From 51a306f20b0a55f404252512fedb7024f5ed1a40 Mon Sep 17 00:00:00 2001 From: Ludwig Bedacht Date: Fri, 26 Sep 2025 16:09:13 +0200 Subject: [PATCH 01/72] feat: implement field resolver logic in the planner --- .../grpc_datasource/execution_plan.go | 277 +++- .../execution_plan_field_resolvers_test.go | 182 +++ .../grpc_datasource/execution_plan_test.go | 13 + .../grpc_datasource/execution_plan_visitor.go | 416 ++--- .../grpc_datasource/mapping_test_helper.go | 20 + .../required_fields_visitor.go | 15 +- v2/pkg/grpctest/Makefile | 4 + v2/pkg/grpctest/mapping/mapping.go | 20 + v2/pkg/grpctest/mockservice.go | 16 + v2/pkg/grpctest/product.proto | 35 + v2/pkg/grpctest/productv1/product.pb.go | 1398 +++++++++++------ v2/pkg/grpctest/productv1/product_grpc.pb.go | 40 +- v2/pkg/grpctest/testdata/products.graphqls | 12 + 13 files changed, 1746 insertions(+), 702 deletions(-) create mode 100644 v2/pkg/engine/datasource/grpc_datasource/execution_plan_field_resolvers_test.go diff --git a/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go b/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go index ee91b528e..f5d822e22 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go +++ b/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go @@ -2,9 +2,11 @@ package grpcdatasource import ( "fmt" + "slices" "strings" "github.com/wundergraph/graphql-go-tools/v2/pkg/ast" + "github.com/wundergraph/graphql-go-tools/v2/pkg/astvisitor" "github.com/wundergraph/graphql-go-tools/v2/pkg/engine/plan" ) @@ -160,7 +162,6 @@ type RPCField struct { // Alias can be used to rename the field in the request message // This is needed to make sure that during the json composition, // the field names match the GraphQL request naming. - // TODO implement alias handling Alias string // Repeated indicates if the field is a repeated field (array/list) Repeated bool @@ -168,9 +169,11 @@ type RPCField struct { Name string // TypeName is the name of the type of the field in the protobuf definition TypeName string - // JSONPath defines the path within the variables to provide the value for the field - // This is used to extract data from the GraphQL variables + // JSONPath either holds the path to the variable definition for the request message, + // or defines the name of the response field in the message. JSONPath string + // ResolvePath is used to resolve values from another message. + ResolvePath ast.Path // EnumName is the name of the enum if the field is an enum type EnumName string // StaticValue is the static value of the field @@ -452,12 +455,19 @@ func (r *rpcPlanningContext) newMessageFromSelectionSet(enclosingTypeNode ast.No // resolveFieldMapping resolves the field mapping for a field. // This applies both for complex types in the input and for all fields in the response. func (r *rpcPlanningContext) resolveFieldMapping(typeName, fieldName string) string { - grpcFieldName, ok := r.mapping.ResolveFieldMapping(typeName, fieldName) - if !ok { - return fieldName + if grpcFieldName, ok := r.mapping.ResolveFieldMapping(typeName, fieldName); ok { + return grpcFieldName } - return grpcFieldName + return fieldName +} + +func (r *rpcPlanningContext) resolveFieldArgumentMapping(typeName, fieldName, argumentName string) string { + if grpcFieldName, ok := r.mapping.ResolveFieldArgumentMapping(typeName, fieldName, argumentName); ok { + return grpcFieldName + } + + return argumentName } func (r *rpcPlanningContext) typeIsNullableOrNestedList(typeRef int) bool { @@ -537,6 +547,145 @@ func (r *rpcPlanningContext) buildField(enclosingTypeNode ast.Node, fd int, fiel return field, nil } +// createRPCFieldFromFieldArgument builds an RPCField from an input value definition. +// It handles scalar, enum, and input object types. +// If the type is an input object type, a message is created and added to the field. +func (r *rpcPlanningContext) createRPCFieldFromFieldArgument(fieldArg fieldArgument) (RPCField, error) { + argDef := r.definition.InputValueDefinitions[fieldArg.argumentDefinitionRef] + argName := r.definition.Input.ByteSliceString(argDef.Name) + underlyingTypeNode, found := r.nodeByTypeRef(argDef.Type) + if !found { + return RPCField{}, fmt.Errorf("unable to resolve underlying type node for argument %s", argName) + } + + var ( + fieldMessage *RPCMessage + err error + dt = DataTypeMessage + ) + + // only scalar, enum and input object types are supported + switch underlyingTypeNode.Kind { + case ast.NodeKindScalarTypeDefinition, ast.NodeKindEnumTypeDefinition: + dt = r.toDataType(&r.definition.Types[argDef.Type]) + case ast.NodeKindInputObjectTypeDefinition: + // If the type is an input object type, a message is created and added to the field. + if fieldMessage, err = r.buildMessageFromInputObjectType(&underlyingTypeNode); err != nil { + return RPCField{}, err + } + default: + return RPCField{}, fmt.Errorf("unsupported type: %s", underlyingTypeNode.Kind) + } + + parentTypeName := fieldArg.parentTypeNode.NameString(r.definition) + fieldName := r.definition.FieldDefinitionNameString(fieldArg.fieldDefinitionRef) + mappedName := r.resolveFieldArgumentMapping(parentTypeName, fieldName, argName) + field, err := r.buildInputMessageField( + argDef.Type, + mappedName, + fieldArg.jsonPath, + dt, + ) + if err != nil { + return RPCField{}, err + } + + field.Message = fieldMessage + return field, nil +} + +// buildMessageFromInputObjectType builds a message from an input object type definition. +func (r *rpcPlanningContext) buildMessageFromInputObjectType(node *ast.Node) (*RPCMessage, error) { + if node.Kind != ast.NodeKindInputObjectTypeDefinition { + return nil, fmt.Errorf("unsupported type: %s", node.Kind) + } + + inputObjectDefinition := r.definition.InputObjectTypeDefinitions[node.Ref] + message := &RPCMessage{ + Name: node.NameString(r.definition), + Fields: make(RPCFields, 0, len(inputObjectDefinition.InputFieldsDefinition.Refs)), + } + for _, inputFieldRef := range inputObjectDefinition.InputFieldsDefinition.Refs { + field, err := r.buildMessageFieldFromInputValueDefinition(inputFieldRef, node) + if err != nil { + return nil, err + } + + message.Fields = append(message.Fields, field) + } + + return message, nil +} + +func (r *rpcPlanningContext) buildMessageFieldFromInputValueDefinition(ivdRef int, node *ast.Node) (RPCField, error) { + ivd := r.definition.InputValueDefinitions[ivdRef] + ivdType := r.definition.Types[ivd.Type] + + underlyingTypeNode, found := r.nodeByTypeRef(ivd.Type) + if !found { + return RPCField{}, fmt.Errorf("unable to resolve underlying type node for input value definition %s", r.definition.Input.ByteSliceString(ivd.Name)) + } + + var ( + fieldMessage *RPCMessage + err error + ) + + dt := DataTypeMessage + switch underlyingTypeNode.Kind { + case ast.NodeKindInputObjectTypeDefinition: + fieldMessage, err = r.buildMessageFromInputObjectType(&underlyingTypeNode) + if err != nil { + return RPCField{}, err + } + default: + dt = r.toDataType(&ivdType) + } + + fieldName := r.definition.Input.ByteSliceString(ivd.Name) + mappedName := r.resolveFieldMapping(node.NameString(r.definition), fieldName) + + field, err := r.buildInputMessageField(ivd.Type, mappedName, fieldName, dt) + if err != nil { + return RPCField{}, err + } + + field.Message = fieldMessage + return field, nil +} + +func (r *rpcPlanningContext) buildInputMessageField(typeRef int, fieldName, jsonPath string, dt DataType) (RPCField, error) { + field := RPCField{ + Name: fieldName, + Optional: !r.definition.TypeIsNonNull(typeRef), + TypeName: dt.String(), + JSONPath: jsonPath, + } + + if r.definition.TypeIsList(typeRef) { + switch { + // for nullable or nested lists we need to build a wrapper message + // Nullability is handled by the datasource during the execution. + case r.typeIsNullableOrNestedList(typeRef): + md, err := r.createListMetadata(typeRef) + if err != nil { + return field, err + } + field.ListMetadata = md + field.IsListType = true + default: + // For non-nullable single lists we can directly use the repeated syntax in protobuf. + field.Repeated = true + } + } + + if dt == DataTypeEnum { + field.EnumName = r.definition.ResolveTypeNameString(typeRef) + } + + return field, nil +} + func (r *rpcPlanningContext) resolveServiceName(subgraphName string) string { if r.mapping == nil || r.mapping.Service == "" { return subgraphName @@ -544,3 +693,117 @@ func (r *rpcPlanningContext) resolveServiceName(subgraphName string) string { return r.mapping.Service } + +func (r *rpcPlanningContext) resolveContextFields(walker *astvisitor.Walker, fd int) ([]int, error) { + contextDirectiveRef, exists := r.definition.FieldDefinitionDirectiveByName(fd, ast.ByteSlice("resolved")) + if exists { + fields, err := r.getFieldsFromContext(walker.EnclosingTypeDefinition, contextDirectiveRef) + if err != nil { + return nil, err + } + + return fields, nil + } + + idFieldRef, err := r.findIDField(walker.EnclosingTypeDefinition, fd) + return []int{idFieldRef}, err +} + +func (r *rpcPlanningContext) getFieldsFromContext(parentNode ast.Node, contextRef int) ([]int, error) { + val, exists := r.definition.DirectiveArgumentValueByName(contextRef, []byte("context")) + if !exists { + return nil, fmt.Errorf("context directive argument not found") + } + + fieldsString := r.definition.ValueContentString(val) + + walker := astvisitor.NewDefaultWalker() + + v := newRequiredFieldsVisitor(&walker, &RPCMessage{}, r) + if err := v.visitRequiredFields(r.definition, parentNode.NameString(r.definition), fieldsString); err != nil { + return nil, err + } + + return v.fieldDefinitionRefs, nil +} + +func (r *rpcPlanningContext) findIDField(parentNode ast.Node, fd int) (int, error) { + switch parentNode.Kind { + case ast.NodeKindObjectTypeDefinition: + o := r.definition.ObjectTypeDefinitions[parentNode.Ref] + result := slices.Collect(r.filterIDFieldsFunc(o, fd)) + + if len(result) == 0 { + return ast.InvalidRef, fmt.Errorf("unable to determine ID field in object type %s", parentNode.NameString(r.definition)) + } + + if len(result) > 1 { + return ast.InvalidRef, fmt.Errorf("multiple ID fields found in object type %s", parentNode.NameString(r.definition)) + } + + return result[0], nil + default: + return ast.InvalidRef, fmt.Errorf("invalid parent node kind: %s, expected ObjectTypeDefinition", parentNode.Kind) + } +} + +func (r *rpcPlanningContext) filterIDFieldsFunc(o ast.ObjectTypeDefinition, fd int) func(yield func(int) bool) { + fieldRefs := o.FieldsDefinition.Refs + return func(yield func(int) bool) { + for _, ref := range fieldRefs { + if ref == fd { + continue + } + + typeName := r.definition.FieldDefinitionTypeNameString(ref) + if typeName != "ID" { + continue + } + + if !yield(ref) { + return + } + } + } +} + +func (r *rpcPlanningContext) parseFieldArguments(walker *astvisitor.Walker, fd int, fieldArgs []int) ([]fieldArgument, error) { + result := make([]fieldArgument, 0, len(fieldArgs)) + for _, fieldArgRef := range fieldArgs { + arg := r.operation.Arguments[fieldArgRef] + fieldArg := r.operation.ArgumentNameString(fieldArgRef) + fieldType := arg.Value.Kind + + argDefRef := r.definition.NodeFieldDefinitionArgumentDefinitionByName( + walker.EnclosingTypeDefinition, + r.definition.FieldDefinitionNameBytes(fd), + r.operation.ArgumentNameBytes(fieldArgRef), + ) + + if argDefRef == ast.InvalidRef { + return nil, fmt.Errorf("unable to resolve argument input value definition for argument %s", fieldArg) + } + + jsonValue := fieldArg + if fieldType == ast.ValueKindVariable { + jsonValue = r.operation.Input.ByteSliceString(r.operation.VariableValues[arg.Value.Ref].Name) + } + + result = append(result, fieldArgument{ + fieldDefinitionRef: fd, + argumentDefinitionRef: argDefRef, + parentTypeNode: walker.EnclosingTypeDefinition, + jsonPath: jsonValue, + }) + + } + + return result, nil + +} + +// nodeByTypeRef is a helper function to resolve the underlying type node for a given type reference. +func (r *rpcPlanningContext) nodeByTypeRef(typeRef int) (ast.Node, bool) { + underlyingTypeName := r.definition.ResolveTypeNameString(typeRef) + return r.definition.NodeByNameStr(underlyingTypeName) +} diff --git a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_field_resolvers_test.go b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_field_resolvers_test.go new file mode 100644 index 000000000..397513c65 --- /dev/null +++ b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_field_resolvers_test.go @@ -0,0 +1,182 @@ +package grpcdatasource + +import ( + "testing" +) + +func TestExecutionPlanFieldResolvers(t *testing.T) { + tests := []struct { + name string + query string + expectedPlan *RPCExecutionPlan + expectedError string + }{ + { + name: "Should create an execution plan for a query with nullable fields type", + query: "query CategoriesWithFieldResolvers($whoop: ProductCountFilter) { categories { id name kind productCount(filters: $whoop) } }", + expectedPlan: &RPCExecutionPlan{ + Calls: []RPCCall{ + { + ServiceName: "Products", + MethodName: "QueryCategories", + Request: RPCMessage{ + Name: "QueryCategoriesRequest", + }, + Response: RPCMessage{ + Name: "QueryCategoriesResponse", + Fields: []RPCField{ + { + Name: "categories", + TypeName: string(DataTypeMessage), + JSONPath: "categories", + Repeated: true, + Message: &RPCMessage{ + Name: "Category", + Fields: []RPCField{ + { + Name: "id", + TypeName: string(DataTypeString), + JSONPath: "id", + }, + { + Name: "name", + TypeName: string(DataTypeString), + JSONPath: "name", + }, + { + Name: "kind", + TypeName: string(DataTypeEnum), + JSONPath: "kind", + EnumName: "CategoryKind", + }, + }, + }, + }, + }, + }, + }, + { + DependentCalls: []int{0}, + ServiceName: "Products", + MethodName: "ResolveCategoryProductCount", + Request: RPCMessage{ + Name: "ResolveCategoryProductCountRequest", + Fields: []RPCField{ + { + Name: "key", + TypeName: string(DataTypeMessage), + JSONPath: "key", + Repeated: true, + Message: &RPCMessage{ + Name: "ResolveCategoryProductCountRequestKey", + Fields: []RPCField{ + { + Name: "context", + TypeName: string(DataTypeMessage), + JSONPath: "", + Message: &RPCMessage{ + Name: "CategoryProductCountContext", + Fields: []RPCField{ + { + Name: "id", + TypeName: string(DataTypeString), + JSONPath: "id", + ResolvePath: buildPath("categories.id"), + }, + { + Name: "name", + TypeName: string(DataTypeString), + JSONPath: "name", + ResolvePath: buildPath("categories.name"), + }, + }, + }, + }, + { + Name: "field_args", + TypeName: string(DataTypeMessage), + JSONPath: "", + Message: &RPCMessage{ + Name: "CategoryProductCountArgs", + Fields: []RPCField{ + { + Name: "filters", + TypeName: string(DataTypeMessage), + JSONPath: "whoop", + Optional: true, + Message: &RPCMessage{ + Name: "ProductCountFilter", + Fields: []RPCField{ + { + Name: "min_price", + TypeName: string(DataTypeDouble), + JSONPath: "minPrice", + Optional: true, + }, + { + Name: "max_price", + TypeName: string(DataTypeDouble), + JSONPath: "maxPrice", + Optional: true, + }, + { + Name: "in_stock", + TypeName: string(DataTypeBool), + JSONPath: "inStock", + Optional: true, + }, + { + Name: "search_term", + TypeName: string(DataTypeString), + JSONPath: "searchTerm", + Optional: true, + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + Response: RPCMessage{ + Name: "ResolveCategoryProductCountResponse", + Fields: []RPCField{ + { + Name: "result", + TypeName: string(DataTypeMessage), + JSONPath: "result", + Repeated: true, + Message: &RPCMessage{ + Name: "ResolveCategoryProductCountResponseResult", + Fields: []RPCField{ + { + Name: "product_count", + TypeName: string(DataTypeInt32), + JSONPath: "productCount", + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + } + + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + t.Parallel() + runTest(t, testCase{ + query: tt.query, + expectedPlan: tt.expectedPlan, + expectedError: tt.expectedError, + }) + }) + } +} diff --git a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_test.go b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_test.go index a19ef4a8f..90bd0cf5f 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_test.go +++ b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_test.go @@ -5,6 +5,7 @@ import ( "testing" "github.com/stretchr/testify/require" + "github.com/wundergraph/graphql-go-tools/v2/pkg/ast" "github.com/wundergraph/graphql-go-tools/v2/pkg/astvalidation" "github.com/wundergraph/graphql-go-tools/v2/pkg/grpctest" "github.com/wundergraph/graphql-go-tools/v2/pkg/operationreport" @@ -49,6 +50,18 @@ func runTest(t *testing.T, testCase testCase) { } } +func buildPath(path string) ast.Path { + pathElements := strings.Split(path, ".") + pathItems := make([]ast.PathItem, 0, len(pathElements)) + for _, element := range pathElements { + pathItems = append(pathItems, ast.PathItem{ + Kind: ast.FieldName, + FieldName: []byte(element), + }) + } + return pathItems +} + func TestQueryExecutionPlans(t *testing.T) { tests := []struct { name string diff --git a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor.go b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor.go index 431357af1..e8422e194 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor.go +++ b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor.go @@ -16,8 +16,7 @@ type planningInfo struct { operationType ast.OperationType operationFieldName string - requestMessageAncestors []*RPCMessage - currentRequestMessage *RPCMessage + currentRequestMessage *RPCMessage responseMessageAncestors []*RPCMessage currentResponseMessage *RPCMessage @@ -26,12 +25,33 @@ type planningInfo struct { responseFieldIndexAncestors []int } +type contextField struct { + fieldRef int + resolvePath ast.Path +} + +type fieldArgument struct { + parentTypeNode ast.Node + jsonPath string + fieldDefinitionRef int + argumentDefinitionRef int +} + +type resolvedField struct { + callerRef int + parentTypeRef int + fieldRef int + + contextFields []contextField + fieldArguments []fieldArgument +} + type rpcPlanVisitor struct { walker *astvisitor.Walker operation *ast.Document definition *ast.Document - planCtx *rpcPlanningContext planInfo planningInfo + planCtx *rpcPlanningContext subgraphName string mapping *GRPCMapping @@ -40,6 +60,9 @@ type rpcPlanVisitor struct { operationFieldRefs []int currentCall *RPCCall currentCallID int + + relatedCallID int + resolvedFields []resolvedField } type rpcPlanVisitorConfig struct { @@ -58,9 +81,11 @@ func newRPCPlanVisitor(config rpcPlanVisitorConfig) *rpcPlanVisitor { subgraphName: cases.Title(language.Und, cases.NoLower).String(config.subgraphName), mapping: config.mapping, operationFieldRef: -1, + resolvedFields: make([]resolvedField, 0), + relatedCallID: -1, } - walker.RegisterEnterDocumentVisitor(visitor) + walker.RegisterDocumentVisitor(visitor) walker.RegisterEnterOperationVisitor(visitor) walker.RegisterFieldVisitor(visitor) walker.RegisterSelectionSetVisitor(visitor) @@ -87,6 +112,123 @@ func (r *rpcPlanVisitor) EnterDocument(operation *ast.Document, definition *ast. r.planCtx = newRPCPlanningContext(operation, definition, r.mapping) } +// LeaveDocument implements astvisitor.DocumentVisitor. +func (r *rpcPlanVisitor) LeaveDocument(_, _ *ast.Document) { + if len(r.resolvedFields) == 0 { + return + } + + // We need to create a new call for each resolved field. + calls := make([]RPCCall, 0, len(r.resolvedFields)) + + for _, resolvedField := range r.resolvedFields { + + contextMessage := &RPCMessage{ + Name: "CategoryProductCountContext", + } + + fieldArgsMessage := &RPCMessage{ + Name: "CategoryProductCountArgs", + } + + // Base resolve call can be templated in plan context. + call := RPCCall{ + DependentCalls: []int{resolvedField.callerRef}, + ServiceName: r.planCtx.resolveServiceName(r.subgraphName), + MethodName: "ResolveCategoryProductCount", + Request: RPCMessage{ + Name: "ResolveCategoryProductCountRequest", + Fields: RPCFields{ + { + Name: "key", + TypeName: string(DataTypeMessage), + JSONPath: "key", + Repeated: true, + Message: &RPCMessage{ + Name: "ResolveCategoryProductCountRequestKey", + Fields: RPCFields{ + { + Name: "context", + TypeName: string(DataTypeMessage), + Message: contextMessage, + }, + { + Name: "field_args", + TypeName: string(DataTypeMessage), + Message: fieldArgsMessage, + }, + }, + }, + }, + }, + }, + Response: RPCMessage{ + Name: "ResolveCategoryProductCountResponse", + Fields: RPCFields{ + { + Name: "result", + TypeName: string(DataTypeMessage), + JSONPath: "result", + Repeated: true, + Message: &RPCMessage{ + Name: "ResolveCategoryProductCountResponseResult", + Fields: RPCFields{ + { + Name: "product_count", + TypeName: string(DataTypeInt32), + JSONPath: "productCount", + }, + }, + }, + }, + }, + }, + } + + contextMessage.Fields = make(RPCFields, 0, len(resolvedField.contextFields)) + for _, contextField := range resolvedField.contextFields { + typeDefNode, found := r.definition.NodeByNameStr(r.definition.ResolveTypeNameString(resolvedField.parentTypeRef)) + if !found { + r.walker.StopWithInternalErr(fmt.Errorf("type definition node not found for type: %s", r.definition.ResolveTypeNameString(resolvedField.parentTypeRef))) + return + } + + field, err := r.planCtx.buildField( + typeDefNode, + contextField.fieldRef, + r.definition.FieldDefinitionNameString(contextField.fieldRef), + "", + ) + + field.ResolvePath = contextField.resolvePath + + if err != nil { + r.walker.StopWithInternalErr(err) + return + } + + contextMessage.Fields = append(contextMessage.Fields, field) + } + + fieldArgsMessage.Fields = make(RPCFields, 0, len(resolvedField.fieldArguments)) + for _, fieldArgument := range resolvedField.fieldArguments { + field, err := r.planCtx.createRPCFieldFromFieldArgument(fieldArgument) + + if err != nil { + r.walker.StopWithInternalErr(err) + return + } + + fieldArgsMessage.Fields = append(fieldArgsMessage.Fields, field) + } + + calls = append(calls, call) + } + + r.plan.Calls = append(r.plan.Calls, calls...) + r.resolvedFields = nil +} + // EnterOperationDefinition implements astvisitor.EnterOperationDefinitionVisitor. // This is called when entering the operation definition node. // It retrieves information about the operation @@ -108,8 +250,8 @@ func (r *rpcPlanVisitor) EnterOperationDefinition(ref int) { // // TODO handle field arguments to define resolvers func (r *rpcPlanVisitor) EnterArgument(ref int) { - a := r.walker.Ancestor() - if a.Kind != ast.NodeKindField && a.Ref != r.operationFieldRef { + ancestor := r.walker.Ancestor() + if ancestor.Kind != ast.NodeKindField || ancestor.Ref != r.operationFieldRef { return } argumentInputValueDefinitionRef, exists := r.walker.ArgumentInputValueDefinition(ref) @@ -117,9 +259,34 @@ func (r *rpcPlanVisitor) EnterArgument(ref int) { return } + // As we check that we are inside of a field we can safely access the second to last type definition. + parentTypeNode := r.walker.TypeDefinitions[len(r.walker.TypeDefinitions)-2] + fieldDefinitionRef, exists := r.definition.NodeFieldDefinitionByName(parentTypeNode, r.operation.FieldNameBytes(ancestor.Ref)) + if !exists { + return + } + + argument := r.operation.ArgumentValue(ref) + jsonPath := r.operation.ArgumentNameString(ref) + if argument.Kind == ast.ValueKindVariable { + jsonPath = r.operation.Input.ByteSliceString(r.operation.VariableValues[argument.Ref].Name) + } + // Retrieve the type of the input value definition, and build the request message - inputValueDefinitionTypeRef := r.definition.InputValueDefinitionType(argumentInputValueDefinitionRef) - r.enrichRequestMessageFromInputArgument(ref, inputValueDefinitionTypeRef) + field, err := r.planCtx.createRPCFieldFromFieldArgument(fieldArgument{ + fieldDefinitionRef: fieldDefinitionRef, + parentTypeNode: parentTypeNode, + argumentDefinitionRef: argumentInputValueDefinitionRef, + jsonPath: jsonPath, + }) + + if err != nil { + r.walker.StopWithInternalErr(err) + return + } + + r.planInfo.currentRequestMessage.Fields = append(r.planInfo.currentRequestMessage.Fields, field) + } // EnterSelectionSet implements astvisitor.EnterSelectionSetVisitor. @@ -214,7 +381,11 @@ func (r *rpcPlanVisitor) LeaveSelectionSet(ref int) { } } -func (r *rpcPlanVisitor) handleRootField(ref int) error { +func (r *rpcPlanVisitor) handleRootField(isRootField bool, ref int) error { + if !isRootField { + return nil + } + r.operationFieldRef = ref r.planInfo.operationFieldName = r.operation.FieldNameString(ref) @@ -241,11 +412,10 @@ func (r *rpcPlanVisitor) handleRootField(ref int) error { // EnterField implements astvisitor.EnterFieldVisitor. func (r *rpcPlanVisitor) EnterField(ref int) { fieldName := r.operation.FieldNameString(ref) - if r.walker.InRootField() { - if err := r.handleRootField(ref); err != nil { - r.walker.StopWithInternalErr(err) - return - } + inRootField := r.walker.InRootField() + if err := r.handleRootField(inRootField, ref); err != nil { + r.walker.StopWithInternalErr(err) + return } if fieldName == "_entities" { @@ -267,6 +437,47 @@ func (r *rpcPlanVisitor) EnterField(ref int) { return } + // Field arguments for non root types will be handled as resolver calls. + // We need to make sure to handle a hierarchy of arguments in order to perform parallel calls in order to retrieve the data. + // TODO: this needs to be available for both visitors and added to the plancontext + if fieldArgs := r.operation.FieldArguments(ref); !inRootField && len(fieldArgs) > 0 { + r.relatedCallID++ + resolvedField := resolvedField{ + callerRef: r.relatedCallID, + parentTypeRef: r.walker.EnclosingTypeDefinition.Ref, + fieldRef: fd, + } + + contextFields, err := r.planCtx.resolveContextFields(r.walker, fd) + if err != nil { + r.walker.StopWithInternalErr(err) + return + } + + for _, contextFieldRef := range contextFields { + contextFieldName := r.definition.FieldDefinitionNameBytes(contextFieldRef) // TODO handle aliases + resolvedPath := append(r.walker.Path[1:].WithoutInlineFragmentNames(), ast.PathItem{ + Kind: ast.FieldName, + FieldName: contextFieldName, + }) + + resolvedField.contextFields = append(resolvedField.contextFields, contextField{ + fieldRef: contextFieldRef, + resolvePath: resolvedPath, + }) + } + + fieldArguments, err := r.planCtx.parseFieldArguments(r.walker, fd, fieldArgs) + if err != nil { + r.walker.StopWithInternalErr(err) + return + } + + resolvedField.fieldArguments = fieldArguments + r.resolvedFields = append(r.resolvedFields, resolvedField) + return + } + field, err := r.planCtx.buildField(r.walker.EnclosingTypeDefinition, fd, fieldName, fieldAlias) if err != nil { r.walker.StopWithInternalErr(err) @@ -291,6 +502,13 @@ func (r *rpcPlanVisitor) EnterField(ref int) { func (r *rpcPlanVisitor) LeaveField(ref int) { // If we are not in the operation field, we can increment the response field index. if !r.walker.InRootField() { + // If the field has arguments, we need to decrement the related call ID. + // This is because we can also have nested arguments, which require the underlying field to be resolved + // by values provided by the parent call. + if r.operation.FieldHasArguments(ref) { + r.relatedCallID-- + } + r.planInfo.currentResponseFieldIndex++ return } @@ -305,173 +523,3 @@ func (r *rpcPlanVisitor) LeaveField(ref int) { r.planInfo.currentResponseFieldIndex = 0 } - -// enrichRequestMessageFromInputArgument constructs a request message from an input argument based on its type. -// It retrieves the underlying type and builds the request message from the underlying type. -// If the underlying type is an input object type, it creates a new message and adds it to the current request message. -// Otherwise, it adds the field to the current request message. -func (r *rpcPlanVisitor) enrichRequestMessageFromInputArgument(argRef, typeRef int) { - underlyingTypeName := r.definition.ResolveTypeNameString(typeRef) - underlyingTypeNode, found := r.definition.NodeByNameStr(underlyingTypeName) - if !found { - return - } - - fieldName := r.operation.ArgumentNameString(argRef) - jsonPath := fieldName - argument := r.operation.Arguments[argRef] - - // TODO: We should only work with variables as after normalization we don't have and direct input values. - // Therefore we should error out when we don't have a variable. - if argument.Value.Kind == ast.ValueKindVariable { - jsonPath = r.operation.Input.ByteSliceString(r.operation.VariableValues[argument.Value.Ref].Name) - } - - rootNode := r.walker.TypeDefinitions[len(r.walker.TypeDefinitions)-2] - baseType := r.definition.NodeNameString(rootNode) - mappedInputName := r.resolveInputArgument(baseType, r.walker.Ancestor().Ref, fieldName) - - // If the underlying type is an input object type, create a new message and add it to the current request message. - switch underlyingTypeNode.Kind { - case ast.NodeKindInputObjectTypeDefinition: - msg := &RPCMessage{ - Name: underlyingTypeName, - Fields: RPCFields{}, - } - - field := r.buildInputMessageField(typeRef, mappedInputName, jsonPath, DataTypeMessage) - field.Message = msg - r.planInfo.currentRequestMessage.Fields = append(r.planInfo.currentRequestMessage.Fields, field) - - // Add the current request message to the ancestors and set the current request message to the new message. - r.planInfo.requestMessageAncestors = append(r.planInfo.requestMessageAncestors, r.planInfo.currentRequestMessage) - r.planInfo.currentRequestMessage = msg - - r.buildMessageFromNode(underlyingTypeNode) - - r.planInfo.currentRequestMessage = r.planInfo.requestMessageAncestors[len(r.planInfo.requestMessageAncestors)-1] - r.planInfo.requestMessageAncestors = r.planInfo.requestMessageAncestors[:len(r.planInfo.requestMessageAncestors)-1] - - case ast.NodeKindScalarTypeDefinition, ast.NodeKindEnumTypeDefinition: - dt := r.planCtx.toDataType(&r.definition.Types[typeRef]) - - r.planInfo.currentRequestMessage.Fields = append(r.planInfo.currentRequestMessage.Fields, - r.buildInputMessageField(typeRef, mappedInputName, jsonPath, dt)) - default: - // TODO unions, interfaces, etc. - r.walker.Report.AddInternalError(fmt.Errorf("unsupported type: %s", underlyingTypeNode.Kind)) - r.walker.Stop() - return - } -} - -// buildMessageFromNode builds a message structure from an AST node. -func (r *rpcPlanVisitor) buildMessageFromNode(node ast.Node) { - switch node.Kind { - case ast.NodeKindInputObjectTypeDefinition: - inputObjectDefinition := r.definition.InputObjectTypeDefinitions[node.Ref] - r.planInfo.currentRequestMessage.Fields = make(RPCFields, 0, len(inputObjectDefinition.InputFieldsDefinition.Refs)) - - for _, inputFieldRef := range inputObjectDefinition.InputFieldsDefinition.Refs { - fieldDefinition := r.definition.InputValueDefinitions[inputFieldRef] - fieldName := r.definition.Input.ByteSliceString(fieldDefinition.Name) - r.buildMessageField(fieldName, fieldDefinition.Type, node.Ref) - } - } -} - -// buildMessageField creates a field in the current request message based on the field type. -func (r *rpcPlanVisitor) buildMessageField(fieldName string, typeRef, parentTypeRef int) { - inputValueDefinitionType := r.definition.Types[typeRef] - underlyingTypeName := r.definition.ResolveTypeNameString(typeRef) - underlyingTypeNode, found := r.definition.NodeByNameStr(underlyingTypeName) - if !found { - return - } - - parentTypeName := r.definition.InputObjectTypeDefinitionNameString(parentTypeRef) - mappedName := r.resolveFieldMapping(parentTypeName, fieldName) - - // If the type is not an object, directly add the field to the request message - if underlyingTypeNode.Kind != ast.NodeKindInputObjectTypeDefinition { - dt := r.planCtx.toDataType(&inputValueDefinitionType) - - r.planInfo.currentRequestMessage.Fields = append(r.planInfo.currentRequestMessage.Fields, - r.buildInputMessageField(typeRef, mappedName, fieldName, dt)) - - return - } - - msg := &RPCMessage{ - Name: underlyingTypeName, - } - - field := r.buildInputMessageField(typeRef, mappedName, fieldName, DataTypeMessage) - field.Message = msg - - r.planInfo.currentRequestMessage.Fields = append(r.planInfo.currentRequestMessage.Fields, field) - - r.planInfo.requestMessageAncestors = append(r.planInfo.requestMessageAncestors, r.planInfo.currentRequestMessage) - r.planInfo.currentRequestMessage = msg - - r.buildMessageFromNode(underlyingTypeNode) - - r.planInfo.currentRequestMessage = r.planInfo.requestMessageAncestors[len(r.planInfo.requestMessageAncestors)-1] - r.planInfo.requestMessageAncestors = r.planInfo.requestMessageAncestors[:len(r.planInfo.requestMessageAncestors)-1] -} - -func (r *rpcPlanVisitor) buildInputMessageField(typeRef int, fieldName, jsonPath string, dt DataType) RPCField { - field := RPCField{ - Name: fieldName, - Optional: !r.definition.TypeIsNonNull(typeRef), - TypeName: dt.String(), - JSONPath: jsonPath, - } - - if r.definition.TypeIsList(typeRef) { - switch { - // for nullable or nested lists we need to build a wrapper message - // Nullability is handled by the datasource during the execution. - case r.planCtx.typeIsNullableOrNestedList(typeRef): - md, err := r.planCtx.createListMetadata(typeRef) - if err != nil { - r.walker.StopWithInternalErr(err) - return field - } - field.ListMetadata = md - field.IsListType = true - default: - // For non-nullable single lists we can directly use the repeated syntax in protobuf. - field.Repeated = true - } - } - - if dt == DataTypeEnum { - field.EnumName = r.definition.ResolveTypeNameString(typeRef) - } - - return field -} - -// This applies both for complex types in the input and for all fields in the response. -func (r *rpcPlanVisitor) resolveFieldMapping(typeName, fieldName string) string { - grpcFieldName, ok := r.mapping.ResolveFieldMapping(typeName, fieldName) - if !ok { - return fieldName - } - - return grpcFieldName -} - -// resolveInputArgument resolves the input argument mapping for a field. -// This only applies if the input arguments are scalar values. -// If the input argument is a message, the mapping is resolved by the -// resolveFieldMapping function. -func (r *rpcPlanVisitor) resolveInputArgument(baseType string, fieldRef int, argumentName string) string { - grpcFieldName, ok := r.mapping.ResolveFieldArgumentMapping(baseType, r.operation.FieldNameString(fieldRef), argumentName) - if !ok { - return argumentName - } - - return grpcFieldName -} diff --git a/v2/pkg/engine/datasource/grpc_datasource/mapping_test_helper.go b/v2/pkg/engine/datasource/grpc_datasource/mapping_test_helper.go index 6878c6bba..057ffe204 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/mapping_test_helper.go +++ b/v2/pkg/engine/datasource/grpc_datasource/mapping_test_helper.go @@ -667,6 +667,12 @@ func testMapping() *GRPCMapping { "kind": { TargetName: "kind", }, + "productCount": { + TargetName: "product_count", + ArgumentMappings: map[string]string{ + "filters": "filters", + }, + }, }, "CategoryFilter": { "category": { @@ -1041,6 +1047,20 @@ func testMapping() *GRPCMapping { TargetName: "skill_count", }, }, + "ProductCountFilter": { + "minPrice": { + TargetName: "min_price", + }, + "maxPrice": { + TargetName: "max_price", + }, + "inStock": { + TargetName: "in_stock", + }, + "searchTerm": { + TargetName: "search_term", + }, + }, }, } } diff --git a/v2/pkg/engine/datasource/grpc_datasource/required_fields_visitor.go b/v2/pkg/engine/datasource/grpc_datasource/required_fields_visitor.go index cd54e7ccf..3271a624a 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/required_fields_visitor.go +++ b/v2/pkg/engine/datasource/grpc_datasource/required_fields_visitor.go @@ -15,8 +15,9 @@ type requiredFieldsVisitor struct { operation *ast.Document definition *ast.Document - walker *astvisitor.Walker - message *RPCMessage + walker *astvisitor.Walker + message *RPCMessage + fieldDefinitionRefs []int planCtx *rpcPlanningContext @@ -27,10 +28,11 @@ type requiredFieldsVisitor struct { // It registers the visitor with the walker and returns it. func newRequiredFieldsVisitor(walker *astvisitor.Walker, message *RPCMessage, planCtx *rpcPlanningContext) *requiredFieldsVisitor { visitor := &requiredFieldsVisitor{ - walker: walker, - message: message, - planCtx: planCtx, - messageAncestors: []*RPCMessage{}, + walker: walker, + message: message, + planCtx: planCtx, + messageAncestors: []*RPCMessage{}, + fieldDefinitionRefs: []int{}, } walker.RegisterEnterDocumentVisitor(visitor) @@ -130,6 +132,7 @@ func (r *requiredFieldsVisitor) EnterField(ref int) { return } + r.fieldDefinitionRefs = append(r.fieldDefinitionRefs, fd) r.message.Fields = append(r.message.Fields, field) } diff --git a/v2/pkg/grpctest/Makefile b/v2/pkg/grpctest/Makefile index 847d67b72..e7ea6737d 100644 --- a/v2/pkg/grpctest/Makefile +++ b/v2/pkg/grpctest/Makefile @@ -16,3 +16,7 @@ build-plugin: .PHONY: regenerate-proto regenerate-proto: pnpx wgc@latest grpc-service generate -i testdata/products.graphqls -o testdata/ -p productv1 -g "cosmo/pkg/proto/productv1;productv1" Product + +.PHONY: regenerate-proto-local +regenerate-proto-local: + pnpx tsx --env-file ../../../../cosmo/cli/.env ../../../../cosmo/cli/src/index.ts grpc-service generate -i testdata/products.graphqls -o testdata/ -p productv1 -g "cosmo/pkg/proto/productv1;productv1" Product \ No newline at end of file diff --git a/v2/pkg/grpctest/mapping/mapping.go b/v2/pkg/grpctest/mapping/mapping.go index 7e14f5f64..f4561a51a 100644 --- a/v2/pkg/grpctest/mapping/mapping.go +++ b/v2/pkg/grpctest/mapping/mapping.go @@ -666,6 +666,12 @@ func DefaultGRPCMapping() *grpcdatasource.GRPCMapping { "kind": { TargetName: "kind", }, + "productCount": { + TargetName: "product_count", + ArgumentMappings: map[string]string{ + "filters": "filters", + }, + }, }, "CategoryFilter": { "category": { @@ -1040,6 +1046,20 @@ func DefaultGRPCMapping() *grpcdatasource.GRPCMapping { TargetName: "skill_count", }, }, + "ProductCountFilter": { + "minPrice": { + TargetName: "min_price", + }, + "maxPrice": { + TargetName: "max_price", + }, + "inStock": { + TargetName: "in_stock", + }, + "searchTerm": { + TargetName: "search_term", + }, + }, }, } } diff --git a/v2/pkg/grpctest/mockservice.go b/v2/pkg/grpctest/mockservice.go index 893d6c4f7..9164cb938 100644 --- a/v2/pkg/grpctest/mockservice.go +++ b/v2/pkg/grpctest/mockservice.go @@ -19,6 +19,22 @@ type MockService struct { productv1.UnimplementedProductServiceServer } +// ResolveCategoryProductCount implements productv1.ProductServiceServer. +func (s *MockService) ResolveCategoryProductCount(_ context.Context, req *productv1.ResolveCategoryProductCountRequest) (*productv1.ResolveCategoryProductCountResponse, error) { + results := make([]*productv1.ResolveCategoryProductCountResponseResult, len(req.GetKey())) + for range req.GetKey() { + results = append(results, &productv1.ResolveCategoryProductCountResponseResult{ + ProductCount: int32(rand.Intn(100)), + }) + } + + resp := &productv1.ResolveCategoryProductCountResponse{ + Result: results, + } + + return resp, nil +} + // LookupWarehouseById implements productv1.ProductServiceServer. func (s *MockService) LookupWarehouseById(ctx context.Context, in *productv1.LookupWarehouseByIdRequest) (*productv1.LookupWarehouseByIdResponse, error) { var results []*productv1.Warehouse diff --git a/v2/pkg/grpctest/product.proto b/v2/pkg/grpctest/product.proto index bce3eb569..dcde431bc 100644 --- a/v2/pkg/grpctest/product.proto +++ b/v2/pkg/grpctest/product.proto @@ -55,6 +55,7 @@ service ProductService { rpc QueryTypeWithMultipleFilterFields(QueryTypeWithMultipleFilterFieldsRequest) returns (QueryTypeWithMultipleFilterFieldsResponse) {} rpc QueryUser(QueryUserRequest) returns (QueryUserResponse) {} rpc QueryUsers(QueryUsersRequest) returns (QueryUsersResponse) {} + rpc ResolveCategoryProductCount(ResolveCategoryProductCountRequest) returns (ResolveCategoryProductCountResponse) {} } // Wrapper message for a list of AuthorFilter. @@ -619,6 +620,33 @@ message MutationBulkUpdateBlogPostsRequest { message MutationBulkUpdateBlogPostsResponse { repeated BlogPost bulk_update_blog_posts = 1; } +message CategoryProductCountArgs { + ProductCountFilter filters = 1; +} + +message CategoryProductCountContext { + string id = 1; + string name = 2; +} + +message ResolveCategoryProductCountRequestKey { + // context provides the resolver context for the field productCount of type Category. + CategoryProductCountContext context = 1; + // field_args provides the arguments for the resolver field productCount of type Category. + CategoryProductCountArgs field_args = 2; +} + +message ResolveCategoryProductCountRequest { + repeated ResolveCategoryProductCountRequestKey key = 1; +} + +message ResolveCategoryProductCountResponseResult { + int32 product_count = 1; +} + +message ResolveCategoryProductCountResponse { + repeated ResolveCategoryProductCountResponseResult result = 1; +} message Product { string id = 1; @@ -918,4 +946,11 @@ message ActionError { message CategoryInput { string name = 1; CategoryKind kind = 2; +} + +message ProductCountFilter { + google.protobuf.DoubleValue min_price = 1; + google.protobuf.DoubleValue max_price = 2; + google.protobuf.BoolValue in_stock = 3; + google.protobuf.StringValue search_term = 4; } \ No newline at end of file diff --git a/v2/pkg/grpctest/productv1/product.pb.go b/v2/pkg/grpctest/productv1/product.pb.go index a8ab1fc87..e57fb0c9e 100644 --- a/v2/pkg/grpctest/productv1/product.pb.go +++ b/v2/pkg/grpctest/productv1/product.pb.go @@ -1,7 +1,7 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: -// protoc-gen-go v1.36.8 -// protoc v5.29.3 +// protoc-gen-go v1.36.9 +// protoc v6.32.0 // source: product.proto package productv1 @@ -5093,6 +5093,288 @@ func (x *MutationBulkUpdateBlogPostsResponse) GetBulkUpdateBlogPosts() []*BlogPo return nil } +type CategoryProductCountArgs struct { + state protoimpl.MessageState `protogen:"open.v1"` + Filters *ProductCountFilter `protobuf:"bytes,1,opt,name=filters,proto3" json:"filters,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *CategoryProductCountArgs) Reset() { + *x = CategoryProductCountArgs{} + mi := &file_product_proto_msgTypes[112] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *CategoryProductCountArgs) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*CategoryProductCountArgs) ProtoMessage() {} + +func (x *CategoryProductCountArgs) ProtoReflect() protoreflect.Message { + mi := &file_product_proto_msgTypes[112] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use CategoryProductCountArgs.ProtoReflect.Descriptor instead. +func (*CategoryProductCountArgs) Descriptor() ([]byte, []int) { + return file_product_proto_rawDescGZIP(), []int{112} +} + +func (x *CategoryProductCountArgs) GetFilters() *ProductCountFilter { + if x != nil { + return x.Filters + } + return nil +} + +type CategoryProductCountContext struct { + state protoimpl.MessageState `protogen:"open.v1"` + Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` + Name string `protobuf:"bytes,2,opt,name=name,proto3" json:"name,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *CategoryProductCountContext) Reset() { + *x = CategoryProductCountContext{} + mi := &file_product_proto_msgTypes[113] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *CategoryProductCountContext) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*CategoryProductCountContext) ProtoMessage() {} + +func (x *CategoryProductCountContext) ProtoReflect() protoreflect.Message { + mi := &file_product_proto_msgTypes[113] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use CategoryProductCountContext.ProtoReflect.Descriptor instead. +func (*CategoryProductCountContext) Descriptor() ([]byte, []int) { + return file_product_proto_rawDescGZIP(), []int{113} +} + +func (x *CategoryProductCountContext) GetId() string { + if x != nil { + return x.Id + } + return "" +} + +func (x *CategoryProductCountContext) GetName() string { + if x != nil { + return x.Name + } + return "" +} + +type ResolveCategoryProductCountRequestKey struct { + state protoimpl.MessageState `protogen:"open.v1"` + // context provides the resolver context for the field productCount of type Category. + Context *CategoryProductCountContext `protobuf:"bytes,1,opt,name=context,proto3" json:"context,omitempty"` + // field_args provides the arguments for the resolver field productCount of type Category. + FieldArgs *CategoryProductCountArgs `protobuf:"bytes,2,opt,name=field_args,json=fieldArgs,proto3" json:"field_args,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ResolveCategoryProductCountRequestKey) Reset() { + *x = ResolveCategoryProductCountRequestKey{} + mi := &file_product_proto_msgTypes[114] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ResolveCategoryProductCountRequestKey) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ResolveCategoryProductCountRequestKey) ProtoMessage() {} + +func (x *ResolveCategoryProductCountRequestKey) ProtoReflect() protoreflect.Message { + mi := &file_product_proto_msgTypes[114] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ResolveCategoryProductCountRequestKey.ProtoReflect.Descriptor instead. +func (*ResolveCategoryProductCountRequestKey) Descriptor() ([]byte, []int) { + return file_product_proto_rawDescGZIP(), []int{114} +} + +func (x *ResolveCategoryProductCountRequestKey) GetContext() *CategoryProductCountContext { + if x != nil { + return x.Context + } + return nil +} + +func (x *ResolveCategoryProductCountRequestKey) GetFieldArgs() *CategoryProductCountArgs { + if x != nil { + return x.FieldArgs + } + return nil +} + +type ResolveCategoryProductCountRequest struct { + state protoimpl.MessageState `protogen:"open.v1"` + Key []*ResolveCategoryProductCountRequestKey `protobuf:"bytes,1,rep,name=key,proto3" json:"key,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ResolveCategoryProductCountRequest) Reset() { + *x = ResolveCategoryProductCountRequest{} + mi := &file_product_proto_msgTypes[115] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ResolveCategoryProductCountRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ResolveCategoryProductCountRequest) ProtoMessage() {} + +func (x *ResolveCategoryProductCountRequest) ProtoReflect() protoreflect.Message { + mi := &file_product_proto_msgTypes[115] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ResolveCategoryProductCountRequest.ProtoReflect.Descriptor instead. +func (*ResolveCategoryProductCountRequest) Descriptor() ([]byte, []int) { + return file_product_proto_rawDescGZIP(), []int{115} +} + +func (x *ResolveCategoryProductCountRequest) GetKey() []*ResolveCategoryProductCountRequestKey { + if x != nil { + return x.Key + } + return nil +} + +type ResolveCategoryProductCountResponseResult struct { + state protoimpl.MessageState `protogen:"open.v1"` + ProductCount int32 `protobuf:"varint,1,opt,name=product_count,json=productCount,proto3" json:"product_count,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ResolveCategoryProductCountResponseResult) Reset() { + *x = ResolveCategoryProductCountResponseResult{} + mi := &file_product_proto_msgTypes[116] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ResolveCategoryProductCountResponseResult) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ResolveCategoryProductCountResponseResult) ProtoMessage() {} + +func (x *ResolveCategoryProductCountResponseResult) ProtoReflect() protoreflect.Message { + mi := &file_product_proto_msgTypes[116] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ResolveCategoryProductCountResponseResult.ProtoReflect.Descriptor instead. +func (*ResolveCategoryProductCountResponseResult) Descriptor() ([]byte, []int) { + return file_product_proto_rawDescGZIP(), []int{116} +} + +func (x *ResolveCategoryProductCountResponseResult) GetProductCount() int32 { + if x != nil { + return x.ProductCount + } + return 0 +} + +type ResolveCategoryProductCountResponse struct { + state protoimpl.MessageState `protogen:"open.v1"` + Result []*ResolveCategoryProductCountResponseResult `protobuf:"bytes,1,rep,name=result,proto3" json:"result,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ResolveCategoryProductCountResponse) Reset() { + *x = ResolveCategoryProductCountResponse{} + mi := &file_product_proto_msgTypes[117] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ResolveCategoryProductCountResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ResolveCategoryProductCountResponse) ProtoMessage() {} + +func (x *ResolveCategoryProductCountResponse) ProtoReflect() protoreflect.Message { + mi := &file_product_proto_msgTypes[117] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ResolveCategoryProductCountResponse.ProtoReflect.Descriptor instead. +func (*ResolveCategoryProductCountResponse) Descriptor() ([]byte, []int) { + return file_product_proto_rawDescGZIP(), []int{117} +} + +func (x *ResolveCategoryProductCountResponse) GetResult() []*ResolveCategoryProductCountResponseResult { + if x != nil { + return x.Result + } + return nil +} + type Product struct { state protoimpl.MessageState `protogen:"open.v1"` Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` @@ -5104,7 +5386,7 @@ type Product struct { func (x *Product) Reset() { *x = Product{} - mi := &file_product_proto_msgTypes[112] + mi := &file_product_proto_msgTypes[118] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5116,7 +5398,7 @@ func (x *Product) String() string { func (*Product) ProtoMessage() {} func (x *Product) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[112] + mi := &file_product_proto_msgTypes[118] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5129,7 +5411,7 @@ func (x *Product) ProtoReflect() protoreflect.Message { // Deprecated: Use Product.ProtoReflect.Descriptor instead. func (*Product) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{112} + return file_product_proto_rawDescGZIP(), []int{118} } func (x *Product) GetId() string { @@ -5164,7 +5446,7 @@ type Storage struct { func (x *Storage) Reset() { *x = Storage{} - mi := &file_product_proto_msgTypes[113] + mi := &file_product_proto_msgTypes[119] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5176,7 +5458,7 @@ func (x *Storage) String() string { func (*Storage) ProtoMessage() {} func (x *Storage) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[113] + mi := &file_product_proto_msgTypes[119] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5189,7 +5471,7 @@ func (x *Storage) ProtoReflect() protoreflect.Message { // Deprecated: Use Storage.ProtoReflect.Descriptor instead. func (*Storage) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{113} + return file_product_proto_rawDescGZIP(), []int{119} } func (x *Storage) GetId() string { @@ -5224,7 +5506,7 @@ type Warehouse struct { func (x *Warehouse) Reset() { *x = Warehouse{} - mi := &file_product_proto_msgTypes[114] + mi := &file_product_proto_msgTypes[120] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5236,7 +5518,7 @@ func (x *Warehouse) String() string { func (*Warehouse) ProtoMessage() {} func (x *Warehouse) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[114] + mi := &file_product_proto_msgTypes[120] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5249,7 +5531,7 @@ func (x *Warehouse) ProtoReflect() protoreflect.Message { // Deprecated: Use Warehouse.ProtoReflect.Descriptor instead. func (*Warehouse) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{114} + return file_product_proto_rawDescGZIP(), []int{120} } func (x *Warehouse) GetId() string { @@ -5283,7 +5565,7 @@ type User struct { func (x *User) Reset() { *x = User{} - mi := &file_product_proto_msgTypes[115] + mi := &file_product_proto_msgTypes[121] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5295,7 +5577,7 @@ func (x *User) String() string { func (*User) ProtoMessage() {} func (x *User) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[115] + mi := &file_product_proto_msgTypes[121] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5308,7 +5590,7 @@ func (x *User) ProtoReflect() protoreflect.Message { // Deprecated: Use User.ProtoReflect.Descriptor instead. func (*User) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{115} + return file_product_proto_rawDescGZIP(), []int{121} } func (x *User) GetId() string { @@ -5336,7 +5618,7 @@ type NestedTypeA struct { func (x *NestedTypeA) Reset() { *x = NestedTypeA{} - mi := &file_product_proto_msgTypes[116] + mi := &file_product_proto_msgTypes[122] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5348,7 +5630,7 @@ func (x *NestedTypeA) String() string { func (*NestedTypeA) ProtoMessage() {} func (x *NestedTypeA) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[116] + mi := &file_product_proto_msgTypes[122] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5361,7 +5643,7 @@ func (x *NestedTypeA) ProtoReflect() protoreflect.Message { // Deprecated: Use NestedTypeA.ProtoReflect.Descriptor instead. func (*NestedTypeA) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{116} + return file_product_proto_rawDescGZIP(), []int{122} } func (x *NestedTypeA) GetId() string { @@ -5396,7 +5678,7 @@ type RecursiveType struct { func (x *RecursiveType) Reset() { *x = RecursiveType{} - mi := &file_product_proto_msgTypes[117] + mi := &file_product_proto_msgTypes[123] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5408,7 +5690,7 @@ func (x *RecursiveType) String() string { func (*RecursiveType) ProtoMessage() {} func (x *RecursiveType) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[117] + mi := &file_product_proto_msgTypes[123] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5421,7 +5703,7 @@ func (x *RecursiveType) ProtoReflect() protoreflect.Message { // Deprecated: Use RecursiveType.ProtoReflect.Descriptor instead. func (*RecursiveType) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{117} + return file_product_proto_rawDescGZIP(), []int{123} } func (x *RecursiveType) GetId() string { @@ -5457,7 +5739,7 @@ type TypeWithMultipleFilterFields struct { func (x *TypeWithMultipleFilterFields) Reset() { *x = TypeWithMultipleFilterFields{} - mi := &file_product_proto_msgTypes[118] + mi := &file_product_proto_msgTypes[124] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5469,7 +5751,7 @@ func (x *TypeWithMultipleFilterFields) String() string { func (*TypeWithMultipleFilterFields) ProtoMessage() {} func (x *TypeWithMultipleFilterFields) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[118] + mi := &file_product_proto_msgTypes[124] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5482,7 +5764,7 @@ func (x *TypeWithMultipleFilterFields) ProtoReflect() protoreflect.Message { // Deprecated: Use TypeWithMultipleFilterFields.ProtoReflect.Descriptor instead. func (*TypeWithMultipleFilterFields) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{118} + return file_product_proto_rawDescGZIP(), []int{124} } func (x *TypeWithMultipleFilterFields) GetId() string { @@ -5523,7 +5805,7 @@ type FilterTypeInput struct { func (x *FilterTypeInput) Reset() { *x = FilterTypeInput{} - mi := &file_product_proto_msgTypes[119] + mi := &file_product_proto_msgTypes[125] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5535,7 +5817,7 @@ func (x *FilterTypeInput) String() string { func (*FilterTypeInput) ProtoMessage() {} func (x *FilterTypeInput) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[119] + mi := &file_product_proto_msgTypes[125] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5548,7 +5830,7 @@ func (x *FilterTypeInput) ProtoReflect() protoreflect.Message { // Deprecated: Use FilterTypeInput.ProtoReflect.Descriptor instead. func (*FilterTypeInput) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{119} + return file_product_proto_rawDescGZIP(), []int{125} } func (x *FilterTypeInput) GetFilterField_1() string { @@ -5574,7 +5856,7 @@ type ComplexFilterTypeInput struct { func (x *ComplexFilterTypeInput) Reset() { *x = ComplexFilterTypeInput{} - mi := &file_product_proto_msgTypes[120] + mi := &file_product_proto_msgTypes[126] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5586,7 +5868,7 @@ func (x *ComplexFilterTypeInput) String() string { func (*ComplexFilterTypeInput) ProtoMessage() {} func (x *ComplexFilterTypeInput) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[120] + mi := &file_product_proto_msgTypes[126] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5599,7 +5881,7 @@ func (x *ComplexFilterTypeInput) ProtoReflect() protoreflect.Message { // Deprecated: Use ComplexFilterTypeInput.ProtoReflect.Descriptor instead. func (*ComplexFilterTypeInput) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{120} + return file_product_proto_rawDescGZIP(), []int{126} } func (x *ComplexFilterTypeInput) GetFilter() *FilterType { @@ -5619,7 +5901,7 @@ type TypeWithComplexFilterInput struct { func (x *TypeWithComplexFilterInput) Reset() { *x = TypeWithComplexFilterInput{} - mi := &file_product_proto_msgTypes[121] + mi := &file_product_proto_msgTypes[127] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5631,7 +5913,7 @@ func (x *TypeWithComplexFilterInput) String() string { func (*TypeWithComplexFilterInput) ProtoMessage() {} func (x *TypeWithComplexFilterInput) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[121] + mi := &file_product_proto_msgTypes[127] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5644,7 +5926,7 @@ func (x *TypeWithComplexFilterInput) ProtoReflect() protoreflect.Message { // Deprecated: Use TypeWithComplexFilterInput.ProtoReflect.Descriptor instead. func (*TypeWithComplexFilterInput) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{121} + return file_product_proto_rawDescGZIP(), []int{127} } func (x *TypeWithComplexFilterInput) GetId() string { @@ -5672,7 +5954,7 @@ type OrderInput struct { func (x *OrderInput) Reset() { *x = OrderInput{} - mi := &file_product_proto_msgTypes[122] + mi := &file_product_proto_msgTypes[128] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5684,7 +5966,7 @@ func (x *OrderInput) String() string { func (*OrderInput) ProtoMessage() {} func (x *OrderInput) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[122] + mi := &file_product_proto_msgTypes[128] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5697,7 +5979,7 @@ func (x *OrderInput) ProtoReflect() protoreflect.Message { // Deprecated: Use OrderInput.ProtoReflect.Descriptor instead. func (*OrderInput) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{122} + return file_product_proto_rawDescGZIP(), []int{128} } func (x *OrderInput) GetOrderId() string { @@ -5733,7 +6015,7 @@ type Order struct { func (x *Order) Reset() { *x = Order{} - mi := &file_product_proto_msgTypes[123] + mi := &file_product_proto_msgTypes[129] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5745,7 +6027,7 @@ func (x *Order) String() string { func (*Order) ProtoMessage() {} func (x *Order) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[123] + mi := &file_product_proto_msgTypes[129] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5758,7 +6040,7 @@ func (x *Order) ProtoReflect() protoreflect.Message { // Deprecated: Use Order.ProtoReflect.Descriptor instead. func (*Order) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{123} + return file_product_proto_rawDescGZIP(), []int{129} } func (x *Order) GetOrderId() string { @@ -5800,7 +6082,7 @@ type Category struct { func (x *Category) Reset() { *x = Category{} - mi := &file_product_proto_msgTypes[124] + mi := &file_product_proto_msgTypes[130] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5812,7 +6094,7 @@ func (x *Category) String() string { func (*Category) ProtoMessage() {} func (x *Category) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[124] + mi := &file_product_proto_msgTypes[130] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5825,7 +6107,7 @@ func (x *Category) ProtoReflect() protoreflect.Message { // Deprecated: Use Category.ProtoReflect.Descriptor instead. func (*Category) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{124} + return file_product_proto_rawDescGZIP(), []int{130} } func (x *Category) GetId() string { @@ -5859,7 +6141,7 @@ type CategoryFilter struct { func (x *CategoryFilter) Reset() { *x = CategoryFilter{} - mi := &file_product_proto_msgTypes[125] + mi := &file_product_proto_msgTypes[131] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5871,7 +6153,7 @@ func (x *CategoryFilter) String() string { func (*CategoryFilter) ProtoMessage() {} func (x *CategoryFilter) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[125] + mi := &file_product_proto_msgTypes[131] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5884,7 +6166,7 @@ func (x *CategoryFilter) ProtoReflect() protoreflect.Message { // Deprecated: Use CategoryFilter.ProtoReflect.Descriptor instead. func (*CategoryFilter) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{125} + return file_product_proto_rawDescGZIP(), []int{131} } func (x *CategoryFilter) GetCategory() CategoryKind { @@ -5914,7 +6196,7 @@ type Animal struct { func (x *Animal) Reset() { *x = Animal{} - mi := &file_product_proto_msgTypes[126] + mi := &file_product_proto_msgTypes[132] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5926,7 +6208,7 @@ func (x *Animal) String() string { func (*Animal) ProtoMessage() {} func (x *Animal) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[126] + mi := &file_product_proto_msgTypes[132] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5939,7 +6221,7 @@ func (x *Animal) ProtoReflect() protoreflect.Message { // Deprecated: Use Animal.ProtoReflect.Descriptor instead. func (*Animal) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{126} + return file_product_proto_rawDescGZIP(), []int{132} } func (x *Animal) GetInstance() isAnimal_Instance { @@ -5993,7 +6275,7 @@ type SearchInput struct { func (x *SearchInput) Reset() { *x = SearchInput{} - mi := &file_product_proto_msgTypes[127] + mi := &file_product_proto_msgTypes[133] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6005,7 +6287,7 @@ func (x *SearchInput) String() string { func (*SearchInput) ProtoMessage() {} func (x *SearchInput) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[127] + mi := &file_product_proto_msgTypes[133] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6018,7 +6300,7 @@ func (x *SearchInput) ProtoReflect() protoreflect.Message { // Deprecated: Use SearchInput.ProtoReflect.Descriptor instead. func (*SearchInput) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{127} + return file_product_proto_rawDescGZIP(), []int{133} } func (x *SearchInput) GetQuery() string { @@ -6049,7 +6331,7 @@ type SearchResult struct { func (x *SearchResult) Reset() { *x = SearchResult{} - mi := &file_product_proto_msgTypes[128] + mi := &file_product_proto_msgTypes[134] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6061,7 +6343,7 @@ func (x *SearchResult) String() string { func (*SearchResult) ProtoMessage() {} func (x *SearchResult) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[128] + mi := &file_product_proto_msgTypes[134] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6074,7 +6356,7 @@ func (x *SearchResult) ProtoReflect() protoreflect.Message { // Deprecated: Use SearchResult.ProtoReflect.Descriptor instead. func (*SearchResult) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{128} + return file_product_proto_rawDescGZIP(), []int{134} } func (x *SearchResult) GetValue() isSearchResult_Value { @@ -6149,7 +6431,7 @@ type NullableFieldsType struct { func (x *NullableFieldsType) Reset() { *x = NullableFieldsType{} - mi := &file_product_proto_msgTypes[129] + mi := &file_product_proto_msgTypes[135] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6161,7 +6443,7 @@ func (x *NullableFieldsType) String() string { func (*NullableFieldsType) ProtoMessage() {} func (x *NullableFieldsType) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[129] + mi := &file_product_proto_msgTypes[135] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6174,7 +6456,7 @@ func (x *NullableFieldsType) ProtoReflect() protoreflect.Message { // Deprecated: Use NullableFieldsType.ProtoReflect.Descriptor instead. func (*NullableFieldsType) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{129} + return file_product_proto_rawDescGZIP(), []int{135} } func (x *NullableFieldsType) GetId() string { @@ -6244,7 +6526,7 @@ type NullableFieldsFilter struct { func (x *NullableFieldsFilter) Reset() { *x = NullableFieldsFilter{} - mi := &file_product_proto_msgTypes[130] + mi := &file_product_proto_msgTypes[136] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6256,7 +6538,7 @@ func (x *NullableFieldsFilter) String() string { func (*NullableFieldsFilter) ProtoMessage() {} func (x *NullableFieldsFilter) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[130] + mi := &file_product_proto_msgTypes[136] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6269,7 +6551,7 @@ func (x *NullableFieldsFilter) ProtoReflect() protoreflect.Message { // Deprecated: Use NullableFieldsFilter.ProtoReflect.Descriptor instead. func (*NullableFieldsFilter) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{130} + return file_product_proto_rawDescGZIP(), []int{136} } func (x *NullableFieldsFilter) GetName() *wrapperspb.StringValue { @@ -6321,7 +6603,7 @@ type BlogPost struct { func (x *BlogPost) Reset() { *x = BlogPost{} - mi := &file_product_proto_msgTypes[131] + mi := &file_product_proto_msgTypes[137] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6333,7 +6615,7 @@ func (x *BlogPost) String() string { func (*BlogPost) ProtoMessage() {} func (x *BlogPost) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[131] + mi := &file_product_proto_msgTypes[137] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6346,7 +6628,7 @@ func (x *BlogPost) ProtoReflect() protoreflect.Message { // Deprecated: Use BlogPost.ProtoReflect.Descriptor instead. func (*BlogPost) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{131} + return file_product_proto_rawDescGZIP(), []int{137} } func (x *BlogPost) GetId() string { @@ -6500,7 +6782,7 @@ type BlogPostFilter struct { func (x *BlogPostFilter) Reset() { *x = BlogPostFilter{} - mi := &file_product_proto_msgTypes[132] + mi := &file_product_proto_msgTypes[138] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6512,7 +6794,7 @@ func (x *BlogPostFilter) String() string { func (*BlogPostFilter) ProtoMessage() {} func (x *BlogPostFilter) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[132] + mi := &file_product_proto_msgTypes[138] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6525,7 +6807,7 @@ func (x *BlogPostFilter) ProtoReflect() protoreflect.Message { // Deprecated: Use BlogPostFilter.ProtoReflect.Descriptor instead. func (*BlogPostFilter) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{132} + return file_product_proto_rawDescGZIP(), []int{138} } func (x *BlogPostFilter) GetTitle() *wrapperspb.StringValue { @@ -6572,7 +6854,7 @@ type Author struct { func (x *Author) Reset() { *x = Author{} - mi := &file_product_proto_msgTypes[133] + mi := &file_product_proto_msgTypes[139] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6584,7 +6866,7 @@ func (x *Author) String() string { func (*Author) ProtoMessage() {} func (x *Author) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[133] + mi := &file_product_proto_msgTypes[139] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6597,7 +6879,7 @@ func (x *Author) ProtoReflect() protoreflect.Message { // Deprecated: Use Author.ProtoReflect.Descriptor instead. func (*Author) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{133} + return file_product_proto_rawDescGZIP(), []int{139} } func (x *Author) GetId() string { @@ -6716,7 +6998,7 @@ type AuthorFilter struct { func (x *AuthorFilter) Reset() { *x = AuthorFilter{} - mi := &file_product_proto_msgTypes[134] + mi := &file_product_proto_msgTypes[140] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6728,7 +7010,7 @@ func (x *AuthorFilter) String() string { func (*AuthorFilter) ProtoMessage() {} func (x *AuthorFilter) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[134] + mi := &file_product_proto_msgTypes[140] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6741,7 +7023,7 @@ func (x *AuthorFilter) ProtoReflect() protoreflect.Message { // Deprecated: Use AuthorFilter.ProtoReflect.Descriptor instead. func (*AuthorFilter) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{134} + return file_product_proto_rawDescGZIP(), []int{140} } func (x *AuthorFilter) GetName() *wrapperspb.StringValue { @@ -6774,7 +7056,7 @@ type UserInput struct { func (x *UserInput) Reset() { *x = UserInput{} - mi := &file_product_proto_msgTypes[135] + mi := &file_product_proto_msgTypes[141] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6786,7 +7068,7 @@ func (x *UserInput) String() string { func (*UserInput) ProtoMessage() {} func (x *UserInput) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[135] + mi := &file_product_proto_msgTypes[141] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6799,7 +7081,7 @@ func (x *UserInput) ProtoReflect() protoreflect.Message { // Deprecated: Use UserInput.ProtoReflect.Descriptor instead. func (*UserInput) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{135} + return file_product_proto_rawDescGZIP(), []int{141} } func (x *UserInput) GetName() string { @@ -6819,7 +7101,7 @@ type ActionInput struct { func (x *ActionInput) Reset() { *x = ActionInput{} - mi := &file_product_proto_msgTypes[136] + mi := &file_product_proto_msgTypes[142] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6831,7 +7113,7 @@ func (x *ActionInput) String() string { func (*ActionInput) ProtoMessage() {} func (x *ActionInput) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[136] + mi := &file_product_proto_msgTypes[142] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6844,7 +7126,7 @@ func (x *ActionInput) ProtoReflect() protoreflect.Message { // Deprecated: Use ActionInput.ProtoReflect.Descriptor instead. func (*ActionInput) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{136} + return file_product_proto_rawDescGZIP(), []int{142} } func (x *ActionInput) GetType() string { @@ -6874,7 +7156,7 @@ type ActionResult struct { func (x *ActionResult) Reset() { *x = ActionResult{} - mi := &file_product_proto_msgTypes[137] + mi := &file_product_proto_msgTypes[143] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6886,7 +7168,7 @@ func (x *ActionResult) String() string { func (*ActionResult) ProtoMessage() {} func (x *ActionResult) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[137] + mi := &file_product_proto_msgTypes[143] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6899,7 +7181,7 @@ func (x *ActionResult) ProtoReflect() protoreflect.Message { // Deprecated: Use ActionResult.ProtoReflect.Descriptor instead. func (*ActionResult) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{137} + return file_product_proto_rawDescGZIP(), []int{143} } func (x *ActionResult) GetValue() isActionResult_Value { @@ -6958,7 +7240,7 @@ type NullableFieldsInput struct { func (x *NullableFieldsInput) Reset() { *x = NullableFieldsInput{} - mi := &file_product_proto_msgTypes[138] + mi := &file_product_proto_msgTypes[144] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6970,7 +7252,7 @@ func (x *NullableFieldsInput) String() string { func (*NullableFieldsInput) ProtoMessage() {} func (x *NullableFieldsInput) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[138] + mi := &file_product_proto_msgTypes[144] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6983,7 +7265,7 @@ func (x *NullableFieldsInput) ProtoReflect() protoreflect.Message { // Deprecated: Use NullableFieldsInput.ProtoReflect.Descriptor instead. func (*NullableFieldsInput) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{138} + return file_product_proto_rawDescGZIP(), []int{144} } func (x *NullableFieldsInput) GetName() string { @@ -7059,7 +7341,7 @@ type BlogPostInput struct { func (x *BlogPostInput) Reset() { *x = BlogPostInput{} - mi := &file_product_proto_msgTypes[139] + mi := &file_product_proto_msgTypes[145] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7071,7 +7353,7 @@ func (x *BlogPostInput) String() string { func (*BlogPostInput) ProtoMessage() {} func (x *BlogPostInput) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[139] + mi := &file_product_proto_msgTypes[145] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7084,7 +7366,7 @@ func (x *BlogPostInput) ProtoReflect() protoreflect.Message { // Deprecated: Use BlogPostInput.ProtoReflect.Descriptor instead. func (*BlogPostInput) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{139} + return file_product_proto_rawDescGZIP(), []int{145} } func (x *BlogPostInput) GetTitle() string { @@ -7217,7 +7499,7 @@ type AuthorInput struct { func (x *AuthorInput) Reset() { *x = AuthorInput{} - mi := &file_product_proto_msgTypes[140] + mi := &file_product_proto_msgTypes[146] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7229,7 +7511,7 @@ func (x *AuthorInput) String() string { func (*AuthorInput) ProtoMessage() {} func (x *AuthorInput) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[140] + mi := &file_product_proto_msgTypes[146] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7242,7 +7524,7 @@ func (x *AuthorInput) ProtoReflect() protoreflect.Message { // Deprecated: Use AuthorInput.ProtoReflect.Descriptor instead. func (*AuthorInput) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{140} + return file_product_proto_rawDescGZIP(), []int{146} } func (x *AuthorInput) GetName() string { @@ -7326,7 +7608,7 @@ type NestedTypeB struct { func (x *NestedTypeB) Reset() { *x = NestedTypeB{} - mi := &file_product_proto_msgTypes[141] + mi := &file_product_proto_msgTypes[147] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7338,7 +7620,7 @@ func (x *NestedTypeB) String() string { func (*NestedTypeB) ProtoMessage() {} func (x *NestedTypeB) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[141] + mi := &file_product_proto_msgTypes[147] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7351,7 +7633,7 @@ func (x *NestedTypeB) ProtoReflect() protoreflect.Message { // Deprecated: Use NestedTypeB.ProtoReflect.Descriptor instead. func (*NestedTypeB) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{141} + return file_product_proto_rawDescGZIP(), []int{147} } func (x *NestedTypeB) GetId() string { @@ -7385,7 +7667,7 @@ type NestedTypeC struct { func (x *NestedTypeC) Reset() { *x = NestedTypeC{} - mi := &file_product_proto_msgTypes[142] + mi := &file_product_proto_msgTypes[148] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7397,7 +7679,7 @@ func (x *NestedTypeC) String() string { func (*NestedTypeC) ProtoMessage() {} func (x *NestedTypeC) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[142] + mi := &file_product_proto_msgTypes[148] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7410,7 +7692,7 @@ func (x *NestedTypeC) ProtoReflect() protoreflect.Message { // Deprecated: Use NestedTypeC.ProtoReflect.Descriptor instead. func (*NestedTypeC) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{142} + return file_product_proto_rawDescGZIP(), []int{148} } func (x *NestedTypeC) GetId() string { @@ -7439,7 +7721,7 @@ type FilterType struct { func (x *FilterType) Reset() { *x = FilterType{} - mi := &file_product_proto_msgTypes[143] + mi := &file_product_proto_msgTypes[149] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7451,7 +7733,7 @@ func (x *FilterType) String() string { func (*FilterType) ProtoMessage() {} func (x *FilterType) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[143] + mi := &file_product_proto_msgTypes[149] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7464,7 +7746,7 @@ func (x *FilterType) ProtoReflect() protoreflect.Message { // Deprecated: Use FilterType.ProtoReflect.Descriptor instead. func (*FilterType) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{143} + return file_product_proto_rawDescGZIP(), []int{149} } func (x *FilterType) GetName() string { @@ -7505,7 +7787,7 @@ type Pagination struct { func (x *Pagination) Reset() { *x = Pagination{} - mi := &file_product_proto_msgTypes[144] + mi := &file_product_proto_msgTypes[150] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7517,7 +7799,7 @@ func (x *Pagination) String() string { func (*Pagination) ProtoMessage() {} func (x *Pagination) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[144] + mi := &file_product_proto_msgTypes[150] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7530,7 +7812,7 @@ func (x *Pagination) ProtoReflect() protoreflect.Message { // Deprecated: Use Pagination.ProtoReflect.Descriptor instead. func (*Pagination) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{144} + return file_product_proto_rawDescGZIP(), []int{150} } func (x *Pagination) GetPage() int32 { @@ -7558,7 +7840,7 @@ type OrderLineInput struct { func (x *OrderLineInput) Reset() { *x = OrderLineInput{} - mi := &file_product_proto_msgTypes[145] + mi := &file_product_proto_msgTypes[151] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7570,7 +7852,7 @@ func (x *OrderLineInput) String() string { func (*OrderLineInput) ProtoMessage() {} func (x *OrderLineInput) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[145] + mi := &file_product_proto_msgTypes[151] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7583,7 +7865,7 @@ func (x *OrderLineInput) ProtoReflect() protoreflect.Message { // Deprecated: Use OrderLineInput.ProtoReflect.Descriptor instead. func (*OrderLineInput) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{145} + return file_product_proto_rawDescGZIP(), []int{151} } func (x *OrderLineInput) GetProductId() string { @@ -7618,7 +7900,7 @@ type OrderLine struct { func (x *OrderLine) Reset() { *x = OrderLine{} - mi := &file_product_proto_msgTypes[146] + mi := &file_product_proto_msgTypes[152] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7630,7 +7912,7 @@ func (x *OrderLine) String() string { func (*OrderLine) ProtoMessage() {} func (x *OrderLine) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[146] + mi := &file_product_proto_msgTypes[152] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7643,7 +7925,7 @@ func (x *OrderLine) ProtoReflect() protoreflect.Message { // Deprecated: Use OrderLine.ProtoReflect.Descriptor instead. func (*OrderLine) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{146} + return file_product_proto_rawDescGZIP(), []int{152} } func (x *OrderLine) GetProductId() string { @@ -7679,7 +7961,7 @@ type Cat struct { func (x *Cat) Reset() { *x = Cat{} - mi := &file_product_proto_msgTypes[147] + mi := &file_product_proto_msgTypes[153] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7691,7 +7973,7 @@ func (x *Cat) String() string { func (*Cat) ProtoMessage() {} func (x *Cat) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[147] + mi := &file_product_proto_msgTypes[153] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7704,7 +7986,7 @@ func (x *Cat) ProtoReflect() protoreflect.Message { // Deprecated: Use Cat.ProtoReflect.Descriptor instead. func (*Cat) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{147} + return file_product_proto_rawDescGZIP(), []int{153} } func (x *Cat) GetId() string { @@ -7747,7 +8029,7 @@ type Dog struct { func (x *Dog) Reset() { *x = Dog{} - mi := &file_product_proto_msgTypes[148] + mi := &file_product_proto_msgTypes[154] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7759,7 +8041,7 @@ func (x *Dog) String() string { func (*Dog) ProtoMessage() {} func (x *Dog) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[148] + mi := &file_product_proto_msgTypes[154] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7772,7 +8054,7 @@ func (x *Dog) ProtoReflect() protoreflect.Message { // Deprecated: Use Dog.ProtoReflect.Descriptor instead. func (*Dog) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{148} + return file_product_proto_rawDescGZIP(), []int{154} } func (x *Dog) GetId() string { @@ -7813,7 +8095,7 @@ type ActionSuccess struct { func (x *ActionSuccess) Reset() { *x = ActionSuccess{} - mi := &file_product_proto_msgTypes[149] + mi := &file_product_proto_msgTypes[155] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7825,7 +8107,7 @@ func (x *ActionSuccess) String() string { func (*ActionSuccess) ProtoMessage() {} func (x *ActionSuccess) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[149] + mi := &file_product_proto_msgTypes[155] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7838,7 +8120,7 @@ func (x *ActionSuccess) ProtoReflect() protoreflect.Message { // Deprecated: Use ActionSuccess.ProtoReflect.Descriptor instead. func (*ActionSuccess) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{149} + return file_product_proto_rawDescGZIP(), []int{155} } func (x *ActionSuccess) GetMessage() string { @@ -7865,7 +8147,7 @@ type ActionError struct { func (x *ActionError) Reset() { *x = ActionError{} - mi := &file_product_proto_msgTypes[150] + mi := &file_product_proto_msgTypes[156] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7877,7 +8159,7 @@ func (x *ActionError) String() string { func (*ActionError) ProtoMessage() {} func (x *ActionError) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[150] + mi := &file_product_proto_msgTypes[156] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7890,7 +8172,7 @@ func (x *ActionError) ProtoReflect() protoreflect.Message { // Deprecated: Use ActionError.ProtoReflect.Descriptor instead. func (*ActionError) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{150} + return file_product_proto_rawDescGZIP(), []int{156} } func (x *ActionError) GetMessage() string { @@ -7917,7 +8199,7 @@ type CategoryInput struct { func (x *CategoryInput) Reset() { *x = CategoryInput{} - mi := &file_product_proto_msgTypes[151] + mi := &file_product_proto_msgTypes[157] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7929,7 +8211,7 @@ func (x *CategoryInput) String() string { func (*CategoryInput) ProtoMessage() {} func (x *CategoryInput) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[151] + mi := &file_product_proto_msgTypes[157] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7942,7 +8224,7 @@ func (x *CategoryInput) ProtoReflect() protoreflect.Message { // Deprecated: Use CategoryInput.ProtoReflect.Descriptor instead. func (*CategoryInput) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{151} + return file_product_proto_rawDescGZIP(), []int{157} } func (x *CategoryInput) GetName() string { @@ -7959,6 +8241,74 @@ func (x *CategoryInput) GetKind() CategoryKind { return CategoryKind_CATEGORY_KIND_UNSPECIFIED } +type ProductCountFilter struct { + state protoimpl.MessageState `protogen:"open.v1"` + MinPrice *wrapperspb.DoubleValue `protobuf:"bytes,1,opt,name=min_price,json=minPrice,proto3" json:"min_price,omitempty"` + MaxPrice *wrapperspb.DoubleValue `protobuf:"bytes,2,opt,name=max_price,json=maxPrice,proto3" json:"max_price,omitempty"` + InStock *wrapperspb.BoolValue `protobuf:"bytes,3,opt,name=in_stock,json=inStock,proto3" json:"in_stock,omitempty"` + SearchTerm *wrapperspb.StringValue `protobuf:"bytes,4,opt,name=search_term,json=searchTerm,proto3" json:"search_term,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ProductCountFilter) Reset() { + *x = ProductCountFilter{} + mi := &file_product_proto_msgTypes[158] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ProductCountFilter) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ProductCountFilter) ProtoMessage() {} + +func (x *ProductCountFilter) ProtoReflect() protoreflect.Message { + mi := &file_product_proto_msgTypes[158] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ProductCountFilter.ProtoReflect.Descriptor instead. +func (*ProductCountFilter) Descriptor() ([]byte, []int) { + return file_product_proto_rawDescGZIP(), []int{158} +} + +func (x *ProductCountFilter) GetMinPrice() *wrapperspb.DoubleValue { + if x != nil { + return x.MinPrice + } + return nil +} + +func (x *ProductCountFilter) GetMaxPrice() *wrapperspb.DoubleValue { + if x != nil { + return x.MaxPrice + } + return nil +} + +func (x *ProductCountFilter) GetInStock() *wrapperspb.BoolValue { + if x != nil { + return x.InStock + } + return nil +} + +func (x *ProductCountFilter) GetSearchTerm() *wrapperspb.StringValue { + if x != nil { + return x.SearchTerm + } + return nil +} + type ListOfAuthorFilter_List struct { state protoimpl.MessageState `protogen:"open.v1"` Items []*AuthorFilter `protobuf:"bytes,1,rep,name=items,proto3" json:"items,omitempty"` @@ -7968,7 +8318,7 @@ type ListOfAuthorFilter_List struct { func (x *ListOfAuthorFilter_List) Reset() { *x = ListOfAuthorFilter_List{} - mi := &file_product_proto_msgTypes[152] + mi := &file_product_proto_msgTypes[159] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7980,7 +8330,7 @@ func (x *ListOfAuthorFilter_List) String() string { func (*ListOfAuthorFilter_List) ProtoMessage() {} func (x *ListOfAuthorFilter_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[152] + mi := &file_product_proto_msgTypes[159] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8012,7 +8362,7 @@ type ListOfAuthorInput_List struct { func (x *ListOfAuthorInput_List) Reset() { *x = ListOfAuthorInput_List{} - mi := &file_product_proto_msgTypes[153] + mi := &file_product_proto_msgTypes[160] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8024,7 +8374,7 @@ func (x *ListOfAuthorInput_List) String() string { func (*ListOfAuthorInput_List) ProtoMessage() {} func (x *ListOfAuthorInput_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[153] + mi := &file_product_proto_msgTypes[160] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8056,7 +8406,7 @@ type ListOfBlogPost_List struct { func (x *ListOfBlogPost_List) Reset() { *x = ListOfBlogPost_List{} - mi := &file_product_proto_msgTypes[154] + mi := &file_product_proto_msgTypes[161] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8068,7 +8418,7 @@ func (x *ListOfBlogPost_List) String() string { func (*ListOfBlogPost_List) ProtoMessage() {} func (x *ListOfBlogPost_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[154] + mi := &file_product_proto_msgTypes[161] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8100,7 +8450,7 @@ type ListOfBlogPostFilter_List struct { func (x *ListOfBlogPostFilter_List) Reset() { *x = ListOfBlogPostFilter_List{} - mi := &file_product_proto_msgTypes[155] + mi := &file_product_proto_msgTypes[162] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8112,7 +8462,7 @@ func (x *ListOfBlogPostFilter_List) String() string { func (*ListOfBlogPostFilter_List) ProtoMessage() {} func (x *ListOfBlogPostFilter_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[155] + mi := &file_product_proto_msgTypes[162] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8144,7 +8494,7 @@ type ListOfBlogPostInput_List struct { func (x *ListOfBlogPostInput_List) Reset() { *x = ListOfBlogPostInput_List{} - mi := &file_product_proto_msgTypes[156] + mi := &file_product_proto_msgTypes[163] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8156,7 +8506,7 @@ func (x *ListOfBlogPostInput_List) String() string { func (*ListOfBlogPostInput_List) ProtoMessage() {} func (x *ListOfBlogPostInput_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[156] + mi := &file_product_proto_msgTypes[163] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8188,7 +8538,7 @@ type ListOfBoolean_List struct { func (x *ListOfBoolean_List) Reset() { *x = ListOfBoolean_List{} - mi := &file_product_proto_msgTypes[157] + mi := &file_product_proto_msgTypes[164] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8200,7 +8550,7 @@ func (x *ListOfBoolean_List) String() string { func (*ListOfBoolean_List) ProtoMessage() {} func (x *ListOfBoolean_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[157] + mi := &file_product_proto_msgTypes[164] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8232,7 +8582,7 @@ type ListOfCategory_List struct { func (x *ListOfCategory_List) Reset() { *x = ListOfCategory_List{} - mi := &file_product_proto_msgTypes[158] + mi := &file_product_proto_msgTypes[165] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8244,7 +8594,7 @@ func (x *ListOfCategory_List) String() string { func (*ListOfCategory_List) ProtoMessage() {} func (x *ListOfCategory_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[158] + mi := &file_product_proto_msgTypes[165] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8276,7 +8626,7 @@ type ListOfCategoryInput_List struct { func (x *ListOfCategoryInput_List) Reset() { *x = ListOfCategoryInput_List{} - mi := &file_product_proto_msgTypes[159] + mi := &file_product_proto_msgTypes[166] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8288,7 +8638,7 @@ func (x *ListOfCategoryInput_List) String() string { func (*ListOfCategoryInput_List) ProtoMessage() {} func (x *ListOfCategoryInput_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[159] + mi := &file_product_proto_msgTypes[166] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8320,7 +8670,7 @@ type ListOfFloat_List struct { func (x *ListOfFloat_List) Reset() { *x = ListOfFloat_List{} - mi := &file_product_proto_msgTypes[160] + mi := &file_product_proto_msgTypes[167] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8332,7 +8682,7 @@ func (x *ListOfFloat_List) String() string { func (*ListOfFloat_List) ProtoMessage() {} func (x *ListOfFloat_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[160] + mi := &file_product_proto_msgTypes[167] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8364,7 +8714,7 @@ type ListOfListOfCategory_List struct { func (x *ListOfListOfCategory_List) Reset() { *x = ListOfListOfCategory_List{} - mi := &file_product_proto_msgTypes[161] + mi := &file_product_proto_msgTypes[168] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8376,7 +8726,7 @@ func (x *ListOfListOfCategory_List) String() string { func (*ListOfListOfCategory_List) ProtoMessage() {} func (x *ListOfListOfCategory_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[161] + mi := &file_product_proto_msgTypes[168] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8408,7 +8758,7 @@ type ListOfListOfCategoryInput_List struct { func (x *ListOfListOfCategoryInput_List) Reset() { *x = ListOfListOfCategoryInput_List{} - mi := &file_product_proto_msgTypes[162] + mi := &file_product_proto_msgTypes[169] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8420,7 +8770,7 @@ func (x *ListOfListOfCategoryInput_List) String() string { func (*ListOfListOfCategoryInput_List) ProtoMessage() {} func (x *ListOfListOfCategoryInput_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[162] + mi := &file_product_proto_msgTypes[169] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8452,7 +8802,7 @@ type ListOfListOfString_List struct { func (x *ListOfListOfString_List) Reset() { *x = ListOfListOfString_List{} - mi := &file_product_proto_msgTypes[163] + mi := &file_product_proto_msgTypes[170] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8464,7 +8814,7 @@ func (x *ListOfListOfString_List) String() string { func (*ListOfListOfString_List) ProtoMessage() {} func (x *ListOfListOfString_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[163] + mi := &file_product_proto_msgTypes[170] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8496,7 +8846,7 @@ type ListOfListOfUser_List struct { func (x *ListOfListOfUser_List) Reset() { *x = ListOfListOfUser_List{} - mi := &file_product_proto_msgTypes[164] + mi := &file_product_proto_msgTypes[171] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8508,7 +8858,7 @@ func (x *ListOfListOfUser_List) String() string { func (*ListOfListOfUser_List) ProtoMessage() {} func (x *ListOfListOfUser_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[164] + mi := &file_product_proto_msgTypes[171] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8540,7 +8890,7 @@ type ListOfListOfUserInput_List struct { func (x *ListOfListOfUserInput_List) Reset() { *x = ListOfListOfUserInput_List{} - mi := &file_product_proto_msgTypes[165] + mi := &file_product_proto_msgTypes[172] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8552,7 +8902,7 @@ func (x *ListOfListOfUserInput_List) String() string { func (*ListOfListOfUserInput_List) ProtoMessage() {} func (x *ListOfListOfUserInput_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[165] + mi := &file_product_proto_msgTypes[172] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8584,7 +8934,7 @@ type ListOfOrderLine_List struct { func (x *ListOfOrderLine_List) Reset() { *x = ListOfOrderLine_List{} - mi := &file_product_proto_msgTypes[166] + mi := &file_product_proto_msgTypes[173] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8596,7 +8946,7 @@ func (x *ListOfOrderLine_List) String() string { func (*ListOfOrderLine_List) ProtoMessage() {} func (x *ListOfOrderLine_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[166] + mi := &file_product_proto_msgTypes[173] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8628,7 +8978,7 @@ type ListOfProduct_List struct { func (x *ListOfProduct_List) Reset() { *x = ListOfProduct_List{} - mi := &file_product_proto_msgTypes[167] + mi := &file_product_proto_msgTypes[174] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8640,7 +8990,7 @@ func (x *ListOfProduct_List) String() string { func (*ListOfProduct_List) ProtoMessage() {} func (x *ListOfProduct_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[167] + mi := &file_product_proto_msgTypes[174] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8672,7 +9022,7 @@ type ListOfString_List struct { func (x *ListOfString_List) Reset() { *x = ListOfString_List{} - mi := &file_product_proto_msgTypes[168] + mi := &file_product_proto_msgTypes[175] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8684,7 +9034,7 @@ func (x *ListOfString_List) String() string { func (*ListOfString_List) ProtoMessage() {} func (x *ListOfString_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[168] + mi := &file_product_proto_msgTypes[175] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8716,7 +9066,7 @@ type ListOfUser_List struct { func (x *ListOfUser_List) Reset() { *x = ListOfUser_List{} - mi := &file_product_proto_msgTypes[169] + mi := &file_product_proto_msgTypes[176] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8728,7 +9078,7 @@ func (x *ListOfUser_List) String() string { func (*ListOfUser_List) ProtoMessage() {} func (x *ListOfUser_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[169] + mi := &file_product_proto_msgTypes[176] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8760,7 +9110,7 @@ type ListOfUserInput_List struct { func (x *ListOfUserInput_List) Reset() { *x = ListOfUserInput_List{} - mi := &file_product_proto_msgTypes[170] + mi := &file_product_proto_msgTypes[177] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8772,7 +9122,7 @@ func (x *ListOfUserInput_List) String() string { func (*ListOfUserInput_List) ProtoMessage() {} func (x *ListOfUserInput_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[170] + mi := &file_product_proto_msgTypes[177] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9062,7 +9412,22 @@ const file_product_proto_rawDesc = "" + "\n" + "blog_posts\x18\x01 \x01(\v2\x1e.productv1.ListOfBlogPostInputR\tblogPosts\"o\n" + "#MutationBulkUpdateBlogPostsResponse\x12H\n" + - "\x16bulk_update_blog_posts\x18\x01 \x03(\v2\x13.productv1.BlogPostR\x13bulkUpdateBlogPosts\"C\n" + + "\x16bulk_update_blog_posts\x18\x01 \x03(\v2\x13.productv1.BlogPostR\x13bulkUpdateBlogPosts\"S\n" + + "\x18CategoryProductCountArgs\x127\n" + + "\afilters\x18\x01 \x01(\v2\x1d.productv1.ProductCountFilterR\afilters\"A\n" + + "\x1bCategoryProductCountContext\x12\x0e\n" + + "\x02id\x18\x01 \x01(\tR\x02id\x12\x12\n" + + "\x04name\x18\x02 \x01(\tR\x04name\"\xad\x01\n" + + "%ResolveCategoryProductCountRequestKey\x12@\n" + + "\acontext\x18\x01 \x01(\v2&.productv1.CategoryProductCountContextR\acontext\x12B\n" + + "\n" + + "field_args\x18\x02 \x01(\v2#.productv1.CategoryProductCountArgsR\tfieldArgs\"h\n" + + "\"ResolveCategoryProductCountRequest\x12B\n" + + "\x03key\x18\x01 \x03(\v20.productv1.ResolveCategoryProductCountRequestKeyR\x03key\"P\n" + + ")ResolveCategoryProductCountResponseResult\x12#\n" + + "\rproduct_count\x18\x01 \x01(\x05R\fproductCount\"s\n" + + "#ResolveCategoryProductCountResponse\x12L\n" + + "\x06result\x18\x01 \x03(\v24.productv1.ResolveCategoryProductCountResponseResultR\x06result\"C\n" + "\aProduct\x12\x0e\n" + "\x02id\x18\x01 \x01(\tR\x02id\x12\x12\n" + "\x04name\x18\x02 \x01(\tR\x04name\x12\x14\n" + @@ -9298,13 +9663,19 @@ const file_product_proto_rawDesc = "" + "\x04code\x18\x02 \x01(\tR\x04code\"P\n" + "\rCategoryInput\x12\x12\n" + "\x04name\x18\x01 \x01(\tR\x04name\x12+\n" + - "\x04kind\x18\x02 \x01(\x0e2\x17.productv1.CategoryKindR\x04kind*\x9a\x01\n" + + "\x04kind\x18\x02 \x01(\x0e2\x17.productv1.CategoryKindR\x04kind\"\x80\x02\n" + + "\x12ProductCountFilter\x129\n" + + "\tmin_price\x18\x01 \x01(\v2\x1c.google.protobuf.DoubleValueR\bminPrice\x129\n" + + "\tmax_price\x18\x02 \x01(\v2\x1c.google.protobuf.DoubleValueR\bmaxPrice\x125\n" + + "\bin_stock\x18\x03 \x01(\v2\x1a.google.protobuf.BoolValueR\ainStock\x12=\n" + + "\vsearch_term\x18\x04 \x01(\v2\x1c.google.protobuf.StringValueR\n" + + "searchTerm*\x9a\x01\n" + "\fCategoryKind\x12\x1d\n" + "\x19CATEGORY_KIND_UNSPECIFIED\x10\x00\x12\x16\n" + "\x12CATEGORY_KIND_BOOK\x10\x01\x12\x1d\n" + "\x19CATEGORY_KIND_ELECTRONICS\x10\x02\x12\x1b\n" + "\x17CATEGORY_KIND_FURNITURE\x10\x03\x12\x17\n" + - "\x13CATEGORY_KIND_OTHER\x10\x042\xb8&\n" + + "\x13CATEGORY_KIND_OTHER\x10\x042\xb8'\n" + "\x0eProductService\x12`\n" + "\x11LookupProductById\x12#.productv1.LookupProductByIdRequest\x1a$.productv1.LookupProductByIdResponse\"\x00\x12`\n" + "\x11LookupStorageById\x12#.productv1.LookupStorageByIdRequest\x1a$.productv1.LookupStorageByIdResponse\"\x00\x12f\n" + @@ -9351,7 +9722,8 @@ const file_product_proto_rawDesc = "" + "!QueryTypeWithMultipleFilterFields\x123.productv1.QueryTypeWithMultipleFilterFieldsRequest\x1a4.productv1.QueryTypeWithMultipleFilterFieldsResponse\"\x00\x12H\n" + "\tQueryUser\x12\x1b.productv1.QueryUserRequest\x1a\x1c.productv1.QueryUserResponse\"\x00\x12K\n" + "\n" + - "QueryUsers\x12\x1c.productv1.QueryUsersRequest\x1a\x1d.productv1.QueryUsersResponse\"\x00B%Z#cosmo/pkg/proto/productv1;productv1b\x06proto3" + "QueryUsers\x12\x1c.productv1.QueryUsersRequest\x1a\x1d.productv1.QueryUsersResponse\"\x00\x12~\n" + + "\x1bResolveCategoryProductCount\x12-.productv1.ResolveCategoryProductCountRequest\x1a..productv1.ResolveCategoryProductCountResponse\"\x00B%Z#cosmo/pkg/proto/productv1;productv1b\x06proto3" var ( file_product_proto_rawDescOnce sync.Once @@ -9366,7 +9738,7 @@ func file_product_proto_rawDescGZIP() []byte { } var file_product_proto_enumTypes = make([]protoimpl.EnumInfo, 1) -var file_product_proto_msgTypes = make([]protoimpl.MessageInfo, 171) +var file_product_proto_msgTypes = make([]protoimpl.MessageInfo, 178) var file_product_proto_goTypes = []any{ (CategoryKind)(0), // 0: productv1.CategoryKind (*ListOfAuthorFilter)(nil), // 1: productv1.ListOfAuthorFilter @@ -9481,354 +9853,372 @@ var file_product_proto_goTypes = []any{ (*MutationBulkCreateBlogPostsResponse)(nil), // 110: productv1.MutationBulkCreateBlogPostsResponse (*MutationBulkUpdateBlogPostsRequest)(nil), // 111: productv1.MutationBulkUpdateBlogPostsRequest (*MutationBulkUpdateBlogPostsResponse)(nil), // 112: productv1.MutationBulkUpdateBlogPostsResponse - (*Product)(nil), // 113: productv1.Product - (*Storage)(nil), // 114: productv1.Storage - (*Warehouse)(nil), // 115: productv1.Warehouse - (*User)(nil), // 116: productv1.User - (*NestedTypeA)(nil), // 117: productv1.NestedTypeA - (*RecursiveType)(nil), // 118: productv1.RecursiveType - (*TypeWithMultipleFilterFields)(nil), // 119: productv1.TypeWithMultipleFilterFields - (*FilterTypeInput)(nil), // 120: productv1.FilterTypeInput - (*ComplexFilterTypeInput)(nil), // 121: productv1.ComplexFilterTypeInput - (*TypeWithComplexFilterInput)(nil), // 122: productv1.TypeWithComplexFilterInput - (*OrderInput)(nil), // 123: productv1.OrderInput - (*Order)(nil), // 124: productv1.Order - (*Category)(nil), // 125: productv1.Category - (*CategoryFilter)(nil), // 126: productv1.CategoryFilter - (*Animal)(nil), // 127: productv1.Animal - (*SearchInput)(nil), // 128: productv1.SearchInput - (*SearchResult)(nil), // 129: productv1.SearchResult - (*NullableFieldsType)(nil), // 130: productv1.NullableFieldsType - (*NullableFieldsFilter)(nil), // 131: productv1.NullableFieldsFilter - (*BlogPost)(nil), // 132: productv1.BlogPost - (*BlogPostFilter)(nil), // 133: productv1.BlogPostFilter - (*Author)(nil), // 134: productv1.Author - (*AuthorFilter)(nil), // 135: productv1.AuthorFilter - (*UserInput)(nil), // 136: productv1.UserInput - (*ActionInput)(nil), // 137: productv1.ActionInput - (*ActionResult)(nil), // 138: productv1.ActionResult - (*NullableFieldsInput)(nil), // 139: productv1.NullableFieldsInput - (*BlogPostInput)(nil), // 140: productv1.BlogPostInput - (*AuthorInput)(nil), // 141: productv1.AuthorInput - (*NestedTypeB)(nil), // 142: productv1.NestedTypeB - (*NestedTypeC)(nil), // 143: productv1.NestedTypeC - (*FilterType)(nil), // 144: productv1.FilterType - (*Pagination)(nil), // 145: productv1.Pagination - (*OrderLineInput)(nil), // 146: productv1.OrderLineInput - (*OrderLine)(nil), // 147: productv1.OrderLine - (*Cat)(nil), // 148: productv1.Cat - (*Dog)(nil), // 149: productv1.Dog - (*ActionSuccess)(nil), // 150: productv1.ActionSuccess - (*ActionError)(nil), // 151: productv1.ActionError - (*CategoryInput)(nil), // 152: productv1.CategoryInput - (*ListOfAuthorFilter_List)(nil), // 153: productv1.ListOfAuthorFilter.List - (*ListOfAuthorInput_List)(nil), // 154: productv1.ListOfAuthorInput.List - (*ListOfBlogPost_List)(nil), // 155: productv1.ListOfBlogPost.List - (*ListOfBlogPostFilter_List)(nil), // 156: productv1.ListOfBlogPostFilter.List - (*ListOfBlogPostInput_List)(nil), // 157: productv1.ListOfBlogPostInput.List - (*ListOfBoolean_List)(nil), // 158: productv1.ListOfBoolean.List - (*ListOfCategory_List)(nil), // 159: productv1.ListOfCategory.List - (*ListOfCategoryInput_List)(nil), // 160: productv1.ListOfCategoryInput.List - (*ListOfFloat_List)(nil), // 161: productv1.ListOfFloat.List - (*ListOfListOfCategory_List)(nil), // 162: productv1.ListOfListOfCategory.List - (*ListOfListOfCategoryInput_List)(nil), // 163: productv1.ListOfListOfCategoryInput.List - (*ListOfListOfString_List)(nil), // 164: productv1.ListOfListOfString.List - (*ListOfListOfUser_List)(nil), // 165: productv1.ListOfListOfUser.List - (*ListOfListOfUserInput_List)(nil), // 166: productv1.ListOfListOfUserInput.List - (*ListOfOrderLine_List)(nil), // 167: productv1.ListOfOrderLine.List - (*ListOfProduct_List)(nil), // 168: productv1.ListOfProduct.List - (*ListOfString_List)(nil), // 169: productv1.ListOfString.List - (*ListOfUser_List)(nil), // 170: productv1.ListOfUser.List - (*ListOfUserInput_List)(nil), // 171: productv1.ListOfUserInput.List - (*wrapperspb.Int32Value)(nil), // 172: google.protobuf.Int32Value - (*wrapperspb.StringValue)(nil), // 173: google.protobuf.StringValue - (*wrapperspb.DoubleValue)(nil), // 174: google.protobuf.DoubleValue - (*wrapperspb.BoolValue)(nil), // 175: google.protobuf.BoolValue + (*CategoryProductCountArgs)(nil), // 113: productv1.CategoryProductCountArgs + (*CategoryProductCountContext)(nil), // 114: productv1.CategoryProductCountContext + (*ResolveCategoryProductCountRequestKey)(nil), // 115: productv1.ResolveCategoryProductCountRequestKey + (*ResolveCategoryProductCountRequest)(nil), // 116: productv1.ResolveCategoryProductCountRequest + (*ResolveCategoryProductCountResponseResult)(nil), // 117: productv1.ResolveCategoryProductCountResponseResult + (*ResolveCategoryProductCountResponse)(nil), // 118: productv1.ResolveCategoryProductCountResponse + (*Product)(nil), // 119: productv1.Product + (*Storage)(nil), // 120: productv1.Storage + (*Warehouse)(nil), // 121: productv1.Warehouse + (*User)(nil), // 122: productv1.User + (*NestedTypeA)(nil), // 123: productv1.NestedTypeA + (*RecursiveType)(nil), // 124: productv1.RecursiveType + (*TypeWithMultipleFilterFields)(nil), // 125: productv1.TypeWithMultipleFilterFields + (*FilterTypeInput)(nil), // 126: productv1.FilterTypeInput + (*ComplexFilterTypeInput)(nil), // 127: productv1.ComplexFilterTypeInput + (*TypeWithComplexFilterInput)(nil), // 128: productv1.TypeWithComplexFilterInput + (*OrderInput)(nil), // 129: productv1.OrderInput + (*Order)(nil), // 130: productv1.Order + (*Category)(nil), // 131: productv1.Category + (*CategoryFilter)(nil), // 132: productv1.CategoryFilter + (*Animal)(nil), // 133: productv1.Animal + (*SearchInput)(nil), // 134: productv1.SearchInput + (*SearchResult)(nil), // 135: productv1.SearchResult + (*NullableFieldsType)(nil), // 136: productv1.NullableFieldsType + (*NullableFieldsFilter)(nil), // 137: productv1.NullableFieldsFilter + (*BlogPost)(nil), // 138: productv1.BlogPost + (*BlogPostFilter)(nil), // 139: productv1.BlogPostFilter + (*Author)(nil), // 140: productv1.Author + (*AuthorFilter)(nil), // 141: productv1.AuthorFilter + (*UserInput)(nil), // 142: productv1.UserInput + (*ActionInput)(nil), // 143: productv1.ActionInput + (*ActionResult)(nil), // 144: productv1.ActionResult + (*NullableFieldsInput)(nil), // 145: productv1.NullableFieldsInput + (*BlogPostInput)(nil), // 146: productv1.BlogPostInput + (*AuthorInput)(nil), // 147: productv1.AuthorInput + (*NestedTypeB)(nil), // 148: productv1.NestedTypeB + (*NestedTypeC)(nil), // 149: productv1.NestedTypeC + (*FilterType)(nil), // 150: productv1.FilterType + (*Pagination)(nil), // 151: productv1.Pagination + (*OrderLineInput)(nil), // 152: productv1.OrderLineInput + (*OrderLine)(nil), // 153: productv1.OrderLine + (*Cat)(nil), // 154: productv1.Cat + (*Dog)(nil), // 155: productv1.Dog + (*ActionSuccess)(nil), // 156: productv1.ActionSuccess + (*ActionError)(nil), // 157: productv1.ActionError + (*CategoryInput)(nil), // 158: productv1.CategoryInput + (*ProductCountFilter)(nil), // 159: productv1.ProductCountFilter + (*ListOfAuthorFilter_List)(nil), // 160: productv1.ListOfAuthorFilter.List + (*ListOfAuthorInput_List)(nil), // 161: productv1.ListOfAuthorInput.List + (*ListOfBlogPost_List)(nil), // 162: productv1.ListOfBlogPost.List + (*ListOfBlogPostFilter_List)(nil), // 163: productv1.ListOfBlogPostFilter.List + (*ListOfBlogPostInput_List)(nil), // 164: productv1.ListOfBlogPostInput.List + (*ListOfBoolean_List)(nil), // 165: productv1.ListOfBoolean.List + (*ListOfCategory_List)(nil), // 166: productv1.ListOfCategory.List + (*ListOfCategoryInput_List)(nil), // 167: productv1.ListOfCategoryInput.List + (*ListOfFloat_List)(nil), // 168: productv1.ListOfFloat.List + (*ListOfListOfCategory_List)(nil), // 169: productv1.ListOfListOfCategory.List + (*ListOfListOfCategoryInput_List)(nil), // 170: productv1.ListOfListOfCategoryInput.List + (*ListOfListOfString_List)(nil), // 171: productv1.ListOfListOfString.List + (*ListOfListOfUser_List)(nil), // 172: productv1.ListOfListOfUser.List + (*ListOfListOfUserInput_List)(nil), // 173: productv1.ListOfListOfUserInput.List + (*ListOfOrderLine_List)(nil), // 174: productv1.ListOfOrderLine.List + (*ListOfProduct_List)(nil), // 175: productv1.ListOfProduct.List + (*ListOfString_List)(nil), // 176: productv1.ListOfString.List + (*ListOfUser_List)(nil), // 177: productv1.ListOfUser.List + (*ListOfUserInput_List)(nil), // 178: productv1.ListOfUserInput.List + (*wrapperspb.Int32Value)(nil), // 179: google.protobuf.Int32Value + (*wrapperspb.StringValue)(nil), // 180: google.protobuf.StringValue + (*wrapperspb.DoubleValue)(nil), // 181: google.protobuf.DoubleValue + (*wrapperspb.BoolValue)(nil), // 182: google.protobuf.BoolValue } var file_product_proto_depIdxs = []int32{ - 153, // 0: productv1.ListOfAuthorFilter.list:type_name -> productv1.ListOfAuthorFilter.List - 154, // 1: productv1.ListOfAuthorInput.list:type_name -> productv1.ListOfAuthorInput.List - 155, // 2: productv1.ListOfBlogPost.list:type_name -> productv1.ListOfBlogPost.List - 156, // 3: productv1.ListOfBlogPostFilter.list:type_name -> productv1.ListOfBlogPostFilter.List - 157, // 4: productv1.ListOfBlogPostInput.list:type_name -> productv1.ListOfBlogPostInput.List - 158, // 5: productv1.ListOfBoolean.list:type_name -> productv1.ListOfBoolean.List - 159, // 6: productv1.ListOfCategory.list:type_name -> productv1.ListOfCategory.List - 160, // 7: productv1.ListOfCategoryInput.list:type_name -> productv1.ListOfCategoryInput.List - 161, // 8: productv1.ListOfFloat.list:type_name -> productv1.ListOfFloat.List - 162, // 9: productv1.ListOfListOfCategory.list:type_name -> productv1.ListOfListOfCategory.List - 163, // 10: productv1.ListOfListOfCategoryInput.list:type_name -> productv1.ListOfListOfCategoryInput.List - 164, // 11: productv1.ListOfListOfString.list:type_name -> productv1.ListOfListOfString.List - 165, // 12: productv1.ListOfListOfUser.list:type_name -> productv1.ListOfListOfUser.List - 166, // 13: productv1.ListOfListOfUserInput.list:type_name -> productv1.ListOfListOfUserInput.List - 167, // 14: productv1.ListOfOrderLine.list:type_name -> productv1.ListOfOrderLine.List - 168, // 15: productv1.ListOfProduct.list:type_name -> productv1.ListOfProduct.List - 169, // 16: productv1.ListOfString.list:type_name -> productv1.ListOfString.List - 170, // 17: productv1.ListOfUser.list:type_name -> productv1.ListOfUser.List - 171, // 18: productv1.ListOfUserInput.list:type_name -> productv1.ListOfUserInput.List + 160, // 0: productv1.ListOfAuthorFilter.list:type_name -> productv1.ListOfAuthorFilter.List + 161, // 1: productv1.ListOfAuthorInput.list:type_name -> productv1.ListOfAuthorInput.List + 162, // 2: productv1.ListOfBlogPost.list:type_name -> productv1.ListOfBlogPost.List + 163, // 3: productv1.ListOfBlogPostFilter.list:type_name -> productv1.ListOfBlogPostFilter.List + 164, // 4: productv1.ListOfBlogPostInput.list:type_name -> productv1.ListOfBlogPostInput.List + 165, // 5: productv1.ListOfBoolean.list:type_name -> productv1.ListOfBoolean.List + 166, // 6: productv1.ListOfCategory.list:type_name -> productv1.ListOfCategory.List + 167, // 7: productv1.ListOfCategoryInput.list:type_name -> productv1.ListOfCategoryInput.List + 168, // 8: productv1.ListOfFloat.list:type_name -> productv1.ListOfFloat.List + 169, // 9: productv1.ListOfListOfCategory.list:type_name -> productv1.ListOfListOfCategory.List + 170, // 10: productv1.ListOfListOfCategoryInput.list:type_name -> productv1.ListOfListOfCategoryInput.List + 171, // 11: productv1.ListOfListOfString.list:type_name -> productv1.ListOfListOfString.List + 172, // 12: productv1.ListOfListOfUser.list:type_name -> productv1.ListOfListOfUser.List + 173, // 13: productv1.ListOfListOfUserInput.list:type_name -> productv1.ListOfListOfUserInput.List + 174, // 14: productv1.ListOfOrderLine.list:type_name -> productv1.ListOfOrderLine.List + 175, // 15: productv1.ListOfProduct.list:type_name -> productv1.ListOfProduct.List + 176, // 16: productv1.ListOfString.list:type_name -> productv1.ListOfString.List + 177, // 17: productv1.ListOfUser.list:type_name -> productv1.ListOfUser.List + 178, // 18: productv1.ListOfUserInput.list:type_name -> productv1.ListOfUserInput.List 20, // 19: productv1.LookupProductByIdRequest.keys:type_name -> productv1.LookupProductByIdRequestKey - 113, // 20: productv1.LookupProductByIdResponse.result:type_name -> productv1.Product + 119, // 20: productv1.LookupProductByIdResponse.result:type_name -> productv1.Product 23, // 21: productv1.LookupStorageByIdRequest.keys:type_name -> productv1.LookupStorageByIdRequestKey - 114, // 22: productv1.LookupStorageByIdResponse.result:type_name -> productv1.Storage + 120, // 22: productv1.LookupStorageByIdResponse.result:type_name -> productv1.Storage 26, // 23: productv1.LookupWarehouseByIdRequest.keys:type_name -> productv1.LookupWarehouseByIdRequestKey - 115, // 24: productv1.LookupWarehouseByIdResponse.result:type_name -> productv1.Warehouse - 116, // 25: productv1.QueryUsersResponse.users:type_name -> productv1.User - 116, // 26: productv1.QueryUserResponse.user:type_name -> productv1.User - 117, // 27: productv1.QueryNestedTypeResponse.nested_type:type_name -> productv1.NestedTypeA - 118, // 28: productv1.QueryRecursiveTypeResponse.recursive_type:type_name -> productv1.RecursiveType - 119, // 29: productv1.QueryTypeFilterWithArgumentsResponse.type_filter_with_arguments:type_name -> productv1.TypeWithMultipleFilterFields - 120, // 30: productv1.QueryTypeWithMultipleFilterFieldsRequest.filter:type_name -> productv1.FilterTypeInput - 119, // 31: productv1.QueryTypeWithMultipleFilterFieldsResponse.type_with_multiple_filter_fields:type_name -> productv1.TypeWithMultipleFilterFields - 121, // 32: productv1.QueryComplexFilterTypeRequest.filter:type_name -> productv1.ComplexFilterTypeInput - 122, // 33: productv1.QueryComplexFilterTypeResponse.complex_filter_type:type_name -> productv1.TypeWithComplexFilterInput - 123, // 34: productv1.QueryCalculateTotalsRequest.orders:type_name -> productv1.OrderInput - 124, // 35: productv1.QueryCalculateTotalsResponse.calculate_totals:type_name -> productv1.Order - 125, // 36: productv1.QueryCategoriesResponse.categories:type_name -> productv1.Category + 121, // 24: productv1.LookupWarehouseByIdResponse.result:type_name -> productv1.Warehouse + 122, // 25: productv1.QueryUsersResponse.users:type_name -> productv1.User + 122, // 26: productv1.QueryUserResponse.user:type_name -> productv1.User + 123, // 27: productv1.QueryNestedTypeResponse.nested_type:type_name -> productv1.NestedTypeA + 124, // 28: productv1.QueryRecursiveTypeResponse.recursive_type:type_name -> productv1.RecursiveType + 125, // 29: productv1.QueryTypeFilterWithArgumentsResponse.type_filter_with_arguments:type_name -> productv1.TypeWithMultipleFilterFields + 126, // 30: productv1.QueryTypeWithMultipleFilterFieldsRequest.filter:type_name -> productv1.FilterTypeInput + 125, // 31: productv1.QueryTypeWithMultipleFilterFieldsResponse.type_with_multiple_filter_fields:type_name -> productv1.TypeWithMultipleFilterFields + 127, // 32: productv1.QueryComplexFilterTypeRequest.filter:type_name -> productv1.ComplexFilterTypeInput + 128, // 33: productv1.QueryComplexFilterTypeResponse.complex_filter_type:type_name -> productv1.TypeWithComplexFilterInput + 129, // 34: productv1.QueryCalculateTotalsRequest.orders:type_name -> productv1.OrderInput + 130, // 35: productv1.QueryCalculateTotalsResponse.calculate_totals:type_name -> productv1.Order + 131, // 36: productv1.QueryCategoriesResponse.categories:type_name -> productv1.Category 0, // 37: productv1.QueryCategoriesByKindRequest.kind:type_name -> productv1.CategoryKind - 125, // 38: productv1.QueryCategoriesByKindResponse.categories_by_kind:type_name -> productv1.Category + 131, // 38: productv1.QueryCategoriesByKindResponse.categories_by_kind:type_name -> productv1.Category 0, // 39: productv1.QueryCategoriesByKindsRequest.kinds:type_name -> productv1.CategoryKind - 125, // 40: productv1.QueryCategoriesByKindsResponse.categories_by_kinds:type_name -> productv1.Category - 126, // 41: productv1.QueryFilterCategoriesRequest.filter:type_name -> productv1.CategoryFilter - 125, // 42: productv1.QueryFilterCategoriesResponse.filter_categories:type_name -> productv1.Category - 127, // 43: productv1.QueryRandomPetResponse.random_pet:type_name -> productv1.Animal - 127, // 44: productv1.QueryAllPetsResponse.all_pets:type_name -> productv1.Animal - 128, // 45: productv1.QuerySearchRequest.input:type_name -> productv1.SearchInput - 129, // 46: productv1.QuerySearchResponse.search:type_name -> productv1.SearchResult - 129, // 47: productv1.QueryRandomSearchResultResponse.random_search_result:type_name -> productv1.SearchResult - 130, // 48: productv1.QueryNullableFieldsTypeResponse.nullable_fields_type:type_name -> productv1.NullableFieldsType - 130, // 49: productv1.QueryNullableFieldsTypeByIdResponse.nullable_fields_type_by_id:type_name -> productv1.NullableFieldsType - 131, // 50: productv1.QueryNullableFieldsTypeWithFilterRequest.filter:type_name -> productv1.NullableFieldsFilter - 130, // 51: productv1.QueryNullableFieldsTypeWithFilterResponse.nullable_fields_type_with_filter:type_name -> productv1.NullableFieldsType - 130, // 52: productv1.QueryAllNullableFieldsTypesResponse.all_nullable_fields_types:type_name -> productv1.NullableFieldsType - 132, // 53: productv1.QueryBlogPostResponse.blog_post:type_name -> productv1.BlogPost - 132, // 54: productv1.QueryBlogPostByIdResponse.blog_post_by_id:type_name -> productv1.BlogPost - 133, // 55: productv1.QueryBlogPostsWithFilterRequest.filter:type_name -> productv1.BlogPostFilter - 132, // 56: productv1.QueryBlogPostsWithFilterResponse.blog_posts_with_filter:type_name -> productv1.BlogPost - 132, // 57: productv1.QueryAllBlogPostsResponse.all_blog_posts:type_name -> productv1.BlogPost - 134, // 58: productv1.QueryAuthorResponse.author:type_name -> productv1.Author - 134, // 59: productv1.QueryAuthorByIdResponse.author_by_id:type_name -> productv1.Author - 135, // 60: productv1.QueryAuthorsWithFilterRequest.filter:type_name -> productv1.AuthorFilter - 134, // 61: productv1.QueryAuthorsWithFilterResponse.authors_with_filter:type_name -> productv1.Author - 134, // 62: productv1.QueryAllAuthorsResponse.all_authors:type_name -> productv1.Author + 131, // 40: productv1.QueryCategoriesByKindsResponse.categories_by_kinds:type_name -> productv1.Category + 132, // 41: productv1.QueryFilterCategoriesRequest.filter:type_name -> productv1.CategoryFilter + 131, // 42: productv1.QueryFilterCategoriesResponse.filter_categories:type_name -> productv1.Category + 133, // 43: productv1.QueryRandomPetResponse.random_pet:type_name -> productv1.Animal + 133, // 44: productv1.QueryAllPetsResponse.all_pets:type_name -> productv1.Animal + 134, // 45: productv1.QuerySearchRequest.input:type_name -> productv1.SearchInput + 135, // 46: productv1.QuerySearchResponse.search:type_name -> productv1.SearchResult + 135, // 47: productv1.QueryRandomSearchResultResponse.random_search_result:type_name -> productv1.SearchResult + 136, // 48: productv1.QueryNullableFieldsTypeResponse.nullable_fields_type:type_name -> productv1.NullableFieldsType + 136, // 49: productv1.QueryNullableFieldsTypeByIdResponse.nullable_fields_type_by_id:type_name -> productv1.NullableFieldsType + 137, // 50: productv1.QueryNullableFieldsTypeWithFilterRequest.filter:type_name -> productv1.NullableFieldsFilter + 136, // 51: productv1.QueryNullableFieldsTypeWithFilterResponse.nullable_fields_type_with_filter:type_name -> productv1.NullableFieldsType + 136, // 52: productv1.QueryAllNullableFieldsTypesResponse.all_nullable_fields_types:type_name -> productv1.NullableFieldsType + 138, // 53: productv1.QueryBlogPostResponse.blog_post:type_name -> productv1.BlogPost + 138, // 54: productv1.QueryBlogPostByIdResponse.blog_post_by_id:type_name -> productv1.BlogPost + 139, // 55: productv1.QueryBlogPostsWithFilterRequest.filter:type_name -> productv1.BlogPostFilter + 138, // 56: productv1.QueryBlogPostsWithFilterResponse.blog_posts_with_filter:type_name -> productv1.BlogPost + 138, // 57: productv1.QueryAllBlogPostsResponse.all_blog_posts:type_name -> productv1.BlogPost + 140, // 58: productv1.QueryAuthorResponse.author:type_name -> productv1.Author + 140, // 59: productv1.QueryAuthorByIdResponse.author_by_id:type_name -> productv1.Author + 141, // 60: productv1.QueryAuthorsWithFilterRequest.filter:type_name -> productv1.AuthorFilter + 140, // 61: productv1.QueryAuthorsWithFilterResponse.authors_with_filter:type_name -> productv1.Author + 140, // 62: productv1.QueryAllAuthorsResponse.all_authors:type_name -> productv1.Author 1, // 63: productv1.QueryBulkSearchAuthorsRequest.filters:type_name -> productv1.ListOfAuthorFilter - 134, // 64: productv1.QueryBulkSearchAuthorsResponse.bulk_search_authors:type_name -> productv1.Author + 140, // 64: productv1.QueryBulkSearchAuthorsResponse.bulk_search_authors:type_name -> productv1.Author 4, // 65: productv1.QueryBulkSearchBlogPostsRequest.filters:type_name -> productv1.ListOfBlogPostFilter - 132, // 66: productv1.QueryBulkSearchBlogPostsResponse.bulk_search_blog_posts:type_name -> productv1.BlogPost - 136, // 67: productv1.MutationCreateUserRequest.input:type_name -> productv1.UserInput - 116, // 68: productv1.MutationCreateUserResponse.create_user:type_name -> productv1.User - 137, // 69: productv1.MutationPerformActionRequest.input:type_name -> productv1.ActionInput - 138, // 70: productv1.MutationPerformActionResponse.perform_action:type_name -> productv1.ActionResult - 139, // 71: productv1.MutationCreateNullableFieldsTypeRequest.input:type_name -> productv1.NullableFieldsInput - 130, // 72: productv1.MutationCreateNullableFieldsTypeResponse.create_nullable_fields_type:type_name -> productv1.NullableFieldsType - 139, // 73: productv1.MutationUpdateNullableFieldsTypeRequest.input:type_name -> productv1.NullableFieldsInput - 130, // 74: productv1.MutationUpdateNullableFieldsTypeResponse.update_nullable_fields_type:type_name -> productv1.NullableFieldsType - 140, // 75: productv1.MutationCreateBlogPostRequest.input:type_name -> productv1.BlogPostInput - 132, // 76: productv1.MutationCreateBlogPostResponse.create_blog_post:type_name -> productv1.BlogPost - 140, // 77: productv1.MutationUpdateBlogPostRequest.input:type_name -> productv1.BlogPostInput - 132, // 78: productv1.MutationUpdateBlogPostResponse.update_blog_post:type_name -> productv1.BlogPost - 141, // 79: productv1.MutationCreateAuthorRequest.input:type_name -> productv1.AuthorInput - 134, // 80: productv1.MutationCreateAuthorResponse.create_author:type_name -> productv1.Author - 141, // 81: productv1.MutationUpdateAuthorRequest.input:type_name -> productv1.AuthorInput - 134, // 82: productv1.MutationUpdateAuthorResponse.update_author:type_name -> productv1.Author + 138, // 66: productv1.QueryBulkSearchBlogPostsResponse.bulk_search_blog_posts:type_name -> productv1.BlogPost + 142, // 67: productv1.MutationCreateUserRequest.input:type_name -> productv1.UserInput + 122, // 68: productv1.MutationCreateUserResponse.create_user:type_name -> productv1.User + 143, // 69: productv1.MutationPerformActionRequest.input:type_name -> productv1.ActionInput + 144, // 70: productv1.MutationPerformActionResponse.perform_action:type_name -> productv1.ActionResult + 145, // 71: productv1.MutationCreateNullableFieldsTypeRequest.input:type_name -> productv1.NullableFieldsInput + 136, // 72: productv1.MutationCreateNullableFieldsTypeResponse.create_nullable_fields_type:type_name -> productv1.NullableFieldsType + 145, // 73: productv1.MutationUpdateNullableFieldsTypeRequest.input:type_name -> productv1.NullableFieldsInput + 136, // 74: productv1.MutationUpdateNullableFieldsTypeResponse.update_nullable_fields_type:type_name -> productv1.NullableFieldsType + 146, // 75: productv1.MutationCreateBlogPostRequest.input:type_name -> productv1.BlogPostInput + 138, // 76: productv1.MutationCreateBlogPostResponse.create_blog_post:type_name -> productv1.BlogPost + 146, // 77: productv1.MutationUpdateBlogPostRequest.input:type_name -> productv1.BlogPostInput + 138, // 78: productv1.MutationUpdateBlogPostResponse.update_blog_post:type_name -> productv1.BlogPost + 147, // 79: productv1.MutationCreateAuthorRequest.input:type_name -> productv1.AuthorInput + 140, // 80: productv1.MutationCreateAuthorResponse.create_author:type_name -> productv1.Author + 147, // 81: productv1.MutationUpdateAuthorRequest.input:type_name -> productv1.AuthorInput + 140, // 82: productv1.MutationUpdateAuthorResponse.update_author:type_name -> productv1.Author 2, // 83: productv1.MutationBulkCreateAuthorsRequest.authors:type_name -> productv1.ListOfAuthorInput - 134, // 84: productv1.MutationBulkCreateAuthorsResponse.bulk_create_authors:type_name -> productv1.Author + 140, // 84: productv1.MutationBulkCreateAuthorsResponse.bulk_create_authors:type_name -> productv1.Author 2, // 85: productv1.MutationBulkUpdateAuthorsRequest.authors:type_name -> productv1.ListOfAuthorInput - 134, // 86: productv1.MutationBulkUpdateAuthorsResponse.bulk_update_authors:type_name -> productv1.Author + 140, // 86: productv1.MutationBulkUpdateAuthorsResponse.bulk_update_authors:type_name -> productv1.Author 5, // 87: productv1.MutationBulkCreateBlogPostsRequest.blog_posts:type_name -> productv1.ListOfBlogPostInput - 132, // 88: productv1.MutationBulkCreateBlogPostsResponse.bulk_create_blog_posts:type_name -> productv1.BlogPost + 138, // 88: productv1.MutationBulkCreateBlogPostsResponse.bulk_create_blog_posts:type_name -> productv1.BlogPost 5, // 89: productv1.MutationBulkUpdateBlogPostsRequest.blog_posts:type_name -> productv1.ListOfBlogPostInput - 132, // 90: productv1.MutationBulkUpdateBlogPostsResponse.bulk_update_blog_posts:type_name -> productv1.BlogPost - 142, // 91: productv1.NestedTypeA.b:type_name -> productv1.NestedTypeB - 118, // 92: productv1.RecursiveType.recursive_type:type_name -> productv1.RecursiveType - 144, // 93: productv1.ComplexFilterTypeInput.filter:type_name -> productv1.FilterType - 146, // 94: productv1.OrderInput.lines:type_name -> productv1.OrderLineInput - 15, // 95: productv1.Order.order_lines:type_name -> productv1.ListOfOrderLine - 0, // 96: productv1.Category.kind:type_name -> productv1.CategoryKind - 0, // 97: productv1.CategoryFilter.category:type_name -> productv1.CategoryKind - 145, // 98: productv1.CategoryFilter.pagination:type_name -> productv1.Pagination - 148, // 99: productv1.Animal.cat:type_name -> productv1.Cat - 149, // 100: productv1.Animal.dog:type_name -> productv1.Dog - 172, // 101: productv1.SearchInput.limit:type_name -> google.protobuf.Int32Value - 113, // 102: productv1.SearchResult.product:type_name -> productv1.Product - 116, // 103: productv1.SearchResult.user:type_name -> productv1.User - 125, // 104: productv1.SearchResult.category:type_name -> productv1.Category - 173, // 105: productv1.NullableFieldsType.optional_string:type_name -> google.protobuf.StringValue - 172, // 106: productv1.NullableFieldsType.optional_int:type_name -> google.protobuf.Int32Value - 174, // 107: productv1.NullableFieldsType.optional_float:type_name -> google.protobuf.DoubleValue - 175, // 108: productv1.NullableFieldsType.optional_boolean:type_name -> google.protobuf.BoolValue - 173, // 109: productv1.NullableFieldsFilter.name:type_name -> google.protobuf.StringValue - 173, // 110: productv1.NullableFieldsFilter.optional_string:type_name -> google.protobuf.StringValue - 175, // 111: productv1.NullableFieldsFilter.include_nulls:type_name -> google.protobuf.BoolValue - 17, // 112: productv1.BlogPost.optional_tags:type_name -> productv1.ListOfString - 17, // 113: productv1.BlogPost.keywords:type_name -> productv1.ListOfString - 9, // 114: productv1.BlogPost.ratings:type_name -> productv1.ListOfFloat - 6, // 115: productv1.BlogPost.is_published:type_name -> productv1.ListOfBoolean - 12, // 116: productv1.BlogPost.tag_groups:type_name -> productv1.ListOfListOfString - 12, // 117: productv1.BlogPost.related_topics:type_name -> productv1.ListOfListOfString - 12, // 118: productv1.BlogPost.comment_threads:type_name -> productv1.ListOfListOfString - 12, // 119: productv1.BlogPost.suggestions:type_name -> productv1.ListOfListOfString - 125, // 120: productv1.BlogPost.related_categories:type_name -> productv1.Category - 116, // 121: productv1.BlogPost.contributors:type_name -> productv1.User - 16, // 122: productv1.BlogPost.mentioned_products:type_name -> productv1.ListOfProduct - 18, // 123: productv1.BlogPost.mentioned_users:type_name -> productv1.ListOfUser - 10, // 124: productv1.BlogPost.category_groups:type_name -> productv1.ListOfListOfCategory - 13, // 125: productv1.BlogPost.contributor_teams:type_name -> productv1.ListOfListOfUser - 173, // 126: productv1.BlogPostFilter.title:type_name -> google.protobuf.StringValue - 175, // 127: productv1.BlogPostFilter.has_categories:type_name -> google.protobuf.BoolValue - 172, // 128: productv1.BlogPostFilter.min_tags:type_name -> google.protobuf.Int32Value - 173, // 129: productv1.Author.email:type_name -> google.protobuf.StringValue - 17, // 130: productv1.Author.social_links:type_name -> productv1.ListOfString - 12, // 131: productv1.Author.teams_by_project:type_name -> productv1.ListOfListOfString - 12, // 132: productv1.Author.collaborations:type_name -> productv1.ListOfListOfString - 3, // 133: productv1.Author.written_posts:type_name -> productv1.ListOfBlogPost - 125, // 134: productv1.Author.favorite_categories:type_name -> productv1.Category - 18, // 135: productv1.Author.related_authors:type_name -> productv1.ListOfUser - 16, // 136: productv1.Author.product_reviews:type_name -> productv1.ListOfProduct - 13, // 137: productv1.Author.author_groups:type_name -> productv1.ListOfListOfUser - 10, // 138: productv1.Author.category_preferences:type_name -> productv1.ListOfListOfCategory - 13, // 139: productv1.Author.project_teams:type_name -> productv1.ListOfListOfUser - 173, // 140: productv1.AuthorFilter.name:type_name -> google.protobuf.StringValue - 175, // 141: productv1.AuthorFilter.has_teams:type_name -> google.protobuf.BoolValue - 172, // 142: productv1.AuthorFilter.skill_count:type_name -> google.protobuf.Int32Value - 150, // 143: productv1.ActionResult.action_success:type_name -> productv1.ActionSuccess - 151, // 144: productv1.ActionResult.action_error:type_name -> productv1.ActionError - 173, // 145: productv1.NullableFieldsInput.optional_string:type_name -> google.protobuf.StringValue - 172, // 146: productv1.NullableFieldsInput.optional_int:type_name -> google.protobuf.Int32Value - 174, // 147: productv1.NullableFieldsInput.optional_float:type_name -> google.protobuf.DoubleValue - 175, // 148: productv1.NullableFieldsInput.optional_boolean:type_name -> google.protobuf.BoolValue - 17, // 149: productv1.BlogPostInput.optional_tags:type_name -> productv1.ListOfString - 17, // 150: productv1.BlogPostInput.keywords:type_name -> productv1.ListOfString - 9, // 151: productv1.BlogPostInput.ratings:type_name -> productv1.ListOfFloat - 6, // 152: productv1.BlogPostInput.is_published:type_name -> productv1.ListOfBoolean - 12, // 153: productv1.BlogPostInput.tag_groups:type_name -> productv1.ListOfListOfString - 12, // 154: productv1.BlogPostInput.related_topics:type_name -> productv1.ListOfListOfString - 12, // 155: productv1.BlogPostInput.comment_threads:type_name -> productv1.ListOfListOfString - 12, // 156: productv1.BlogPostInput.suggestions:type_name -> productv1.ListOfListOfString - 8, // 157: productv1.BlogPostInput.related_categories:type_name -> productv1.ListOfCategoryInput - 19, // 158: productv1.BlogPostInput.contributors:type_name -> productv1.ListOfUserInput - 11, // 159: productv1.BlogPostInput.category_groups:type_name -> productv1.ListOfListOfCategoryInput - 173, // 160: productv1.AuthorInput.email:type_name -> google.protobuf.StringValue - 17, // 161: productv1.AuthorInput.social_links:type_name -> productv1.ListOfString - 12, // 162: productv1.AuthorInput.teams_by_project:type_name -> productv1.ListOfListOfString - 12, // 163: productv1.AuthorInput.collaborations:type_name -> productv1.ListOfListOfString - 152, // 164: productv1.AuthorInput.favorite_categories:type_name -> productv1.CategoryInput - 14, // 165: productv1.AuthorInput.author_groups:type_name -> productv1.ListOfListOfUserInput - 14, // 166: productv1.AuthorInput.project_teams:type_name -> productv1.ListOfListOfUserInput - 143, // 167: productv1.NestedTypeB.c:type_name -> productv1.NestedTypeC - 145, // 168: productv1.FilterType.pagination:type_name -> productv1.Pagination - 17, // 169: productv1.OrderLineInput.modifiers:type_name -> productv1.ListOfString - 17, // 170: productv1.OrderLine.modifiers:type_name -> productv1.ListOfString - 0, // 171: productv1.CategoryInput.kind:type_name -> productv1.CategoryKind - 135, // 172: productv1.ListOfAuthorFilter.List.items:type_name -> productv1.AuthorFilter - 141, // 173: productv1.ListOfAuthorInput.List.items:type_name -> productv1.AuthorInput - 132, // 174: productv1.ListOfBlogPost.List.items:type_name -> productv1.BlogPost - 133, // 175: productv1.ListOfBlogPostFilter.List.items:type_name -> productv1.BlogPostFilter - 140, // 176: productv1.ListOfBlogPostInput.List.items:type_name -> productv1.BlogPostInput - 125, // 177: productv1.ListOfCategory.List.items:type_name -> productv1.Category - 152, // 178: productv1.ListOfCategoryInput.List.items:type_name -> productv1.CategoryInput - 7, // 179: productv1.ListOfListOfCategory.List.items:type_name -> productv1.ListOfCategory - 8, // 180: productv1.ListOfListOfCategoryInput.List.items:type_name -> productv1.ListOfCategoryInput - 17, // 181: productv1.ListOfListOfString.List.items:type_name -> productv1.ListOfString - 18, // 182: productv1.ListOfListOfUser.List.items:type_name -> productv1.ListOfUser - 19, // 183: productv1.ListOfListOfUserInput.List.items:type_name -> productv1.ListOfUserInput - 147, // 184: productv1.ListOfOrderLine.List.items:type_name -> productv1.OrderLine - 113, // 185: productv1.ListOfProduct.List.items:type_name -> productv1.Product - 116, // 186: productv1.ListOfUser.List.items:type_name -> productv1.User - 136, // 187: productv1.ListOfUserInput.List.items:type_name -> productv1.UserInput - 21, // 188: productv1.ProductService.LookupProductById:input_type -> productv1.LookupProductByIdRequest - 24, // 189: productv1.ProductService.LookupStorageById:input_type -> productv1.LookupStorageByIdRequest - 27, // 190: productv1.ProductService.LookupWarehouseById:input_type -> productv1.LookupWarehouseByIdRequest - 105, // 191: productv1.ProductService.MutationBulkCreateAuthors:input_type -> productv1.MutationBulkCreateAuthorsRequest - 109, // 192: productv1.ProductService.MutationBulkCreateBlogPosts:input_type -> productv1.MutationBulkCreateBlogPostsRequest - 107, // 193: productv1.ProductService.MutationBulkUpdateAuthors:input_type -> productv1.MutationBulkUpdateAuthorsRequest - 111, // 194: productv1.ProductService.MutationBulkUpdateBlogPosts:input_type -> productv1.MutationBulkUpdateBlogPostsRequest - 101, // 195: productv1.ProductService.MutationCreateAuthor:input_type -> productv1.MutationCreateAuthorRequest - 97, // 196: productv1.ProductService.MutationCreateBlogPost:input_type -> productv1.MutationCreateBlogPostRequest - 93, // 197: productv1.ProductService.MutationCreateNullableFieldsType:input_type -> productv1.MutationCreateNullableFieldsTypeRequest - 89, // 198: productv1.ProductService.MutationCreateUser:input_type -> productv1.MutationCreateUserRequest - 91, // 199: productv1.ProductService.MutationPerformAction:input_type -> productv1.MutationPerformActionRequest - 103, // 200: productv1.ProductService.MutationUpdateAuthor:input_type -> productv1.MutationUpdateAuthorRequest - 99, // 201: productv1.ProductService.MutationUpdateBlogPost:input_type -> productv1.MutationUpdateBlogPostRequest - 95, // 202: productv1.ProductService.MutationUpdateNullableFieldsType:input_type -> productv1.MutationUpdateNullableFieldsTypeRequest - 83, // 203: productv1.ProductService.QueryAllAuthors:input_type -> productv1.QueryAllAuthorsRequest - 75, // 204: productv1.ProductService.QueryAllBlogPosts:input_type -> productv1.QueryAllBlogPostsRequest - 67, // 205: productv1.ProductService.QueryAllNullableFieldsTypes:input_type -> productv1.QueryAllNullableFieldsTypesRequest - 55, // 206: productv1.ProductService.QueryAllPets:input_type -> productv1.QueryAllPetsRequest - 77, // 207: productv1.ProductService.QueryAuthor:input_type -> productv1.QueryAuthorRequest - 79, // 208: productv1.ProductService.QueryAuthorById:input_type -> productv1.QueryAuthorByIdRequest - 81, // 209: productv1.ProductService.QueryAuthorsWithFilter:input_type -> productv1.QueryAuthorsWithFilterRequest - 69, // 210: productv1.ProductService.QueryBlogPost:input_type -> productv1.QueryBlogPostRequest - 71, // 211: productv1.ProductService.QueryBlogPostById:input_type -> productv1.QueryBlogPostByIdRequest - 73, // 212: productv1.ProductService.QueryBlogPostsWithFilter:input_type -> productv1.QueryBlogPostsWithFilterRequest - 85, // 213: productv1.ProductService.QueryBulkSearchAuthors:input_type -> productv1.QueryBulkSearchAuthorsRequest - 87, // 214: productv1.ProductService.QueryBulkSearchBlogPosts:input_type -> productv1.QueryBulkSearchBlogPostsRequest - 43, // 215: productv1.ProductService.QueryCalculateTotals:input_type -> productv1.QueryCalculateTotalsRequest - 45, // 216: productv1.ProductService.QueryCategories:input_type -> productv1.QueryCategoriesRequest - 47, // 217: productv1.ProductService.QueryCategoriesByKind:input_type -> productv1.QueryCategoriesByKindRequest - 49, // 218: productv1.ProductService.QueryCategoriesByKinds:input_type -> productv1.QueryCategoriesByKindsRequest - 41, // 219: productv1.ProductService.QueryComplexFilterType:input_type -> productv1.QueryComplexFilterTypeRequest - 51, // 220: productv1.ProductService.QueryFilterCategories:input_type -> productv1.QueryFilterCategoriesRequest - 33, // 221: productv1.ProductService.QueryNestedType:input_type -> productv1.QueryNestedTypeRequest - 61, // 222: productv1.ProductService.QueryNullableFieldsType:input_type -> productv1.QueryNullableFieldsTypeRequest - 63, // 223: productv1.ProductService.QueryNullableFieldsTypeById:input_type -> productv1.QueryNullableFieldsTypeByIdRequest - 65, // 224: productv1.ProductService.QueryNullableFieldsTypeWithFilter:input_type -> productv1.QueryNullableFieldsTypeWithFilterRequest - 53, // 225: productv1.ProductService.QueryRandomPet:input_type -> productv1.QueryRandomPetRequest - 59, // 226: productv1.ProductService.QueryRandomSearchResult:input_type -> productv1.QueryRandomSearchResultRequest - 35, // 227: productv1.ProductService.QueryRecursiveType:input_type -> productv1.QueryRecursiveTypeRequest - 57, // 228: productv1.ProductService.QuerySearch:input_type -> productv1.QuerySearchRequest - 37, // 229: productv1.ProductService.QueryTypeFilterWithArguments:input_type -> productv1.QueryTypeFilterWithArgumentsRequest - 39, // 230: productv1.ProductService.QueryTypeWithMultipleFilterFields:input_type -> productv1.QueryTypeWithMultipleFilterFieldsRequest - 31, // 231: productv1.ProductService.QueryUser:input_type -> productv1.QueryUserRequest - 29, // 232: productv1.ProductService.QueryUsers:input_type -> productv1.QueryUsersRequest - 22, // 233: productv1.ProductService.LookupProductById:output_type -> productv1.LookupProductByIdResponse - 25, // 234: productv1.ProductService.LookupStorageById:output_type -> productv1.LookupStorageByIdResponse - 28, // 235: productv1.ProductService.LookupWarehouseById:output_type -> productv1.LookupWarehouseByIdResponse - 106, // 236: productv1.ProductService.MutationBulkCreateAuthors:output_type -> productv1.MutationBulkCreateAuthorsResponse - 110, // 237: productv1.ProductService.MutationBulkCreateBlogPosts:output_type -> productv1.MutationBulkCreateBlogPostsResponse - 108, // 238: productv1.ProductService.MutationBulkUpdateAuthors:output_type -> productv1.MutationBulkUpdateAuthorsResponse - 112, // 239: productv1.ProductService.MutationBulkUpdateBlogPosts:output_type -> productv1.MutationBulkUpdateBlogPostsResponse - 102, // 240: productv1.ProductService.MutationCreateAuthor:output_type -> productv1.MutationCreateAuthorResponse - 98, // 241: productv1.ProductService.MutationCreateBlogPost:output_type -> productv1.MutationCreateBlogPostResponse - 94, // 242: productv1.ProductService.MutationCreateNullableFieldsType:output_type -> productv1.MutationCreateNullableFieldsTypeResponse - 90, // 243: productv1.ProductService.MutationCreateUser:output_type -> productv1.MutationCreateUserResponse - 92, // 244: productv1.ProductService.MutationPerformAction:output_type -> productv1.MutationPerformActionResponse - 104, // 245: productv1.ProductService.MutationUpdateAuthor:output_type -> productv1.MutationUpdateAuthorResponse - 100, // 246: productv1.ProductService.MutationUpdateBlogPost:output_type -> productv1.MutationUpdateBlogPostResponse - 96, // 247: productv1.ProductService.MutationUpdateNullableFieldsType:output_type -> productv1.MutationUpdateNullableFieldsTypeResponse - 84, // 248: productv1.ProductService.QueryAllAuthors:output_type -> productv1.QueryAllAuthorsResponse - 76, // 249: productv1.ProductService.QueryAllBlogPosts:output_type -> productv1.QueryAllBlogPostsResponse - 68, // 250: productv1.ProductService.QueryAllNullableFieldsTypes:output_type -> productv1.QueryAllNullableFieldsTypesResponse - 56, // 251: productv1.ProductService.QueryAllPets:output_type -> productv1.QueryAllPetsResponse - 78, // 252: productv1.ProductService.QueryAuthor:output_type -> productv1.QueryAuthorResponse - 80, // 253: productv1.ProductService.QueryAuthorById:output_type -> productv1.QueryAuthorByIdResponse - 82, // 254: productv1.ProductService.QueryAuthorsWithFilter:output_type -> productv1.QueryAuthorsWithFilterResponse - 70, // 255: productv1.ProductService.QueryBlogPost:output_type -> productv1.QueryBlogPostResponse - 72, // 256: productv1.ProductService.QueryBlogPostById:output_type -> productv1.QueryBlogPostByIdResponse - 74, // 257: productv1.ProductService.QueryBlogPostsWithFilter:output_type -> productv1.QueryBlogPostsWithFilterResponse - 86, // 258: productv1.ProductService.QueryBulkSearchAuthors:output_type -> productv1.QueryBulkSearchAuthorsResponse - 88, // 259: productv1.ProductService.QueryBulkSearchBlogPosts:output_type -> productv1.QueryBulkSearchBlogPostsResponse - 44, // 260: productv1.ProductService.QueryCalculateTotals:output_type -> productv1.QueryCalculateTotalsResponse - 46, // 261: productv1.ProductService.QueryCategories:output_type -> productv1.QueryCategoriesResponse - 48, // 262: productv1.ProductService.QueryCategoriesByKind:output_type -> productv1.QueryCategoriesByKindResponse - 50, // 263: productv1.ProductService.QueryCategoriesByKinds:output_type -> productv1.QueryCategoriesByKindsResponse - 42, // 264: productv1.ProductService.QueryComplexFilterType:output_type -> productv1.QueryComplexFilterTypeResponse - 52, // 265: productv1.ProductService.QueryFilterCategories:output_type -> productv1.QueryFilterCategoriesResponse - 34, // 266: productv1.ProductService.QueryNestedType:output_type -> productv1.QueryNestedTypeResponse - 62, // 267: productv1.ProductService.QueryNullableFieldsType:output_type -> productv1.QueryNullableFieldsTypeResponse - 64, // 268: productv1.ProductService.QueryNullableFieldsTypeById:output_type -> productv1.QueryNullableFieldsTypeByIdResponse - 66, // 269: productv1.ProductService.QueryNullableFieldsTypeWithFilter:output_type -> productv1.QueryNullableFieldsTypeWithFilterResponse - 54, // 270: productv1.ProductService.QueryRandomPet:output_type -> productv1.QueryRandomPetResponse - 60, // 271: productv1.ProductService.QueryRandomSearchResult:output_type -> productv1.QueryRandomSearchResultResponse - 36, // 272: productv1.ProductService.QueryRecursiveType:output_type -> productv1.QueryRecursiveTypeResponse - 58, // 273: productv1.ProductService.QuerySearch:output_type -> productv1.QuerySearchResponse - 38, // 274: productv1.ProductService.QueryTypeFilterWithArguments:output_type -> productv1.QueryTypeFilterWithArgumentsResponse - 40, // 275: productv1.ProductService.QueryTypeWithMultipleFilterFields:output_type -> productv1.QueryTypeWithMultipleFilterFieldsResponse - 32, // 276: productv1.ProductService.QueryUser:output_type -> productv1.QueryUserResponse - 30, // 277: productv1.ProductService.QueryUsers:output_type -> productv1.QueryUsersResponse - 233, // [233:278] is the sub-list for method output_type - 188, // [188:233] is the sub-list for method input_type - 188, // [188:188] is the sub-list for extension type_name - 188, // [188:188] is the sub-list for extension extendee - 0, // [0:188] is the sub-list for field type_name + 138, // 90: productv1.MutationBulkUpdateBlogPostsResponse.bulk_update_blog_posts:type_name -> productv1.BlogPost + 159, // 91: productv1.CategoryProductCountArgs.filters:type_name -> productv1.ProductCountFilter + 114, // 92: productv1.ResolveCategoryProductCountRequestKey.context:type_name -> productv1.CategoryProductCountContext + 113, // 93: productv1.ResolveCategoryProductCountRequestKey.field_args:type_name -> productv1.CategoryProductCountArgs + 115, // 94: productv1.ResolveCategoryProductCountRequest.key:type_name -> productv1.ResolveCategoryProductCountRequestKey + 117, // 95: productv1.ResolveCategoryProductCountResponse.result:type_name -> productv1.ResolveCategoryProductCountResponseResult + 148, // 96: productv1.NestedTypeA.b:type_name -> productv1.NestedTypeB + 124, // 97: productv1.RecursiveType.recursive_type:type_name -> productv1.RecursiveType + 150, // 98: productv1.ComplexFilterTypeInput.filter:type_name -> productv1.FilterType + 152, // 99: productv1.OrderInput.lines:type_name -> productv1.OrderLineInput + 15, // 100: productv1.Order.order_lines:type_name -> productv1.ListOfOrderLine + 0, // 101: productv1.Category.kind:type_name -> productv1.CategoryKind + 0, // 102: productv1.CategoryFilter.category:type_name -> productv1.CategoryKind + 151, // 103: productv1.CategoryFilter.pagination:type_name -> productv1.Pagination + 154, // 104: productv1.Animal.cat:type_name -> productv1.Cat + 155, // 105: productv1.Animal.dog:type_name -> productv1.Dog + 179, // 106: productv1.SearchInput.limit:type_name -> google.protobuf.Int32Value + 119, // 107: productv1.SearchResult.product:type_name -> productv1.Product + 122, // 108: productv1.SearchResult.user:type_name -> productv1.User + 131, // 109: productv1.SearchResult.category:type_name -> productv1.Category + 180, // 110: productv1.NullableFieldsType.optional_string:type_name -> google.protobuf.StringValue + 179, // 111: productv1.NullableFieldsType.optional_int:type_name -> google.protobuf.Int32Value + 181, // 112: productv1.NullableFieldsType.optional_float:type_name -> google.protobuf.DoubleValue + 182, // 113: productv1.NullableFieldsType.optional_boolean:type_name -> google.protobuf.BoolValue + 180, // 114: productv1.NullableFieldsFilter.name:type_name -> google.protobuf.StringValue + 180, // 115: productv1.NullableFieldsFilter.optional_string:type_name -> google.protobuf.StringValue + 182, // 116: productv1.NullableFieldsFilter.include_nulls:type_name -> google.protobuf.BoolValue + 17, // 117: productv1.BlogPost.optional_tags:type_name -> productv1.ListOfString + 17, // 118: productv1.BlogPost.keywords:type_name -> productv1.ListOfString + 9, // 119: productv1.BlogPost.ratings:type_name -> productv1.ListOfFloat + 6, // 120: productv1.BlogPost.is_published:type_name -> productv1.ListOfBoolean + 12, // 121: productv1.BlogPost.tag_groups:type_name -> productv1.ListOfListOfString + 12, // 122: productv1.BlogPost.related_topics:type_name -> productv1.ListOfListOfString + 12, // 123: productv1.BlogPost.comment_threads:type_name -> productv1.ListOfListOfString + 12, // 124: productv1.BlogPost.suggestions:type_name -> productv1.ListOfListOfString + 131, // 125: productv1.BlogPost.related_categories:type_name -> productv1.Category + 122, // 126: productv1.BlogPost.contributors:type_name -> productv1.User + 16, // 127: productv1.BlogPost.mentioned_products:type_name -> productv1.ListOfProduct + 18, // 128: productv1.BlogPost.mentioned_users:type_name -> productv1.ListOfUser + 10, // 129: productv1.BlogPost.category_groups:type_name -> productv1.ListOfListOfCategory + 13, // 130: productv1.BlogPost.contributor_teams:type_name -> productv1.ListOfListOfUser + 180, // 131: productv1.BlogPostFilter.title:type_name -> google.protobuf.StringValue + 182, // 132: productv1.BlogPostFilter.has_categories:type_name -> google.protobuf.BoolValue + 179, // 133: productv1.BlogPostFilter.min_tags:type_name -> google.protobuf.Int32Value + 180, // 134: productv1.Author.email:type_name -> google.protobuf.StringValue + 17, // 135: productv1.Author.social_links:type_name -> productv1.ListOfString + 12, // 136: productv1.Author.teams_by_project:type_name -> productv1.ListOfListOfString + 12, // 137: productv1.Author.collaborations:type_name -> productv1.ListOfListOfString + 3, // 138: productv1.Author.written_posts:type_name -> productv1.ListOfBlogPost + 131, // 139: productv1.Author.favorite_categories:type_name -> productv1.Category + 18, // 140: productv1.Author.related_authors:type_name -> productv1.ListOfUser + 16, // 141: productv1.Author.product_reviews:type_name -> productv1.ListOfProduct + 13, // 142: productv1.Author.author_groups:type_name -> productv1.ListOfListOfUser + 10, // 143: productv1.Author.category_preferences:type_name -> productv1.ListOfListOfCategory + 13, // 144: productv1.Author.project_teams:type_name -> productv1.ListOfListOfUser + 180, // 145: productv1.AuthorFilter.name:type_name -> google.protobuf.StringValue + 182, // 146: productv1.AuthorFilter.has_teams:type_name -> google.protobuf.BoolValue + 179, // 147: productv1.AuthorFilter.skill_count:type_name -> google.protobuf.Int32Value + 156, // 148: productv1.ActionResult.action_success:type_name -> productv1.ActionSuccess + 157, // 149: productv1.ActionResult.action_error:type_name -> productv1.ActionError + 180, // 150: productv1.NullableFieldsInput.optional_string:type_name -> google.protobuf.StringValue + 179, // 151: productv1.NullableFieldsInput.optional_int:type_name -> google.protobuf.Int32Value + 181, // 152: productv1.NullableFieldsInput.optional_float:type_name -> google.protobuf.DoubleValue + 182, // 153: productv1.NullableFieldsInput.optional_boolean:type_name -> google.protobuf.BoolValue + 17, // 154: productv1.BlogPostInput.optional_tags:type_name -> productv1.ListOfString + 17, // 155: productv1.BlogPostInput.keywords:type_name -> productv1.ListOfString + 9, // 156: productv1.BlogPostInput.ratings:type_name -> productv1.ListOfFloat + 6, // 157: productv1.BlogPostInput.is_published:type_name -> productv1.ListOfBoolean + 12, // 158: productv1.BlogPostInput.tag_groups:type_name -> productv1.ListOfListOfString + 12, // 159: productv1.BlogPostInput.related_topics:type_name -> productv1.ListOfListOfString + 12, // 160: productv1.BlogPostInput.comment_threads:type_name -> productv1.ListOfListOfString + 12, // 161: productv1.BlogPostInput.suggestions:type_name -> productv1.ListOfListOfString + 8, // 162: productv1.BlogPostInput.related_categories:type_name -> productv1.ListOfCategoryInput + 19, // 163: productv1.BlogPostInput.contributors:type_name -> productv1.ListOfUserInput + 11, // 164: productv1.BlogPostInput.category_groups:type_name -> productv1.ListOfListOfCategoryInput + 180, // 165: productv1.AuthorInput.email:type_name -> google.protobuf.StringValue + 17, // 166: productv1.AuthorInput.social_links:type_name -> productv1.ListOfString + 12, // 167: productv1.AuthorInput.teams_by_project:type_name -> productv1.ListOfListOfString + 12, // 168: productv1.AuthorInput.collaborations:type_name -> productv1.ListOfListOfString + 158, // 169: productv1.AuthorInput.favorite_categories:type_name -> productv1.CategoryInput + 14, // 170: productv1.AuthorInput.author_groups:type_name -> productv1.ListOfListOfUserInput + 14, // 171: productv1.AuthorInput.project_teams:type_name -> productv1.ListOfListOfUserInput + 149, // 172: productv1.NestedTypeB.c:type_name -> productv1.NestedTypeC + 151, // 173: productv1.FilterType.pagination:type_name -> productv1.Pagination + 17, // 174: productv1.OrderLineInput.modifiers:type_name -> productv1.ListOfString + 17, // 175: productv1.OrderLine.modifiers:type_name -> productv1.ListOfString + 0, // 176: productv1.CategoryInput.kind:type_name -> productv1.CategoryKind + 181, // 177: productv1.ProductCountFilter.min_price:type_name -> google.protobuf.DoubleValue + 181, // 178: productv1.ProductCountFilter.max_price:type_name -> google.protobuf.DoubleValue + 182, // 179: productv1.ProductCountFilter.in_stock:type_name -> google.protobuf.BoolValue + 180, // 180: productv1.ProductCountFilter.search_term:type_name -> google.protobuf.StringValue + 141, // 181: productv1.ListOfAuthorFilter.List.items:type_name -> productv1.AuthorFilter + 147, // 182: productv1.ListOfAuthorInput.List.items:type_name -> productv1.AuthorInput + 138, // 183: productv1.ListOfBlogPost.List.items:type_name -> productv1.BlogPost + 139, // 184: productv1.ListOfBlogPostFilter.List.items:type_name -> productv1.BlogPostFilter + 146, // 185: productv1.ListOfBlogPostInput.List.items:type_name -> productv1.BlogPostInput + 131, // 186: productv1.ListOfCategory.List.items:type_name -> productv1.Category + 158, // 187: productv1.ListOfCategoryInput.List.items:type_name -> productv1.CategoryInput + 7, // 188: productv1.ListOfListOfCategory.List.items:type_name -> productv1.ListOfCategory + 8, // 189: productv1.ListOfListOfCategoryInput.List.items:type_name -> productv1.ListOfCategoryInput + 17, // 190: productv1.ListOfListOfString.List.items:type_name -> productv1.ListOfString + 18, // 191: productv1.ListOfListOfUser.List.items:type_name -> productv1.ListOfUser + 19, // 192: productv1.ListOfListOfUserInput.List.items:type_name -> productv1.ListOfUserInput + 153, // 193: productv1.ListOfOrderLine.List.items:type_name -> productv1.OrderLine + 119, // 194: productv1.ListOfProduct.List.items:type_name -> productv1.Product + 122, // 195: productv1.ListOfUser.List.items:type_name -> productv1.User + 142, // 196: productv1.ListOfUserInput.List.items:type_name -> productv1.UserInput + 21, // 197: productv1.ProductService.LookupProductById:input_type -> productv1.LookupProductByIdRequest + 24, // 198: productv1.ProductService.LookupStorageById:input_type -> productv1.LookupStorageByIdRequest + 27, // 199: productv1.ProductService.LookupWarehouseById:input_type -> productv1.LookupWarehouseByIdRequest + 105, // 200: productv1.ProductService.MutationBulkCreateAuthors:input_type -> productv1.MutationBulkCreateAuthorsRequest + 109, // 201: productv1.ProductService.MutationBulkCreateBlogPosts:input_type -> productv1.MutationBulkCreateBlogPostsRequest + 107, // 202: productv1.ProductService.MutationBulkUpdateAuthors:input_type -> productv1.MutationBulkUpdateAuthorsRequest + 111, // 203: productv1.ProductService.MutationBulkUpdateBlogPosts:input_type -> productv1.MutationBulkUpdateBlogPostsRequest + 101, // 204: productv1.ProductService.MutationCreateAuthor:input_type -> productv1.MutationCreateAuthorRequest + 97, // 205: productv1.ProductService.MutationCreateBlogPost:input_type -> productv1.MutationCreateBlogPostRequest + 93, // 206: productv1.ProductService.MutationCreateNullableFieldsType:input_type -> productv1.MutationCreateNullableFieldsTypeRequest + 89, // 207: productv1.ProductService.MutationCreateUser:input_type -> productv1.MutationCreateUserRequest + 91, // 208: productv1.ProductService.MutationPerformAction:input_type -> productv1.MutationPerformActionRequest + 103, // 209: productv1.ProductService.MutationUpdateAuthor:input_type -> productv1.MutationUpdateAuthorRequest + 99, // 210: productv1.ProductService.MutationUpdateBlogPost:input_type -> productv1.MutationUpdateBlogPostRequest + 95, // 211: productv1.ProductService.MutationUpdateNullableFieldsType:input_type -> productv1.MutationUpdateNullableFieldsTypeRequest + 83, // 212: productv1.ProductService.QueryAllAuthors:input_type -> productv1.QueryAllAuthorsRequest + 75, // 213: productv1.ProductService.QueryAllBlogPosts:input_type -> productv1.QueryAllBlogPostsRequest + 67, // 214: productv1.ProductService.QueryAllNullableFieldsTypes:input_type -> productv1.QueryAllNullableFieldsTypesRequest + 55, // 215: productv1.ProductService.QueryAllPets:input_type -> productv1.QueryAllPetsRequest + 77, // 216: productv1.ProductService.QueryAuthor:input_type -> productv1.QueryAuthorRequest + 79, // 217: productv1.ProductService.QueryAuthorById:input_type -> productv1.QueryAuthorByIdRequest + 81, // 218: productv1.ProductService.QueryAuthorsWithFilter:input_type -> productv1.QueryAuthorsWithFilterRequest + 69, // 219: productv1.ProductService.QueryBlogPost:input_type -> productv1.QueryBlogPostRequest + 71, // 220: productv1.ProductService.QueryBlogPostById:input_type -> productv1.QueryBlogPostByIdRequest + 73, // 221: productv1.ProductService.QueryBlogPostsWithFilter:input_type -> productv1.QueryBlogPostsWithFilterRequest + 85, // 222: productv1.ProductService.QueryBulkSearchAuthors:input_type -> productv1.QueryBulkSearchAuthorsRequest + 87, // 223: productv1.ProductService.QueryBulkSearchBlogPosts:input_type -> productv1.QueryBulkSearchBlogPostsRequest + 43, // 224: productv1.ProductService.QueryCalculateTotals:input_type -> productv1.QueryCalculateTotalsRequest + 45, // 225: productv1.ProductService.QueryCategories:input_type -> productv1.QueryCategoriesRequest + 47, // 226: productv1.ProductService.QueryCategoriesByKind:input_type -> productv1.QueryCategoriesByKindRequest + 49, // 227: productv1.ProductService.QueryCategoriesByKinds:input_type -> productv1.QueryCategoriesByKindsRequest + 41, // 228: productv1.ProductService.QueryComplexFilterType:input_type -> productv1.QueryComplexFilterTypeRequest + 51, // 229: productv1.ProductService.QueryFilterCategories:input_type -> productv1.QueryFilterCategoriesRequest + 33, // 230: productv1.ProductService.QueryNestedType:input_type -> productv1.QueryNestedTypeRequest + 61, // 231: productv1.ProductService.QueryNullableFieldsType:input_type -> productv1.QueryNullableFieldsTypeRequest + 63, // 232: productv1.ProductService.QueryNullableFieldsTypeById:input_type -> productv1.QueryNullableFieldsTypeByIdRequest + 65, // 233: productv1.ProductService.QueryNullableFieldsTypeWithFilter:input_type -> productv1.QueryNullableFieldsTypeWithFilterRequest + 53, // 234: productv1.ProductService.QueryRandomPet:input_type -> productv1.QueryRandomPetRequest + 59, // 235: productv1.ProductService.QueryRandomSearchResult:input_type -> productv1.QueryRandomSearchResultRequest + 35, // 236: productv1.ProductService.QueryRecursiveType:input_type -> productv1.QueryRecursiveTypeRequest + 57, // 237: productv1.ProductService.QuerySearch:input_type -> productv1.QuerySearchRequest + 37, // 238: productv1.ProductService.QueryTypeFilterWithArguments:input_type -> productv1.QueryTypeFilterWithArgumentsRequest + 39, // 239: productv1.ProductService.QueryTypeWithMultipleFilterFields:input_type -> productv1.QueryTypeWithMultipleFilterFieldsRequest + 31, // 240: productv1.ProductService.QueryUser:input_type -> productv1.QueryUserRequest + 29, // 241: productv1.ProductService.QueryUsers:input_type -> productv1.QueryUsersRequest + 116, // 242: productv1.ProductService.ResolveCategoryProductCount:input_type -> productv1.ResolveCategoryProductCountRequest + 22, // 243: productv1.ProductService.LookupProductById:output_type -> productv1.LookupProductByIdResponse + 25, // 244: productv1.ProductService.LookupStorageById:output_type -> productv1.LookupStorageByIdResponse + 28, // 245: productv1.ProductService.LookupWarehouseById:output_type -> productv1.LookupWarehouseByIdResponse + 106, // 246: productv1.ProductService.MutationBulkCreateAuthors:output_type -> productv1.MutationBulkCreateAuthorsResponse + 110, // 247: productv1.ProductService.MutationBulkCreateBlogPosts:output_type -> productv1.MutationBulkCreateBlogPostsResponse + 108, // 248: productv1.ProductService.MutationBulkUpdateAuthors:output_type -> productv1.MutationBulkUpdateAuthorsResponse + 112, // 249: productv1.ProductService.MutationBulkUpdateBlogPosts:output_type -> productv1.MutationBulkUpdateBlogPostsResponse + 102, // 250: productv1.ProductService.MutationCreateAuthor:output_type -> productv1.MutationCreateAuthorResponse + 98, // 251: productv1.ProductService.MutationCreateBlogPost:output_type -> productv1.MutationCreateBlogPostResponse + 94, // 252: productv1.ProductService.MutationCreateNullableFieldsType:output_type -> productv1.MutationCreateNullableFieldsTypeResponse + 90, // 253: productv1.ProductService.MutationCreateUser:output_type -> productv1.MutationCreateUserResponse + 92, // 254: productv1.ProductService.MutationPerformAction:output_type -> productv1.MutationPerformActionResponse + 104, // 255: productv1.ProductService.MutationUpdateAuthor:output_type -> productv1.MutationUpdateAuthorResponse + 100, // 256: productv1.ProductService.MutationUpdateBlogPost:output_type -> productv1.MutationUpdateBlogPostResponse + 96, // 257: productv1.ProductService.MutationUpdateNullableFieldsType:output_type -> productv1.MutationUpdateNullableFieldsTypeResponse + 84, // 258: productv1.ProductService.QueryAllAuthors:output_type -> productv1.QueryAllAuthorsResponse + 76, // 259: productv1.ProductService.QueryAllBlogPosts:output_type -> productv1.QueryAllBlogPostsResponse + 68, // 260: productv1.ProductService.QueryAllNullableFieldsTypes:output_type -> productv1.QueryAllNullableFieldsTypesResponse + 56, // 261: productv1.ProductService.QueryAllPets:output_type -> productv1.QueryAllPetsResponse + 78, // 262: productv1.ProductService.QueryAuthor:output_type -> productv1.QueryAuthorResponse + 80, // 263: productv1.ProductService.QueryAuthorById:output_type -> productv1.QueryAuthorByIdResponse + 82, // 264: productv1.ProductService.QueryAuthorsWithFilter:output_type -> productv1.QueryAuthorsWithFilterResponse + 70, // 265: productv1.ProductService.QueryBlogPost:output_type -> productv1.QueryBlogPostResponse + 72, // 266: productv1.ProductService.QueryBlogPostById:output_type -> productv1.QueryBlogPostByIdResponse + 74, // 267: productv1.ProductService.QueryBlogPostsWithFilter:output_type -> productv1.QueryBlogPostsWithFilterResponse + 86, // 268: productv1.ProductService.QueryBulkSearchAuthors:output_type -> productv1.QueryBulkSearchAuthorsResponse + 88, // 269: productv1.ProductService.QueryBulkSearchBlogPosts:output_type -> productv1.QueryBulkSearchBlogPostsResponse + 44, // 270: productv1.ProductService.QueryCalculateTotals:output_type -> productv1.QueryCalculateTotalsResponse + 46, // 271: productv1.ProductService.QueryCategories:output_type -> productv1.QueryCategoriesResponse + 48, // 272: productv1.ProductService.QueryCategoriesByKind:output_type -> productv1.QueryCategoriesByKindResponse + 50, // 273: productv1.ProductService.QueryCategoriesByKinds:output_type -> productv1.QueryCategoriesByKindsResponse + 42, // 274: productv1.ProductService.QueryComplexFilterType:output_type -> productv1.QueryComplexFilterTypeResponse + 52, // 275: productv1.ProductService.QueryFilterCategories:output_type -> productv1.QueryFilterCategoriesResponse + 34, // 276: productv1.ProductService.QueryNestedType:output_type -> productv1.QueryNestedTypeResponse + 62, // 277: productv1.ProductService.QueryNullableFieldsType:output_type -> productv1.QueryNullableFieldsTypeResponse + 64, // 278: productv1.ProductService.QueryNullableFieldsTypeById:output_type -> productv1.QueryNullableFieldsTypeByIdResponse + 66, // 279: productv1.ProductService.QueryNullableFieldsTypeWithFilter:output_type -> productv1.QueryNullableFieldsTypeWithFilterResponse + 54, // 280: productv1.ProductService.QueryRandomPet:output_type -> productv1.QueryRandomPetResponse + 60, // 281: productv1.ProductService.QueryRandomSearchResult:output_type -> productv1.QueryRandomSearchResultResponse + 36, // 282: productv1.ProductService.QueryRecursiveType:output_type -> productv1.QueryRecursiveTypeResponse + 58, // 283: productv1.ProductService.QuerySearch:output_type -> productv1.QuerySearchResponse + 38, // 284: productv1.ProductService.QueryTypeFilterWithArguments:output_type -> productv1.QueryTypeFilterWithArgumentsResponse + 40, // 285: productv1.ProductService.QueryTypeWithMultipleFilterFields:output_type -> productv1.QueryTypeWithMultipleFilterFieldsResponse + 32, // 286: productv1.ProductService.QueryUser:output_type -> productv1.QueryUserResponse + 30, // 287: productv1.ProductService.QueryUsers:output_type -> productv1.QueryUsersResponse + 118, // 288: productv1.ProductService.ResolveCategoryProductCount:output_type -> productv1.ResolveCategoryProductCountResponse + 243, // [243:289] is the sub-list for method output_type + 197, // [197:243] is the sub-list for method input_type + 197, // [197:197] is the sub-list for extension type_name + 197, // [197:197] is the sub-list for extension extendee + 0, // [0:197] is the sub-list for field type_name } func init() { file_product_proto_init() } @@ -9836,16 +10226,16 @@ func file_product_proto_init() { if File_product_proto != nil { return } - file_product_proto_msgTypes[126].OneofWrappers = []any{ + file_product_proto_msgTypes[132].OneofWrappers = []any{ (*Animal_Cat)(nil), (*Animal_Dog)(nil), } - file_product_proto_msgTypes[128].OneofWrappers = []any{ + file_product_proto_msgTypes[134].OneofWrappers = []any{ (*SearchResult_Product)(nil), (*SearchResult_User)(nil), (*SearchResult_Category)(nil), } - file_product_proto_msgTypes[137].OneofWrappers = []any{ + file_product_proto_msgTypes[143].OneofWrappers = []any{ (*ActionResult_ActionSuccess)(nil), (*ActionResult_ActionError)(nil), } @@ -9855,7 +10245,7 @@ func file_product_proto_init() { GoPackagePath: reflect.TypeOf(x{}).PkgPath(), RawDescriptor: unsafe.Slice(unsafe.StringData(file_product_proto_rawDesc), len(file_product_proto_rawDesc)), NumEnums: 1, - NumMessages: 171, + NumMessages: 178, NumExtensions: 0, NumServices: 1, }, diff --git a/v2/pkg/grpctest/productv1/product_grpc.pb.go b/v2/pkg/grpctest/productv1/product_grpc.pb.go index 9918bf4a6..3821e954b 100644 --- a/v2/pkg/grpctest/productv1/product_grpc.pb.go +++ b/v2/pkg/grpctest/productv1/product_grpc.pb.go @@ -1,7 +1,7 @@ // Code generated by protoc-gen-go-grpc. DO NOT EDIT. // versions: // - protoc-gen-go-grpc v1.5.1 -// - protoc v5.29.3 +// - protoc v6.32.0 // source: product.proto package productv1 @@ -64,6 +64,7 @@ const ( ProductService_QueryTypeWithMultipleFilterFields_FullMethodName = "/productv1.ProductService/QueryTypeWithMultipleFilterFields" ProductService_QueryUser_FullMethodName = "/productv1.ProductService/QueryUser" ProductService_QueryUsers_FullMethodName = "/productv1.ProductService/QueryUsers" + ProductService_ResolveCategoryProductCount_FullMethodName = "/productv1.ProductService/ResolveCategoryProductCount" ) // ProductServiceClient is the client API for ProductService service. @@ -120,6 +121,7 @@ type ProductServiceClient interface { QueryTypeWithMultipleFilterFields(ctx context.Context, in *QueryTypeWithMultipleFilterFieldsRequest, opts ...grpc.CallOption) (*QueryTypeWithMultipleFilterFieldsResponse, error) QueryUser(ctx context.Context, in *QueryUserRequest, opts ...grpc.CallOption) (*QueryUserResponse, error) QueryUsers(ctx context.Context, in *QueryUsersRequest, opts ...grpc.CallOption) (*QueryUsersResponse, error) + ResolveCategoryProductCount(ctx context.Context, in *ResolveCategoryProductCountRequest, opts ...grpc.CallOption) (*ResolveCategoryProductCountResponse, error) } type productServiceClient struct { @@ -580,6 +582,16 @@ func (c *productServiceClient) QueryUsers(ctx context.Context, in *QueryUsersReq return out, nil } +func (c *productServiceClient) ResolveCategoryProductCount(ctx context.Context, in *ResolveCategoryProductCountRequest, opts ...grpc.CallOption) (*ResolveCategoryProductCountResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(ResolveCategoryProductCountResponse) + err := c.cc.Invoke(ctx, ProductService_ResolveCategoryProductCount_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + // ProductServiceServer is the server API for ProductService service. // All implementations must embed UnimplementedProductServiceServer // for forward compatibility. @@ -634,6 +646,7 @@ type ProductServiceServer interface { QueryTypeWithMultipleFilterFields(context.Context, *QueryTypeWithMultipleFilterFieldsRequest) (*QueryTypeWithMultipleFilterFieldsResponse, error) QueryUser(context.Context, *QueryUserRequest) (*QueryUserResponse, error) QueryUsers(context.Context, *QueryUsersRequest) (*QueryUsersResponse, error) + ResolveCategoryProductCount(context.Context, *ResolveCategoryProductCountRequest) (*ResolveCategoryProductCountResponse, error) mustEmbedUnimplementedProductServiceServer() } @@ -779,6 +792,9 @@ func (UnimplementedProductServiceServer) QueryUser(context.Context, *QueryUserRe func (UnimplementedProductServiceServer) QueryUsers(context.Context, *QueryUsersRequest) (*QueryUsersResponse, error) { return nil, status.Errorf(codes.Unimplemented, "method QueryUsers not implemented") } +func (UnimplementedProductServiceServer) ResolveCategoryProductCount(context.Context, *ResolveCategoryProductCountRequest) (*ResolveCategoryProductCountResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method ResolveCategoryProductCount not implemented") +} func (UnimplementedProductServiceServer) mustEmbedUnimplementedProductServiceServer() {} func (UnimplementedProductServiceServer) testEmbeddedByValue() {} @@ -1610,6 +1626,24 @@ func _ProductService_QueryUsers_Handler(srv interface{}, ctx context.Context, de return interceptor(ctx, in, info, handler) } +func _ProductService_ResolveCategoryProductCount_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(ResolveCategoryProductCountRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(ProductServiceServer).ResolveCategoryProductCount(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: ProductService_ResolveCategoryProductCount_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(ProductServiceServer).ResolveCategoryProductCount(ctx, req.(*ResolveCategoryProductCountRequest)) + } + return interceptor(ctx, in, info, handler) +} + // ProductService_ServiceDesc is the grpc.ServiceDesc for ProductService service. // It's only intended for direct use with grpc.RegisterService, // and not to be introspected or modified (even as a copy) @@ -1797,6 +1831,10 @@ var ProductService_ServiceDesc = grpc.ServiceDesc{ MethodName: "QueryUsers", Handler: _ProductService_QueryUsers_Handler, }, + { + MethodName: "ResolveCategoryProductCount", + Handler: _ProductService_ResolveCategoryProductCount_Handler, + }, }, Streams: []grpc.StreamDesc{}, Metadata: "product.proto", diff --git a/v2/pkg/grpctest/testdata/products.graphqls b/v2/pkg/grpctest/testdata/products.graphqls index 8a422655c..a273796a4 100644 --- a/v2/pkg/grpctest/testdata/products.graphqls +++ b/v2/pkg/grpctest/testdata/products.graphqls @@ -1,3 +1,6 @@ +# Directive for gRPC field resolvers +directive @resolved(context: openfed__FieldSet!) on FIELD_DEFINITION + type Product @key(fields: "id") { id: ID! name: String! @@ -113,6 +116,7 @@ type Category { id: ID! name: String! kind: CategoryKind! + productCount(filters: ProductCountFilter): Int! @resolved(context: "id name") } enum CategoryKind { @@ -314,6 +318,13 @@ input CategoryInput { kind: CategoryKind! } +input ProductCountFilter { + minPrice: Float + maxPrice: Float + inStock: Boolean + searchTerm: String +} + type Query { _entities(representations: [_Any!]!): [_Entity!]! users: [User!]! @@ -402,3 +413,4 @@ type Mutation { union _Entity = Product | Storage | Warehouse scalar _Any +scalar openfed__FieldSet From 49ab0e844af928d838df4b4f3abfcc5d886fe7b5 Mon Sep 17 00:00:00 2001 From: Ludwig Bedacht Date: Mon, 6 Oct 2025 17:17:38 +0200 Subject: [PATCH 02/72] feat: update compiler and json builder to resolve fields --- v2/pkg/ast/path.go | 11 + .../datasource/grpc_datasource/compiler.go | 451 ++++++- .../grpc_datasource/compiler_test.go | 10 +- .../grpc_datasource/execution_plan.go | 32 +- .../execution_plan_field_resolvers_test.go | 105 +- .../grpc_datasource/execution_plan_visitor.go | 35 +- .../datasource/grpc_datasource/fetch.go | 144 +++ .../datasource/grpc_datasource/fetch_test.go | 178 +++ .../grpc_datasource/grpc_datasource.go | 107 +- .../grpc_datasource/grpc_datasource_test.go | 93 ++ .../grpc_datasource/json_builder.go | 92 +- .../grpc_datasource/mapping_test_helper.go | 374 +++--- v2/pkg/grpctest/cmd/mapping_helper/main.go | 191 +++ v2/pkg/grpctest/mapping/mapping.go | 367 +++--- v2/pkg/grpctest/mockservice.go | 6 +- v2/pkg/grpctest/product.proto | 8 +- v2/pkg/grpctest/productv1/product.pb.go | 1122 ++++++++--------- 17 files changed, 2191 insertions(+), 1135 deletions(-) create mode 100644 v2/pkg/engine/datasource/grpc_datasource/fetch.go create mode 100644 v2/pkg/engine/datasource/grpc_datasource/fetch_test.go create mode 100644 v2/pkg/grpctest/cmd/mapping_helper/main.go diff --git a/v2/pkg/ast/path.go b/v2/pkg/ast/path.go index 2ff55f3bf..8d28790e8 100644 --- a/v2/pkg/ast/path.go +++ b/v2/pkg/ast/path.go @@ -33,6 +33,10 @@ type PathItem struct { type Path []PathItem +func (p Path) Len() int { + return len(p) +} + func (p Path) Equals(another Path) bool { if len(p) != len(another) { return false @@ -77,6 +81,13 @@ func (p Path) WithoutInlineFragmentNames() Path { return out } +func (p Path) WithPathElement(element PathItem) Path { + res := make(Path, len(p)+1) + copy(res, p) + res[len(res)-1] = element + return res +} + func (p Path) String() string { out := "[" for i := range p { diff --git a/v2/pkg/engine/datasource/grpc_datasource/compiler.go b/v2/pkg/engine/datasource/grpc_datasource/compiler.go index 1f6b0ec1f..0df1098ac 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/compiler.go +++ b/v2/pkg/engine/datasource/grpc_datasource/compiler.go @@ -6,6 +6,7 @@ import ( "slices" "github.com/bufbuild/protocompile" + "github.com/cespare/xxhash/v2" "github.com/tidwall/gjson" protoref "google.golang.org/protobuf/reflect/protoreflect" "google.golang.org/protobuf/types/dynamicpb" @@ -14,6 +15,11 @@ import ( "github.com/wundergraph/graphql-go-tools/v2/pkg/operationreport" ) +const ( + // InvalidRef is a constant used to indicate that a reference is invalid. + InvalidRef = -1 +) + // DataType represents the different types of data that can be stored in a protobuf field. type DataType string @@ -89,8 +95,23 @@ func parseDataType(name string) DataType { return dataTypeMap[name] } +type NodeKind int + +const ( + NodeKindMessage NodeKind = iota + 1 + NodeKindEnum + NodeKindService + NodeKindUnknown +) + +type node struct { + ref int + kind NodeKind +} + // Document represents a compiled protobuf document with all its services, messages, and methods. type Document struct { + nodes map[uint64]node Package string // The package name of the protobuf document Imports []string // The imports of the protobuf document Services []Service // All services defined in the document @@ -99,6 +120,41 @@ type Document struct { Methods []Method // All methods from all services in the document } +// newNode creates a new node in the document. +func (d *Document) newNode(ref int, name string, kind NodeKind) { + h := xxhash.Sum64String(name) + d.nodes[h] = node{ + ref: ref, + kind: kind, + } +} + +// nodeByName returns a node by its name. +// Returns false if the node does not exist. +func (d *Document) nodeByName(name string) (node, bool) { + h := xxhash.Sum64String(name) + node, exists := d.nodes[h] + return node, exists +} + +// appendMessage appends a message to the document and returns the reference index. +func (d *Document) appendMessage(message Message) int { + d.Messages = append(d.Messages, message) + return len(d.Messages) - 1 +} + +// appendEnum appends an enum to the document and returns the reference index. +func (d *Document) appendEnum(enum Enum) int { + d.Enums = append(d.Enums, enum) + return len(d.Enums) - 1 +} + +// appendService appends a service to the document and returns the reference index. +func (d *Document) appendService(service Service) int { + d.Services = append(d.Services, service) + return len(d.Services) - 1 +} + // Service represents a gRPC service with methods. type Service struct { Name string // The name of the service @@ -122,6 +178,18 @@ type Message struct { Desc protoref.MessageDescriptor // The protobuf descriptor for the message } +// FieldByName returns a field by its name. +// Returns nil if no field with the given name exists. +func (m *Message) FieldByName(name string) *Field { + for _, field := range m.Fields { + if field.Name == name { + return &field + } + } + + return nil +} + // Field represents a field in a protobuf message. type Field struct { Name string // The name of the field @@ -229,13 +297,12 @@ func (d *Document) MessageByName(name string) (Message, bool) { // MessageRefByName returns the index of a Message in the Messages slice by its name. // Returns -1 if no message with the given name exists. func (d *Document) MessageRefByName(name string) int { - for i, m := range d.Messages { - if m.Name == name { - return i - } + node, found := d.nodeByName(name) + if !found || node.kind != NodeKindMessage { + return InvalidRef } + return node.ref - return -1 } // MessageByRef returns a Message by its reference index. @@ -243,6 +310,8 @@ func (d *Document) MessageByRef(ref int) Message { return d.Messages[ref] } +// EnumByName returns an Enum by its name. +// Returns false if the enum does not exist. func (d *Document) EnumByName(name string) (Enum, bool) { for _, e := range d.Enums { if e.Name == name { @@ -279,6 +348,7 @@ func NewProtoCompiler(schema string, mapping *GRPCMapping) (*RPCCompiler, error) schemaFile := fd[0] pc := &RPCCompiler{ doc: &Document{ + nodes: make(map[uint64]node), Package: string(schemaFile.Package()), }, report: operationreport.Report{}, @@ -304,18 +374,28 @@ func NewProtoCompiler(schema string, mapping *GRPCMapping) (*RPCCompiler, error) func (p *RPCCompiler) processFile(f protoref.FileDescriptor, mapping *GRPCMapping) { // Process all enums in the schema for i := 0; i < f.Enums().Len(); i++ { - p.doc.Enums = append(p.doc.Enums, p.parseEnum(f.Enums().Get(i), mapping)) + enum := p.parseEnum(f.Enums().Get(i), mapping) + ref := p.doc.appendEnum(enum) + p.doc.newNode(ref, enum.Name, NodeKindEnum) } // Process all messages in the schema - p.doc.Messages = append(p.doc.Messages, p.parseMessageDefinitions(f.Messages())...) + messages := p.parseMessageDefinitions(f.Messages()) + for _, message := range messages { + ref := p.doc.appendMessage(message) + p.doc.newNode(ref, message.Name, NodeKindMessage) + } + + // We need to reiterate over the messages to handle recursive types. for ref, message := range p.doc.Messages { p.enrichMessageData(ref, message.Desc) } // Process all services in the schema for i := 0; i < f.Services().Len(); i++ { - p.doc.Services = append(p.doc.Services, p.parseService(f.Services().Get(i))) + service := p.parseService(f.Services().Get(i)) + ref := p.doc.appendService(service) + p.doc.newNode(ref, service.Name, NodeKindService) } } @@ -325,19 +405,125 @@ func (p *RPCCompiler) ConstructExecutionPlan(operation, schema *ast.Document) (* return nil, nil } -// Invocation represents a single gRPC invocation with its input and output messages. -type Invocation struct { +// ServiceCall represents a single gRPC service call with its input and output messages. +type ServiceCall struct { + // ServiceName is the name of the gRPC service to call ServiceName string - MethodName string - Input *dynamicpb.Message - Output *dynamicpb.Message - Call *RPCCall + // MethodName is the name of the method on the service to call + MethodName string + // Input is the input message for the gRPC call + Input *dynamicpb.Message + // Output is the output message for the gRPC call + Output *dynamicpb.Message + // Call is the call that was made to the gRPC service + Call *RPCCall // TODO: Might be not needed anymore when we are using the DependencyGraph +} + +// func (p *RPCCompiler) CompileFetches(graph *DependencyGraph, fetches []FetchItem, inputData gjson.Result) ([]Invocation, error) { +// invocations := make([]Invocation, 0, len(fetches)) + +// resultChan := make(chan Invocation, len(fetches)) +// errChan := make(chan error, len(fetches)) + +// wg := sync.WaitGroup{} +// wg.Add(len(fetches)) + +// for _, node := range fetches { +// go func() { +// defer wg.Done() +// invocation, err := p.CompileNode(graph, node, inputData) +// if err != nil { +// errChan <- err +// return +// } + +// resultChan <- invocation +// node.Invocation = &invocation +// }() +// } + +// close(resultChan) +// close(errChan) + +// var joinErr error +// for err := range errChan { +// joinErr = errors.Join(joinErr, err) +// } + +// if joinErr != nil { +// return nil, joinErr +// } + +// for invocation := range resultChan { +// invocations = append(invocations, invocation) +// } + +// return invocations, nil +// } + +func (p *RPCCompiler) CompileFetches(graph *DependencyGraph, fetches []FetchItem, inputData gjson.Result) ([]ServiceCall, error) { + serviceCalls := make([]ServiceCall, 0, len(fetches)) + + for _, node := range fetches { + serviceCall, err := p.CompileNode(graph, node, inputData) + if err != nil { + return nil, err + } + + graph.SetFetchData(node.ID, &serviceCall) + serviceCalls = append(serviceCalls, serviceCall) + } + + return serviceCalls, nil +} + +func (p *RPCCompiler) CompileNode(graph *DependencyGraph, fetch FetchItem, inputData gjson.Result) (ServiceCall, error) { + call := fetch.Plan + + inputMessage, ok := p.doc.MessageByName(call.Request.Name) + if !ok { + return ServiceCall{}, fmt.Errorf("input message %s not found in document", call.Request.Name) + } + + outputMessage, ok := p.doc.MessageByName(call.Response.Name) + if !ok { + return ServiceCall{}, fmt.Errorf("output message %s not found in document", call.Response.Name) + } + + request, response := p.newEmptyMessage(inputMessage), p.newEmptyMessage(outputMessage) + + switch call.Kind { + case CallKindStandard, CallKindEntity: + request = p.buildProtoMessage(inputMessage, &call.Request, inputData) + case CallKindResolve: + context, err := graph.FetchDependencies(&fetch) + if err != nil { + return ServiceCall{}, err + } + + request = p.buildProtoMessageWithContext(inputMessage, &call.Request, inputData, context) + } + + serviceName, ok := p.resolveServiceName(call.MethodName) + if !ok { + return ServiceCall{}, fmt.Errorf("failed to resolve service name for method %s from the protobuf definition", call.MethodName) + } + + return ServiceCall{ + ServiceName: serviceName, + MethodName: call.MethodName, + Input: request, + Output: response, + Call: call, + }, nil + } // Compile processes an RPCExecutionPlan and builds protobuf messages from JSON data // based on the compiled schema. -func (p *RPCCompiler) Compile(executionPlan *RPCExecutionPlan, inputData gjson.Result) ([]Invocation, error) { - invocations := make([]Invocation, 0, len(executionPlan.Calls)) +// Deprecated: Use CompileFetches instead. +func (p *RPCCompiler) Compile(executionPlan *RPCExecutionPlan, inputData gjson.Result) ([]ServiceCall, error) { + serviceCalls := make([]ServiceCall, 0, len(executionPlan.Calls)) for _, call := range executionPlan.Calls { inputMessage, ok := p.doc.MessageByName(call.Request.Name) @@ -362,7 +548,7 @@ func (p *RPCCompiler) Compile(executionPlan *RPCExecutionPlan, inputData gjson.R return nil, fmt.Errorf("failed to resolve service name for method %s from the protobuf definition", call.MethodName) } - invocations = append(invocations, Invocation{ + serviceCalls = append(serviceCalls, ServiceCall{ ServiceName: serviceName, MethodName: call.MethodName, Input: request, @@ -371,7 +557,7 @@ func (p *RPCCompiler) Compile(executionPlan *RPCExecutionPlan, inputData gjson.R }) } - return invocations, nil + return serviceCalls, nil } func (p *RPCCompiler) resolveServiceName(methodName string) (string, bool) { @@ -388,7 +574,7 @@ func (p *RPCCompiler) resolveServiceName(methodName string) (string, bool) { // newEmptyMessage creates a new empty dynamicpb.Message from a Message definition. func (p *RPCCompiler) newEmptyMessage(message Message) *dynamicpb.Message { - if p.doc.MessageRefByName(message.Name) == -1 { + if p.doc.MessageRefByName(message.Name) == InvalidRef { p.report.AddInternalError(fmt.Errorf("message %s not found in document", message.Name)) return nil } @@ -396,15 +582,233 @@ func (p *RPCCompiler) newEmptyMessage(message Message) *dynamicpb.Message { return dynamicpb.NewMessage(message.Desc) } +// buildProtoMessageWithContext builds a protobuf message from an RPCMessage definition +// and JSON data. It handles nested messages and repeated fields. +// +// Example: +// +// message ResolveCategoryProductCountRequest { +// repeated CategoryProductCountContext context = 1; +// CategoryProductCountArgs field_args = 2; +// } +// +// message ResolveCategoryProductCountRequestKey { + +// } +func (p *RPCCompiler) buildProtoMessageWithContext(inputMessage Message, rpcMessage *RPCMessage, data gjson.Result, context []FetchItem) *dynamicpb.Message { + if rpcMessage == nil { + return nil + } + + if p.doc.MessageRefByName(rpcMessage.Name) == InvalidRef { + p.report.AddInternalError(fmt.Errorf("message %s not found in document", rpcMessage.Name)) + return nil + } + + rootMessage := dynamicpb.NewMessage(inputMessage.Desc) + + if len(inputMessage.Fields) != 2 { + p.report.AddInternalError(fmt.Errorf("message %s must have exactly one key field", inputMessage.Name)) + return nil + } + + // schemaField := inputMessage.FieldByName("key") + // if schemaField == nil { + // p.report.AddInternalError(fmt.Errorf("key field not found in message %s", inputMessage.Name)) + // return nil + // } + + // // the key field must be a repeated field + // // Get the RPC field + // planKeyField := rpcMessage.Fields.ByName(schemaField.Name) + // if planKeyField == nil { + // p.report.AddInternalError(fmt.Errorf("key field not found in message %s", rpcMessage.Name)) + // return nil + // } + + contextSchemaField := inputMessage.FieldByName("context") + if contextSchemaField == nil { + p.report.AddInternalError(fmt.Errorf("context field not found in message %s", inputMessage.Name)) + return nil + } + + contextRPCField := rpcMessage.Fields.ByName(contextSchemaField.Name) + if contextRPCField == nil { + p.report.AddInternalError(fmt.Errorf("context field not found in message %s", rpcMessage.Name)) + return nil + } + + contextField := rootMessage.Descriptor().Fields().ByNumber(protoref.FieldNumber(contextSchemaField.Number)) + if contextField == nil { + p.report.AddInternalError(fmt.Errorf("context field not found in message %s", inputMessage.Name)) + return nil + } + + contextList := p.newEmptyListMessageByName(rootMessage, contextSchemaField.Name) + contextData := p.resolveContextData(context[0], contextRPCField) + + for _, data := range contextData { + val := contextList.NewElement() + valMsg := val.Message() + for fieldName, value := range data { + p.setMessageValue(valMsg, fieldName, value) + } + + contextList.Append(val) + } + + argsMessage := p.doc.Messages[inputMessage.FieldByName("field_args").MessageRef] + argsRPCField := rpcMessage.Fields.ByName("field_args") + if argsRPCField == nil { + p.report.AddInternalError(fmt.Errorf("field_args field not found in message %s", rpcMessage.Name)) + return nil + } + + args := p.buildProtoMessage(argsMessage, rpcMessage.Fields[1].Message, data) + + // // Set the key list + p.setMessageValue(rootMessage, contextSchemaField.Name, protoref.ValueOfList(contextList)) + p.setMessageValue(rootMessage, argsRPCField.Name, protoref.ValueOfMessage(args)) + + return rootMessage +} + +func (p *RPCCompiler) resolveContextData(context FetchItem, contextField *RPCField) []map[string]protoref.Value { + if context.ServiceCall == nil || context.ServiceCall.Output == nil { + return []map[string]protoref.Value{} + } + + contextValues := make([]map[string]protoref.Value, 0) + outputMessage := context.ServiceCall.Output + for _, field := range contextField.Message.Fields { + resolvePath := field.ResolvePath + values := p.resolveContextDataForPath(outputMessage, resolvePath) + + for index, value := range values { + if index >= len(contextValues) { + contextValues = append(contextValues, make(map[string]protoref.Value)) + } + + contextValues[index][field.Name] = value + } + + } + + return contextValues +} + +func (p *RPCCompiler) resolveContextDataForPath(message protoref.Message, path ast.Path) []protoref.Value { + if path.Len() == 0 { + return nil + } + + segment := path[0] + path = path[1:] + + msg, fd := p.getMessageField(message, segment.FieldName.String()) + if !msg.IsValid() { + return nil + } + + if fd.IsList() { + return p.resolveListDataForPath(msg.List(), fd, path) + } + + return []protoref.Value{p.resolveDataForPath(msg.Message(), path)} + +} + +func (p *RPCCompiler) resolveListDataForPath(message protoref.List, fd protoref.FieldDescriptor, path ast.Path) []protoref.Value { + if path.Len() == 0 { + return nil + } + + result := make([]protoref.Value, 0, message.Len()) + + for i := range message.Len() { + item := message.Get(i) + + switch fd.Kind() { + case protoref.MessageKind: + val := p.resolveDataForPath(item.Message(), path) + + if list, isList := val.Interface().(protoref.List); isList { + values := p.resolveListDataForPath(list, fd, path) + result = append(result, values...) + continue + } + + result = append(result, val) + default: + result = append(result, item) + } + + // val := p.resolveDataForPath(item.Message(), path) + // if list, isList := val.Interface().(protoref.List); isList { + // value := p.resolveListDataForPath(list, fd, path) + // result = append(result, value...) + // continue + // } + + // result = append(result, val) + } + + return result +} + +func (p *RPCCompiler) resolveDataForPath(outputMessage protoref.Message, path ast.Path) protoref.Value { + if path.Len() == 0 { + return protoref.Value{} + } + + segment := path[0] + field, fd := p.getMessageField(outputMessage, segment.FieldName.String()) + if !field.IsValid() { + return protoref.Value{} + } + + switch fd.Kind() { + case protoref.MessageKind: + if fd.IsList() { + return field + } + + return p.resolveDataForPath(field.Message(), path[1:]) + } + + return field +} + +func (p *RPCCompiler) getMessageField(message protoref.Message, fieldName string) (protoref.Value, protoref.FieldDescriptor) { + fd := message.Descriptor().Fields().ByName(protoref.Name(fieldName)) + if fd == nil { + return protoref.Value{}, nil + } + + return message.Get(fd), fd +} + +func (p *RPCCompiler) newEmptyListMessageByNumber(msg *dynamicpb.Message, number int32) protoref.List { + return msg.Mutable(msg.Descriptor().Fields().ByNumber(protoref.FieldNumber(number))).List() +} + +func (p *RPCCompiler) newEmptyListMessageByName(msg *dynamicpb.Message, name string) protoref.List { + return msg.Mutable(msg.Descriptor().Fields().ByName(protoref.Name(name))).List() +} + +func (p *RPCCompiler) setMessageValue(message protoref.Message, fieldName string, value protoref.Value) { + message.Set(message.Descriptor().Fields().ByName(protoref.Name(fieldName)), value) +} + // buildProtoMessage recursively builds a protobuf message from an RPCMessage definition // and JSON data. It handles nested messages and repeated fields. -// TODO provide a way to have data func (p *RPCCompiler) buildProtoMessage(inputMessage Message, rpcMessage *RPCMessage, data gjson.Result) *dynamicpb.Message { if rpcMessage == nil { return nil } - if p.doc.MessageRefByName(inputMessage.Name) == -1 { + inputMessageRef := p.doc.MessageRefByName(inputMessage.Name) + if inputMessageRef == InvalidRef { p.report.AddInternalError(fmt.Errorf("message %s not found in document", inputMessage.Name)) return nil } @@ -424,7 +828,6 @@ func (p *RPCCompiler) buildProtoMessage(inputMessage Message, rpcMessage *RPCMes if field.Repeated { // Get a mutable reference to the list field list := message.Mutable(fd.ByName(protoref.Name(field.Name))).List() - // Extract the array elements from the JSON data elements := data.Get(rpcField.JSONPath).Array() if len(elements) == 0 { @@ -765,7 +1168,6 @@ func (p *RPCCompiler) enrichMessageData(ref int, m protoref.MessageDescriptor) { field := p.parseField(f) - // If the field is a message type, recursively parse the nested message if f.Kind() == protoref.MessageKind { // Handle nested messages when they are recursive types field.MessageRef = p.doc.MessageRefByName(string(f.Message().Name())) @@ -775,7 +1177,6 @@ func (p *RPCCompiler) enrichMessageData(ref int, m protoref.MessageDescriptor) { } msg.Fields = fields - p.doc.Messages[ref] = msg } @@ -790,7 +1191,7 @@ func (p *RPCCompiler) parseField(f protoref.FieldDescriptor) Field { Number: int32(f.Number()), Repeated: f.IsList(), Optional: f.Cardinality() == protoref.Optional, - MessageRef: -1, + MessageRef: InvalidRef, } } diff --git a/v2/pkg/engine/datasource/grpc_datasource/compiler_test.go b/v2/pkg/engine/datasource/grpc_datasource/compiler_test.go index 17b46b102..231c75728 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/compiler_test.go +++ b/v2/pkg/engine/datasource/grpc_datasource/compiler_test.go @@ -285,12 +285,12 @@ func TestBuildProtoMessage(t *testing.T) { // Compile the execution plan with the variables // This should build a protobuf message ready to be sent to the gRPC service - invocations, err := compiler.Compile(executionPlan, gjson.ParseBytes(variables)) + serviceCalls, err := compiler.Compile(executionPlan, gjson.ParseBytes(variables)) if err != nil { t.Fatalf("failed to compile proto: %v", err) } - require.Equal(t, 1, len(invocations)) + require.Equal(t, 1, len(serviceCalls)) } @@ -400,11 +400,11 @@ func TestCompileNestedLists(t *testing.T) { }, } - invocations, err := compiler.Compile(plan, gjson.ParseBytes([]byte(`{"orders":[{"orderId":"123","customerName":"John Doe","lines":[{"productId":"123","quantity":1, "modifiers":["modifier1", "modifier2"]}]}]}`))) + serviceCalls, err := compiler.Compile(plan, gjson.ParseBytes([]byte(`{"orders":[{"orderId":"123","customerName":"John Doe","lines":[{"productId":"123","quantity":1, "modifiers":["modifier1", "modifier2"]}]}]}`))) require.NoError(t, err) - require.Equal(t, 1, len(invocations)) + require.Equal(t, 1, len(serviceCalls)) - proto := invocations[0].Input.ProtoReflect() + proto := serviceCalls[0].Input.ProtoReflect() msgDesc := proto.Descriptor() diff --git a/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go b/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go index f5d822e22..323461575 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go +++ b/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go @@ -53,9 +53,24 @@ type RPCExecutionPlan struct { // instead of the planner and the compiler? } +// CallKind is the type of call operation to perform. +type CallKind int + +const ( + // CallKindStandard is a basic fetch operation. + CallKindStandard CallKind = iota + // CallKindEntity is a fetch operation for entities. + CallKindEntity + // CallKindResolve is a fetch operation for resolving field values. + CallKindResolve +) + // RPCCall represents a single call to a gRPC service method. // It contains all the information needed to make the call and process the response. type RPCCall struct { + // Kind of call, used to decide how to execute the call + // This is used to identify the call type and execution behaviour. + Kind CallKind // DependentCalls is a list of calls that must be executed before this call DependentCalls []int // ServiceName is the name of the gRPC service to call @@ -66,6 +81,8 @@ type RPCCall struct { Request RPCMessage // Response contains the message structure for the gRPC response Response RPCMessage + // ResponsePath is the path to the response in the JSON response + ResponsePath ast.Path } // RPCMessage represents a gRPC message structure for requests and responses. @@ -312,22 +329,19 @@ func NewPlanner(subgraphName string, mapping *GRPCMapping, federationConfigs pla mapping = new(GRPCMapping) } - var visitor PlanVisitor if len(federationConfigs) > 0 { - visitor = newRPCPlanVisitorFederation(rpcPlanVisitorConfig{ - subgraphName: subgraphName, - mapping: mapping, - federationConfigs: federationConfigs, - }) - } else { - visitor = newRPCPlanVisitor(rpcPlanVisitorConfig{ + return newRPCPlanVisitorFederation(rpcPlanVisitorConfig{ subgraphName: subgraphName, mapping: mapping, federationConfigs: federationConfigs, }) } - return visitor + return newRPCPlanVisitor(rpcPlanVisitorConfig{ + subgraphName: subgraphName, + mapping: mapping, + federationConfigs: federationConfigs, + }) } // formatRPCMessage formats an RPCMessage and adds it to the string builder with the specified indentation diff --git a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_field_resolvers_test.go b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_field_resolvers_test.go index 397513c65..b63db1fc5 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_field_resolvers_test.go +++ b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_field_resolvers_test.go @@ -5,6 +5,7 @@ import ( ) func TestExecutionPlanFieldResolvers(t *testing.T) { + t.Parallel() tests := []struct { name string query string @@ -59,80 +60,72 @@ func TestExecutionPlanFieldResolvers(t *testing.T) { DependentCalls: []int{0}, ServiceName: "Products", MethodName: "ResolveCategoryProductCount", + Kind: CallKindResolve, + ResponsePath: buildPath("categories.productCount"), Request: RPCMessage{ Name: "ResolveCategoryProductCountRequest", Fields: []RPCField{ { - Name: "key", + Name: "context", TypeName: string(DataTypeMessage), - JSONPath: "key", + JSONPath: "", Repeated: true, Message: &RPCMessage{ - Name: "ResolveCategoryProductCountRequestKey", + Name: "CategoryProductCountContext", Fields: []RPCField{ { - Name: "context", + Name: "id", + TypeName: string(DataTypeString), + JSONPath: "id", + ResolvePath: buildPath("categories.id"), + }, + { + Name: "name", + TypeName: string(DataTypeString), + JSONPath: "name", + ResolvePath: buildPath("categories.name"), + }, + }, + }, + }, + { + Name: "field_args", + TypeName: string(DataTypeMessage), + JSONPath: "", + Message: &RPCMessage{ + Name: "CategoryProductCountArgs", + Fields: []RPCField{ + { + Name: "filters", TypeName: string(DataTypeMessage), - JSONPath: "", + JSONPath: "whoop", + Optional: true, Message: &RPCMessage{ - Name: "CategoryProductCountContext", + Name: "ProductCountFilter", Fields: []RPCField{ { - Name: "id", - TypeName: string(DataTypeString), - JSONPath: "id", - ResolvePath: buildPath("categories.id"), + Name: "min_price", + TypeName: string(DataTypeDouble), + JSONPath: "minPrice", + Optional: true, }, { - Name: "name", - TypeName: string(DataTypeString), - JSONPath: "name", - ResolvePath: buildPath("categories.name"), + Name: "max_price", + TypeName: string(DataTypeDouble), + JSONPath: "maxPrice", + Optional: true, + }, + { + Name: "in_stock", + TypeName: string(DataTypeBool), + JSONPath: "inStock", + Optional: true, }, - }, - }, - }, - { - Name: "field_args", - TypeName: string(DataTypeMessage), - JSONPath: "", - Message: &RPCMessage{ - Name: "CategoryProductCountArgs", - Fields: []RPCField{ { - Name: "filters", - TypeName: string(DataTypeMessage), - JSONPath: "whoop", + Name: "search_term", + TypeName: string(DataTypeString), + JSONPath: "searchTerm", Optional: true, - Message: &RPCMessage{ - Name: "ProductCountFilter", - Fields: []RPCField{ - { - Name: "min_price", - TypeName: string(DataTypeDouble), - JSONPath: "minPrice", - Optional: true, - }, - { - Name: "max_price", - TypeName: string(DataTypeDouble), - JSONPath: "maxPrice", - Optional: true, - }, - { - Name: "in_stock", - TypeName: string(DataTypeBool), - JSONPath: "inStock", - Optional: true, - }, - { - Name: "search_term", - TypeName: string(DataTypeString), - JSONPath: "searchTerm", - Optional: true, - }, - }, - }, }, }, }, diff --git a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor.go b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor.go index 4c44733da..7692ad74b 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor.go +++ b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor.go @@ -41,7 +41,7 @@ type fieldArgument struct { type resolvedField struct { callerRef int parentTypeRef int - fieldRef int + responsePath ast.Path contextFields []contextField fieldArguments []fieldArgument @@ -135,31 +135,25 @@ func (r *rpcPlanVisitor) LeaveDocument(_, _ *ast.Document) { // Base resolve call can be templated in plan context. call := RPCCall{ DependentCalls: []int{resolvedField.callerRef}, + ResponsePath: resolvedField.responsePath, ServiceName: r.planCtx.resolveServiceName(r.subgraphName), MethodName: "ResolveCategoryProductCount", + Kind: CallKindResolve, Request: RPCMessage{ Name: "ResolveCategoryProductCountRequest", Fields: RPCFields{ { - Name: "key", + Name: "context", TypeName: string(DataTypeMessage), - JSONPath: "key", + JSONPath: "", Repeated: true, - Message: &RPCMessage{ - Name: "ResolveCategoryProductCountRequestKey", - Fields: RPCFields{ - { - Name: "context", - TypeName: string(DataTypeMessage), - Message: contextMessage, - }, - { - Name: "field_args", - TypeName: string(DataTypeMessage), - Message: fieldArgsMessage, - }, - }, - }, + Message: contextMessage, + }, + { + Name: "field_args", + TypeName: string(DataTypeMessage), + JSONPath: "", + Message: fieldArgsMessage, }, }, }, @@ -446,7 +440,10 @@ func (r *rpcPlanVisitor) EnterField(ref int) { resolvedField := resolvedField{ callerRef: r.relatedCallID, parentTypeRef: r.walker.EnclosingTypeDefinition.Ref, - fieldRef: fd, + responsePath: r.walker.Path[1:].WithoutInlineFragmentNames().WithPathElement(ast.PathItem{ + Kind: ast.FieldName, + FieldName: r.operation.FieldAliasOrNameBytes(ref), + }), } contextFields, err := r.planCtx.resolveContextFields(r.walker, fd) diff --git a/v2/pkg/engine/datasource/grpc_datasource/fetch.go b/v2/pkg/engine/datasource/grpc_datasource/fetch.go new file mode 100644 index 000000000..19d8edf75 --- /dev/null +++ b/v2/pkg/engine/datasource/grpc_datasource/fetch.go @@ -0,0 +1,144 @@ +package grpcdatasource + +import ( + "fmt" + "sync" +) + +type FetchItem struct { + ID int + Plan *RPCCall + ServiceCall *ServiceCall + DependentFetches []int +} + +type DependencyGraph struct { + mu sync.Mutex + fetches []FetchItem + // nodes is a list of lists of dependent calls. + // Each node index corresponds to a call index in the execution plan + // and the list contains the corresponding dependent calls indices. + nodes [][]int +} + +func NewDependencyGraph(executionPlan *RPCExecutionPlan) *DependencyGraph { + graph := &DependencyGraph{ + nodes: make([][]int, len(executionPlan.Calls)), + fetches: make([]FetchItem, len(executionPlan.Calls)), + } + + for index, call := range executionPlan.Calls { + graph.nodes[index] = call.DependentCalls + graph.fetches[index] = FetchItem{ + ID: index, + Plan: &call, + ServiceCall: nil, + DependentFetches: call.DependentCalls, + } + } + + return graph +} + +func (g *DependencyGraph) TopologicalSortResolve(resolver func(nodes []FetchItem) error) error { + // We are using a slice to store the batch index for each noded ordered. + callHierarchyRefs := initializeSlice(len(g.nodes), -1) + cycleChecks := make([]bool, len(g.nodes)) + + var visit func(index int) error + visit = func(index int) error { + if cycleChecks[index] { + return fmt.Errorf("cycle detected") + } + + cycleChecks[index] = true + + if len(g.nodes[index]) == 0 { + callHierarchyRefs[index] = 0 + return nil + } + + currentLevel := 0 + for _, dep := range g.nodes[index] { + if dep < 0 || dep >= len(g.nodes) { + return fmt.Errorf("unable to find dependent call %d in execution plan", dep) + } + + if depLevel := callHierarchyRefs[dep]; depLevel >= 0 { + if depLevel > currentLevel { + currentLevel = depLevel + } + continue + } + + if err := visit(dep); err != nil { + return err + } + + if l := callHierarchyRefs[dep]; l > currentLevel { + currentLevel = l + } + } + + callHierarchyRefs[index] = currentLevel + 1 + return nil + } + + for node := range g.nodes { + if err := visit(node); err != nil { + return err + } + + clear(cycleChecks) + } + + chunks := make(map[int][]FetchItem) + + for callIndex, chunkIndex := range callHierarchyRefs { + chunks[chunkIndex] = append(chunks[chunkIndex], g.fetches[callIndex]) + } + + for i := 0; i < len(chunks); i++ { + if err := resolver(chunks[i]); err != nil { + return err + } + } + + return nil +} + +func (g *DependencyGraph) Fetch(index int) (FetchItem, error) { + if index < 0 || index >= len(g.fetches) { + return FetchItem{}, fmt.Errorf("unable to find fetch %d in execution plan", index) + } + + return g.fetches[index], nil +} + +func (g *DependencyGraph) FetchDependencies(fetch *FetchItem) ([]FetchItem, error) { + dependencies := make([]FetchItem, 0, len(fetch.DependentFetches)) + + for _, dependentFetch := range fetch.DependentFetches { + dependency, err := g.Fetch(dependentFetch) + if err != nil { + return nil, err + } + dependencies = append(dependencies, dependency) + } + + return dependencies, nil +} + +func (g *DependencyGraph) SetFetchData(index int, serviceCall *ServiceCall) { + g.mu.Lock() + g.fetches[index].ServiceCall = serviceCall + g.mu.Unlock() +} + +func initializeSlice[T any](len int, zero T) []T { + s := make([]T, len) + for i := range s { + s[i] = zero + } + return s +} diff --git a/v2/pkg/engine/datasource/grpc_datasource/fetch_test.go b/v2/pkg/engine/datasource/grpc_datasource/fetch_test.go new file mode 100644 index 000000000..8b9cefd6b --- /dev/null +++ b/v2/pkg/engine/datasource/grpc_datasource/fetch_test.go @@ -0,0 +1,178 @@ +package grpcdatasource + +import ( + "testing" + + "github.com/stretchr/testify/require" +) + +func BenchmarkBuildDependencyGraph(b *testing.B) { + executionPlan := &RPCExecutionPlan{ + Calls: []RPCCall{ + { + Kind: CallKindStandard, + DependentCalls: []int{1}, + MethodName: "Method1", + }, + { + Kind: CallKindStandard, + MethodName: "Method2", + }, + { + Kind: CallKindStandard, + DependentCalls: []int{0}, + MethodName: "Method3", + }, + { + Kind: CallKindStandard, + DependentCalls: []int{0}, + MethodName: "Method4", + }, + { + Kind: CallKindStandard, + DependentCalls: []int{0, 2}, + MethodName: "Method5", + }, + { + Kind: CallKindStandard, + MethodName: "Method6", + }, + }, + } + graph := NewDependencyGraph(executionPlan) + for b.Loop() { + _ = graph.TopologicalSortResolve(func(nodes []FetchItem) error { + return nil + }) + } +} + +func TestBuildDependencyGraph(t *testing.T) { + t.Parallel() + t.Run("Simple execution plan with single root dependencies", func(t *testing.T) { + t.Parallel() + executionPlan := &RPCExecutionPlan{ + Calls: []RPCCall{ + { + Kind: CallKindStandard, + DependentCalls: []int{1}, + MethodName: "Method1", + }, + { + Kind: CallKindStandard, + MethodName: "Method2", + }, + { + Kind: CallKindStandard, + DependentCalls: []int{0}, + MethodName: "Method3", + }, + { + Kind: CallKindStandard, + DependentCalls: []int{0}, + MethodName: "Method4", + }, + { + Kind: CallKindStandard, + DependentCalls: []int{0, 2}, + MethodName: "Method5", + }, + { + Kind: CallKindStandard, + MethodName: "Method6", + }, + }, + } + + graph := NewDependencyGraph(executionPlan) + require.Equal(t, 6, len(graph.nodes)) + + result := make([]FetchItem, 0) + err := graph.TopologicalSortResolve(func(nodes []FetchItem) error { + result = append(result, nodes...) + return nil + }) + require.NoError(t, err) + require.Equal(t, []FetchItem{graph.fetches[1], graph.fetches[5], graph.fetches[0], graph.fetches[2], graph.fetches[3], graph.fetches[4]}, result) + }) + + t.Run("Should resolve the nodes in the correct order", func(t *testing.T) { + t.Parallel() + executionPlan := &RPCExecutionPlan{ + Calls: []RPCCall{ + { + Kind: CallKindStandard, + DependentCalls: []int{1}, + }, + { + Kind: CallKindStandard, + }, + }, + } + + graph := NewDependencyGraph(executionPlan) + require.Equal(t, 2, len(graph.nodes)) + + result := make([]FetchItem, 0) + + err := graph.TopologicalSortResolve(func(nodes []FetchItem) error { + result = append(result, nodes...) + return nil + }) + + require.Equal(t, []FetchItem{graph.fetches[1], graph.fetches[0]}, result) + require.NoError(t, err) + }) + + t.Run("Should raise error if there is a cycle", func(t *testing.T) { + t.Parallel() + executionPlan := &RPCExecutionPlan{ + Calls: []RPCCall{ + { + Kind: CallKindStandard, + DependentCalls: []int{1}, + }, + { + Kind: CallKindStandard, + DependentCalls: []int{0}, + }, + }, + } + + graph := NewDependencyGraph(executionPlan) + require.Equal(t, 2, len(graph.nodes)) + require.Equal(t, []int{1}, graph.nodes[0]) + require.Equal(t, []int{0}, graph.nodes[1]) + + err := graph.TopologicalSortResolve(func(nodes []FetchItem) error { + return nil + }) + require.Error(t, err) + require.Contains(t, err.Error(), "cycle detected") + }) + + t.Run("Should raise an error if the execution plan is missing a call", func(t *testing.T) { + t.Parallel() + executionPlan := &RPCExecutionPlan{ + Calls: []RPCCall{ + { + Kind: CallKindStandard, + DependentCalls: []int{1}, + }, + { + Kind: CallKindStandard, + DependentCalls: []int{2}, + }, + }, + } + + graph := NewDependencyGraph(executionPlan) + require.Equal(t, 2, len(graph.nodes)) + + err := graph.TopologicalSortResolve(func(nodes []FetchItem) error { + return nil + }) + require.Error(t, err) + require.Contains(t, err.Error(), "unable to find dependent call 2 in execution plan") + }) +} diff --git a/v2/pkg/engine/datasource/grpc_datasource/grpc_datasource.go b/v2/pkg/engine/datasource/grpc_datasource/grpc_datasource.go index 4d9babc60..acb428ec2 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/grpc_datasource.go +++ b/v2/pkg/engine/datasource/grpc_datasource/grpc_datasource.go @@ -31,8 +31,8 @@ var _ resolve.DataSource = (*DataSource)(nil) // It handles the conversion of GraphQL queries to gRPC requests and // transforms the responses back to GraphQL format. type DataSource struct { - // Invocations is a list of gRPC invocations to be executed plan *RPCExecutionPlan + graph *DependencyGraph cc grpc.ClientConnInterface rc *RPCCompiler mapping *GRPCMapping @@ -64,6 +64,7 @@ func NewDataSource(client grpc.ClientConnInterface, config DataSourceConfig) (*D return &DataSource{ plan: plan, + graph: NewDependencyGraph(plan), cc: client, rc: config.Compiler, mapping: config.Mapping, @@ -88,66 +89,78 @@ func (d *DataSource) Load(ctx context.Context, input []byte, out *bytes.Buffer) return nil } - // get invocations from plan - invocations, err := d.rc.Compile(d.plan, variables) - if err != nil { - return err - } - - responses := make([]*astjson.Value, len(invocations)) - errGrp, errGrpCtx := errgroup.WithContext(ctx) + arena := astjson.Arena{} + root := arena.NewObject() - mu := sync.Mutex{} - // make gRPC calls - for index, invocation := range invocations { - errGrp.Go(func() error { - a := astjson.Arena{} - // Invoke the gRPC method - this will populate invocation.Output - methodName := fmt.Sprintf("/%s/%s", invocation.ServiceName, invocation.MethodName) + failed := false - err := d.cc.Invoke(errGrpCtx, methodName, invocation.Input, invocation.Output) - if err != nil { - return err - } + if err := d.graph.TopologicalSortResolve(func(nodes []FetchItem) error { + serviceCalls, err := d.rc.CompileFetches(d.graph, nodes, variables) + if err != nil { + return err + } - mu.Lock() - defer mu.Unlock() + responses := make([]*astjson.Value, len(serviceCalls)) + errGrp, errGrpCtx := errgroup.WithContext(ctx) + mu := sync.Mutex{} + + // make gRPC calls + for index, serviceCall := range serviceCalls { + errGrp.Go(func() error { + a := astjson.Arena{} + // Invoke the gRPC method - this will populate serviceCall.Output + methodName := fmt.Sprintf("/%s/%s", serviceCall.ServiceName, serviceCall.MethodName) + + err := d.cc.Invoke(errGrpCtx, methodName, serviceCall.Input, serviceCall.Output) + if err != nil { + return err + } + + response, err := builder.marshalResponseJSON(&a, &serviceCall.Call.Response, serviceCall.Output) + if err != nil { + return nil + } + + if serviceCall.Call.Kind == CallKindResolve { + return builder.mergeWithPath(root, response, serviceCall.Call.ResponsePath) + } + + // In case of a federated response, we need to ensure that the response is valid. + // The number of entities per type must match the number of lookup keys in the variablese + err = builder.validateFederatedResponse(response) + if err != nil { + return err + } + + mu.Lock() + responses[index] = response + mu.Unlock() + + return nil + }) + } - response, err := builder.marshalResponseJSON(&a, &invocation.Call.Response, invocation.Output) - if err != nil { - return err - } + if err := errGrp.Wait(); err != nil { + out.Write(builder.writeErrorBytes(err)) + failed = true + return nil + } - // In case of a federated response, we need to ensure that the response is valid. - // The number of entities per type must match the number of lookup keys in the variables. - err = builder.validateFederatedResponse(response) + for _, response := range responses { + root, err = builder.mergeValues(root, response) if err != nil { + out.Write(builder.writeErrorBytes(err)) return err } + } - responses[index] = response - return nil - }) - } - - if err := errGrp.Wait(); err != nil { - out.Write(builder.writeErrorBytes(err)) return nil - } - - a := astjson.Arena{} - root := a.NewObject() - for _, response := range responses { - root, err = builder.mergeValues(root, response) - if err != nil { - out.Write(builder.writeErrorBytes(err)) - return err - } + }); err != nil || failed { + return err } data := builder.toDataObject(root) out.Write(data.MarshalTo(nil)) - return nil } diff --git a/v2/pkg/engine/datasource/grpc_datasource/grpc_datasource_test.go b/v2/pkg/engine/datasource/grpc_datasource/grpc_datasource_test.go index 3ae711d51..c2feb5561 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/grpc_datasource_test.go +++ b/v2/pkg/engine/datasource/grpc_datasource/grpc_datasource_test.go @@ -3636,3 +3636,96 @@ func Test_DataSource_Load_WithEntity_Calls(t *testing.T) { }) } } + +func Test_Datasource_Load_WithFieldResolvers(t *testing.T) { + conn, cleanup := setupTestGRPCServer(t) + t.Cleanup(cleanup) + + type graphqlError struct { + Message string `json:"message"` + } + type graphqlResponse struct { + Data map[string]interface{} `json:"data"` + Errors []graphqlError `json:"errors,omitempty"` + } + + testCases := []struct { + name string + query string + vars string + federationConfigs plan.FederationFieldConfigurations + validate func(t *testing.T, data map[string]interface{}) + validateError func(t *testing.T, errData []graphqlError) + }{ + { + name: "Query with field resolvers", + query: `query CategoriesWithFieldResolvers($filters: ProductCountFilter) { categories { id name kind productCount(filters: $filters) } }`, + vars: `{"variables":{"filters":{"minPrice":100}}}`, + validate: func(t *testing.T, data map[string]interface{}) { + require.NotEmpty(t, data) + + categories, ok := data["categories"].([]interface{}) + require.True(t, ok, "categories should be an array") + require.NotEmpty(t, categories, "categories should not be empty") + require.Len(t, categories, 4, "Should return 1 category") + + for productCount, category := range categories { + category, ok := category.(map[string]interface{}) + require.True(t, ok, "category should be an object") + require.NotEmpty(t, category["id"]) + require.NotEmpty(t, category["name"]) + require.NotEmpty(t, category["kind"]) + require.Equal(t, float64(productCount), category["productCount"]) + } + + }, + validateError: func(t *testing.T, errData []graphqlError) { + require.Empty(t, errData) + }, + }, + } + + for _, tc := range testCases { + t.Run(tc.name, func(t *testing.T) { + // Parse the GraphQL schema + schemaDoc := grpctest.MustGraphQLSchema(t) + + // Parse the GraphQL query + queryDoc, report := astparser.ParseGraphqlDocumentString(tc.query) + if report.HasErrors() { + t.Fatalf("failed to parse query: %s", report.Error()) + } + + compiler, err := NewProtoCompiler(grpctest.MustProtoSchema(t), testMapping()) + if err != nil { + t.Fatalf("failed to compile proto: %v", err) + } + + // Create the datasource + ds, err := NewDataSource(conn, DataSourceConfig{ + Operation: &queryDoc, + Definition: &schemaDoc, + SubgraphName: "Products", + Mapping: testMapping(), + Compiler: compiler, + FederationConfigs: tc.federationConfigs, + }) + require.NoError(t, err) + + // Execute the query through our datasource + output := new(bytes.Buffer) + input := fmt.Sprintf(`{"query":%q,"body":%s}`, tc.query, tc.vars) + err = ds.Load(context.Background(), []byte(input), output) + require.NoError(t, err) + + // Parse the response + var resp graphqlResponse + + err = json.Unmarshal(output.Bytes(), &resp) + require.NoError(t, err, "Failed to unmarshal response") + + tc.validate(t, resp.Data) + tc.validateError(t, resp.Errors) + }) + } +} diff --git a/v2/pkg/engine/datasource/grpc_datasource/json_builder.go b/v2/pkg/engine/datasource/grpc_datasource/json_builder.go index 7c1fc81d7..cf7dc3da2 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/json_builder.go +++ b/v2/pkg/engine/datasource/grpc_datasource/json_builder.go @@ -11,13 +11,16 @@ import ( protoref "google.golang.org/protobuf/reflect/protoreflect" "github.com/wundergraph/astjson" + + "github.com/wundergraph/graphql-go-tools/v2/pkg/ast" ) // Standard GraphQL response paths -var ( - entityPath = "_entities" // Path for federated entities in response - dataPath = "data" // Standard GraphQL data wrapper - errorsPath = "errors" // Standard GraphQL errors array +const ( + entityPath = "_entities" // Path for federated entities in response + dataPath = "data" // Standard GraphQL data wrapper + errorsPath = "errors" // Standard GraphQL errors array + resolvResponsePath = "result" // Path for resolve response ) // entityIndex represents the mapping between representation order and result order @@ -217,6 +220,87 @@ func (j *jsonBuilder) mergeEntities(left *astjson.Value, right *astjson.Value) ( return entities, nil } +func (j *jsonBuilder) mergeWithPath(base *astjson.Value, resolved *astjson.Value, path ast.Path) error { + resolvedValues := resolved.GetArray(resolvResponsePath) + + searchPath := path[:len(path)-1] + elementName := path[len(path)-1].FieldName.String() + + responseValues := make([]*astjson.Value, 0, len(resolvedValues)) + + current := base + current = current.Get(searchPath[0].FieldName.String()) + switch current.Type() { + case astjson.TypeArray: + arr := current.GetArray() + values, err := j.flattenList(arr, searchPath[1:]) + if err != nil { + return err + } + responseValues = append(responseValues, values...) + default: + values, err := j.flattenObject(current, searchPath[1:]) + if err != nil { + return err + } + responseValues = append(responseValues, values...) + } + + if len(resolvedValues) < len(responseValues) { + return fmt.Errorf("length of values doesn't match the length of the result array, expected %d, got %d", len(responseValues), len(resolvedValues)) + } + + for i := range responseValues { + responseValues[i].Set(elementName, resolvedValues[i].Get(elementName)) + } + + return nil +} + +func (j *jsonBuilder) flattenObject(value *astjson.Value, path ast.Path) ([]*astjson.Value, error) { + if path.Len() == 0 { + return []*astjson.Value{value}, nil + } + + current := value + result := make([]*astjson.Value, 0) + switch current.Type() { + case astjson.TypeObject: + values, err := j.flattenObject(current, path[1:]) + if err != nil { + return nil, err + } + result = append(result, values...) + case astjson.TypeArray: + values, err := j.flattenList(current.GetArray(), path[1:]) + if err != nil { + return nil, err + } + result = append(result, values...) + default: + return nil, fmt.Errorf("expected array or object, got %s", current.Type()) + } + + return result, nil +} + +func (j *jsonBuilder) flattenList(items []*astjson.Value, path ast.Path) ([]*astjson.Value, error) { + if path.Len() == 0 { + return items, nil + } + + result := make([]*astjson.Value, 0) + for _, item := range items { + values, err := j.flattenObject(item, path) + if err != nil { + return nil, err + } + result = append(result, values...) + } + + return result, nil +} + // marshalResponseJSON converts a protobuf message into a GraphQL-compatible JSON response. // This is the core marshaling function that handles all the complex type conversions, // including oneOf types, nested messages, lists, and scalar values. diff --git a/v2/pkg/engine/datasource/grpc_datasource/mapping_test_helper.go b/v2/pkg/engine/datasource/grpc_datasource/mapping_test_helper.go index 057ffe204..24ec4247b 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/mapping_test_helper.go +++ b/v2/pkg/engine/datasource/grpc_datasource/mapping_test_helper.go @@ -44,16 +44,6 @@ func testMapping() *GRPCMapping { Request: "QueryCalculateTotalsRequest", Response: "QueryCalculateTotalsResponse", }, - "randomPet": { - RPC: "QueryRandomPet", - Request: "QueryRandomPetRequest", - Response: "QueryRandomPetResponse", - }, - "allPets": { - RPC: "QueryAllPets", - Request: "QueryAllPetsRequest", - Response: "QueryAllPetsResponse", - }, "categories": { RPC: "QueryCategories", Request: "QueryCategoriesRequest", @@ -74,16 +64,26 @@ func testMapping() *GRPCMapping { Request: "QueryFilterCategoriesRequest", Response: "QueryFilterCategoriesResponse", }, - "randomSearchResult": { - RPC: "QueryRandomSearchResult", - Request: "QueryRandomSearchResultRequest", - Response: "QueryRandomSearchResultResponse", + "randomPet": { + RPC: "QueryRandomPet", + Request: "QueryRandomPetRequest", + Response: "QueryRandomPetResponse", + }, + "allPets": { + RPC: "QueryAllPets", + Request: "QueryAllPetsRequest", + Response: "QueryAllPetsResponse", }, "search": { RPC: "QuerySearch", Request: "QuerySearchRequest", Response: "QuerySearchResponse", }, + "randomSearchResult": { + RPC: "QueryRandomSearchResult", + Request: "QueryRandomSearchResultRequest", + Response: "QueryRandomSearchResultResponse", + }, "nullableFieldsType": { RPC: "QueryNullableFieldsType", Request: "QueryNullableFieldsTypeRequest", @@ -260,9 +260,12 @@ func testMapping() *GRPCMapping { }, Fields: map[string]FieldMap{ "Query": { + "users": { + TargetName: "users", + }, "user": { TargetName: "user", - ArgumentMappings: map[string]string{ + ArgumentMappings: FieldArgumentMap{ "id": "id", }, }, @@ -272,61 +275,61 @@ func testMapping() *GRPCMapping { "recursiveType": { TargetName: "recursive_type", }, - "randomPet": { - TargetName: "random_pet", + "typeFilterWithArguments": { + TargetName: "type_filter_with_arguments", + ArgumentMappings: FieldArgumentMap{ + "filterField1": "filter_field_1", + "filterField2": "filter_field_2", + }, }, - "allPets": { - TargetName: "all_pets", + "typeWithMultipleFilterFields": { + TargetName: "type_with_multiple_filter_fields", + ArgumentMappings: FieldArgumentMap{ + "filter": "filter", + }, + }, + "complexFilterType": { + TargetName: "complex_filter_type", + ArgumentMappings: FieldArgumentMap{ + "filter": "filter", + }, + }, + "calculateTotals": { + TargetName: "calculate_totals", + ArgumentMappings: FieldArgumentMap{ + "orders": "orders", + }, }, "categories": { TargetName: "categories", }, "categoriesByKind": { TargetName: "categories_by_kind", - ArgumentMappings: map[string]string{ + ArgumentMappings: FieldArgumentMap{ "kind": "kind", }, }, "categoriesByKinds": { TargetName: "categories_by_kinds", - ArgumentMappings: map[string]string{ + ArgumentMappings: FieldArgumentMap{ "kinds": "kinds", }, }, "filterCategories": { TargetName: "filter_categories", - ArgumentMappings: map[string]string{ + ArgumentMappings: FieldArgumentMap{ "filter": "filter", }, }, - "typeFilterWithArguments": { - TargetName: "type_filter_with_arguments", - ArgumentMappings: map[string]string{ - "filterField1": "filter_field_1", - "filterField2": "filter_field_2", - }, - }, - "typeWithMultipleFilterFields": { - TargetName: "type_with_multiple_filter_fields", - ArgumentMappings: map[string]string{ - "filter": "filter", - }, - }, - "complexFilterType": { - TargetName: "complex_filter_type", - ArgumentMappings: map[string]string{ - "filter": "filter", - }, + "randomPet": { + TargetName: "random_pet", }, - "calculateTotals": { - TargetName: "calculate_totals", - ArgumentMappings: map[string]string{ - "orders": "orders", - }, + "allPets": { + TargetName: "all_pets", }, "search": { TargetName: "search", - ArgumentMappings: map[string]string{ + ArgumentMappings: FieldArgumentMap{ "input": "input", }, }, @@ -338,13 +341,13 @@ func testMapping() *GRPCMapping { }, "nullableFieldsTypeById": { TargetName: "nullable_fields_type_by_id", - ArgumentMappings: map[string]string{ + ArgumentMappings: FieldArgumentMap{ "id": "id", }, }, "nullableFieldsTypeWithFilter": { TargetName: "nullable_fields_type_with_filter", - ArgumentMappings: map[string]string{ + ArgumentMappings: FieldArgumentMap{ "filter": "filter", }, }, @@ -356,13 +359,13 @@ func testMapping() *GRPCMapping { }, "blogPostById": { TargetName: "blog_post_by_id", - ArgumentMappings: map[string]string{ + ArgumentMappings: FieldArgumentMap{ "id": "id", }, }, "blogPostsWithFilter": { TargetName: "blog_posts_with_filter", - ArgumentMappings: map[string]string{ + ArgumentMappings: FieldArgumentMap{ "filter": "filter", }, }, @@ -374,13 +377,13 @@ func testMapping() *GRPCMapping { }, "authorById": { TargetName: "author_by_id", - ArgumentMappings: map[string]string{ + ArgumentMappings: FieldArgumentMap{ "id": "id", }, }, "authorsWithFilter": { TargetName: "authors_with_filter", - ArgumentMappings: map[string]string{ + ArgumentMappings: FieldArgumentMap{ "filter": "filter", }, }, @@ -389,13 +392,13 @@ func testMapping() *GRPCMapping { }, "bulkSearchAuthors": { TargetName: "bulk_search_authors", - ArgumentMappings: map[string]string{ + ArgumentMappings: FieldArgumentMap{ "filters": "filters", }, }, "bulkSearchBlogPosts": { TargetName: "bulk_search_blog_posts", - ArgumentMappings: map[string]string{ + ArgumentMappings: FieldArgumentMap{ "filters": "filters", }, }, @@ -403,85 +406,80 @@ func testMapping() *GRPCMapping { "Mutation": { "createUser": { TargetName: "create_user", - ArgumentMappings: map[string]string{ + ArgumentMappings: FieldArgumentMap{ "input": "input", }, }, "performAction": { TargetName: "perform_action", - ArgumentMappings: map[string]string{ + ArgumentMappings: FieldArgumentMap{ "input": "input", }, }, "createNullableFieldsType": { TargetName: "create_nullable_fields_type", - ArgumentMappings: map[string]string{ + ArgumentMappings: FieldArgumentMap{ "input": "input", }, }, "updateNullableFieldsType": { TargetName: "update_nullable_fields_type", - ArgumentMappings: map[string]string{ + ArgumentMappings: FieldArgumentMap{ "id": "id", "input": "input", }, }, "createBlogPost": { TargetName: "create_blog_post", - ArgumentMappings: map[string]string{ + ArgumentMappings: FieldArgumentMap{ "input": "input", }, }, "updateBlogPost": { TargetName: "update_blog_post", - ArgumentMappings: map[string]string{ + ArgumentMappings: FieldArgumentMap{ "id": "id", "input": "input", }, }, "createAuthor": { TargetName: "create_author", - ArgumentMappings: map[string]string{ + ArgumentMappings: FieldArgumentMap{ "input": "input", }, }, "updateAuthor": { TargetName: "update_author", - ArgumentMappings: map[string]string{ + ArgumentMappings: FieldArgumentMap{ "id": "id", "input": "input", }, }, "bulkCreateAuthors": { TargetName: "bulk_create_authors", - ArgumentMappings: map[string]string{ + ArgumentMappings: FieldArgumentMap{ "authors": "authors", }, }, "bulkUpdateAuthors": { TargetName: "bulk_update_authors", - ArgumentMappings: map[string]string{ + ArgumentMappings: FieldArgumentMap{ "authors": "authors", }, }, "bulkCreateBlogPosts": { TargetName: "bulk_create_blog_posts", - ArgumentMappings: map[string]string{ + ArgumentMappings: FieldArgumentMap{ "blogPosts": "blog_posts", }, }, "bulkUpdateBlogPosts": { TargetName: "bulk_update_blog_posts", - ArgumentMappings: map[string]string{ + ArgumentMappings: FieldArgumentMap{ "blogPosts": "blog_posts", }, }, }, - "UserInput": { - "name": { - TargetName: "name", - }, - }, "Product": { "id": { TargetName: "id", @@ -578,49 +576,21 @@ func testMapping() *GRPCMapping { TargetName: "filter_field_2", }, }, - "TypeWithComplexFilterInput": { - "id": { - TargetName: "id", - }, - "name": { - TargetName: "name", - }, - }, - "Cat": { - "id": { - TargetName: "id", - }, - "name": { - TargetName: "name", - }, - "kind": { - TargetName: "kind", + "FilterTypeInput": { + "filterField1": { + TargetName: "filter_field_1", }, - "meowVolume": { - TargetName: "meow_volume", + "filterField2": { + TargetName: "filter_field_2", }, }, - "Dog": { + "TypeWithComplexFilterInput": { "id": { TargetName: "id", }, "name": { TargetName: "name", }, - "kind": { - TargetName: "kind", - }, - "barkVolume": { - TargetName: "bark_volume", - }, - }, - "Animal": { - "cat": { - TargetName: "cat", - }, - "dog": { - TargetName: "dog", - }, }, "FilterType": { "name": { @@ -649,37 +619,26 @@ func testMapping() *GRPCMapping { TargetName: "filter", }, }, - "FilterTypeInput": { - "filterField1": { - TargetName: "filter_field_1", - }, - "filterField2": { - TargetName: "filter_field_2", - }, - }, - "Category": { - "id": { - TargetName: "id", - }, - "name": { - TargetName: "name", + "OrderLineInput": { + "productId": { + TargetName: "product_id", }, - "kind": { - TargetName: "kind", + "quantity": { + TargetName: "quantity", }, - "productCount": { - TargetName: "product_count", - ArgumentMappings: map[string]string{ - "filters": "filters", - }, + "modifiers": { + TargetName: "modifiers", }, }, - "CategoryFilter": { - "category": { - TargetName: "category", + "OrderInput": { + "orderId": { + TargetName: "order_id", }, - "pagination": { - TargetName: "pagination", + "customerName": { + TargetName: "customer_name", + }, + "lines": { + TargetName: "lines", }, }, "Order": { @@ -707,26 +666,57 @@ func testMapping() *GRPCMapping { TargetName: "modifiers", }, }, - "OrderInput": { - "orderId": { - TargetName: "order_id", + "CategoryFilter": { + "category": { + TargetName: "category", }, - "customerName": { - TargetName: "customer_name", + "pagination": { + TargetName: "pagination", }, - "lines": { - TargetName: "lines", + }, + "Category": { + "id": { + TargetName: "id", + }, + "name": { + TargetName: "name", + }, + "kind": { + TargetName: "kind", + }, + "productCount": { + TargetName: "product_count", + ArgumentMappings: FieldArgumentMap{ + "filters": "filters", + }, }, }, - "OrderLineInput": { - "productId": { - TargetName: "product_id", + "Cat": { + "id": { + TargetName: "id", }, - "quantity": { - TargetName: "quantity", + "name": { + TargetName: "name", }, - "modifiers": { - TargetName: "modifiers", + "kind": { + TargetName: "kind", + }, + "meowVolume": { + TargetName: "meow_volume", + }, + }, + "Dog": { + "id": { + TargetName: "id", + }, + "name": { + TargetName: "name", + }, + "kind": { + TargetName: "kind", + }, + "barkVolume": { + TargetName: "bark_volume", }, }, "ActionSuccess": { @@ -761,19 +751,6 @@ func testMapping() *GRPCMapping { TargetName: "payload", }, }, - "SearchResult": { - "product": { - TargetName: "product", - }, - }, - "ActionResult": { - "actionSuccess": { - TargetName: "action_success", - }, - "actionError": { - TargetName: "action_error", - }, - }, "NullableFieldsType": { "id": { TargetName: "id", @@ -800,40 +777,6 @@ func testMapping() *GRPCMapping { TargetName: "required_int", }, }, - "NullableFieldsInput": { - "name": { - TargetName: "name", - }, - "optionalString": { - TargetName: "optional_string", - }, - "optionalInt": { - TargetName: "optional_int", - }, - "optionalFloat": { - TargetName: "optional_float", - }, - "optionalBoolean": { - TargetName: "optional_boolean", - }, - "requiredString": { - TargetName: "required_string", - }, - "requiredInt": { - TargetName: "required_int", - }, - }, - "NullableFieldsFilter": { - "name": { - TargetName: "name", - }, - "optionalString": { - TargetName: "optional_string", - }, - "includeNulls": { - TargetName: "include_nulls", - }, - }, "BlogPost": { "id": { TargetName: "id", @@ -1047,6 +990,48 @@ func testMapping() *GRPCMapping { TargetName: "skill_count", }, }, + "NullableFieldsInput": { + "name": { + TargetName: "name", + }, + "optionalString": { + TargetName: "optional_string", + }, + "optionalInt": { + TargetName: "optional_int", + }, + "optionalFloat": { + TargetName: "optional_float", + }, + "optionalBoolean": { + TargetName: "optional_boolean", + }, + "requiredString": { + TargetName: "required_string", + }, + "requiredInt": { + TargetName: "required_int", + }, + }, + "NullableFieldsFilter": { + "name": { + TargetName: "name", + }, + "optionalString": { + TargetName: "optional_string", + }, + "includeNulls": { + TargetName: "include_nulls", + }, + }, + "CategoryInput": { + "name": { + TargetName: "name", + }, + "kind": { + TargetName: "kind", + }, + }, "ProductCountFilter": { "minPrice": { TargetName: "min_price", @@ -1061,6 +1046,11 @@ func testMapping() *GRPCMapping { TargetName: "search_term", }, }, + "UserInput": { + "name": { + TargetName: "name", + }, + }, }, } } diff --git a/v2/pkg/grpctest/cmd/mapping_helper/main.go b/v2/pkg/grpctest/cmd/mapping_helper/main.go new file mode 100644 index 000000000..1fea4f8d0 --- /dev/null +++ b/v2/pkg/grpctest/cmd/mapping_helper/main.go @@ -0,0 +1,191 @@ +package main + +import ( + "bytes" + "encoding/json" + "log" + "os" + "text/template" +) + +type JSONMapping struct { + Version int `json:"version"` + Service string `json:"service"` + OperationMappings []OperationMapping `json:"operationMappings"` + EntityMappings []EntityMapping `json:"entityMappings"` + TypeFieldMappings []TypeFieldMapping `json:"typeFieldMappings"` + EnumMappings []EnumMapping `json:"enumMappings"` +} + +type OperationMapping struct { + Type string `json:"type"` + Original string `json:"original"` + Mapped string `json:"mapped"` + Request string `json:"request"` + Response string `json:"response"` +} + +type EntityMapping struct { + TypeName string `json:"typeName"` + Kind string `json:"kind"` + Key string `json:"key"` + RPC string `json:"rpc"` + Request string `json:"request"` + Response string `json:"response"` +} + +type TypeFieldMapping struct { + Type string `json:"type"` + FieldMappings []FieldMapping `json:"fieldMappings"` +} + +type FieldMapping struct { + Original string `json:"original"` + Mapped string `json:"mapped"` + ArgumentMappings []ArgumentMapping `json:"argumentMappings"` +} + +type ArgumentMapping struct { + Original string `json:"original"` + Mapped string `json:"mapped"` +} + +type EnumMapping struct { + Type string `json:"type"` + Values []EnumValue `json:"values"` +} + +type EnumValue struct { + Original string `json:"original"` + Mapped string `json:"mapped"` +} + +const tpl = `package mapping + +import ( + "testing" + + grpcdatasource "github.com/wundergraph/graphql-go-tools/v2/pkg/engine/datasource/grpc_datasource" +) + +// DefaultGRPCMapping returns a hardcoded default mapping between GraphQL and Protobuf +func DefaultGRPCMapping() *grpcdatasource.GRPCMapping { + return &grpcdatasource.GRPCMapping{ + Service: "{{.Service}}", + QueryRPCs: map[string]grpcdatasource.RPCConfig{ + {{- range $index, $operation := .OperationMappings}} + {{- if eq $operation.Type "OPERATION_TYPE_QUERY"}} + "{{$operation.Original}}": { + RPC: "{{$operation.Mapped}}", + Request: "{{$operation.Request}}", + Response: "{{$operation.Response}}", + }, + {{- end }} + {{- end }} + }, + MutationRPCs: grpcdatasource.RPCConfigMap{ + {{- range $index, $operation := .OperationMappings}} + {{- if eq $operation.Type "OPERATION_TYPE_MUTATION"}} + "{{$operation.Original}}": { + RPC: "{{$operation.Mapped}}", + Request: "{{$operation.Request}}", + Response: "{{$operation.Response}}", + }, + {{- end }} + {{- end }} + }, + SubscriptionRPCs: grpcdatasource.RPCConfigMap{ + {{- range $index, $operation := .OperationMappings}} + {{- if eq $operation.Type "OPERATION_TYPE_SUBSCRIPTION"}} + "{{$operation.Original}}": { + RPC: "{{$operation.Mapped}}", + Request: "{{$operation.Request}}", + Response: "{{$operation.Response}}", + }, + {{- end }} + {{- end }} + }, + EntityRPCs: map[string][]grpcdatasource.EntityRPCConfig{ + {{- range $index, $entity := .EntityMappings}} + "{{$entity.TypeName}}": { + { + Key: "{{$entity.Key}}", + RPCConfig: grpcdatasource.RPCConfig{ + RPC: "{{$entity.RPC}}", + Request: "{{$entity.Request}}", + Response: "{{$entity.Response}}", + }, + }, + }, + {{- end }} + }, + EnumValues: map[string][]grpcdatasource.EnumValueMapping{ + {{- range $index, $enum := .EnumMappings}} + "{{$enum.Type}}": { + {{- range $index, $value := .Values}} + {Value: "{{$value.Original}}", TargetValue: "{{$value.Mapped}}"}, + {{- end }} + }, + {{- end }} + }, + Fields: map[string]grpcdatasource.FieldMap{ + {{- range $index, $typeField := .TypeFieldMappings}} + "{{$typeField.Type}}": { + {{- range $index, $field := $typeField.FieldMappings}} + "{{$field.Original}}": { + TargetName: "{{$field.Mapped}}", + {{- if (gt (len $field.ArgumentMappings) 0)}} + ArgumentMappings: grpcdatasource.FieldArgumentMap{ + {{- range $index, $argument := $field.ArgumentMappings}} + "{{$argument.Original}}": "{{$argument.Mapped}}", + {{- end }} + }, + {{- end }} + }, + {{- end}} + }, + {{- end}} + }, + } +} + + +// MustDefaultGRPCMapping returns the default GRPC mapping +func MustDefaultGRPCMapping(t *testing.T) *grpcdatasource.GRPCMapping { + mapping := DefaultGRPCMapping() + return mapping +} + + +` + +func main() { + args := os.Args[1:] + if len(args) < 2 { + log.Fatal("No input file or output file provided") + } + + inputFile := args[0] + outputFile := args[1] + + jsonBytes, err := os.ReadFile(inputFile) + if err != nil { + log.Fatal(err) + } + var mapping JSONMapping + err = json.Unmarshal(jsonBytes, &mapping) + if err != nil { + log.Fatal(err) + } + + t := template.Must(template.New("mapping").Parse(tpl)) + + buf := &bytes.Buffer{} + if err := t.Execute(buf, mapping); err != nil { + log.Fatal(err) + } + + if err := os.WriteFile(outputFile, buf.Bytes(), 0644); err != nil { + log.Fatal(err) + } +} diff --git a/v2/pkg/grpctest/mapping/mapping.go b/v2/pkg/grpctest/mapping/mapping.go index f4561a51a..b62ad84fb 100644 --- a/v2/pkg/grpctest/mapping/mapping.go +++ b/v2/pkg/grpctest/mapping/mapping.go @@ -51,16 +51,6 @@ func DefaultGRPCMapping() *grpcdatasource.GRPCMapping { Request: "QueryCalculateTotalsRequest", Response: "QueryCalculateTotalsResponse", }, - "randomPet": { - RPC: "QueryRandomPet", - Request: "QueryRandomPetRequest", - Response: "QueryRandomPetResponse", - }, - "allPets": { - RPC: "QueryAllPets", - Request: "QueryAllPetsRequest", - Response: "QueryAllPetsResponse", - }, "categories": { RPC: "QueryCategories", Request: "QueryCategoriesRequest", @@ -81,16 +71,26 @@ func DefaultGRPCMapping() *grpcdatasource.GRPCMapping { Request: "QueryFilterCategoriesRequest", Response: "QueryFilterCategoriesResponse", }, - "randomSearchResult": { - RPC: "QueryRandomSearchResult", - Request: "QueryRandomSearchResultRequest", - Response: "QueryRandomSearchResultResponse", + "randomPet": { + RPC: "QueryRandomPet", + Request: "QueryRandomPetRequest", + Response: "QueryRandomPetResponse", + }, + "allPets": { + RPC: "QueryAllPets", + Request: "QueryAllPetsRequest", + Response: "QueryAllPetsResponse", }, "search": { RPC: "QuerySearch", Request: "QuerySearchRequest", Response: "QuerySearchResponse", }, + "randomSearchResult": { + RPC: "QueryRandomSearchResult", + Request: "QueryRandomSearchResultRequest", + Response: "QueryRandomSearchResultResponse", + }, "nullableFieldsType": { RPC: "QueryNullableFieldsType", Request: "QueryNullableFieldsTypeRequest", @@ -267,9 +267,12 @@ func DefaultGRPCMapping() *grpcdatasource.GRPCMapping { }, Fields: map[string]grpcdatasource.FieldMap{ "Query": { + "users": { + TargetName: "users", + }, "user": { TargetName: "user", - ArgumentMappings: map[string]string{ + ArgumentMappings: grpcdatasource.FieldArgumentMap{ "id": "id", }, }, @@ -279,61 +282,61 @@ func DefaultGRPCMapping() *grpcdatasource.GRPCMapping { "recursiveType": { TargetName: "recursive_type", }, - "randomPet": { - TargetName: "random_pet", + "typeFilterWithArguments": { + TargetName: "type_filter_with_arguments", + ArgumentMappings: grpcdatasource.FieldArgumentMap{ + "filterField1": "filter_field_1", + "filterField2": "filter_field_2", + }, }, - "allPets": { - TargetName: "all_pets", + "typeWithMultipleFilterFields": { + TargetName: "type_with_multiple_filter_fields", + ArgumentMappings: grpcdatasource.FieldArgumentMap{ + "filter": "filter", + }, + }, + "complexFilterType": { + TargetName: "complex_filter_type", + ArgumentMappings: grpcdatasource.FieldArgumentMap{ + "filter": "filter", + }, + }, + "calculateTotals": { + TargetName: "calculate_totals", + ArgumentMappings: grpcdatasource.FieldArgumentMap{ + "orders": "orders", + }, }, "categories": { TargetName: "categories", }, "categoriesByKind": { TargetName: "categories_by_kind", - ArgumentMappings: map[string]string{ + ArgumentMappings: grpcdatasource.FieldArgumentMap{ "kind": "kind", }, }, "categoriesByKinds": { TargetName: "categories_by_kinds", - ArgumentMappings: map[string]string{ + ArgumentMappings: grpcdatasource.FieldArgumentMap{ "kinds": "kinds", }, }, "filterCategories": { TargetName: "filter_categories", - ArgumentMappings: map[string]string{ - "filter": "filter", - }, - }, - "typeFilterWithArguments": { - TargetName: "type_filter_with_arguments", - ArgumentMappings: map[string]string{ - "filterField1": "filter_field_1", - "filterField2": "filter_field_2", - }, - }, - "typeWithMultipleFilterFields": { - TargetName: "type_with_multiple_filter_fields", - ArgumentMappings: map[string]string{ + ArgumentMappings: grpcdatasource.FieldArgumentMap{ "filter": "filter", }, }, - "complexFilterType": { - TargetName: "complex_filter_type", - ArgumentMappings: map[string]string{ - "filter": "filter", - }, + "randomPet": { + TargetName: "random_pet", }, - "calculateTotals": { - TargetName: "calculate_totals", - ArgumentMappings: map[string]string{ - "orders": "orders", - }, + "allPets": { + TargetName: "all_pets", }, "search": { TargetName: "search", - ArgumentMappings: map[string]string{ + ArgumentMappings: grpcdatasource.FieldArgumentMap{ "input": "input", }, }, @@ -345,13 +348,13 @@ func DefaultGRPCMapping() *grpcdatasource.GRPCMapping { }, "nullableFieldsTypeById": { TargetName: "nullable_fields_type_by_id", - ArgumentMappings: map[string]string{ + ArgumentMappings: grpcdatasource.FieldArgumentMap{ "id": "id", }, }, "nullableFieldsTypeWithFilter": { TargetName: "nullable_fields_type_with_filter", - ArgumentMappings: map[string]string{ + ArgumentMappings: grpcdatasource.FieldArgumentMap{ "filter": "filter", }, }, @@ -363,13 +366,13 @@ func DefaultGRPCMapping() *grpcdatasource.GRPCMapping { }, "blogPostById": { TargetName: "blog_post_by_id", - ArgumentMappings: map[string]string{ + ArgumentMappings: grpcdatasource.FieldArgumentMap{ "id": "id", }, }, "blogPostsWithFilter": { TargetName: "blog_posts_with_filter", - ArgumentMappings: map[string]string{ + ArgumentMappings: grpcdatasource.FieldArgumentMap{ "filter": "filter", }, }, @@ -381,13 +384,13 @@ func DefaultGRPCMapping() *grpcdatasource.GRPCMapping { }, "authorById": { TargetName: "author_by_id", - ArgumentMappings: map[string]string{ + ArgumentMappings: grpcdatasource.FieldArgumentMap{ "id": "id", }, }, "authorsWithFilter": { TargetName: "authors_with_filter", - ArgumentMappings: map[string]string{ + ArgumentMappings: grpcdatasource.FieldArgumentMap{ "filter": "filter", }, }, @@ -396,13 +399,13 @@ func DefaultGRPCMapping() *grpcdatasource.GRPCMapping { }, "bulkSearchAuthors": { TargetName: "bulk_search_authors", - ArgumentMappings: map[string]string{ + ArgumentMappings: grpcdatasource.FieldArgumentMap{ "filters": "filters", }, }, "bulkSearchBlogPosts": { TargetName: "bulk_search_blog_posts", - ArgumentMappings: map[string]string{ + ArgumentMappings: grpcdatasource.FieldArgumentMap{ "filters": "filters", }, }, @@ -410,85 +413,80 @@ func DefaultGRPCMapping() *grpcdatasource.GRPCMapping { "Mutation": { "createUser": { TargetName: "create_user", - ArgumentMappings: map[string]string{ + ArgumentMappings: grpcdatasource.FieldArgumentMap{ "input": "input", }, }, "performAction": { TargetName: "perform_action", - ArgumentMappings: map[string]string{ + ArgumentMappings: grpcdatasource.FieldArgumentMap{ "input": "input", }, }, "createNullableFieldsType": { TargetName: "create_nullable_fields_type", - ArgumentMappings: map[string]string{ + ArgumentMappings: grpcdatasource.FieldArgumentMap{ "input": "input", }, }, "updateNullableFieldsType": { TargetName: "update_nullable_fields_type", - ArgumentMappings: map[string]string{ + ArgumentMappings: grpcdatasource.FieldArgumentMap{ "id": "id", "input": "input", }, }, "createBlogPost": { TargetName: "create_blog_post", - ArgumentMappings: map[string]string{ + ArgumentMappings: grpcdatasource.FieldArgumentMap{ "input": "input", }, }, "updateBlogPost": { TargetName: "update_blog_post", - ArgumentMappings: map[string]string{ + ArgumentMappings: grpcdatasource.FieldArgumentMap{ "id": "id", "input": "input", }, }, "createAuthor": { TargetName: "create_author", - ArgumentMappings: map[string]string{ + ArgumentMappings: grpcdatasource.FieldArgumentMap{ "input": "input", }, }, "updateAuthor": { TargetName: "update_author", - ArgumentMappings: map[string]string{ + ArgumentMappings: grpcdatasource.FieldArgumentMap{ "id": "id", "input": "input", }, }, "bulkCreateAuthors": { TargetName: "bulk_create_authors", - ArgumentMappings: map[string]string{ + ArgumentMappings: grpcdatasource.FieldArgumentMap{ "authors": "authors", }, }, "bulkUpdateAuthors": { TargetName: "bulk_update_authors", - ArgumentMappings: map[string]string{ + ArgumentMappings: grpcdatasource.FieldArgumentMap{ "authors": "authors", }, }, "bulkCreateBlogPosts": { TargetName: "bulk_create_blog_posts", - ArgumentMappings: map[string]string{ + ArgumentMappings: grpcdatasource.FieldArgumentMap{ "blogPosts": "blog_posts", }, }, "bulkUpdateBlogPosts": { TargetName: "bulk_update_blog_posts", - ArgumentMappings: map[string]string{ + ArgumentMappings: grpcdatasource.FieldArgumentMap{ "blogPosts": "blog_posts", }, }, }, - "UserInput": { - "name": { - TargetName: "name", - }, - }, "Product": { "id": { TargetName: "id", @@ -585,49 +583,21 @@ func DefaultGRPCMapping() *grpcdatasource.GRPCMapping { TargetName: "filter_field_2", }, }, - "TypeWithComplexFilterInput": { - "id": { - TargetName: "id", - }, - "name": { - TargetName: "name", - }, - }, - "Cat": { - "id": { - TargetName: "id", - }, - "name": { - TargetName: "name", - }, - "kind": { - TargetName: "kind", + "FilterTypeInput": { + "filterField1": { + TargetName: "filter_field_1", }, - "meowVolume": { - TargetName: "meow_volume", + "filterField2": { + TargetName: "filter_field_2", }, }, - "Dog": { + "TypeWithComplexFilterInput": { "id": { TargetName: "id", }, "name": { TargetName: "name", }, - "kind": { - TargetName: "kind", - }, - "barkVolume": { - TargetName: "bark_volume", - }, - }, - "Animal": { - "cat": { - TargetName: "cat", - }, - "dog": { - TargetName: "dog", - }, }, "FilterType": { "name": { @@ -656,29 +626,26 @@ func DefaultGRPCMapping() *grpcdatasource.GRPCMapping { TargetName: "filter", }, }, - "Category": { - "id": { - TargetName: "id", - }, - "name": { - TargetName: "name", + "OrderLineInput": { + "productId": { + TargetName: "product_id", }, - "kind": { - TargetName: "kind", + "quantity": { + TargetName: "quantity", }, - "productCount": { - TargetName: "product_count", - ArgumentMappings: map[string]string{ - "filters": "filters", - }, + "modifiers": { + TargetName: "modifiers", }, }, - "CategoryFilter": { - "category": { - TargetName: "category", + "OrderInput": { + "orderId": { + TargetName: "order_id", }, - "pagination": { - TargetName: "pagination", + "customerName": { + TargetName: "customer_name", + }, + "lines": { + TargetName: "lines", }, }, "Order": { @@ -706,26 +673,57 @@ func DefaultGRPCMapping() *grpcdatasource.GRPCMapping { TargetName: "modifiers", }, }, - "OrderInput": { - "orderId": { - TargetName: "order_id", + "CategoryFilter": { + "category": { + TargetName: "category", }, - "customerName": { - TargetName: "customer_name", + "pagination": { + TargetName: "pagination", }, - "lines": { - TargetName: "lines", + }, + "Category": { + "id": { + TargetName: "id", + }, + "name": { + TargetName: "name", + }, + "kind": { + TargetName: "kind", + }, + "productCount": { + TargetName: "product_count", + ArgumentMappings: grpcdatasource.FieldArgumentMap{ + "filters": "filters", + }, }, }, - "OrderLineInput": { - "productId": { - TargetName: "product_id", + "Cat": { + "id": { + TargetName: "id", }, - "quantity": { - TargetName: "quantity", + "name": { + TargetName: "name", }, - "modifiers": { - TargetName: "modifiers", + "kind": { + TargetName: "kind", + }, + "meowVolume": { + TargetName: "meow_volume", + }, + }, + "Dog": { + "id": { + TargetName: "id", + }, + "name": { + TargetName: "name", + }, + "kind": { + TargetName: "kind", + }, + "barkVolume": { + TargetName: "bark_volume", }, }, "ActionSuccess": { @@ -760,19 +758,6 @@ func DefaultGRPCMapping() *grpcdatasource.GRPCMapping { TargetName: "payload", }, }, - "SearchResult": { - "product": { - TargetName: "product", - }, - }, - "ActionResult": { - "actionSuccess": { - TargetName: "action_success", - }, - "actionError": { - TargetName: "action_error", - }, - }, "NullableFieldsType": { "id": { TargetName: "id", @@ -799,40 +784,6 @@ func DefaultGRPCMapping() *grpcdatasource.GRPCMapping { TargetName: "required_int", }, }, - "NullableFieldsInput": { - "name": { - TargetName: "name", - }, - "optionalString": { - TargetName: "optional_string", - }, - "optionalInt": { - TargetName: "optional_int", - }, - "optionalFloat": { - TargetName: "optional_float", - }, - "optionalBoolean": { - TargetName: "optional_boolean", - }, - "requiredString": { - TargetName: "required_string", - }, - "requiredInt": { - TargetName: "required_int", - }, - }, - "NullableFieldsFilter": { - "name": { - TargetName: "name", - }, - "optionalString": { - TargetName: "optional_string", - }, - "includeNulls": { - TargetName: "include_nulls", - }, - }, "BlogPost": { "id": { TargetName: "id", @@ -1046,6 +997,48 @@ func DefaultGRPCMapping() *grpcdatasource.GRPCMapping { TargetName: "skill_count", }, }, + "NullableFieldsInput": { + "name": { + TargetName: "name", + }, + "optionalString": { + TargetName: "optional_string", + }, + "optionalInt": { + TargetName: "optional_int", + }, + "optionalFloat": { + TargetName: "optional_float", + }, + "optionalBoolean": { + TargetName: "optional_boolean", + }, + "requiredString": { + TargetName: "required_string", + }, + "requiredInt": { + TargetName: "required_int", + }, + }, + "NullableFieldsFilter": { + "name": { + TargetName: "name", + }, + "optionalString": { + TargetName: "optional_string", + }, + "includeNulls": { + TargetName: "include_nulls", + }, + }, + "CategoryInput": { + "name": { + TargetName: "name", + }, + "kind": { + TargetName: "kind", + }, + }, "ProductCountFilter": { "minPrice": { TargetName: "min_price", @@ -1060,10 +1053,16 @@ func DefaultGRPCMapping() *grpcdatasource.GRPCMapping { TargetName: "search_term", }, }, + "UserInput": { + "name": { + TargetName: "name", + }, + }, }, } } +// MustDefaultGRPCMapping returns the default GRPC mapping func MustDefaultGRPCMapping(t *testing.T) *grpcdatasource.GRPCMapping { mapping := DefaultGRPCMapping() return mapping diff --git a/v2/pkg/grpctest/mockservice.go b/v2/pkg/grpctest/mockservice.go index 88d58dce5..ca4e1089e 100644 --- a/v2/pkg/grpctest/mockservice.go +++ b/v2/pkg/grpctest/mockservice.go @@ -22,10 +22,10 @@ type MockService struct { // ResolveCategoryProductCount implements productv1.ProductServiceServer. func (s *MockService) ResolveCategoryProductCount(_ context.Context, req *productv1.ResolveCategoryProductCountRequest) (*productv1.ResolveCategoryProductCountResponse, error) { - results := make([]*productv1.ResolveCategoryProductCountResponseResult, len(req.GetKey())) - for range req.GetKey() { + results := make([]*productv1.ResolveCategoryProductCountResponseResult, 0, len(req.GetContext())) + for i := range req.GetContext() { results = append(results, &productv1.ResolveCategoryProductCountResponseResult{ - ProductCount: int32(rand.Intn(100)), + ProductCount: int32(i), }) } diff --git a/v2/pkg/grpctest/product.proto b/v2/pkg/grpctest/product.proto index dcde431bc..24a170195 100644 --- a/v2/pkg/grpctest/product.proto +++ b/v2/pkg/grpctest/product.proto @@ -629,17 +629,13 @@ message CategoryProductCountContext { string name = 2; } -message ResolveCategoryProductCountRequestKey { +message ResolveCategoryProductCountRequest { // context provides the resolver context for the field productCount of type Category. - CategoryProductCountContext context = 1; + repeated CategoryProductCountContext context = 1; // field_args provides the arguments for the resolver field productCount of type Category. CategoryProductCountArgs field_args = 2; } -message ResolveCategoryProductCountRequest { - repeated ResolveCategoryProductCountRequestKey key = 1; -} - message ResolveCategoryProductCountResponseResult { int32 product_count = 1; } diff --git a/v2/pkg/grpctest/productv1/product.pb.go b/v2/pkg/grpctest/productv1/product.pb.go index e57fb0c9e..c8ff88e62 100644 --- a/v2/pkg/grpctest/productv1/product.pb.go +++ b/v2/pkg/grpctest/productv1/product.pb.go @@ -1,6 +1,6 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: -// protoc-gen-go v1.36.9 +// protoc-gen-go v1.36.10 // protoc v6.32.0 // source: product.proto @@ -5189,30 +5189,30 @@ func (x *CategoryProductCountContext) GetName() string { return "" } -type ResolveCategoryProductCountRequestKey struct { +type ResolveCategoryProductCountRequest struct { state protoimpl.MessageState `protogen:"open.v1"` // context provides the resolver context for the field productCount of type Category. - Context *CategoryProductCountContext `protobuf:"bytes,1,opt,name=context,proto3" json:"context,omitempty"` + Context []*CategoryProductCountContext `protobuf:"bytes,1,rep,name=context,proto3" json:"context,omitempty"` // field_args provides the arguments for the resolver field productCount of type Category. FieldArgs *CategoryProductCountArgs `protobuf:"bytes,2,opt,name=field_args,json=fieldArgs,proto3" json:"field_args,omitempty"` unknownFields protoimpl.UnknownFields sizeCache protoimpl.SizeCache } -func (x *ResolveCategoryProductCountRequestKey) Reset() { - *x = ResolveCategoryProductCountRequestKey{} +func (x *ResolveCategoryProductCountRequest) Reset() { + *x = ResolveCategoryProductCountRequest{} mi := &file_product_proto_msgTypes[114] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } -func (x *ResolveCategoryProductCountRequestKey) String() string { +func (x *ResolveCategoryProductCountRequest) String() string { return protoimpl.X.MessageStringOf(x) } -func (*ResolveCategoryProductCountRequestKey) ProtoMessage() {} +func (*ResolveCategoryProductCountRequest) ProtoMessage() {} -func (x *ResolveCategoryProductCountRequestKey) ProtoReflect() protoreflect.Message { +func (x *ResolveCategoryProductCountRequest) ProtoReflect() protoreflect.Message { mi := &file_product_proto_msgTypes[114] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) @@ -5224,69 +5224,25 @@ func (x *ResolveCategoryProductCountRequestKey) ProtoReflect() protoreflect.Mess return mi.MessageOf(x) } -// Deprecated: Use ResolveCategoryProductCountRequestKey.ProtoReflect.Descriptor instead. -func (*ResolveCategoryProductCountRequestKey) Descriptor() ([]byte, []int) { +// Deprecated: Use ResolveCategoryProductCountRequest.ProtoReflect.Descriptor instead. +func (*ResolveCategoryProductCountRequest) Descriptor() ([]byte, []int) { return file_product_proto_rawDescGZIP(), []int{114} } -func (x *ResolveCategoryProductCountRequestKey) GetContext() *CategoryProductCountContext { +func (x *ResolveCategoryProductCountRequest) GetContext() []*CategoryProductCountContext { if x != nil { return x.Context } return nil } -func (x *ResolveCategoryProductCountRequestKey) GetFieldArgs() *CategoryProductCountArgs { +func (x *ResolveCategoryProductCountRequest) GetFieldArgs() *CategoryProductCountArgs { if x != nil { return x.FieldArgs } return nil } -type ResolveCategoryProductCountRequest struct { - state protoimpl.MessageState `protogen:"open.v1"` - Key []*ResolveCategoryProductCountRequestKey `protobuf:"bytes,1,rep,name=key,proto3" json:"key,omitempty"` - unknownFields protoimpl.UnknownFields - sizeCache protoimpl.SizeCache -} - -func (x *ResolveCategoryProductCountRequest) Reset() { - *x = ResolveCategoryProductCountRequest{} - mi := &file_product_proto_msgTypes[115] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) -} - -func (x *ResolveCategoryProductCountRequest) String() string { - return protoimpl.X.MessageStringOf(x) -} - -func (*ResolveCategoryProductCountRequest) ProtoMessage() {} - -func (x *ResolveCategoryProductCountRequest) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[115] - if x != nil { - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - if ms.LoadMessageInfo() == nil { - ms.StoreMessageInfo(mi) - } - return ms - } - return mi.MessageOf(x) -} - -// Deprecated: Use ResolveCategoryProductCountRequest.ProtoReflect.Descriptor instead. -func (*ResolveCategoryProductCountRequest) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{115} -} - -func (x *ResolveCategoryProductCountRequest) GetKey() []*ResolveCategoryProductCountRequestKey { - if x != nil { - return x.Key - } - return nil -} - type ResolveCategoryProductCountResponseResult struct { state protoimpl.MessageState `protogen:"open.v1"` ProductCount int32 `protobuf:"varint,1,opt,name=product_count,json=productCount,proto3" json:"product_count,omitempty"` @@ -5296,7 +5252,7 @@ type ResolveCategoryProductCountResponseResult struct { func (x *ResolveCategoryProductCountResponseResult) Reset() { *x = ResolveCategoryProductCountResponseResult{} - mi := &file_product_proto_msgTypes[116] + mi := &file_product_proto_msgTypes[115] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5308,7 +5264,7 @@ func (x *ResolveCategoryProductCountResponseResult) String() string { func (*ResolveCategoryProductCountResponseResult) ProtoMessage() {} func (x *ResolveCategoryProductCountResponseResult) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[116] + mi := &file_product_proto_msgTypes[115] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5321,7 +5277,7 @@ func (x *ResolveCategoryProductCountResponseResult) ProtoReflect() protoreflect. // Deprecated: Use ResolveCategoryProductCountResponseResult.ProtoReflect.Descriptor instead. func (*ResolveCategoryProductCountResponseResult) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{116} + return file_product_proto_rawDescGZIP(), []int{115} } func (x *ResolveCategoryProductCountResponseResult) GetProductCount() int32 { @@ -5340,7 +5296,7 @@ type ResolveCategoryProductCountResponse struct { func (x *ResolveCategoryProductCountResponse) Reset() { *x = ResolveCategoryProductCountResponse{} - mi := &file_product_proto_msgTypes[117] + mi := &file_product_proto_msgTypes[116] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5352,7 +5308,7 @@ func (x *ResolveCategoryProductCountResponse) String() string { func (*ResolveCategoryProductCountResponse) ProtoMessage() {} func (x *ResolveCategoryProductCountResponse) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[117] + mi := &file_product_proto_msgTypes[116] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5365,7 +5321,7 @@ func (x *ResolveCategoryProductCountResponse) ProtoReflect() protoreflect.Messag // Deprecated: Use ResolveCategoryProductCountResponse.ProtoReflect.Descriptor instead. func (*ResolveCategoryProductCountResponse) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{117} + return file_product_proto_rawDescGZIP(), []int{116} } func (x *ResolveCategoryProductCountResponse) GetResult() []*ResolveCategoryProductCountResponseResult { @@ -5386,7 +5342,7 @@ type Product struct { func (x *Product) Reset() { *x = Product{} - mi := &file_product_proto_msgTypes[118] + mi := &file_product_proto_msgTypes[117] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5398,7 +5354,7 @@ func (x *Product) String() string { func (*Product) ProtoMessage() {} func (x *Product) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[118] + mi := &file_product_proto_msgTypes[117] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5411,7 +5367,7 @@ func (x *Product) ProtoReflect() protoreflect.Message { // Deprecated: Use Product.ProtoReflect.Descriptor instead. func (*Product) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{118} + return file_product_proto_rawDescGZIP(), []int{117} } func (x *Product) GetId() string { @@ -5446,7 +5402,7 @@ type Storage struct { func (x *Storage) Reset() { *x = Storage{} - mi := &file_product_proto_msgTypes[119] + mi := &file_product_proto_msgTypes[118] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5458,7 +5414,7 @@ func (x *Storage) String() string { func (*Storage) ProtoMessage() {} func (x *Storage) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[119] + mi := &file_product_proto_msgTypes[118] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5471,7 +5427,7 @@ func (x *Storage) ProtoReflect() protoreflect.Message { // Deprecated: Use Storage.ProtoReflect.Descriptor instead. func (*Storage) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{119} + return file_product_proto_rawDescGZIP(), []int{118} } func (x *Storage) GetId() string { @@ -5506,7 +5462,7 @@ type Warehouse struct { func (x *Warehouse) Reset() { *x = Warehouse{} - mi := &file_product_proto_msgTypes[120] + mi := &file_product_proto_msgTypes[119] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5518,7 +5474,7 @@ func (x *Warehouse) String() string { func (*Warehouse) ProtoMessage() {} func (x *Warehouse) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[120] + mi := &file_product_proto_msgTypes[119] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5531,7 +5487,7 @@ func (x *Warehouse) ProtoReflect() protoreflect.Message { // Deprecated: Use Warehouse.ProtoReflect.Descriptor instead. func (*Warehouse) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{120} + return file_product_proto_rawDescGZIP(), []int{119} } func (x *Warehouse) GetId() string { @@ -5565,7 +5521,7 @@ type User struct { func (x *User) Reset() { *x = User{} - mi := &file_product_proto_msgTypes[121] + mi := &file_product_proto_msgTypes[120] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5577,7 +5533,7 @@ func (x *User) String() string { func (*User) ProtoMessage() {} func (x *User) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[121] + mi := &file_product_proto_msgTypes[120] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5590,7 +5546,7 @@ func (x *User) ProtoReflect() protoreflect.Message { // Deprecated: Use User.ProtoReflect.Descriptor instead. func (*User) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{121} + return file_product_proto_rawDescGZIP(), []int{120} } func (x *User) GetId() string { @@ -5618,7 +5574,7 @@ type NestedTypeA struct { func (x *NestedTypeA) Reset() { *x = NestedTypeA{} - mi := &file_product_proto_msgTypes[122] + mi := &file_product_proto_msgTypes[121] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5630,7 +5586,7 @@ func (x *NestedTypeA) String() string { func (*NestedTypeA) ProtoMessage() {} func (x *NestedTypeA) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[122] + mi := &file_product_proto_msgTypes[121] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5643,7 +5599,7 @@ func (x *NestedTypeA) ProtoReflect() protoreflect.Message { // Deprecated: Use NestedTypeA.ProtoReflect.Descriptor instead. func (*NestedTypeA) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{122} + return file_product_proto_rawDescGZIP(), []int{121} } func (x *NestedTypeA) GetId() string { @@ -5678,7 +5634,7 @@ type RecursiveType struct { func (x *RecursiveType) Reset() { *x = RecursiveType{} - mi := &file_product_proto_msgTypes[123] + mi := &file_product_proto_msgTypes[122] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5690,7 +5646,7 @@ func (x *RecursiveType) String() string { func (*RecursiveType) ProtoMessage() {} func (x *RecursiveType) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[123] + mi := &file_product_proto_msgTypes[122] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5703,7 +5659,7 @@ func (x *RecursiveType) ProtoReflect() protoreflect.Message { // Deprecated: Use RecursiveType.ProtoReflect.Descriptor instead. func (*RecursiveType) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{123} + return file_product_proto_rawDescGZIP(), []int{122} } func (x *RecursiveType) GetId() string { @@ -5739,7 +5695,7 @@ type TypeWithMultipleFilterFields struct { func (x *TypeWithMultipleFilterFields) Reset() { *x = TypeWithMultipleFilterFields{} - mi := &file_product_proto_msgTypes[124] + mi := &file_product_proto_msgTypes[123] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5751,7 +5707,7 @@ func (x *TypeWithMultipleFilterFields) String() string { func (*TypeWithMultipleFilterFields) ProtoMessage() {} func (x *TypeWithMultipleFilterFields) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[124] + mi := &file_product_proto_msgTypes[123] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5764,7 +5720,7 @@ func (x *TypeWithMultipleFilterFields) ProtoReflect() protoreflect.Message { // Deprecated: Use TypeWithMultipleFilterFields.ProtoReflect.Descriptor instead. func (*TypeWithMultipleFilterFields) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{124} + return file_product_proto_rawDescGZIP(), []int{123} } func (x *TypeWithMultipleFilterFields) GetId() string { @@ -5805,7 +5761,7 @@ type FilterTypeInput struct { func (x *FilterTypeInput) Reset() { *x = FilterTypeInput{} - mi := &file_product_proto_msgTypes[125] + mi := &file_product_proto_msgTypes[124] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5817,7 +5773,7 @@ func (x *FilterTypeInput) String() string { func (*FilterTypeInput) ProtoMessage() {} func (x *FilterTypeInput) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[125] + mi := &file_product_proto_msgTypes[124] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5830,7 +5786,7 @@ func (x *FilterTypeInput) ProtoReflect() protoreflect.Message { // Deprecated: Use FilterTypeInput.ProtoReflect.Descriptor instead. func (*FilterTypeInput) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{125} + return file_product_proto_rawDescGZIP(), []int{124} } func (x *FilterTypeInput) GetFilterField_1() string { @@ -5856,7 +5812,7 @@ type ComplexFilterTypeInput struct { func (x *ComplexFilterTypeInput) Reset() { *x = ComplexFilterTypeInput{} - mi := &file_product_proto_msgTypes[126] + mi := &file_product_proto_msgTypes[125] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5868,7 +5824,7 @@ func (x *ComplexFilterTypeInput) String() string { func (*ComplexFilterTypeInput) ProtoMessage() {} func (x *ComplexFilterTypeInput) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[126] + mi := &file_product_proto_msgTypes[125] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5881,7 +5837,7 @@ func (x *ComplexFilterTypeInput) ProtoReflect() protoreflect.Message { // Deprecated: Use ComplexFilterTypeInput.ProtoReflect.Descriptor instead. func (*ComplexFilterTypeInput) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{126} + return file_product_proto_rawDescGZIP(), []int{125} } func (x *ComplexFilterTypeInput) GetFilter() *FilterType { @@ -5901,7 +5857,7 @@ type TypeWithComplexFilterInput struct { func (x *TypeWithComplexFilterInput) Reset() { *x = TypeWithComplexFilterInput{} - mi := &file_product_proto_msgTypes[127] + mi := &file_product_proto_msgTypes[126] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5913,7 +5869,7 @@ func (x *TypeWithComplexFilterInput) String() string { func (*TypeWithComplexFilterInput) ProtoMessage() {} func (x *TypeWithComplexFilterInput) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[127] + mi := &file_product_proto_msgTypes[126] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5926,7 +5882,7 @@ func (x *TypeWithComplexFilterInput) ProtoReflect() protoreflect.Message { // Deprecated: Use TypeWithComplexFilterInput.ProtoReflect.Descriptor instead. func (*TypeWithComplexFilterInput) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{127} + return file_product_proto_rawDescGZIP(), []int{126} } func (x *TypeWithComplexFilterInput) GetId() string { @@ -5954,7 +5910,7 @@ type OrderInput struct { func (x *OrderInput) Reset() { *x = OrderInput{} - mi := &file_product_proto_msgTypes[128] + mi := &file_product_proto_msgTypes[127] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5966,7 +5922,7 @@ func (x *OrderInput) String() string { func (*OrderInput) ProtoMessage() {} func (x *OrderInput) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[128] + mi := &file_product_proto_msgTypes[127] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5979,7 +5935,7 @@ func (x *OrderInput) ProtoReflect() protoreflect.Message { // Deprecated: Use OrderInput.ProtoReflect.Descriptor instead. func (*OrderInput) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{128} + return file_product_proto_rawDescGZIP(), []int{127} } func (x *OrderInput) GetOrderId() string { @@ -6015,7 +5971,7 @@ type Order struct { func (x *Order) Reset() { *x = Order{} - mi := &file_product_proto_msgTypes[129] + mi := &file_product_proto_msgTypes[128] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6027,7 +5983,7 @@ func (x *Order) String() string { func (*Order) ProtoMessage() {} func (x *Order) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[129] + mi := &file_product_proto_msgTypes[128] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6040,7 +5996,7 @@ func (x *Order) ProtoReflect() protoreflect.Message { // Deprecated: Use Order.ProtoReflect.Descriptor instead. func (*Order) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{129} + return file_product_proto_rawDescGZIP(), []int{128} } func (x *Order) GetOrderId() string { @@ -6082,7 +6038,7 @@ type Category struct { func (x *Category) Reset() { *x = Category{} - mi := &file_product_proto_msgTypes[130] + mi := &file_product_proto_msgTypes[129] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6094,7 +6050,7 @@ func (x *Category) String() string { func (*Category) ProtoMessage() {} func (x *Category) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[130] + mi := &file_product_proto_msgTypes[129] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6107,7 +6063,7 @@ func (x *Category) ProtoReflect() protoreflect.Message { // Deprecated: Use Category.ProtoReflect.Descriptor instead. func (*Category) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{130} + return file_product_proto_rawDescGZIP(), []int{129} } func (x *Category) GetId() string { @@ -6141,7 +6097,7 @@ type CategoryFilter struct { func (x *CategoryFilter) Reset() { *x = CategoryFilter{} - mi := &file_product_proto_msgTypes[131] + mi := &file_product_proto_msgTypes[130] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6153,7 +6109,7 @@ func (x *CategoryFilter) String() string { func (*CategoryFilter) ProtoMessage() {} func (x *CategoryFilter) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[131] + mi := &file_product_proto_msgTypes[130] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6166,7 +6122,7 @@ func (x *CategoryFilter) ProtoReflect() protoreflect.Message { // Deprecated: Use CategoryFilter.ProtoReflect.Descriptor instead. func (*CategoryFilter) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{131} + return file_product_proto_rawDescGZIP(), []int{130} } func (x *CategoryFilter) GetCategory() CategoryKind { @@ -6196,7 +6152,7 @@ type Animal struct { func (x *Animal) Reset() { *x = Animal{} - mi := &file_product_proto_msgTypes[132] + mi := &file_product_proto_msgTypes[131] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6208,7 +6164,7 @@ func (x *Animal) String() string { func (*Animal) ProtoMessage() {} func (x *Animal) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[132] + mi := &file_product_proto_msgTypes[131] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6221,7 +6177,7 @@ func (x *Animal) ProtoReflect() protoreflect.Message { // Deprecated: Use Animal.ProtoReflect.Descriptor instead. func (*Animal) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{132} + return file_product_proto_rawDescGZIP(), []int{131} } func (x *Animal) GetInstance() isAnimal_Instance { @@ -6275,7 +6231,7 @@ type SearchInput struct { func (x *SearchInput) Reset() { *x = SearchInput{} - mi := &file_product_proto_msgTypes[133] + mi := &file_product_proto_msgTypes[132] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6287,7 +6243,7 @@ func (x *SearchInput) String() string { func (*SearchInput) ProtoMessage() {} func (x *SearchInput) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[133] + mi := &file_product_proto_msgTypes[132] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6300,7 +6256,7 @@ func (x *SearchInput) ProtoReflect() protoreflect.Message { // Deprecated: Use SearchInput.ProtoReflect.Descriptor instead. func (*SearchInput) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{133} + return file_product_proto_rawDescGZIP(), []int{132} } func (x *SearchInput) GetQuery() string { @@ -6331,7 +6287,7 @@ type SearchResult struct { func (x *SearchResult) Reset() { *x = SearchResult{} - mi := &file_product_proto_msgTypes[134] + mi := &file_product_proto_msgTypes[133] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6343,7 +6299,7 @@ func (x *SearchResult) String() string { func (*SearchResult) ProtoMessage() {} func (x *SearchResult) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[134] + mi := &file_product_proto_msgTypes[133] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6356,7 +6312,7 @@ func (x *SearchResult) ProtoReflect() protoreflect.Message { // Deprecated: Use SearchResult.ProtoReflect.Descriptor instead. func (*SearchResult) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{134} + return file_product_proto_rawDescGZIP(), []int{133} } func (x *SearchResult) GetValue() isSearchResult_Value { @@ -6431,7 +6387,7 @@ type NullableFieldsType struct { func (x *NullableFieldsType) Reset() { *x = NullableFieldsType{} - mi := &file_product_proto_msgTypes[135] + mi := &file_product_proto_msgTypes[134] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6443,7 +6399,7 @@ func (x *NullableFieldsType) String() string { func (*NullableFieldsType) ProtoMessage() {} func (x *NullableFieldsType) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[135] + mi := &file_product_proto_msgTypes[134] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6456,7 +6412,7 @@ func (x *NullableFieldsType) ProtoReflect() protoreflect.Message { // Deprecated: Use NullableFieldsType.ProtoReflect.Descriptor instead. func (*NullableFieldsType) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{135} + return file_product_proto_rawDescGZIP(), []int{134} } func (x *NullableFieldsType) GetId() string { @@ -6526,7 +6482,7 @@ type NullableFieldsFilter struct { func (x *NullableFieldsFilter) Reset() { *x = NullableFieldsFilter{} - mi := &file_product_proto_msgTypes[136] + mi := &file_product_proto_msgTypes[135] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6538,7 +6494,7 @@ func (x *NullableFieldsFilter) String() string { func (*NullableFieldsFilter) ProtoMessage() {} func (x *NullableFieldsFilter) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[136] + mi := &file_product_proto_msgTypes[135] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6551,7 +6507,7 @@ func (x *NullableFieldsFilter) ProtoReflect() protoreflect.Message { // Deprecated: Use NullableFieldsFilter.ProtoReflect.Descriptor instead. func (*NullableFieldsFilter) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{136} + return file_product_proto_rawDescGZIP(), []int{135} } func (x *NullableFieldsFilter) GetName() *wrapperspb.StringValue { @@ -6603,7 +6559,7 @@ type BlogPost struct { func (x *BlogPost) Reset() { *x = BlogPost{} - mi := &file_product_proto_msgTypes[137] + mi := &file_product_proto_msgTypes[136] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6615,7 +6571,7 @@ func (x *BlogPost) String() string { func (*BlogPost) ProtoMessage() {} func (x *BlogPost) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[137] + mi := &file_product_proto_msgTypes[136] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6628,7 +6584,7 @@ func (x *BlogPost) ProtoReflect() protoreflect.Message { // Deprecated: Use BlogPost.ProtoReflect.Descriptor instead. func (*BlogPost) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{137} + return file_product_proto_rawDescGZIP(), []int{136} } func (x *BlogPost) GetId() string { @@ -6782,7 +6738,7 @@ type BlogPostFilter struct { func (x *BlogPostFilter) Reset() { *x = BlogPostFilter{} - mi := &file_product_proto_msgTypes[138] + mi := &file_product_proto_msgTypes[137] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6794,7 +6750,7 @@ func (x *BlogPostFilter) String() string { func (*BlogPostFilter) ProtoMessage() {} func (x *BlogPostFilter) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[138] + mi := &file_product_proto_msgTypes[137] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6807,7 +6763,7 @@ func (x *BlogPostFilter) ProtoReflect() protoreflect.Message { // Deprecated: Use BlogPostFilter.ProtoReflect.Descriptor instead. func (*BlogPostFilter) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{138} + return file_product_proto_rawDescGZIP(), []int{137} } func (x *BlogPostFilter) GetTitle() *wrapperspb.StringValue { @@ -6854,7 +6810,7 @@ type Author struct { func (x *Author) Reset() { *x = Author{} - mi := &file_product_proto_msgTypes[139] + mi := &file_product_proto_msgTypes[138] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6866,7 +6822,7 @@ func (x *Author) String() string { func (*Author) ProtoMessage() {} func (x *Author) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[139] + mi := &file_product_proto_msgTypes[138] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6879,7 +6835,7 @@ func (x *Author) ProtoReflect() protoreflect.Message { // Deprecated: Use Author.ProtoReflect.Descriptor instead. func (*Author) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{139} + return file_product_proto_rawDescGZIP(), []int{138} } func (x *Author) GetId() string { @@ -6998,7 +6954,7 @@ type AuthorFilter struct { func (x *AuthorFilter) Reset() { *x = AuthorFilter{} - mi := &file_product_proto_msgTypes[140] + mi := &file_product_proto_msgTypes[139] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7010,7 +6966,7 @@ func (x *AuthorFilter) String() string { func (*AuthorFilter) ProtoMessage() {} func (x *AuthorFilter) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[140] + mi := &file_product_proto_msgTypes[139] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7023,7 +6979,7 @@ func (x *AuthorFilter) ProtoReflect() protoreflect.Message { // Deprecated: Use AuthorFilter.ProtoReflect.Descriptor instead. func (*AuthorFilter) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{140} + return file_product_proto_rawDescGZIP(), []int{139} } func (x *AuthorFilter) GetName() *wrapperspb.StringValue { @@ -7056,7 +7012,7 @@ type UserInput struct { func (x *UserInput) Reset() { *x = UserInput{} - mi := &file_product_proto_msgTypes[141] + mi := &file_product_proto_msgTypes[140] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7068,7 +7024,7 @@ func (x *UserInput) String() string { func (*UserInput) ProtoMessage() {} func (x *UserInput) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[141] + mi := &file_product_proto_msgTypes[140] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7081,7 +7037,7 @@ func (x *UserInput) ProtoReflect() protoreflect.Message { // Deprecated: Use UserInput.ProtoReflect.Descriptor instead. func (*UserInput) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{141} + return file_product_proto_rawDescGZIP(), []int{140} } func (x *UserInput) GetName() string { @@ -7101,7 +7057,7 @@ type ActionInput struct { func (x *ActionInput) Reset() { *x = ActionInput{} - mi := &file_product_proto_msgTypes[142] + mi := &file_product_proto_msgTypes[141] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7113,7 +7069,7 @@ func (x *ActionInput) String() string { func (*ActionInput) ProtoMessage() {} func (x *ActionInput) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[142] + mi := &file_product_proto_msgTypes[141] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7126,7 +7082,7 @@ func (x *ActionInput) ProtoReflect() protoreflect.Message { // Deprecated: Use ActionInput.ProtoReflect.Descriptor instead. func (*ActionInput) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{142} + return file_product_proto_rawDescGZIP(), []int{141} } func (x *ActionInput) GetType() string { @@ -7156,7 +7112,7 @@ type ActionResult struct { func (x *ActionResult) Reset() { *x = ActionResult{} - mi := &file_product_proto_msgTypes[143] + mi := &file_product_proto_msgTypes[142] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7168,7 +7124,7 @@ func (x *ActionResult) String() string { func (*ActionResult) ProtoMessage() {} func (x *ActionResult) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[143] + mi := &file_product_proto_msgTypes[142] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7181,7 +7137,7 @@ func (x *ActionResult) ProtoReflect() protoreflect.Message { // Deprecated: Use ActionResult.ProtoReflect.Descriptor instead. func (*ActionResult) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{143} + return file_product_proto_rawDescGZIP(), []int{142} } func (x *ActionResult) GetValue() isActionResult_Value { @@ -7240,7 +7196,7 @@ type NullableFieldsInput struct { func (x *NullableFieldsInput) Reset() { *x = NullableFieldsInput{} - mi := &file_product_proto_msgTypes[144] + mi := &file_product_proto_msgTypes[143] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7252,7 +7208,7 @@ func (x *NullableFieldsInput) String() string { func (*NullableFieldsInput) ProtoMessage() {} func (x *NullableFieldsInput) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[144] + mi := &file_product_proto_msgTypes[143] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7265,7 +7221,7 @@ func (x *NullableFieldsInput) ProtoReflect() protoreflect.Message { // Deprecated: Use NullableFieldsInput.ProtoReflect.Descriptor instead. func (*NullableFieldsInput) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{144} + return file_product_proto_rawDescGZIP(), []int{143} } func (x *NullableFieldsInput) GetName() string { @@ -7341,7 +7297,7 @@ type BlogPostInput struct { func (x *BlogPostInput) Reset() { *x = BlogPostInput{} - mi := &file_product_proto_msgTypes[145] + mi := &file_product_proto_msgTypes[144] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7353,7 +7309,7 @@ func (x *BlogPostInput) String() string { func (*BlogPostInput) ProtoMessage() {} func (x *BlogPostInput) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[145] + mi := &file_product_proto_msgTypes[144] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7366,7 +7322,7 @@ func (x *BlogPostInput) ProtoReflect() protoreflect.Message { // Deprecated: Use BlogPostInput.ProtoReflect.Descriptor instead. func (*BlogPostInput) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{145} + return file_product_proto_rawDescGZIP(), []int{144} } func (x *BlogPostInput) GetTitle() string { @@ -7499,7 +7455,7 @@ type AuthorInput struct { func (x *AuthorInput) Reset() { *x = AuthorInput{} - mi := &file_product_proto_msgTypes[146] + mi := &file_product_proto_msgTypes[145] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7511,7 +7467,7 @@ func (x *AuthorInput) String() string { func (*AuthorInput) ProtoMessage() {} func (x *AuthorInput) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[146] + mi := &file_product_proto_msgTypes[145] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7524,7 +7480,7 @@ func (x *AuthorInput) ProtoReflect() protoreflect.Message { // Deprecated: Use AuthorInput.ProtoReflect.Descriptor instead. func (*AuthorInput) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{146} + return file_product_proto_rawDescGZIP(), []int{145} } func (x *AuthorInput) GetName() string { @@ -7608,7 +7564,7 @@ type NestedTypeB struct { func (x *NestedTypeB) Reset() { *x = NestedTypeB{} - mi := &file_product_proto_msgTypes[147] + mi := &file_product_proto_msgTypes[146] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7620,7 +7576,7 @@ func (x *NestedTypeB) String() string { func (*NestedTypeB) ProtoMessage() {} func (x *NestedTypeB) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[147] + mi := &file_product_proto_msgTypes[146] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7633,7 +7589,7 @@ func (x *NestedTypeB) ProtoReflect() protoreflect.Message { // Deprecated: Use NestedTypeB.ProtoReflect.Descriptor instead. func (*NestedTypeB) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{147} + return file_product_proto_rawDescGZIP(), []int{146} } func (x *NestedTypeB) GetId() string { @@ -7667,7 +7623,7 @@ type NestedTypeC struct { func (x *NestedTypeC) Reset() { *x = NestedTypeC{} - mi := &file_product_proto_msgTypes[148] + mi := &file_product_proto_msgTypes[147] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7679,7 +7635,7 @@ func (x *NestedTypeC) String() string { func (*NestedTypeC) ProtoMessage() {} func (x *NestedTypeC) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[148] + mi := &file_product_proto_msgTypes[147] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7692,7 +7648,7 @@ func (x *NestedTypeC) ProtoReflect() protoreflect.Message { // Deprecated: Use NestedTypeC.ProtoReflect.Descriptor instead. func (*NestedTypeC) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{148} + return file_product_proto_rawDescGZIP(), []int{147} } func (x *NestedTypeC) GetId() string { @@ -7721,7 +7677,7 @@ type FilterType struct { func (x *FilterType) Reset() { *x = FilterType{} - mi := &file_product_proto_msgTypes[149] + mi := &file_product_proto_msgTypes[148] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7733,7 +7689,7 @@ func (x *FilterType) String() string { func (*FilterType) ProtoMessage() {} func (x *FilterType) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[149] + mi := &file_product_proto_msgTypes[148] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7746,7 +7702,7 @@ func (x *FilterType) ProtoReflect() protoreflect.Message { // Deprecated: Use FilterType.ProtoReflect.Descriptor instead. func (*FilterType) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{149} + return file_product_proto_rawDescGZIP(), []int{148} } func (x *FilterType) GetName() string { @@ -7787,7 +7743,7 @@ type Pagination struct { func (x *Pagination) Reset() { *x = Pagination{} - mi := &file_product_proto_msgTypes[150] + mi := &file_product_proto_msgTypes[149] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7799,7 +7755,7 @@ func (x *Pagination) String() string { func (*Pagination) ProtoMessage() {} func (x *Pagination) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[150] + mi := &file_product_proto_msgTypes[149] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7812,7 +7768,7 @@ func (x *Pagination) ProtoReflect() protoreflect.Message { // Deprecated: Use Pagination.ProtoReflect.Descriptor instead. func (*Pagination) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{150} + return file_product_proto_rawDescGZIP(), []int{149} } func (x *Pagination) GetPage() int32 { @@ -7840,7 +7796,7 @@ type OrderLineInput struct { func (x *OrderLineInput) Reset() { *x = OrderLineInput{} - mi := &file_product_proto_msgTypes[151] + mi := &file_product_proto_msgTypes[150] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7852,7 +7808,7 @@ func (x *OrderLineInput) String() string { func (*OrderLineInput) ProtoMessage() {} func (x *OrderLineInput) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[151] + mi := &file_product_proto_msgTypes[150] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7865,7 +7821,7 @@ func (x *OrderLineInput) ProtoReflect() protoreflect.Message { // Deprecated: Use OrderLineInput.ProtoReflect.Descriptor instead. func (*OrderLineInput) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{151} + return file_product_proto_rawDescGZIP(), []int{150} } func (x *OrderLineInput) GetProductId() string { @@ -7900,7 +7856,7 @@ type OrderLine struct { func (x *OrderLine) Reset() { *x = OrderLine{} - mi := &file_product_proto_msgTypes[152] + mi := &file_product_proto_msgTypes[151] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7912,7 +7868,7 @@ func (x *OrderLine) String() string { func (*OrderLine) ProtoMessage() {} func (x *OrderLine) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[152] + mi := &file_product_proto_msgTypes[151] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7925,7 +7881,7 @@ func (x *OrderLine) ProtoReflect() protoreflect.Message { // Deprecated: Use OrderLine.ProtoReflect.Descriptor instead. func (*OrderLine) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{152} + return file_product_proto_rawDescGZIP(), []int{151} } func (x *OrderLine) GetProductId() string { @@ -7961,7 +7917,7 @@ type Cat struct { func (x *Cat) Reset() { *x = Cat{} - mi := &file_product_proto_msgTypes[153] + mi := &file_product_proto_msgTypes[152] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7973,7 +7929,7 @@ func (x *Cat) String() string { func (*Cat) ProtoMessage() {} func (x *Cat) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[153] + mi := &file_product_proto_msgTypes[152] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7986,7 +7942,7 @@ func (x *Cat) ProtoReflect() protoreflect.Message { // Deprecated: Use Cat.ProtoReflect.Descriptor instead. func (*Cat) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{153} + return file_product_proto_rawDescGZIP(), []int{152} } func (x *Cat) GetId() string { @@ -8029,7 +7985,7 @@ type Dog struct { func (x *Dog) Reset() { *x = Dog{} - mi := &file_product_proto_msgTypes[154] + mi := &file_product_proto_msgTypes[153] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8041,7 +7997,7 @@ func (x *Dog) String() string { func (*Dog) ProtoMessage() {} func (x *Dog) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[154] + mi := &file_product_proto_msgTypes[153] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8054,7 +8010,7 @@ func (x *Dog) ProtoReflect() protoreflect.Message { // Deprecated: Use Dog.ProtoReflect.Descriptor instead. func (*Dog) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{154} + return file_product_proto_rawDescGZIP(), []int{153} } func (x *Dog) GetId() string { @@ -8095,7 +8051,7 @@ type ActionSuccess struct { func (x *ActionSuccess) Reset() { *x = ActionSuccess{} - mi := &file_product_proto_msgTypes[155] + mi := &file_product_proto_msgTypes[154] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8107,7 +8063,7 @@ func (x *ActionSuccess) String() string { func (*ActionSuccess) ProtoMessage() {} func (x *ActionSuccess) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[155] + mi := &file_product_proto_msgTypes[154] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8120,7 +8076,7 @@ func (x *ActionSuccess) ProtoReflect() protoreflect.Message { // Deprecated: Use ActionSuccess.ProtoReflect.Descriptor instead. func (*ActionSuccess) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{155} + return file_product_proto_rawDescGZIP(), []int{154} } func (x *ActionSuccess) GetMessage() string { @@ -8147,7 +8103,7 @@ type ActionError struct { func (x *ActionError) Reset() { *x = ActionError{} - mi := &file_product_proto_msgTypes[156] + mi := &file_product_proto_msgTypes[155] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8159,7 +8115,7 @@ func (x *ActionError) String() string { func (*ActionError) ProtoMessage() {} func (x *ActionError) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[156] + mi := &file_product_proto_msgTypes[155] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8172,7 +8128,7 @@ func (x *ActionError) ProtoReflect() protoreflect.Message { // Deprecated: Use ActionError.ProtoReflect.Descriptor instead. func (*ActionError) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{156} + return file_product_proto_rawDescGZIP(), []int{155} } func (x *ActionError) GetMessage() string { @@ -8199,7 +8155,7 @@ type CategoryInput struct { func (x *CategoryInput) Reset() { *x = CategoryInput{} - mi := &file_product_proto_msgTypes[157] + mi := &file_product_proto_msgTypes[156] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8211,7 +8167,7 @@ func (x *CategoryInput) String() string { func (*CategoryInput) ProtoMessage() {} func (x *CategoryInput) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[157] + mi := &file_product_proto_msgTypes[156] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8224,7 +8180,7 @@ func (x *CategoryInput) ProtoReflect() protoreflect.Message { // Deprecated: Use CategoryInput.ProtoReflect.Descriptor instead. func (*CategoryInput) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{157} + return file_product_proto_rawDescGZIP(), []int{156} } func (x *CategoryInput) GetName() string { @@ -8253,7 +8209,7 @@ type ProductCountFilter struct { func (x *ProductCountFilter) Reset() { *x = ProductCountFilter{} - mi := &file_product_proto_msgTypes[158] + mi := &file_product_proto_msgTypes[157] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8265,7 +8221,7 @@ func (x *ProductCountFilter) String() string { func (*ProductCountFilter) ProtoMessage() {} func (x *ProductCountFilter) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[158] + mi := &file_product_proto_msgTypes[157] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8278,7 +8234,7 @@ func (x *ProductCountFilter) ProtoReflect() protoreflect.Message { // Deprecated: Use ProductCountFilter.ProtoReflect.Descriptor instead. func (*ProductCountFilter) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{158} + return file_product_proto_rawDescGZIP(), []int{157} } func (x *ProductCountFilter) GetMinPrice() *wrapperspb.DoubleValue { @@ -8318,7 +8274,7 @@ type ListOfAuthorFilter_List struct { func (x *ListOfAuthorFilter_List) Reset() { *x = ListOfAuthorFilter_List{} - mi := &file_product_proto_msgTypes[159] + mi := &file_product_proto_msgTypes[158] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8330,7 +8286,7 @@ func (x *ListOfAuthorFilter_List) String() string { func (*ListOfAuthorFilter_List) ProtoMessage() {} func (x *ListOfAuthorFilter_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[159] + mi := &file_product_proto_msgTypes[158] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8362,7 +8318,7 @@ type ListOfAuthorInput_List struct { func (x *ListOfAuthorInput_List) Reset() { *x = ListOfAuthorInput_List{} - mi := &file_product_proto_msgTypes[160] + mi := &file_product_proto_msgTypes[159] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8374,7 +8330,7 @@ func (x *ListOfAuthorInput_List) String() string { func (*ListOfAuthorInput_List) ProtoMessage() {} func (x *ListOfAuthorInput_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[160] + mi := &file_product_proto_msgTypes[159] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8406,7 +8362,7 @@ type ListOfBlogPost_List struct { func (x *ListOfBlogPost_List) Reset() { *x = ListOfBlogPost_List{} - mi := &file_product_proto_msgTypes[161] + mi := &file_product_proto_msgTypes[160] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8418,7 +8374,7 @@ func (x *ListOfBlogPost_List) String() string { func (*ListOfBlogPost_List) ProtoMessage() {} func (x *ListOfBlogPost_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[161] + mi := &file_product_proto_msgTypes[160] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8450,7 +8406,7 @@ type ListOfBlogPostFilter_List struct { func (x *ListOfBlogPostFilter_List) Reset() { *x = ListOfBlogPostFilter_List{} - mi := &file_product_proto_msgTypes[162] + mi := &file_product_proto_msgTypes[161] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8462,7 +8418,7 @@ func (x *ListOfBlogPostFilter_List) String() string { func (*ListOfBlogPostFilter_List) ProtoMessage() {} func (x *ListOfBlogPostFilter_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[162] + mi := &file_product_proto_msgTypes[161] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8494,7 +8450,7 @@ type ListOfBlogPostInput_List struct { func (x *ListOfBlogPostInput_List) Reset() { *x = ListOfBlogPostInput_List{} - mi := &file_product_proto_msgTypes[163] + mi := &file_product_proto_msgTypes[162] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8506,7 +8462,7 @@ func (x *ListOfBlogPostInput_List) String() string { func (*ListOfBlogPostInput_List) ProtoMessage() {} func (x *ListOfBlogPostInput_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[163] + mi := &file_product_proto_msgTypes[162] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8538,7 +8494,7 @@ type ListOfBoolean_List struct { func (x *ListOfBoolean_List) Reset() { *x = ListOfBoolean_List{} - mi := &file_product_proto_msgTypes[164] + mi := &file_product_proto_msgTypes[163] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8550,7 +8506,7 @@ func (x *ListOfBoolean_List) String() string { func (*ListOfBoolean_List) ProtoMessage() {} func (x *ListOfBoolean_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[164] + mi := &file_product_proto_msgTypes[163] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8582,7 +8538,7 @@ type ListOfCategory_List struct { func (x *ListOfCategory_List) Reset() { *x = ListOfCategory_List{} - mi := &file_product_proto_msgTypes[165] + mi := &file_product_proto_msgTypes[164] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8594,7 +8550,7 @@ func (x *ListOfCategory_List) String() string { func (*ListOfCategory_List) ProtoMessage() {} func (x *ListOfCategory_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[165] + mi := &file_product_proto_msgTypes[164] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8626,7 +8582,7 @@ type ListOfCategoryInput_List struct { func (x *ListOfCategoryInput_List) Reset() { *x = ListOfCategoryInput_List{} - mi := &file_product_proto_msgTypes[166] + mi := &file_product_proto_msgTypes[165] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8638,7 +8594,7 @@ func (x *ListOfCategoryInput_List) String() string { func (*ListOfCategoryInput_List) ProtoMessage() {} func (x *ListOfCategoryInput_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[166] + mi := &file_product_proto_msgTypes[165] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8670,7 +8626,7 @@ type ListOfFloat_List struct { func (x *ListOfFloat_List) Reset() { *x = ListOfFloat_List{} - mi := &file_product_proto_msgTypes[167] + mi := &file_product_proto_msgTypes[166] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8682,7 +8638,7 @@ func (x *ListOfFloat_List) String() string { func (*ListOfFloat_List) ProtoMessage() {} func (x *ListOfFloat_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[167] + mi := &file_product_proto_msgTypes[166] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8714,7 +8670,7 @@ type ListOfListOfCategory_List struct { func (x *ListOfListOfCategory_List) Reset() { *x = ListOfListOfCategory_List{} - mi := &file_product_proto_msgTypes[168] + mi := &file_product_proto_msgTypes[167] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8726,7 +8682,7 @@ func (x *ListOfListOfCategory_List) String() string { func (*ListOfListOfCategory_List) ProtoMessage() {} func (x *ListOfListOfCategory_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[168] + mi := &file_product_proto_msgTypes[167] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8758,7 +8714,7 @@ type ListOfListOfCategoryInput_List struct { func (x *ListOfListOfCategoryInput_List) Reset() { *x = ListOfListOfCategoryInput_List{} - mi := &file_product_proto_msgTypes[169] + mi := &file_product_proto_msgTypes[168] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8770,7 +8726,7 @@ func (x *ListOfListOfCategoryInput_List) String() string { func (*ListOfListOfCategoryInput_List) ProtoMessage() {} func (x *ListOfListOfCategoryInput_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[169] + mi := &file_product_proto_msgTypes[168] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8802,7 +8758,7 @@ type ListOfListOfString_List struct { func (x *ListOfListOfString_List) Reset() { *x = ListOfListOfString_List{} - mi := &file_product_proto_msgTypes[170] + mi := &file_product_proto_msgTypes[169] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8814,7 +8770,7 @@ func (x *ListOfListOfString_List) String() string { func (*ListOfListOfString_List) ProtoMessage() {} func (x *ListOfListOfString_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[170] + mi := &file_product_proto_msgTypes[169] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8846,7 +8802,7 @@ type ListOfListOfUser_List struct { func (x *ListOfListOfUser_List) Reset() { *x = ListOfListOfUser_List{} - mi := &file_product_proto_msgTypes[171] + mi := &file_product_proto_msgTypes[170] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8858,7 +8814,7 @@ func (x *ListOfListOfUser_List) String() string { func (*ListOfListOfUser_List) ProtoMessage() {} func (x *ListOfListOfUser_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[171] + mi := &file_product_proto_msgTypes[170] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8890,7 +8846,7 @@ type ListOfListOfUserInput_List struct { func (x *ListOfListOfUserInput_List) Reset() { *x = ListOfListOfUserInput_List{} - mi := &file_product_proto_msgTypes[172] + mi := &file_product_proto_msgTypes[171] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8902,7 +8858,7 @@ func (x *ListOfListOfUserInput_List) String() string { func (*ListOfListOfUserInput_List) ProtoMessage() {} func (x *ListOfListOfUserInput_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[172] + mi := &file_product_proto_msgTypes[171] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8934,7 +8890,7 @@ type ListOfOrderLine_List struct { func (x *ListOfOrderLine_List) Reset() { *x = ListOfOrderLine_List{} - mi := &file_product_proto_msgTypes[173] + mi := &file_product_proto_msgTypes[172] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8946,7 +8902,7 @@ func (x *ListOfOrderLine_List) String() string { func (*ListOfOrderLine_List) ProtoMessage() {} func (x *ListOfOrderLine_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[173] + mi := &file_product_proto_msgTypes[172] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8978,7 +8934,7 @@ type ListOfProduct_List struct { func (x *ListOfProduct_List) Reset() { *x = ListOfProduct_List{} - mi := &file_product_proto_msgTypes[174] + mi := &file_product_proto_msgTypes[173] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8990,7 +8946,7 @@ func (x *ListOfProduct_List) String() string { func (*ListOfProduct_List) ProtoMessage() {} func (x *ListOfProduct_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[174] + mi := &file_product_proto_msgTypes[173] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9022,7 +8978,7 @@ type ListOfString_List struct { func (x *ListOfString_List) Reset() { *x = ListOfString_List{} - mi := &file_product_proto_msgTypes[175] + mi := &file_product_proto_msgTypes[174] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9034,7 +8990,7 @@ func (x *ListOfString_List) String() string { func (*ListOfString_List) ProtoMessage() {} func (x *ListOfString_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[175] + mi := &file_product_proto_msgTypes[174] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9066,7 +9022,7 @@ type ListOfUser_List struct { func (x *ListOfUser_List) Reset() { *x = ListOfUser_List{} - mi := &file_product_proto_msgTypes[176] + mi := &file_product_proto_msgTypes[175] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9078,7 +9034,7 @@ func (x *ListOfUser_List) String() string { func (*ListOfUser_List) ProtoMessage() {} func (x *ListOfUser_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[176] + mi := &file_product_proto_msgTypes[175] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9110,7 +9066,7 @@ type ListOfUserInput_List struct { func (x *ListOfUserInput_List) Reset() { *x = ListOfUserInput_List{} - mi := &file_product_proto_msgTypes[177] + mi := &file_product_proto_msgTypes[176] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9122,7 +9078,7 @@ func (x *ListOfUserInput_List) String() string { func (*ListOfUserInput_List) ProtoMessage() {} func (x *ListOfUserInput_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[177] + mi := &file_product_proto_msgTypes[176] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9417,13 +9373,11 @@ const file_product_proto_rawDesc = "" + "\afilters\x18\x01 \x01(\v2\x1d.productv1.ProductCountFilterR\afilters\"A\n" + "\x1bCategoryProductCountContext\x12\x0e\n" + "\x02id\x18\x01 \x01(\tR\x02id\x12\x12\n" + - "\x04name\x18\x02 \x01(\tR\x04name\"\xad\x01\n" + - "%ResolveCategoryProductCountRequestKey\x12@\n" + - "\acontext\x18\x01 \x01(\v2&.productv1.CategoryProductCountContextR\acontext\x12B\n" + + "\x04name\x18\x02 \x01(\tR\x04name\"\xaa\x01\n" + + "\"ResolveCategoryProductCountRequest\x12@\n" + + "\acontext\x18\x01 \x03(\v2&.productv1.CategoryProductCountContextR\acontext\x12B\n" + "\n" + - "field_args\x18\x02 \x01(\v2#.productv1.CategoryProductCountArgsR\tfieldArgs\"h\n" + - "\"ResolveCategoryProductCountRequest\x12B\n" + - "\x03key\x18\x01 \x03(\v20.productv1.ResolveCategoryProductCountRequestKeyR\x03key\"P\n" + + "field_args\x18\x02 \x01(\v2#.productv1.CategoryProductCountArgsR\tfieldArgs\"P\n" + ")ResolveCategoryProductCountResponseResult\x12#\n" + "\rproduct_count\x18\x01 \x01(\x05R\fproductCount\"s\n" + "#ResolveCategoryProductCountResponse\x12L\n" + @@ -9738,7 +9692,7 @@ func file_product_proto_rawDescGZIP() []byte { } var file_product_proto_enumTypes = make([]protoimpl.EnumInfo, 1) -var file_product_proto_msgTypes = make([]protoimpl.MessageInfo, 178) +var file_product_proto_msgTypes = make([]protoimpl.MessageInfo, 177) var file_product_proto_goTypes = []any{ (CategoryKind)(0), // 0: productv1.CategoryKind (*ListOfAuthorFilter)(nil), // 1: productv1.ListOfAuthorFilter @@ -9855,370 +9809,368 @@ var file_product_proto_goTypes = []any{ (*MutationBulkUpdateBlogPostsResponse)(nil), // 112: productv1.MutationBulkUpdateBlogPostsResponse (*CategoryProductCountArgs)(nil), // 113: productv1.CategoryProductCountArgs (*CategoryProductCountContext)(nil), // 114: productv1.CategoryProductCountContext - (*ResolveCategoryProductCountRequestKey)(nil), // 115: productv1.ResolveCategoryProductCountRequestKey - (*ResolveCategoryProductCountRequest)(nil), // 116: productv1.ResolveCategoryProductCountRequest - (*ResolveCategoryProductCountResponseResult)(nil), // 117: productv1.ResolveCategoryProductCountResponseResult - (*ResolveCategoryProductCountResponse)(nil), // 118: productv1.ResolveCategoryProductCountResponse - (*Product)(nil), // 119: productv1.Product - (*Storage)(nil), // 120: productv1.Storage - (*Warehouse)(nil), // 121: productv1.Warehouse - (*User)(nil), // 122: productv1.User - (*NestedTypeA)(nil), // 123: productv1.NestedTypeA - (*RecursiveType)(nil), // 124: productv1.RecursiveType - (*TypeWithMultipleFilterFields)(nil), // 125: productv1.TypeWithMultipleFilterFields - (*FilterTypeInput)(nil), // 126: productv1.FilterTypeInput - (*ComplexFilterTypeInput)(nil), // 127: productv1.ComplexFilterTypeInput - (*TypeWithComplexFilterInput)(nil), // 128: productv1.TypeWithComplexFilterInput - (*OrderInput)(nil), // 129: productv1.OrderInput - (*Order)(nil), // 130: productv1.Order - (*Category)(nil), // 131: productv1.Category - (*CategoryFilter)(nil), // 132: productv1.CategoryFilter - (*Animal)(nil), // 133: productv1.Animal - (*SearchInput)(nil), // 134: productv1.SearchInput - (*SearchResult)(nil), // 135: productv1.SearchResult - (*NullableFieldsType)(nil), // 136: productv1.NullableFieldsType - (*NullableFieldsFilter)(nil), // 137: productv1.NullableFieldsFilter - (*BlogPost)(nil), // 138: productv1.BlogPost - (*BlogPostFilter)(nil), // 139: productv1.BlogPostFilter - (*Author)(nil), // 140: productv1.Author - (*AuthorFilter)(nil), // 141: productv1.AuthorFilter - (*UserInput)(nil), // 142: productv1.UserInput - (*ActionInput)(nil), // 143: productv1.ActionInput - (*ActionResult)(nil), // 144: productv1.ActionResult - (*NullableFieldsInput)(nil), // 145: productv1.NullableFieldsInput - (*BlogPostInput)(nil), // 146: productv1.BlogPostInput - (*AuthorInput)(nil), // 147: productv1.AuthorInput - (*NestedTypeB)(nil), // 148: productv1.NestedTypeB - (*NestedTypeC)(nil), // 149: productv1.NestedTypeC - (*FilterType)(nil), // 150: productv1.FilterType - (*Pagination)(nil), // 151: productv1.Pagination - (*OrderLineInput)(nil), // 152: productv1.OrderLineInput - (*OrderLine)(nil), // 153: productv1.OrderLine - (*Cat)(nil), // 154: productv1.Cat - (*Dog)(nil), // 155: productv1.Dog - (*ActionSuccess)(nil), // 156: productv1.ActionSuccess - (*ActionError)(nil), // 157: productv1.ActionError - (*CategoryInput)(nil), // 158: productv1.CategoryInput - (*ProductCountFilter)(nil), // 159: productv1.ProductCountFilter - (*ListOfAuthorFilter_List)(nil), // 160: productv1.ListOfAuthorFilter.List - (*ListOfAuthorInput_List)(nil), // 161: productv1.ListOfAuthorInput.List - (*ListOfBlogPost_List)(nil), // 162: productv1.ListOfBlogPost.List - (*ListOfBlogPostFilter_List)(nil), // 163: productv1.ListOfBlogPostFilter.List - (*ListOfBlogPostInput_List)(nil), // 164: productv1.ListOfBlogPostInput.List - (*ListOfBoolean_List)(nil), // 165: productv1.ListOfBoolean.List - (*ListOfCategory_List)(nil), // 166: productv1.ListOfCategory.List - (*ListOfCategoryInput_List)(nil), // 167: productv1.ListOfCategoryInput.List - (*ListOfFloat_List)(nil), // 168: productv1.ListOfFloat.List - (*ListOfListOfCategory_List)(nil), // 169: productv1.ListOfListOfCategory.List - (*ListOfListOfCategoryInput_List)(nil), // 170: productv1.ListOfListOfCategoryInput.List - (*ListOfListOfString_List)(nil), // 171: productv1.ListOfListOfString.List - (*ListOfListOfUser_List)(nil), // 172: productv1.ListOfListOfUser.List - (*ListOfListOfUserInput_List)(nil), // 173: productv1.ListOfListOfUserInput.List - (*ListOfOrderLine_List)(nil), // 174: productv1.ListOfOrderLine.List - (*ListOfProduct_List)(nil), // 175: productv1.ListOfProduct.List - (*ListOfString_List)(nil), // 176: productv1.ListOfString.List - (*ListOfUser_List)(nil), // 177: productv1.ListOfUser.List - (*ListOfUserInput_List)(nil), // 178: productv1.ListOfUserInput.List - (*wrapperspb.Int32Value)(nil), // 179: google.protobuf.Int32Value - (*wrapperspb.StringValue)(nil), // 180: google.protobuf.StringValue - (*wrapperspb.DoubleValue)(nil), // 181: google.protobuf.DoubleValue - (*wrapperspb.BoolValue)(nil), // 182: google.protobuf.BoolValue + (*ResolveCategoryProductCountRequest)(nil), // 115: productv1.ResolveCategoryProductCountRequest + (*ResolveCategoryProductCountResponseResult)(nil), // 116: productv1.ResolveCategoryProductCountResponseResult + (*ResolveCategoryProductCountResponse)(nil), // 117: productv1.ResolveCategoryProductCountResponse + (*Product)(nil), // 118: productv1.Product + (*Storage)(nil), // 119: productv1.Storage + (*Warehouse)(nil), // 120: productv1.Warehouse + (*User)(nil), // 121: productv1.User + (*NestedTypeA)(nil), // 122: productv1.NestedTypeA + (*RecursiveType)(nil), // 123: productv1.RecursiveType + (*TypeWithMultipleFilterFields)(nil), // 124: productv1.TypeWithMultipleFilterFields + (*FilterTypeInput)(nil), // 125: productv1.FilterTypeInput + (*ComplexFilterTypeInput)(nil), // 126: productv1.ComplexFilterTypeInput + (*TypeWithComplexFilterInput)(nil), // 127: productv1.TypeWithComplexFilterInput + (*OrderInput)(nil), // 128: productv1.OrderInput + (*Order)(nil), // 129: productv1.Order + (*Category)(nil), // 130: productv1.Category + (*CategoryFilter)(nil), // 131: productv1.CategoryFilter + (*Animal)(nil), // 132: productv1.Animal + (*SearchInput)(nil), // 133: productv1.SearchInput + (*SearchResult)(nil), // 134: productv1.SearchResult + (*NullableFieldsType)(nil), // 135: productv1.NullableFieldsType + (*NullableFieldsFilter)(nil), // 136: productv1.NullableFieldsFilter + (*BlogPost)(nil), // 137: productv1.BlogPost + (*BlogPostFilter)(nil), // 138: productv1.BlogPostFilter + (*Author)(nil), // 139: productv1.Author + (*AuthorFilter)(nil), // 140: productv1.AuthorFilter + (*UserInput)(nil), // 141: productv1.UserInput + (*ActionInput)(nil), // 142: productv1.ActionInput + (*ActionResult)(nil), // 143: productv1.ActionResult + (*NullableFieldsInput)(nil), // 144: productv1.NullableFieldsInput + (*BlogPostInput)(nil), // 145: productv1.BlogPostInput + (*AuthorInput)(nil), // 146: productv1.AuthorInput + (*NestedTypeB)(nil), // 147: productv1.NestedTypeB + (*NestedTypeC)(nil), // 148: productv1.NestedTypeC + (*FilterType)(nil), // 149: productv1.FilterType + (*Pagination)(nil), // 150: productv1.Pagination + (*OrderLineInput)(nil), // 151: productv1.OrderLineInput + (*OrderLine)(nil), // 152: productv1.OrderLine + (*Cat)(nil), // 153: productv1.Cat + (*Dog)(nil), // 154: productv1.Dog + (*ActionSuccess)(nil), // 155: productv1.ActionSuccess + (*ActionError)(nil), // 156: productv1.ActionError + (*CategoryInput)(nil), // 157: productv1.CategoryInput + (*ProductCountFilter)(nil), // 158: productv1.ProductCountFilter + (*ListOfAuthorFilter_List)(nil), // 159: productv1.ListOfAuthorFilter.List + (*ListOfAuthorInput_List)(nil), // 160: productv1.ListOfAuthorInput.List + (*ListOfBlogPost_List)(nil), // 161: productv1.ListOfBlogPost.List + (*ListOfBlogPostFilter_List)(nil), // 162: productv1.ListOfBlogPostFilter.List + (*ListOfBlogPostInput_List)(nil), // 163: productv1.ListOfBlogPostInput.List + (*ListOfBoolean_List)(nil), // 164: productv1.ListOfBoolean.List + (*ListOfCategory_List)(nil), // 165: productv1.ListOfCategory.List + (*ListOfCategoryInput_List)(nil), // 166: productv1.ListOfCategoryInput.List + (*ListOfFloat_List)(nil), // 167: productv1.ListOfFloat.List + (*ListOfListOfCategory_List)(nil), // 168: productv1.ListOfListOfCategory.List + (*ListOfListOfCategoryInput_List)(nil), // 169: productv1.ListOfListOfCategoryInput.List + (*ListOfListOfString_List)(nil), // 170: productv1.ListOfListOfString.List + (*ListOfListOfUser_List)(nil), // 171: productv1.ListOfListOfUser.List + (*ListOfListOfUserInput_List)(nil), // 172: productv1.ListOfListOfUserInput.List + (*ListOfOrderLine_List)(nil), // 173: productv1.ListOfOrderLine.List + (*ListOfProduct_List)(nil), // 174: productv1.ListOfProduct.List + (*ListOfString_List)(nil), // 175: productv1.ListOfString.List + (*ListOfUser_List)(nil), // 176: productv1.ListOfUser.List + (*ListOfUserInput_List)(nil), // 177: productv1.ListOfUserInput.List + (*wrapperspb.Int32Value)(nil), // 178: google.protobuf.Int32Value + (*wrapperspb.StringValue)(nil), // 179: google.protobuf.StringValue + (*wrapperspb.DoubleValue)(nil), // 180: google.protobuf.DoubleValue + (*wrapperspb.BoolValue)(nil), // 181: google.protobuf.BoolValue } var file_product_proto_depIdxs = []int32{ - 160, // 0: productv1.ListOfAuthorFilter.list:type_name -> productv1.ListOfAuthorFilter.List - 161, // 1: productv1.ListOfAuthorInput.list:type_name -> productv1.ListOfAuthorInput.List - 162, // 2: productv1.ListOfBlogPost.list:type_name -> productv1.ListOfBlogPost.List - 163, // 3: productv1.ListOfBlogPostFilter.list:type_name -> productv1.ListOfBlogPostFilter.List - 164, // 4: productv1.ListOfBlogPostInput.list:type_name -> productv1.ListOfBlogPostInput.List - 165, // 5: productv1.ListOfBoolean.list:type_name -> productv1.ListOfBoolean.List - 166, // 6: productv1.ListOfCategory.list:type_name -> productv1.ListOfCategory.List - 167, // 7: productv1.ListOfCategoryInput.list:type_name -> productv1.ListOfCategoryInput.List - 168, // 8: productv1.ListOfFloat.list:type_name -> productv1.ListOfFloat.List - 169, // 9: productv1.ListOfListOfCategory.list:type_name -> productv1.ListOfListOfCategory.List - 170, // 10: productv1.ListOfListOfCategoryInput.list:type_name -> productv1.ListOfListOfCategoryInput.List - 171, // 11: productv1.ListOfListOfString.list:type_name -> productv1.ListOfListOfString.List - 172, // 12: productv1.ListOfListOfUser.list:type_name -> productv1.ListOfListOfUser.List - 173, // 13: productv1.ListOfListOfUserInput.list:type_name -> productv1.ListOfListOfUserInput.List - 174, // 14: productv1.ListOfOrderLine.list:type_name -> productv1.ListOfOrderLine.List - 175, // 15: productv1.ListOfProduct.list:type_name -> productv1.ListOfProduct.List - 176, // 16: productv1.ListOfString.list:type_name -> productv1.ListOfString.List - 177, // 17: productv1.ListOfUser.list:type_name -> productv1.ListOfUser.List - 178, // 18: productv1.ListOfUserInput.list:type_name -> productv1.ListOfUserInput.List + 159, // 0: productv1.ListOfAuthorFilter.list:type_name -> productv1.ListOfAuthorFilter.List + 160, // 1: productv1.ListOfAuthorInput.list:type_name -> productv1.ListOfAuthorInput.List + 161, // 2: productv1.ListOfBlogPost.list:type_name -> productv1.ListOfBlogPost.List + 162, // 3: productv1.ListOfBlogPostFilter.list:type_name -> productv1.ListOfBlogPostFilter.List + 163, // 4: productv1.ListOfBlogPostInput.list:type_name -> productv1.ListOfBlogPostInput.List + 164, // 5: productv1.ListOfBoolean.list:type_name -> productv1.ListOfBoolean.List + 165, // 6: productv1.ListOfCategory.list:type_name -> productv1.ListOfCategory.List + 166, // 7: productv1.ListOfCategoryInput.list:type_name -> productv1.ListOfCategoryInput.List + 167, // 8: productv1.ListOfFloat.list:type_name -> productv1.ListOfFloat.List + 168, // 9: productv1.ListOfListOfCategory.list:type_name -> productv1.ListOfListOfCategory.List + 169, // 10: productv1.ListOfListOfCategoryInput.list:type_name -> productv1.ListOfListOfCategoryInput.List + 170, // 11: productv1.ListOfListOfString.list:type_name -> productv1.ListOfListOfString.List + 171, // 12: productv1.ListOfListOfUser.list:type_name -> productv1.ListOfListOfUser.List + 172, // 13: productv1.ListOfListOfUserInput.list:type_name -> productv1.ListOfListOfUserInput.List + 173, // 14: productv1.ListOfOrderLine.list:type_name -> productv1.ListOfOrderLine.List + 174, // 15: productv1.ListOfProduct.list:type_name -> productv1.ListOfProduct.List + 175, // 16: productv1.ListOfString.list:type_name -> productv1.ListOfString.List + 176, // 17: productv1.ListOfUser.list:type_name -> productv1.ListOfUser.List + 177, // 18: productv1.ListOfUserInput.list:type_name -> productv1.ListOfUserInput.List 20, // 19: productv1.LookupProductByIdRequest.keys:type_name -> productv1.LookupProductByIdRequestKey - 119, // 20: productv1.LookupProductByIdResponse.result:type_name -> productv1.Product + 118, // 20: productv1.LookupProductByIdResponse.result:type_name -> productv1.Product 23, // 21: productv1.LookupStorageByIdRequest.keys:type_name -> productv1.LookupStorageByIdRequestKey - 120, // 22: productv1.LookupStorageByIdResponse.result:type_name -> productv1.Storage + 119, // 22: productv1.LookupStorageByIdResponse.result:type_name -> productv1.Storage 26, // 23: productv1.LookupWarehouseByIdRequest.keys:type_name -> productv1.LookupWarehouseByIdRequestKey - 121, // 24: productv1.LookupWarehouseByIdResponse.result:type_name -> productv1.Warehouse - 122, // 25: productv1.QueryUsersResponse.users:type_name -> productv1.User - 122, // 26: productv1.QueryUserResponse.user:type_name -> productv1.User - 123, // 27: productv1.QueryNestedTypeResponse.nested_type:type_name -> productv1.NestedTypeA - 124, // 28: productv1.QueryRecursiveTypeResponse.recursive_type:type_name -> productv1.RecursiveType - 125, // 29: productv1.QueryTypeFilterWithArgumentsResponse.type_filter_with_arguments:type_name -> productv1.TypeWithMultipleFilterFields - 126, // 30: productv1.QueryTypeWithMultipleFilterFieldsRequest.filter:type_name -> productv1.FilterTypeInput - 125, // 31: productv1.QueryTypeWithMultipleFilterFieldsResponse.type_with_multiple_filter_fields:type_name -> productv1.TypeWithMultipleFilterFields - 127, // 32: productv1.QueryComplexFilterTypeRequest.filter:type_name -> productv1.ComplexFilterTypeInput - 128, // 33: productv1.QueryComplexFilterTypeResponse.complex_filter_type:type_name -> productv1.TypeWithComplexFilterInput - 129, // 34: productv1.QueryCalculateTotalsRequest.orders:type_name -> productv1.OrderInput - 130, // 35: productv1.QueryCalculateTotalsResponse.calculate_totals:type_name -> productv1.Order - 131, // 36: productv1.QueryCategoriesResponse.categories:type_name -> productv1.Category + 120, // 24: productv1.LookupWarehouseByIdResponse.result:type_name -> productv1.Warehouse + 121, // 25: productv1.QueryUsersResponse.users:type_name -> productv1.User + 121, // 26: productv1.QueryUserResponse.user:type_name -> productv1.User + 122, // 27: productv1.QueryNestedTypeResponse.nested_type:type_name -> productv1.NestedTypeA + 123, // 28: productv1.QueryRecursiveTypeResponse.recursive_type:type_name -> productv1.RecursiveType + 124, // 29: productv1.QueryTypeFilterWithArgumentsResponse.type_filter_with_arguments:type_name -> productv1.TypeWithMultipleFilterFields + 125, // 30: productv1.QueryTypeWithMultipleFilterFieldsRequest.filter:type_name -> productv1.FilterTypeInput + 124, // 31: productv1.QueryTypeWithMultipleFilterFieldsResponse.type_with_multiple_filter_fields:type_name -> productv1.TypeWithMultipleFilterFields + 126, // 32: productv1.QueryComplexFilterTypeRequest.filter:type_name -> productv1.ComplexFilterTypeInput + 127, // 33: productv1.QueryComplexFilterTypeResponse.complex_filter_type:type_name -> productv1.TypeWithComplexFilterInput + 128, // 34: productv1.QueryCalculateTotalsRequest.orders:type_name -> productv1.OrderInput + 129, // 35: productv1.QueryCalculateTotalsResponse.calculate_totals:type_name -> productv1.Order + 130, // 36: productv1.QueryCategoriesResponse.categories:type_name -> productv1.Category 0, // 37: productv1.QueryCategoriesByKindRequest.kind:type_name -> productv1.CategoryKind - 131, // 38: productv1.QueryCategoriesByKindResponse.categories_by_kind:type_name -> productv1.Category + 130, // 38: productv1.QueryCategoriesByKindResponse.categories_by_kind:type_name -> productv1.Category 0, // 39: productv1.QueryCategoriesByKindsRequest.kinds:type_name -> productv1.CategoryKind - 131, // 40: productv1.QueryCategoriesByKindsResponse.categories_by_kinds:type_name -> productv1.Category - 132, // 41: productv1.QueryFilterCategoriesRequest.filter:type_name -> productv1.CategoryFilter - 131, // 42: productv1.QueryFilterCategoriesResponse.filter_categories:type_name -> productv1.Category - 133, // 43: productv1.QueryRandomPetResponse.random_pet:type_name -> productv1.Animal - 133, // 44: productv1.QueryAllPetsResponse.all_pets:type_name -> productv1.Animal - 134, // 45: productv1.QuerySearchRequest.input:type_name -> productv1.SearchInput - 135, // 46: productv1.QuerySearchResponse.search:type_name -> productv1.SearchResult - 135, // 47: productv1.QueryRandomSearchResultResponse.random_search_result:type_name -> productv1.SearchResult - 136, // 48: productv1.QueryNullableFieldsTypeResponse.nullable_fields_type:type_name -> productv1.NullableFieldsType - 136, // 49: productv1.QueryNullableFieldsTypeByIdResponse.nullable_fields_type_by_id:type_name -> productv1.NullableFieldsType - 137, // 50: productv1.QueryNullableFieldsTypeWithFilterRequest.filter:type_name -> productv1.NullableFieldsFilter - 136, // 51: productv1.QueryNullableFieldsTypeWithFilterResponse.nullable_fields_type_with_filter:type_name -> productv1.NullableFieldsType - 136, // 52: productv1.QueryAllNullableFieldsTypesResponse.all_nullable_fields_types:type_name -> productv1.NullableFieldsType - 138, // 53: productv1.QueryBlogPostResponse.blog_post:type_name -> productv1.BlogPost - 138, // 54: productv1.QueryBlogPostByIdResponse.blog_post_by_id:type_name -> productv1.BlogPost - 139, // 55: productv1.QueryBlogPostsWithFilterRequest.filter:type_name -> productv1.BlogPostFilter - 138, // 56: productv1.QueryBlogPostsWithFilterResponse.blog_posts_with_filter:type_name -> productv1.BlogPost - 138, // 57: productv1.QueryAllBlogPostsResponse.all_blog_posts:type_name -> productv1.BlogPost - 140, // 58: productv1.QueryAuthorResponse.author:type_name -> productv1.Author - 140, // 59: productv1.QueryAuthorByIdResponse.author_by_id:type_name -> productv1.Author - 141, // 60: productv1.QueryAuthorsWithFilterRequest.filter:type_name -> productv1.AuthorFilter - 140, // 61: productv1.QueryAuthorsWithFilterResponse.authors_with_filter:type_name -> productv1.Author - 140, // 62: productv1.QueryAllAuthorsResponse.all_authors:type_name -> productv1.Author + 130, // 40: productv1.QueryCategoriesByKindsResponse.categories_by_kinds:type_name -> productv1.Category + 131, // 41: productv1.QueryFilterCategoriesRequest.filter:type_name -> productv1.CategoryFilter + 130, // 42: productv1.QueryFilterCategoriesResponse.filter_categories:type_name -> productv1.Category + 132, // 43: productv1.QueryRandomPetResponse.random_pet:type_name -> productv1.Animal + 132, // 44: productv1.QueryAllPetsResponse.all_pets:type_name -> productv1.Animal + 133, // 45: productv1.QuerySearchRequest.input:type_name -> productv1.SearchInput + 134, // 46: productv1.QuerySearchResponse.search:type_name -> productv1.SearchResult + 134, // 47: productv1.QueryRandomSearchResultResponse.random_search_result:type_name -> productv1.SearchResult + 135, // 48: productv1.QueryNullableFieldsTypeResponse.nullable_fields_type:type_name -> productv1.NullableFieldsType + 135, // 49: productv1.QueryNullableFieldsTypeByIdResponse.nullable_fields_type_by_id:type_name -> productv1.NullableFieldsType + 136, // 50: productv1.QueryNullableFieldsTypeWithFilterRequest.filter:type_name -> productv1.NullableFieldsFilter + 135, // 51: productv1.QueryNullableFieldsTypeWithFilterResponse.nullable_fields_type_with_filter:type_name -> productv1.NullableFieldsType + 135, // 52: productv1.QueryAllNullableFieldsTypesResponse.all_nullable_fields_types:type_name -> productv1.NullableFieldsType + 137, // 53: productv1.QueryBlogPostResponse.blog_post:type_name -> productv1.BlogPost + 137, // 54: productv1.QueryBlogPostByIdResponse.blog_post_by_id:type_name -> productv1.BlogPost + 138, // 55: productv1.QueryBlogPostsWithFilterRequest.filter:type_name -> productv1.BlogPostFilter + 137, // 56: productv1.QueryBlogPostsWithFilterResponse.blog_posts_with_filter:type_name -> productv1.BlogPost + 137, // 57: productv1.QueryAllBlogPostsResponse.all_blog_posts:type_name -> productv1.BlogPost + 139, // 58: productv1.QueryAuthorResponse.author:type_name -> productv1.Author + 139, // 59: productv1.QueryAuthorByIdResponse.author_by_id:type_name -> productv1.Author + 140, // 60: productv1.QueryAuthorsWithFilterRequest.filter:type_name -> productv1.AuthorFilter + 139, // 61: productv1.QueryAuthorsWithFilterResponse.authors_with_filter:type_name -> productv1.Author + 139, // 62: productv1.QueryAllAuthorsResponse.all_authors:type_name -> productv1.Author 1, // 63: productv1.QueryBulkSearchAuthorsRequest.filters:type_name -> productv1.ListOfAuthorFilter - 140, // 64: productv1.QueryBulkSearchAuthorsResponse.bulk_search_authors:type_name -> productv1.Author + 139, // 64: productv1.QueryBulkSearchAuthorsResponse.bulk_search_authors:type_name -> productv1.Author 4, // 65: productv1.QueryBulkSearchBlogPostsRequest.filters:type_name -> productv1.ListOfBlogPostFilter - 138, // 66: productv1.QueryBulkSearchBlogPostsResponse.bulk_search_blog_posts:type_name -> productv1.BlogPost - 142, // 67: productv1.MutationCreateUserRequest.input:type_name -> productv1.UserInput - 122, // 68: productv1.MutationCreateUserResponse.create_user:type_name -> productv1.User - 143, // 69: productv1.MutationPerformActionRequest.input:type_name -> productv1.ActionInput - 144, // 70: productv1.MutationPerformActionResponse.perform_action:type_name -> productv1.ActionResult - 145, // 71: productv1.MutationCreateNullableFieldsTypeRequest.input:type_name -> productv1.NullableFieldsInput - 136, // 72: productv1.MutationCreateNullableFieldsTypeResponse.create_nullable_fields_type:type_name -> productv1.NullableFieldsType - 145, // 73: productv1.MutationUpdateNullableFieldsTypeRequest.input:type_name -> productv1.NullableFieldsInput - 136, // 74: productv1.MutationUpdateNullableFieldsTypeResponse.update_nullable_fields_type:type_name -> productv1.NullableFieldsType - 146, // 75: productv1.MutationCreateBlogPostRequest.input:type_name -> productv1.BlogPostInput - 138, // 76: productv1.MutationCreateBlogPostResponse.create_blog_post:type_name -> productv1.BlogPost - 146, // 77: productv1.MutationUpdateBlogPostRequest.input:type_name -> productv1.BlogPostInput - 138, // 78: productv1.MutationUpdateBlogPostResponse.update_blog_post:type_name -> productv1.BlogPost - 147, // 79: productv1.MutationCreateAuthorRequest.input:type_name -> productv1.AuthorInput - 140, // 80: productv1.MutationCreateAuthorResponse.create_author:type_name -> productv1.Author - 147, // 81: productv1.MutationUpdateAuthorRequest.input:type_name -> productv1.AuthorInput - 140, // 82: productv1.MutationUpdateAuthorResponse.update_author:type_name -> productv1.Author + 137, // 66: productv1.QueryBulkSearchBlogPostsResponse.bulk_search_blog_posts:type_name -> productv1.BlogPost + 141, // 67: productv1.MutationCreateUserRequest.input:type_name -> productv1.UserInput + 121, // 68: productv1.MutationCreateUserResponse.create_user:type_name -> productv1.User + 142, // 69: productv1.MutationPerformActionRequest.input:type_name -> productv1.ActionInput + 143, // 70: productv1.MutationPerformActionResponse.perform_action:type_name -> productv1.ActionResult + 144, // 71: productv1.MutationCreateNullableFieldsTypeRequest.input:type_name -> productv1.NullableFieldsInput + 135, // 72: productv1.MutationCreateNullableFieldsTypeResponse.create_nullable_fields_type:type_name -> productv1.NullableFieldsType + 144, // 73: productv1.MutationUpdateNullableFieldsTypeRequest.input:type_name -> productv1.NullableFieldsInput + 135, // 74: productv1.MutationUpdateNullableFieldsTypeResponse.update_nullable_fields_type:type_name -> productv1.NullableFieldsType + 145, // 75: productv1.MutationCreateBlogPostRequest.input:type_name -> productv1.BlogPostInput + 137, // 76: productv1.MutationCreateBlogPostResponse.create_blog_post:type_name -> productv1.BlogPost + 145, // 77: productv1.MutationUpdateBlogPostRequest.input:type_name -> productv1.BlogPostInput + 137, // 78: productv1.MutationUpdateBlogPostResponse.update_blog_post:type_name -> productv1.BlogPost + 146, // 79: productv1.MutationCreateAuthorRequest.input:type_name -> productv1.AuthorInput + 139, // 80: productv1.MutationCreateAuthorResponse.create_author:type_name -> productv1.Author + 146, // 81: productv1.MutationUpdateAuthorRequest.input:type_name -> productv1.AuthorInput + 139, // 82: productv1.MutationUpdateAuthorResponse.update_author:type_name -> productv1.Author 2, // 83: productv1.MutationBulkCreateAuthorsRequest.authors:type_name -> productv1.ListOfAuthorInput - 140, // 84: productv1.MutationBulkCreateAuthorsResponse.bulk_create_authors:type_name -> productv1.Author + 139, // 84: productv1.MutationBulkCreateAuthorsResponse.bulk_create_authors:type_name -> productv1.Author 2, // 85: productv1.MutationBulkUpdateAuthorsRequest.authors:type_name -> productv1.ListOfAuthorInput - 140, // 86: productv1.MutationBulkUpdateAuthorsResponse.bulk_update_authors:type_name -> productv1.Author + 139, // 86: productv1.MutationBulkUpdateAuthorsResponse.bulk_update_authors:type_name -> productv1.Author 5, // 87: productv1.MutationBulkCreateBlogPostsRequest.blog_posts:type_name -> productv1.ListOfBlogPostInput - 138, // 88: productv1.MutationBulkCreateBlogPostsResponse.bulk_create_blog_posts:type_name -> productv1.BlogPost + 137, // 88: productv1.MutationBulkCreateBlogPostsResponse.bulk_create_blog_posts:type_name -> productv1.BlogPost 5, // 89: productv1.MutationBulkUpdateBlogPostsRequest.blog_posts:type_name -> productv1.ListOfBlogPostInput - 138, // 90: productv1.MutationBulkUpdateBlogPostsResponse.bulk_update_blog_posts:type_name -> productv1.BlogPost - 159, // 91: productv1.CategoryProductCountArgs.filters:type_name -> productv1.ProductCountFilter - 114, // 92: productv1.ResolveCategoryProductCountRequestKey.context:type_name -> productv1.CategoryProductCountContext - 113, // 93: productv1.ResolveCategoryProductCountRequestKey.field_args:type_name -> productv1.CategoryProductCountArgs - 115, // 94: productv1.ResolveCategoryProductCountRequest.key:type_name -> productv1.ResolveCategoryProductCountRequestKey - 117, // 95: productv1.ResolveCategoryProductCountResponse.result:type_name -> productv1.ResolveCategoryProductCountResponseResult - 148, // 96: productv1.NestedTypeA.b:type_name -> productv1.NestedTypeB - 124, // 97: productv1.RecursiveType.recursive_type:type_name -> productv1.RecursiveType - 150, // 98: productv1.ComplexFilterTypeInput.filter:type_name -> productv1.FilterType - 152, // 99: productv1.OrderInput.lines:type_name -> productv1.OrderLineInput - 15, // 100: productv1.Order.order_lines:type_name -> productv1.ListOfOrderLine - 0, // 101: productv1.Category.kind:type_name -> productv1.CategoryKind - 0, // 102: productv1.CategoryFilter.category:type_name -> productv1.CategoryKind - 151, // 103: productv1.CategoryFilter.pagination:type_name -> productv1.Pagination - 154, // 104: productv1.Animal.cat:type_name -> productv1.Cat - 155, // 105: productv1.Animal.dog:type_name -> productv1.Dog - 179, // 106: productv1.SearchInput.limit:type_name -> google.protobuf.Int32Value - 119, // 107: productv1.SearchResult.product:type_name -> productv1.Product - 122, // 108: productv1.SearchResult.user:type_name -> productv1.User - 131, // 109: productv1.SearchResult.category:type_name -> productv1.Category - 180, // 110: productv1.NullableFieldsType.optional_string:type_name -> google.protobuf.StringValue - 179, // 111: productv1.NullableFieldsType.optional_int:type_name -> google.protobuf.Int32Value - 181, // 112: productv1.NullableFieldsType.optional_float:type_name -> google.protobuf.DoubleValue - 182, // 113: productv1.NullableFieldsType.optional_boolean:type_name -> google.protobuf.BoolValue - 180, // 114: productv1.NullableFieldsFilter.name:type_name -> google.protobuf.StringValue - 180, // 115: productv1.NullableFieldsFilter.optional_string:type_name -> google.protobuf.StringValue - 182, // 116: productv1.NullableFieldsFilter.include_nulls:type_name -> google.protobuf.BoolValue - 17, // 117: productv1.BlogPost.optional_tags:type_name -> productv1.ListOfString - 17, // 118: productv1.BlogPost.keywords:type_name -> productv1.ListOfString - 9, // 119: productv1.BlogPost.ratings:type_name -> productv1.ListOfFloat - 6, // 120: productv1.BlogPost.is_published:type_name -> productv1.ListOfBoolean - 12, // 121: productv1.BlogPost.tag_groups:type_name -> productv1.ListOfListOfString - 12, // 122: productv1.BlogPost.related_topics:type_name -> productv1.ListOfListOfString - 12, // 123: productv1.BlogPost.comment_threads:type_name -> productv1.ListOfListOfString - 12, // 124: productv1.BlogPost.suggestions:type_name -> productv1.ListOfListOfString - 131, // 125: productv1.BlogPost.related_categories:type_name -> productv1.Category - 122, // 126: productv1.BlogPost.contributors:type_name -> productv1.User - 16, // 127: productv1.BlogPost.mentioned_products:type_name -> productv1.ListOfProduct - 18, // 128: productv1.BlogPost.mentioned_users:type_name -> productv1.ListOfUser - 10, // 129: productv1.BlogPost.category_groups:type_name -> productv1.ListOfListOfCategory - 13, // 130: productv1.BlogPost.contributor_teams:type_name -> productv1.ListOfListOfUser - 180, // 131: productv1.BlogPostFilter.title:type_name -> google.protobuf.StringValue - 182, // 132: productv1.BlogPostFilter.has_categories:type_name -> google.protobuf.BoolValue - 179, // 133: productv1.BlogPostFilter.min_tags:type_name -> google.protobuf.Int32Value - 180, // 134: productv1.Author.email:type_name -> google.protobuf.StringValue - 17, // 135: productv1.Author.social_links:type_name -> productv1.ListOfString - 12, // 136: productv1.Author.teams_by_project:type_name -> productv1.ListOfListOfString - 12, // 137: productv1.Author.collaborations:type_name -> productv1.ListOfListOfString - 3, // 138: productv1.Author.written_posts:type_name -> productv1.ListOfBlogPost - 131, // 139: productv1.Author.favorite_categories:type_name -> productv1.Category - 18, // 140: productv1.Author.related_authors:type_name -> productv1.ListOfUser - 16, // 141: productv1.Author.product_reviews:type_name -> productv1.ListOfProduct - 13, // 142: productv1.Author.author_groups:type_name -> productv1.ListOfListOfUser - 10, // 143: productv1.Author.category_preferences:type_name -> productv1.ListOfListOfCategory - 13, // 144: productv1.Author.project_teams:type_name -> productv1.ListOfListOfUser - 180, // 145: productv1.AuthorFilter.name:type_name -> google.protobuf.StringValue - 182, // 146: productv1.AuthorFilter.has_teams:type_name -> google.protobuf.BoolValue - 179, // 147: productv1.AuthorFilter.skill_count:type_name -> google.protobuf.Int32Value - 156, // 148: productv1.ActionResult.action_success:type_name -> productv1.ActionSuccess - 157, // 149: productv1.ActionResult.action_error:type_name -> productv1.ActionError - 180, // 150: productv1.NullableFieldsInput.optional_string:type_name -> google.protobuf.StringValue - 179, // 151: productv1.NullableFieldsInput.optional_int:type_name -> google.protobuf.Int32Value - 181, // 152: productv1.NullableFieldsInput.optional_float:type_name -> google.protobuf.DoubleValue - 182, // 153: productv1.NullableFieldsInput.optional_boolean:type_name -> google.protobuf.BoolValue - 17, // 154: productv1.BlogPostInput.optional_tags:type_name -> productv1.ListOfString - 17, // 155: productv1.BlogPostInput.keywords:type_name -> productv1.ListOfString - 9, // 156: productv1.BlogPostInput.ratings:type_name -> productv1.ListOfFloat - 6, // 157: productv1.BlogPostInput.is_published:type_name -> productv1.ListOfBoolean - 12, // 158: productv1.BlogPostInput.tag_groups:type_name -> productv1.ListOfListOfString - 12, // 159: productv1.BlogPostInput.related_topics:type_name -> productv1.ListOfListOfString - 12, // 160: productv1.BlogPostInput.comment_threads:type_name -> productv1.ListOfListOfString - 12, // 161: productv1.BlogPostInput.suggestions:type_name -> productv1.ListOfListOfString - 8, // 162: productv1.BlogPostInput.related_categories:type_name -> productv1.ListOfCategoryInput - 19, // 163: productv1.BlogPostInput.contributors:type_name -> productv1.ListOfUserInput - 11, // 164: productv1.BlogPostInput.category_groups:type_name -> productv1.ListOfListOfCategoryInput - 180, // 165: productv1.AuthorInput.email:type_name -> google.protobuf.StringValue - 17, // 166: productv1.AuthorInput.social_links:type_name -> productv1.ListOfString - 12, // 167: productv1.AuthorInput.teams_by_project:type_name -> productv1.ListOfListOfString - 12, // 168: productv1.AuthorInput.collaborations:type_name -> productv1.ListOfListOfString - 158, // 169: productv1.AuthorInput.favorite_categories:type_name -> productv1.CategoryInput - 14, // 170: productv1.AuthorInput.author_groups:type_name -> productv1.ListOfListOfUserInput - 14, // 171: productv1.AuthorInput.project_teams:type_name -> productv1.ListOfListOfUserInput - 149, // 172: productv1.NestedTypeB.c:type_name -> productv1.NestedTypeC - 151, // 173: productv1.FilterType.pagination:type_name -> productv1.Pagination - 17, // 174: productv1.OrderLineInput.modifiers:type_name -> productv1.ListOfString - 17, // 175: productv1.OrderLine.modifiers:type_name -> productv1.ListOfString - 0, // 176: productv1.CategoryInput.kind:type_name -> productv1.CategoryKind - 181, // 177: productv1.ProductCountFilter.min_price:type_name -> google.protobuf.DoubleValue - 181, // 178: productv1.ProductCountFilter.max_price:type_name -> google.protobuf.DoubleValue - 182, // 179: productv1.ProductCountFilter.in_stock:type_name -> google.protobuf.BoolValue - 180, // 180: productv1.ProductCountFilter.search_term:type_name -> google.protobuf.StringValue - 141, // 181: productv1.ListOfAuthorFilter.List.items:type_name -> productv1.AuthorFilter - 147, // 182: productv1.ListOfAuthorInput.List.items:type_name -> productv1.AuthorInput - 138, // 183: productv1.ListOfBlogPost.List.items:type_name -> productv1.BlogPost - 139, // 184: productv1.ListOfBlogPostFilter.List.items:type_name -> productv1.BlogPostFilter - 146, // 185: productv1.ListOfBlogPostInput.List.items:type_name -> productv1.BlogPostInput - 131, // 186: productv1.ListOfCategory.List.items:type_name -> productv1.Category - 158, // 187: productv1.ListOfCategoryInput.List.items:type_name -> productv1.CategoryInput - 7, // 188: productv1.ListOfListOfCategory.List.items:type_name -> productv1.ListOfCategory - 8, // 189: productv1.ListOfListOfCategoryInput.List.items:type_name -> productv1.ListOfCategoryInput - 17, // 190: productv1.ListOfListOfString.List.items:type_name -> productv1.ListOfString - 18, // 191: productv1.ListOfListOfUser.List.items:type_name -> productv1.ListOfUser - 19, // 192: productv1.ListOfListOfUserInput.List.items:type_name -> productv1.ListOfUserInput - 153, // 193: productv1.ListOfOrderLine.List.items:type_name -> productv1.OrderLine - 119, // 194: productv1.ListOfProduct.List.items:type_name -> productv1.Product - 122, // 195: productv1.ListOfUser.List.items:type_name -> productv1.User - 142, // 196: productv1.ListOfUserInput.List.items:type_name -> productv1.UserInput - 21, // 197: productv1.ProductService.LookupProductById:input_type -> productv1.LookupProductByIdRequest - 24, // 198: productv1.ProductService.LookupStorageById:input_type -> productv1.LookupStorageByIdRequest - 27, // 199: productv1.ProductService.LookupWarehouseById:input_type -> productv1.LookupWarehouseByIdRequest - 105, // 200: productv1.ProductService.MutationBulkCreateAuthors:input_type -> productv1.MutationBulkCreateAuthorsRequest - 109, // 201: productv1.ProductService.MutationBulkCreateBlogPosts:input_type -> productv1.MutationBulkCreateBlogPostsRequest - 107, // 202: productv1.ProductService.MutationBulkUpdateAuthors:input_type -> productv1.MutationBulkUpdateAuthorsRequest - 111, // 203: productv1.ProductService.MutationBulkUpdateBlogPosts:input_type -> productv1.MutationBulkUpdateBlogPostsRequest - 101, // 204: productv1.ProductService.MutationCreateAuthor:input_type -> productv1.MutationCreateAuthorRequest - 97, // 205: productv1.ProductService.MutationCreateBlogPost:input_type -> productv1.MutationCreateBlogPostRequest - 93, // 206: productv1.ProductService.MutationCreateNullableFieldsType:input_type -> productv1.MutationCreateNullableFieldsTypeRequest - 89, // 207: productv1.ProductService.MutationCreateUser:input_type -> productv1.MutationCreateUserRequest - 91, // 208: productv1.ProductService.MutationPerformAction:input_type -> productv1.MutationPerformActionRequest - 103, // 209: productv1.ProductService.MutationUpdateAuthor:input_type -> productv1.MutationUpdateAuthorRequest - 99, // 210: productv1.ProductService.MutationUpdateBlogPost:input_type -> productv1.MutationUpdateBlogPostRequest - 95, // 211: productv1.ProductService.MutationUpdateNullableFieldsType:input_type -> productv1.MutationUpdateNullableFieldsTypeRequest - 83, // 212: productv1.ProductService.QueryAllAuthors:input_type -> productv1.QueryAllAuthorsRequest - 75, // 213: productv1.ProductService.QueryAllBlogPosts:input_type -> productv1.QueryAllBlogPostsRequest - 67, // 214: productv1.ProductService.QueryAllNullableFieldsTypes:input_type -> productv1.QueryAllNullableFieldsTypesRequest - 55, // 215: productv1.ProductService.QueryAllPets:input_type -> productv1.QueryAllPetsRequest - 77, // 216: productv1.ProductService.QueryAuthor:input_type -> productv1.QueryAuthorRequest - 79, // 217: productv1.ProductService.QueryAuthorById:input_type -> productv1.QueryAuthorByIdRequest - 81, // 218: productv1.ProductService.QueryAuthorsWithFilter:input_type -> productv1.QueryAuthorsWithFilterRequest - 69, // 219: productv1.ProductService.QueryBlogPost:input_type -> productv1.QueryBlogPostRequest - 71, // 220: productv1.ProductService.QueryBlogPostById:input_type -> productv1.QueryBlogPostByIdRequest - 73, // 221: productv1.ProductService.QueryBlogPostsWithFilter:input_type -> productv1.QueryBlogPostsWithFilterRequest - 85, // 222: productv1.ProductService.QueryBulkSearchAuthors:input_type -> productv1.QueryBulkSearchAuthorsRequest - 87, // 223: productv1.ProductService.QueryBulkSearchBlogPosts:input_type -> productv1.QueryBulkSearchBlogPostsRequest - 43, // 224: productv1.ProductService.QueryCalculateTotals:input_type -> productv1.QueryCalculateTotalsRequest - 45, // 225: productv1.ProductService.QueryCategories:input_type -> productv1.QueryCategoriesRequest - 47, // 226: productv1.ProductService.QueryCategoriesByKind:input_type -> productv1.QueryCategoriesByKindRequest - 49, // 227: productv1.ProductService.QueryCategoriesByKinds:input_type -> productv1.QueryCategoriesByKindsRequest - 41, // 228: productv1.ProductService.QueryComplexFilterType:input_type -> productv1.QueryComplexFilterTypeRequest - 51, // 229: productv1.ProductService.QueryFilterCategories:input_type -> productv1.QueryFilterCategoriesRequest - 33, // 230: productv1.ProductService.QueryNestedType:input_type -> productv1.QueryNestedTypeRequest - 61, // 231: productv1.ProductService.QueryNullableFieldsType:input_type -> productv1.QueryNullableFieldsTypeRequest - 63, // 232: productv1.ProductService.QueryNullableFieldsTypeById:input_type -> productv1.QueryNullableFieldsTypeByIdRequest - 65, // 233: productv1.ProductService.QueryNullableFieldsTypeWithFilter:input_type -> productv1.QueryNullableFieldsTypeWithFilterRequest - 53, // 234: productv1.ProductService.QueryRandomPet:input_type -> productv1.QueryRandomPetRequest - 59, // 235: productv1.ProductService.QueryRandomSearchResult:input_type -> productv1.QueryRandomSearchResultRequest - 35, // 236: productv1.ProductService.QueryRecursiveType:input_type -> productv1.QueryRecursiveTypeRequest - 57, // 237: productv1.ProductService.QuerySearch:input_type -> productv1.QuerySearchRequest - 37, // 238: productv1.ProductService.QueryTypeFilterWithArguments:input_type -> productv1.QueryTypeFilterWithArgumentsRequest - 39, // 239: productv1.ProductService.QueryTypeWithMultipleFilterFields:input_type -> productv1.QueryTypeWithMultipleFilterFieldsRequest - 31, // 240: productv1.ProductService.QueryUser:input_type -> productv1.QueryUserRequest - 29, // 241: productv1.ProductService.QueryUsers:input_type -> productv1.QueryUsersRequest - 116, // 242: productv1.ProductService.ResolveCategoryProductCount:input_type -> productv1.ResolveCategoryProductCountRequest - 22, // 243: productv1.ProductService.LookupProductById:output_type -> productv1.LookupProductByIdResponse - 25, // 244: productv1.ProductService.LookupStorageById:output_type -> productv1.LookupStorageByIdResponse - 28, // 245: productv1.ProductService.LookupWarehouseById:output_type -> productv1.LookupWarehouseByIdResponse - 106, // 246: productv1.ProductService.MutationBulkCreateAuthors:output_type -> productv1.MutationBulkCreateAuthorsResponse - 110, // 247: productv1.ProductService.MutationBulkCreateBlogPosts:output_type -> productv1.MutationBulkCreateBlogPostsResponse - 108, // 248: productv1.ProductService.MutationBulkUpdateAuthors:output_type -> productv1.MutationBulkUpdateAuthorsResponse - 112, // 249: productv1.ProductService.MutationBulkUpdateBlogPosts:output_type -> productv1.MutationBulkUpdateBlogPostsResponse - 102, // 250: productv1.ProductService.MutationCreateAuthor:output_type -> productv1.MutationCreateAuthorResponse - 98, // 251: productv1.ProductService.MutationCreateBlogPost:output_type -> productv1.MutationCreateBlogPostResponse - 94, // 252: productv1.ProductService.MutationCreateNullableFieldsType:output_type -> productv1.MutationCreateNullableFieldsTypeResponse - 90, // 253: productv1.ProductService.MutationCreateUser:output_type -> productv1.MutationCreateUserResponse - 92, // 254: productv1.ProductService.MutationPerformAction:output_type -> productv1.MutationPerformActionResponse - 104, // 255: productv1.ProductService.MutationUpdateAuthor:output_type -> productv1.MutationUpdateAuthorResponse - 100, // 256: productv1.ProductService.MutationUpdateBlogPost:output_type -> productv1.MutationUpdateBlogPostResponse - 96, // 257: productv1.ProductService.MutationUpdateNullableFieldsType:output_type -> productv1.MutationUpdateNullableFieldsTypeResponse - 84, // 258: productv1.ProductService.QueryAllAuthors:output_type -> productv1.QueryAllAuthorsResponse - 76, // 259: productv1.ProductService.QueryAllBlogPosts:output_type -> productv1.QueryAllBlogPostsResponse - 68, // 260: productv1.ProductService.QueryAllNullableFieldsTypes:output_type -> productv1.QueryAllNullableFieldsTypesResponse - 56, // 261: productv1.ProductService.QueryAllPets:output_type -> productv1.QueryAllPetsResponse - 78, // 262: productv1.ProductService.QueryAuthor:output_type -> productv1.QueryAuthorResponse - 80, // 263: productv1.ProductService.QueryAuthorById:output_type -> productv1.QueryAuthorByIdResponse - 82, // 264: productv1.ProductService.QueryAuthorsWithFilter:output_type -> productv1.QueryAuthorsWithFilterResponse - 70, // 265: productv1.ProductService.QueryBlogPost:output_type -> productv1.QueryBlogPostResponse - 72, // 266: productv1.ProductService.QueryBlogPostById:output_type -> productv1.QueryBlogPostByIdResponse - 74, // 267: productv1.ProductService.QueryBlogPostsWithFilter:output_type -> productv1.QueryBlogPostsWithFilterResponse - 86, // 268: productv1.ProductService.QueryBulkSearchAuthors:output_type -> productv1.QueryBulkSearchAuthorsResponse - 88, // 269: productv1.ProductService.QueryBulkSearchBlogPosts:output_type -> productv1.QueryBulkSearchBlogPostsResponse - 44, // 270: productv1.ProductService.QueryCalculateTotals:output_type -> productv1.QueryCalculateTotalsResponse - 46, // 271: productv1.ProductService.QueryCategories:output_type -> productv1.QueryCategoriesResponse - 48, // 272: productv1.ProductService.QueryCategoriesByKind:output_type -> productv1.QueryCategoriesByKindResponse - 50, // 273: productv1.ProductService.QueryCategoriesByKinds:output_type -> productv1.QueryCategoriesByKindsResponse - 42, // 274: productv1.ProductService.QueryComplexFilterType:output_type -> productv1.QueryComplexFilterTypeResponse - 52, // 275: productv1.ProductService.QueryFilterCategories:output_type -> productv1.QueryFilterCategoriesResponse - 34, // 276: productv1.ProductService.QueryNestedType:output_type -> productv1.QueryNestedTypeResponse - 62, // 277: productv1.ProductService.QueryNullableFieldsType:output_type -> productv1.QueryNullableFieldsTypeResponse - 64, // 278: productv1.ProductService.QueryNullableFieldsTypeById:output_type -> productv1.QueryNullableFieldsTypeByIdResponse - 66, // 279: productv1.ProductService.QueryNullableFieldsTypeWithFilter:output_type -> productv1.QueryNullableFieldsTypeWithFilterResponse - 54, // 280: productv1.ProductService.QueryRandomPet:output_type -> productv1.QueryRandomPetResponse - 60, // 281: productv1.ProductService.QueryRandomSearchResult:output_type -> productv1.QueryRandomSearchResultResponse - 36, // 282: productv1.ProductService.QueryRecursiveType:output_type -> productv1.QueryRecursiveTypeResponse - 58, // 283: productv1.ProductService.QuerySearch:output_type -> productv1.QuerySearchResponse - 38, // 284: productv1.ProductService.QueryTypeFilterWithArguments:output_type -> productv1.QueryTypeFilterWithArgumentsResponse - 40, // 285: productv1.ProductService.QueryTypeWithMultipleFilterFields:output_type -> productv1.QueryTypeWithMultipleFilterFieldsResponse - 32, // 286: productv1.ProductService.QueryUser:output_type -> productv1.QueryUserResponse - 30, // 287: productv1.ProductService.QueryUsers:output_type -> productv1.QueryUsersResponse - 118, // 288: productv1.ProductService.ResolveCategoryProductCount:output_type -> productv1.ResolveCategoryProductCountResponse - 243, // [243:289] is the sub-list for method output_type - 197, // [197:243] is the sub-list for method input_type - 197, // [197:197] is the sub-list for extension type_name - 197, // [197:197] is the sub-list for extension extendee - 0, // [0:197] is the sub-list for field type_name + 137, // 90: productv1.MutationBulkUpdateBlogPostsResponse.bulk_update_blog_posts:type_name -> productv1.BlogPost + 158, // 91: productv1.CategoryProductCountArgs.filters:type_name -> productv1.ProductCountFilter + 114, // 92: productv1.ResolveCategoryProductCountRequest.context:type_name -> productv1.CategoryProductCountContext + 113, // 93: productv1.ResolveCategoryProductCountRequest.field_args:type_name -> productv1.CategoryProductCountArgs + 116, // 94: productv1.ResolveCategoryProductCountResponse.result:type_name -> productv1.ResolveCategoryProductCountResponseResult + 147, // 95: productv1.NestedTypeA.b:type_name -> productv1.NestedTypeB + 123, // 96: productv1.RecursiveType.recursive_type:type_name -> productv1.RecursiveType + 149, // 97: productv1.ComplexFilterTypeInput.filter:type_name -> productv1.FilterType + 151, // 98: productv1.OrderInput.lines:type_name -> productv1.OrderLineInput + 15, // 99: productv1.Order.order_lines:type_name -> productv1.ListOfOrderLine + 0, // 100: productv1.Category.kind:type_name -> productv1.CategoryKind + 0, // 101: productv1.CategoryFilter.category:type_name -> productv1.CategoryKind + 150, // 102: productv1.CategoryFilter.pagination:type_name -> productv1.Pagination + 153, // 103: productv1.Animal.cat:type_name -> productv1.Cat + 154, // 104: productv1.Animal.dog:type_name -> productv1.Dog + 178, // 105: productv1.SearchInput.limit:type_name -> google.protobuf.Int32Value + 118, // 106: productv1.SearchResult.product:type_name -> productv1.Product + 121, // 107: productv1.SearchResult.user:type_name -> productv1.User + 130, // 108: productv1.SearchResult.category:type_name -> productv1.Category + 179, // 109: productv1.NullableFieldsType.optional_string:type_name -> google.protobuf.StringValue + 178, // 110: productv1.NullableFieldsType.optional_int:type_name -> google.protobuf.Int32Value + 180, // 111: productv1.NullableFieldsType.optional_float:type_name -> google.protobuf.DoubleValue + 181, // 112: productv1.NullableFieldsType.optional_boolean:type_name -> google.protobuf.BoolValue + 179, // 113: productv1.NullableFieldsFilter.name:type_name -> google.protobuf.StringValue + 179, // 114: productv1.NullableFieldsFilter.optional_string:type_name -> google.protobuf.StringValue + 181, // 115: productv1.NullableFieldsFilter.include_nulls:type_name -> google.protobuf.BoolValue + 17, // 116: productv1.BlogPost.optional_tags:type_name -> productv1.ListOfString + 17, // 117: productv1.BlogPost.keywords:type_name -> productv1.ListOfString + 9, // 118: productv1.BlogPost.ratings:type_name -> productv1.ListOfFloat + 6, // 119: productv1.BlogPost.is_published:type_name -> productv1.ListOfBoolean + 12, // 120: productv1.BlogPost.tag_groups:type_name -> productv1.ListOfListOfString + 12, // 121: productv1.BlogPost.related_topics:type_name -> productv1.ListOfListOfString + 12, // 122: productv1.BlogPost.comment_threads:type_name -> productv1.ListOfListOfString + 12, // 123: productv1.BlogPost.suggestions:type_name -> productv1.ListOfListOfString + 130, // 124: productv1.BlogPost.related_categories:type_name -> productv1.Category + 121, // 125: productv1.BlogPost.contributors:type_name -> productv1.User + 16, // 126: productv1.BlogPost.mentioned_products:type_name -> productv1.ListOfProduct + 18, // 127: productv1.BlogPost.mentioned_users:type_name -> productv1.ListOfUser + 10, // 128: productv1.BlogPost.category_groups:type_name -> productv1.ListOfListOfCategory + 13, // 129: productv1.BlogPost.contributor_teams:type_name -> productv1.ListOfListOfUser + 179, // 130: productv1.BlogPostFilter.title:type_name -> google.protobuf.StringValue + 181, // 131: productv1.BlogPostFilter.has_categories:type_name -> google.protobuf.BoolValue + 178, // 132: productv1.BlogPostFilter.min_tags:type_name -> google.protobuf.Int32Value + 179, // 133: productv1.Author.email:type_name -> google.protobuf.StringValue + 17, // 134: productv1.Author.social_links:type_name -> productv1.ListOfString + 12, // 135: productv1.Author.teams_by_project:type_name -> productv1.ListOfListOfString + 12, // 136: productv1.Author.collaborations:type_name -> productv1.ListOfListOfString + 3, // 137: productv1.Author.written_posts:type_name -> productv1.ListOfBlogPost + 130, // 138: productv1.Author.favorite_categories:type_name -> productv1.Category + 18, // 139: productv1.Author.related_authors:type_name -> productv1.ListOfUser + 16, // 140: productv1.Author.product_reviews:type_name -> productv1.ListOfProduct + 13, // 141: productv1.Author.author_groups:type_name -> productv1.ListOfListOfUser + 10, // 142: productv1.Author.category_preferences:type_name -> productv1.ListOfListOfCategory + 13, // 143: productv1.Author.project_teams:type_name -> productv1.ListOfListOfUser + 179, // 144: productv1.AuthorFilter.name:type_name -> google.protobuf.StringValue + 181, // 145: productv1.AuthorFilter.has_teams:type_name -> google.protobuf.BoolValue + 178, // 146: productv1.AuthorFilter.skill_count:type_name -> google.protobuf.Int32Value + 155, // 147: productv1.ActionResult.action_success:type_name -> productv1.ActionSuccess + 156, // 148: productv1.ActionResult.action_error:type_name -> productv1.ActionError + 179, // 149: productv1.NullableFieldsInput.optional_string:type_name -> google.protobuf.StringValue + 178, // 150: productv1.NullableFieldsInput.optional_int:type_name -> google.protobuf.Int32Value + 180, // 151: productv1.NullableFieldsInput.optional_float:type_name -> google.protobuf.DoubleValue + 181, // 152: productv1.NullableFieldsInput.optional_boolean:type_name -> google.protobuf.BoolValue + 17, // 153: productv1.BlogPostInput.optional_tags:type_name -> productv1.ListOfString + 17, // 154: productv1.BlogPostInput.keywords:type_name -> productv1.ListOfString + 9, // 155: productv1.BlogPostInput.ratings:type_name -> productv1.ListOfFloat + 6, // 156: productv1.BlogPostInput.is_published:type_name -> productv1.ListOfBoolean + 12, // 157: productv1.BlogPostInput.tag_groups:type_name -> productv1.ListOfListOfString + 12, // 158: productv1.BlogPostInput.related_topics:type_name -> productv1.ListOfListOfString + 12, // 159: productv1.BlogPostInput.comment_threads:type_name -> productv1.ListOfListOfString + 12, // 160: productv1.BlogPostInput.suggestions:type_name -> productv1.ListOfListOfString + 8, // 161: productv1.BlogPostInput.related_categories:type_name -> productv1.ListOfCategoryInput + 19, // 162: productv1.BlogPostInput.contributors:type_name -> productv1.ListOfUserInput + 11, // 163: productv1.BlogPostInput.category_groups:type_name -> productv1.ListOfListOfCategoryInput + 179, // 164: productv1.AuthorInput.email:type_name -> google.protobuf.StringValue + 17, // 165: productv1.AuthorInput.social_links:type_name -> productv1.ListOfString + 12, // 166: productv1.AuthorInput.teams_by_project:type_name -> productv1.ListOfListOfString + 12, // 167: productv1.AuthorInput.collaborations:type_name -> productv1.ListOfListOfString + 157, // 168: productv1.AuthorInput.favorite_categories:type_name -> productv1.CategoryInput + 14, // 169: productv1.AuthorInput.author_groups:type_name -> productv1.ListOfListOfUserInput + 14, // 170: productv1.AuthorInput.project_teams:type_name -> productv1.ListOfListOfUserInput + 148, // 171: productv1.NestedTypeB.c:type_name -> productv1.NestedTypeC + 150, // 172: productv1.FilterType.pagination:type_name -> productv1.Pagination + 17, // 173: productv1.OrderLineInput.modifiers:type_name -> productv1.ListOfString + 17, // 174: productv1.OrderLine.modifiers:type_name -> productv1.ListOfString + 0, // 175: productv1.CategoryInput.kind:type_name -> productv1.CategoryKind + 180, // 176: productv1.ProductCountFilter.min_price:type_name -> google.protobuf.DoubleValue + 180, // 177: productv1.ProductCountFilter.max_price:type_name -> google.protobuf.DoubleValue + 181, // 178: productv1.ProductCountFilter.in_stock:type_name -> google.protobuf.BoolValue + 179, // 179: productv1.ProductCountFilter.search_term:type_name -> google.protobuf.StringValue + 140, // 180: productv1.ListOfAuthorFilter.List.items:type_name -> productv1.AuthorFilter + 146, // 181: productv1.ListOfAuthorInput.List.items:type_name -> productv1.AuthorInput + 137, // 182: productv1.ListOfBlogPost.List.items:type_name -> productv1.BlogPost + 138, // 183: productv1.ListOfBlogPostFilter.List.items:type_name -> productv1.BlogPostFilter + 145, // 184: productv1.ListOfBlogPostInput.List.items:type_name -> productv1.BlogPostInput + 130, // 185: productv1.ListOfCategory.List.items:type_name -> productv1.Category + 157, // 186: productv1.ListOfCategoryInput.List.items:type_name -> productv1.CategoryInput + 7, // 187: productv1.ListOfListOfCategory.List.items:type_name -> productv1.ListOfCategory + 8, // 188: productv1.ListOfListOfCategoryInput.List.items:type_name -> productv1.ListOfCategoryInput + 17, // 189: productv1.ListOfListOfString.List.items:type_name -> productv1.ListOfString + 18, // 190: productv1.ListOfListOfUser.List.items:type_name -> productv1.ListOfUser + 19, // 191: productv1.ListOfListOfUserInput.List.items:type_name -> productv1.ListOfUserInput + 152, // 192: productv1.ListOfOrderLine.List.items:type_name -> productv1.OrderLine + 118, // 193: productv1.ListOfProduct.List.items:type_name -> productv1.Product + 121, // 194: productv1.ListOfUser.List.items:type_name -> productv1.User + 141, // 195: productv1.ListOfUserInput.List.items:type_name -> productv1.UserInput + 21, // 196: productv1.ProductService.LookupProductById:input_type -> productv1.LookupProductByIdRequest + 24, // 197: productv1.ProductService.LookupStorageById:input_type -> productv1.LookupStorageByIdRequest + 27, // 198: productv1.ProductService.LookupWarehouseById:input_type -> productv1.LookupWarehouseByIdRequest + 105, // 199: productv1.ProductService.MutationBulkCreateAuthors:input_type -> productv1.MutationBulkCreateAuthorsRequest + 109, // 200: productv1.ProductService.MutationBulkCreateBlogPosts:input_type -> productv1.MutationBulkCreateBlogPostsRequest + 107, // 201: productv1.ProductService.MutationBulkUpdateAuthors:input_type -> productv1.MutationBulkUpdateAuthorsRequest + 111, // 202: productv1.ProductService.MutationBulkUpdateBlogPosts:input_type -> productv1.MutationBulkUpdateBlogPostsRequest + 101, // 203: productv1.ProductService.MutationCreateAuthor:input_type -> productv1.MutationCreateAuthorRequest + 97, // 204: productv1.ProductService.MutationCreateBlogPost:input_type -> productv1.MutationCreateBlogPostRequest + 93, // 205: productv1.ProductService.MutationCreateNullableFieldsType:input_type -> productv1.MutationCreateNullableFieldsTypeRequest + 89, // 206: productv1.ProductService.MutationCreateUser:input_type -> productv1.MutationCreateUserRequest + 91, // 207: productv1.ProductService.MutationPerformAction:input_type -> productv1.MutationPerformActionRequest + 103, // 208: productv1.ProductService.MutationUpdateAuthor:input_type -> productv1.MutationUpdateAuthorRequest + 99, // 209: productv1.ProductService.MutationUpdateBlogPost:input_type -> productv1.MutationUpdateBlogPostRequest + 95, // 210: productv1.ProductService.MutationUpdateNullableFieldsType:input_type -> productv1.MutationUpdateNullableFieldsTypeRequest + 83, // 211: productv1.ProductService.QueryAllAuthors:input_type -> productv1.QueryAllAuthorsRequest + 75, // 212: productv1.ProductService.QueryAllBlogPosts:input_type -> productv1.QueryAllBlogPostsRequest + 67, // 213: productv1.ProductService.QueryAllNullableFieldsTypes:input_type -> productv1.QueryAllNullableFieldsTypesRequest + 55, // 214: productv1.ProductService.QueryAllPets:input_type -> productv1.QueryAllPetsRequest + 77, // 215: productv1.ProductService.QueryAuthor:input_type -> productv1.QueryAuthorRequest + 79, // 216: productv1.ProductService.QueryAuthorById:input_type -> productv1.QueryAuthorByIdRequest + 81, // 217: productv1.ProductService.QueryAuthorsWithFilter:input_type -> productv1.QueryAuthorsWithFilterRequest + 69, // 218: productv1.ProductService.QueryBlogPost:input_type -> productv1.QueryBlogPostRequest + 71, // 219: productv1.ProductService.QueryBlogPostById:input_type -> productv1.QueryBlogPostByIdRequest + 73, // 220: productv1.ProductService.QueryBlogPostsWithFilter:input_type -> productv1.QueryBlogPostsWithFilterRequest + 85, // 221: productv1.ProductService.QueryBulkSearchAuthors:input_type -> productv1.QueryBulkSearchAuthorsRequest + 87, // 222: productv1.ProductService.QueryBulkSearchBlogPosts:input_type -> productv1.QueryBulkSearchBlogPostsRequest + 43, // 223: productv1.ProductService.QueryCalculateTotals:input_type -> productv1.QueryCalculateTotalsRequest + 45, // 224: productv1.ProductService.QueryCategories:input_type -> productv1.QueryCategoriesRequest + 47, // 225: productv1.ProductService.QueryCategoriesByKind:input_type -> productv1.QueryCategoriesByKindRequest + 49, // 226: productv1.ProductService.QueryCategoriesByKinds:input_type -> productv1.QueryCategoriesByKindsRequest + 41, // 227: productv1.ProductService.QueryComplexFilterType:input_type -> productv1.QueryComplexFilterTypeRequest + 51, // 228: productv1.ProductService.QueryFilterCategories:input_type -> productv1.QueryFilterCategoriesRequest + 33, // 229: productv1.ProductService.QueryNestedType:input_type -> productv1.QueryNestedTypeRequest + 61, // 230: productv1.ProductService.QueryNullableFieldsType:input_type -> productv1.QueryNullableFieldsTypeRequest + 63, // 231: productv1.ProductService.QueryNullableFieldsTypeById:input_type -> productv1.QueryNullableFieldsTypeByIdRequest + 65, // 232: productv1.ProductService.QueryNullableFieldsTypeWithFilter:input_type -> productv1.QueryNullableFieldsTypeWithFilterRequest + 53, // 233: productv1.ProductService.QueryRandomPet:input_type -> productv1.QueryRandomPetRequest + 59, // 234: productv1.ProductService.QueryRandomSearchResult:input_type -> productv1.QueryRandomSearchResultRequest + 35, // 235: productv1.ProductService.QueryRecursiveType:input_type -> productv1.QueryRecursiveTypeRequest + 57, // 236: productv1.ProductService.QuerySearch:input_type -> productv1.QuerySearchRequest + 37, // 237: productv1.ProductService.QueryTypeFilterWithArguments:input_type -> productv1.QueryTypeFilterWithArgumentsRequest + 39, // 238: productv1.ProductService.QueryTypeWithMultipleFilterFields:input_type -> productv1.QueryTypeWithMultipleFilterFieldsRequest + 31, // 239: productv1.ProductService.QueryUser:input_type -> productv1.QueryUserRequest + 29, // 240: productv1.ProductService.QueryUsers:input_type -> productv1.QueryUsersRequest + 115, // 241: productv1.ProductService.ResolveCategoryProductCount:input_type -> productv1.ResolveCategoryProductCountRequest + 22, // 242: productv1.ProductService.LookupProductById:output_type -> productv1.LookupProductByIdResponse + 25, // 243: productv1.ProductService.LookupStorageById:output_type -> productv1.LookupStorageByIdResponse + 28, // 244: productv1.ProductService.LookupWarehouseById:output_type -> productv1.LookupWarehouseByIdResponse + 106, // 245: productv1.ProductService.MutationBulkCreateAuthors:output_type -> productv1.MutationBulkCreateAuthorsResponse + 110, // 246: productv1.ProductService.MutationBulkCreateBlogPosts:output_type -> productv1.MutationBulkCreateBlogPostsResponse + 108, // 247: productv1.ProductService.MutationBulkUpdateAuthors:output_type -> productv1.MutationBulkUpdateAuthorsResponse + 112, // 248: productv1.ProductService.MutationBulkUpdateBlogPosts:output_type -> productv1.MutationBulkUpdateBlogPostsResponse + 102, // 249: productv1.ProductService.MutationCreateAuthor:output_type -> productv1.MutationCreateAuthorResponse + 98, // 250: productv1.ProductService.MutationCreateBlogPost:output_type -> productv1.MutationCreateBlogPostResponse + 94, // 251: productv1.ProductService.MutationCreateNullableFieldsType:output_type -> productv1.MutationCreateNullableFieldsTypeResponse + 90, // 252: productv1.ProductService.MutationCreateUser:output_type -> productv1.MutationCreateUserResponse + 92, // 253: productv1.ProductService.MutationPerformAction:output_type -> productv1.MutationPerformActionResponse + 104, // 254: productv1.ProductService.MutationUpdateAuthor:output_type -> productv1.MutationUpdateAuthorResponse + 100, // 255: productv1.ProductService.MutationUpdateBlogPost:output_type -> productv1.MutationUpdateBlogPostResponse + 96, // 256: productv1.ProductService.MutationUpdateNullableFieldsType:output_type -> productv1.MutationUpdateNullableFieldsTypeResponse + 84, // 257: productv1.ProductService.QueryAllAuthors:output_type -> productv1.QueryAllAuthorsResponse + 76, // 258: productv1.ProductService.QueryAllBlogPosts:output_type -> productv1.QueryAllBlogPostsResponse + 68, // 259: productv1.ProductService.QueryAllNullableFieldsTypes:output_type -> productv1.QueryAllNullableFieldsTypesResponse + 56, // 260: productv1.ProductService.QueryAllPets:output_type -> productv1.QueryAllPetsResponse + 78, // 261: productv1.ProductService.QueryAuthor:output_type -> productv1.QueryAuthorResponse + 80, // 262: productv1.ProductService.QueryAuthorById:output_type -> productv1.QueryAuthorByIdResponse + 82, // 263: productv1.ProductService.QueryAuthorsWithFilter:output_type -> productv1.QueryAuthorsWithFilterResponse + 70, // 264: productv1.ProductService.QueryBlogPost:output_type -> productv1.QueryBlogPostResponse + 72, // 265: productv1.ProductService.QueryBlogPostById:output_type -> productv1.QueryBlogPostByIdResponse + 74, // 266: productv1.ProductService.QueryBlogPostsWithFilter:output_type -> productv1.QueryBlogPostsWithFilterResponse + 86, // 267: productv1.ProductService.QueryBulkSearchAuthors:output_type -> productv1.QueryBulkSearchAuthorsResponse + 88, // 268: productv1.ProductService.QueryBulkSearchBlogPosts:output_type -> productv1.QueryBulkSearchBlogPostsResponse + 44, // 269: productv1.ProductService.QueryCalculateTotals:output_type -> productv1.QueryCalculateTotalsResponse + 46, // 270: productv1.ProductService.QueryCategories:output_type -> productv1.QueryCategoriesResponse + 48, // 271: productv1.ProductService.QueryCategoriesByKind:output_type -> productv1.QueryCategoriesByKindResponse + 50, // 272: productv1.ProductService.QueryCategoriesByKinds:output_type -> productv1.QueryCategoriesByKindsResponse + 42, // 273: productv1.ProductService.QueryComplexFilterType:output_type -> productv1.QueryComplexFilterTypeResponse + 52, // 274: productv1.ProductService.QueryFilterCategories:output_type -> productv1.QueryFilterCategoriesResponse + 34, // 275: productv1.ProductService.QueryNestedType:output_type -> productv1.QueryNestedTypeResponse + 62, // 276: productv1.ProductService.QueryNullableFieldsType:output_type -> productv1.QueryNullableFieldsTypeResponse + 64, // 277: productv1.ProductService.QueryNullableFieldsTypeById:output_type -> productv1.QueryNullableFieldsTypeByIdResponse + 66, // 278: productv1.ProductService.QueryNullableFieldsTypeWithFilter:output_type -> productv1.QueryNullableFieldsTypeWithFilterResponse + 54, // 279: productv1.ProductService.QueryRandomPet:output_type -> productv1.QueryRandomPetResponse + 60, // 280: productv1.ProductService.QueryRandomSearchResult:output_type -> productv1.QueryRandomSearchResultResponse + 36, // 281: productv1.ProductService.QueryRecursiveType:output_type -> productv1.QueryRecursiveTypeResponse + 58, // 282: productv1.ProductService.QuerySearch:output_type -> productv1.QuerySearchResponse + 38, // 283: productv1.ProductService.QueryTypeFilterWithArguments:output_type -> productv1.QueryTypeFilterWithArgumentsResponse + 40, // 284: productv1.ProductService.QueryTypeWithMultipleFilterFields:output_type -> productv1.QueryTypeWithMultipleFilterFieldsResponse + 32, // 285: productv1.ProductService.QueryUser:output_type -> productv1.QueryUserResponse + 30, // 286: productv1.ProductService.QueryUsers:output_type -> productv1.QueryUsersResponse + 117, // 287: productv1.ProductService.ResolveCategoryProductCount:output_type -> productv1.ResolveCategoryProductCountResponse + 242, // [242:288] is the sub-list for method output_type + 196, // [196:242] is the sub-list for method input_type + 196, // [196:196] is the sub-list for extension type_name + 196, // [196:196] is the sub-list for extension extendee + 0, // [0:196] is the sub-list for field type_name } func init() { file_product_proto_init() } @@ -10226,16 +10178,16 @@ func file_product_proto_init() { if File_product_proto != nil { return } - file_product_proto_msgTypes[132].OneofWrappers = []any{ + file_product_proto_msgTypes[131].OneofWrappers = []any{ (*Animal_Cat)(nil), (*Animal_Dog)(nil), } - file_product_proto_msgTypes[134].OneofWrappers = []any{ + file_product_proto_msgTypes[133].OneofWrappers = []any{ (*SearchResult_Product)(nil), (*SearchResult_User)(nil), (*SearchResult_Category)(nil), } - file_product_proto_msgTypes[143].OneofWrappers = []any{ + file_product_proto_msgTypes[142].OneofWrappers = []any{ (*ActionResult_ActionSuccess)(nil), (*ActionResult_ActionError)(nil), } @@ -10245,7 +10197,7 @@ func file_product_proto_init() { GoPackagePath: reflect.TypeOf(x{}).PkgPath(), RawDescriptor: unsafe.Slice(unsafe.StringData(file_product_proto_rawDesc), len(file_product_proto_rawDesc)), NumEnums: 1, - NumMessages: 178, + NumMessages: 177, NumExtensions: 0, NumServices: 1, }, From 4e6385dc20987a949135bfd9c52f51d93728e0c6 Mon Sep 17 00:00:00 2001 From: Ludwig Bedacht Date: Tue, 7 Oct 2025 14:16:15 +0200 Subject: [PATCH 03/72] chore: add mapping for resolve calls --- .../grpc_datasource/configuration.go | 59 +- .../grpc_datasource/execution_plan.go | 65 +- .../execution_plan_field_resolvers_test.go | 6 +- .../grpc_datasource/execution_plan_visitor.go | 71 +- .../execution_plan_visitor_federation.go | 2 +- .../datasource/grpc_datasource/fetch.go | 12 +- .../grpc_datasource/grpc_datasource_test.go | 8 +- .../grpc_datasource/json_builder.go | 4 +- .../grpc_datasource/mapping_test_helper.go | 22 +- v2/pkg/grpctest/cmd/mapping_helper/main.go | 41 +- v2/pkg/grpctest/mapping/mapping.go | 1479 +++++++++-------- v2/pkg/grpctest/product.proto | 12 +- 12 files changed, 963 insertions(+), 818 deletions(-) diff --git a/v2/pkg/engine/datasource/grpc_datasource/configuration.go b/v2/pkg/engine/datasource/grpc_datasource/configuration.go index e52172b23..ad9863e09 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/configuration.go +++ b/v2/pkg/engine/datasource/grpc_datasource/configuration.go @@ -6,11 +6,17 @@ import ( "github.com/wundergraph/graphql-go-tools/v2/pkg/lexer/runes" ) +// LookupFieldMapDefinition defines the mapping between a GraphQL field and a gRPC field +type RPCMapping interface { + RPCConfig | ResolveRPCMapping + // TODO: add requires RPC mapping +} + type ( // RPCConfigMap is a map of RPC names to RPC configurations // The key is the field name in the GraphQL operation type (query, mutation, subscription). // The value is the RPC configuration for that field. - RPCConfigMap map[string]RPCConfig + RPCConfigMap[T RPCMapping] map[string]T // FieldMap defines the mapping between a GraphQL field and a gRPC field // The key is the field name in the GraphQL type. // The value is the FieldMapData for that field which contains the target name and the argument mappings. @@ -21,11 +27,13 @@ type GRPCMapping struct { // Service is the name of the gRPC service to use Service string // QueryRPCs maps GraphQL query fields to the corresponding gRPC RPC configurations - QueryRPCs RPCConfigMap + QueryRPCs RPCConfigMap[RPCConfig] // MutationRPCs maps GraphQL mutation fields to the corresponding gRPC RPC configurations - MutationRPCs RPCConfigMap + MutationRPCs RPCConfigMap[RPCConfig] // SubscriptionRPCs maps GraphQL subscription fields to the corresponding gRPC RPC configurations - SubscriptionRPCs RPCConfigMap + SubscriptionRPCs RPCConfigMap[RPCConfig] + // ResolveRPCs maps GraphQL resolve fields to the corresponding gRPC RPC configurations + ResolveRPCs RPCConfigMap[ResolveRPCMapping] // EntityRPCs defines how GraphQL types are resolved as entities using specific RPCs // The key is the type name and the value is a list of EntityRPCConfig for that type EntityRPCs map[string][]EntityRPCConfig @@ -78,8 +86,19 @@ type FieldMapData struct { // FieldArgumentMap defines the mapping between a GraphQL field argument and a gRPC field type FieldArgumentMap map[string]string -// ResolveFieldMapping resolves the gRPC field name for a given GraphQL field name and type -func (g *GRPCMapping) ResolveFieldMapping(typeName string, fieldName string) (string, bool) { +// ResolveRPCMapping defines the mapping between a GraphQL field resolver and a gRPC RPC configuration +type ResolveRPCMapping map[string]ResolveRPCTypeField + +// ResolveRPCTypeField defines the mapping between a GraphQL resolve field and a gRPC RPC configuration +type ResolveRPCTypeField struct { + FieldMappingData FieldMapData // The mapping between GraphQL field arguments and gRPC request arguments + RPC string // The name of the RPC method to call + Request string // The name of the request message type + Response string // The name of the response message type +} + +// FindFieldMapping finds the gRPC field name for a given GraphQL field name and type +func (g *GRPCMapping) FindFieldMapping(typeName string, fieldName string) (string, bool) { if g == nil || g.Fields == nil { return "", false } @@ -97,7 +116,8 @@ func (g *GRPCMapping) ResolveFieldMapping(typeName string, fieldName string) (st return field.TargetName, true } -func (g *GRPCMapping) ResolveFieldArgumentMapping(typeName, fieldName, argumentName string) (string, bool) { +// FindFieldArgumentMapping finds the gRPC field name for a given GraphQL field name and argument name +func (g *GRPCMapping) FindFieldArgumentMapping(typeName, fieldName, argumentName string) (string, bool) { if g == nil || g.Fields == nil { return "", false } @@ -116,7 +136,8 @@ func (g *GRPCMapping) ResolveFieldArgumentMapping(typeName, fieldName, argumentN return grpcFieldName, ok } -func (g *GRPCMapping) ResolveEnumValue(enumName, enumValue string) (string, bool) { +// FindEnumValueMapping finds the gRPC enum value for a given GraphQL enum value +func (g *GRPCMapping) FindEnumValueMapping(enumName, enumValue string) (string, bool) { if g == nil || g.EnumValues == nil { return "", false } @@ -139,7 +160,8 @@ func (g *GRPCMapping) ResolveEnumValue(enumName, enumValue string) (string, bool return "", false } -func (g *GRPCMapping) ResolveEntityRPCConfig(typeName, key string) (RPCConfig, bool) { +// FindEntityRPCConfig finds the gRPC RPC config for a given GraphQL type and key +func (g *GRPCMapping) FindEntityRPCConfig(typeName, key string) (RPCConfig, bool) { rpcConfig, ok := g.EntityRPCs[typeName] if !ok { return RPCConfig{}, false @@ -155,6 +177,25 @@ func (g *GRPCMapping) ResolveEntityRPCConfig(typeName, key string) (RPCConfig, b return RPCConfig{}, false } +// FindResolveTypeFieldMapping finds the gRPC field name for a given GraphQL field name and type +func (g *GRPCMapping) FindResolveTypeFieldMapping(typeName, fieldName string) (ResolveRPCTypeField, bool) { + if g == nil || g.ResolveRPCs == nil { + return ResolveRPCTypeField{}, false + } + + fieldMappings, ok := g.ResolveRPCs[typeName] + if !ok { + return ResolveRPCTypeField{}, false + } + + field, ok := fieldMappings[fieldName] + if !ok { + return ResolveRPCTypeField{}, false + } + + return field, true +} + type keySet map[string]struct{} func (k keySet) add(keys ...string) { diff --git a/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go b/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go index 323461575..dde7348ba 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go +++ b/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go @@ -469,7 +469,7 @@ func (r *rpcPlanningContext) newMessageFromSelectionSet(enclosingTypeNode ast.No // resolveFieldMapping resolves the field mapping for a field. // This applies both for complex types in the input and for all fields in the response. func (r *rpcPlanningContext) resolveFieldMapping(typeName, fieldName string) string { - if grpcFieldName, ok := r.mapping.ResolveFieldMapping(typeName, fieldName); ok { + if grpcFieldName, ok := r.mapping.FindFieldMapping(typeName, fieldName); ok { return grpcFieldName } @@ -477,7 +477,7 @@ func (r *rpcPlanningContext) resolveFieldMapping(typeName, fieldName string) str } func (r *rpcPlanningContext) resolveFieldArgumentMapping(typeName, fieldName, argumentName string) string { - if grpcFieldName, ok := r.mapping.ResolveFieldArgumentMapping(typeName, fieldName, argumentName); ok { + if grpcFieldName, ok := r.mapping.FindFieldArgumentMapping(typeName, fieldName, argumentName); ok { return grpcFieldName } @@ -821,3 +821,64 @@ func (r *rpcPlanningContext) nodeByTypeRef(typeRef int) (ast.Node, bool) { underlyingTypeName := r.definition.ResolveTypeNameString(typeRef) return r.definition.NodeByNameStr(underlyingTypeName) } + +type resolveRPCCallConfig struct { + serviceName string + typeName string + fieldName string + resolveConfig ResolveRPCTypeField + resolvedField resolvedField + contextMessage *RPCMessage + fieldArgsMessage *RPCMessage +} + +func (r *rpcPlanningContext) newResolveRPCCall(config resolveRPCCallConfig) RPCCall { + resolveConfig := config.resolveConfig + resolvedField := config.resolvedField + + return RPCCall{ + DependentCalls: []int{resolvedField.callerRef}, + ResponsePath: resolvedField.responsePath, + ServiceName: r.resolveServiceName(config.serviceName), + MethodName: resolveConfig.RPC, + Kind: CallKindResolve, + Request: RPCMessage{ + Name: resolveConfig.Request, + Fields: RPCFields{ + { + Name: "context", + TypeName: string(DataTypeMessage), + Repeated: true, + Message: config.contextMessage, + }, + { + Name: "field_args", + TypeName: string(DataTypeMessage), + JSONPath: "", + Message: config.fieldArgsMessage, + }, + }, + }, + Response: RPCMessage{ + Name: resolveConfig.Response, + Fields: RPCFields{ + { + Name: "result", + TypeName: string(DataTypeMessage), + JSONPath: "result", + Repeated: true, + Message: &RPCMessage{ + Name: resolveConfig.RPC + "Result", + Fields: RPCFields{ + { + Name: resolveConfig.FieldMappingData.TargetName, + TypeName: string(DataTypeInt32), + JSONPath: config.fieldName, + }, + }, + }, + }, + }, + }, + } +} diff --git a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_field_resolvers_test.go b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_field_resolvers_test.go index b63db1fc5..a9861b3fa 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_field_resolvers_test.go +++ b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_field_resolvers_test.go @@ -71,7 +71,7 @@ func TestExecutionPlanFieldResolvers(t *testing.T) { JSONPath: "", Repeated: true, Message: &RPCMessage{ - Name: "CategoryProductCountContext", + Name: "ResolveCategoryProductCountContext", Fields: []RPCField{ { Name: "id", @@ -93,7 +93,7 @@ func TestExecutionPlanFieldResolvers(t *testing.T) { TypeName: string(DataTypeMessage), JSONPath: "", Message: &RPCMessage{ - Name: "CategoryProductCountArgs", + Name: "ResolveCategoryProductCountArgs", Fields: []RPCField{ { Name: "filters", @@ -144,7 +144,7 @@ func TestExecutionPlanFieldResolvers(t *testing.T) { JSONPath: "result", Repeated: true, Message: &RPCMessage{ - Name: "ResolveCategoryProductCountResponseResult", + Name: "ResolveCategoryProductCountResult", Fields: []RPCField{ { Name: "product_count", diff --git a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor.go b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor.go index 7692ad74b..f18b46d3c 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor.go +++ b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor.go @@ -41,6 +41,7 @@ type fieldArgument struct { type resolvedField struct { callerRef int parentTypeRef int + fieldRef int responsePath ast.Path contextFields []contextField @@ -123,62 +124,33 @@ func (r *rpcPlanVisitor) LeaveDocument(_, _ *ast.Document) { calls := make([]RPCCall, 0, len(r.resolvedFields)) for _, resolvedField := range r.resolvedFields { + resolveConfig, exists := r.mapping.FindResolveTypeFieldMapping( + r.definition.ObjectTypeDefinitionNameString(resolvedField.parentTypeRef), + r.operation.FieldNameString(resolvedField.fieldRef), + ) + + if !exists { + r.walker.StopWithInternalErr(fmt.Errorf("resolve config not found for type: %s, field: %s", r.definition.ResolveTypeNameString(resolvedField.parentTypeRef), r.operation.FieldAliasString(resolvedField.fieldRef))) + return + } contextMessage := &RPCMessage{ - Name: "CategoryProductCountContext", + Name: resolveConfig.RPC + "Context", } fieldArgsMessage := &RPCMessage{ - Name: "CategoryProductCountArgs", + Name: resolveConfig.RPC + "Args", } - // Base resolve call can be templated in plan context. - call := RPCCall{ - DependentCalls: []int{resolvedField.callerRef}, - ResponsePath: resolvedField.responsePath, - ServiceName: r.planCtx.resolveServiceName(r.subgraphName), - MethodName: "ResolveCategoryProductCount", - Kind: CallKindResolve, - Request: RPCMessage{ - Name: "ResolveCategoryProductCountRequest", - Fields: RPCFields{ - { - Name: "context", - TypeName: string(DataTypeMessage), - JSONPath: "", - Repeated: true, - Message: contextMessage, - }, - { - Name: "field_args", - TypeName: string(DataTypeMessage), - JSONPath: "", - Message: fieldArgsMessage, - }, - }, - }, - Response: RPCMessage{ - Name: "ResolveCategoryProductCountResponse", - Fields: RPCFields{ - { - Name: "result", - TypeName: string(DataTypeMessage), - JSONPath: "result", - Repeated: true, - Message: &RPCMessage{ - Name: "ResolveCategoryProductCountResponseResult", - Fields: RPCFields{ - { - Name: "product_count", - TypeName: string(DataTypeInt32), - JSONPath: "productCount", - }, - }, - }, - }, - }, - }, - } + call := r.planCtx.newResolveRPCCall(resolveRPCCallConfig{ + serviceName: r.subgraphName, + typeName: r.definition.ResolveTypeNameString(resolvedField.parentTypeRef), + fieldName: r.operation.FieldAliasOrNameString(resolvedField.fieldRef), + resolveConfig: resolveConfig, + resolvedField: resolvedField, + contextMessage: contextMessage, + fieldArgsMessage: fieldArgsMessage, + }) contextMessage.Fields = make(RPCFields, 0, len(resolvedField.contextFields)) for _, contextField := range resolvedField.contextFields { @@ -440,6 +412,7 @@ func (r *rpcPlanVisitor) EnterField(ref int) { resolvedField := resolvedField{ callerRef: r.relatedCallID, parentTypeRef: r.walker.EnclosingTypeDefinition.Ref, + fieldRef: ref, responsePath: r.walker.Path[1:].WithoutInlineFragmentNames().WithPathElement(ast.PathItem{ Kind: ast.FieldName, FieldName: r.operation.FieldAliasOrNameBytes(ref), diff --git a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor_federation.go b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor_federation.go index c8d85be51..303158135 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor_federation.go +++ b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor_federation.go @@ -323,7 +323,7 @@ func (r *rpcPlanVisitorFederation) resolveEntityInformation(inlineFragmentRef in return nil } - rpcConfig, exists := r.mapping.ResolveEntityRPCConfig(fc.entityTypeName, fc.keyFields) + rpcConfig, exists := r.mapping.FindEntityRPCConfig(fc.entityTypeName, fc.keyFields) if !exists { return fmt.Errorf("entity type %s not found in mapping", fc.entityTypeName) } diff --git a/v2/pkg/engine/datasource/grpc_datasource/fetch.go b/v2/pkg/engine/datasource/grpc_datasource/fetch.go index 19d8edf75..74f0c012e 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/fetch.go +++ b/v2/pkg/engine/datasource/grpc_datasource/fetch.go @@ -59,23 +59,23 @@ func (g *DependencyGraph) TopologicalSortResolve(resolver func(nodes []FetchItem } currentLevel := 0 - for _, dep := range g.nodes[index] { - if dep < 0 || dep >= len(g.nodes) { - return fmt.Errorf("unable to find dependent call %d in execution plan", dep) + for _, depCallIndex := range g.nodes[index] { + if depCallIndex < 0 || depCallIndex >= len(g.nodes) { + return fmt.Errorf("unable to find dependent call %d in execution plan", depCallIndex) } - if depLevel := callHierarchyRefs[dep]; depLevel >= 0 { + if depLevel := callHierarchyRefs[depCallIndex]; depLevel >= 0 { if depLevel > currentLevel { currentLevel = depLevel } continue } - if err := visit(dep); err != nil { + if err := visit(depCallIndex); err != nil { return err } - if l := callHierarchyRefs[dep]; l > currentLevel { + if l := callHierarchyRefs[depCallIndex]; l > currentLevel { currentLevel = l } } diff --git a/v2/pkg/engine/datasource/grpc_datasource/grpc_datasource_test.go b/v2/pkg/engine/datasource/grpc_datasource/grpc_datasource_test.go index c2feb5561..44ff808ca 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/grpc_datasource_test.go +++ b/v2/pkg/engine/datasource/grpc_datasource/grpc_datasource_test.go @@ -130,7 +130,7 @@ func Test_DataSource_Load(t *testing.T) { Compiler: compiler, Mapping: &GRPCMapping{ Service: "Products", - QueryRPCs: RPCConfigMap{ + QueryRPCs: RPCConfigMap[RPCConfig]{ "complexFilterType": { RPC: "QueryComplexFilterType", Request: "QueryComplexFilterTypeRequest", @@ -189,7 +189,7 @@ func Test_DataSource_Load_WithMockService(t *testing.T) { Compiler: compiler, Mapping: &GRPCMapping{ Service: "Products", - QueryRPCs: RPCConfigMap{ + QueryRPCs: RPCConfigMap[RPCConfig]{ "complexFilterType": { RPC: "QueryComplexFilterType", Request: "QueryComplexFilterTypeRequest", @@ -279,7 +279,7 @@ func Test_DataSource_Load_WithMockService_WithResponseMapping(t *testing.T) { Compiler: compiler, Mapping: &GRPCMapping{ Service: "Products", - QueryRPCs: RPCConfigMap{ + QueryRPCs: RPCConfigMap[RPCConfig]{ "complexFilterType": { RPC: "QueryComplexFilterType", Request: "QueryComplexFilterTypeRequest", @@ -382,7 +382,7 @@ func Test_DataSource_Load_WithGrpcError(t *testing.T) { Compiler: compiler, Mapping: &GRPCMapping{ Service: "Products", - QueryRPCs: RPCConfigMap{ + QueryRPCs: RPCConfigMap[RPCConfig]{ "user": { RPC: "QueryUser", Request: "QueryUserRequest", diff --git a/v2/pkg/engine/datasource/grpc_datasource/json_builder.go b/v2/pkg/engine/datasource/grpc_datasource/json_builder.go index cf7dc3da2..d533d9d92 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/json_builder.go +++ b/v2/pkg/engine/datasource/grpc_datasource/json_builder.go @@ -614,7 +614,7 @@ func (j *jsonBuilder) setJSONValue(arena *astjson.Arena, root *astjson.Value, na } // Look up the GraphQL enum value mapping - graphqlValue, ok := j.mapping.ResolveEnumValue(string(enumDesc.Name()), string(enumValueDesc.Name())) + graphqlValue, ok := j.mapping.FindEnumValueMapping(string(enumDesc.Name()), string(enumValueDesc.Name())) if !ok { // No mapping found - set to null root.Set(name, arena.NewNull()) @@ -663,7 +663,7 @@ func (j *jsonBuilder) setArrayItem(index int, arena *astjson.Arena, array *astjs } // Look up GraphQL enum mapping - graphqlValue, ok := j.mapping.ResolveEnumValue(string(enumDesc.Name()), string(enumValueDesc.Name())) + graphqlValue, ok := j.mapping.FindEnumValueMapping(string(enumDesc.Name()), string(enumValueDesc.Name())) if !ok { // No mapping found - use null array.SetArrayItem(index, arena.NewNull()) diff --git a/v2/pkg/engine/datasource/grpc_datasource/mapping_test_helper.go b/v2/pkg/engine/datasource/grpc_datasource/mapping_test_helper.go index 24ec4247b..ca7ee1adb 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/mapping_test_helper.go +++ b/v2/pkg/engine/datasource/grpc_datasource/mapping_test_helper.go @@ -3,7 +3,7 @@ package grpcdatasource func testMapping() *GRPCMapping { return &GRPCMapping{ Service: "Products", - QueryRPCs: map[string]RPCConfig{ + QueryRPCs: RPCConfigMap[RPCConfig]{ "users": { RPC: "QueryUsers", Request: "QueryUsersRequest", @@ -155,7 +155,7 @@ func testMapping() *GRPCMapping { Response: "QueryBulkSearchBlogPostsResponse", }, }, - MutationRPCs: RPCConfigMap{ + MutationRPCs: RPCConfigMap[RPCConfig]{ "createUser": { RPC: "MutationCreateUser", Request: "MutationCreateUserRequest", @@ -217,7 +217,22 @@ func testMapping() *GRPCMapping { Response: "MutationBulkUpdateBlogPostsResponse", }, }, - SubscriptionRPCs: RPCConfigMap{}, + SubscriptionRPCs: RPCConfigMap[RPCConfig]{}, + ResolveRPCs: RPCConfigMap[ResolveRPCMapping]{ + "Category": { + "productCount": { + FieldMappingData: FieldMapData{ + TargetName: "product_count", + ArgumentMappings: FieldArgumentMap{ + "filters": "filters", + }, + }, + RPC: "ResolveCategoryProductCount", + Request: "ResolveCategoryProductCountRequest", + Response: "ResolveCategoryProductCountResponse", + }, + }, + }, EntityRPCs: map[string][]EntityRPCConfig{ "Product": { { @@ -1053,4 +1068,5 @@ func testMapping() *GRPCMapping { }, }, } + } diff --git a/v2/pkg/grpctest/cmd/mapping_helper/main.go b/v2/pkg/grpctest/cmd/mapping_helper/main.go index 1fea4f8d0..550af405c 100644 --- a/v2/pkg/grpctest/cmd/mapping_helper/main.go +++ b/v2/pkg/grpctest/cmd/mapping_helper/main.go @@ -15,6 +15,7 @@ type JSONMapping struct { EntityMappings []EntityMapping `json:"entityMappings"` TypeFieldMappings []TypeFieldMapping `json:"typeFieldMappings"` EnumMappings []EnumMapping `json:"enumMappings"` + ResolveMappings []ResolveMapping `json:"resolveMappings"` } type OperationMapping struct { @@ -34,6 +35,19 @@ type EntityMapping struct { Response string `json:"response"` } +type ResolveMapping struct { + Type string `json:"type"` + LookupMapping LookupMapping `json:"lookupMapping"` + RPC string `json:"rpc"` + Request string `json:"request"` + Response string `json:"response"` +} + +type LookupMapping struct { + Type string `json:"type"` + FieldMapping FieldMapping `json:"fieldMapping"` +} + type TypeFieldMapping struct { Type string `json:"type"` FieldMappings []FieldMapping `json:"fieldMappings"` @@ -72,7 +86,7 @@ import ( func DefaultGRPCMapping() *grpcdatasource.GRPCMapping { return &grpcdatasource.GRPCMapping{ Service: "{{.Service}}", - QueryRPCs: map[string]grpcdatasource.RPCConfig{ + QueryRPCs: grpcdatasource.RPCConfigMap[grpcdatasource.RPCConfig]{ {{- range $index, $operation := .OperationMappings}} {{- if eq $operation.Type "OPERATION_TYPE_QUERY"}} "{{$operation.Original}}": { @@ -83,7 +97,7 @@ func DefaultGRPCMapping() *grpcdatasource.GRPCMapping { {{- end }} {{- end }} }, - MutationRPCs: grpcdatasource.RPCConfigMap{ + MutationRPCs: grpcdatasource.RPCConfigMap[grpcdatasource.RPCConfig]{ {{- range $index, $operation := .OperationMappings}} {{- if eq $operation.Type "OPERATION_TYPE_MUTATION"}} "{{$operation.Original}}": { @@ -94,7 +108,7 @@ func DefaultGRPCMapping() *grpcdatasource.GRPCMapping { {{- end }} {{- end }} }, - SubscriptionRPCs: grpcdatasource.RPCConfigMap{ + SubscriptionRPCs: grpcdatasource.RPCConfigMap[grpcdatasource.RPCConfig]{ {{- range $index, $operation := .OperationMappings}} {{- if eq $operation.Type "OPERATION_TYPE_SUBSCRIPTION"}} "{{$operation.Original}}": { @@ -105,6 +119,27 @@ func DefaultGRPCMapping() *grpcdatasource.GRPCMapping { {{- end }} {{- end }} }, + ResolveRPCs: grpcdatasource.RPCConfigMap[grpcdatasource.ResolveRPCMapping]{ + {{- range $index, $resolve := .ResolveMappings}} + {{- if eq $resolve.Type "LOOKUP_TYPE_RESOLVE"}} + "{{$resolve.LookupMapping.Type}}": { + "{{$resolve.LookupMapping.FieldMapping.Original}}": { + FieldMappingData: grpcdatasource.FieldMapData{ + TargetName: "{{$resolve.LookupMapping.FieldMapping.Mapped}}", + ArgumentMappings: grpcdatasource.FieldArgumentMap{ + {{- range $index, $argument := $resolve.LookupMapping.FieldMapping.ArgumentMappings}} + "{{$argument.Original}}": "{{$argument.Mapped}}", + {{- end }} + }, + }, + RPC: "{{$resolve.RPC}}", + Request: "{{$resolve.Request}}", + Response: "{{$resolve.Response}}", + }, + }, + {{- end }} + {{- end }} + }, EntityRPCs: map[string][]grpcdatasource.EntityRPCConfig{ {{- range $index, $entity := .EntityMappings}} "{{$entity.TypeName}}": { diff --git a/v2/pkg/grpctest/mapping/mapping.go b/v2/pkg/grpctest/mapping/mapping.go index b62ad84fb..81f5c8e43 100644 --- a/v2/pkg/grpctest/mapping/mapping.go +++ b/v2/pkg/grpctest/mapping/mapping.go @@ -9,8 +9,8 @@ import ( // DefaultGRPCMapping returns a hardcoded default mapping between GraphQL and Protobuf func DefaultGRPCMapping() *grpcdatasource.GRPCMapping { return &grpcdatasource.GRPCMapping{ - Service: "Products", - QueryRPCs: map[string]grpcdatasource.RPCConfig{ + Service: "ProductService", + QueryRPCs: grpcdatasource.RPCConfigMap[grpcdatasource.RPCConfig]{ "users": { RPC: "QueryUsers", Request: "QueryUsersRequest", @@ -162,7 +162,7 @@ func DefaultGRPCMapping() *grpcdatasource.GRPCMapping { Response: "QueryBulkSearchBlogPostsResponse", }, }, - MutationRPCs: grpcdatasource.RPCConfigMap{ + MutationRPCs: grpcdatasource.RPCConfigMap[grpcdatasource.RPCConfig]{ "createUser": { RPC: "MutationCreateUser", Request: "MutationCreateUserRequest", @@ -224,7 +224,23 @@ func DefaultGRPCMapping() *grpcdatasource.GRPCMapping { Response: "MutationBulkUpdateBlogPostsResponse", }, }, - SubscriptionRPCs: grpcdatasource.RPCConfigMap{}, + SubscriptionRPCs: grpcdatasource.RPCConfigMap[grpcdatasource.RPCConfig]{ + }, + ResolveRPCs: grpcdatasource.RPCConfigMap[grpcdatasource.ResolveRPCMapping]{ + "Category": { + "productCount": { + FieldMappingData: grpcdatasource.FieldMapData{ + TargetName: "product_count", + ArgumentMappings: grpcdatasource.FieldArgumentMap{ + "filters": "filters", + }, + }, + RPC: "ResolveCategoryProductCount", + Request: "ResolveCategoryProductCountRequest", + Response: "ResolveCategoryProductCountResponse", + }, + }, + }, EntityRPCs: map[string][]grpcdatasource.EntityRPCConfig{ "Product": { { @@ -259,811 +275,814 @@ func DefaultGRPCMapping() *grpcdatasource.GRPCMapping { }, EnumValues: map[string][]grpcdatasource.EnumValueMapping{ "CategoryKind": { - {Value: "BOOK", TargetValue: "CATEGORY_KIND_BOOK"}, - {Value: "ELECTRONICS", TargetValue: "CATEGORY_KIND_ELECTRONICS"}, - {Value: "FURNITURE", TargetValue: "CATEGORY_KIND_FURNITURE"}, - {Value: "OTHER", TargetValue: "CATEGORY_KIND_OTHER"}, + {Value: "BOOK", TargetValue: "CATEGORY_KIND_BOOK"}, + {Value: "ELECTRONICS", TargetValue: "CATEGORY_KIND_ELECTRONICS"}, + {Value: "FURNITURE", TargetValue: "CATEGORY_KIND_FURNITURE"}, + {Value: "OTHER", TargetValue: "CATEGORY_KIND_OTHER"}, }, }, Fields: map[string]grpcdatasource.FieldMap{ - "Query": { - "users": { - TargetName: "users", - }, - "user": { - TargetName: "user", - ArgumentMappings: grpcdatasource.FieldArgumentMap{ - "id": "id", - }, - }, - "nestedType": { - TargetName: "nested_type", - }, - "recursiveType": { - TargetName: "recursive_type", - }, - "typeFilterWithArguments": { - TargetName: "type_filter_with_arguments", - ArgumentMappings: grpcdatasource.FieldArgumentMap{ - "filterField1": "filter_field_1", - "filterField2": "filter_field_2", - }, - }, - "typeWithMultipleFilterFields": { - TargetName: "type_with_multiple_filter_fields", - ArgumentMappings: grpcdatasource.FieldArgumentMap{ - "filter": "filter", - }, - }, - "complexFilterType": { - TargetName: "complex_filter_type", - ArgumentMappings: grpcdatasource.FieldArgumentMap{ - "filter": "filter", - }, - }, - "calculateTotals": { - TargetName: "calculate_totals", - ArgumentMappings: grpcdatasource.FieldArgumentMap{ - "orders": "orders", - }, - }, - "categories": { - TargetName: "categories", - }, - "categoriesByKind": { - TargetName: "categories_by_kind", - ArgumentMappings: grpcdatasource.FieldArgumentMap{ - "kind": "kind", - }, - }, - "categoriesByKinds": { - TargetName: "categories_by_kinds", - ArgumentMappings: grpcdatasource.FieldArgumentMap{ - "kinds": "kinds", - }, - }, - "filterCategories": { - TargetName: "filter_categories", - ArgumentMappings: grpcdatasource.FieldArgumentMap{ - "filter": "filter", - }, - }, - "randomPet": { - TargetName: "random_pet", - }, - "allPets": { - TargetName: "all_pets", - }, - "search": { - TargetName: "search", - ArgumentMappings: grpcdatasource.FieldArgumentMap{ - "input": "input", - }, - }, - "randomSearchResult": { - TargetName: "random_search_result", - }, - "nullableFieldsType": { - TargetName: "nullable_fields_type", - }, - "nullableFieldsTypeById": { - TargetName: "nullable_fields_type_by_id", - ArgumentMappings: grpcdatasource.FieldArgumentMap{ - "id": "id", - }, - }, - "nullableFieldsTypeWithFilter": { - TargetName: "nullable_fields_type_with_filter", - ArgumentMappings: grpcdatasource.FieldArgumentMap{ - "filter": "filter", - }, - }, - "allNullableFieldsTypes": { - TargetName: "all_nullable_fields_types", - }, - "blogPost": { - TargetName: "blog_post", - }, - "blogPostById": { - TargetName: "blog_post_by_id", - ArgumentMappings: grpcdatasource.FieldArgumentMap{ - "id": "id", - }, - }, - "blogPostsWithFilter": { - TargetName: "blog_posts_with_filter", - ArgumentMappings: grpcdatasource.FieldArgumentMap{ - "filter": "filter", - }, - }, - "allBlogPosts": { - TargetName: "all_blog_posts", - }, - "author": { - TargetName: "author", - }, - "authorById": { - TargetName: "author_by_id", - ArgumentMappings: grpcdatasource.FieldArgumentMap{ - "id": "id", - }, - }, - "authorsWithFilter": { - TargetName: "authors_with_filter", - ArgumentMappings: grpcdatasource.FieldArgumentMap{ - "filter": "filter", - }, - }, - "allAuthors": { - TargetName: "all_authors", - }, - "bulkSearchAuthors": { - TargetName: "bulk_search_authors", - ArgumentMappings: grpcdatasource.FieldArgumentMap{ - "filters": "filters", - }, - }, - "bulkSearchBlogPosts": { - TargetName: "bulk_search_blog_posts", - ArgumentMappings: grpcdatasource.FieldArgumentMap{ - "filters": "filters", - }, - }, + "Query": { + "users": { + TargetName: "users", }, - "Mutation": { - "createUser": { - TargetName: "create_user", - ArgumentMappings: grpcdatasource.FieldArgumentMap{ - "input": "input", - }, - }, - "performAction": { - TargetName: "perform_action", - ArgumentMappings: grpcdatasource.FieldArgumentMap{ - "input": "input", - }, - }, - "createNullableFieldsType": { - TargetName: "create_nullable_fields_type", - ArgumentMappings: grpcdatasource.FieldArgumentMap{ - "input": "input", - }, - }, - "updateNullableFieldsType": { - TargetName: "update_nullable_fields_type", - ArgumentMappings: grpcdatasource.FieldArgumentMap{ - "id": "id", - "input": "input", - }, - }, - "createBlogPost": { - TargetName: "create_blog_post", - ArgumentMappings: grpcdatasource.FieldArgumentMap{ - "input": "input", - }, - }, - "updateBlogPost": { - TargetName: "update_blog_post", - ArgumentMappings: grpcdatasource.FieldArgumentMap{ - "id": "id", - "input": "input", - }, - }, - "createAuthor": { - TargetName: "create_author", - ArgumentMappings: grpcdatasource.FieldArgumentMap{ - "input": "input", - }, - }, - "updateAuthor": { - TargetName: "update_author", - ArgumentMappings: grpcdatasource.FieldArgumentMap{ - "id": "id", - "input": "input", - }, - }, - "bulkCreateAuthors": { - TargetName: "bulk_create_authors", - ArgumentMappings: grpcdatasource.FieldArgumentMap{ - "authors": "authors", - }, - }, - "bulkUpdateAuthors": { - TargetName: "bulk_update_authors", - ArgumentMappings: grpcdatasource.FieldArgumentMap{ - "authors": "authors", - }, - }, - "bulkCreateBlogPosts": { - TargetName: "bulk_create_blog_posts", - ArgumentMappings: grpcdatasource.FieldArgumentMap{ - "blogPosts": "blog_posts", - }, - }, - "bulkUpdateBlogPosts": { - TargetName: "bulk_update_blog_posts", - ArgumentMappings: grpcdatasource.FieldArgumentMap{ - "blogPosts": "blog_posts", - }, + "user": { + TargetName: "user", + ArgumentMappings: grpcdatasource.FieldArgumentMap{ + "id": "id", }, }, - "Product": { - "id": { - TargetName: "id", - }, - "name": { - TargetName: "name", - }, - "price": { - TargetName: "price", - }, + "nestedType": { + TargetName: "nested_type", }, - "Storage": { - "id": { - TargetName: "id", - }, - "name": { - TargetName: "name", - }, - "location": { - TargetName: "location", - }, + "recursiveType": { + TargetName: "recursive_type", }, - "Warehouse": { - "id": { - TargetName: "id", - }, - "name": { - TargetName: "name", - }, - "location": { - TargetName: "location", + "typeFilterWithArguments": { + TargetName: "type_filter_with_arguments", + ArgumentMappings: grpcdatasource.FieldArgumentMap{ + "filterField1": "filter_field_1", + "filterField2": "filter_field_2", }, }, - "User": { - "id": { - TargetName: "id", - }, - "name": { - TargetName: "name", + "typeWithMultipleFilterFields": { + TargetName: "type_with_multiple_filter_fields", + ArgumentMappings: grpcdatasource.FieldArgumentMap{ + "filter": "filter", }, }, - "NestedTypeA": { - "id": { - TargetName: "id", - }, - "name": { - TargetName: "name", - }, - "b": { - TargetName: "b", + "complexFilterType": { + TargetName: "complex_filter_type", + ArgumentMappings: grpcdatasource.FieldArgumentMap{ + "filter": "filter", }, }, - "NestedTypeB": { - "id": { - TargetName: "id", - }, - "name": { - TargetName: "name", - }, - "c": { - TargetName: "c", + "calculateTotals": { + TargetName: "calculate_totals", + ArgumentMappings: grpcdatasource.FieldArgumentMap{ + "orders": "orders", }, }, - "NestedTypeC": { - "id": { - TargetName: "id", - }, - "name": { - TargetName: "name", - }, + "categories": { + TargetName: "categories", }, - "RecursiveType": { - "id": { - TargetName: "id", - }, - "name": { - TargetName: "name", - }, - "recursiveType": { - TargetName: "recursive_type", + "categoriesByKind": { + TargetName: "categories_by_kind", + ArgumentMappings: grpcdatasource.FieldArgumentMap{ + "kind": "kind", }, }, - "TypeWithMultipleFilterFields": { - "id": { - TargetName: "id", - }, - "name": { - TargetName: "name", - }, - "filterField1": { - TargetName: "filter_field_1", - }, - "filterField2": { - TargetName: "filter_field_2", + "categoriesByKinds": { + TargetName: "categories_by_kinds", + ArgumentMappings: grpcdatasource.FieldArgumentMap{ + "kinds": "kinds", }, }, - "FilterTypeInput": { - "filterField1": { - TargetName: "filter_field_1", - }, - "filterField2": { - TargetName: "filter_field_2", + "filterCategories": { + TargetName: "filter_categories", + ArgumentMappings: grpcdatasource.FieldArgumentMap{ + "filter": "filter", }, }, - "TypeWithComplexFilterInput": { - "id": { - TargetName: "id", - }, - "name": { - TargetName: "name", - }, + "randomPet": { + TargetName: "random_pet", }, - "FilterType": { - "name": { - TargetName: "name", - }, - "filterField1": { - TargetName: "filter_field_1", - }, - "filterField2": { - TargetName: "filter_field_2", - }, - "pagination": { - TargetName: "pagination", - }, + "allPets": { + TargetName: "all_pets", }, - "Pagination": { - "page": { - TargetName: "page", - }, - "perPage": { - TargetName: "per_page", + "search": { + TargetName: "search", + ArgumentMappings: grpcdatasource.FieldArgumentMap{ + "input": "input", }, }, - "ComplexFilterTypeInput": { - "filter": { - TargetName: "filter", - }, + "randomSearchResult": { + TargetName: "random_search_result", }, - "OrderLineInput": { - "productId": { - TargetName: "product_id", - }, - "quantity": { - TargetName: "quantity", - }, - "modifiers": { - TargetName: "modifiers", - }, + "nullableFieldsType": { + TargetName: "nullable_fields_type", }, - "OrderInput": { - "orderId": { - TargetName: "order_id", - }, - "customerName": { - TargetName: "customer_name", - }, - "lines": { - TargetName: "lines", + "nullableFieldsTypeById": { + TargetName: "nullable_fields_type_by_id", + ArgumentMappings: grpcdatasource.FieldArgumentMap{ + "id": "id", }, }, - "Order": { - "orderId": { - TargetName: "order_id", - }, - "customerName": { - TargetName: "customer_name", - }, - "totalItems": { - TargetName: "total_items", - }, - "orderLines": { - TargetName: "order_lines", + "nullableFieldsTypeWithFilter": { + TargetName: "nullable_fields_type_with_filter", + ArgumentMappings: grpcdatasource.FieldArgumentMap{ + "filter": "filter", }, }, - "OrderLine": { - "productId": { - TargetName: "product_id", - }, - "quantity": { - TargetName: "quantity", - }, - "modifiers": { - TargetName: "modifiers", - }, + "allNullableFieldsTypes": { + TargetName: "all_nullable_fields_types", }, - "CategoryFilter": { - "category": { - TargetName: "category", - }, - "pagination": { - TargetName: "pagination", - }, + "blogPost": { + TargetName: "blog_post", }, - "Category": { - "id": { - TargetName: "id", - }, - "name": { - TargetName: "name", - }, - "kind": { - TargetName: "kind", - }, - "productCount": { - TargetName: "product_count", - ArgumentMappings: grpcdatasource.FieldArgumentMap{ - "filters": "filters", - }, + "blogPostById": { + TargetName: "blog_post_by_id", + ArgumentMappings: grpcdatasource.FieldArgumentMap{ + "id": "id", }, }, - "Cat": { - "id": { - TargetName: "id", - }, - "name": { - TargetName: "name", - }, - "kind": { - TargetName: "kind", - }, - "meowVolume": { - TargetName: "meow_volume", + "blogPostsWithFilter": { + TargetName: "blog_posts_with_filter", + ArgumentMappings: grpcdatasource.FieldArgumentMap{ + "filter": "filter", }, }, - "Dog": { - "id": { - TargetName: "id", - }, - "name": { - TargetName: "name", - }, - "kind": { - TargetName: "kind", - }, - "barkVolume": { - TargetName: "bark_volume", - }, + "allBlogPosts": { + TargetName: "all_blog_posts", }, - "ActionSuccess": { - "message": { - TargetName: "message", - }, - "timestamp": { - TargetName: "timestamp", - }, + "author": { + TargetName: "author", }, - "ActionError": { - "message": { - TargetName: "message", - }, - "code": { - TargetName: "code", + "authorById": { + TargetName: "author_by_id", + ArgumentMappings: grpcdatasource.FieldArgumentMap{ + "id": "id", }, }, - "SearchInput": { - "query": { - TargetName: "query", - }, - "limit": { - TargetName: "limit", + "authorsWithFilter": { + TargetName: "authors_with_filter", + ArgumentMappings: grpcdatasource.FieldArgumentMap{ + "filter": "filter", }, }, - "ActionInput": { - "type": { - TargetName: "type", - }, - "payload": { - TargetName: "payload", - }, + "allAuthors": { + TargetName: "all_authors", }, - "NullableFieldsType": { - "id": { - TargetName: "id", - }, - "name": { - TargetName: "name", - }, - "optionalString": { - TargetName: "optional_string", - }, - "optionalInt": { - TargetName: "optional_int", - }, - "optionalFloat": { - TargetName: "optional_float", - }, - "optionalBoolean": { - TargetName: "optional_boolean", - }, - "requiredString": { - TargetName: "required_string", - }, - "requiredInt": { - TargetName: "required_int", + "bulkSearchAuthors": { + TargetName: "bulk_search_authors", + ArgumentMappings: grpcdatasource.FieldArgumentMap{ + "filters": "filters", }, }, - "BlogPost": { - "id": { - TargetName: "id", - }, - "title": { - TargetName: "title", - }, - "content": { - TargetName: "content", - }, - "tags": { - TargetName: "tags", - }, - "optionalTags": { - TargetName: "optional_tags", - }, - "categories": { - TargetName: "categories", - }, - "keywords": { - TargetName: "keywords", - }, - "viewCounts": { - TargetName: "view_counts", - }, - "ratings": { - TargetName: "ratings", - }, - "isPublished": { - TargetName: "is_published", - }, - "tagGroups": { - TargetName: "tag_groups", - }, - "relatedTopics": { - TargetName: "related_topics", - }, - "commentThreads": { - TargetName: "comment_threads", - }, - "suggestions": { - TargetName: "suggestions", - }, - "relatedCategories": { - TargetName: "related_categories", - }, - "contributors": { - TargetName: "contributors", - }, - "mentionedProducts": { - TargetName: "mentioned_products", - }, - "mentionedUsers": { - TargetName: "mentioned_users", - }, - "categoryGroups": { - TargetName: "category_groups", - }, - "contributorTeams": { - TargetName: "contributor_teams", + "bulkSearchBlogPosts": { + TargetName: "bulk_search_blog_posts", + ArgumentMappings: grpcdatasource.FieldArgumentMap{ + "filters": "filters", }, }, - "Author": { - "id": { - TargetName: "id", - }, - "name": { - TargetName: "name", - }, - "email": { - TargetName: "email", - }, - "skills": { - TargetName: "skills", - }, - "languages": { - TargetName: "languages", - }, - "socialLinks": { - TargetName: "social_links", - }, - "teamsByProject": { - TargetName: "teams_by_project", - }, - "collaborations": { - TargetName: "collaborations", - }, - "writtenPosts": { - TargetName: "written_posts", - }, - "favoriteCategories": { - TargetName: "favorite_categories", - }, - "relatedAuthors": { - TargetName: "related_authors", - }, - "productReviews": { - TargetName: "product_reviews", - }, - "authorGroups": { - TargetName: "author_groups", - }, - "categoryPreferences": { - TargetName: "category_preferences", - }, - "projectTeams": { - TargetName: "project_teams", - }, }, - "BlogPostInput": { - "title": { - TargetName: "title", - }, - "content": { - TargetName: "content", - }, - "tags": { - TargetName: "tags", - }, - "optionalTags": { - TargetName: "optional_tags", - }, - "categories": { - TargetName: "categories", - }, - "keywords": { - TargetName: "keywords", - }, - "viewCounts": { - TargetName: "view_counts", - }, - "ratings": { - TargetName: "ratings", - }, - "isPublished": { - TargetName: "is_published", - }, - "tagGroups": { - TargetName: "tag_groups", - }, - "relatedTopics": { - TargetName: "related_topics", - }, - "commentThreads": { - TargetName: "comment_threads", - }, - "suggestions": { - TargetName: "suggestions", - }, - "relatedCategories": { - TargetName: "related_categories", - }, - "contributors": { - TargetName: "contributors", - }, - "categoryGroups": { - TargetName: "category_groups", + "Mutation": { + "createUser": { + TargetName: "create_user", + ArgumentMappings: grpcdatasource.FieldArgumentMap{ + "input": "input", }, }, - "AuthorInput": { - "name": { - TargetName: "name", - }, - "email": { - TargetName: "email", - }, - "skills": { - TargetName: "skills", - }, - "languages": { - TargetName: "languages", - }, - "socialLinks": { - TargetName: "social_links", - }, - "teamsByProject": { - TargetName: "teams_by_project", - }, - "collaborations": { - TargetName: "collaborations", - }, - "favoriteCategories": { - TargetName: "favorite_categories", - }, - "authorGroups": { - TargetName: "author_groups", - }, - "projectTeams": { - TargetName: "project_teams", + "performAction": { + TargetName: "perform_action", + ArgumentMappings: grpcdatasource.FieldArgumentMap{ + "input": "input", }, }, - "BlogPostFilter": { - "title": { - TargetName: "title", - }, - "hasCategories": { - TargetName: "has_categories", - }, - "minTags": { - TargetName: "min_tags", + "createNullableFieldsType": { + TargetName: "create_nullable_fields_type", + ArgumentMappings: grpcdatasource.FieldArgumentMap{ + "input": "input", }, }, - "AuthorFilter": { - "name": { - TargetName: "name", - }, - "hasTeams": { - TargetName: "has_teams", - }, - "skillCount": { - TargetName: "skill_count", + "updateNullableFieldsType": { + TargetName: "update_nullable_fields_type", + ArgumentMappings: grpcdatasource.FieldArgumentMap{ + "id": "id", + "input": "input", }, }, - "NullableFieldsInput": { - "name": { - TargetName: "name", - }, - "optionalString": { - TargetName: "optional_string", - }, - "optionalInt": { - TargetName: "optional_int", - }, - "optionalFloat": { - TargetName: "optional_float", - }, - "optionalBoolean": { - TargetName: "optional_boolean", - }, - "requiredString": { - TargetName: "required_string", - }, - "requiredInt": { - TargetName: "required_int", + "createBlogPost": { + TargetName: "create_blog_post", + ArgumentMappings: grpcdatasource.FieldArgumentMap{ + "input": "input", }, }, - "NullableFieldsFilter": { - "name": { - TargetName: "name", - }, - "optionalString": { - TargetName: "optional_string", - }, - "includeNulls": { - TargetName: "include_nulls", + "updateBlogPost": { + TargetName: "update_blog_post", + ArgumentMappings: grpcdatasource.FieldArgumentMap{ + "id": "id", + "input": "input", }, }, - "CategoryInput": { - "name": { - TargetName: "name", + "createAuthor": { + TargetName: "create_author", + ArgumentMappings: grpcdatasource.FieldArgumentMap{ + "input": "input", }, - "kind": { - TargetName: "kind", + }, + "updateAuthor": { + TargetName: "update_author", + ArgumentMappings: grpcdatasource.FieldArgumentMap{ + "id": "id", + "input": "input", }, }, - "ProductCountFilter": { - "minPrice": { - TargetName: "min_price", + "bulkCreateAuthors": { + TargetName: "bulk_create_authors", + ArgumentMappings: grpcdatasource.FieldArgumentMap{ + "authors": "authors", }, - "maxPrice": { - TargetName: "max_price", + }, + "bulkUpdateAuthors": { + TargetName: "bulk_update_authors", + ArgumentMappings: grpcdatasource.FieldArgumentMap{ + "authors": "authors", }, - "inStock": { - TargetName: "in_stock", + }, + "bulkCreateBlogPosts": { + TargetName: "bulk_create_blog_posts", + ArgumentMappings: grpcdatasource.FieldArgumentMap{ + "blogPosts": "blog_posts", }, - "searchTerm": { - TargetName: "search_term", + }, + "bulkUpdateBlogPosts": { + TargetName: "bulk_update_blog_posts", + ArgumentMappings: grpcdatasource.FieldArgumentMap{ + "blogPosts": "blog_posts", }, }, - "UserInput": { - "name": { - TargetName: "name", + }, + "Product": { + "id": { + TargetName: "id", + }, + "name": { + TargetName: "name", + }, + "price": { + TargetName: "price", + }, + }, + "Storage": { + "id": { + TargetName: "id", + }, + "name": { + TargetName: "name", + }, + "location": { + TargetName: "location", + }, + }, + "Warehouse": { + "id": { + TargetName: "id", + }, + "name": { + TargetName: "name", + }, + "location": { + TargetName: "location", + }, + }, + "User": { + "id": { + TargetName: "id", + }, + "name": { + TargetName: "name", + }, + }, + "NestedTypeA": { + "id": { + TargetName: "id", + }, + "name": { + TargetName: "name", + }, + "b": { + TargetName: "b", + }, + }, + "NestedTypeB": { + "id": { + TargetName: "id", + }, + "name": { + TargetName: "name", + }, + "c": { + TargetName: "c", + }, + }, + "NestedTypeC": { + "id": { + TargetName: "id", + }, + "name": { + TargetName: "name", + }, + }, + "RecursiveType": { + "id": { + TargetName: "id", + }, + "name": { + TargetName: "name", + }, + "recursiveType": { + TargetName: "recursive_type", + }, + }, + "TypeWithMultipleFilterFields": { + "id": { + TargetName: "id", + }, + "name": { + TargetName: "name", + }, + "filterField1": { + TargetName: "filter_field_1", + }, + "filterField2": { + TargetName: "filter_field_2", + }, + }, + "FilterTypeInput": { + "filterField1": { + TargetName: "filter_field_1", + }, + "filterField2": { + TargetName: "filter_field_2", + }, + }, + "TypeWithComplexFilterInput": { + "id": { + TargetName: "id", + }, + "name": { + TargetName: "name", + }, + }, + "FilterType": { + "name": { + TargetName: "name", + }, + "filterField1": { + TargetName: "filter_field_1", + }, + "filterField2": { + TargetName: "filter_field_2", + }, + "pagination": { + TargetName: "pagination", + }, + }, + "Pagination": { + "page": { + TargetName: "page", + }, + "perPage": { + TargetName: "per_page", + }, + }, + "ComplexFilterTypeInput": { + "filter": { + TargetName: "filter", + }, + }, + "OrderLineInput": { + "productId": { + TargetName: "product_id", + }, + "quantity": { + TargetName: "quantity", + }, + "modifiers": { + TargetName: "modifiers", + }, + }, + "OrderInput": { + "orderId": { + TargetName: "order_id", + }, + "customerName": { + TargetName: "customer_name", + }, + "lines": { + TargetName: "lines", + }, + }, + "Order": { + "orderId": { + TargetName: "order_id", + }, + "customerName": { + TargetName: "customer_name", + }, + "totalItems": { + TargetName: "total_items", + }, + "orderLines": { + TargetName: "order_lines", + }, + }, + "OrderLine": { + "productId": { + TargetName: "product_id", + }, + "quantity": { + TargetName: "quantity", + }, + "modifiers": { + TargetName: "modifiers", + }, + }, + "CategoryFilter": { + "category": { + TargetName: "category", + }, + "pagination": { + TargetName: "pagination", + }, + }, + "Category": { + "id": { + TargetName: "id", + }, + "name": { + TargetName: "name", + }, + "kind": { + TargetName: "kind", + }, + "productCount": { + TargetName: "product_count", + ArgumentMappings: grpcdatasource.FieldArgumentMap{ + "filters": "filters", }, }, + }, + "Cat": { + "id": { + TargetName: "id", + }, + "name": { + TargetName: "name", + }, + "kind": { + TargetName: "kind", + }, + "meowVolume": { + TargetName: "meow_volume", + }, + }, + "Dog": { + "id": { + TargetName: "id", + }, + "name": { + TargetName: "name", + }, + "kind": { + TargetName: "kind", + }, + "barkVolume": { + TargetName: "bark_volume", + }, + }, + "ActionSuccess": { + "message": { + TargetName: "message", + }, + "timestamp": { + TargetName: "timestamp", + }, + }, + "ActionError": { + "message": { + TargetName: "message", + }, + "code": { + TargetName: "code", + }, + }, + "SearchInput": { + "query": { + TargetName: "query", + }, + "limit": { + TargetName: "limit", + }, + }, + "ActionInput": { + "type": { + TargetName: "type", + }, + "payload": { + TargetName: "payload", + }, + }, + "NullableFieldsType": { + "id": { + TargetName: "id", + }, + "name": { + TargetName: "name", + }, + "optionalString": { + TargetName: "optional_string", + }, + "optionalInt": { + TargetName: "optional_int", + }, + "optionalFloat": { + TargetName: "optional_float", + }, + "optionalBoolean": { + TargetName: "optional_boolean", + }, + "requiredString": { + TargetName: "required_string", + }, + "requiredInt": { + TargetName: "required_int", + }, + }, + "BlogPost": { + "id": { + TargetName: "id", + }, + "title": { + TargetName: "title", + }, + "content": { + TargetName: "content", + }, + "tags": { + TargetName: "tags", + }, + "optionalTags": { + TargetName: "optional_tags", + }, + "categories": { + TargetName: "categories", + }, + "keywords": { + TargetName: "keywords", + }, + "viewCounts": { + TargetName: "view_counts", + }, + "ratings": { + TargetName: "ratings", + }, + "isPublished": { + TargetName: "is_published", + }, + "tagGroups": { + TargetName: "tag_groups", + }, + "relatedTopics": { + TargetName: "related_topics", + }, + "commentThreads": { + TargetName: "comment_threads", + }, + "suggestions": { + TargetName: "suggestions", + }, + "relatedCategories": { + TargetName: "related_categories", + }, + "contributors": { + TargetName: "contributors", + }, + "mentionedProducts": { + TargetName: "mentioned_products", + }, + "mentionedUsers": { + TargetName: "mentioned_users", + }, + "categoryGroups": { + TargetName: "category_groups", + }, + "contributorTeams": { + TargetName: "contributor_teams", + }, + }, + "Author": { + "id": { + TargetName: "id", + }, + "name": { + TargetName: "name", + }, + "email": { + TargetName: "email", + }, + "skills": { + TargetName: "skills", + }, + "languages": { + TargetName: "languages", + }, + "socialLinks": { + TargetName: "social_links", + }, + "teamsByProject": { + TargetName: "teams_by_project", + }, + "collaborations": { + TargetName: "collaborations", + }, + "writtenPosts": { + TargetName: "written_posts", + }, + "favoriteCategories": { + TargetName: "favorite_categories", + }, + "relatedAuthors": { + TargetName: "related_authors", + }, + "productReviews": { + TargetName: "product_reviews", + }, + "authorGroups": { + TargetName: "author_groups", + }, + "categoryPreferences": { + TargetName: "category_preferences", + }, + "projectTeams": { + TargetName: "project_teams", + }, + }, + "BlogPostInput": { + "title": { + TargetName: "title", + }, + "content": { + TargetName: "content", + }, + "tags": { + TargetName: "tags", + }, + "optionalTags": { + TargetName: "optional_tags", + }, + "categories": { + TargetName: "categories", + }, + "keywords": { + TargetName: "keywords", + }, + "viewCounts": { + TargetName: "view_counts", + }, + "ratings": { + TargetName: "ratings", + }, + "isPublished": { + TargetName: "is_published", + }, + "tagGroups": { + TargetName: "tag_groups", + }, + "relatedTopics": { + TargetName: "related_topics", + }, + "commentThreads": { + TargetName: "comment_threads", + }, + "suggestions": { + TargetName: "suggestions", + }, + "relatedCategories": { + TargetName: "related_categories", + }, + "contributors": { + TargetName: "contributors", + }, + "categoryGroups": { + TargetName: "category_groups", + }, + }, + "AuthorInput": { + "name": { + TargetName: "name", + }, + "email": { + TargetName: "email", + }, + "skills": { + TargetName: "skills", + }, + "languages": { + TargetName: "languages", + }, + "socialLinks": { + TargetName: "social_links", + }, + "teamsByProject": { + TargetName: "teams_by_project", + }, + "collaborations": { + TargetName: "collaborations", + }, + "favoriteCategories": { + TargetName: "favorite_categories", + }, + "authorGroups": { + TargetName: "author_groups", + }, + "projectTeams": { + TargetName: "project_teams", + }, + }, + "BlogPostFilter": { + "title": { + TargetName: "title", + }, + "hasCategories": { + TargetName: "has_categories", + }, + "minTags": { + TargetName: "min_tags", + }, + }, + "AuthorFilter": { + "name": { + TargetName: "name", + }, + "hasTeams": { + TargetName: "has_teams", + }, + "skillCount": { + TargetName: "skill_count", + }, + }, + "NullableFieldsInput": { + "name": { + TargetName: "name", + }, + "optionalString": { + TargetName: "optional_string", + }, + "optionalInt": { + TargetName: "optional_int", + }, + "optionalFloat": { + TargetName: "optional_float", + }, + "optionalBoolean": { + TargetName: "optional_boolean", + }, + "requiredString": { + TargetName: "required_string", + }, + "requiredInt": { + TargetName: "required_int", + }, + }, + "NullableFieldsFilter": { + "name": { + TargetName: "name", + }, + "optionalString": { + TargetName: "optional_string", + }, + "includeNulls": { + TargetName: "include_nulls", + }, + }, + "CategoryInput": { + "name": { + TargetName: "name", + }, + "kind": { + TargetName: "kind", + }, + }, + "ProductCountFilter": { + "minPrice": { + TargetName: "min_price", + }, + "maxPrice": { + TargetName: "max_price", + }, + "inStock": { + TargetName: "in_stock", + }, + "searchTerm": { + TargetName: "search_term", + }, + }, + "UserInput": { + "name": { + TargetName: "name", + }, + }, }, } } + // MustDefaultGRPCMapping returns the default GRPC mapping func MustDefaultGRPCMapping(t *testing.T) *grpcdatasource.GRPCMapping { mapping := DefaultGRPCMapping() return mapping } + + diff --git a/v2/pkg/grpctest/product.proto b/v2/pkg/grpctest/product.proto index 24a170195..43ee9aab4 100644 --- a/v2/pkg/grpctest/product.proto +++ b/v2/pkg/grpctest/product.proto @@ -620,28 +620,28 @@ message MutationBulkUpdateBlogPostsRequest { message MutationBulkUpdateBlogPostsResponse { repeated BlogPost bulk_update_blog_posts = 1; } -message CategoryProductCountArgs { +message ResolveCategoryProductCountArgs { ProductCountFilter filters = 1; } -message CategoryProductCountContext { +message ResolveCategoryProductCountContext { string id = 1; string name = 2; } message ResolveCategoryProductCountRequest { // context provides the resolver context for the field productCount of type Category. - repeated CategoryProductCountContext context = 1; + repeated ResolveCategoryProductCountContext context = 1; // field_args provides the arguments for the resolver field productCount of type Category. - CategoryProductCountArgs field_args = 2; + ResolveCategoryProductCountArgs field_args = 2; } -message ResolveCategoryProductCountResponseResult { +message ResolveCategoryProductCountResult { int32 product_count = 1; } message ResolveCategoryProductCountResponse { - repeated ResolveCategoryProductCountResponseResult result = 1; + repeated ResolveCategoryProductCountResult result = 1; } message Product { From 234bf5dbb1fb3325bc5c34de8229acfc86eb8712 Mon Sep 17 00:00:00 2001 From: Ludwig Bedacht Date: Wed, 8 Oct 2025 12:06:53 +0200 Subject: [PATCH 04/72] feat: handle list types --- v2/pkg/ast/path.go | 15 + .../datasource/grpc_datasource/compiler.go | 141 +- .../execution_plan_field_resolvers_test.go | 409 ++- .../grpc_datasource/execution_plan_visitor.go | 25 +- .../grpc_datasource/grpc_datasource.go | 33 +- .../grpc_datasource/grpc_datasource_test.go | 56 +- .../grpc_datasource/json_builder.go | 19 +- .../grpc_datasource/mapping_test_helper.go | 53 + v2/pkg/grpctest/mapping/mapping.go | 1511 +++++----- v2/pkg/grpctest/mockservice.go | 121 +- v2/pkg/grpctest/product.proto | 47 + v2/pkg/grpctest/productv1/product.pb.go | 2538 ++++++++++------- v2/pkg/grpctest/productv1/product_grpc.pb.go | 38 + v2/pkg/grpctest/testdata/products.graphqls | 17 + 14 files changed, 3183 insertions(+), 1840 deletions(-) diff --git a/v2/pkg/ast/path.go b/v2/pkg/ast/path.go index 8d28790e8..28f66fa7a 100644 --- a/v2/pkg/ast/path.go +++ b/v2/pkg/ast/path.go @@ -88,6 +88,21 @@ func (p Path) WithPathElement(element PathItem) Path { return res } +func (p Path) WithFieldNameItem(fieldName []byte) Path { + return p.WithPathElement(PathItem{ + Kind: FieldName, + FieldName: fieldName, + }) +} + +func (p Path) RemoveLastItem() Path { + if len(p) == 0 { + return p + } + + return p[:len(p)-1] +} + func (p Path) String() string { out := "[" for i := range p { diff --git a/v2/pkg/engine/datasource/grpc_datasource/compiler.go b/v2/pkg/engine/datasource/grpc_datasource/compiler.go index 0df1098ac..76f061f51 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/compiler.go +++ b/v2/pkg/engine/datasource/grpc_datasource/compiler.go @@ -4,6 +4,7 @@ import ( "context" "fmt" "slices" + "strings" "github.com/bufbuild/protocompile" "github.com/cespare/xxhash/v2" @@ -415,8 +416,8 @@ type ServiceCall struct { Input *dynamicpb.Message // Output is the output message for the gRPC call Output *dynamicpb.Message - // Call is the call that was made to the gRPC service - Call *RPCCall // TODO: Might be not needed anymore when we are using the DependencyGraph + // RPC is the call that was made to the gRPC service + RPC *RPCCall } // func (p *RPCCompiler) CompileFetches(graph *DependencyGraph, fetches []FetchItem, inputData gjson.Result) ([]Invocation, error) { @@ -514,7 +515,7 @@ func (p *RPCCompiler) CompileNode(graph *DependencyGraph, fetch FetchItem, input MethodName: call.MethodName, Input: request, Output: response, - Call: call, + RPC: call, }, nil } @@ -553,7 +554,7 @@ func (p *RPCCompiler) Compile(executionPlan *RPCExecutionPlan, inputData gjson.R MethodName: call.MethodName, Input: request, Output: response, - Call: &call, + RPC: &call, }) } @@ -612,20 +613,6 @@ func (p *RPCCompiler) buildProtoMessageWithContext(inputMessage Message, rpcMess return nil } - // schemaField := inputMessage.FieldByName("key") - // if schemaField == nil { - // p.report.AddInternalError(fmt.Errorf("key field not found in message %s", inputMessage.Name)) - // return nil - // } - - // // the key field must be a repeated field - // // Get the RPC field - // planKeyField := rpcMessage.Fields.ByName(schemaField.Name) - // if planKeyField == nil { - // p.report.AddInternalError(fmt.Errorf("key field not found in message %s", rpcMessage.Name)) - // return nil - // } - contextSchemaField := inputMessage.FieldByName("context") if contextSchemaField == nil { p.report.AddInternalError(fmt.Errorf("context field not found in message %s", inputMessage.Name)) @@ -714,7 +701,7 @@ func (p *RPCCompiler) resolveContextDataForPath(message protoref.Message, path a return p.resolveListDataForPath(msg.List(), fd, path) } - return []protoref.Value{p.resolveDataForPath(msg.Message(), path)} + return p.resolveDataForPath(msg.Message(), path) } @@ -730,56 +717,64 @@ func (p *RPCCompiler) resolveListDataForPath(message protoref.List, fd protoref. switch fd.Kind() { case protoref.MessageKind: - val := p.resolveDataForPath(item.Message(), path) + values := p.resolveDataForPath(item.Message(), path) - if list, isList := val.Interface().(protoref.List); isList { - values := p.resolveListDataForPath(list, fd, path) - result = append(result, values...) - continue + for _, val := range values { + if list, isList := val.Interface().(protoref.List); isList { + values := p.resolveListDataForPath(list, fd, path) + result = append(result, values...) + continue + } } - result = append(result, val) + result = append(result, values...) default: result = append(result, item) } - - // val := p.resolveDataForPath(item.Message(), path) - // if list, isList := val.Interface().(protoref.List); isList { - // value := p.resolveListDataForPath(list, fd, path) - // result = append(result, value...) - // continue - // } - - // result = append(result, val) } return result } -func (p *RPCCompiler) resolveDataForPath(outputMessage protoref.Message, path ast.Path) protoref.Value { +func (p *RPCCompiler) resolveDataForPath(messsage protoref.Message, path ast.Path) []protoref.Value { if path.Len() == 0 { - return protoref.Value{} + return nil } segment := path[0] - field, fd := p.getMessageField(outputMessage, segment.FieldName.String()) + + if fn := segment.FieldName.String(); strings.HasPrefix(fn, "@") { + list := p.resolveUnderlyingList(messsage, fn) + + result := make([]protoref.Value, 0, len(list)) + for _, item := range list { + result = append(result, p.resolveDataForPath(item.Message(), path[1:])...) + } + + return result + } + + field, fd := p.getMessageField(messsage, segment.FieldName.String()) if !field.IsValid() { - return protoref.Value{} + return nil } switch fd.Kind() { case protoref.MessageKind: if fd.IsList() { - return field + return []protoref.Value{field} } return p.resolveDataForPath(field.Message(), path[1:]) + default: + return []protoref.Value{field} } - - return field } +// getMessageField gets the field from the message by its name. +// It also handles nested lists and nullable lists. func (p *RPCCompiler) getMessageField(message protoref.Message, fieldName string) (protoref.Value, protoref.FieldDescriptor) { + fd := message.Descriptor().Fields().ByName(protoref.Name(fieldName)) if fd == nil { return protoref.Value{}, nil @@ -788,8 +783,68 @@ func (p *RPCCompiler) getMessageField(message protoref.Message, fieldName string return message.Get(fd), fd } -func (p *RPCCompiler) newEmptyListMessageByNumber(msg *dynamicpb.Message, number int32) protoref.List { - return msg.Mutable(msg.Descriptor().Fields().ByNumber(protoref.FieldNumber(number))).List() +// resolveUnderlyingList resolves the underlying list message from a nested list message. +// +// message ListOfFloat { +// message List { +// repeated double items = 1; +// } +// List list = 1; +// } +func (p *RPCCompiler) resolveUnderlyingList(msg protoref.Message, fieldName string) []protoref.Value { + nestingLevel := 0 + for _, char := range fieldName { + if char != '@' { + break + } + nestingLevel++ + } + + listFieldValue := msg.Get(msg.Descriptor().Fields().ByName(protoref.Name(fieldName[nestingLevel:]))) + if !listFieldValue.IsValid() { + return nil + } + + return p.resolveUnderlyingListItems(listFieldValue, nestingLevel) + +} + +func (p *RPCCompiler) resolveUnderlyingListItems(value protoref.Value, nestingLevel int) []protoref.Value { + msg := value.Message() + fd := msg.Descriptor().Fields().ByNumber(1) + if fd == nil { + return nil + } + + listMsg := msg.Get(fd) + if !listMsg.IsValid() { + return nil + } + + itemsValue := listMsg.Message().Get(listMsg.Message().Descriptor().Fields().ByNumber(1)) + if !itemsValue.IsValid() { + return nil + } + + if itemsValue.List().Len() == 0 { + return nil + } + + if nestingLevel > 1 { + items := make([]protoref.Value, 0, listMsg.List().Len()) + for i := 0; i < listMsg.List().Len(); i++ { + items = append(items, p.resolveUnderlyingListItems(listMsg.List().Get(i), nestingLevel-1)...) + } + + return items + } + + result := make([]protoref.Value, itemsValue.List().Len()) + for i := 0; i < itemsValue.List().Len(); i++ { + result[i] = itemsValue.List().Get(i) + } + + return result } func (p *RPCCompiler) newEmptyListMessageByName(msg *dynamicpb.Message, name string) protoref.List { diff --git a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_field_resolvers_test.go b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_field_resolvers_test.go index a9861b3fa..f189a671e 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_field_resolvers_test.go +++ b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_field_resolvers_test.go @@ -13,7 +13,7 @@ func TestExecutionPlanFieldResolvers(t *testing.T) { expectedError string }{ { - name: "Should create an execution plan for a query with nullable fields type", + name: "Should create an execution plan for a query with a field resolver", query: "query CategoriesWithFieldResolvers($whoop: ProductCountFilter) { categories { id name kind productCount(filters: $whoop) } }", expectedPlan: &RPCExecutionPlan{ Calls: []RPCCall{ @@ -160,6 +160,413 @@ func TestExecutionPlanFieldResolvers(t *testing.T) { }, }, }, + { + name: "Should create an execution plan for a query with a field resolver and aliases", + query: "query CategoriesWithFieldResolversAndAliases($p1: ProductCountFilter, $p2: ProductCountFilter) { categories { productCount1: productCount(filters: $p1) productCount2: productCount(filters: $p2) } }", + expectedPlan: &RPCExecutionPlan{ + Calls: []RPCCall{ + { + ServiceName: "Products", + MethodName: "QueryCategories", + Request: RPCMessage{ + Name: "QueryCategoriesRequest", + }, + Response: RPCMessage{ + Name: "QueryCategoriesResponse", + Fields: []RPCField{ + { + Name: "categories", + TypeName: string(DataTypeMessage), + JSONPath: "categories", + Repeated: true, + Message: &RPCMessage{ + Name: "Category", + Fields: []RPCField{}, + }, + }, + }, + }, + }, + { + DependentCalls: []int{0}, + ServiceName: "Products", + MethodName: "ResolveCategoryProductCount", + Kind: CallKindResolve, + ResponsePath: buildPath("categories.productCount1"), + Request: RPCMessage{ + Name: "ResolveCategoryProductCountRequest", + Fields: []RPCField{ + { + Name: "context", + TypeName: string(DataTypeMessage), + JSONPath: "", + Repeated: true, + Message: &RPCMessage{ + Name: "ResolveCategoryProductCountContext", + Fields: []RPCField{ + { + Name: "id", + TypeName: string(DataTypeString), + JSONPath: "id", + ResolvePath: buildPath("categories.id"), + }, + { + Name: "name", + TypeName: string(DataTypeString), + JSONPath: "name", + ResolvePath: buildPath("categories.name"), + }, + }, + }, + }, + { + Name: "field_args", + TypeName: string(DataTypeMessage), + JSONPath: "", + Message: &RPCMessage{ + Name: "ResolveCategoryProductCountArgs", + Fields: []RPCField{ + { + Name: "filters", + TypeName: string(DataTypeMessage), + JSONPath: "p1", + Optional: true, + Message: &RPCMessage{ + Name: "ProductCountFilter", + Fields: []RPCField{ + { + Name: "min_price", + TypeName: string(DataTypeDouble), + JSONPath: "minPrice", + Optional: true, + }, + { + Name: "max_price", + TypeName: string(DataTypeDouble), + JSONPath: "maxPrice", + Optional: true, + }, + { + Name: "in_stock", + TypeName: string(DataTypeBool), + JSONPath: "inStock", + Optional: true, + }, + { + Name: "search_term", + TypeName: string(DataTypeString), + JSONPath: "searchTerm", + Optional: true, + }, + }, + }, + }, + }, + }, + }, + }, + }, + Response: RPCMessage{ + Name: "ResolveCategoryProductCountResponse", + Fields: []RPCField{ + { + Name: "result", + TypeName: string(DataTypeMessage), + JSONPath: "result", + Repeated: true, + Message: &RPCMessage{ + Name: "ResolveCategoryProductCountResult", + Fields: []RPCField{ + { + Name: "product_count", + TypeName: string(DataTypeInt32), + JSONPath: "productCount1", + }, + }, + }, + }, + }, + }, + }, + { + DependentCalls: []int{0}, + ServiceName: "Products", + MethodName: "ResolveCategoryProductCount", + Kind: CallKindResolve, + ResponsePath: buildPath("categories.productCount2"), + Request: RPCMessage{ + Name: "ResolveCategoryProductCountRequest", + Fields: []RPCField{ + { + Name: "context", + TypeName: string(DataTypeMessage), + JSONPath: "", + Repeated: true, + Message: &RPCMessage{ + Name: "ResolveCategoryProductCountContext", + Fields: []RPCField{ + { + Name: "id", + TypeName: string(DataTypeString), + JSONPath: "id", + ResolvePath: buildPath("categories.id"), + }, + { + Name: "name", + TypeName: string(DataTypeString), + JSONPath: "name", + ResolvePath: buildPath("categories.name"), + }, + }, + }, + }, + { + Name: "field_args", + TypeName: string(DataTypeMessage), + JSONPath: "", + Message: &RPCMessage{ + Name: "ResolveCategoryProductCountArgs", + Fields: []RPCField{ + { + Name: "filters", + TypeName: string(DataTypeMessage), + JSONPath: "p2", + Optional: true, + Message: &RPCMessage{ + Name: "ProductCountFilter", + Fields: []RPCField{ + { + Name: "min_price", + TypeName: string(DataTypeDouble), + JSONPath: "minPrice", + Optional: true, + }, + { + Name: "max_price", + TypeName: string(DataTypeDouble), + JSONPath: "maxPrice", + Optional: true, + }, + { + Name: "in_stock", + TypeName: string(DataTypeBool), + JSONPath: "inStock", + Optional: true, + }, + { + Name: "search_term", + TypeName: string(DataTypeString), + JSONPath: "searchTerm", + Optional: true, + }, + }, + }, + }, + }, + }, + }, + }, + }, + Response: RPCMessage{ + Name: "ResolveCategoryProductCountResponse", + Fields: []RPCField{ + { + Name: "result", + TypeName: string(DataTypeMessage), + JSONPath: "result", + Repeated: true, + Message: &RPCMessage{ + Name: "ResolveCategoryProductCountResult", + Fields: []RPCField{ + { + Name: "product_count", + TypeName: string(DataTypeInt32), + JSONPath: "productCount2", + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + { + name: "Should create an execution plan for a query with nullable fields type", + query: "query SubcategoriesWithFieldResolvers($filter: SubcategoryItemFilter) { categories { id subcategories { id name description isActive itemCount(filters: $filter) } } }", + expectedPlan: &RPCExecutionPlan{ + Calls: []RPCCall{ + { + ServiceName: "Products", + MethodName: "QueryCategories", + Request: RPCMessage{ + Name: "QueryCategoriesRequest", + }, + Response: RPCMessage{ + Name: "QueryCategoriesResponse", + Fields: []RPCField{ + { + Name: "categories", + TypeName: string(DataTypeMessage), + JSONPath: "categories", + Repeated: true, + Message: &RPCMessage{ + Name: "Category", + Fields: []RPCField{ + { + Name: "id", + TypeName: string(DataTypeString), + JSONPath: "id", + }, + { + Name: "subcategories", + TypeName: string(DataTypeMessage), + JSONPath: "subcategories", + Repeated: false, + IsListType: true, + Optional: true, + ListMetadata: &ListMetadata{ + NestingLevel: 1, + LevelInfo: []LevelInfo{{Optional: true}}, + }, + Message: &RPCMessage{ + Name: "Subcategory", + Fields: []RPCField{ + { + Name: "id", + TypeName: string(DataTypeString), + JSONPath: "id", + }, + { + Name: "name", + TypeName: string(DataTypeString), + JSONPath: "name", + }, + { + Name: "description", + TypeName: string(DataTypeString), + JSONPath: "description", + Optional: true, + }, + { + Name: "is_active", + TypeName: string(DataTypeBool), + JSONPath: "isActive", + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + { + DependentCalls: []int{0}, + ServiceName: "Products", + MethodName: "ResolveSubcategoryItemCount", + Kind: CallKindResolve, + ResponsePath: buildPath("categories.subcategories.itemCount"), + Request: RPCMessage{ + Name: "ResolveSubcategoryItemCountRequest", + Fields: []RPCField{ + { + Name: "context", + TypeName: string(DataTypeMessage), + JSONPath: "", + Repeated: true, + Message: &RPCMessage{ + Name: "ResolveSubcategoryItemCountContext", + Fields: []RPCField{ + { + Name: "id", + TypeName: string(DataTypeString), + JSONPath: "id", + ResolvePath: buildPath("categories.@subcategories.id"), + }, + }, + }, + }, + { + Name: "field_args", + TypeName: string(DataTypeMessage), + JSONPath: "", + Message: &RPCMessage{ + Name: "ResolveSubcategoryItemCountArgs", + Fields: []RPCField{ + { + Name: "filters", + TypeName: string(DataTypeMessage), + JSONPath: "filter", + Optional: true, + Message: &RPCMessage{ + Name: "SubcategoryItemFilter", + Fields: []RPCField{ + { + Name: "min_price", + TypeName: string(DataTypeDouble), + JSONPath: "minPrice", + Optional: true, + }, + { + Name: "max_price", + TypeName: string(DataTypeDouble), + JSONPath: "maxPrice", + Optional: true, + }, + { + Name: "in_stock", + TypeName: string(DataTypeBool), + JSONPath: "inStock", + Optional: true, + }, + { + Name: "is_active", + TypeName: string(DataTypeBool), + JSONPath: "isActive", + Optional: true, + }, + { + Name: "search_term", + TypeName: string(DataTypeString), + JSONPath: "searchTerm", + Optional: true, + }, + }, + }, + }, + }, + }, + }, + }, + }, + Response: RPCMessage{ + Name: "ResolveSubcategoryItemCountResponse", + Fields: []RPCField{ + { + Name: "result", + TypeName: string(DataTypeMessage), + JSONPath: "result", + Repeated: true, + Message: &RPCMessage{ + Name: "ResolveSubcategoryItemCountResult", + Fields: []RPCField{ + { + Name: "item_count", + TypeName: string(DataTypeInt32), + JSONPath: "itemCount", + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, } for _, tt := range tests { diff --git a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor.go b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor.go index f18b46d3c..27e530557 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor.go +++ b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor.go @@ -3,6 +3,7 @@ package grpcdatasource import ( "errors" "fmt" + "strings" "golang.org/x/text/cases" "golang.org/x/text/language" @@ -65,6 +66,8 @@ type rpcPlanVisitor struct { relatedCallID int resolvedFields []resolvedField + + fieldPath ast.Path } type rpcPlanVisitorConfig struct { @@ -85,6 +88,7 @@ func newRPCPlanVisitor(config rpcPlanVisitorConfig) *rpcPlanVisitor { operationFieldRef: -1, resolvedFields: make([]resolvedField, 0), relatedCallID: -1, + fieldPath: make(ast.Path, 0), } walker.RegisterDocumentVisitor(visitor) @@ -413,10 +417,7 @@ func (r *rpcPlanVisitor) EnterField(ref int) { callerRef: r.relatedCallID, parentTypeRef: r.walker.EnclosingTypeDefinition.Ref, fieldRef: ref, - responsePath: r.walker.Path[1:].WithoutInlineFragmentNames().WithPathElement(ast.PathItem{ - Kind: ast.FieldName, - FieldName: r.operation.FieldAliasOrNameBytes(ref), - }), + responsePath: r.walker.Path[1:].WithoutInlineFragmentNames().WithFieldNameItem(r.operation.FieldAliasOrNameBytes(ref)), } contextFields, err := r.planCtx.resolveContextFields(r.walker, fd) @@ -427,10 +428,7 @@ func (r *rpcPlanVisitor) EnterField(ref int) { for _, contextFieldRef := range contextFields { contextFieldName := r.definition.FieldDefinitionNameBytes(contextFieldRef) // TODO handle aliases - resolvedPath := append(r.walker.Path[1:].WithoutInlineFragmentNames(), ast.PathItem{ - Kind: ast.FieldName, - FieldName: contextFieldName, - }) + resolvedPath := r.fieldPath.WithFieldNameItem(contextFieldName) resolvedField.contextFields = append(resolvedField.contextFields, contextField{ fieldRef: contextFieldRef, @@ -446,6 +444,7 @@ func (r *rpcPlanVisitor) EnterField(ref int) { resolvedField.fieldArguments = fieldArguments r.resolvedFields = append(r.resolvedFields, resolvedField) + r.fieldPath = r.fieldPath.WithFieldNameItem(r.operation.FieldNameBytes(ref)) return } @@ -455,6 +454,14 @@ func (r *rpcPlanVisitor) EnterField(ref int) { return } + // If we have a nested or nullable list, we add a @ prefix to indicate the nesting level. + prefix := "" + if field.ListMetadata != nil { + prefix = strings.Repeat("@", field.ListMetadata.NestingLevel) + } + + r.fieldPath = r.fieldPath.WithFieldNameItem([]byte(prefix + field.Name)) + // check if we are inside of an inline fragment if ref, ok := r.walker.ResolveInlineFragment(); ok { if r.planInfo.currentResponseMessage.FieldSelectionSet == nil { @@ -471,6 +478,8 @@ func (r *rpcPlanVisitor) EnterField(ref int) { // LeaveField implements astvisitor.FieldVisitor. func (r *rpcPlanVisitor) LeaveField(ref int) { + r.fieldPath = r.fieldPath.RemoveLastItem() + // If we are not in the operation field, we can increment the response field index. if !r.walker.InRootField() { // If the field has arguments, we need to decrement the related call ID. diff --git a/v2/pkg/engine/datasource/grpc_datasource/grpc_datasource.go b/v2/pkg/engine/datasource/grpc_datasource/grpc_datasource.go index acb428ec2..1ae2b4688 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/grpc_datasource.go +++ b/v2/pkg/engine/datasource/grpc_datasource/grpc_datasource.go @@ -24,6 +24,12 @@ import ( "github.com/wundergraph/graphql-go-tools/v2/pkg/engine/resolve" ) +type resultData struct { + kind CallKind + response *astjson.Value + responsePath ast.Path +} + // Verify DataSource implements the resolve.DataSource interface var _ resolve.DataSource = (*DataSource)(nil) @@ -100,7 +106,7 @@ func (d *DataSource) Load(ctx context.Context, input []byte, out *bytes.Buffer) return err } - responses := make([]*astjson.Value, len(serviceCalls)) + results := make([]resultData, len(serviceCalls)) errGrp, errGrpCtx := errgroup.WithContext(ctx) mu := sync.Mutex{} @@ -116,24 +122,30 @@ func (d *DataSource) Load(ctx context.Context, input []byte, out *bytes.Buffer) return err } - response, err := builder.marshalResponseJSON(&a, &serviceCall.Call.Response, serviceCall.Output) + response, err := builder.marshalResponseJSON(&a, &serviceCall.RPC.Response, serviceCall.Output) if err != nil { return nil } - if serviceCall.Call.Kind == CallKindResolve { - return builder.mergeWithPath(root, response, serviceCall.Call.ResponsePath) - } + // if serviceCall.RPC.Kind == CallKindResolve { + // return builder.mergeWithPath(root, response, serviceCall.RPC.ResponsePath) + // } // In case of a federated response, we need to ensure that the response is valid. // The number of entities per type must match the number of lookup keys in the variablese + // if serviceCall.RPC.Kind == CallKindEntity { err = builder.validateFederatedResponse(response) if err != nil { return err } + // } mu.Lock() - responses[index] = response + results[index] = resultData{ + kind: serviceCall.RPC.Kind, + response: response, + responsePath: serviceCall.RPC.ResponsePath, + } mu.Unlock() return nil @@ -146,8 +158,13 @@ func (d *DataSource) Load(ctx context.Context, input []byte, out *bytes.Buffer) return nil } - for _, response := range responses { - root, err = builder.mergeValues(root, response) + for _, result := range results { + switch result.kind { + case CallKindResolve: + err = builder.mergeWithPath(root, result.response, result.responsePath) + default: + root, err = builder.mergeValues(root, result.response) + } if err != nil { out.Write(builder.writeErrorBytes(err)) return err diff --git a/v2/pkg/engine/datasource/grpc_datasource/grpc_datasource_test.go b/v2/pkg/engine/datasource/grpc_datasource/grpc_datasource_test.go index 44ff808ca..2f56967fd 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/grpc_datasource_test.go +++ b/v2/pkg/engine/datasource/grpc_datasource/grpc_datasource_test.go @@ -3657,10 +3657,52 @@ func Test_Datasource_Load_WithFieldResolvers(t *testing.T) { validate func(t *testing.T, data map[string]interface{}) validateError func(t *testing.T, errData []graphqlError) }{ - { - name: "Query with field resolvers", - query: `query CategoriesWithFieldResolvers($filters: ProductCountFilter) { categories { id name kind productCount(filters: $filters) } }`, - vars: `{"variables":{"filters":{"minPrice":100}}}`, + // { + // name: "Query with field resolvers", + // query: `query CategoriesWithFieldResolvers($filters: ProductCountFilter) { categories { id name kind productCount(filters: $filters) } }`, + // vars: `{"variables":{"filters":{"minPrice":100}}}`, + // validate: func(t *testing.T, data map[string]interface{}) { + // require.NotEmpty(t, data) + + // categories, ok := data["categories"].([]interface{}) + // require.True(t, ok, "categories should be an array") + // require.NotEmpty(t, categories, "categories should not be empty") + // require.Len(t, categories, 4, "Should return 1 category") + + // for productCount, category := range categories { + // category, ok := category.(map[string]interface{}) + // require.True(t, ok, "category should be an object") + // require.NotEmpty(t, category["id"]) + // require.NotEmpty(t, category["name"]) + // require.NotEmpty(t, category["kind"]) + // require.Equal(t, float64(productCount), category["productCount"]) + // } + + // }, + // validateError: func(t *testing.T, errData []graphqlError) { + // require.Empty(t, errData) + // }, + // }, + // { + // name: "Query with field resolvers and nullable lists", + // query: "query SubcategoriesWithFieldResolvers($filter: SubcategoryItemFilter) { categories { id subcategories { id name description isActive itemCount(filters: $filter) } } }", + // vars: `{"variables":{"filter":{"isActive":true}}}`, + // validate: func(t *testing.T, data map[string]interface{}) { + // require.NotEmpty(t, data) + + // categories, ok := data["categories"].([]interface{}) + // require.True(t, ok, "categories should be an array") + // require.NotEmpty(t, categories, "categories should not be empty") + // require.Len(t, categories, 4, "Should return 1 category") + // }, + // validateError: func(t *testing.T, errData []graphqlError) { + // require.Empty(t, errData) + // }, + // }, + { + name: "Query with field resolvers and aliases", + query: "query CategoriesWithFieldResolversAndAliases($filter1: ProductCountFilter, $filter2: ProductCountFilter) { categories { productCount1: productCount(filters: $filter1) productCount2: productCount(filters: $filter2) } }", + vars: `{"variables":{"filter1":{"minPrice":100},"filter2":{"minPrice":200}}}`, validate: func(t *testing.T, data map[string]interface{}) { require.NotEmpty(t, data) @@ -3672,10 +3714,8 @@ func Test_Datasource_Load_WithFieldResolvers(t *testing.T) { for productCount, category := range categories { category, ok := category.(map[string]interface{}) require.True(t, ok, "category should be an object") - require.NotEmpty(t, category["id"]) - require.NotEmpty(t, category["name"]) - require.NotEmpty(t, category["kind"]) - require.Equal(t, float64(productCount), category["productCount"]) + require.Equal(t, float64(productCount), category["productCount1"]) + require.Equal(t, float64(productCount), category["productCount2"]) } }, diff --git a/v2/pkg/engine/datasource/grpc_datasource/json_builder.go b/v2/pkg/engine/datasource/grpc_datasource/json_builder.go index d533d9d92..e6133e320 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/json_builder.go +++ b/v2/pkg/engine/datasource/grpc_datasource/json_builder.go @@ -17,10 +17,10 @@ import ( // Standard GraphQL response paths const ( - entityPath = "_entities" // Path for federated entities in response - dataPath = "data" // Standard GraphQL data wrapper - errorsPath = "errors" // Standard GraphQL errors array - resolvResponsePath = "result" // Path for resolve response + entityPath = "_entities" // Path for federated entities in response + dataPath = "data" // Standard GraphQL data wrapper + errorsPath = "errors" // Standard GraphQL errors array + resolveResponsePath = "result" // Path for resolve response ) // entityIndex represents the mapping between representation order and result order @@ -189,11 +189,11 @@ func (j *jsonBuilder) mergeValues(left *astjson.Value, right *astjson.Value) (*a // This function ensures that entities are placed in the correct positions in the final response // array based on their original representation order, which is critical for GraphQL federation. func (j *jsonBuilder) mergeEntities(left *astjson.Value, right *astjson.Value) (*astjson.Value, error) { - root := astjson.Arena{} + arena := astjson.Arena{} // Create the response structure with _entities array - entities := root.NewObject() - entities.Set(entityPath, root.NewArray()) + entities := arena.NewObject() + entities.Set(entityPath, arena.NewArray()) arr := entities.Get(entityPath) // Extract entity arrays from both responses @@ -221,7 +221,7 @@ func (j *jsonBuilder) mergeEntities(left *astjson.Value, right *astjson.Value) ( } func (j *jsonBuilder) mergeWithPath(base *astjson.Value, resolved *astjson.Value, path ast.Path) error { - resolvedValues := resolved.GetArray(resolvResponsePath) + resolvedValues := resolved.GetArray(resolveResponsePath) searchPath := path[:len(path)-1] elementName := path[len(path)-1].FieldName.String() @@ -262,7 +262,8 @@ func (j *jsonBuilder) flattenObject(value *astjson.Value, path ast.Path) ([]*ast return []*astjson.Value{value}, nil } - current := value + segment := path[0] + current := value.Get(segment.FieldName.String()) result := make([]*astjson.Value, 0) switch current.Type() { case astjson.TypeObject: diff --git a/v2/pkg/engine/datasource/grpc_datasource/mapping_test_helper.go b/v2/pkg/engine/datasource/grpc_datasource/mapping_test_helper.go index ca7ee1adb..64d673cd6 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/mapping_test_helper.go +++ b/v2/pkg/engine/datasource/grpc_datasource/mapping_test_helper.go @@ -232,6 +232,19 @@ func testMapping() *GRPCMapping { Response: "ResolveCategoryProductCountResponse", }, }, + "Subcategory": { + "itemCount": { + FieldMappingData: FieldMapData{ + TargetName: "item_count", + ArgumentMappings: FieldArgumentMap{ + "filters": "filters", + }, + }, + RPC: "ResolveSubcategoryItemCount", + Request: "ResolveSubcategoryItemCountRequest", + Response: "ResolveSubcategoryItemCountResponse", + }, + }, }, EntityRPCs: map[string][]EntityRPCConfig{ "Product": { @@ -705,6 +718,29 @@ func testMapping() *GRPCMapping { "filters": "filters", }, }, + "subcategories": { + TargetName: "subcategories", + }, + }, + "Subcategory": { + "id": { + TargetName: "id", + }, + "name": { + TargetName: "name", + }, + "description": { + TargetName: "description", + }, + "isActive": { + TargetName: "is_active", + }, + "itemCount": { + TargetName: "item_count", + ArgumentMappings: FieldArgumentMap{ + "filters": "filters", + }, + }, }, "Cat": { "id": { @@ -1061,6 +1097,23 @@ func testMapping() *GRPCMapping { TargetName: "search_term", }, }, + "SubcategoryItemFilter": { + "minPrice": { + TargetName: "min_price", + }, + "maxPrice": { + TargetName: "max_price", + }, + "inStock": { + TargetName: "in_stock", + }, + "isActive": { + TargetName: "is_active", + }, + "searchTerm": { + TargetName: "search_term", + }, + }, "UserInput": { "name": { TargetName: "name", diff --git a/v2/pkg/grpctest/mapping/mapping.go b/v2/pkg/grpctest/mapping/mapping.go index 81f5c8e43..839545834 100644 --- a/v2/pkg/grpctest/mapping/mapping.go +++ b/v2/pkg/grpctest/mapping/mapping.go @@ -9,7 +9,7 @@ import ( // DefaultGRPCMapping returns a hardcoded default mapping between GraphQL and Protobuf func DefaultGRPCMapping() *grpcdatasource.GRPCMapping { return &grpcdatasource.GRPCMapping{ - Service: "ProductService", + Service: "Products", QueryRPCs: grpcdatasource.RPCConfigMap[grpcdatasource.RPCConfig]{ "users": { RPC: "QueryUsers", @@ -224,15 +224,14 @@ func DefaultGRPCMapping() *grpcdatasource.GRPCMapping { Response: "MutationBulkUpdateBlogPostsResponse", }, }, - SubscriptionRPCs: grpcdatasource.RPCConfigMap[grpcdatasource.RPCConfig]{ - }, + SubscriptionRPCs: grpcdatasource.RPCConfigMap[grpcdatasource.RPCConfig]{}, ResolveRPCs: grpcdatasource.RPCConfigMap[grpcdatasource.ResolveRPCMapping]{ "Category": { "productCount": { FieldMappingData: grpcdatasource.FieldMapData{ TargetName: "product_count", ArgumentMappings: grpcdatasource.FieldArgumentMap{ - "filters": "filters", + "filters": "filters", }, }, RPC: "ResolveCategoryProductCount", @@ -240,6 +239,19 @@ func DefaultGRPCMapping() *grpcdatasource.GRPCMapping { Response: "ResolveCategoryProductCountResponse", }, }, + "Subcategory": { + "itemCount": { + FieldMappingData: grpcdatasource.FieldMapData{ + TargetName: "item_count", + ArgumentMappings: grpcdatasource.FieldArgumentMap{ + "filters": "filters", + }, + }, + RPC: "ResolveSubcategoryItemCount", + Request: "ResolveSubcategoryItemCountRequest", + Response: "ResolveSubcategoryItemCountResponse", + }, + }, }, EntityRPCs: map[string][]grpcdatasource.EntityRPCConfig{ "Product": { @@ -275,814 +287,851 @@ func DefaultGRPCMapping() *grpcdatasource.GRPCMapping { }, EnumValues: map[string][]grpcdatasource.EnumValueMapping{ "CategoryKind": { - {Value: "BOOK", TargetValue: "CATEGORY_KIND_BOOK"}, - {Value: "ELECTRONICS", TargetValue: "CATEGORY_KIND_ELECTRONICS"}, - {Value: "FURNITURE", TargetValue: "CATEGORY_KIND_FURNITURE"}, - {Value: "OTHER", TargetValue: "CATEGORY_KIND_OTHER"}, + {Value: "BOOK", TargetValue: "CATEGORY_KIND_BOOK"}, + {Value: "ELECTRONICS", TargetValue: "CATEGORY_KIND_ELECTRONICS"}, + {Value: "FURNITURE", TargetValue: "CATEGORY_KIND_FURNITURE"}, + {Value: "OTHER", TargetValue: "CATEGORY_KIND_OTHER"}, }, }, Fields: map[string]grpcdatasource.FieldMap{ - "Query": { - "users": { - TargetName: "users", - }, - "user": { - TargetName: "user", - ArgumentMappings: grpcdatasource.FieldArgumentMap{ - "id": "id", + "Query": { + "users": { + TargetName: "users", }, - }, - "nestedType": { - TargetName: "nested_type", - }, - "recursiveType": { - TargetName: "recursive_type", - }, - "typeFilterWithArguments": { - TargetName: "type_filter_with_arguments", - ArgumentMappings: grpcdatasource.FieldArgumentMap{ - "filterField1": "filter_field_1", - "filterField2": "filter_field_2", + "user": { + TargetName: "user", + ArgumentMappings: grpcdatasource.FieldArgumentMap{ + "id": "id", + }, }, - }, - "typeWithMultipleFilterFields": { - TargetName: "type_with_multiple_filter_fields", - ArgumentMappings: grpcdatasource.FieldArgumentMap{ - "filter": "filter", + "nestedType": { + TargetName: "nested_type", }, - }, - "complexFilterType": { - TargetName: "complex_filter_type", - ArgumentMappings: grpcdatasource.FieldArgumentMap{ - "filter": "filter", + "recursiveType": { + TargetName: "recursive_type", }, - }, - "calculateTotals": { - TargetName: "calculate_totals", - ArgumentMappings: grpcdatasource.FieldArgumentMap{ - "orders": "orders", + "typeFilterWithArguments": { + TargetName: "type_filter_with_arguments", + ArgumentMappings: grpcdatasource.FieldArgumentMap{ + "filterField1": "filter_field_1", + "filterField2": "filter_field_2", + }, }, - }, - "categories": { - TargetName: "categories", - }, - "categoriesByKind": { - TargetName: "categories_by_kind", - ArgumentMappings: grpcdatasource.FieldArgumentMap{ - "kind": "kind", + "typeWithMultipleFilterFields": { + TargetName: "type_with_multiple_filter_fields", + ArgumentMappings: grpcdatasource.FieldArgumentMap{ + "filter": "filter", + }, }, - }, - "categoriesByKinds": { - TargetName: "categories_by_kinds", - ArgumentMappings: grpcdatasource.FieldArgumentMap{ - "kinds": "kinds", + "complexFilterType": { + TargetName: "complex_filter_type", + ArgumentMappings: grpcdatasource.FieldArgumentMap{ + "filter": "filter", + }, }, - }, - "filterCategories": { - TargetName: "filter_categories", - ArgumentMappings: grpcdatasource.FieldArgumentMap{ - "filter": "filter", + "calculateTotals": { + TargetName: "calculate_totals", + ArgumentMappings: grpcdatasource.FieldArgumentMap{ + "orders": "orders", + }, }, - }, - "randomPet": { - TargetName: "random_pet", - }, - "allPets": { - TargetName: "all_pets", - }, - "search": { - TargetName: "search", - ArgumentMappings: grpcdatasource.FieldArgumentMap{ - "input": "input", + "categories": { + TargetName: "categories", }, - }, - "randomSearchResult": { - TargetName: "random_search_result", - }, - "nullableFieldsType": { - TargetName: "nullable_fields_type", - }, - "nullableFieldsTypeById": { - TargetName: "nullable_fields_type_by_id", - ArgumentMappings: grpcdatasource.FieldArgumentMap{ - "id": "id", + "categoriesByKind": { + TargetName: "categories_by_kind", + ArgumentMappings: grpcdatasource.FieldArgumentMap{ + "kind": "kind", + }, }, - }, - "nullableFieldsTypeWithFilter": { - TargetName: "nullable_fields_type_with_filter", - ArgumentMappings: grpcdatasource.FieldArgumentMap{ - "filter": "filter", + "categoriesByKinds": { + TargetName: "categories_by_kinds", + ArgumentMappings: grpcdatasource.FieldArgumentMap{ + "kinds": "kinds", + }, }, - }, - "allNullableFieldsTypes": { - TargetName: "all_nullable_fields_types", - }, - "blogPost": { - TargetName: "blog_post", - }, - "blogPostById": { - TargetName: "blog_post_by_id", - ArgumentMappings: grpcdatasource.FieldArgumentMap{ - "id": "id", + "filterCategories": { + TargetName: "filter_categories", + ArgumentMappings: grpcdatasource.FieldArgumentMap{ + "filter": "filter", + }, }, - }, - "blogPostsWithFilter": { - TargetName: "blog_posts_with_filter", - ArgumentMappings: grpcdatasource.FieldArgumentMap{ - "filter": "filter", + "randomPet": { + TargetName: "random_pet", }, - }, - "allBlogPosts": { - TargetName: "all_blog_posts", - }, - "author": { - TargetName: "author", - }, - "authorById": { - TargetName: "author_by_id", - ArgumentMappings: grpcdatasource.FieldArgumentMap{ - "id": "id", + "allPets": { + TargetName: "all_pets", }, - }, - "authorsWithFilter": { - TargetName: "authors_with_filter", - ArgumentMappings: grpcdatasource.FieldArgumentMap{ - "filter": "filter", + "search": { + TargetName: "search", + ArgumentMappings: grpcdatasource.FieldArgumentMap{ + "input": "input", + }, }, - }, - "allAuthors": { - TargetName: "all_authors", - }, - "bulkSearchAuthors": { - TargetName: "bulk_search_authors", - ArgumentMappings: grpcdatasource.FieldArgumentMap{ - "filters": "filters", + "randomSearchResult": { + TargetName: "random_search_result", }, - }, - "bulkSearchBlogPosts": { - TargetName: "bulk_search_blog_posts", - ArgumentMappings: grpcdatasource.FieldArgumentMap{ - "filters": "filters", + "nullableFieldsType": { + TargetName: "nullable_fields_type", }, - }, - }, - "Mutation": { - "createUser": { - TargetName: "create_user", - ArgumentMappings: grpcdatasource.FieldArgumentMap{ - "input": "input", + "nullableFieldsTypeById": { + TargetName: "nullable_fields_type_by_id", + ArgumentMappings: grpcdatasource.FieldArgumentMap{ + "id": "id", + }, }, - }, - "performAction": { - TargetName: "perform_action", - ArgumentMappings: grpcdatasource.FieldArgumentMap{ - "input": "input", + "nullableFieldsTypeWithFilter": { + TargetName: "nullable_fields_type_with_filter", + ArgumentMappings: grpcdatasource.FieldArgumentMap{ + "filter": "filter", + }, }, - }, - "createNullableFieldsType": { - TargetName: "create_nullable_fields_type", - ArgumentMappings: grpcdatasource.FieldArgumentMap{ - "input": "input", + "allNullableFieldsTypes": { + TargetName: "all_nullable_fields_types", }, - }, - "updateNullableFieldsType": { - TargetName: "update_nullable_fields_type", - ArgumentMappings: grpcdatasource.FieldArgumentMap{ - "id": "id", - "input": "input", + "blogPost": { + TargetName: "blog_post", }, - }, - "createBlogPost": { - TargetName: "create_blog_post", - ArgumentMappings: grpcdatasource.FieldArgumentMap{ - "input": "input", + "blogPostById": { + TargetName: "blog_post_by_id", + ArgumentMappings: grpcdatasource.FieldArgumentMap{ + "id": "id", + }, }, - }, - "updateBlogPost": { - TargetName: "update_blog_post", - ArgumentMappings: grpcdatasource.FieldArgumentMap{ - "id": "id", - "input": "input", + "blogPostsWithFilter": { + TargetName: "blog_posts_with_filter", + ArgumentMappings: grpcdatasource.FieldArgumentMap{ + "filter": "filter", + }, }, - }, - "createAuthor": { - TargetName: "create_author", - ArgumentMappings: grpcdatasource.FieldArgumentMap{ - "input": "input", + "allBlogPosts": { + TargetName: "all_blog_posts", }, - }, - "updateAuthor": { - TargetName: "update_author", - ArgumentMappings: grpcdatasource.FieldArgumentMap{ - "id": "id", - "input": "input", + "author": { + TargetName: "author", }, - }, - "bulkCreateAuthors": { - TargetName: "bulk_create_authors", - ArgumentMappings: grpcdatasource.FieldArgumentMap{ - "authors": "authors", + "authorById": { + TargetName: "author_by_id", + ArgumentMappings: grpcdatasource.FieldArgumentMap{ + "id": "id", + }, }, - }, - "bulkUpdateAuthors": { - TargetName: "bulk_update_authors", - ArgumentMappings: grpcdatasource.FieldArgumentMap{ - "authors": "authors", + "authorsWithFilter": { + TargetName: "authors_with_filter", + ArgumentMappings: grpcdatasource.FieldArgumentMap{ + "filter": "filter", + }, }, - }, - "bulkCreateBlogPosts": { - TargetName: "bulk_create_blog_posts", - ArgumentMappings: grpcdatasource.FieldArgumentMap{ - "blogPosts": "blog_posts", + "allAuthors": { + TargetName: "all_authors", }, - }, - "bulkUpdateBlogPosts": { - TargetName: "bulk_update_blog_posts", - ArgumentMappings: grpcdatasource.FieldArgumentMap{ - "blogPosts": "blog_posts", + "bulkSearchAuthors": { + TargetName: "bulk_search_authors", + ArgumentMappings: grpcdatasource.FieldArgumentMap{ + "filters": "filters", + }, }, - }, - }, - "Product": { - "id": { - TargetName: "id", - }, - "name": { - TargetName: "name", - }, - "price": { - TargetName: "price", - }, - }, - "Storage": { - "id": { - TargetName: "id", - }, - "name": { - TargetName: "name", - }, - "location": { - TargetName: "location", - }, - }, - "Warehouse": { - "id": { - TargetName: "id", - }, - "name": { - TargetName: "name", - }, - "location": { - TargetName: "location", - }, - }, - "User": { - "id": { - TargetName: "id", - }, - "name": { - TargetName: "name", - }, - }, - "NestedTypeA": { - "id": { - TargetName: "id", - }, - "name": { - TargetName: "name", - }, - "b": { - TargetName: "b", - }, - }, - "NestedTypeB": { - "id": { - TargetName: "id", - }, - "name": { - TargetName: "name", - }, - "c": { - TargetName: "c", - }, - }, - "NestedTypeC": { - "id": { - TargetName: "id", - }, - "name": { - TargetName: "name", - }, - }, - "RecursiveType": { - "id": { - TargetName: "id", - }, - "name": { - TargetName: "name", - }, - "recursiveType": { - TargetName: "recursive_type", - }, - }, - "TypeWithMultipleFilterFields": { - "id": { - TargetName: "id", - }, - "name": { - TargetName: "name", - }, - "filterField1": { - TargetName: "filter_field_1", - }, - "filterField2": { - TargetName: "filter_field_2", - }, - }, - "FilterTypeInput": { - "filterField1": { - TargetName: "filter_field_1", - }, - "filterField2": { - TargetName: "filter_field_2", - }, - }, - "TypeWithComplexFilterInput": { - "id": { - TargetName: "id", - }, - "name": { - TargetName: "name", - }, - }, - "FilterType": { - "name": { - TargetName: "name", - }, - "filterField1": { - TargetName: "filter_field_1", - }, - "filterField2": { - TargetName: "filter_field_2", - }, - "pagination": { - TargetName: "pagination", - }, - }, - "Pagination": { - "page": { - TargetName: "page", - }, - "perPage": { - TargetName: "per_page", - }, - }, - "ComplexFilterTypeInput": { - "filter": { - TargetName: "filter", - }, - }, - "OrderLineInput": { - "productId": { - TargetName: "product_id", - }, - "quantity": { - TargetName: "quantity", - }, - "modifiers": { - TargetName: "modifiers", - }, - }, - "OrderInput": { - "orderId": { - TargetName: "order_id", - }, - "customerName": { - TargetName: "customer_name", - }, - "lines": { - TargetName: "lines", - }, - }, - "Order": { - "orderId": { - TargetName: "order_id", - }, - "customerName": { - TargetName: "customer_name", - }, - "totalItems": { - TargetName: "total_items", - }, - "orderLines": { - TargetName: "order_lines", - }, - }, - "OrderLine": { - "productId": { - TargetName: "product_id", - }, - "quantity": { - TargetName: "quantity", - }, - "modifiers": { - TargetName: "modifiers", - }, - }, - "CategoryFilter": { - "category": { - TargetName: "category", - }, - "pagination": { - TargetName: "pagination", - }, - }, - "Category": { - "id": { - TargetName: "id", - }, - "name": { - TargetName: "name", - }, - "kind": { - TargetName: "kind", - }, - "productCount": { - TargetName: "product_count", - ArgumentMappings: grpcdatasource.FieldArgumentMap{ - "filters": "filters", + "bulkSearchBlogPosts": { + TargetName: "bulk_search_blog_posts", + ArgumentMappings: grpcdatasource.FieldArgumentMap{ + "filters": "filters", + }, }, }, + "Mutation": { + "createUser": { + TargetName: "create_user", + ArgumentMappings: grpcdatasource.FieldArgumentMap{ + "input": "input", + }, + }, + "performAction": { + TargetName: "perform_action", + ArgumentMappings: grpcdatasource.FieldArgumentMap{ + "input": "input", + }, + }, + "createNullableFieldsType": { + TargetName: "create_nullable_fields_type", + ArgumentMappings: grpcdatasource.FieldArgumentMap{ + "input": "input", + }, + }, + "updateNullableFieldsType": { + TargetName: "update_nullable_fields_type", + ArgumentMappings: grpcdatasource.FieldArgumentMap{ + "id": "id", + "input": "input", + }, + }, + "createBlogPost": { + TargetName: "create_blog_post", + ArgumentMappings: grpcdatasource.FieldArgumentMap{ + "input": "input", + }, + }, + "updateBlogPost": { + TargetName: "update_blog_post", + ArgumentMappings: grpcdatasource.FieldArgumentMap{ + "id": "id", + "input": "input", + }, + }, + "createAuthor": { + TargetName: "create_author", + ArgumentMappings: grpcdatasource.FieldArgumentMap{ + "input": "input", + }, + }, + "updateAuthor": { + TargetName: "update_author", + ArgumentMappings: grpcdatasource.FieldArgumentMap{ + "id": "id", + "input": "input", + }, + }, + "bulkCreateAuthors": { + TargetName: "bulk_create_authors", + ArgumentMappings: grpcdatasource.FieldArgumentMap{ + "authors": "authors", + }, + }, + "bulkUpdateAuthors": { + TargetName: "bulk_update_authors", + ArgumentMappings: grpcdatasource.FieldArgumentMap{ + "authors": "authors", + }, + }, + "bulkCreateBlogPosts": { + TargetName: "bulk_create_blog_posts", + ArgumentMappings: grpcdatasource.FieldArgumentMap{ + "blogPosts": "blog_posts", + }, + }, + "bulkUpdateBlogPosts": { + TargetName: "bulk_update_blog_posts", + ArgumentMappings: grpcdatasource.FieldArgumentMap{ + "blogPosts": "blog_posts", + }, + }, }, - "Cat": { - "id": { - TargetName: "id", - }, - "name": { - TargetName: "name", - }, - "kind": { - TargetName: "kind", - }, - "meowVolume": { - TargetName: "meow_volume", - }, - }, - "Dog": { - "id": { - TargetName: "id", - }, - "name": { - TargetName: "name", - }, - "kind": { - TargetName: "kind", - }, - "barkVolume": { - TargetName: "bark_volume", - }, - }, - "ActionSuccess": { - "message": { - TargetName: "message", - }, - "timestamp": { - TargetName: "timestamp", - }, - }, - "ActionError": { - "message": { - TargetName: "message", - }, - "code": { - TargetName: "code", - }, - }, - "SearchInput": { - "query": { - TargetName: "query", - }, - "limit": { - TargetName: "limit", - }, - }, - "ActionInput": { - "type": { - TargetName: "type", - }, - "payload": { - TargetName: "payload", - }, - }, - "NullableFieldsType": { - "id": { - TargetName: "id", - }, - "name": { - TargetName: "name", - }, - "optionalString": { - TargetName: "optional_string", - }, - "optionalInt": { - TargetName: "optional_int", - }, - "optionalFloat": { - TargetName: "optional_float", - }, - "optionalBoolean": { - TargetName: "optional_boolean", - }, - "requiredString": { - TargetName: "required_string", - }, - "requiredInt": { - TargetName: "required_int", - }, - }, - "BlogPost": { - "id": { - TargetName: "id", - }, - "title": { - TargetName: "title", - }, - "content": { - TargetName: "content", - }, - "tags": { - TargetName: "tags", - }, - "optionalTags": { - TargetName: "optional_tags", - }, - "categories": { - TargetName: "categories", - }, - "keywords": { - TargetName: "keywords", - }, - "viewCounts": { - TargetName: "view_counts", - }, - "ratings": { - TargetName: "ratings", - }, - "isPublished": { - TargetName: "is_published", - }, - "tagGroups": { - TargetName: "tag_groups", - }, - "relatedTopics": { - TargetName: "related_topics", - }, - "commentThreads": { - TargetName: "comment_threads", - }, - "suggestions": { - TargetName: "suggestions", - }, - "relatedCategories": { - TargetName: "related_categories", - }, - "contributors": { - TargetName: "contributors", - }, - "mentionedProducts": { - TargetName: "mentioned_products", - }, - "mentionedUsers": { - TargetName: "mentioned_users", - }, - "categoryGroups": { - TargetName: "category_groups", - }, - "contributorTeams": { - TargetName: "contributor_teams", - }, - }, - "Author": { - "id": { - TargetName: "id", - }, - "name": { - TargetName: "name", - }, - "email": { - TargetName: "email", - }, - "skills": { - TargetName: "skills", - }, - "languages": { - TargetName: "languages", - }, - "socialLinks": { - TargetName: "social_links", - }, - "teamsByProject": { - TargetName: "teams_by_project", - }, - "collaborations": { - TargetName: "collaborations", - }, - "writtenPosts": { - TargetName: "written_posts", - }, - "favoriteCategories": { - TargetName: "favorite_categories", - }, - "relatedAuthors": { - TargetName: "related_authors", - }, - "productReviews": { - TargetName: "product_reviews", - }, - "authorGroups": { - TargetName: "author_groups", - }, - "categoryPreferences": { - TargetName: "category_preferences", - }, - "projectTeams": { - TargetName: "project_teams", - }, - }, - "BlogPostInput": { - "title": { - TargetName: "title", - }, - "content": { - TargetName: "content", - }, - "tags": { - TargetName: "tags", - }, - "optionalTags": { - TargetName: "optional_tags", - }, - "categories": { - TargetName: "categories", - }, - "keywords": { - TargetName: "keywords", - }, - "viewCounts": { - TargetName: "view_counts", - }, - "ratings": { - TargetName: "ratings", - }, - "isPublished": { - TargetName: "is_published", - }, - "tagGroups": { - TargetName: "tag_groups", - }, - "relatedTopics": { - TargetName: "related_topics", - }, - "commentThreads": { - TargetName: "comment_threads", - }, - "suggestions": { - TargetName: "suggestions", - }, - "relatedCategories": { - TargetName: "related_categories", - }, - "contributors": { - TargetName: "contributors", - }, - "categoryGroups": { - TargetName: "category_groups", - }, - }, - "AuthorInput": { - "name": { - TargetName: "name", - }, - "email": { - TargetName: "email", + "Product": { + "id": { + TargetName: "id", + }, + "name": { + TargetName: "name", + }, + "price": { + TargetName: "price", + }, }, - "skills": { - TargetName: "skills", + "Storage": { + "id": { + TargetName: "id", + }, + "name": { + TargetName: "name", + }, + "location": { + TargetName: "location", + }, }, - "languages": { - TargetName: "languages", + "Warehouse": { + "id": { + TargetName: "id", + }, + "name": { + TargetName: "name", + }, + "location": { + TargetName: "location", + }, }, - "socialLinks": { - TargetName: "social_links", + "User": { + "id": { + TargetName: "id", + }, + "name": { + TargetName: "name", + }, }, - "teamsByProject": { - TargetName: "teams_by_project", + "NestedTypeA": { + "id": { + TargetName: "id", + }, + "name": { + TargetName: "name", + }, + "b": { + TargetName: "b", + }, }, - "collaborations": { - TargetName: "collaborations", + "NestedTypeB": { + "id": { + TargetName: "id", + }, + "name": { + TargetName: "name", + }, + "c": { + TargetName: "c", + }, }, - "favoriteCategories": { - TargetName: "favorite_categories", + "NestedTypeC": { + "id": { + TargetName: "id", + }, + "name": { + TargetName: "name", + }, }, - "authorGroups": { - TargetName: "author_groups", + "RecursiveType": { + "id": { + TargetName: "id", + }, + "name": { + TargetName: "name", + }, + "recursiveType": { + TargetName: "recursive_type", + }, }, - "projectTeams": { - TargetName: "project_teams", + "TypeWithMultipleFilterFields": { + "id": { + TargetName: "id", + }, + "name": { + TargetName: "name", + }, + "filterField1": { + TargetName: "filter_field_1", + }, + "filterField2": { + TargetName: "filter_field_2", + }, }, + "FilterTypeInput": { + "filterField1": { + TargetName: "filter_field_1", + }, + "filterField2": { + TargetName: "filter_field_2", + }, }, - "BlogPostFilter": { - "title": { - TargetName: "title", + "TypeWithComplexFilterInput": { + "id": { + TargetName: "id", + }, + "name": { + TargetName: "name", + }, }, - "hasCategories": { - TargetName: "has_categories", + "FilterType": { + "name": { + TargetName: "name", + }, + "filterField1": { + TargetName: "filter_field_1", + }, + "filterField2": { + TargetName: "filter_field_2", + }, + "pagination": { + TargetName: "pagination", + }, }, - "minTags": { - TargetName: "min_tags", + "Pagination": { + "page": { + TargetName: "page", + }, + "perPage": { + TargetName: "per_page", + }, }, + "ComplexFilterTypeInput": { + "filter": { + TargetName: "filter", + }, }, - "AuthorFilter": { - "name": { - TargetName: "name", + "OrderLineInput": { + "productId": { + TargetName: "product_id", + }, + "quantity": { + TargetName: "quantity", + }, + "modifiers": { + TargetName: "modifiers", + }, }, - "hasTeams": { - TargetName: "has_teams", + "OrderInput": { + "orderId": { + TargetName: "order_id", + }, + "customerName": { + TargetName: "customer_name", + }, + "lines": { + TargetName: "lines", + }, }, - "skillCount": { - TargetName: "skill_count", + "Order": { + "orderId": { + TargetName: "order_id", + }, + "customerName": { + TargetName: "customer_name", + }, + "totalItems": { + TargetName: "total_items", + }, + "orderLines": { + TargetName: "order_lines", + }, }, + "OrderLine": { + "productId": { + TargetName: "product_id", + }, + "quantity": { + TargetName: "quantity", + }, + "modifiers": { + TargetName: "modifiers", + }, }, - "NullableFieldsInput": { - "name": { - TargetName: "name", + "CategoryFilter": { + "category": { + TargetName: "category", + }, + "pagination": { + TargetName: "pagination", + }, }, - "optionalString": { - TargetName: "optional_string", + "Category": { + "id": { + TargetName: "id", + }, + "name": { + TargetName: "name", + }, + "kind": { + TargetName: "kind", + }, + "productCount": { + TargetName: "product_count", + ArgumentMappings: grpcdatasource.FieldArgumentMap{ + "filters": "filters", + }, + }, + "subcategories": { + TargetName: "subcategories", + }, }, - "optionalInt": { - TargetName: "optional_int", + "Subcategory": { + "id": { + TargetName: "id", + }, + "name": { + TargetName: "name", + }, + "description": { + TargetName: "description", + }, + "isActive": { + TargetName: "is_active", + }, + "itemCount": { + TargetName: "item_count", + ArgumentMappings: grpcdatasource.FieldArgumentMap{ + "filters": "filters", + }, + }, }, - "optionalFloat": { - TargetName: "optional_float", + "Cat": { + "id": { + TargetName: "id", + }, + "name": { + TargetName: "name", + }, + "kind": { + TargetName: "kind", + }, + "meowVolume": { + TargetName: "meow_volume", + }, }, - "optionalBoolean": { - TargetName: "optional_boolean", + "Dog": { + "id": { + TargetName: "id", + }, + "name": { + TargetName: "name", + }, + "kind": { + TargetName: "kind", + }, + "barkVolume": { + TargetName: "bark_volume", + }, }, - "requiredString": { - TargetName: "required_string", + "ActionSuccess": { + "message": { + TargetName: "message", + }, + "timestamp": { + TargetName: "timestamp", + }, }, - "requiredInt": { - TargetName: "required_int", + "ActionError": { + "message": { + TargetName: "message", + }, + "code": { + TargetName: "code", + }, }, + "SearchInput": { + "query": { + TargetName: "query", + }, + "limit": { + TargetName: "limit", + }, }, - "NullableFieldsFilter": { - "name": { - TargetName: "name", + "ActionInput": { + "type": { + TargetName: "type", + }, + "payload": { + TargetName: "payload", + }, }, - "optionalString": { - TargetName: "optional_string", + "NullableFieldsType": { + "id": { + TargetName: "id", + }, + "name": { + TargetName: "name", + }, + "optionalString": { + TargetName: "optional_string", + }, + "optionalInt": { + TargetName: "optional_int", + }, + "optionalFloat": { + TargetName: "optional_float", + }, + "optionalBoolean": { + TargetName: "optional_boolean", + }, + "requiredString": { + TargetName: "required_string", + }, + "requiredInt": { + TargetName: "required_int", + }, }, - "includeNulls": { - TargetName: "include_nulls", + "BlogPost": { + "id": { + TargetName: "id", + }, + "title": { + TargetName: "title", + }, + "content": { + TargetName: "content", + }, + "tags": { + TargetName: "tags", + }, + "optionalTags": { + TargetName: "optional_tags", + }, + "categories": { + TargetName: "categories", + }, + "keywords": { + TargetName: "keywords", + }, + "viewCounts": { + TargetName: "view_counts", + }, + "ratings": { + TargetName: "ratings", + }, + "isPublished": { + TargetName: "is_published", + }, + "tagGroups": { + TargetName: "tag_groups", + }, + "relatedTopics": { + TargetName: "related_topics", + }, + "commentThreads": { + TargetName: "comment_threads", + }, + "suggestions": { + TargetName: "suggestions", + }, + "relatedCategories": { + TargetName: "related_categories", + }, + "contributors": { + TargetName: "contributors", + }, + "mentionedProducts": { + TargetName: "mentioned_products", + }, + "mentionedUsers": { + TargetName: "mentioned_users", + }, + "categoryGroups": { + TargetName: "category_groups", + }, + "contributorTeams": { + TargetName: "contributor_teams", + }, }, + "Author": { + "id": { + TargetName: "id", + }, + "name": { + TargetName: "name", + }, + "email": { + TargetName: "email", + }, + "skills": { + TargetName: "skills", + }, + "languages": { + TargetName: "languages", + }, + "socialLinks": { + TargetName: "social_links", + }, + "teamsByProject": { + TargetName: "teams_by_project", + }, + "collaborations": { + TargetName: "collaborations", + }, + "writtenPosts": { + TargetName: "written_posts", + }, + "favoriteCategories": { + TargetName: "favorite_categories", + }, + "relatedAuthors": { + TargetName: "related_authors", + }, + "productReviews": { + TargetName: "product_reviews", + }, + "authorGroups": { + TargetName: "author_groups", + }, + "categoryPreferences": { + TargetName: "category_preferences", + }, + "projectTeams": { + TargetName: "project_teams", + }, }, - "CategoryInput": { - "name": { - TargetName: "name", + "BlogPostInput": { + "title": { + TargetName: "title", + }, + "content": { + TargetName: "content", + }, + "tags": { + TargetName: "tags", + }, + "optionalTags": { + TargetName: "optional_tags", + }, + "categories": { + TargetName: "categories", + }, + "keywords": { + TargetName: "keywords", + }, + "viewCounts": { + TargetName: "view_counts", + }, + "ratings": { + TargetName: "ratings", + }, + "isPublished": { + TargetName: "is_published", + }, + "tagGroups": { + TargetName: "tag_groups", + }, + "relatedTopics": { + TargetName: "related_topics", + }, + "commentThreads": { + TargetName: "comment_threads", + }, + "suggestions": { + TargetName: "suggestions", + }, + "relatedCategories": { + TargetName: "related_categories", + }, + "contributors": { + TargetName: "contributors", + }, + "categoryGroups": { + TargetName: "category_groups", + }, }, - "kind": { - TargetName: "kind", + "AuthorInput": { + "name": { + TargetName: "name", + }, + "email": { + TargetName: "email", + }, + "skills": { + TargetName: "skills", + }, + "languages": { + TargetName: "languages", + }, + "socialLinks": { + TargetName: "social_links", + }, + "teamsByProject": { + TargetName: "teams_by_project", + }, + "collaborations": { + TargetName: "collaborations", + }, + "favoriteCategories": { + TargetName: "favorite_categories", + }, + "authorGroups": { + TargetName: "author_groups", + }, + "projectTeams": { + TargetName: "project_teams", + }, }, + "BlogPostFilter": { + "title": { + TargetName: "title", + }, + "hasCategories": { + TargetName: "has_categories", + }, + "minTags": { + TargetName: "min_tags", + }, }, - "ProductCountFilter": { - "minPrice": { - TargetName: "min_price", + "AuthorFilter": { + "name": { + TargetName: "name", + }, + "hasTeams": { + TargetName: "has_teams", + }, + "skillCount": { + TargetName: "skill_count", + }, }, - "maxPrice": { - TargetName: "max_price", + "NullableFieldsInput": { + "name": { + TargetName: "name", + }, + "optionalString": { + TargetName: "optional_string", + }, + "optionalInt": { + TargetName: "optional_int", + }, + "optionalFloat": { + TargetName: "optional_float", + }, + "optionalBoolean": { + TargetName: "optional_boolean", + }, + "requiredString": { + TargetName: "required_string", + }, + "requiredInt": { + TargetName: "required_int", + }, }, - "inStock": { - TargetName: "in_stock", + "NullableFieldsFilter": { + "name": { + TargetName: "name", + }, + "optionalString": { + TargetName: "optional_string", + }, + "includeNulls": { + TargetName: "include_nulls", + }, }, - "searchTerm": { - TargetName: "search_term", + "CategoryInput": { + "name": { + TargetName: "name", + }, + "kind": { + TargetName: "kind", + }, }, + "ProductCountFilter": { + "minPrice": { + TargetName: "min_price", + }, + "maxPrice": { + TargetName: "max_price", + }, + "inStock": { + TargetName: "in_stock", + }, + "searchTerm": { + TargetName: "search_term", + }, }, - "UserInput": { - "name": { - TargetName: "name", + "SubcategoryItemFilter": { + "minPrice": { + TargetName: "min_price", + }, + "maxPrice": { + TargetName: "max_price", + }, + "inStock": { + TargetName: "in_stock", + }, + "isActive": { + TargetName: "is_active", + }, + "searchTerm": { + TargetName: "search_term", + }, }, + "UserInput": { + "name": { + TargetName: "name", + }, }, }, } } - // MustDefaultGRPCMapping returns the default GRPC mapping func MustDefaultGRPCMapping(t *testing.T) *grpcdatasource.GRPCMapping { mapping := DefaultGRPCMapping() return mapping } - - diff --git a/v2/pkg/grpctest/mockservice.go b/v2/pkg/grpctest/mockservice.go index ca4e1089e..4132fda37 100644 --- a/v2/pkg/grpctest/mockservice.go +++ b/v2/pkg/grpctest/mockservice.go @@ -20,11 +20,27 @@ type MockService struct { productv1.UnimplementedProductServiceServer } +// ResolveSubcategoryItemCount implements productv1.ProductServiceServer. +func (s *MockService) ResolveSubcategoryItemCount(_ context.Context, req *productv1.ResolveSubcategoryItemCountRequest) (*productv1.ResolveSubcategoryItemCountResponse, error) { + results := make([]*productv1.ResolveSubcategoryItemCountResult, 0, len(req.GetContext())) + for i := range req.GetContext() { + results = append(results, &productv1.ResolveSubcategoryItemCountResult{ + ItemCount: int32(i * 10), // Different multiplier to distinguish from productCount + }) + } + + resp := &productv1.ResolveSubcategoryItemCountResponse{ + Result: results, + } + + return resp, nil +} + // ResolveCategoryProductCount implements productv1.ProductServiceServer. func (s *MockService) ResolveCategoryProductCount(_ context.Context, req *productv1.ResolveCategoryProductCountRequest) (*productv1.ResolveCategoryProductCountResponse, error) { - results := make([]*productv1.ResolveCategoryProductCountResponseResult, 0, len(req.GetContext())) + results := make([]*productv1.ResolveCategoryProductCountResult, 0, len(req.GetContext())) for i := range req.GetContext() { - results = append(results, &productv1.ResolveCategoryProductCountResponseResult{ + results = append(results, &productv1.ResolveCategoryProductCountResult{ ProductCount: int32(i), }) } @@ -69,6 +85,33 @@ func (s *MockService) LookupWarehouseById(ctx context.Context, in *productv1.Loo }, nil } +// Helper function to create subcategories for a category +func createSubcategories(categoryId string, kind productv1.CategoryKind, count int) *productv1.ListOfSubcategory { + if count <= 0 { + return &productv1.ListOfSubcategory{ + List: &productv1.ListOfSubcategory_List{ + Items: []*productv1.Subcategory{}, + }, + } + } + + subcategories := make([]*productv1.Subcategory, 0, count) + for j := 1; j <= count; j++ { + subcategories = append(subcategories, &productv1.Subcategory{ + Id: fmt.Sprintf("%s-subcategory-%d", categoryId, j), + Name: fmt.Sprintf("%s Subcategory %d", kind.String(), j), + Description: &wrapperspb.StringValue{Value: fmt.Sprintf("Subcategory %d for %s", j, categoryId)}, + IsActive: true, + }) + } + + return &productv1.ListOfSubcategory{ + List: &productv1.ListOfSubcategory_List{ + Items: subcategories, + }, + } +} + // Helper functions to convert input types to output types func convertCategoryInputsToCategories(inputs []*productv1.CategoryInput) []*productv1.Category { if inputs == nil { @@ -77,9 +120,10 @@ func convertCategoryInputsToCategories(inputs []*productv1.CategoryInput) []*pro results := make([]*productv1.Category, len(inputs)) for i, input := range inputs { results[i] = &productv1.Category{ - Id: fmt.Sprintf("cat-input-%d", i), - Name: input.GetName(), - Kind: input.GetKind(), + Id: fmt.Sprintf("cat-input-%d", i), + Name: input.GetName(), + Kind: input.GetKind(), + Subcategories: createSubcategories(fmt.Sprintf("cat-input-%d", i), input.GetKind(), i+1), } } return results @@ -92,9 +136,10 @@ func convertCategoryInputListToCategories(inputs *productv1.ListOfCategoryInput) results := make([]*productv1.Category, len(inputs.List.Items)) for i, input := range inputs.List.Items { results[i] = &productv1.Category{ - Id: fmt.Sprintf("cat-list-input-%d", i), - Name: input.GetName(), - Kind: input.GetKind(), + Id: fmt.Sprintf("cat-list-input-%d", i), + Name: input.GetName(), + Kind: input.GetKind(), + Subcategories: createSubcategories(fmt.Sprintf("cat-list-input-%d", i), input.GetKind(), i+1), } } return results @@ -160,9 +205,10 @@ func convertNestedCategoryInputsToCategories(nestedInputs *productv1.ListOfListO categories := make([]*productv1.Category, len(categoryList.List.Items)) for j, categoryInput := range categoryList.List.Items { categories[j] = &productv1.Category{ - Id: fmt.Sprintf("nested-cat-%d-%d", i, j), - Name: categoryInput.GetName(), - Kind: categoryInput.GetKind(), + Id: fmt.Sprintf("nested-cat-%d-%d", i, j), + Name: categoryInput.GetName(), + Kind: categoryInput.GetKind(), + Subcategories: createSubcategories(fmt.Sprintf("nested-cat-%d-%d", i, j), categoryInput.GetKind(), j+1), } } results[i] = &productv1.ListOfCategory{ @@ -535,12 +581,14 @@ func (s *MockService) QueryRandomSearchResult(ctx context.Context, in *productv1 } default: // Return a Category + kind := productv1.CategoryKind_CATEGORY_KIND_ELECTRONICS result = &productv1.SearchResult{ Value: &productv1.SearchResult_Category{ Category: &productv1.Category{ - Id: "category-random-1", - Name: "Random Category", - Kind: productv1.CategoryKind_CATEGORY_KIND_ELECTRONICS, + Id: "category-random-1", + Name: "Random Category", + Kind: kind, + Subcategories: createSubcategories("category-random-1", kind, 2), }, }, } @@ -595,12 +643,14 @@ func (s *MockService) QuerySearch(ctx context.Context, in *productv1.QuerySearch productv1.CategoryKind_CATEGORY_KIND_ELECTRONICS, productv1.CategoryKind_CATEGORY_KIND_FURNITURE, } + kind := kinds[i%int32(len(kinds))] results = append(results, &productv1.SearchResult{ Value: &productv1.SearchResult_Category{ Category: &productv1.Category{ - Id: fmt.Sprintf("category-search-%d", i+1), - Name: fmt.Sprintf("Category matching '%s' #%d", query, i+1), - Kind: kinds[i%int32(len(kinds))], + Id: fmt.Sprintf("category-search-%d", i+1), + Name: fmt.Sprintf("Category matching '%s' #%d", query, i+1), + Kind: kind, + Subcategories: createSubcategories(fmt.Sprintf("category-search-%d", i+1), kind, int(i%3)+1), }, }, }) @@ -871,9 +921,10 @@ func (s *MockService) QueryCategories(ctx context.Context, in *productv1.QueryCa for i, kind := range categoryKinds { categories = append(categories, &productv1.Category{ - Id: fmt.Sprintf("category-%d", i+1), - Name: fmt.Sprintf("%s Category", kind.String()), - Kind: kind, + Id: fmt.Sprintf("category-%d", i+1), + Name: fmt.Sprintf("%s Category", kind.String()), + Kind: kind, + Subcategories: createSubcategories(fmt.Sprintf("category-%d", i+1), kind, i+1), }) } @@ -891,10 +942,26 @@ func (s *MockService) QueryCategoriesByKind(ctx context.Context, in *productv1.Q // Create 3 categories of the requested kind for i := 1; i <= 3; i++ { + + subcategoties := make([]*productv1.Subcategory, 0, i) + for j := 1; j <= i; j++ { + subcategoties = append(subcategoties, &productv1.Subcategory{ + Id: fmt.Sprintf("%s-subcategory-%d", kind.String(), j), + Name: fmt.Sprintf("%s Subcategory %d", kind.String(), j), + Description: &wrapperspb.StringValue{Value: fmt.Sprintf("%s Subcategory %d", kind.String(), j)}, + IsActive: true, + }) + } + categories = append(categories, &productv1.Category{ Id: fmt.Sprintf("%s-category-%d", kind.String(), i), Name: fmt.Sprintf("%s Category %d", kind.String(), i), Kind: kind, + Subcategories: &productv1.ListOfSubcategory{ + List: &productv1.ListOfSubcategory_List{ + Items: subcategoties, + }, + }, }) } @@ -910,9 +977,10 @@ func (s *MockService) QueryCategoriesByKinds(ctx context.Context, in *productv1. for i, kind := range kinds { categories = append(categories, &productv1.Category{ - Id: fmt.Sprintf("%s-category-%d", kind.String(), i), - Name: fmt.Sprintf("%s Category %d", kind.String(), i), - Kind: kind, + Id: fmt.Sprintf("%s-category-%d", kind.String(), i), + Name: fmt.Sprintf("%s Category %d", kind.String(), i), + Kind: kind, + Subcategories: createSubcategories(fmt.Sprintf("%s-category-%d", kind.String(), i), kind, i+1), }) } @@ -939,9 +1007,10 @@ func (s *MockService) QueryFilterCategories(ctx context.Context, in *productv1.Q // Create categories that match the filter for i := 1; i <= 5; i++ { categories = append(categories, &productv1.Category{ - Id: fmt.Sprintf("filtered-%s-category-%d", kind.String(), i), - Name: fmt.Sprintf("Filtered %s Category %d", kind.String(), i), - Kind: kind, + Id: fmt.Sprintf("filtered-%s-category-%d", kind.String(), i), + Name: fmt.Sprintf("Filtered %s Category %d", kind.String(), i), + Kind: kind, + Subcategories: createSubcategories(fmt.Sprintf("filtered-%s-category-%d", kind.String(), i), kind, i), }) } diff --git a/v2/pkg/grpctest/product.proto b/v2/pkg/grpctest/product.proto index 43ee9aab4..542fe4286 100644 --- a/v2/pkg/grpctest/product.proto +++ b/v2/pkg/grpctest/product.proto @@ -56,6 +56,7 @@ service ProductService { rpc QueryUser(QueryUserRequest) returns (QueryUserResponse) {} rpc QueryUsers(QueryUsersRequest) returns (QueryUsersResponse) {} rpc ResolveCategoryProductCount(ResolveCategoryProductCountRequest) returns (ResolveCategoryProductCountResponse) {} + rpc ResolveSubcategoryItemCount(ResolveSubcategoryItemCountRequest) returns (ResolveSubcategoryItemCountResponse) {} } // Wrapper message for a list of AuthorFilter. @@ -177,6 +178,13 @@ message ListOfString { } List list = 1; } +// Wrapper message for a list of Subcategory. +message ListOfSubcategory { + message List { + repeated Subcategory items = 1; + } + List list = 1; +} // Wrapper message for a list of User. message ListOfUser { message List { @@ -644,6 +652,29 @@ message ResolveCategoryProductCountResponse { repeated ResolveCategoryProductCountResult result = 1; } +message ResolveSubcategoryItemCountArgs { + SubcategoryItemFilter filters = 1; +} + +message ResolveSubcategoryItemCountContext { + string id = 1; +} + +message ResolveSubcategoryItemCountRequest { + // context provides the resolver context for the field itemCount of type Subcategory. + repeated ResolveSubcategoryItemCountContext context = 1; + // field_args provides the arguments for the resolver field itemCount of type Subcategory. + ResolveSubcategoryItemCountArgs field_args = 2; +} + +message ResolveSubcategoryItemCountResult { + int32 item_count = 1; +} + +message ResolveSubcategoryItemCountResponse { + repeated ResolveSubcategoryItemCountResult result = 1; +} + message Product { string id = 1; string name = 2; @@ -717,6 +748,7 @@ message Category { string id = 1; string name = 2; CategoryKind kind = 3; + ListOfSubcategory subcategories = 5; } message CategoryFilter { @@ -907,6 +939,13 @@ message OrderLine { ListOfString modifiers = 3; } +message Subcategory { + string id = 1; + string name = 2; + google.protobuf.StringValue description = 3; + bool is_active = 4; +} + enum CategoryKind { CATEGORY_KIND_UNSPECIFIED = 0; CATEGORY_KIND_BOOK = 1; @@ -949,4 +988,12 @@ message ProductCountFilter { google.protobuf.DoubleValue max_price = 2; google.protobuf.BoolValue in_stock = 3; google.protobuf.StringValue search_term = 4; +} + +message SubcategoryItemFilter { + google.protobuf.DoubleValue min_price = 1; + google.protobuf.DoubleValue max_price = 2; + google.protobuf.BoolValue in_stock = 3; + google.protobuf.BoolValue is_active = 4; + google.protobuf.StringValue search_term = 5; } \ No newline at end of file diff --git a/v2/pkg/grpctest/productv1/product.pb.go b/v2/pkg/grpctest/productv1/product.pb.go index c8ff88e62..b31f73663 100644 --- a/v2/pkg/grpctest/productv1/product.pb.go +++ b/v2/pkg/grpctest/productv1/product.pb.go @@ -842,6 +842,51 @@ func (x *ListOfString) GetList() *ListOfString_List { return nil } +// Wrapper message for a list of Subcategory. +type ListOfSubcategory struct { + state protoimpl.MessageState `protogen:"open.v1"` + List *ListOfSubcategory_List `protobuf:"bytes,1,opt,name=list,proto3" json:"list,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ListOfSubcategory) Reset() { + *x = ListOfSubcategory{} + mi := &file_product_proto_msgTypes[17] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ListOfSubcategory) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ListOfSubcategory) ProtoMessage() {} + +func (x *ListOfSubcategory) ProtoReflect() protoreflect.Message { + mi := &file_product_proto_msgTypes[17] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ListOfSubcategory.ProtoReflect.Descriptor instead. +func (*ListOfSubcategory) Descriptor() ([]byte, []int) { + return file_product_proto_rawDescGZIP(), []int{17} +} + +func (x *ListOfSubcategory) GetList() *ListOfSubcategory_List { + if x != nil { + return x.List + } + return nil +} + // Wrapper message for a list of User. type ListOfUser struct { state protoimpl.MessageState `protogen:"open.v1"` @@ -852,7 +897,7 @@ type ListOfUser struct { func (x *ListOfUser) Reset() { *x = ListOfUser{} - mi := &file_product_proto_msgTypes[17] + mi := &file_product_proto_msgTypes[18] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -864,7 +909,7 @@ func (x *ListOfUser) String() string { func (*ListOfUser) ProtoMessage() {} func (x *ListOfUser) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[17] + mi := &file_product_proto_msgTypes[18] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -877,7 +922,7 @@ func (x *ListOfUser) ProtoReflect() protoreflect.Message { // Deprecated: Use ListOfUser.ProtoReflect.Descriptor instead. func (*ListOfUser) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{17} + return file_product_proto_rawDescGZIP(), []int{18} } func (x *ListOfUser) GetList() *ListOfUser_List { @@ -897,7 +942,7 @@ type ListOfUserInput struct { func (x *ListOfUserInput) Reset() { *x = ListOfUserInput{} - mi := &file_product_proto_msgTypes[18] + mi := &file_product_proto_msgTypes[19] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -909,7 +954,7 @@ func (x *ListOfUserInput) String() string { func (*ListOfUserInput) ProtoMessage() {} func (x *ListOfUserInput) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[18] + mi := &file_product_proto_msgTypes[19] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -922,7 +967,7 @@ func (x *ListOfUserInput) ProtoReflect() protoreflect.Message { // Deprecated: Use ListOfUserInput.ProtoReflect.Descriptor instead. func (*ListOfUserInput) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{18} + return file_product_proto_rawDescGZIP(), []int{19} } func (x *ListOfUserInput) GetList() *ListOfUserInput_List { @@ -943,7 +988,7 @@ type LookupProductByIdRequestKey struct { func (x *LookupProductByIdRequestKey) Reset() { *x = LookupProductByIdRequestKey{} - mi := &file_product_proto_msgTypes[19] + mi := &file_product_proto_msgTypes[20] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -955,7 +1000,7 @@ func (x *LookupProductByIdRequestKey) String() string { func (*LookupProductByIdRequestKey) ProtoMessage() {} func (x *LookupProductByIdRequestKey) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[19] + mi := &file_product_proto_msgTypes[20] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -968,7 +1013,7 @@ func (x *LookupProductByIdRequestKey) ProtoReflect() protoreflect.Message { // Deprecated: Use LookupProductByIdRequestKey.ProtoReflect.Descriptor instead. func (*LookupProductByIdRequestKey) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{19} + return file_product_proto_rawDescGZIP(), []int{20} } func (x *LookupProductByIdRequestKey) GetId() string { @@ -990,7 +1035,7 @@ type LookupProductByIdRequest struct { func (x *LookupProductByIdRequest) Reset() { *x = LookupProductByIdRequest{} - mi := &file_product_proto_msgTypes[20] + mi := &file_product_proto_msgTypes[21] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1002,7 +1047,7 @@ func (x *LookupProductByIdRequest) String() string { func (*LookupProductByIdRequest) ProtoMessage() {} func (x *LookupProductByIdRequest) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[20] + mi := &file_product_proto_msgTypes[21] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1015,7 +1060,7 @@ func (x *LookupProductByIdRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use LookupProductByIdRequest.ProtoReflect.Descriptor instead. func (*LookupProductByIdRequest) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{20} + return file_product_proto_rawDescGZIP(), []int{21} } func (x *LookupProductByIdRequest) GetKeys() []*LookupProductByIdRequestKey { @@ -1048,7 +1093,7 @@ type LookupProductByIdResponse struct { func (x *LookupProductByIdResponse) Reset() { *x = LookupProductByIdResponse{} - mi := &file_product_proto_msgTypes[21] + mi := &file_product_proto_msgTypes[22] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1060,7 +1105,7 @@ func (x *LookupProductByIdResponse) String() string { func (*LookupProductByIdResponse) ProtoMessage() {} func (x *LookupProductByIdResponse) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[21] + mi := &file_product_proto_msgTypes[22] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1073,7 +1118,7 @@ func (x *LookupProductByIdResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use LookupProductByIdResponse.ProtoReflect.Descriptor instead. func (*LookupProductByIdResponse) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{21} + return file_product_proto_rawDescGZIP(), []int{22} } func (x *LookupProductByIdResponse) GetResult() []*Product { @@ -1094,7 +1139,7 @@ type LookupStorageByIdRequestKey struct { func (x *LookupStorageByIdRequestKey) Reset() { *x = LookupStorageByIdRequestKey{} - mi := &file_product_proto_msgTypes[22] + mi := &file_product_proto_msgTypes[23] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1106,7 +1151,7 @@ func (x *LookupStorageByIdRequestKey) String() string { func (*LookupStorageByIdRequestKey) ProtoMessage() {} func (x *LookupStorageByIdRequestKey) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[22] + mi := &file_product_proto_msgTypes[23] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1119,7 +1164,7 @@ func (x *LookupStorageByIdRequestKey) ProtoReflect() protoreflect.Message { // Deprecated: Use LookupStorageByIdRequestKey.ProtoReflect.Descriptor instead. func (*LookupStorageByIdRequestKey) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{22} + return file_product_proto_rawDescGZIP(), []int{23} } func (x *LookupStorageByIdRequestKey) GetId() string { @@ -1141,7 +1186,7 @@ type LookupStorageByIdRequest struct { func (x *LookupStorageByIdRequest) Reset() { *x = LookupStorageByIdRequest{} - mi := &file_product_proto_msgTypes[23] + mi := &file_product_proto_msgTypes[24] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1153,7 +1198,7 @@ func (x *LookupStorageByIdRequest) String() string { func (*LookupStorageByIdRequest) ProtoMessage() {} func (x *LookupStorageByIdRequest) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[23] + mi := &file_product_proto_msgTypes[24] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1166,7 +1211,7 @@ func (x *LookupStorageByIdRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use LookupStorageByIdRequest.ProtoReflect.Descriptor instead. func (*LookupStorageByIdRequest) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{23} + return file_product_proto_rawDescGZIP(), []int{24} } func (x *LookupStorageByIdRequest) GetKeys() []*LookupStorageByIdRequestKey { @@ -1199,7 +1244,7 @@ type LookupStorageByIdResponse struct { func (x *LookupStorageByIdResponse) Reset() { *x = LookupStorageByIdResponse{} - mi := &file_product_proto_msgTypes[24] + mi := &file_product_proto_msgTypes[25] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1211,7 +1256,7 @@ func (x *LookupStorageByIdResponse) String() string { func (*LookupStorageByIdResponse) ProtoMessage() {} func (x *LookupStorageByIdResponse) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[24] + mi := &file_product_proto_msgTypes[25] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1224,7 +1269,7 @@ func (x *LookupStorageByIdResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use LookupStorageByIdResponse.ProtoReflect.Descriptor instead. func (*LookupStorageByIdResponse) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{24} + return file_product_proto_rawDescGZIP(), []int{25} } func (x *LookupStorageByIdResponse) GetResult() []*Storage { @@ -1245,7 +1290,7 @@ type LookupWarehouseByIdRequestKey struct { func (x *LookupWarehouseByIdRequestKey) Reset() { *x = LookupWarehouseByIdRequestKey{} - mi := &file_product_proto_msgTypes[25] + mi := &file_product_proto_msgTypes[26] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1257,7 +1302,7 @@ func (x *LookupWarehouseByIdRequestKey) String() string { func (*LookupWarehouseByIdRequestKey) ProtoMessage() {} func (x *LookupWarehouseByIdRequestKey) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[25] + mi := &file_product_proto_msgTypes[26] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1270,7 +1315,7 @@ func (x *LookupWarehouseByIdRequestKey) ProtoReflect() protoreflect.Message { // Deprecated: Use LookupWarehouseByIdRequestKey.ProtoReflect.Descriptor instead. func (*LookupWarehouseByIdRequestKey) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{25} + return file_product_proto_rawDescGZIP(), []int{26} } func (x *LookupWarehouseByIdRequestKey) GetId() string { @@ -1292,7 +1337,7 @@ type LookupWarehouseByIdRequest struct { func (x *LookupWarehouseByIdRequest) Reset() { *x = LookupWarehouseByIdRequest{} - mi := &file_product_proto_msgTypes[26] + mi := &file_product_proto_msgTypes[27] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1304,7 +1349,7 @@ func (x *LookupWarehouseByIdRequest) String() string { func (*LookupWarehouseByIdRequest) ProtoMessage() {} func (x *LookupWarehouseByIdRequest) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[26] + mi := &file_product_proto_msgTypes[27] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1317,7 +1362,7 @@ func (x *LookupWarehouseByIdRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use LookupWarehouseByIdRequest.ProtoReflect.Descriptor instead. func (*LookupWarehouseByIdRequest) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{26} + return file_product_proto_rawDescGZIP(), []int{27} } func (x *LookupWarehouseByIdRequest) GetKeys() []*LookupWarehouseByIdRequestKey { @@ -1350,7 +1395,7 @@ type LookupWarehouseByIdResponse struct { func (x *LookupWarehouseByIdResponse) Reset() { *x = LookupWarehouseByIdResponse{} - mi := &file_product_proto_msgTypes[27] + mi := &file_product_proto_msgTypes[28] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1362,7 +1407,7 @@ func (x *LookupWarehouseByIdResponse) String() string { func (*LookupWarehouseByIdResponse) ProtoMessage() {} func (x *LookupWarehouseByIdResponse) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[27] + mi := &file_product_proto_msgTypes[28] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1375,7 +1420,7 @@ func (x *LookupWarehouseByIdResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use LookupWarehouseByIdResponse.ProtoReflect.Descriptor instead. func (*LookupWarehouseByIdResponse) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{27} + return file_product_proto_rawDescGZIP(), []int{28} } func (x *LookupWarehouseByIdResponse) GetResult() []*Warehouse { @@ -1394,7 +1439,7 @@ type QueryUsersRequest struct { func (x *QueryUsersRequest) Reset() { *x = QueryUsersRequest{} - mi := &file_product_proto_msgTypes[28] + mi := &file_product_proto_msgTypes[29] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1406,7 +1451,7 @@ func (x *QueryUsersRequest) String() string { func (*QueryUsersRequest) ProtoMessage() {} func (x *QueryUsersRequest) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[28] + mi := &file_product_proto_msgTypes[29] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1419,7 +1464,7 @@ func (x *QueryUsersRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use QueryUsersRequest.ProtoReflect.Descriptor instead. func (*QueryUsersRequest) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{28} + return file_product_proto_rawDescGZIP(), []int{29} } // Response message for users operation. @@ -1432,7 +1477,7 @@ type QueryUsersResponse struct { func (x *QueryUsersResponse) Reset() { *x = QueryUsersResponse{} - mi := &file_product_proto_msgTypes[29] + mi := &file_product_proto_msgTypes[30] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1444,7 +1489,7 @@ func (x *QueryUsersResponse) String() string { func (*QueryUsersResponse) ProtoMessage() {} func (x *QueryUsersResponse) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[29] + mi := &file_product_proto_msgTypes[30] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1457,7 +1502,7 @@ func (x *QueryUsersResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use QueryUsersResponse.ProtoReflect.Descriptor instead. func (*QueryUsersResponse) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{29} + return file_product_proto_rawDescGZIP(), []int{30} } func (x *QueryUsersResponse) GetUsers() []*User { @@ -1477,7 +1522,7 @@ type QueryUserRequest struct { func (x *QueryUserRequest) Reset() { *x = QueryUserRequest{} - mi := &file_product_proto_msgTypes[30] + mi := &file_product_proto_msgTypes[31] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1489,7 +1534,7 @@ func (x *QueryUserRequest) String() string { func (*QueryUserRequest) ProtoMessage() {} func (x *QueryUserRequest) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[30] + mi := &file_product_proto_msgTypes[31] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1502,7 +1547,7 @@ func (x *QueryUserRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use QueryUserRequest.ProtoReflect.Descriptor instead. func (*QueryUserRequest) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{30} + return file_product_proto_rawDescGZIP(), []int{31} } func (x *QueryUserRequest) GetId() string { @@ -1522,7 +1567,7 @@ type QueryUserResponse struct { func (x *QueryUserResponse) Reset() { *x = QueryUserResponse{} - mi := &file_product_proto_msgTypes[31] + mi := &file_product_proto_msgTypes[32] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1534,7 +1579,7 @@ func (x *QueryUserResponse) String() string { func (*QueryUserResponse) ProtoMessage() {} func (x *QueryUserResponse) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[31] + mi := &file_product_proto_msgTypes[32] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1547,7 +1592,7 @@ func (x *QueryUserResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use QueryUserResponse.ProtoReflect.Descriptor instead. func (*QueryUserResponse) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{31} + return file_product_proto_rawDescGZIP(), []int{32} } func (x *QueryUserResponse) GetUser() *User { @@ -1566,7 +1611,7 @@ type QueryNestedTypeRequest struct { func (x *QueryNestedTypeRequest) Reset() { *x = QueryNestedTypeRequest{} - mi := &file_product_proto_msgTypes[32] + mi := &file_product_proto_msgTypes[33] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1578,7 +1623,7 @@ func (x *QueryNestedTypeRequest) String() string { func (*QueryNestedTypeRequest) ProtoMessage() {} func (x *QueryNestedTypeRequest) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[32] + mi := &file_product_proto_msgTypes[33] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1591,7 +1636,7 @@ func (x *QueryNestedTypeRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use QueryNestedTypeRequest.ProtoReflect.Descriptor instead. func (*QueryNestedTypeRequest) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{32} + return file_product_proto_rawDescGZIP(), []int{33} } // Response message for nestedType operation. @@ -1604,7 +1649,7 @@ type QueryNestedTypeResponse struct { func (x *QueryNestedTypeResponse) Reset() { *x = QueryNestedTypeResponse{} - mi := &file_product_proto_msgTypes[33] + mi := &file_product_proto_msgTypes[34] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1616,7 +1661,7 @@ func (x *QueryNestedTypeResponse) String() string { func (*QueryNestedTypeResponse) ProtoMessage() {} func (x *QueryNestedTypeResponse) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[33] + mi := &file_product_proto_msgTypes[34] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1629,7 +1674,7 @@ func (x *QueryNestedTypeResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use QueryNestedTypeResponse.ProtoReflect.Descriptor instead. func (*QueryNestedTypeResponse) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{33} + return file_product_proto_rawDescGZIP(), []int{34} } func (x *QueryNestedTypeResponse) GetNestedType() []*NestedTypeA { @@ -1648,7 +1693,7 @@ type QueryRecursiveTypeRequest struct { func (x *QueryRecursiveTypeRequest) Reset() { *x = QueryRecursiveTypeRequest{} - mi := &file_product_proto_msgTypes[34] + mi := &file_product_proto_msgTypes[35] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1660,7 +1705,7 @@ func (x *QueryRecursiveTypeRequest) String() string { func (*QueryRecursiveTypeRequest) ProtoMessage() {} func (x *QueryRecursiveTypeRequest) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[34] + mi := &file_product_proto_msgTypes[35] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1673,7 +1718,7 @@ func (x *QueryRecursiveTypeRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use QueryRecursiveTypeRequest.ProtoReflect.Descriptor instead. func (*QueryRecursiveTypeRequest) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{34} + return file_product_proto_rawDescGZIP(), []int{35} } // Response message for recursiveType operation. @@ -1686,7 +1731,7 @@ type QueryRecursiveTypeResponse struct { func (x *QueryRecursiveTypeResponse) Reset() { *x = QueryRecursiveTypeResponse{} - mi := &file_product_proto_msgTypes[35] + mi := &file_product_proto_msgTypes[36] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1698,7 +1743,7 @@ func (x *QueryRecursiveTypeResponse) String() string { func (*QueryRecursiveTypeResponse) ProtoMessage() {} func (x *QueryRecursiveTypeResponse) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[35] + mi := &file_product_proto_msgTypes[36] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1711,7 +1756,7 @@ func (x *QueryRecursiveTypeResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use QueryRecursiveTypeResponse.ProtoReflect.Descriptor instead. func (*QueryRecursiveTypeResponse) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{35} + return file_product_proto_rawDescGZIP(), []int{36} } func (x *QueryRecursiveTypeResponse) GetRecursiveType() *RecursiveType { @@ -1732,7 +1777,7 @@ type QueryTypeFilterWithArgumentsRequest struct { func (x *QueryTypeFilterWithArgumentsRequest) Reset() { *x = QueryTypeFilterWithArgumentsRequest{} - mi := &file_product_proto_msgTypes[36] + mi := &file_product_proto_msgTypes[37] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1744,7 +1789,7 @@ func (x *QueryTypeFilterWithArgumentsRequest) String() string { func (*QueryTypeFilterWithArgumentsRequest) ProtoMessage() {} func (x *QueryTypeFilterWithArgumentsRequest) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[36] + mi := &file_product_proto_msgTypes[37] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1757,7 +1802,7 @@ func (x *QueryTypeFilterWithArgumentsRequest) ProtoReflect() protoreflect.Messag // Deprecated: Use QueryTypeFilterWithArgumentsRequest.ProtoReflect.Descriptor instead. func (*QueryTypeFilterWithArgumentsRequest) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{36} + return file_product_proto_rawDescGZIP(), []int{37} } func (x *QueryTypeFilterWithArgumentsRequest) GetFilterField_1() string { @@ -1784,7 +1829,7 @@ type QueryTypeFilterWithArgumentsResponse struct { func (x *QueryTypeFilterWithArgumentsResponse) Reset() { *x = QueryTypeFilterWithArgumentsResponse{} - mi := &file_product_proto_msgTypes[37] + mi := &file_product_proto_msgTypes[38] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1796,7 +1841,7 @@ func (x *QueryTypeFilterWithArgumentsResponse) String() string { func (*QueryTypeFilterWithArgumentsResponse) ProtoMessage() {} func (x *QueryTypeFilterWithArgumentsResponse) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[37] + mi := &file_product_proto_msgTypes[38] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1809,7 +1854,7 @@ func (x *QueryTypeFilterWithArgumentsResponse) ProtoReflect() protoreflect.Messa // Deprecated: Use QueryTypeFilterWithArgumentsResponse.ProtoReflect.Descriptor instead. func (*QueryTypeFilterWithArgumentsResponse) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{37} + return file_product_proto_rawDescGZIP(), []int{38} } func (x *QueryTypeFilterWithArgumentsResponse) GetTypeFilterWithArguments() []*TypeWithMultipleFilterFields { @@ -1829,7 +1874,7 @@ type QueryTypeWithMultipleFilterFieldsRequest struct { func (x *QueryTypeWithMultipleFilterFieldsRequest) Reset() { *x = QueryTypeWithMultipleFilterFieldsRequest{} - mi := &file_product_proto_msgTypes[38] + mi := &file_product_proto_msgTypes[39] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1841,7 +1886,7 @@ func (x *QueryTypeWithMultipleFilterFieldsRequest) String() string { func (*QueryTypeWithMultipleFilterFieldsRequest) ProtoMessage() {} func (x *QueryTypeWithMultipleFilterFieldsRequest) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[38] + mi := &file_product_proto_msgTypes[39] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1854,7 +1899,7 @@ func (x *QueryTypeWithMultipleFilterFieldsRequest) ProtoReflect() protoreflect.M // Deprecated: Use QueryTypeWithMultipleFilterFieldsRequest.ProtoReflect.Descriptor instead. func (*QueryTypeWithMultipleFilterFieldsRequest) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{38} + return file_product_proto_rawDescGZIP(), []int{39} } func (x *QueryTypeWithMultipleFilterFieldsRequest) GetFilter() *FilterTypeInput { @@ -1874,7 +1919,7 @@ type QueryTypeWithMultipleFilterFieldsResponse struct { func (x *QueryTypeWithMultipleFilterFieldsResponse) Reset() { *x = QueryTypeWithMultipleFilterFieldsResponse{} - mi := &file_product_proto_msgTypes[39] + mi := &file_product_proto_msgTypes[40] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1886,7 +1931,7 @@ func (x *QueryTypeWithMultipleFilterFieldsResponse) String() string { func (*QueryTypeWithMultipleFilterFieldsResponse) ProtoMessage() {} func (x *QueryTypeWithMultipleFilterFieldsResponse) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[39] + mi := &file_product_proto_msgTypes[40] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1899,7 +1944,7 @@ func (x *QueryTypeWithMultipleFilterFieldsResponse) ProtoReflect() protoreflect. // Deprecated: Use QueryTypeWithMultipleFilterFieldsResponse.ProtoReflect.Descriptor instead. func (*QueryTypeWithMultipleFilterFieldsResponse) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{39} + return file_product_proto_rawDescGZIP(), []int{40} } func (x *QueryTypeWithMultipleFilterFieldsResponse) GetTypeWithMultipleFilterFields() []*TypeWithMultipleFilterFields { @@ -1919,7 +1964,7 @@ type QueryComplexFilterTypeRequest struct { func (x *QueryComplexFilterTypeRequest) Reset() { *x = QueryComplexFilterTypeRequest{} - mi := &file_product_proto_msgTypes[40] + mi := &file_product_proto_msgTypes[41] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1931,7 +1976,7 @@ func (x *QueryComplexFilterTypeRequest) String() string { func (*QueryComplexFilterTypeRequest) ProtoMessage() {} func (x *QueryComplexFilterTypeRequest) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[40] + mi := &file_product_proto_msgTypes[41] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1944,7 +1989,7 @@ func (x *QueryComplexFilterTypeRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use QueryComplexFilterTypeRequest.ProtoReflect.Descriptor instead. func (*QueryComplexFilterTypeRequest) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{40} + return file_product_proto_rawDescGZIP(), []int{41} } func (x *QueryComplexFilterTypeRequest) GetFilter() *ComplexFilterTypeInput { @@ -1964,7 +2009,7 @@ type QueryComplexFilterTypeResponse struct { func (x *QueryComplexFilterTypeResponse) Reset() { *x = QueryComplexFilterTypeResponse{} - mi := &file_product_proto_msgTypes[41] + mi := &file_product_proto_msgTypes[42] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1976,7 +2021,7 @@ func (x *QueryComplexFilterTypeResponse) String() string { func (*QueryComplexFilterTypeResponse) ProtoMessage() {} func (x *QueryComplexFilterTypeResponse) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[41] + mi := &file_product_proto_msgTypes[42] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1989,7 +2034,7 @@ func (x *QueryComplexFilterTypeResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use QueryComplexFilterTypeResponse.ProtoReflect.Descriptor instead. func (*QueryComplexFilterTypeResponse) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{41} + return file_product_proto_rawDescGZIP(), []int{42} } func (x *QueryComplexFilterTypeResponse) GetComplexFilterType() []*TypeWithComplexFilterInput { @@ -2009,7 +2054,7 @@ type QueryCalculateTotalsRequest struct { func (x *QueryCalculateTotalsRequest) Reset() { *x = QueryCalculateTotalsRequest{} - mi := &file_product_proto_msgTypes[42] + mi := &file_product_proto_msgTypes[43] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2021,7 +2066,7 @@ func (x *QueryCalculateTotalsRequest) String() string { func (*QueryCalculateTotalsRequest) ProtoMessage() {} func (x *QueryCalculateTotalsRequest) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[42] + mi := &file_product_proto_msgTypes[43] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2034,7 +2079,7 @@ func (x *QueryCalculateTotalsRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use QueryCalculateTotalsRequest.ProtoReflect.Descriptor instead. func (*QueryCalculateTotalsRequest) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{42} + return file_product_proto_rawDescGZIP(), []int{43} } func (x *QueryCalculateTotalsRequest) GetOrders() []*OrderInput { @@ -2054,7 +2099,7 @@ type QueryCalculateTotalsResponse struct { func (x *QueryCalculateTotalsResponse) Reset() { *x = QueryCalculateTotalsResponse{} - mi := &file_product_proto_msgTypes[43] + mi := &file_product_proto_msgTypes[44] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2066,7 +2111,7 @@ func (x *QueryCalculateTotalsResponse) String() string { func (*QueryCalculateTotalsResponse) ProtoMessage() {} func (x *QueryCalculateTotalsResponse) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[43] + mi := &file_product_proto_msgTypes[44] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2079,7 +2124,7 @@ func (x *QueryCalculateTotalsResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use QueryCalculateTotalsResponse.ProtoReflect.Descriptor instead. func (*QueryCalculateTotalsResponse) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{43} + return file_product_proto_rawDescGZIP(), []int{44} } func (x *QueryCalculateTotalsResponse) GetCalculateTotals() []*Order { @@ -2098,7 +2143,7 @@ type QueryCategoriesRequest struct { func (x *QueryCategoriesRequest) Reset() { *x = QueryCategoriesRequest{} - mi := &file_product_proto_msgTypes[44] + mi := &file_product_proto_msgTypes[45] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2110,7 +2155,7 @@ func (x *QueryCategoriesRequest) String() string { func (*QueryCategoriesRequest) ProtoMessage() {} func (x *QueryCategoriesRequest) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[44] + mi := &file_product_proto_msgTypes[45] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2123,7 +2168,7 @@ func (x *QueryCategoriesRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use QueryCategoriesRequest.ProtoReflect.Descriptor instead. func (*QueryCategoriesRequest) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{44} + return file_product_proto_rawDescGZIP(), []int{45} } // Response message for categories operation. @@ -2136,7 +2181,7 @@ type QueryCategoriesResponse struct { func (x *QueryCategoriesResponse) Reset() { *x = QueryCategoriesResponse{} - mi := &file_product_proto_msgTypes[45] + mi := &file_product_proto_msgTypes[46] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2148,7 +2193,7 @@ func (x *QueryCategoriesResponse) String() string { func (*QueryCategoriesResponse) ProtoMessage() {} func (x *QueryCategoriesResponse) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[45] + mi := &file_product_proto_msgTypes[46] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2161,7 +2206,7 @@ func (x *QueryCategoriesResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use QueryCategoriesResponse.ProtoReflect.Descriptor instead. func (*QueryCategoriesResponse) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{45} + return file_product_proto_rawDescGZIP(), []int{46} } func (x *QueryCategoriesResponse) GetCategories() []*Category { @@ -2181,7 +2226,7 @@ type QueryCategoriesByKindRequest struct { func (x *QueryCategoriesByKindRequest) Reset() { *x = QueryCategoriesByKindRequest{} - mi := &file_product_proto_msgTypes[46] + mi := &file_product_proto_msgTypes[47] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2193,7 +2238,7 @@ func (x *QueryCategoriesByKindRequest) String() string { func (*QueryCategoriesByKindRequest) ProtoMessage() {} func (x *QueryCategoriesByKindRequest) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[46] + mi := &file_product_proto_msgTypes[47] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2206,7 +2251,7 @@ func (x *QueryCategoriesByKindRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use QueryCategoriesByKindRequest.ProtoReflect.Descriptor instead. func (*QueryCategoriesByKindRequest) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{46} + return file_product_proto_rawDescGZIP(), []int{47} } func (x *QueryCategoriesByKindRequest) GetKind() CategoryKind { @@ -2226,7 +2271,7 @@ type QueryCategoriesByKindResponse struct { func (x *QueryCategoriesByKindResponse) Reset() { *x = QueryCategoriesByKindResponse{} - mi := &file_product_proto_msgTypes[47] + mi := &file_product_proto_msgTypes[48] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2238,7 +2283,7 @@ func (x *QueryCategoriesByKindResponse) String() string { func (*QueryCategoriesByKindResponse) ProtoMessage() {} func (x *QueryCategoriesByKindResponse) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[47] + mi := &file_product_proto_msgTypes[48] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2251,7 +2296,7 @@ func (x *QueryCategoriesByKindResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use QueryCategoriesByKindResponse.ProtoReflect.Descriptor instead. func (*QueryCategoriesByKindResponse) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{47} + return file_product_proto_rawDescGZIP(), []int{48} } func (x *QueryCategoriesByKindResponse) GetCategoriesByKind() []*Category { @@ -2271,7 +2316,7 @@ type QueryCategoriesByKindsRequest struct { func (x *QueryCategoriesByKindsRequest) Reset() { *x = QueryCategoriesByKindsRequest{} - mi := &file_product_proto_msgTypes[48] + mi := &file_product_proto_msgTypes[49] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2283,7 +2328,7 @@ func (x *QueryCategoriesByKindsRequest) String() string { func (*QueryCategoriesByKindsRequest) ProtoMessage() {} func (x *QueryCategoriesByKindsRequest) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[48] + mi := &file_product_proto_msgTypes[49] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2296,7 +2341,7 @@ func (x *QueryCategoriesByKindsRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use QueryCategoriesByKindsRequest.ProtoReflect.Descriptor instead. func (*QueryCategoriesByKindsRequest) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{48} + return file_product_proto_rawDescGZIP(), []int{49} } func (x *QueryCategoriesByKindsRequest) GetKinds() []CategoryKind { @@ -2316,7 +2361,7 @@ type QueryCategoriesByKindsResponse struct { func (x *QueryCategoriesByKindsResponse) Reset() { *x = QueryCategoriesByKindsResponse{} - mi := &file_product_proto_msgTypes[49] + mi := &file_product_proto_msgTypes[50] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2328,7 +2373,7 @@ func (x *QueryCategoriesByKindsResponse) String() string { func (*QueryCategoriesByKindsResponse) ProtoMessage() {} func (x *QueryCategoriesByKindsResponse) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[49] + mi := &file_product_proto_msgTypes[50] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2341,7 +2386,7 @@ func (x *QueryCategoriesByKindsResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use QueryCategoriesByKindsResponse.ProtoReflect.Descriptor instead. func (*QueryCategoriesByKindsResponse) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{49} + return file_product_proto_rawDescGZIP(), []int{50} } func (x *QueryCategoriesByKindsResponse) GetCategoriesByKinds() []*Category { @@ -2361,7 +2406,7 @@ type QueryFilterCategoriesRequest struct { func (x *QueryFilterCategoriesRequest) Reset() { *x = QueryFilterCategoriesRequest{} - mi := &file_product_proto_msgTypes[50] + mi := &file_product_proto_msgTypes[51] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2373,7 +2418,7 @@ func (x *QueryFilterCategoriesRequest) String() string { func (*QueryFilterCategoriesRequest) ProtoMessage() {} func (x *QueryFilterCategoriesRequest) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[50] + mi := &file_product_proto_msgTypes[51] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2386,7 +2431,7 @@ func (x *QueryFilterCategoriesRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use QueryFilterCategoriesRequest.ProtoReflect.Descriptor instead. func (*QueryFilterCategoriesRequest) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{50} + return file_product_proto_rawDescGZIP(), []int{51} } func (x *QueryFilterCategoriesRequest) GetFilter() *CategoryFilter { @@ -2406,7 +2451,7 @@ type QueryFilterCategoriesResponse struct { func (x *QueryFilterCategoriesResponse) Reset() { *x = QueryFilterCategoriesResponse{} - mi := &file_product_proto_msgTypes[51] + mi := &file_product_proto_msgTypes[52] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2418,7 +2463,7 @@ func (x *QueryFilterCategoriesResponse) String() string { func (*QueryFilterCategoriesResponse) ProtoMessage() {} func (x *QueryFilterCategoriesResponse) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[51] + mi := &file_product_proto_msgTypes[52] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2431,7 +2476,7 @@ func (x *QueryFilterCategoriesResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use QueryFilterCategoriesResponse.ProtoReflect.Descriptor instead. func (*QueryFilterCategoriesResponse) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{51} + return file_product_proto_rawDescGZIP(), []int{52} } func (x *QueryFilterCategoriesResponse) GetFilterCategories() []*Category { @@ -2450,7 +2495,7 @@ type QueryRandomPetRequest struct { func (x *QueryRandomPetRequest) Reset() { *x = QueryRandomPetRequest{} - mi := &file_product_proto_msgTypes[52] + mi := &file_product_proto_msgTypes[53] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2462,7 +2507,7 @@ func (x *QueryRandomPetRequest) String() string { func (*QueryRandomPetRequest) ProtoMessage() {} func (x *QueryRandomPetRequest) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[52] + mi := &file_product_proto_msgTypes[53] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2475,7 +2520,7 @@ func (x *QueryRandomPetRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use QueryRandomPetRequest.ProtoReflect.Descriptor instead. func (*QueryRandomPetRequest) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{52} + return file_product_proto_rawDescGZIP(), []int{53} } // Response message for randomPet operation. @@ -2488,7 +2533,7 @@ type QueryRandomPetResponse struct { func (x *QueryRandomPetResponse) Reset() { *x = QueryRandomPetResponse{} - mi := &file_product_proto_msgTypes[53] + mi := &file_product_proto_msgTypes[54] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2500,7 +2545,7 @@ func (x *QueryRandomPetResponse) String() string { func (*QueryRandomPetResponse) ProtoMessage() {} func (x *QueryRandomPetResponse) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[53] + mi := &file_product_proto_msgTypes[54] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2513,7 +2558,7 @@ func (x *QueryRandomPetResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use QueryRandomPetResponse.ProtoReflect.Descriptor instead. func (*QueryRandomPetResponse) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{53} + return file_product_proto_rawDescGZIP(), []int{54} } func (x *QueryRandomPetResponse) GetRandomPet() *Animal { @@ -2532,7 +2577,7 @@ type QueryAllPetsRequest struct { func (x *QueryAllPetsRequest) Reset() { *x = QueryAllPetsRequest{} - mi := &file_product_proto_msgTypes[54] + mi := &file_product_proto_msgTypes[55] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2544,7 +2589,7 @@ func (x *QueryAllPetsRequest) String() string { func (*QueryAllPetsRequest) ProtoMessage() {} func (x *QueryAllPetsRequest) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[54] + mi := &file_product_proto_msgTypes[55] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2557,7 +2602,7 @@ func (x *QueryAllPetsRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use QueryAllPetsRequest.ProtoReflect.Descriptor instead. func (*QueryAllPetsRequest) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{54} + return file_product_proto_rawDescGZIP(), []int{55} } // Response message for allPets operation. @@ -2570,7 +2615,7 @@ type QueryAllPetsResponse struct { func (x *QueryAllPetsResponse) Reset() { *x = QueryAllPetsResponse{} - mi := &file_product_proto_msgTypes[55] + mi := &file_product_proto_msgTypes[56] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2582,7 +2627,7 @@ func (x *QueryAllPetsResponse) String() string { func (*QueryAllPetsResponse) ProtoMessage() {} func (x *QueryAllPetsResponse) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[55] + mi := &file_product_proto_msgTypes[56] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2595,7 +2640,7 @@ func (x *QueryAllPetsResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use QueryAllPetsResponse.ProtoReflect.Descriptor instead. func (*QueryAllPetsResponse) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{55} + return file_product_proto_rawDescGZIP(), []int{56} } func (x *QueryAllPetsResponse) GetAllPets() []*Animal { @@ -2615,7 +2660,7 @@ type QuerySearchRequest struct { func (x *QuerySearchRequest) Reset() { *x = QuerySearchRequest{} - mi := &file_product_proto_msgTypes[56] + mi := &file_product_proto_msgTypes[57] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2627,7 +2672,7 @@ func (x *QuerySearchRequest) String() string { func (*QuerySearchRequest) ProtoMessage() {} func (x *QuerySearchRequest) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[56] + mi := &file_product_proto_msgTypes[57] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2640,7 +2685,7 @@ func (x *QuerySearchRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use QuerySearchRequest.ProtoReflect.Descriptor instead. func (*QuerySearchRequest) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{56} + return file_product_proto_rawDescGZIP(), []int{57} } func (x *QuerySearchRequest) GetInput() *SearchInput { @@ -2660,7 +2705,7 @@ type QuerySearchResponse struct { func (x *QuerySearchResponse) Reset() { *x = QuerySearchResponse{} - mi := &file_product_proto_msgTypes[57] + mi := &file_product_proto_msgTypes[58] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2672,7 +2717,7 @@ func (x *QuerySearchResponse) String() string { func (*QuerySearchResponse) ProtoMessage() {} func (x *QuerySearchResponse) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[57] + mi := &file_product_proto_msgTypes[58] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2685,7 +2730,7 @@ func (x *QuerySearchResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use QuerySearchResponse.ProtoReflect.Descriptor instead. func (*QuerySearchResponse) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{57} + return file_product_proto_rawDescGZIP(), []int{58} } func (x *QuerySearchResponse) GetSearch() []*SearchResult { @@ -2704,7 +2749,7 @@ type QueryRandomSearchResultRequest struct { func (x *QueryRandomSearchResultRequest) Reset() { *x = QueryRandomSearchResultRequest{} - mi := &file_product_proto_msgTypes[58] + mi := &file_product_proto_msgTypes[59] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2716,7 +2761,7 @@ func (x *QueryRandomSearchResultRequest) String() string { func (*QueryRandomSearchResultRequest) ProtoMessage() {} func (x *QueryRandomSearchResultRequest) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[58] + mi := &file_product_proto_msgTypes[59] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2729,7 +2774,7 @@ func (x *QueryRandomSearchResultRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use QueryRandomSearchResultRequest.ProtoReflect.Descriptor instead. func (*QueryRandomSearchResultRequest) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{58} + return file_product_proto_rawDescGZIP(), []int{59} } // Response message for randomSearchResult operation. @@ -2742,7 +2787,7 @@ type QueryRandomSearchResultResponse struct { func (x *QueryRandomSearchResultResponse) Reset() { *x = QueryRandomSearchResultResponse{} - mi := &file_product_proto_msgTypes[59] + mi := &file_product_proto_msgTypes[60] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2754,7 +2799,7 @@ func (x *QueryRandomSearchResultResponse) String() string { func (*QueryRandomSearchResultResponse) ProtoMessage() {} func (x *QueryRandomSearchResultResponse) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[59] + mi := &file_product_proto_msgTypes[60] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2767,7 +2812,7 @@ func (x *QueryRandomSearchResultResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use QueryRandomSearchResultResponse.ProtoReflect.Descriptor instead. func (*QueryRandomSearchResultResponse) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{59} + return file_product_proto_rawDescGZIP(), []int{60} } func (x *QueryRandomSearchResultResponse) GetRandomSearchResult() *SearchResult { @@ -2786,7 +2831,7 @@ type QueryNullableFieldsTypeRequest struct { func (x *QueryNullableFieldsTypeRequest) Reset() { *x = QueryNullableFieldsTypeRequest{} - mi := &file_product_proto_msgTypes[60] + mi := &file_product_proto_msgTypes[61] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2798,7 +2843,7 @@ func (x *QueryNullableFieldsTypeRequest) String() string { func (*QueryNullableFieldsTypeRequest) ProtoMessage() {} func (x *QueryNullableFieldsTypeRequest) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[60] + mi := &file_product_proto_msgTypes[61] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2811,7 +2856,7 @@ func (x *QueryNullableFieldsTypeRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use QueryNullableFieldsTypeRequest.ProtoReflect.Descriptor instead. func (*QueryNullableFieldsTypeRequest) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{60} + return file_product_proto_rawDescGZIP(), []int{61} } // Response message for nullableFieldsType operation. @@ -2824,7 +2869,7 @@ type QueryNullableFieldsTypeResponse struct { func (x *QueryNullableFieldsTypeResponse) Reset() { *x = QueryNullableFieldsTypeResponse{} - mi := &file_product_proto_msgTypes[61] + mi := &file_product_proto_msgTypes[62] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2836,7 +2881,7 @@ func (x *QueryNullableFieldsTypeResponse) String() string { func (*QueryNullableFieldsTypeResponse) ProtoMessage() {} func (x *QueryNullableFieldsTypeResponse) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[61] + mi := &file_product_proto_msgTypes[62] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2849,7 +2894,7 @@ func (x *QueryNullableFieldsTypeResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use QueryNullableFieldsTypeResponse.ProtoReflect.Descriptor instead. func (*QueryNullableFieldsTypeResponse) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{61} + return file_product_proto_rawDescGZIP(), []int{62} } func (x *QueryNullableFieldsTypeResponse) GetNullableFieldsType() *NullableFieldsType { @@ -2869,7 +2914,7 @@ type QueryNullableFieldsTypeByIdRequest struct { func (x *QueryNullableFieldsTypeByIdRequest) Reset() { *x = QueryNullableFieldsTypeByIdRequest{} - mi := &file_product_proto_msgTypes[62] + mi := &file_product_proto_msgTypes[63] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2881,7 +2926,7 @@ func (x *QueryNullableFieldsTypeByIdRequest) String() string { func (*QueryNullableFieldsTypeByIdRequest) ProtoMessage() {} func (x *QueryNullableFieldsTypeByIdRequest) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[62] + mi := &file_product_proto_msgTypes[63] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2894,7 +2939,7 @@ func (x *QueryNullableFieldsTypeByIdRequest) ProtoReflect() protoreflect.Message // Deprecated: Use QueryNullableFieldsTypeByIdRequest.ProtoReflect.Descriptor instead. func (*QueryNullableFieldsTypeByIdRequest) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{62} + return file_product_proto_rawDescGZIP(), []int{63} } func (x *QueryNullableFieldsTypeByIdRequest) GetId() string { @@ -2914,7 +2959,7 @@ type QueryNullableFieldsTypeByIdResponse struct { func (x *QueryNullableFieldsTypeByIdResponse) Reset() { *x = QueryNullableFieldsTypeByIdResponse{} - mi := &file_product_proto_msgTypes[63] + mi := &file_product_proto_msgTypes[64] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2926,7 +2971,7 @@ func (x *QueryNullableFieldsTypeByIdResponse) String() string { func (*QueryNullableFieldsTypeByIdResponse) ProtoMessage() {} func (x *QueryNullableFieldsTypeByIdResponse) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[63] + mi := &file_product_proto_msgTypes[64] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2939,7 +2984,7 @@ func (x *QueryNullableFieldsTypeByIdResponse) ProtoReflect() protoreflect.Messag // Deprecated: Use QueryNullableFieldsTypeByIdResponse.ProtoReflect.Descriptor instead. func (*QueryNullableFieldsTypeByIdResponse) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{63} + return file_product_proto_rawDescGZIP(), []int{64} } func (x *QueryNullableFieldsTypeByIdResponse) GetNullableFieldsTypeById() *NullableFieldsType { @@ -2959,7 +3004,7 @@ type QueryNullableFieldsTypeWithFilterRequest struct { func (x *QueryNullableFieldsTypeWithFilterRequest) Reset() { *x = QueryNullableFieldsTypeWithFilterRequest{} - mi := &file_product_proto_msgTypes[64] + mi := &file_product_proto_msgTypes[65] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2971,7 +3016,7 @@ func (x *QueryNullableFieldsTypeWithFilterRequest) String() string { func (*QueryNullableFieldsTypeWithFilterRequest) ProtoMessage() {} func (x *QueryNullableFieldsTypeWithFilterRequest) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[64] + mi := &file_product_proto_msgTypes[65] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2984,7 +3029,7 @@ func (x *QueryNullableFieldsTypeWithFilterRequest) ProtoReflect() protoreflect.M // Deprecated: Use QueryNullableFieldsTypeWithFilterRequest.ProtoReflect.Descriptor instead. func (*QueryNullableFieldsTypeWithFilterRequest) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{64} + return file_product_proto_rawDescGZIP(), []int{65} } func (x *QueryNullableFieldsTypeWithFilterRequest) GetFilter() *NullableFieldsFilter { @@ -3004,7 +3049,7 @@ type QueryNullableFieldsTypeWithFilterResponse struct { func (x *QueryNullableFieldsTypeWithFilterResponse) Reset() { *x = QueryNullableFieldsTypeWithFilterResponse{} - mi := &file_product_proto_msgTypes[65] + mi := &file_product_proto_msgTypes[66] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3016,7 +3061,7 @@ func (x *QueryNullableFieldsTypeWithFilterResponse) String() string { func (*QueryNullableFieldsTypeWithFilterResponse) ProtoMessage() {} func (x *QueryNullableFieldsTypeWithFilterResponse) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[65] + mi := &file_product_proto_msgTypes[66] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3029,7 +3074,7 @@ func (x *QueryNullableFieldsTypeWithFilterResponse) ProtoReflect() protoreflect. // Deprecated: Use QueryNullableFieldsTypeWithFilterResponse.ProtoReflect.Descriptor instead. func (*QueryNullableFieldsTypeWithFilterResponse) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{65} + return file_product_proto_rawDescGZIP(), []int{66} } func (x *QueryNullableFieldsTypeWithFilterResponse) GetNullableFieldsTypeWithFilter() []*NullableFieldsType { @@ -3048,7 +3093,7 @@ type QueryAllNullableFieldsTypesRequest struct { func (x *QueryAllNullableFieldsTypesRequest) Reset() { *x = QueryAllNullableFieldsTypesRequest{} - mi := &file_product_proto_msgTypes[66] + mi := &file_product_proto_msgTypes[67] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3060,7 +3105,7 @@ func (x *QueryAllNullableFieldsTypesRequest) String() string { func (*QueryAllNullableFieldsTypesRequest) ProtoMessage() {} func (x *QueryAllNullableFieldsTypesRequest) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[66] + mi := &file_product_proto_msgTypes[67] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3073,7 +3118,7 @@ func (x *QueryAllNullableFieldsTypesRequest) ProtoReflect() protoreflect.Message // Deprecated: Use QueryAllNullableFieldsTypesRequest.ProtoReflect.Descriptor instead. func (*QueryAllNullableFieldsTypesRequest) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{66} + return file_product_proto_rawDescGZIP(), []int{67} } // Response message for allNullableFieldsTypes operation. @@ -3086,7 +3131,7 @@ type QueryAllNullableFieldsTypesResponse struct { func (x *QueryAllNullableFieldsTypesResponse) Reset() { *x = QueryAllNullableFieldsTypesResponse{} - mi := &file_product_proto_msgTypes[67] + mi := &file_product_proto_msgTypes[68] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3098,7 +3143,7 @@ func (x *QueryAllNullableFieldsTypesResponse) String() string { func (*QueryAllNullableFieldsTypesResponse) ProtoMessage() {} func (x *QueryAllNullableFieldsTypesResponse) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[67] + mi := &file_product_proto_msgTypes[68] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3111,7 +3156,7 @@ func (x *QueryAllNullableFieldsTypesResponse) ProtoReflect() protoreflect.Messag // Deprecated: Use QueryAllNullableFieldsTypesResponse.ProtoReflect.Descriptor instead. func (*QueryAllNullableFieldsTypesResponse) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{67} + return file_product_proto_rawDescGZIP(), []int{68} } func (x *QueryAllNullableFieldsTypesResponse) GetAllNullableFieldsTypes() []*NullableFieldsType { @@ -3130,7 +3175,7 @@ type QueryBlogPostRequest struct { func (x *QueryBlogPostRequest) Reset() { *x = QueryBlogPostRequest{} - mi := &file_product_proto_msgTypes[68] + mi := &file_product_proto_msgTypes[69] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3142,7 +3187,7 @@ func (x *QueryBlogPostRequest) String() string { func (*QueryBlogPostRequest) ProtoMessage() {} func (x *QueryBlogPostRequest) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[68] + mi := &file_product_proto_msgTypes[69] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3155,7 +3200,7 @@ func (x *QueryBlogPostRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use QueryBlogPostRequest.ProtoReflect.Descriptor instead. func (*QueryBlogPostRequest) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{68} + return file_product_proto_rawDescGZIP(), []int{69} } // Response message for blogPost operation. @@ -3168,7 +3213,7 @@ type QueryBlogPostResponse struct { func (x *QueryBlogPostResponse) Reset() { *x = QueryBlogPostResponse{} - mi := &file_product_proto_msgTypes[69] + mi := &file_product_proto_msgTypes[70] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3180,7 +3225,7 @@ func (x *QueryBlogPostResponse) String() string { func (*QueryBlogPostResponse) ProtoMessage() {} func (x *QueryBlogPostResponse) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[69] + mi := &file_product_proto_msgTypes[70] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3193,7 +3238,7 @@ func (x *QueryBlogPostResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use QueryBlogPostResponse.ProtoReflect.Descriptor instead. func (*QueryBlogPostResponse) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{69} + return file_product_proto_rawDescGZIP(), []int{70} } func (x *QueryBlogPostResponse) GetBlogPost() *BlogPost { @@ -3213,7 +3258,7 @@ type QueryBlogPostByIdRequest struct { func (x *QueryBlogPostByIdRequest) Reset() { *x = QueryBlogPostByIdRequest{} - mi := &file_product_proto_msgTypes[70] + mi := &file_product_proto_msgTypes[71] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3225,7 +3270,7 @@ func (x *QueryBlogPostByIdRequest) String() string { func (*QueryBlogPostByIdRequest) ProtoMessage() {} func (x *QueryBlogPostByIdRequest) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[70] + mi := &file_product_proto_msgTypes[71] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3238,7 +3283,7 @@ func (x *QueryBlogPostByIdRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use QueryBlogPostByIdRequest.ProtoReflect.Descriptor instead. func (*QueryBlogPostByIdRequest) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{70} + return file_product_proto_rawDescGZIP(), []int{71} } func (x *QueryBlogPostByIdRequest) GetId() string { @@ -3258,7 +3303,7 @@ type QueryBlogPostByIdResponse struct { func (x *QueryBlogPostByIdResponse) Reset() { *x = QueryBlogPostByIdResponse{} - mi := &file_product_proto_msgTypes[71] + mi := &file_product_proto_msgTypes[72] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3270,7 +3315,7 @@ func (x *QueryBlogPostByIdResponse) String() string { func (*QueryBlogPostByIdResponse) ProtoMessage() {} func (x *QueryBlogPostByIdResponse) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[71] + mi := &file_product_proto_msgTypes[72] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3283,7 +3328,7 @@ func (x *QueryBlogPostByIdResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use QueryBlogPostByIdResponse.ProtoReflect.Descriptor instead. func (*QueryBlogPostByIdResponse) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{71} + return file_product_proto_rawDescGZIP(), []int{72} } func (x *QueryBlogPostByIdResponse) GetBlogPostById() *BlogPost { @@ -3303,7 +3348,7 @@ type QueryBlogPostsWithFilterRequest struct { func (x *QueryBlogPostsWithFilterRequest) Reset() { *x = QueryBlogPostsWithFilterRequest{} - mi := &file_product_proto_msgTypes[72] + mi := &file_product_proto_msgTypes[73] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3315,7 +3360,7 @@ func (x *QueryBlogPostsWithFilterRequest) String() string { func (*QueryBlogPostsWithFilterRequest) ProtoMessage() {} func (x *QueryBlogPostsWithFilterRequest) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[72] + mi := &file_product_proto_msgTypes[73] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3328,7 +3373,7 @@ func (x *QueryBlogPostsWithFilterRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use QueryBlogPostsWithFilterRequest.ProtoReflect.Descriptor instead. func (*QueryBlogPostsWithFilterRequest) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{72} + return file_product_proto_rawDescGZIP(), []int{73} } func (x *QueryBlogPostsWithFilterRequest) GetFilter() *BlogPostFilter { @@ -3348,7 +3393,7 @@ type QueryBlogPostsWithFilterResponse struct { func (x *QueryBlogPostsWithFilterResponse) Reset() { *x = QueryBlogPostsWithFilterResponse{} - mi := &file_product_proto_msgTypes[73] + mi := &file_product_proto_msgTypes[74] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3360,7 +3405,7 @@ func (x *QueryBlogPostsWithFilterResponse) String() string { func (*QueryBlogPostsWithFilterResponse) ProtoMessage() {} func (x *QueryBlogPostsWithFilterResponse) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[73] + mi := &file_product_proto_msgTypes[74] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3373,7 +3418,7 @@ func (x *QueryBlogPostsWithFilterResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use QueryBlogPostsWithFilterResponse.ProtoReflect.Descriptor instead. func (*QueryBlogPostsWithFilterResponse) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{73} + return file_product_proto_rawDescGZIP(), []int{74} } func (x *QueryBlogPostsWithFilterResponse) GetBlogPostsWithFilter() []*BlogPost { @@ -3392,7 +3437,7 @@ type QueryAllBlogPostsRequest struct { func (x *QueryAllBlogPostsRequest) Reset() { *x = QueryAllBlogPostsRequest{} - mi := &file_product_proto_msgTypes[74] + mi := &file_product_proto_msgTypes[75] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3404,7 +3449,7 @@ func (x *QueryAllBlogPostsRequest) String() string { func (*QueryAllBlogPostsRequest) ProtoMessage() {} func (x *QueryAllBlogPostsRequest) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[74] + mi := &file_product_proto_msgTypes[75] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3417,7 +3462,7 @@ func (x *QueryAllBlogPostsRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use QueryAllBlogPostsRequest.ProtoReflect.Descriptor instead. func (*QueryAllBlogPostsRequest) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{74} + return file_product_proto_rawDescGZIP(), []int{75} } // Response message for allBlogPosts operation. @@ -3430,7 +3475,7 @@ type QueryAllBlogPostsResponse struct { func (x *QueryAllBlogPostsResponse) Reset() { *x = QueryAllBlogPostsResponse{} - mi := &file_product_proto_msgTypes[75] + mi := &file_product_proto_msgTypes[76] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3442,7 +3487,7 @@ func (x *QueryAllBlogPostsResponse) String() string { func (*QueryAllBlogPostsResponse) ProtoMessage() {} func (x *QueryAllBlogPostsResponse) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[75] + mi := &file_product_proto_msgTypes[76] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3455,7 +3500,7 @@ func (x *QueryAllBlogPostsResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use QueryAllBlogPostsResponse.ProtoReflect.Descriptor instead. func (*QueryAllBlogPostsResponse) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{75} + return file_product_proto_rawDescGZIP(), []int{76} } func (x *QueryAllBlogPostsResponse) GetAllBlogPosts() []*BlogPost { @@ -3474,7 +3519,7 @@ type QueryAuthorRequest struct { func (x *QueryAuthorRequest) Reset() { *x = QueryAuthorRequest{} - mi := &file_product_proto_msgTypes[76] + mi := &file_product_proto_msgTypes[77] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3486,7 +3531,7 @@ func (x *QueryAuthorRequest) String() string { func (*QueryAuthorRequest) ProtoMessage() {} func (x *QueryAuthorRequest) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[76] + mi := &file_product_proto_msgTypes[77] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3499,7 +3544,7 @@ func (x *QueryAuthorRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use QueryAuthorRequest.ProtoReflect.Descriptor instead. func (*QueryAuthorRequest) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{76} + return file_product_proto_rawDescGZIP(), []int{77} } // Response message for author operation. @@ -3512,7 +3557,7 @@ type QueryAuthorResponse struct { func (x *QueryAuthorResponse) Reset() { *x = QueryAuthorResponse{} - mi := &file_product_proto_msgTypes[77] + mi := &file_product_proto_msgTypes[78] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3524,7 +3569,7 @@ func (x *QueryAuthorResponse) String() string { func (*QueryAuthorResponse) ProtoMessage() {} func (x *QueryAuthorResponse) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[77] + mi := &file_product_proto_msgTypes[78] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3537,7 +3582,7 @@ func (x *QueryAuthorResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use QueryAuthorResponse.ProtoReflect.Descriptor instead. func (*QueryAuthorResponse) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{77} + return file_product_proto_rawDescGZIP(), []int{78} } func (x *QueryAuthorResponse) GetAuthor() *Author { @@ -3557,7 +3602,7 @@ type QueryAuthorByIdRequest struct { func (x *QueryAuthorByIdRequest) Reset() { *x = QueryAuthorByIdRequest{} - mi := &file_product_proto_msgTypes[78] + mi := &file_product_proto_msgTypes[79] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3569,7 +3614,7 @@ func (x *QueryAuthorByIdRequest) String() string { func (*QueryAuthorByIdRequest) ProtoMessage() {} func (x *QueryAuthorByIdRequest) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[78] + mi := &file_product_proto_msgTypes[79] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3582,7 +3627,7 @@ func (x *QueryAuthorByIdRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use QueryAuthorByIdRequest.ProtoReflect.Descriptor instead. func (*QueryAuthorByIdRequest) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{78} + return file_product_proto_rawDescGZIP(), []int{79} } func (x *QueryAuthorByIdRequest) GetId() string { @@ -3602,7 +3647,7 @@ type QueryAuthorByIdResponse struct { func (x *QueryAuthorByIdResponse) Reset() { *x = QueryAuthorByIdResponse{} - mi := &file_product_proto_msgTypes[79] + mi := &file_product_proto_msgTypes[80] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3614,7 +3659,7 @@ func (x *QueryAuthorByIdResponse) String() string { func (*QueryAuthorByIdResponse) ProtoMessage() {} func (x *QueryAuthorByIdResponse) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[79] + mi := &file_product_proto_msgTypes[80] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3627,7 +3672,7 @@ func (x *QueryAuthorByIdResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use QueryAuthorByIdResponse.ProtoReflect.Descriptor instead. func (*QueryAuthorByIdResponse) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{79} + return file_product_proto_rawDescGZIP(), []int{80} } func (x *QueryAuthorByIdResponse) GetAuthorById() *Author { @@ -3647,7 +3692,7 @@ type QueryAuthorsWithFilterRequest struct { func (x *QueryAuthorsWithFilterRequest) Reset() { *x = QueryAuthorsWithFilterRequest{} - mi := &file_product_proto_msgTypes[80] + mi := &file_product_proto_msgTypes[81] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3659,7 +3704,7 @@ func (x *QueryAuthorsWithFilterRequest) String() string { func (*QueryAuthorsWithFilterRequest) ProtoMessage() {} func (x *QueryAuthorsWithFilterRequest) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[80] + mi := &file_product_proto_msgTypes[81] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3672,7 +3717,7 @@ func (x *QueryAuthorsWithFilterRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use QueryAuthorsWithFilterRequest.ProtoReflect.Descriptor instead. func (*QueryAuthorsWithFilterRequest) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{80} + return file_product_proto_rawDescGZIP(), []int{81} } func (x *QueryAuthorsWithFilterRequest) GetFilter() *AuthorFilter { @@ -3692,7 +3737,7 @@ type QueryAuthorsWithFilterResponse struct { func (x *QueryAuthorsWithFilterResponse) Reset() { *x = QueryAuthorsWithFilterResponse{} - mi := &file_product_proto_msgTypes[81] + mi := &file_product_proto_msgTypes[82] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3704,7 +3749,7 @@ func (x *QueryAuthorsWithFilterResponse) String() string { func (*QueryAuthorsWithFilterResponse) ProtoMessage() {} func (x *QueryAuthorsWithFilterResponse) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[81] + mi := &file_product_proto_msgTypes[82] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3717,7 +3762,7 @@ func (x *QueryAuthorsWithFilterResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use QueryAuthorsWithFilterResponse.ProtoReflect.Descriptor instead. func (*QueryAuthorsWithFilterResponse) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{81} + return file_product_proto_rawDescGZIP(), []int{82} } func (x *QueryAuthorsWithFilterResponse) GetAuthorsWithFilter() []*Author { @@ -3736,7 +3781,7 @@ type QueryAllAuthorsRequest struct { func (x *QueryAllAuthorsRequest) Reset() { *x = QueryAllAuthorsRequest{} - mi := &file_product_proto_msgTypes[82] + mi := &file_product_proto_msgTypes[83] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3748,7 +3793,7 @@ func (x *QueryAllAuthorsRequest) String() string { func (*QueryAllAuthorsRequest) ProtoMessage() {} func (x *QueryAllAuthorsRequest) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[82] + mi := &file_product_proto_msgTypes[83] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3761,7 +3806,7 @@ func (x *QueryAllAuthorsRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use QueryAllAuthorsRequest.ProtoReflect.Descriptor instead. func (*QueryAllAuthorsRequest) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{82} + return file_product_proto_rawDescGZIP(), []int{83} } // Response message for allAuthors operation. @@ -3774,7 +3819,7 @@ type QueryAllAuthorsResponse struct { func (x *QueryAllAuthorsResponse) Reset() { *x = QueryAllAuthorsResponse{} - mi := &file_product_proto_msgTypes[83] + mi := &file_product_proto_msgTypes[84] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3786,7 +3831,7 @@ func (x *QueryAllAuthorsResponse) String() string { func (*QueryAllAuthorsResponse) ProtoMessage() {} func (x *QueryAllAuthorsResponse) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[83] + mi := &file_product_proto_msgTypes[84] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3799,7 +3844,7 @@ func (x *QueryAllAuthorsResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use QueryAllAuthorsResponse.ProtoReflect.Descriptor instead. func (*QueryAllAuthorsResponse) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{83} + return file_product_proto_rawDescGZIP(), []int{84} } func (x *QueryAllAuthorsResponse) GetAllAuthors() []*Author { @@ -3819,7 +3864,7 @@ type QueryBulkSearchAuthorsRequest struct { func (x *QueryBulkSearchAuthorsRequest) Reset() { *x = QueryBulkSearchAuthorsRequest{} - mi := &file_product_proto_msgTypes[84] + mi := &file_product_proto_msgTypes[85] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3831,7 +3876,7 @@ func (x *QueryBulkSearchAuthorsRequest) String() string { func (*QueryBulkSearchAuthorsRequest) ProtoMessage() {} func (x *QueryBulkSearchAuthorsRequest) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[84] + mi := &file_product_proto_msgTypes[85] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3844,7 +3889,7 @@ func (x *QueryBulkSearchAuthorsRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use QueryBulkSearchAuthorsRequest.ProtoReflect.Descriptor instead. func (*QueryBulkSearchAuthorsRequest) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{84} + return file_product_proto_rawDescGZIP(), []int{85} } func (x *QueryBulkSearchAuthorsRequest) GetFilters() *ListOfAuthorFilter { @@ -3864,7 +3909,7 @@ type QueryBulkSearchAuthorsResponse struct { func (x *QueryBulkSearchAuthorsResponse) Reset() { *x = QueryBulkSearchAuthorsResponse{} - mi := &file_product_proto_msgTypes[85] + mi := &file_product_proto_msgTypes[86] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3876,7 +3921,7 @@ func (x *QueryBulkSearchAuthorsResponse) String() string { func (*QueryBulkSearchAuthorsResponse) ProtoMessage() {} func (x *QueryBulkSearchAuthorsResponse) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[85] + mi := &file_product_proto_msgTypes[86] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3889,7 +3934,7 @@ func (x *QueryBulkSearchAuthorsResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use QueryBulkSearchAuthorsResponse.ProtoReflect.Descriptor instead. func (*QueryBulkSearchAuthorsResponse) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{85} + return file_product_proto_rawDescGZIP(), []int{86} } func (x *QueryBulkSearchAuthorsResponse) GetBulkSearchAuthors() []*Author { @@ -3909,7 +3954,7 @@ type QueryBulkSearchBlogPostsRequest struct { func (x *QueryBulkSearchBlogPostsRequest) Reset() { *x = QueryBulkSearchBlogPostsRequest{} - mi := &file_product_proto_msgTypes[86] + mi := &file_product_proto_msgTypes[87] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3921,7 +3966,7 @@ func (x *QueryBulkSearchBlogPostsRequest) String() string { func (*QueryBulkSearchBlogPostsRequest) ProtoMessage() {} func (x *QueryBulkSearchBlogPostsRequest) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[86] + mi := &file_product_proto_msgTypes[87] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3934,7 +3979,7 @@ func (x *QueryBulkSearchBlogPostsRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use QueryBulkSearchBlogPostsRequest.ProtoReflect.Descriptor instead. func (*QueryBulkSearchBlogPostsRequest) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{86} + return file_product_proto_rawDescGZIP(), []int{87} } func (x *QueryBulkSearchBlogPostsRequest) GetFilters() *ListOfBlogPostFilter { @@ -3954,7 +3999,7 @@ type QueryBulkSearchBlogPostsResponse struct { func (x *QueryBulkSearchBlogPostsResponse) Reset() { *x = QueryBulkSearchBlogPostsResponse{} - mi := &file_product_proto_msgTypes[87] + mi := &file_product_proto_msgTypes[88] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3966,7 +4011,7 @@ func (x *QueryBulkSearchBlogPostsResponse) String() string { func (*QueryBulkSearchBlogPostsResponse) ProtoMessage() {} func (x *QueryBulkSearchBlogPostsResponse) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[87] + mi := &file_product_proto_msgTypes[88] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3979,7 +4024,7 @@ func (x *QueryBulkSearchBlogPostsResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use QueryBulkSearchBlogPostsResponse.ProtoReflect.Descriptor instead. func (*QueryBulkSearchBlogPostsResponse) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{87} + return file_product_proto_rawDescGZIP(), []int{88} } func (x *QueryBulkSearchBlogPostsResponse) GetBulkSearchBlogPosts() []*BlogPost { @@ -3999,7 +4044,7 @@ type MutationCreateUserRequest struct { func (x *MutationCreateUserRequest) Reset() { *x = MutationCreateUserRequest{} - mi := &file_product_proto_msgTypes[88] + mi := &file_product_proto_msgTypes[89] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -4011,7 +4056,7 @@ func (x *MutationCreateUserRequest) String() string { func (*MutationCreateUserRequest) ProtoMessage() {} func (x *MutationCreateUserRequest) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[88] + mi := &file_product_proto_msgTypes[89] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -4024,7 +4069,7 @@ func (x *MutationCreateUserRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use MutationCreateUserRequest.ProtoReflect.Descriptor instead. func (*MutationCreateUserRequest) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{88} + return file_product_proto_rawDescGZIP(), []int{89} } func (x *MutationCreateUserRequest) GetInput() *UserInput { @@ -4044,7 +4089,7 @@ type MutationCreateUserResponse struct { func (x *MutationCreateUserResponse) Reset() { *x = MutationCreateUserResponse{} - mi := &file_product_proto_msgTypes[89] + mi := &file_product_proto_msgTypes[90] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -4056,7 +4101,7 @@ func (x *MutationCreateUserResponse) String() string { func (*MutationCreateUserResponse) ProtoMessage() {} func (x *MutationCreateUserResponse) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[89] + mi := &file_product_proto_msgTypes[90] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -4069,7 +4114,7 @@ func (x *MutationCreateUserResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use MutationCreateUserResponse.ProtoReflect.Descriptor instead. func (*MutationCreateUserResponse) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{89} + return file_product_proto_rawDescGZIP(), []int{90} } func (x *MutationCreateUserResponse) GetCreateUser() *User { @@ -4089,7 +4134,7 @@ type MutationPerformActionRequest struct { func (x *MutationPerformActionRequest) Reset() { *x = MutationPerformActionRequest{} - mi := &file_product_proto_msgTypes[90] + mi := &file_product_proto_msgTypes[91] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -4101,7 +4146,7 @@ func (x *MutationPerformActionRequest) String() string { func (*MutationPerformActionRequest) ProtoMessage() {} func (x *MutationPerformActionRequest) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[90] + mi := &file_product_proto_msgTypes[91] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -4114,7 +4159,7 @@ func (x *MutationPerformActionRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use MutationPerformActionRequest.ProtoReflect.Descriptor instead. func (*MutationPerformActionRequest) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{90} + return file_product_proto_rawDescGZIP(), []int{91} } func (x *MutationPerformActionRequest) GetInput() *ActionInput { @@ -4134,7 +4179,7 @@ type MutationPerformActionResponse struct { func (x *MutationPerformActionResponse) Reset() { *x = MutationPerformActionResponse{} - mi := &file_product_proto_msgTypes[91] + mi := &file_product_proto_msgTypes[92] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -4146,7 +4191,7 @@ func (x *MutationPerformActionResponse) String() string { func (*MutationPerformActionResponse) ProtoMessage() {} func (x *MutationPerformActionResponse) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[91] + mi := &file_product_proto_msgTypes[92] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -4159,7 +4204,7 @@ func (x *MutationPerformActionResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use MutationPerformActionResponse.ProtoReflect.Descriptor instead. func (*MutationPerformActionResponse) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{91} + return file_product_proto_rawDescGZIP(), []int{92} } func (x *MutationPerformActionResponse) GetPerformAction() *ActionResult { @@ -4179,7 +4224,7 @@ type MutationCreateNullableFieldsTypeRequest struct { func (x *MutationCreateNullableFieldsTypeRequest) Reset() { *x = MutationCreateNullableFieldsTypeRequest{} - mi := &file_product_proto_msgTypes[92] + mi := &file_product_proto_msgTypes[93] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -4191,7 +4236,7 @@ func (x *MutationCreateNullableFieldsTypeRequest) String() string { func (*MutationCreateNullableFieldsTypeRequest) ProtoMessage() {} func (x *MutationCreateNullableFieldsTypeRequest) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[92] + mi := &file_product_proto_msgTypes[93] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -4204,7 +4249,7 @@ func (x *MutationCreateNullableFieldsTypeRequest) ProtoReflect() protoreflect.Me // Deprecated: Use MutationCreateNullableFieldsTypeRequest.ProtoReflect.Descriptor instead. func (*MutationCreateNullableFieldsTypeRequest) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{92} + return file_product_proto_rawDescGZIP(), []int{93} } func (x *MutationCreateNullableFieldsTypeRequest) GetInput() *NullableFieldsInput { @@ -4224,7 +4269,7 @@ type MutationCreateNullableFieldsTypeResponse struct { func (x *MutationCreateNullableFieldsTypeResponse) Reset() { *x = MutationCreateNullableFieldsTypeResponse{} - mi := &file_product_proto_msgTypes[93] + mi := &file_product_proto_msgTypes[94] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -4236,7 +4281,7 @@ func (x *MutationCreateNullableFieldsTypeResponse) String() string { func (*MutationCreateNullableFieldsTypeResponse) ProtoMessage() {} func (x *MutationCreateNullableFieldsTypeResponse) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[93] + mi := &file_product_proto_msgTypes[94] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -4249,7 +4294,7 @@ func (x *MutationCreateNullableFieldsTypeResponse) ProtoReflect() protoreflect.M // Deprecated: Use MutationCreateNullableFieldsTypeResponse.ProtoReflect.Descriptor instead. func (*MutationCreateNullableFieldsTypeResponse) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{93} + return file_product_proto_rawDescGZIP(), []int{94} } func (x *MutationCreateNullableFieldsTypeResponse) GetCreateNullableFieldsType() *NullableFieldsType { @@ -4270,7 +4315,7 @@ type MutationUpdateNullableFieldsTypeRequest struct { func (x *MutationUpdateNullableFieldsTypeRequest) Reset() { *x = MutationUpdateNullableFieldsTypeRequest{} - mi := &file_product_proto_msgTypes[94] + mi := &file_product_proto_msgTypes[95] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -4282,7 +4327,7 @@ func (x *MutationUpdateNullableFieldsTypeRequest) String() string { func (*MutationUpdateNullableFieldsTypeRequest) ProtoMessage() {} func (x *MutationUpdateNullableFieldsTypeRequest) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[94] + mi := &file_product_proto_msgTypes[95] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -4295,7 +4340,7 @@ func (x *MutationUpdateNullableFieldsTypeRequest) ProtoReflect() protoreflect.Me // Deprecated: Use MutationUpdateNullableFieldsTypeRequest.ProtoReflect.Descriptor instead. func (*MutationUpdateNullableFieldsTypeRequest) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{94} + return file_product_proto_rawDescGZIP(), []int{95} } func (x *MutationUpdateNullableFieldsTypeRequest) GetId() string { @@ -4322,7 +4367,7 @@ type MutationUpdateNullableFieldsTypeResponse struct { func (x *MutationUpdateNullableFieldsTypeResponse) Reset() { *x = MutationUpdateNullableFieldsTypeResponse{} - mi := &file_product_proto_msgTypes[95] + mi := &file_product_proto_msgTypes[96] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -4334,7 +4379,7 @@ func (x *MutationUpdateNullableFieldsTypeResponse) String() string { func (*MutationUpdateNullableFieldsTypeResponse) ProtoMessage() {} func (x *MutationUpdateNullableFieldsTypeResponse) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[95] + mi := &file_product_proto_msgTypes[96] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -4347,7 +4392,7 @@ func (x *MutationUpdateNullableFieldsTypeResponse) ProtoReflect() protoreflect.M // Deprecated: Use MutationUpdateNullableFieldsTypeResponse.ProtoReflect.Descriptor instead. func (*MutationUpdateNullableFieldsTypeResponse) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{95} + return file_product_proto_rawDescGZIP(), []int{96} } func (x *MutationUpdateNullableFieldsTypeResponse) GetUpdateNullableFieldsType() *NullableFieldsType { @@ -4367,7 +4412,7 @@ type MutationCreateBlogPostRequest struct { func (x *MutationCreateBlogPostRequest) Reset() { *x = MutationCreateBlogPostRequest{} - mi := &file_product_proto_msgTypes[96] + mi := &file_product_proto_msgTypes[97] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -4379,7 +4424,7 @@ func (x *MutationCreateBlogPostRequest) String() string { func (*MutationCreateBlogPostRequest) ProtoMessage() {} func (x *MutationCreateBlogPostRequest) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[96] + mi := &file_product_proto_msgTypes[97] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -4392,7 +4437,7 @@ func (x *MutationCreateBlogPostRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use MutationCreateBlogPostRequest.ProtoReflect.Descriptor instead. func (*MutationCreateBlogPostRequest) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{96} + return file_product_proto_rawDescGZIP(), []int{97} } func (x *MutationCreateBlogPostRequest) GetInput() *BlogPostInput { @@ -4412,7 +4457,7 @@ type MutationCreateBlogPostResponse struct { func (x *MutationCreateBlogPostResponse) Reset() { *x = MutationCreateBlogPostResponse{} - mi := &file_product_proto_msgTypes[97] + mi := &file_product_proto_msgTypes[98] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -4424,7 +4469,7 @@ func (x *MutationCreateBlogPostResponse) String() string { func (*MutationCreateBlogPostResponse) ProtoMessage() {} func (x *MutationCreateBlogPostResponse) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[97] + mi := &file_product_proto_msgTypes[98] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -4437,7 +4482,7 @@ func (x *MutationCreateBlogPostResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use MutationCreateBlogPostResponse.ProtoReflect.Descriptor instead. func (*MutationCreateBlogPostResponse) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{97} + return file_product_proto_rawDescGZIP(), []int{98} } func (x *MutationCreateBlogPostResponse) GetCreateBlogPost() *BlogPost { @@ -4458,7 +4503,7 @@ type MutationUpdateBlogPostRequest struct { func (x *MutationUpdateBlogPostRequest) Reset() { *x = MutationUpdateBlogPostRequest{} - mi := &file_product_proto_msgTypes[98] + mi := &file_product_proto_msgTypes[99] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -4470,7 +4515,7 @@ func (x *MutationUpdateBlogPostRequest) String() string { func (*MutationUpdateBlogPostRequest) ProtoMessage() {} func (x *MutationUpdateBlogPostRequest) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[98] + mi := &file_product_proto_msgTypes[99] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -4483,7 +4528,7 @@ func (x *MutationUpdateBlogPostRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use MutationUpdateBlogPostRequest.ProtoReflect.Descriptor instead. func (*MutationUpdateBlogPostRequest) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{98} + return file_product_proto_rawDescGZIP(), []int{99} } func (x *MutationUpdateBlogPostRequest) GetId() string { @@ -4510,7 +4555,7 @@ type MutationUpdateBlogPostResponse struct { func (x *MutationUpdateBlogPostResponse) Reset() { *x = MutationUpdateBlogPostResponse{} - mi := &file_product_proto_msgTypes[99] + mi := &file_product_proto_msgTypes[100] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -4522,7 +4567,7 @@ func (x *MutationUpdateBlogPostResponse) String() string { func (*MutationUpdateBlogPostResponse) ProtoMessage() {} func (x *MutationUpdateBlogPostResponse) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[99] + mi := &file_product_proto_msgTypes[100] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -4535,7 +4580,7 @@ func (x *MutationUpdateBlogPostResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use MutationUpdateBlogPostResponse.ProtoReflect.Descriptor instead. func (*MutationUpdateBlogPostResponse) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{99} + return file_product_proto_rawDescGZIP(), []int{100} } func (x *MutationUpdateBlogPostResponse) GetUpdateBlogPost() *BlogPost { @@ -4555,7 +4600,7 @@ type MutationCreateAuthorRequest struct { func (x *MutationCreateAuthorRequest) Reset() { *x = MutationCreateAuthorRequest{} - mi := &file_product_proto_msgTypes[100] + mi := &file_product_proto_msgTypes[101] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -4567,7 +4612,7 @@ func (x *MutationCreateAuthorRequest) String() string { func (*MutationCreateAuthorRequest) ProtoMessage() {} func (x *MutationCreateAuthorRequest) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[100] + mi := &file_product_proto_msgTypes[101] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -4580,7 +4625,7 @@ func (x *MutationCreateAuthorRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use MutationCreateAuthorRequest.ProtoReflect.Descriptor instead. func (*MutationCreateAuthorRequest) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{100} + return file_product_proto_rawDescGZIP(), []int{101} } func (x *MutationCreateAuthorRequest) GetInput() *AuthorInput { @@ -4600,7 +4645,7 @@ type MutationCreateAuthorResponse struct { func (x *MutationCreateAuthorResponse) Reset() { *x = MutationCreateAuthorResponse{} - mi := &file_product_proto_msgTypes[101] + mi := &file_product_proto_msgTypes[102] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -4612,7 +4657,7 @@ func (x *MutationCreateAuthorResponse) String() string { func (*MutationCreateAuthorResponse) ProtoMessage() {} func (x *MutationCreateAuthorResponse) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[101] + mi := &file_product_proto_msgTypes[102] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -4625,7 +4670,7 @@ func (x *MutationCreateAuthorResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use MutationCreateAuthorResponse.ProtoReflect.Descriptor instead. func (*MutationCreateAuthorResponse) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{101} + return file_product_proto_rawDescGZIP(), []int{102} } func (x *MutationCreateAuthorResponse) GetCreateAuthor() *Author { @@ -4646,7 +4691,7 @@ type MutationUpdateAuthorRequest struct { func (x *MutationUpdateAuthorRequest) Reset() { *x = MutationUpdateAuthorRequest{} - mi := &file_product_proto_msgTypes[102] + mi := &file_product_proto_msgTypes[103] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -4658,7 +4703,7 @@ func (x *MutationUpdateAuthorRequest) String() string { func (*MutationUpdateAuthorRequest) ProtoMessage() {} func (x *MutationUpdateAuthorRequest) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[102] + mi := &file_product_proto_msgTypes[103] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -4671,7 +4716,7 @@ func (x *MutationUpdateAuthorRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use MutationUpdateAuthorRequest.ProtoReflect.Descriptor instead. func (*MutationUpdateAuthorRequest) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{102} + return file_product_proto_rawDescGZIP(), []int{103} } func (x *MutationUpdateAuthorRequest) GetId() string { @@ -4698,7 +4743,7 @@ type MutationUpdateAuthorResponse struct { func (x *MutationUpdateAuthorResponse) Reset() { *x = MutationUpdateAuthorResponse{} - mi := &file_product_proto_msgTypes[103] + mi := &file_product_proto_msgTypes[104] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -4710,7 +4755,7 @@ func (x *MutationUpdateAuthorResponse) String() string { func (*MutationUpdateAuthorResponse) ProtoMessage() {} func (x *MutationUpdateAuthorResponse) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[103] + mi := &file_product_proto_msgTypes[104] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -4723,7 +4768,7 @@ func (x *MutationUpdateAuthorResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use MutationUpdateAuthorResponse.ProtoReflect.Descriptor instead. func (*MutationUpdateAuthorResponse) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{103} + return file_product_proto_rawDescGZIP(), []int{104} } func (x *MutationUpdateAuthorResponse) GetUpdateAuthor() *Author { @@ -4743,7 +4788,7 @@ type MutationBulkCreateAuthorsRequest struct { func (x *MutationBulkCreateAuthorsRequest) Reset() { *x = MutationBulkCreateAuthorsRequest{} - mi := &file_product_proto_msgTypes[104] + mi := &file_product_proto_msgTypes[105] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -4755,7 +4800,7 @@ func (x *MutationBulkCreateAuthorsRequest) String() string { func (*MutationBulkCreateAuthorsRequest) ProtoMessage() {} func (x *MutationBulkCreateAuthorsRequest) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[104] + mi := &file_product_proto_msgTypes[105] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -4768,7 +4813,7 @@ func (x *MutationBulkCreateAuthorsRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use MutationBulkCreateAuthorsRequest.ProtoReflect.Descriptor instead. func (*MutationBulkCreateAuthorsRequest) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{104} + return file_product_proto_rawDescGZIP(), []int{105} } func (x *MutationBulkCreateAuthorsRequest) GetAuthors() *ListOfAuthorInput { @@ -4788,7 +4833,7 @@ type MutationBulkCreateAuthorsResponse struct { func (x *MutationBulkCreateAuthorsResponse) Reset() { *x = MutationBulkCreateAuthorsResponse{} - mi := &file_product_proto_msgTypes[105] + mi := &file_product_proto_msgTypes[106] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -4800,7 +4845,7 @@ func (x *MutationBulkCreateAuthorsResponse) String() string { func (*MutationBulkCreateAuthorsResponse) ProtoMessage() {} func (x *MutationBulkCreateAuthorsResponse) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[105] + mi := &file_product_proto_msgTypes[106] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -4813,7 +4858,7 @@ func (x *MutationBulkCreateAuthorsResponse) ProtoReflect() protoreflect.Message // Deprecated: Use MutationBulkCreateAuthorsResponse.ProtoReflect.Descriptor instead. func (*MutationBulkCreateAuthorsResponse) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{105} + return file_product_proto_rawDescGZIP(), []int{106} } func (x *MutationBulkCreateAuthorsResponse) GetBulkCreateAuthors() []*Author { @@ -4833,7 +4878,7 @@ type MutationBulkUpdateAuthorsRequest struct { func (x *MutationBulkUpdateAuthorsRequest) Reset() { *x = MutationBulkUpdateAuthorsRequest{} - mi := &file_product_proto_msgTypes[106] + mi := &file_product_proto_msgTypes[107] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -4845,7 +4890,7 @@ func (x *MutationBulkUpdateAuthorsRequest) String() string { func (*MutationBulkUpdateAuthorsRequest) ProtoMessage() {} func (x *MutationBulkUpdateAuthorsRequest) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[106] + mi := &file_product_proto_msgTypes[107] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -4858,7 +4903,7 @@ func (x *MutationBulkUpdateAuthorsRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use MutationBulkUpdateAuthorsRequest.ProtoReflect.Descriptor instead. func (*MutationBulkUpdateAuthorsRequest) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{106} + return file_product_proto_rawDescGZIP(), []int{107} } func (x *MutationBulkUpdateAuthorsRequest) GetAuthors() *ListOfAuthorInput { @@ -4878,7 +4923,7 @@ type MutationBulkUpdateAuthorsResponse struct { func (x *MutationBulkUpdateAuthorsResponse) Reset() { *x = MutationBulkUpdateAuthorsResponse{} - mi := &file_product_proto_msgTypes[107] + mi := &file_product_proto_msgTypes[108] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -4890,7 +4935,7 @@ func (x *MutationBulkUpdateAuthorsResponse) String() string { func (*MutationBulkUpdateAuthorsResponse) ProtoMessage() {} func (x *MutationBulkUpdateAuthorsResponse) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[107] + mi := &file_product_proto_msgTypes[108] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -4903,7 +4948,7 @@ func (x *MutationBulkUpdateAuthorsResponse) ProtoReflect() protoreflect.Message // Deprecated: Use MutationBulkUpdateAuthorsResponse.ProtoReflect.Descriptor instead. func (*MutationBulkUpdateAuthorsResponse) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{107} + return file_product_proto_rawDescGZIP(), []int{108} } func (x *MutationBulkUpdateAuthorsResponse) GetBulkUpdateAuthors() []*Author { @@ -4923,7 +4968,7 @@ type MutationBulkCreateBlogPostsRequest struct { func (x *MutationBulkCreateBlogPostsRequest) Reset() { *x = MutationBulkCreateBlogPostsRequest{} - mi := &file_product_proto_msgTypes[108] + mi := &file_product_proto_msgTypes[109] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -4935,7 +4980,7 @@ func (x *MutationBulkCreateBlogPostsRequest) String() string { func (*MutationBulkCreateBlogPostsRequest) ProtoMessage() {} func (x *MutationBulkCreateBlogPostsRequest) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[108] + mi := &file_product_proto_msgTypes[109] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -4948,7 +4993,7 @@ func (x *MutationBulkCreateBlogPostsRequest) ProtoReflect() protoreflect.Message // Deprecated: Use MutationBulkCreateBlogPostsRequest.ProtoReflect.Descriptor instead. func (*MutationBulkCreateBlogPostsRequest) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{108} + return file_product_proto_rawDescGZIP(), []int{109} } func (x *MutationBulkCreateBlogPostsRequest) GetBlogPosts() *ListOfBlogPostInput { @@ -4968,7 +5013,7 @@ type MutationBulkCreateBlogPostsResponse struct { func (x *MutationBulkCreateBlogPostsResponse) Reset() { *x = MutationBulkCreateBlogPostsResponse{} - mi := &file_product_proto_msgTypes[109] + mi := &file_product_proto_msgTypes[110] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -4980,7 +5025,7 @@ func (x *MutationBulkCreateBlogPostsResponse) String() string { func (*MutationBulkCreateBlogPostsResponse) ProtoMessage() {} func (x *MutationBulkCreateBlogPostsResponse) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[109] + mi := &file_product_proto_msgTypes[110] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -4993,7 +5038,7 @@ func (x *MutationBulkCreateBlogPostsResponse) ProtoReflect() protoreflect.Messag // Deprecated: Use MutationBulkCreateBlogPostsResponse.ProtoReflect.Descriptor instead. func (*MutationBulkCreateBlogPostsResponse) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{109} + return file_product_proto_rawDescGZIP(), []int{110} } func (x *MutationBulkCreateBlogPostsResponse) GetBulkCreateBlogPosts() []*BlogPost { @@ -5013,7 +5058,7 @@ type MutationBulkUpdateBlogPostsRequest struct { func (x *MutationBulkUpdateBlogPostsRequest) Reset() { *x = MutationBulkUpdateBlogPostsRequest{} - mi := &file_product_proto_msgTypes[110] + mi := &file_product_proto_msgTypes[111] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5025,7 +5070,7 @@ func (x *MutationBulkUpdateBlogPostsRequest) String() string { func (*MutationBulkUpdateBlogPostsRequest) ProtoMessage() {} func (x *MutationBulkUpdateBlogPostsRequest) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[110] + mi := &file_product_proto_msgTypes[111] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5038,7 +5083,7 @@ func (x *MutationBulkUpdateBlogPostsRequest) ProtoReflect() protoreflect.Message // Deprecated: Use MutationBulkUpdateBlogPostsRequest.ProtoReflect.Descriptor instead. func (*MutationBulkUpdateBlogPostsRequest) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{110} + return file_product_proto_rawDescGZIP(), []int{111} } func (x *MutationBulkUpdateBlogPostsRequest) GetBlogPosts() *ListOfBlogPostInput { @@ -5058,7 +5103,7 @@ type MutationBulkUpdateBlogPostsResponse struct { func (x *MutationBulkUpdateBlogPostsResponse) Reset() { *x = MutationBulkUpdateBlogPostsResponse{} - mi := &file_product_proto_msgTypes[111] + mi := &file_product_proto_msgTypes[112] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5070,7 +5115,7 @@ func (x *MutationBulkUpdateBlogPostsResponse) String() string { func (*MutationBulkUpdateBlogPostsResponse) ProtoMessage() {} func (x *MutationBulkUpdateBlogPostsResponse) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[111] + mi := &file_product_proto_msgTypes[112] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5083,7 +5128,7 @@ func (x *MutationBulkUpdateBlogPostsResponse) ProtoReflect() protoreflect.Messag // Deprecated: Use MutationBulkUpdateBlogPostsResponse.ProtoReflect.Descriptor instead. func (*MutationBulkUpdateBlogPostsResponse) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{111} + return file_product_proto_rawDescGZIP(), []int{112} } func (x *MutationBulkUpdateBlogPostsResponse) GetBulkUpdateBlogPosts() []*BlogPost { @@ -5093,28 +5138,266 @@ func (x *MutationBulkUpdateBlogPostsResponse) GetBulkUpdateBlogPosts() []*BlogPo return nil } -type CategoryProductCountArgs struct { +type ResolveCategoryProductCountArgs struct { + state protoimpl.MessageState `protogen:"open.v1"` + Filters *ProductCountFilter `protobuf:"bytes,1,opt,name=filters,proto3" json:"filters,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ResolveCategoryProductCountArgs) Reset() { + *x = ResolveCategoryProductCountArgs{} + mi := &file_product_proto_msgTypes[113] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ResolveCategoryProductCountArgs) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ResolveCategoryProductCountArgs) ProtoMessage() {} + +func (x *ResolveCategoryProductCountArgs) ProtoReflect() protoreflect.Message { + mi := &file_product_proto_msgTypes[113] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ResolveCategoryProductCountArgs.ProtoReflect.Descriptor instead. +func (*ResolveCategoryProductCountArgs) Descriptor() ([]byte, []int) { + return file_product_proto_rawDescGZIP(), []int{113} +} + +func (x *ResolveCategoryProductCountArgs) GetFilters() *ProductCountFilter { + if x != nil { + return x.Filters + } + return nil +} + +type ResolveCategoryProductCountContext struct { + state protoimpl.MessageState `protogen:"open.v1"` + Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` + Name string `protobuf:"bytes,2,opt,name=name,proto3" json:"name,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ResolveCategoryProductCountContext) Reset() { + *x = ResolveCategoryProductCountContext{} + mi := &file_product_proto_msgTypes[114] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ResolveCategoryProductCountContext) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ResolveCategoryProductCountContext) ProtoMessage() {} + +func (x *ResolveCategoryProductCountContext) ProtoReflect() protoreflect.Message { + mi := &file_product_proto_msgTypes[114] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ResolveCategoryProductCountContext.ProtoReflect.Descriptor instead. +func (*ResolveCategoryProductCountContext) Descriptor() ([]byte, []int) { + return file_product_proto_rawDescGZIP(), []int{114} +} + +func (x *ResolveCategoryProductCountContext) GetId() string { + if x != nil { + return x.Id + } + return "" +} + +func (x *ResolveCategoryProductCountContext) GetName() string { + if x != nil { + return x.Name + } + return "" +} + +type ResolveCategoryProductCountRequest struct { + state protoimpl.MessageState `protogen:"open.v1"` + // context provides the resolver context for the field productCount of type Category. + Context []*ResolveCategoryProductCountContext `protobuf:"bytes,1,rep,name=context,proto3" json:"context,omitempty"` + // field_args provides the arguments for the resolver field productCount of type Category. + FieldArgs *ResolveCategoryProductCountArgs `protobuf:"bytes,2,opt,name=field_args,json=fieldArgs,proto3" json:"field_args,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ResolveCategoryProductCountRequest) Reset() { + *x = ResolveCategoryProductCountRequest{} + mi := &file_product_proto_msgTypes[115] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ResolveCategoryProductCountRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ResolveCategoryProductCountRequest) ProtoMessage() {} + +func (x *ResolveCategoryProductCountRequest) ProtoReflect() protoreflect.Message { + mi := &file_product_proto_msgTypes[115] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ResolveCategoryProductCountRequest.ProtoReflect.Descriptor instead. +func (*ResolveCategoryProductCountRequest) Descriptor() ([]byte, []int) { + return file_product_proto_rawDescGZIP(), []int{115} +} + +func (x *ResolveCategoryProductCountRequest) GetContext() []*ResolveCategoryProductCountContext { + if x != nil { + return x.Context + } + return nil +} + +func (x *ResolveCategoryProductCountRequest) GetFieldArgs() *ResolveCategoryProductCountArgs { + if x != nil { + return x.FieldArgs + } + return nil +} + +type ResolveCategoryProductCountResult struct { + state protoimpl.MessageState `protogen:"open.v1"` + ProductCount int32 `protobuf:"varint,1,opt,name=product_count,json=productCount,proto3" json:"product_count,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ResolveCategoryProductCountResult) Reset() { + *x = ResolveCategoryProductCountResult{} + mi := &file_product_proto_msgTypes[116] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ResolveCategoryProductCountResult) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ResolveCategoryProductCountResult) ProtoMessage() {} + +func (x *ResolveCategoryProductCountResult) ProtoReflect() protoreflect.Message { + mi := &file_product_proto_msgTypes[116] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ResolveCategoryProductCountResult.ProtoReflect.Descriptor instead. +func (*ResolveCategoryProductCountResult) Descriptor() ([]byte, []int) { + return file_product_proto_rawDescGZIP(), []int{116} +} + +func (x *ResolveCategoryProductCountResult) GetProductCount() int32 { + if x != nil { + return x.ProductCount + } + return 0 +} + +type ResolveCategoryProductCountResponse struct { + state protoimpl.MessageState `protogen:"open.v1"` + Result []*ResolveCategoryProductCountResult `protobuf:"bytes,1,rep,name=result,proto3" json:"result,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ResolveCategoryProductCountResponse) Reset() { + *x = ResolveCategoryProductCountResponse{} + mi := &file_product_proto_msgTypes[117] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ResolveCategoryProductCountResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ResolveCategoryProductCountResponse) ProtoMessage() {} + +func (x *ResolveCategoryProductCountResponse) ProtoReflect() protoreflect.Message { + mi := &file_product_proto_msgTypes[117] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ResolveCategoryProductCountResponse.ProtoReflect.Descriptor instead. +func (*ResolveCategoryProductCountResponse) Descriptor() ([]byte, []int) { + return file_product_proto_rawDescGZIP(), []int{117} +} + +func (x *ResolveCategoryProductCountResponse) GetResult() []*ResolveCategoryProductCountResult { + if x != nil { + return x.Result + } + return nil +} + +type ResolveSubcategoryItemCountArgs struct { state protoimpl.MessageState `protogen:"open.v1"` - Filters *ProductCountFilter `protobuf:"bytes,1,opt,name=filters,proto3" json:"filters,omitempty"` + Filters *SubcategoryItemFilter `protobuf:"bytes,1,opt,name=filters,proto3" json:"filters,omitempty"` unknownFields protoimpl.UnknownFields sizeCache protoimpl.SizeCache } -func (x *CategoryProductCountArgs) Reset() { - *x = CategoryProductCountArgs{} - mi := &file_product_proto_msgTypes[112] +func (x *ResolveSubcategoryItemCountArgs) Reset() { + *x = ResolveSubcategoryItemCountArgs{} + mi := &file_product_proto_msgTypes[118] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } -func (x *CategoryProductCountArgs) String() string { +func (x *ResolveSubcategoryItemCountArgs) String() string { return protoimpl.X.MessageStringOf(x) } -func (*CategoryProductCountArgs) ProtoMessage() {} +func (*ResolveSubcategoryItemCountArgs) ProtoMessage() {} -func (x *CategoryProductCountArgs) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[112] +func (x *ResolveSubcategoryItemCountArgs) ProtoReflect() protoreflect.Message { + mi := &file_product_proto_msgTypes[118] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5125,41 +5408,40 @@ func (x *CategoryProductCountArgs) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use CategoryProductCountArgs.ProtoReflect.Descriptor instead. -func (*CategoryProductCountArgs) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{112} +// Deprecated: Use ResolveSubcategoryItemCountArgs.ProtoReflect.Descriptor instead. +func (*ResolveSubcategoryItemCountArgs) Descriptor() ([]byte, []int) { + return file_product_proto_rawDescGZIP(), []int{118} } -func (x *CategoryProductCountArgs) GetFilters() *ProductCountFilter { +func (x *ResolveSubcategoryItemCountArgs) GetFilters() *SubcategoryItemFilter { if x != nil { return x.Filters } return nil } -type CategoryProductCountContext struct { +type ResolveSubcategoryItemCountContext struct { state protoimpl.MessageState `protogen:"open.v1"` Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` - Name string `protobuf:"bytes,2,opt,name=name,proto3" json:"name,omitempty"` unknownFields protoimpl.UnknownFields sizeCache protoimpl.SizeCache } -func (x *CategoryProductCountContext) Reset() { - *x = CategoryProductCountContext{} - mi := &file_product_proto_msgTypes[113] +func (x *ResolveSubcategoryItemCountContext) Reset() { + *x = ResolveSubcategoryItemCountContext{} + mi := &file_product_proto_msgTypes[119] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } -func (x *CategoryProductCountContext) String() string { +func (x *ResolveSubcategoryItemCountContext) String() string { return protoimpl.X.MessageStringOf(x) } -func (*CategoryProductCountContext) ProtoMessage() {} +func (*ResolveSubcategoryItemCountContext) ProtoMessage() {} -func (x *CategoryProductCountContext) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[113] +func (x *ResolveSubcategoryItemCountContext) ProtoReflect() protoreflect.Message { + mi := &file_product_proto_msgTypes[119] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5170,50 +5452,43 @@ func (x *CategoryProductCountContext) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use CategoryProductCountContext.ProtoReflect.Descriptor instead. -func (*CategoryProductCountContext) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{113} +// Deprecated: Use ResolveSubcategoryItemCountContext.ProtoReflect.Descriptor instead. +func (*ResolveSubcategoryItemCountContext) Descriptor() ([]byte, []int) { + return file_product_proto_rawDescGZIP(), []int{119} } -func (x *CategoryProductCountContext) GetId() string { +func (x *ResolveSubcategoryItemCountContext) GetId() string { if x != nil { return x.Id } return "" } -func (x *CategoryProductCountContext) GetName() string { - if x != nil { - return x.Name - } - return "" -} - -type ResolveCategoryProductCountRequest struct { +type ResolveSubcategoryItemCountRequest struct { state protoimpl.MessageState `protogen:"open.v1"` - // context provides the resolver context for the field productCount of type Category. - Context []*CategoryProductCountContext `protobuf:"bytes,1,rep,name=context,proto3" json:"context,omitempty"` - // field_args provides the arguments for the resolver field productCount of type Category. - FieldArgs *CategoryProductCountArgs `protobuf:"bytes,2,opt,name=field_args,json=fieldArgs,proto3" json:"field_args,omitempty"` + // context provides the resolver context for the field itemCount of type Subcategory. + Context []*ResolveSubcategoryItemCountContext `protobuf:"bytes,1,rep,name=context,proto3" json:"context,omitempty"` + // field_args provides the arguments for the resolver field itemCount of type Subcategory. + FieldArgs *ResolveSubcategoryItemCountArgs `protobuf:"bytes,2,opt,name=field_args,json=fieldArgs,proto3" json:"field_args,omitempty"` unknownFields protoimpl.UnknownFields sizeCache protoimpl.SizeCache } -func (x *ResolveCategoryProductCountRequest) Reset() { - *x = ResolveCategoryProductCountRequest{} - mi := &file_product_proto_msgTypes[114] +func (x *ResolveSubcategoryItemCountRequest) Reset() { + *x = ResolveSubcategoryItemCountRequest{} + mi := &file_product_proto_msgTypes[120] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } -func (x *ResolveCategoryProductCountRequest) String() string { +func (x *ResolveSubcategoryItemCountRequest) String() string { return protoimpl.X.MessageStringOf(x) } -func (*ResolveCategoryProductCountRequest) ProtoMessage() {} +func (*ResolveSubcategoryItemCountRequest) ProtoMessage() {} -func (x *ResolveCategoryProductCountRequest) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[114] +func (x *ResolveSubcategoryItemCountRequest) ProtoReflect() protoreflect.Message { + mi := &file_product_proto_msgTypes[120] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5224,47 +5499,47 @@ func (x *ResolveCategoryProductCountRequest) ProtoReflect() protoreflect.Message return mi.MessageOf(x) } -// Deprecated: Use ResolveCategoryProductCountRequest.ProtoReflect.Descriptor instead. -func (*ResolveCategoryProductCountRequest) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{114} +// Deprecated: Use ResolveSubcategoryItemCountRequest.ProtoReflect.Descriptor instead. +func (*ResolveSubcategoryItemCountRequest) Descriptor() ([]byte, []int) { + return file_product_proto_rawDescGZIP(), []int{120} } -func (x *ResolveCategoryProductCountRequest) GetContext() []*CategoryProductCountContext { +func (x *ResolveSubcategoryItemCountRequest) GetContext() []*ResolveSubcategoryItemCountContext { if x != nil { return x.Context } return nil } -func (x *ResolveCategoryProductCountRequest) GetFieldArgs() *CategoryProductCountArgs { +func (x *ResolveSubcategoryItemCountRequest) GetFieldArgs() *ResolveSubcategoryItemCountArgs { if x != nil { return x.FieldArgs } return nil } -type ResolveCategoryProductCountResponseResult struct { +type ResolveSubcategoryItemCountResult struct { state protoimpl.MessageState `protogen:"open.v1"` - ProductCount int32 `protobuf:"varint,1,opt,name=product_count,json=productCount,proto3" json:"product_count,omitempty"` + ItemCount int32 `protobuf:"varint,1,opt,name=item_count,json=itemCount,proto3" json:"item_count,omitempty"` unknownFields protoimpl.UnknownFields sizeCache protoimpl.SizeCache } -func (x *ResolveCategoryProductCountResponseResult) Reset() { - *x = ResolveCategoryProductCountResponseResult{} - mi := &file_product_proto_msgTypes[115] +func (x *ResolveSubcategoryItemCountResult) Reset() { + *x = ResolveSubcategoryItemCountResult{} + mi := &file_product_proto_msgTypes[121] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } -func (x *ResolveCategoryProductCountResponseResult) String() string { +func (x *ResolveSubcategoryItemCountResult) String() string { return protoimpl.X.MessageStringOf(x) } -func (*ResolveCategoryProductCountResponseResult) ProtoMessage() {} +func (*ResolveSubcategoryItemCountResult) ProtoMessage() {} -func (x *ResolveCategoryProductCountResponseResult) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[115] +func (x *ResolveSubcategoryItemCountResult) ProtoReflect() protoreflect.Message { + mi := &file_product_proto_msgTypes[121] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5275,40 +5550,40 @@ func (x *ResolveCategoryProductCountResponseResult) ProtoReflect() protoreflect. return mi.MessageOf(x) } -// Deprecated: Use ResolveCategoryProductCountResponseResult.ProtoReflect.Descriptor instead. -func (*ResolveCategoryProductCountResponseResult) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{115} +// Deprecated: Use ResolveSubcategoryItemCountResult.ProtoReflect.Descriptor instead. +func (*ResolveSubcategoryItemCountResult) Descriptor() ([]byte, []int) { + return file_product_proto_rawDescGZIP(), []int{121} } -func (x *ResolveCategoryProductCountResponseResult) GetProductCount() int32 { +func (x *ResolveSubcategoryItemCountResult) GetItemCount() int32 { if x != nil { - return x.ProductCount + return x.ItemCount } return 0 } -type ResolveCategoryProductCountResponse struct { - state protoimpl.MessageState `protogen:"open.v1"` - Result []*ResolveCategoryProductCountResponseResult `protobuf:"bytes,1,rep,name=result,proto3" json:"result,omitempty"` +type ResolveSubcategoryItemCountResponse struct { + state protoimpl.MessageState `protogen:"open.v1"` + Result []*ResolveSubcategoryItemCountResult `protobuf:"bytes,1,rep,name=result,proto3" json:"result,omitempty"` unknownFields protoimpl.UnknownFields sizeCache protoimpl.SizeCache } -func (x *ResolveCategoryProductCountResponse) Reset() { - *x = ResolveCategoryProductCountResponse{} - mi := &file_product_proto_msgTypes[116] +func (x *ResolveSubcategoryItemCountResponse) Reset() { + *x = ResolveSubcategoryItemCountResponse{} + mi := &file_product_proto_msgTypes[122] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } -func (x *ResolveCategoryProductCountResponse) String() string { +func (x *ResolveSubcategoryItemCountResponse) String() string { return protoimpl.X.MessageStringOf(x) } -func (*ResolveCategoryProductCountResponse) ProtoMessage() {} +func (*ResolveSubcategoryItemCountResponse) ProtoMessage() {} -func (x *ResolveCategoryProductCountResponse) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[116] +func (x *ResolveSubcategoryItemCountResponse) ProtoReflect() protoreflect.Message { + mi := &file_product_proto_msgTypes[122] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5319,12 +5594,12 @@ func (x *ResolveCategoryProductCountResponse) ProtoReflect() protoreflect.Messag return mi.MessageOf(x) } -// Deprecated: Use ResolveCategoryProductCountResponse.ProtoReflect.Descriptor instead. -func (*ResolveCategoryProductCountResponse) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{116} +// Deprecated: Use ResolveSubcategoryItemCountResponse.ProtoReflect.Descriptor instead. +func (*ResolveSubcategoryItemCountResponse) Descriptor() ([]byte, []int) { + return file_product_proto_rawDescGZIP(), []int{122} } -func (x *ResolveCategoryProductCountResponse) GetResult() []*ResolveCategoryProductCountResponseResult { +func (x *ResolveSubcategoryItemCountResponse) GetResult() []*ResolveSubcategoryItemCountResult { if x != nil { return x.Result } @@ -5342,7 +5617,7 @@ type Product struct { func (x *Product) Reset() { *x = Product{} - mi := &file_product_proto_msgTypes[117] + mi := &file_product_proto_msgTypes[123] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5354,7 +5629,7 @@ func (x *Product) String() string { func (*Product) ProtoMessage() {} func (x *Product) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[117] + mi := &file_product_proto_msgTypes[123] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5367,7 +5642,7 @@ func (x *Product) ProtoReflect() protoreflect.Message { // Deprecated: Use Product.ProtoReflect.Descriptor instead. func (*Product) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{117} + return file_product_proto_rawDescGZIP(), []int{123} } func (x *Product) GetId() string { @@ -5402,7 +5677,7 @@ type Storage struct { func (x *Storage) Reset() { *x = Storage{} - mi := &file_product_proto_msgTypes[118] + mi := &file_product_proto_msgTypes[124] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5414,7 +5689,7 @@ func (x *Storage) String() string { func (*Storage) ProtoMessage() {} func (x *Storage) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[118] + mi := &file_product_proto_msgTypes[124] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5427,7 +5702,7 @@ func (x *Storage) ProtoReflect() protoreflect.Message { // Deprecated: Use Storage.ProtoReflect.Descriptor instead. func (*Storage) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{118} + return file_product_proto_rawDescGZIP(), []int{124} } func (x *Storage) GetId() string { @@ -5462,7 +5737,7 @@ type Warehouse struct { func (x *Warehouse) Reset() { *x = Warehouse{} - mi := &file_product_proto_msgTypes[119] + mi := &file_product_proto_msgTypes[125] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5474,7 +5749,7 @@ func (x *Warehouse) String() string { func (*Warehouse) ProtoMessage() {} func (x *Warehouse) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[119] + mi := &file_product_proto_msgTypes[125] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5487,7 +5762,7 @@ func (x *Warehouse) ProtoReflect() protoreflect.Message { // Deprecated: Use Warehouse.ProtoReflect.Descriptor instead. func (*Warehouse) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{119} + return file_product_proto_rawDescGZIP(), []int{125} } func (x *Warehouse) GetId() string { @@ -5521,7 +5796,7 @@ type User struct { func (x *User) Reset() { *x = User{} - mi := &file_product_proto_msgTypes[120] + mi := &file_product_proto_msgTypes[126] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5533,7 +5808,7 @@ func (x *User) String() string { func (*User) ProtoMessage() {} func (x *User) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[120] + mi := &file_product_proto_msgTypes[126] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5546,7 +5821,7 @@ func (x *User) ProtoReflect() protoreflect.Message { // Deprecated: Use User.ProtoReflect.Descriptor instead. func (*User) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{120} + return file_product_proto_rawDescGZIP(), []int{126} } func (x *User) GetId() string { @@ -5574,7 +5849,7 @@ type NestedTypeA struct { func (x *NestedTypeA) Reset() { *x = NestedTypeA{} - mi := &file_product_proto_msgTypes[121] + mi := &file_product_proto_msgTypes[127] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5586,7 +5861,7 @@ func (x *NestedTypeA) String() string { func (*NestedTypeA) ProtoMessage() {} func (x *NestedTypeA) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[121] + mi := &file_product_proto_msgTypes[127] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5599,7 +5874,7 @@ func (x *NestedTypeA) ProtoReflect() protoreflect.Message { // Deprecated: Use NestedTypeA.ProtoReflect.Descriptor instead. func (*NestedTypeA) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{121} + return file_product_proto_rawDescGZIP(), []int{127} } func (x *NestedTypeA) GetId() string { @@ -5634,7 +5909,7 @@ type RecursiveType struct { func (x *RecursiveType) Reset() { *x = RecursiveType{} - mi := &file_product_proto_msgTypes[122] + mi := &file_product_proto_msgTypes[128] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5646,7 +5921,7 @@ func (x *RecursiveType) String() string { func (*RecursiveType) ProtoMessage() {} func (x *RecursiveType) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[122] + mi := &file_product_proto_msgTypes[128] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5659,7 +5934,7 @@ func (x *RecursiveType) ProtoReflect() protoreflect.Message { // Deprecated: Use RecursiveType.ProtoReflect.Descriptor instead. func (*RecursiveType) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{122} + return file_product_proto_rawDescGZIP(), []int{128} } func (x *RecursiveType) GetId() string { @@ -5695,7 +5970,7 @@ type TypeWithMultipleFilterFields struct { func (x *TypeWithMultipleFilterFields) Reset() { *x = TypeWithMultipleFilterFields{} - mi := &file_product_proto_msgTypes[123] + mi := &file_product_proto_msgTypes[129] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5707,7 +5982,7 @@ func (x *TypeWithMultipleFilterFields) String() string { func (*TypeWithMultipleFilterFields) ProtoMessage() {} func (x *TypeWithMultipleFilterFields) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[123] + mi := &file_product_proto_msgTypes[129] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5720,7 +5995,7 @@ func (x *TypeWithMultipleFilterFields) ProtoReflect() protoreflect.Message { // Deprecated: Use TypeWithMultipleFilterFields.ProtoReflect.Descriptor instead. func (*TypeWithMultipleFilterFields) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{123} + return file_product_proto_rawDescGZIP(), []int{129} } func (x *TypeWithMultipleFilterFields) GetId() string { @@ -5761,7 +6036,7 @@ type FilterTypeInput struct { func (x *FilterTypeInput) Reset() { *x = FilterTypeInput{} - mi := &file_product_proto_msgTypes[124] + mi := &file_product_proto_msgTypes[130] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5773,7 +6048,7 @@ func (x *FilterTypeInput) String() string { func (*FilterTypeInput) ProtoMessage() {} func (x *FilterTypeInput) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[124] + mi := &file_product_proto_msgTypes[130] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5786,7 +6061,7 @@ func (x *FilterTypeInput) ProtoReflect() protoreflect.Message { // Deprecated: Use FilterTypeInput.ProtoReflect.Descriptor instead. func (*FilterTypeInput) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{124} + return file_product_proto_rawDescGZIP(), []int{130} } func (x *FilterTypeInput) GetFilterField_1() string { @@ -5812,7 +6087,7 @@ type ComplexFilterTypeInput struct { func (x *ComplexFilterTypeInput) Reset() { *x = ComplexFilterTypeInput{} - mi := &file_product_proto_msgTypes[125] + mi := &file_product_proto_msgTypes[131] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5824,7 +6099,7 @@ func (x *ComplexFilterTypeInput) String() string { func (*ComplexFilterTypeInput) ProtoMessage() {} func (x *ComplexFilterTypeInput) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[125] + mi := &file_product_proto_msgTypes[131] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5837,7 +6112,7 @@ func (x *ComplexFilterTypeInput) ProtoReflect() protoreflect.Message { // Deprecated: Use ComplexFilterTypeInput.ProtoReflect.Descriptor instead. func (*ComplexFilterTypeInput) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{125} + return file_product_proto_rawDescGZIP(), []int{131} } func (x *ComplexFilterTypeInput) GetFilter() *FilterType { @@ -5857,7 +6132,7 @@ type TypeWithComplexFilterInput struct { func (x *TypeWithComplexFilterInput) Reset() { *x = TypeWithComplexFilterInput{} - mi := &file_product_proto_msgTypes[126] + mi := &file_product_proto_msgTypes[132] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5869,7 +6144,7 @@ func (x *TypeWithComplexFilterInput) String() string { func (*TypeWithComplexFilterInput) ProtoMessage() {} func (x *TypeWithComplexFilterInput) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[126] + mi := &file_product_proto_msgTypes[132] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5882,7 +6157,7 @@ func (x *TypeWithComplexFilterInput) ProtoReflect() protoreflect.Message { // Deprecated: Use TypeWithComplexFilterInput.ProtoReflect.Descriptor instead. func (*TypeWithComplexFilterInput) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{126} + return file_product_proto_rawDescGZIP(), []int{132} } func (x *TypeWithComplexFilterInput) GetId() string { @@ -5910,7 +6185,7 @@ type OrderInput struct { func (x *OrderInput) Reset() { *x = OrderInput{} - mi := &file_product_proto_msgTypes[127] + mi := &file_product_proto_msgTypes[133] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5922,7 +6197,7 @@ func (x *OrderInput) String() string { func (*OrderInput) ProtoMessage() {} func (x *OrderInput) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[127] + mi := &file_product_proto_msgTypes[133] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5935,7 +6210,7 @@ func (x *OrderInput) ProtoReflect() protoreflect.Message { // Deprecated: Use OrderInput.ProtoReflect.Descriptor instead. func (*OrderInput) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{127} + return file_product_proto_rawDescGZIP(), []int{133} } func (x *OrderInput) GetOrderId() string { @@ -5971,7 +6246,7 @@ type Order struct { func (x *Order) Reset() { *x = Order{} - mi := &file_product_proto_msgTypes[128] + mi := &file_product_proto_msgTypes[134] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5983,7 +6258,7 @@ func (x *Order) String() string { func (*Order) ProtoMessage() {} func (x *Order) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[128] + mi := &file_product_proto_msgTypes[134] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5996,7 +6271,7 @@ func (x *Order) ProtoReflect() protoreflect.Message { // Deprecated: Use Order.ProtoReflect.Descriptor instead. func (*Order) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{128} + return file_product_proto_rawDescGZIP(), []int{134} } func (x *Order) GetOrderId() string { @@ -6032,13 +6307,14 @@ type Category struct { Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` Name string `protobuf:"bytes,2,opt,name=name,proto3" json:"name,omitempty"` Kind CategoryKind `protobuf:"varint,3,opt,name=kind,proto3,enum=productv1.CategoryKind" json:"kind,omitempty"` + Subcategories *ListOfSubcategory `protobuf:"bytes,5,opt,name=subcategories,proto3" json:"subcategories,omitempty"` unknownFields protoimpl.UnknownFields sizeCache protoimpl.SizeCache } func (x *Category) Reset() { *x = Category{} - mi := &file_product_proto_msgTypes[129] + mi := &file_product_proto_msgTypes[135] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6050,7 +6326,7 @@ func (x *Category) String() string { func (*Category) ProtoMessage() {} func (x *Category) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[129] + mi := &file_product_proto_msgTypes[135] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6063,7 +6339,7 @@ func (x *Category) ProtoReflect() protoreflect.Message { // Deprecated: Use Category.ProtoReflect.Descriptor instead. func (*Category) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{129} + return file_product_proto_rawDescGZIP(), []int{135} } func (x *Category) GetId() string { @@ -6087,6 +6363,13 @@ func (x *Category) GetKind() CategoryKind { return CategoryKind_CATEGORY_KIND_UNSPECIFIED } +func (x *Category) GetSubcategories() *ListOfSubcategory { + if x != nil { + return x.Subcategories + } + return nil +} + type CategoryFilter struct { state protoimpl.MessageState `protogen:"open.v1"` Category CategoryKind `protobuf:"varint,1,opt,name=category,proto3,enum=productv1.CategoryKind" json:"category,omitempty"` @@ -6097,7 +6380,7 @@ type CategoryFilter struct { func (x *CategoryFilter) Reset() { *x = CategoryFilter{} - mi := &file_product_proto_msgTypes[130] + mi := &file_product_proto_msgTypes[136] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6109,7 +6392,7 @@ func (x *CategoryFilter) String() string { func (*CategoryFilter) ProtoMessage() {} func (x *CategoryFilter) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[130] + mi := &file_product_proto_msgTypes[136] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6122,7 +6405,7 @@ func (x *CategoryFilter) ProtoReflect() protoreflect.Message { // Deprecated: Use CategoryFilter.ProtoReflect.Descriptor instead. func (*CategoryFilter) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{130} + return file_product_proto_rawDescGZIP(), []int{136} } func (x *CategoryFilter) GetCategory() CategoryKind { @@ -6152,7 +6435,7 @@ type Animal struct { func (x *Animal) Reset() { *x = Animal{} - mi := &file_product_proto_msgTypes[131] + mi := &file_product_proto_msgTypes[137] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6164,7 +6447,7 @@ func (x *Animal) String() string { func (*Animal) ProtoMessage() {} func (x *Animal) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[131] + mi := &file_product_proto_msgTypes[137] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6177,7 +6460,7 @@ func (x *Animal) ProtoReflect() protoreflect.Message { // Deprecated: Use Animal.ProtoReflect.Descriptor instead. func (*Animal) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{131} + return file_product_proto_rawDescGZIP(), []int{137} } func (x *Animal) GetInstance() isAnimal_Instance { @@ -6231,7 +6514,7 @@ type SearchInput struct { func (x *SearchInput) Reset() { *x = SearchInput{} - mi := &file_product_proto_msgTypes[132] + mi := &file_product_proto_msgTypes[138] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6243,7 +6526,7 @@ func (x *SearchInput) String() string { func (*SearchInput) ProtoMessage() {} func (x *SearchInput) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[132] + mi := &file_product_proto_msgTypes[138] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6256,7 +6539,7 @@ func (x *SearchInput) ProtoReflect() protoreflect.Message { // Deprecated: Use SearchInput.ProtoReflect.Descriptor instead. func (*SearchInput) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{132} + return file_product_proto_rawDescGZIP(), []int{138} } func (x *SearchInput) GetQuery() string { @@ -6287,7 +6570,7 @@ type SearchResult struct { func (x *SearchResult) Reset() { *x = SearchResult{} - mi := &file_product_proto_msgTypes[133] + mi := &file_product_proto_msgTypes[139] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6299,7 +6582,7 @@ func (x *SearchResult) String() string { func (*SearchResult) ProtoMessage() {} func (x *SearchResult) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[133] + mi := &file_product_proto_msgTypes[139] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6312,7 +6595,7 @@ func (x *SearchResult) ProtoReflect() protoreflect.Message { // Deprecated: Use SearchResult.ProtoReflect.Descriptor instead. func (*SearchResult) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{133} + return file_product_proto_rawDescGZIP(), []int{139} } func (x *SearchResult) GetValue() isSearchResult_Value { @@ -6387,7 +6670,7 @@ type NullableFieldsType struct { func (x *NullableFieldsType) Reset() { *x = NullableFieldsType{} - mi := &file_product_proto_msgTypes[134] + mi := &file_product_proto_msgTypes[140] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6399,7 +6682,7 @@ func (x *NullableFieldsType) String() string { func (*NullableFieldsType) ProtoMessage() {} func (x *NullableFieldsType) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[134] + mi := &file_product_proto_msgTypes[140] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6412,7 +6695,7 @@ func (x *NullableFieldsType) ProtoReflect() protoreflect.Message { // Deprecated: Use NullableFieldsType.ProtoReflect.Descriptor instead. func (*NullableFieldsType) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{134} + return file_product_proto_rawDescGZIP(), []int{140} } func (x *NullableFieldsType) GetId() string { @@ -6482,7 +6765,7 @@ type NullableFieldsFilter struct { func (x *NullableFieldsFilter) Reset() { *x = NullableFieldsFilter{} - mi := &file_product_proto_msgTypes[135] + mi := &file_product_proto_msgTypes[141] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6494,7 +6777,7 @@ func (x *NullableFieldsFilter) String() string { func (*NullableFieldsFilter) ProtoMessage() {} func (x *NullableFieldsFilter) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[135] + mi := &file_product_proto_msgTypes[141] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6507,7 +6790,7 @@ func (x *NullableFieldsFilter) ProtoReflect() protoreflect.Message { // Deprecated: Use NullableFieldsFilter.ProtoReflect.Descriptor instead. func (*NullableFieldsFilter) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{135} + return file_product_proto_rawDescGZIP(), []int{141} } func (x *NullableFieldsFilter) GetName() *wrapperspb.StringValue { @@ -6559,7 +6842,7 @@ type BlogPost struct { func (x *BlogPost) Reset() { *x = BlogPost{} - mi := &file_product_proto_msgTypes[136] + mi := &file_product_proto_msgTypes[142] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6571,7 +6854,7 @@ func (x *BlogPost) String() string { func (*BlogPost) ProtoMessage() {} func (x *BlogPost) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[136] + mi := &file_product_proto_msgTypes[142] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6584,7 +6867,7 @@ func (x *BlogPost) ProtoReflect() protoreflect.Message { // Deprecated: Use BlogPost.ProtoReflect.Descriptor instead. func (*BlogPost) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{136} + return file_product_proto_rawDescGZIP(), []int{142} } func (x *BlogPost) GetId() string { @@ -6738,7 +7021,7 @@ type BlogPostFilter struct { func (x *BlogPostFilter) Reset() { *x = BlogPostFilter{} - mi := &file_product_proto_msgTypes[137] + mi := &file_product_proto_msgTypes[143] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6750,7 +7033,7 @@ func (x *BlogPostFilter) String() string { func (*BlogPostFilter) ProtoMessage() {} func (x *BlogPostFilter) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[137] + mi := &file_product_proto_msgTypes[143] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6763,7 +7046,7 @@ func (x *BlogPostFilter) ProtoReflect() protoreflect.Message { // Deprecated: Use BlogPostFilter.ProtoReflect.Descriptor instead. func (*BlogPostFilter) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{137} + return file_product_proto_rawDescGZIP(), []int{143} } func (x *BlogPostFilter) GetTitle() *wrapperspb.StringValue { @@ -6810,7 +7093,7 @@ type Author struct { func (x *Author) Reset() { *x = Author{} - mi := &file_product_proto_msgTypes[138] + mi := &file_product_proto_msgTypes[144] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6822,7 +7105,7 @@ func (x *Author) String() string { func (*Author) ProtoMessage() {} func (x *Author) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[138] + mi := &file_product_proto_msgTypes[144] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6835,7 +7118,7 @@ func (x *Author) ProtoReflect() protoreflect.Message { // Deprecated: Use Author.ProtoReflect.Descriptor instead. func (*Author) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{138} + return file_product_proto_rawDescGZIP(), []int{144} } func (x *Author) GetId() string { @@ -6954,7 +7237,7 @@ type AuthorFilter struct { func (x *AuthorFilter) Reset() { *x = AuthorFilter{} - mi := &file_product_proto_msgTypes[139] + mi := &file_product_proto_msgTypes[145] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6966,7 +7249,7 @@ func (x *AuthorFilter) String() string { func (*AuthorFilter) ProtoMessage() {} func (x *AuthorFilter) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[139] + mi := &file_product_proto_msgTypes[145] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6979,7 +7262,7 @@ func (x *AuthorFilter) ProtoReflect() protoreflect.Message { // Deprecated: Use AuthorFilter.ProtoReflect.Descriptor instead. func (*AuthorFilter) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{139} + return file_product_proto_rawDescGZIP(), []int{145} } func (x *AuthorFilter) GetName() *wrapperspb.StringValue { @@ -7012,7 +7295,7 @@ type UserInput struct { func (x *UserInput) Reset() { *x = UserInput{} - mi := &file_product_proto_msgTypes[140] + mi := &file_product_proto_msgTypes[146] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7024,7 +7307,7 @@ func (x *UserInput) String() string { func (*UserInput) ProtoMessage() {} func (x *UserInput) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[140] + mi := &file_product_proto_msgTypes[146] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7037,7 +7320,7 @@ func (x *UserInput) ProtoReflect() protoreflect.Message { // Deprecated: Use UserInput.ProtoReflect.Descriptor instead. func (*UserInput) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{140} + return file_product_proto_rawDescGZIP(), []int{146} } func (x *UserInput) GetName() string { @@ -7057,7 +7340,7 @@ type ActionInput struct { func (x *ActionInput) Reset() { *x = ActionInput{} - mi := &file_product_proto_msgTypes[141] + mi := &file_product_proto_msgTypes[147] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7069,7 +7352,7 @@ func (x *ActionInput) String() string { func (*ActionInput) ProtoMessage() {} func (x *ActionInput) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[141] + mi := &file_product_proto_msgTypes[147] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7082,7 +7365,7 @@ func (x *ActionInput) ProtoReflect() protoreflect.Message { // Deprecated: Use ActionInput.ProtoReflect.Descriptor instead. func (*ActionInput) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{141} + return file_product_proto_rawDescGZIP(), []int{147} } func (x *ActionInput) GetType() string { @@ -7112,7 +7395,7 @@ type ActionResult struct { func (x *ActionResult) Reset() { *x = ActionResult{} - mi := &file_product_proto_msgTypes[142] + mi := &file_product_proto_msgTypes[148] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7124,7 +7407,7 @@ func (x *ActionResult) String() string { func (*ActionResult) ProtoMessage() {} func (x *ActionResult) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[142] + mi := &file_product_proto_msgTypes[148] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7137,7 +7420,7 @@ func (x *ActionResult) ProtoReflect() protoreflect.Message { // Deprecated: Use ActionResult.ProtoReflect.Descriptor instead. func (*ActionResult) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{142} + return file_product_proto_rawDescGZIP(), []int{148} } func (x *ActionResult) GetValue() isActionResult_Value { @@ -7196,7 +7479,7 @@ type NullableFieldsInput struct { func (x *NullableFieldsInput) Reset() { *x = NullableFieldsInput{} - mi := &file_product_proto_msgTypes[143] + mi := &file_product_proto_msgTypes[149] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7208,7 +7491,7 @@ func (x *NullableFieldsInput) String() string { func (*NullableFieldsInput) ProtoMessage() {} func (x *NullableFieldsInput) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[143] + mi := &file_product_proto_msgTypes[149] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7221,7 +7504,7 @@ func (x *NullableFieldsInput) ProtoReflect() protoreflect.Message { // Deprecated: Use NullableFieldsInput.ProtoReflect.Descriptor instead. func (*NullableFieldsInput) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{143} + return file_product_proto_rawDescGZIP(), []int{149} } func (x *NullableFieldsInput) GetName() string { @@ -7297,7 +7580,7 @@ type BlogPostInput struct { func (x *BlogPostInput) Reset() { *x = BlogPostInput{} - mi := &file_product_proto_msgTypes[144] + mi := &file_product_proto_msgTypes[150] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7309,7 +7592,7 @@ func (x *BlogPostInput) String() string { func (*BlogPostInput) ProtoMessage() {} func (x *BlogPostInput) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[144] + mi := &file_product_proto_msgTypes[150] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7322,7 +7605,7 @@ func (x *BlogPostInput) ProtoReflect() protoreflect.Message { // Deprecated: Use BlogPostInput.ProtoReflect.Descriptor instead. func (*BlogPostInput) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{144} + return file_product_proto_rawDescGZIP(), []int{150} } func (x *BlogPostInput) GetTitle() string { @@ -7455,7 +7738,7 @@ type AuthorInput struct { func (x *AuthorInput) Reset() { *x = AuthorInput{} - mi := &file_product_proto_msgTypes[145] + mi := &file_product_proto_msgTypes[151] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7467,7 +7750,7 @@ func (x *AuthorInput) String() string { func (*AuthorInput) ProtoMessage() {} func (x *AuthorInput) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[145] + mi := &file_product_proto_msgTypes[151] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7480,7 +7763,7 @@ func (x *AuthorInput) ProtoReflect() protoreflect.Message { // Deprecated: Use AuthorInput.ProtoReflect.Descriptor instead. func (*AuthorInput) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{145} + return file_product_proto_rawDescGZIP(), []int{151} } func (x *AuthorInput) GetName() string { @@ -7564,7 +7847,7 @@ type NestedTypeB struct { func (x *NestedTypeB) Reset() { *x = NestedTypeB{} - mi := &file_product_proto_msgTypes[146] + mi := &file_product_proto_msgTypes[152] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7576,7 +7859,7 @@ func (x *NestedTypeB) String() string { func (*NestedTypeB) ProtoMessage() {} func (x *NestedTypeB) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[146] + mi := &file_product_proto_msgTypes[152] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7589,7 +7872,7 @@ func (x *NestedTypeB) ProtoReflect() protoreflect.Message { // Deprecated: Use NestedTypeB.ProtoReflect.Descriptor instead. func (*NestedTypeB) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{146} + return file_product_proto_rawDescGZIP(), []int{152} } func (x *NestedTypeB) GetId() string { @@ -7623,7 +7906,7 @@ type NestedTypeC struct { func (x *NestedTypeC) Reset() { *x = NestedTypeC{} - mi := &file_product_proto_msgTypes[147] + mi := &file_product_proto_msgTypes[153] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7635,7 +7918,7 @@ func (x *NestedTypeC) String() string { func (*NestedTypeC) ProtoMessage() {} func (x *NestedTypeC) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[147] + mi := &file_product_proto_msgTypes[153] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7648,7 +7931,7 @@ func (x *NestedTypeC) ProtoReflect() protoreflect.Message { // Deprecated: Use NestedTypeC.ProtoReflect.Descriptor instead. func (*NestedTypeC) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{147} + return file_product_proto_rawDescGZIP(), []int{153} } func (x *NestedTypeC) GetId() string { @@ -7677,7 +7960,7 @@ type FilterType struct { func (x *FilterType) Reset() { *x = FilterType{} - mi := &file_product_proto_msgTypes[148] + mi := &file_product_proto_msgTypes[154] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7689,7 +7972,7 @@ func (x *FilterType) String() string { func (*FilterType) ProtoMessage() {} func (x *FilterType) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[148] + mi := &file_product_proto_msgTypes[154] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7702,7 +7985,7 @@ func (x *FilterType) ProtoReflect() protoreflect.Message { // Deprecated: Use FilterType.ProtoReflect.Descriptor instead. func (*FilterType) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{148} + return file_product_proto_rawDescGZIP(), []int{154} } func (x *FilterType) GetName() string { @@ -7743,7 +8026,7 @@ type Pagination struct { func (x *Pagination) Reset() { *x = Pagination{} - mi := &file_product_proto_msgTypes[149] + mi := &file_product_proto_msgTypes[155] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7755,7 +8038,7 @@ func (x *Pagination) String() string { func (*Pagination) ProtoMessage() {} func (x *Pagination) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[149] + mi := &file_product_proto_msgTypes[155] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7768,7 +8051,7 @@ func (x *Pagination) ProtoReflect() protoreflect.Message { // Deprecated: Use Pagination.ProtoReflect.Descriptor instead. func (*Pagination) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{149} + return file_product_proto_rawDescGZIP(), []int{155} } func (x *Pagination) GetPage() int32 { @@ -7796,7 +8079,7 @@ type OrderLineInput struct { func (x *OrderLineInput) Reset() { *x = OrderLineInput{} - mi := &file_product_proto_msgTypes[150] + mi := &file_product_proto_msgTypes[156] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7808,7 +8091,7 @@ func (x *OrderLineInput) String() string { func (*OrderLineInput) ProtoMessage() {} func (x *OrderLineInput) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[150] + mi := &file_product_proto_msgTypes[156] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7821,7 +8104,7 @@ func (x *OrderLineInput) ProtoReflect() protoreflect.Message { // Deprecated: Use OrderLineInput.ProtoReflect.Descriptor instead. func (*OrderLineInput) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{150} + return file_product_proto_rawDescGZIP(), []int{156} } func (x *OrderLineInput) GetProductId() string { @@ -7856,7 +8139,7 @@ type OrderLine struct { func (x *OrderLine) Reset() { *x = OrderLine{} - mi := &file_product_proto_msgTypes[151] + mi := &file_product_proto_msgTypes[157] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7868,7 +8151,7 @@ func (x *OrderLine) String() string { func (*OrderLine) ProtoMessage() {} func (x *OrderLine) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[151] + mi := &file_product_proto_msgTypes[157] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7881,7 +8164,7 @@ func (x *OrderLine) ProtoReflect() protoreflect.Message { // Deprecated: Use OrderLine.ProtoReflect.Descriptor instead. func (*OrderLine) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{151} + return file_product_proto_rawDescGZIP(), []int{157} } func (x *OrderLine) GetProductId() string { @@ -7905,6 +8188,74 @@ func (x *OrderLine) GetModifiers() *ListOfString { return nil } +type Subcategory struct { + state protoimpl.MessageState `protogen:"open.v1"` + Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` + Name string `protobuf:"bytes,2,opt,name=name,proto3" json:"name,omitempty"` + Description *wrapperspb.StringValue `protobuf:"bytes,3,opt,name=description,proto3" json:"description,omitempty"` + IsActive bool `protobuf:"varint,4,opt,name=is_active,json=isActive,proto3" json:"is_active,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *Subcategory) Reset() { + *x = Subcategory{} + mi := &file_product_proto_msgTypes[158] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *Subcategory) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Subcategory) ProtoMessage() {} + +func (x *Subcategory) ProtoReflect() protoreflect.Message { + mi := &file_product_proto_msgTypes[158] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use Subcategory.ProtoReflect.Descriptor instead. +func (*Subcategory) Descriptor() ([]byte, []int) { + return file_product_proto_rawDescGZIP(), []int{158} +} + +func (x *Subcategory) GetId() string { + if x != nil { + return x.Id + } + return "" +} + +func (x *Subcategory) GetName() string { + if x != nil { + return x.Name + } + return "" +} + +func (x *Subcategory) GetDescription() *wrapperspb.StringValue { + if x != nil { + return x.Description + } + return nil +} + +func (x *Subcategory) GetIsActive() bool { + if x != nil { + return x.IsActive + } + return false +} + type Cat struct { state protoimpl.MessageState `protogen:"open.v1"` Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` @@ -7917,7 +8268,7 @@ type Cat struct { func (x *Cat) Reset() { *x = Cat{} - mi := &file_product_proto_msgTypes[152] + mi := &file_product_proto_msgTypes[159] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7929,7 +8280,7 @@ func (x *Cat) String() string { func (*Cat) ProtoMessage() {} func (x *Cat) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[152] + mi := &file_product_proto_msgTypes[159] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7942,7 +8293,7 @@ func (x *Cat) ProtoReflect() protoreflect.Message { // Deprecated: Use Cat.ProtoReflect.Descriptor instead. func (*Cat) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{152} + return file_product_proto_rawDescGZIP(), []int{159} } func (x *Cat) GetId() string { @@ -7985,7 +8336,7 @@ type Dog struct { func (x *Dog) Reset() { *x = Dog{} - mi := &file_product_proto_msgTypes[153] + mi := &file_product_proto_msgTypes[160] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7997,7 +8348,7 @@ func (x *Dog) String() string { func (*Dog) ProtoMessage() {} func (x *Dog) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[153] + mi := &file_product_proto_msgTypes[160] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8010,7 +8361,7 @@ func (x *Dog) ProtoReflect() protoreflect.Message { // Deprecated: Use Dog.ProtoReflect.Descriptor instead. func (*Dog) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{153} + return file_product_proto_rawDescGZIP(), []int{160} } func (x *Dog) GetId() string { @@ -8051,7 +8402,7 @@ type ActionSuccess struct { func (x *ActionSuccess) Reset() { *x = ActionSuccess{} - mi := &file_product_proto_msgTypes[154] + mi := &file_product_proto_msgTypes[161] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8063,7 +8414,7 @@ func (x *ActionSuccess) String() string { func (*ActionSuccess) ProtoMessage() {} func (x *ActionSuccess) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[154] + mi := &file_product_proto_msgTypes[161] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8076,7 +8427,7 @@ func (x *ActionSuccess) ProtoReflect() protoreflect.Message { // Deprecated: Use ActionSuccess.ProtoReflect.Descriptor instead. func (*ActionSuccess) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{154} + return file_product_proto_rawDescGZIP(), []int{161} } func (x *ActionSuccess) GetMessage() string { @@ -8103,7 +8454,7 @@ type ActionError struct { func (x *ActionError) Reset() { *x = ActionError{} - mi := &file_product_proto_msgTypes[155] + mi := &file_product_proto_msgTypes[162] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8115,7 +8466,7 @@ func (x *ActionError) String() string { func (*ActionError) ProtoMessage() {} func (x *ActionError) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[155] + mi := &file_product_proto_msgTypes[162] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8128,7 +8479,7 @@ func (x *ActionError) ProtoReflect() protoreflect.Message { // Deprecated: Use ActionError.ProtoReflect.Descriptor instead. func (*ActionError) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{155} + return file_product_proto_rawDescGZIP(), []int{162} } func (x *ActionError) GetMessage() string { @@ -8155,7 +8506,7 @@ type CategoryInput struct { func (x *CategoryInput) Reset() { *x = CategoryInput{} - mi := &file_product_proto_msgTypes[156] + mi := &file_product_proto_msgTypes[163] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8167,7 +8518,7 @@ func (x *CategoryInput) String() string { func (*CategoryInput) ProtoMessage() {} func (x *CategoryInput) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[156] + mi := &file_product_proto_msgTypes[163] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8180,7 +8531,7 @@ func (x *CategoryInput) ProtoReflect() protoreflect.Message { // Deprecated: Use CategoryInput.ProtoReflect.Descriptor instead. func (*CategoryInput) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{156} + return file_product_proto_rawDescGZIP(), []int{163} } func (x *CategoryInput) GetName() string { @@ -8209,7 +8560,7 @@ type ProductCountFilter struct { func (x *ProductCountFilter) Reset() { *x = ProductCountFilter{} - mi := &file_product_proto_msgTypes[157] + mi := &file_product_proto_msgTypes[164] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8221,7 +8572,7 @@ func (x *ProductCountFilter) String() string { func (*ProductCountFilter) ProtoMessage() {} func (x *ProductCountFilter) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[157] + mi := &file_product_proto_msgTypes[164] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8234,7 +8585,7 @@ func (x *ProductCountFilter) ProtoReflect() protoreflect.Message { // Deprecated: Use ProductCountFilter.ProtoReflect.Descriptor instead. func (*ProductCountFilter) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{157} + return file_product_proto_rawDescGZIP(), []int{164} } func (x *ProductCountFilter) GetMinPrice() *wrapperspb.DoubleValue { @@ -8265,6 +8616,82 @@ func (x *ProductCountFilter) GetSearchTerm() *wrapperspb.StringValue { return nil } +type SubcategoryItemFilter struct { + state protoimpl.MessageState `protogen:"open.v1"` + MinPrice *wrapperspb.DoubleValue `protobuf:"bytes,1,opt,name=min_price,json=minPrice,proto3" json:"min_price,omitempty"` + MaxPrice *wrapperspb.DoubleValue `protobuf:"bytes,2,opt,name=max_price,json=maxPrice,proto3" json:"max_price,omitempty"` + InStock *wrapperspb.BoolValue `protobuf:"bytes,3,opt,name=in_stock,json=inStock,proto3" json:"in_stock,omitempty"` + IsActive *wrapperspb.BoolValue `protobuf:"bytes,4,opt,name=is_active,json=isActive,proto3" json:"is_active,omitempty"` + SearchTerm *wrapperspb.StringValue `protobuf:"bytes,5,opt,name=search_term,json=searchTerm,proto3" json:"search_term,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *SubcategoryItemFilter) Reset() { + *x = SubcategoryItemFilter{} + mi := &file_product_proto_msgTypes[165] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *SubcategoryItemFilter) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*SubcategoryItemFilter) ProtoMessage() {} + +func (x *SubcategoryItemFilter) ProtoReflect() protoreflect.Message { + mi := &file_product_proto_msgTypes[165] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use SubcategoryItemFilter.ProtoReflect.Descriptor instead. +func (*SubcategoryItemFilter) Descriptor() ([]byte, []int) { + return file_product_proto_rawDescGZIP(), []int{165} +} + +func (x *SubcategoryItemFilter) GetMinPrice() *wrapperspb.DoubleValue { + if x != nil { + return x.MinPrice + } + return nil +} + +func (x *SubcategoryItemFilter) GetMaxPrice() *wrapperspb.DoubleValue { + if x != nil { + return x.MaxPrice + } + return nil +} + +func (x *SubcategoryItemFilter) GetInStock() *wrapperspb.BoolValue { + if x != nil { + return x.InStock + } + return nil +} + +func (x *SubcategoryItemFilter) GetIsActive() *wrapperspb.BoolValue { + if x != nil { + return x.IsActive + } + return nil +} + +func (x *SubcategoryItemFilter) GetSearchTerm() *wrapperspb.StringValue { + if x != nil { + return x.SearchTerm + } + return nil +} + type ListOfAuthorFilter_List struct { state protoimpl.MessageState `protogen:"open.v1"` Items []*AuthorFilter `protobuf:"bytes,1,rep,name=items,proto3" json:"items,omitempty"` @@ -8274,7 +8701,7 @@ type ListOfAuthorFilter_List struct { func (x *ListOfAuthorFilter_List) Reset() { *x = ListOfAuthorFilter_List{} - mi := &file_product_proto_msgTypes[158] + mi := &file_product_proto_msgTypes[166] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8286,7 +8713,7 @@ func (x *ListOfAuthorFilter_List) String() string { func (*ListOfAuthorFilter_List) ProtoMessage() {} func (x *ListOfAuthorFilter_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[158] + mi := &file_product_proto_msgTypes[166] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8318,7 +8745,7 @@ type ListOfAuthorInput_List struct { func (x *ListOfAuthorInput_List) Reset() { *x = ListOfAuthorInput_List{} - mi := &file_product_proto_msgTypes[159] + mi := &file_product_proto_msgTypes[167] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8330,7 +8757,7 @@ func (x *ListOfAuthorInput_List) String() string { func (*ListOfAuthorInput_List) ProtoMessage() {} func (x *ListOfAuthorInput_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[159] + mi := &file_product_proto_msgTypes[167] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8362,7 +8789,7 @@ type ListOfBlogPost_List struct { func (x *ListOfBlogPost_List) Reset() { *x = ListOfBlogPost_List{} - mi := &file_product_proto_msgTypes[160] + mi := &file_product_proto_msgTypes[168] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8374,7 +8801,7 @@ func (x *ListOfBlogPost_List) String() string { func (*ListOfBlogPost_List) ProtoMessage() {} func (x *ListOfBlogPost_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[160] + mi := &file_product_proto_msgTypes[168] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8406,7 +8833,7 @@ type ListOfBlogPostFilter_List struct { func (x *ListOfBlogPostFilter_List) Reset() { *x = ListOfBlogPostFilter_List{} - mi := &file_product_proto_msgTypes[161] + mi := &file_product_proto_msgTypes[169] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8418,7 +8845,7 @@ func (x *ListOfBlogPostFilter_List) String() string { func (*ListOfBlogPostFilter_List) ProtoMessage() {} func (x *ListOfBlogPostFilter_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[161] + mi := &file_product_proto_msgTypes[169] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8450,7 +8877,7 @@ type ListOfBlogPostInput_List struct { func (x *ListOfBlogPostInput_List) Reset() { *x = ListOfBlogPostInput_List{} - mi := &file_product_proto_msgTypes[162] + mi := &file_product_proto_msgTypes[170] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8462,7 +8889,7 @@ func (x *ListOfBlogPostInput_List) String() string { func (*ListOfBlogPostInput_List) ProtoMessage() {} func (x *ListOfBlogPostInput_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[162] + mi := &file_product_proto_msgTypes[170] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8494,7 +8921,7 @@ type ListOfBoolean_List struct { func (x *ListOfBoolean_List) Reset() { *x = ListOfBoolean_List{} - mi := &file_product_proto_msgTypes[163] + mi := &file_product_proto_msgTypes[171] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8506,7 +8933,7 @@ func (x *ListOfBoolean_List) String() string { func (*ListOfBoolean_List) ProtoMessage() {} func (x *ListOfBoolean_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[163] + mi := &file_product_proto_msgTypes[171] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8538,7 +8965,7 @@ type ListOfCategory_List struct { func (x *ListOfCategory_List) Reset() { *x = ListOfCategory_List{} - mi := &file_product_proto_msgTypes[164] + mi := &file_product_proto_msgTypes[172] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8550,7 +8977,7 @@ func (x *ListOfCategory_List) String() string { func (*ListOfCategory_List) ProtoMessage() {} func (x *ListOfCategory_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[164] + mi := &file_product_proto_msgTypes[172] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8582,7 +9009,7 @@ type ListOfCategoryInput_List struct { func (x *ListOfCategoryInput_List) Reset() { *x = ListOfCategoryInput_List{} - mi := &file_product_proto_msgTypes[165] + mi := &file_product_proto_msgTypes[173] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8594,7 +9021,7 @@ func (x *ListOfCategoryInput_List) String() string { func (*ListOfCategoryInput_List) ProtoMessage() {} func (x *ListOfCategoryInput_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[165] + mi := &file_product_proto_msgTypes[173] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8626,7 +9053,7 @@ type ListOfFloat_List struct { func (x *ListOfFloat_List) Reset() { *x = ListOfFloat_List{} - mi := &file_product_proto_msgTypes[166] + mi := &file_product_proto_msgTypes[174] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8638,7 +9065,7 @@ func (x *ListOfFloat_List) String() string { func (*ListOfFloat_List) ProtoMessage() {} func (x *ListOfFloat_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[166] + mi := &file_product_proto_msgTypes[174] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8670,7 +9097,7 @@ type ListOfListOfCategory_List struct { func (x *ListOfListOfCategory_List) Reset() { *x = ListOfListOfCategory_List{} - mi := &file_product_proto_msgTypes[167] + mi := &file_product_proto_msgTypes[175] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8682,7 +9109,7 @@ func (x *ListOfListOfCategory_List) String() string { func (*ListOfListOfCategory_List) ProtoMessage() {} func (x *ListOfListOfCategory_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[167] + mi := &file_product_proto_msgTypes[175] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8714,7 +9141,7 @@ type ListOfListOfCategoryInput_List struct { func (x *ListOfListOfCategoryInput_List) Reset() { *x = ListOfListOfCategoryInput_List{} - mi := &file_product_proto_msgTypes[168] + mi := &file_product_proto_msgTypes[176] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8726,7 +9153,7 @@ func (x *ListOfListOfCategoryInput_List) String() string { func (*ListOfListOfCategoryInput_List) ProtoMessage() {} func (x *ListOfListOfCategoryInput_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[168] + mi := &file_product_proto_msgTypes[176] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8758,7 +9185,7 @@ type ListOfListOfString_List struct { func (x *ListOfListOfString_List) Reset() { *x = ListOfListOfString_List{} - mi := &file_product_proto_msgTypes[169] + mi := &file_product_proto_msgTypes[177] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8770,7 +9197,7 @@ func (x *ListOfListOfString_List) String() string { func (*ListOfListOfString_List) ProtoMessage() {} func (x *ListOfListOfString_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[169] + mi := &file_product_proto_msgTypes[177] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8802,7 +9229,7 @@ type ListOfListOfUser_List struct { func (x *ListOfListOfUser_List) Reset() { *x = ListOfListOfUser_List{} - mi := &file_product_proto_msgTypes[170] + mi := &file_product_proto_msgTypes[178] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8814,7 +9241,7 @@ func (x *ListOfListOfUser_List) String() string { func (*ListOfListOfUser_List) ProtoMessage() {} func (x *ListOfListOfUser_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[170] + mi := &file_product_proto_msgTypes[178] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8846,7 +9273,7 @@ type ListOfListOfUserInput_List struct { func (x *ListOfListOfUserInput_List) Reset() { *x = ListOfListOfUserInput_List{} - mi := &file_product_proto_msgTypes[171] + mi := &file_product_proto_msgTypes[179] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8858,7 +9285,7 @@ func (x *ListOfListOfUserInput_List) String() string { func (*ListOfListOfUserInput_List) ProtoMessage() {} func (x *ListOfListOfUserInput_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[171] + mi := &file_product_proto_msgTypes[179] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8890,7 +9317,7 @@ type ListOfOrderLine_List struct { func (x *ListOfOrderLine_List) Reset() { *x = ListOfOrderLine_List{} - mi := &file_product_proto_msgTypes[172] + mi := &file_product_proto_msgTypes[180] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8902,7 +9329,7 @@ func (x *ListOfOrderLine_List) String() string { func (*ListOfOrderLine_List) ProtoMessage() {} func (x *ListOfOrderLine_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[172] + mi := &file_product_proto_msgTypes[180] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8934,7 +9361,7 @@ type ListOfProduct_List struct { func (x *ListOfProduct_List) Reset() { *x = ListOfProduct_List{} - mi := &file_product_proto_msgTypes[173] + mi := &file_product_proto_msgTypes[181] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8946,7 +9373,7 @@ func (x *ListOfProduct_List) String() string { func (*ListOfProduct_List) ProtoMessage() {} func (x *ListOfProduct_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[173] + mi := &file_product_proto_msgTypes[181] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8978,7 +9405,7 @@ type ListOfString_List struct { func (x *ListOfString_List) Reset() { *x = ListOfString_List{} - mi := &file_product_proto_msgTypes[174] + mi := &file_product_proto_msgTypes[182] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8990,7 +9417,7 @@ func (x *ListOfString_List) String() string { func (*ListOfString_List) ProtoMessage() {} func (x *ListOfString_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[174] + mi := &file_product_proto_msgTypes[182] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9013,6 +9440,50 @@ func (x *ListOfString_List) GetItems() []string { return nil } +type ListOfSubcategory_List struct { + state protoimpl.MessageState `protogen:"open.v1"` + Items []*Subcategory `protobuf:"bytes,1,rep,name=items,proto3" json:"items,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ListOfSubcategory_List) Reset() { + *x = ListOfSubcategory_List{} + mi := &file_product_proto_msgTypes[183] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ListOfSubcategory_List) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ListOfSubcategory_List) ProtoMessage() {} + +func (x *ListOfSubcategory_List) ProtoReflect() protoreflect.Message { + mi := &file_product_proto_msgTypes[183] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ListOfSubcategory_List.ProtoReflect.Descriptor instead. +func (*ListOfSubcategory_List) Descriptor() ([]byte, []int) { + return file_product_proto_rawDescGZIP(), []int{17, 0} +} + +func (x *ListOfSubcategory_List) GetItems() []*Subcategory { + if x != nil { + return x.Items + } + return nil +} + type ListOfUser_List struct { state protoimpl.MessageState `protogen:"open.v1"` Items []*User `protobuf:"bytes,1,rep,name=items,proto3" json:"items,omitempty"` @@ -9022,7 +9493,7 @@ type ListOfUser_List struct { func (x *ListOfUser_List) Reset() { *x = ListOfUser_List{} - mi := &file_product_proto_msgTypes[175] + mi := &file_product_proto_msgTypes[184] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9034,7 +9505,7 @@ func (x *ListOfUser_List) String() string { func (*ListOfUser_List) ProtoMessage() {} func (x *ListOfUser_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[175] + mi := &file_product_proto_msgTypes[184] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9047,7 +9518,7 @@ func (x *ListOfUser_List) ProtoReflect() protoreflect.Message { // Deprecated: Use ListOfUser_List.ProtoReflect.Descriptor instead. func (*ListOfUser_List) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{17, 0} + return file_product_proto_rawDescGZIP(), []int{18, 0} } func (x *ListOfUser_List) GetItems() []*User { @@ -9066,7 +9537,7 @@ type ListOfUserInput_List struct { func (x *ListOfUserInput_List) Reset() { *x = ListOfUserInput_List{} - mi := &file_product_proto_msgTypes[176] + mi := &file_product_proto_msgTypes[185] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9078,7 +9549,7 @@ func (x *ListOfUserInput_List) String() string { func (*ListOfUserInput_List) ProtoMessage() {} func (x *ListOfUserInput_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[176] + mi := &file_product_proto_msgTypes[185] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9091,7 +9562,7 @@ func (x *ListOfUserInput_List) ProtoReflect() protoreflect.Message { // Deprecated: Use ListOfUserInput_List.ProtoReflect.Descriptor instead. func (*ListOfUserInput_List) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{18, 0} + return file_product_proto_rawDescGZIP(), []int{19, 0} } func (x *ListOfUserInput_List) GetItems() []*UserInput { @@ -9173,7 +9644,11 @@ const file_product_proto_rawDesc = "" + "\fListOfString\x120\n" + "\x04list\x18\x01 \x01(\v2\x1c.productv1.ListOfString.ListR\x04list\x1a\x1c\n" + "\x04List\x12\x14\n" + - "\x05items\x18\x01 \x03(\tR\x05items\"k\n" + + "\x05items\x18\x01 \x03(\tR\x05items\"\x80\x01\n" + + "\x11ListOfSubcategory\x125\n" + + "\x04list\x18\x01 \x01(\v2!.productv1.ListOfSubcategory.ListR\x04list\x1a4\n" + + "\x04List\x12,\n" + + "\x05items\x18\x01 \x03(\v2\x16.productv1.SubcategoryR\x05items\"k\n" + "\n" + "ListOfUser\x12.\n" + "\x04list\x18\x01 \x01(\v2\x1a.productv1.ListOfUser.ListR\x04list\x1a-\n" + @@ -9368,20 +9843,33 @@ const file_product_proto_rawDesc = "" + "\n" + "blog_posts\x18\x01 \x01(\v2\x1e.productv1.ListOfBlogPostInputR\tblogPosts\"o\n" + "#MutationBulkUpdateBlogPostsResponse\x12H\n" + - "\x16bulk_update_blog_posts\x18\x01 \x03(\v2\x13.productv1.BlogPostR\x13bulkUpdateBlogPosts\"S\n" + - "\x18CategoryProductCountArgs\x127\n" + - "\afilters\x18\x01 \x01(\v2\x1d.productv1.ProductCountFilterR\afilters\"A\n" + - "\x1bCategoryProductCountContext\x12\x0e\n" + + "\x16bulk_update_blog_posts\x18\x01 \x03(\v2\x13.productv1.BlogPostR\x13bulkUpdateBlogPosts\"Z\n" + + "\x1fResolveCategoryProductCountArgs\x127\n" + + "\afilters\x18\x01 \x01(\v2\x1d.productv1.ProductCountFilterR\afilters\"H\n" + + "\"ResolveCategoryProductCountContext\x12\x0e\n" + "\x02id\x18\x01 \x01(\tR\x02id\x12\x12\n" + - "\x04name\x18\x02 \x01(\tR\x04name\"\xaa\x01\n" + - "\"ResolveCategoryProductCountRequest\x12@\n" + - "\acontext\x18\x01 \x03(\v2&.productv1.CategoryProductCountContextR\acontext\x12B\n" + + "\x04name\x18\x02 \x01(\tR\x04name\"\xb8\x01\n" + + "\"ResolveCategoryProductCountRequest\x12G\n" + + "\acontext\x18\x01 \x03(\v2-.productv1.ResolveCategoryProductCountContextR\acontext\x12I\n" + + "\n" + + "field_args\x18\x02 \x01(\v2*.productv1.ResolveCategoryProductCountArgsR\tfieldArgs\"H\n" + + "!ResolveCategoryProductCountResult\x12#\n" + + "\rproduct_count\x18\x01 \x01(\x05R\fproductCount\"k\n" + + "#ResolveCategoryProductCountResponse\x12D\n" + + "\x06result\x18\x01 \x03(\v2,.productv1.ResolveCategoryProductCountResultR\x06result\"]\n" + + "\x1fResolveSubcategoryItemCountArgs\x12:\n" + + "\afilters\x18\x01 \x01(\v2 .productv1.SubcategoryItemFilterR\afilters\"4\n" + + "\"ResolveSubcategoryItemCountContext\x12\x0e\n" + + "\x02id\x18\x01 \x01(\tR\x02id\"\xb8\x01\n" + + "\"ResolveSubcategoryItemCountRequest\x12G\n" + + "\acontext\x18\x01 \x03(\v2-.productv1.ResolveSubcategoryItemCountContextR\acontext\x12I\n" + "\n" + - "field_args\x18\x02 \x01(\v2#.productv1.CategoryProductCountArgsR\tfieldArgs\"P\n" + - ")ResolveCategoryProductCountResponseResult\x12#\n" + - "\rproduct_count\x18\x01 \x01(\x05R\fproductCount\"s\n" + - "#ResolveCategoryProductCountResponse\x12L\n" + - "\x06result\x18\x01 \x03(\v24.productv1.ResolveCategoryProductCountResponseResultR\x06result\"C\n" + + "field_args\x18\x02 \x01(\v2*.productv1.ResolveSubcategoryItemCountArgsR\tfieldArgs\"B\n" + + "!ResolveSubcategoryItemCountResult\x12\x1d\n" + + "\n" + + "item_count\x18\x01 \x01(\x05R\titemCount\"k\n" + + "#ResolveSubcategoryItemCountResponse\x12D\n" + + "\x06result\x18\x01 \x03(\v2,.productv1.ResolveSubcategoryItemCountResultR\x06result\"C\n" + "\aProduct\x12\x0e\n" + "\x02id\x18\x01 \x01(\tR\x02id\x12\x12\n" + "\x04name\x18\x02 \x01(\tR\x04name\x12\x14\n" + @@ -9429,11 +9917,12 @@ const file_product_proto_rawDesc = "" + "\vtotal_items\x18\x03 \x01(\x05R\n" + "totalItems\x12;\n" + "\vorder_lines\x18\x04 \x01(\v2\x1a.productv1.ListOfOrderLineR\n" + - "orderLines\"[\n" + + "orderLines\"\x9f\x01\n" + "\bCategory\x12\x0e\n" + "\x02id\x18\x01 \x01(\tR\x02id\x12\x12\n" + "\x04name\x18\x02 \x01(\tR\x04name\x12+\n" + - "\x04kind\x18\x03 \x01(\x0e2\x17.productv1.CategoryKindR\x04kind\"|\n" + + "\x04kind\x18\x03 \x01(\x0e2\x17.productv1.CategoryKindR\x04kind\x12B\n" + + "\rsubcategories\x18\x05 \x01(\v2\x1c.productv1.ListOfSubcategoryR\rsubcategories\"|\n" + "\x0eCategoryFilter\x123\n" + "\bcategory\x18\x01 \x01(\x0e2\x17.productv1.CategoryKindR\bcategory\x125\n" + "\n" + @@ -9596,7 +10085,12 @@ const file_product_proto_rawDesc = "" + "\n" + "product_id\x18\x01 \x01(\tR\tproductId\x12\x1a\n" + "\bquantity\x18\x02 \x01(\x05R\bquantity\x125\n" + - "\tmodifiers\x18\x03 \x01(\v2\x17.productv1.ListOfStringR\tmodifiers\"^\n" + + "\tmodifiers\x18\x03 \x01(\v2\x17.productv1.ListOfStringR\tmodifiers\"\x8e\x01\n" + + "\vSubcategory\x12\x0e\n" + + "\x02id\x18\x01 \x01(\tR\x02id\x12\x12\n" + + "\x04name\x18\x02 \x01(\tR\x04name\x12>\n" + + "\vdescription\x18\x03 \x01(\v2\x1c.google.protobuf.StringValueR\vdescription\x12\x1b\n" + + "\tis_active\x18\x04 \x01(\bR\bisActive\"^\n" + "\x03Cat\x12\x0e\n" + "\x02id\x18\x01 \x01(\tR\x02id\x12\x12\n" + "\x04name\x18\x02 \x01(\tR\x04name\x12\x12\n" + @@ -9623,13 +10117,20 @@ const file_product_proto_rawDesc = "" + "\tmax_price\x18\x02 \x01(\v2\x1c.google.protobuf.DoubleValueR\bmaxPrice\x125\n" + "\bin_stock\x18\x03 \x01(\v2\x1a.google.protobuf.BoolValueR\ainStock\x12=\n" + "\vsearch_term\x18\x04 \x01(\v2\x1c.google.protobuf.StringValueR\n" + + "searchTerm\"\xbc\x02\n" + + "\x15SubcategoryItemFilter\x129\n" + + "\tmin_price\x18\x01 \x01(\v2\x1c.google.protobuf.DoubleValueR\bminPrice\x129\n" + + "\tmax_price\x18\x02 \x01(\v2\x1c.google.protobuf.DoubleValueR\bmaxPrice\x125\n" + + "\bin_stock\x18\x03 \x01(\v2\x1a.google.protobuf.BoolValueR\ainStock\x127\n" + + "\tis_active\x18\x04 \x01(\v2\x1a.google.protobuf.BoolValueR\bisActive\x12=\n" + + "\vsearch_term\x18\x05 \x01(\v2\x1c.google.protobuf.StringValueR\n" + "searchTerm*\x9a\x01\n" + "\fCategoryKind\x12\x1d\n" + "\x19CATEGORY_KIND_UNSPECIFIED\x10\x00\x12\x16\n" + "\x12CATEGORY_KIND_BOOK\x10\x01\x12\x1d\n" + "\x19CATEGORY_KIND_ELECTRONICS\x10\x02\x12\x1b\n" + "\x17CATEGORY_KIND_FURNITURE\x10\x03\x12\x17\n" + - "\x13CATEGORY_KIND_OTHER\x10\x042\xb8'\n" + + "\x13CATEGORY_KIND_OTHER\x10\x042\xb8(\n" + "\x0eProductService\x12`\n" + "\x11LookupProductById\x12#.productv1.LookupProductByIdRequest\x1a$.productv1.LookupProductByIdResponse\"\x00\x12`\n" + "\x11LookupStorageById\x12#.productv1.LookupStorageByIdRequest\x1a$.productv1.LookupStorageByIdResponse\"\x00\x12f\n" + @@ -9677,7 +10178,8 @@ const file_product_proto_rawDesc = "" + "\tQueryUser\x12\x1b.productv1.QueryUserRequest\x1a\x1c.productv1.QueryUserResponse\"\x00\x12K\n" + "\n" + "QueryUsers\x12\x1c.productv1.QueryUsersRequest\x1a\x1d.productv1.QueryUsersResponse\"\x00\x12~\n" + - "\x1bResolveCategoryProductCount\x12-.productv1.ResolveCategoryProductCountRequest\x1a..productv1.ResolveCategoryProductCountResponse\"\x00B%Z#cosmo/pkg/proto/productv1;productv1b\x06proto3" + "\x1bResolveCategoryProductCount\x12-.productv1.ResolveCategoryProductCountRequest\x1a..productv1.ResolveCategoryProductCountResponse\"\x00\x12~\n" + + "\x1bResolveSubcategoryItemCount\x12-.productv1.ResolveSubcategoryItemCountRequest\x1a..productv1.ResolveSubcategoryItemCountResponse\"\x00B%Z#cosmo/pkg/proto/productv1;productv1b\x06proto3" var ( file_product_proto_rawDescOnce sync.Once @@ -9692,7 +10194,7 @@ func file_product_proto_rawDescGZIP() []byte { } var file_product_proto_enumTypes = make([]protoimpl.EnumInfo, 1) -var file_product_proto_msgTypes = make([]protoimpl.MessageInfo, 177) +var file_product_proto_msgTypes = make([]protoimpl.MessageInfo, 186) var file_product_proto_goTypes = []any{ (CategoryKind)(0), // 0: productv1.CategoryKind (*ListOfAuthorFilter)(nil), // 1: productv1.ListOfAuthorFilter @@ -9712,465 +10214,489 @@ var file_product_proto_goTypes = []any{ (*ListOfOrderLine)(nil), // 15: productv1.ListOfOrderLine (*ListOfProduct)(nil), // 16: productv1.ListOfProduct (*ListOfString)(nil), // 17: productv1.ListOfString - (*ListOfUser)(nil), // 18: productv1.ListOfUser - (*ListOfUserInput)(nil), // 19: productv1.ListOfUserInput - (*LookupProductByIdRequestKey)(nil), // 20: productv1.LookupProductByIdRequestKey - (*LookupProductByIdRequest)(nil), // 21: productv1.LookupProductByIdRequest - (*LookupProductByIdResponse)(nil), // 22: productv1.LookupProductByIdResponse - (*LookupStorageByIdRequestKey)(nil), // 23: productv1.LookupStorageByIdRequestKey - (*LookupStorageByIdRequest)(nil), // 24: productv1.LookupStorageByIdRequest - (*LookupStorageByIdResponse)(nil), // 25: productv1.LookupStorageByIdResponse - (*LookupWarehouseByIdRequestKey)(nil), // 26: productv1.LookupWarehouseByIdRequestKey - (*LookupWarehouseByIdRequest)(nil), // 27: productv1.LookupWarehouseByIdRequest - (*LookupWarehouseByIdResponse)(nil), // 28: productv1.LookupWarehouseByIdResponse - (*QueryUsersRequest)(nil), // 29: productv1.QueryUsersRequest - (*QueryUsersResponse)(nil), // 30: productv1.QueryUsersResponse - (*QueryUserRequest)(nil), // 31: productv1.QueryUserRequest - (*QueryUserResponse)(nil), // 32: productv1.QueryUserResponse - (*QueryNestedTypeRequest)(nil), // 33: productv1.QueryNestedTypeRequest - (*QueryNestedTypeResponse)(nil), // 34: productv1.QueryNestedTypeResponse - (*QueryRecursiveTypeRequest)(nil), // 35: productv1.QueryRecursiveTypeRequest - (*QueryRecursiveTypeResponse)(nil), // 36: productv1.QueryRecursiveTypeResponse - (*QueryTypeFilterWithArgumentsRequest)(nil), // 37: productv1.QueryTypeFilterWithArgumentsRequest - (*QueryTypeFilterWithArgumentsResponse)(nil), // 38: productv1.QueryTypeFilterWithArgumentsResponse - (*QueryTypeWithMultipleFilterFieldsRequest)(nil), // 39: productv1.QueryTypeWithMultipleFilterFieldsRequest - (*QueryTypeWithMultipleFilterFieldsResponse)(nil), // 40: productv1.QueryTypeWithMultipleFilterFieldsResponse - (*QueryComplexFilterTypeRequest)(nil), // 41: productv1.QueryComplexFilterTypeRequest - (*QueryComplexFilterTypeResponse)(nil), // 42: productv1.QueryComplexFilterTypeResponse - (*QueryCalculateTotalsRequest)(nil), // 43: productv1.QueryCalculateTotalsRequest - (*QueryCalculateTotalsResponse)(nil), // 44: productv1.QueryCalculateTotalsResponse - (*QueryCategoriesRequest)(nil), // 45: productv1.QueryCategoriesRequest - (*QueryCategoriesResponse)(nil), // 46: productv1.QueryCategoriesResponse - (*QueryCategoriesByKindRequest)(nil), // 47: productv1.QueryCategoriesByKindRequest - (*QueryCategoriesByKindResponse)(nil), // 48: productv1.QueryCategoriesByKindResponse - (*QueryCategoriesByKindsRequest)(nil), // 49: productv1.QueryCategoriesByKindsRequest - (*QueryCategoriesByKindsResponse)(nil), // 50: productv1.QueryCategoriesByKindsResponse - (*QueryFilterCategoriesRequest)(nil), // 51: productv1.QueryFilterCategoriesRequest - (*QueryFilterCategoriesResponse)(nil), // 52: productv1.QueryFilterCategoriesResponse - (*QueryRandomPetRequest)(nil), // 53: productv1.QueryRandomPetRequest - (*QueryRandomPetResponse)(nil), // 54: productv1.QueryRandomPetResponse - (*QueryAllPetsRequest)(nil), // 55: productv1.QueryAllPetsRequest - (*QueryAllPetsResponse)(nil), // 56: productv1.QueryAllPetsResponse - (*QuerySearchRequest)(nil), // 57: productv1.QuerySearchRequest - (*QuerySearchResponse)(nil), // 58: productv1.QuerySearchResponse - (*QueryRandomSearchResultRequest)(nil), // 59: productv1.QueryRandomSearchResultRequest - (*QueryRandomSearchResultResponse)(nil), // 60: productv1.QueryRandomSearchResultResponse - (*QueryNullableFieldsTypeRequest)(nil), // 61: productv1.QueryNullableFieldsTypeRequest - (*QueryNullableFieldsTypeResponse)(nil), // 62: productv1.QueryNullableFieldsTypeResponse - (*QueryNullableFieldsTypeByIdRequest)(nil), // 63: productv1.QueryNullableFieldsTypeByIdRequest - (*QueryNullableFieldsTypeByIdResponse)(nil), // 64: productv1.QueryNullableFieldsTypeByIdResponse - (*QueryNullableFieldsTypeWithFilterRequest)(nil), // 65: productv1.QueryNullableFieldsTypeWithFilterRequest - (*QueryNullableFieldsTypeWithFilterResponse)(nil), // 66: productv1.QueryNullableFieldsTypeWithFilterResponse - (*QueryAllNullableFieldsTypesRequest)(nil), // 67: productv1.QueryAllNullableFieldsTypesRequest - (*QueryAllNullableFieldsTypesResponse)(nil), // 68: productv1.QueryAllNullableFieldsTypesResponse - (*QueryBlogPostRequest)(nil), // 69: productv1.QueryBlogPostRequest - (*QueryBlogPostResponse)(nil), // 70: productv1.QueryBlogPostResponse - (*QueryBlogPostByIdRequest)(nil), // 71: productv1.QueryBlogPostByIdRequest - (*QueryBlogPostByIdResponse)(nil), // 72: productv1.QueryBlogPostByIdResponse - (*QueryBlogPostsWithFilterRequest)(nil), // 73: productv1.QueryBlogPostsWithFilterRequest - (*QueryBlogPostsWithFilterResponse)(nil), // 74: productv1.QueryBlogPostsWithFilterResponse - (*QueryAllBlogPostsRequest)(nil), // 75: productv1.QueryAllBlogPostsRequest - (*QueryAllBlogPostsResponse)(nil), // 76: productv1.QueryAllBlogPostsResponse - (*QueryAuthorRequest)(nil), // 77: productv1.QueryAuthorRequest - (*QueryAuthorResponse)(nil), // 78: productv1.QueryAuthorResponse - (*QueryAuthorByIdRequest)(nil), // 79: productv1.QueryAuthorByIdRequest - (*QueryAuthorByIdResponse)(nil), // 80: productv1.QueryAuthorByIdResponse - (*QueryAuthorsWithFilterRequest)(nil), // 81: productv1.QueryAuthorsWithFilterRequest - (*QueryAuthorsWithFilterResponse)(nil), // 82: productv1.QueryAuthorsWithFilterResponse - (*QueryAllAuthorsRequest)(nil), // 83: productv1.QueryAllAuthorsRequest - (*QueryAllAuthorsResponse)(nil), // 84: productv1.QueryAllAuthorsResponse - (*QueryBulkSearchAuthorsRequest)(nil), // 85: productv1.QueryBulkSearchAuthorsRequest - (*QueryBulkSearchAuthorsResponse)(nil), // 86: productv1.QueryBulkSearchAuthorsResponse - (*QueryBulkSearchBlogPostsRequest)(nil), // 87: productv1.QueryBulkSearchBlogPostsRequest - (*QueryBulkSearchBlogPostsResponse)(nil), // 88: productv1.QueryBulkSearchBlogPostsResponse - (*MutationCreateUserRequest)(nil), // 89: productv1.MutationCreateUserRequest - (*MutationCreateUserResponse)(nil), // 90: productv1.MutationCreateUserResponse - (*MutationPerformActionRequest)(nil), // 91: productv1.MutationPerformActionRequest - (*MutationPerformActionResponse)(nil), // 92: productv1.MutationPerformActionResponse - (*MutationCreateNullableFieldsTypeRequest)(nil), // 93: productv1.MutationCreateNullableFieldsTypeRequest - (*MutationCreateNullableFieldsTypeResponse)(nil), // 94: productv1.MutationCreateNullableFieldsTypeResponse - (*MutationUpdateNullableFieldsTypeRequest)(nil), // 95: productv1.MutationUpdateNullableFieldsTypeRequest - (*MutationUpdateNullableFieldsTypeResponse)(nil), // 96: productv1.MutationUpdateNullableFieldsTypeResponse - (*MutationCreateBlogPostRequest)(nil), // 97: productv1.MutationCreateBlogPostRequest - (*MutationCreateBlogPostResponse)(nil), // 98: productv1.MutationCreateBlogPostResponse - (*MutationUpdateBlogPostRequest)(nil), // 99: productv1.MutationUpdateBlogPostRequest - (*MutationUpdateBlogPostResponse)(nil), // 100: productv1.MutationUpdateBlogPostResponse - (*MutationCreateAuthorRequest)(nil), // 101: productv1.MutationCreateAuthorRequest - (*MutationCreateAuthorResponse)(nil), // 102: productv1.MutationCreateAuthorResponse - (*MutationUpdateAuthorRequest)(nil), // 103: productv1.MutationUpdateAuthorRequest - (*MutationUpdateAuthorResponse)(nil), // 104: productv1.MutationUpdateAuthorResponse - (*MutationBulkCreateAuthorsRequest)(nil), // 105: productv1.MutationBulkCreateAuthorsRequest - (*MutationBulkCreateAuthorsResponse)(nil), // 106: productv1.MutationBulkCreateAuthorsResponse - (*MutationBulkUpdateAuthorsRequest)(nil), // 107: productv1.MutationBulkUpdateAuthorsRequest - (*MutationBulkUpdateAuthorsResponse)(nil), // 108: productv1.MutationBulkUpdateAuthorsResponse - (*MutationBulkCreateBlogPostsRequest)(nil), // 109: productv1.MutationBulkCreateBlogPostsRequest - (*MutationBulkCreateBlogPostsResponse)(nil), // 110: productv1.MutationBulkCreateBlogPostsResponse - (*MutationBulkUpdateBlogPostsRequest)(nil), // 111: productv1.MutationBulkUpdateBlogPostsRequest - (*MutationBulkUpdateBlogPostsResponse)(nil), // 112: productv1.MutationBulkUpdateBlogPostsResponse - (*CategoryProductCountArgs)(nil), // 113: productv1.CategoryProductCountArgs - (*CategoryProductCountContext)(nil), // 114: productv1.CategoryProductCountContext - (*ResolveCategoryProductCountRequest)(nil), // 115: productv1.ResolveCategoryProductCountRequest - (*ResolveCategoryProductCountResponseResult)(nil), // 116: productv1.ResolveCategoryProductCountResponseResult - (*ResolveCategoryProductCountResponse)(nil), // 117: productv1.ResolveCategoryProductCountResponse - (*Product)(nil), // 118: productv1.Product - (*Storage)(nil), // 119: productv1.Storage - (*Warehouse)(nil), // 120: productv1.Warehouse - (*User)(nil), // 121: productv1.User - (*NestedTypeA)(nil), // 122: productv1.NestedTypeA - (*RecursiveType)(nil), // 123: productv1.RecursiveType - (*TypeWithMultipleFilterFields)(nil), // 124: productv1.TypeWithMultipleFilterFields - (*FilterTypeInput)(nil), // 125: productv1.FilterTypeInput - (*ComplexFilterTypeInput)(nil), // 126: productv1.ComplexFilterTypeInput - (*TypeWithComplexFilterInput)(nil), // 127: productv1.TypeWithComplexFilterInput - (*OrderInput)(nil), // 128: productv1.OrderInput - (*Order)(nil), // 129: productv1.Order - (*Category)(nil), // 130: productv1.Category - (*CategoryFilter)(nil), // 131: productv1.CategoryFilter - (*Animal)(nil), // 132: productv1.Animal - (*SearchInput)(nil), // 133: productv1.SearchInput - (*SearchResult)(nil), // 134: productv1.SearchResult - (*NullableFieldsType)(nil), // 135: productv1.NullableFieldsType - (*NullableFieldsFilter)(nil), // 136: productv1.NullableFieldsFilter - (*BlogPost)(nil), // 137: productv1.BlogPost - (*BlogPostFilter)(nil), // 138: productv1.BlogPostFilter - (*Author)(nil), // 139: productv1.Author - (*AuthorFilter)(nil), // 140: productv1.AuthorFilter - (*UserInput)(nil), // 141: productv1.UserInput - (*ActionInput)(nil), // 142: productv1.ActionInput - (*ActionResult)(nil), // 143: productv1.ActionResult - (*NullableFieldsInput)(nil), // 144: productv1.NullableFieldsInput - (*BlogPostInput)(nil), // 145: productv1.BlogPostInput - (*AuthorInput)(nil), // 146: productv1.AuthorInput - (*NestedTypeB)(nil), // 147: productv1.NestedTypeB - (*NestedTypeC)(nil), // 148: productv1.NestedTypeC - (*FilterType)(nil), // 149: productv1.FilterType - (*Pagination)(nil), // 150: productv1.Pagination - (*OrderLineInput)(nil), // 151: productv1.OrderLineInput - (*OrderLine)(nil), // 152: productv1.OrderLine - (*Cat)(nil), // 153: productv1.Cat - (*Dog)(nil), // 154: productv1.Dog - (*ActionSuccess)(nil), // 155: productv1.ActionSuccess - (*ActionError)(nil), // 156: productv1.ActionError - (*CategoryInput)(nil), // 157: productv1.CategoryInput - (*ProductCountFilter)(nil), // 158: productv1.ProductCountFilter - (*ListOfAuthorFilter_List)(nil), // 159: productv1.ListOfAuthorFilter.List - (*ListOfAuthorInput_List)(nil), // 160: productv1.ListOfAuthorInput.List - (*ListOfBlogPost_List)(nil), // 161: productv1.ListOfBlogPost.List - (*ListOfBlogPostFilter_List)(nil), // 162: productv1.ListOfBlogPostFilter.List - (*ListOfBlogPostInput_List)(nil), // 163: productv1.ListOfBlogPostInput.List - (*ListOfBoolean_List)(nil), // 164: productv1.ListOfBoolean.List - (*ListOfCategory_List)(nil), // 165: productv1.ListOfCategory.List - (*ListOfCategoryInput_List)(nil), // 166: productv1.ListOfCategoryInput.List - (*ListOfFloat_List)(nil), // 167: productv1.ListOfFloat.List - (*ListOfListOfCategory_List)(nil), // 168: productv1.ListOfListOfCategory.List - (*ListOfListOfCategoryInput_List)(nil), // 169: productv1.ListOfListOfCategoryInput.List - (*ListOfListOfString_List)(nil), // 170: productv1.ListOfListOfString.List - (*ListOfListOfUser_List)(nil), // 171: productv1.ListOfListOfUser.List - (*ListOfListOfUserInput_List)(nil), // 172: productv1.ListOfListOfUserInput.List - (*ListOfOrderLine_List)(nil), // 173: productv1.ListOfOrderLine.List - (*ListOfProduct_List)(nil), // 174: productv1.ListOfProduct.List - (*ListOfString_List)(nil), // 175: productv1.ListOfString.List - (*ListOfUser_List)(nil), // 176: productv1.ListOfUser.List - (*ListOfUserInput_List)(nil), // 177: productv1.ListOfUserInput.List - (*wrapperspb.Int32Value)(nil), // 178: google.protobuf.Int32Value - (*wrapperspb.StringValue)(nil), // 179: google.protobuf.StringValue - (*wrapperspb.DoubleValue)(nil), // 180: google.protobuf.DoubleValue - (*wrapperspb.BoolValue)(nil), // 181: google.protobuf.BoolValue + (*ListOfSubcategory)(nil), // 18: productv1.ListOfSubcategory + (*ListOfUser)(nil), // 19: productv1.ListOfUser + (*ListOfUserInput)(nil), // 20: productv1.ListOfUserInput + (*LookupProductByIdRequestKey)(nil), // 21: productv1.LookupProductByIdRequestKey + (*LookupProductByIdRequest)(nil), // 22: productv1.LookupProductByIdRequest + (*LookupProductByIdResponse)(nil), // 23: productv1.LookupProductByIdResponse + (*LookupStorageByIdRequestKey)(nil), // 24: productv1.LookupStorageByIdRequestKey + (*LookupStorageByIdRequest)(nil), // 25: productv1.LookupStorageByIdRequest + (*LookupStorageByIdResponse)(nil), // 26: productv1.LookupStorageByIdResponse + (*LookupWarehouseByIdRequestKey)(nil), // 27: productv1.LookupWarehouseByIdRequestKey + (*LookupWarehouseByIdRequest)(nil), // 28: productv1.LookupWarehouseByIdRequest + (*LookupWarehouseByIdResponse)(nil), // 29: productv1.LookupWarehouseByIdResponse + (*QueryUsersRequest)(nil), // 30: productv1.QueryUsersRequest + (*QueryUsersResponse)(nil), // 31: productv1.QueryUsersResponse + (*QueryUserRequest)(nil), // 32: productv1.QueryUserRequest + (*QueryUserResponse)(nil), // 33: productv1.QueryUserResponse + (*QueryNestedTypeRequest)(nil), // 34: productv1.QueryNestedTypeRequest + (*QueryNestedTypeResponse)(nil), // 35: productv1.QueryNestedTypeResponse + (*QueryRecursiveTypeRequest)(nil), // 36: productv1.QueryRecursiveTypeRequest + (*QueryRecursiveTypeResponse)(nil), // 37: productv1.QueryRecursiveTypeResponse + (*QueryTypeFilterWithArgumentsRequest)(nil), // 38: productv1.QueryTypeFilterWithArgumentsRequest + (*QueryTypeFilterWithArgumentsResponse)(nil), // 39: productv1.QueryTypeFilterWithArgumentsResponse + (*QueryTypeWithMultipleFilterFieldsRequest)(nil), // 40: productv1.QueryTypeWithMultipleFilterFieldsRequest + (*QueryTypeWithMultipleFilterFieldsResponse)(nil), // 41: productv1.QueryTypeWithMultipleFilterFieldsResponse + (*QueryComplexFilterTypeRequest)(nil), // 42: productv1.QueryComplexFilterTypeRequest + (*QueryComplexFilterTypeResponse)(nil), // 43: productv1.QueryComplexFilterTypeResponse + (*QueryCalculateTotalsRequest)(nil), // 44: productv1.QueryCalculateTotalsRequest + (*QueryCalculateTotalsResponse)(nil), // 45: productv1.QueryCalculateTotalsResponse + (*QueryCategoriesRequest)(nil), // 46: productv1.QueryCategoriesRequest + (*QueryCategoriesResponse)(nil), // 47: productv1.QueryCategoriesResponse + (*QueryCategoriesByKindRequest)(nil), // 48: productv1.QueryCategoriesByKindRequest + (*QueryCategoriesByKindResponse)(nil), // 49: productv1.QueryCategoriesByKindResponse + (*QueryCategoriesByKindsRequest)(nil), // 50: productv1.QueryCategoriesByKindsRequest + (*QueryCategoriesByKindsResponse)(nil), // 51: productv1.QueryCategoriesByKindsResponse + (*QueryFilterCategoriesRequest)(nil), // 52: productv1.QueryFilterCategoriesRequest + (*QueryFilterCategoriesResponse)(nil), // 53: productv1.QueryFilterCategoriesResponse + (*QueryRandomPetRequest)(nil), // 54: productv1.QueryRandomPetRequest + (*QueryRandomPetResponse)(nil), // 55: productv1.QueryRandomPetResponse + (*QueryAllPetsRequest)(nil), // 56: productv1.QueryAllPetsRequest + (*QueryAllPetsResponse)(nil), // 57: productv1.QueryAllPetsResponse + (*QuerySearchRequest)(nil), // 58: productv1.QuerySearchRequest + (*QuerySearchResponse)(nil), // 59: productv1.QuerySearchResponse + (*QueryRandomSearchResultRequest)(nil), // 60: productv1.QueryRandomSearchResultRequest + (*QueryRandomSearchResultResponse)(nil), // 61: productv1.QueryRandomSearchResultResponse + (*QueryNullableFieldsTypeRequest)(nil), // 62: productv1.QueryNullableFieldsTypeRequest + (*QueryNullableFieldsTypeResponse)(nil), // 63: productv1.QueryNullableFieldsTypeResponse + (*QueryNullableFieldsTypeByIdRequest)(nil), // 64: productv1.QueryNullableFieldsTypeByIdRequest + (*QueryNullableFieldsTypeByIdResponse)(nil), // 65: productv1.QueryNullableFieldsTypeByIdResponse + (*QueryNullableFieldsTypeWithFilterRequest)(nil), // 66: productv1.QueryNullableFieldsTypeWithFilterRequest + (*QueryNullableFieldsTypeWithFilterResponse)(nil), // 67: productv1.QueryNullableFieldsTypeWithFilterResponse + (*QueryAllNullableFieldsTypesRequest)(nil), // 68: productv1.QueryAllNullableFieldsTypesRequest + (*QueryAllNullableFieldsTypesResponse)(nil), // 69: productv1.QueryAllNullableFieldsTypesResponse + (*QueryBlogPostRequest)(nil), // 70: productv1.QueryBlogPostRequest + (*QueryBlogPostResponse)(nil), // 71: productv1.QueryBlogPostResponse + (*QueryBlogPostByIdRequest)(nil), // 72: productv1.QueryBlogPostByIdRequest + (*QueryBlogPostByIdResponse)(nil), // 73: productv1.QueryBlogPostByIdResponse + (*QueryBlogPostsWithFilterRequest)(nil), // 74: productv1.QueryBlogPostsWithFilterRequest + (*QueryBlogPostsWithFilterResponse)(nil), // 75: productv1.QueryBlogPostsWithFilterResponse + (*QueryAllBlogPostsRequest)(nil), // 76: productv1.QueryAllBlogPostsRequest + (*QueryAllBlogPostsResponse)(nil), // 77: productv1.QueryAllBlogPostsResponse + (*QueryAuthorRequest)(nil), // 78: productv1.QueryAuthorRequest + (*QueryAuthorResponse)(nil), // 79: productv1.QueryAuthorResponse + (*QueryAuthorByIdRequest)(nil), // 80: productv1.QueryAuthorByIdRequest + (*QueryAuthorByIdResponse)(nil), // 81: productv1.QueryAuthorByIdResponse + (*QueryAuthorsWithFilterRequest)(nil), // 82: productv1.QueryAuthorsWithFilterRequest + (*QueryAuthorsWithFilterResponse)(nil), // 83: productv1.QueryAuthorsWithFilterResponse + (*QueryAllAuthorsRequest)(nil), // 84: productv1.QueryAllAuthorsRequest + (*QueryAllAuthorsResponse)(nil), // 85: productv1.QueryAllAuthorsResponse + (*QueryBulkSearchAuthorsRequest)(nil), // 86: productv1.QueryBulkSearchAuthorsRequest + (*QueryBulkSearchAuthorsResponse)(nil), // 87: productv1.QueryBulkSearchAuthorsResponse + (*QueryBulkSearchBlogPostsRequest)(nil), // 88: productv1.QueryBulkSearchBlogPostsRequest + (*QueryBulkSearchBlogPostsResponse)(nil), // 89: productv1.QueryBulkSearchBlogPostsResponse + (*MutationCreateUserRequest)(nil), // 90: productv1.MutationCreateUserRequest + (*MutationCreateUserResponse)(nil), // 91: productv1.MutationCreateUserResponse + (*MutationPerformActionRequest)(nil), // 92: productv1.MutationPerformActionRequest + (*MutationPerformActionResponse)(nil), // 93: productv1.MutationPerformActionResponse + (*MutationCreateNullableFieldsTypeRequest)(nil), // 94: productv1.MutationCreateNullableFieldsTypeRequest + (*MutationCreateNullableFieldsTypeResponse)(nil), // 95: productv1.MutationCreateNullableFieldsTypeResponse + (*MutationUpdateNullableFieldsTypeRequest)(nil), // 96: productv1.MutationUpdateNullableFieldsTypeRequest + (*MutationUpdateNullableFieldsTypeResponse)(nil), // 97: productv1.MutationUpdateNullableFieldsTypeResponse + (*MutationCreateBlogPostRequest)(nil), // 98: productv1.MutationCreateBlogPostRequest + (*MutationCreateBlogPostResponse)(nil), // 99: productv1.MutationCreateBlogPostResponse + (*MutationUpdateBlogPostRequest)(nil), // 100: productv1.MutationUpdateBlogPostRequest + (*MutationUpdateBlogPostResponse)(nil), // 101: productv1.MutationUpdateBlogPostResponse + (*MutationCreateAuthorRequest)(nil), // 102: productv1.MutationCreateAuthorRequest + (*MutationCreateAuthorResponse)(nil), // 103: productv1.MutationCreateAuthorResponse + (*MutationUpdateAuthorRequest)(nil), // 104: productv1.MutationUpdateAuthorRequest + (*MutationUpdateAuthorResponse)(nil), // 105: productv1.MutationUpdateAuthorResponse + (*MutationBulkCreateAuthorsRequest)(nil), // 106: productv1.MutationBulkCreateAuthorsRequest + (*MutationBulkCreateAuthorsResponse)(nil), // 107: productv1.MutationBulkCreateAuthorsResponse + (*MutationBulkUpdateAuthorsRequest)(nil), // 108: productv1.MutationBulkUpdateAuthorsRequest + (*MutationBulkUpdateAuthorsResponse)(nil), // 109: productv1.MutationBulkUpdateAuthorsResponse + (*MutationBulkCreateBlogPostsRequest)(nil), // 110: productv1.MutationBulkCreateBlogPostsRequest + (*MutationBulkCreateBlogPostsResponse)(nil), // 111: productv1.MutationBulkCreateBlogPostsResponse + (*MutationBulkUpdateBlogPostsRequest)(nil), // 112: productv1.MutationBulkUpdateBlogPostsRequest + (*MutationBulkUpdateBlogPostsResponse)(nil), // 113: productv1.MutationBulkUpdateBlogPostsResponse + (*ResolveCategoryProductCountArgs)(nil), // 114: productv1.ResolveCategoryProductCountArgs + (*ResolveCategoryProductCountContext)(nil), // 115: productv1.ResolveCategoryProductCountContext + (*ResolveCategoryProductCountRequest)(nil), // 116: productv1.ResolveCategoryProductCountRequest + (*ResolveCategoryProductCountResult)(nil), // 117: productv1.ResolveCategoryProductCountResult + (*ResolveCategoryProductCountResponse)(nil), // 118: productv1.ResolveCategoryProductCountResponse + (*ResolveSubcategoryItemCountArgs)(nil), // 119: productv1.ResolveSubcategoryItemCountArgs + (*ResolveSubcategoryItemCountContext)(nil), // 120: productv1.ResolveSubcategoryItemCountContext + (*ResolveSubcategoryItemCountRequest)(nil), // 121: productv1.ResolveSubcategoryItemCountRequest + (*ResolveSubcategoryItemCountResult)(nil), // 122: productv1.ResolveSubcategoryItemCountResult + (*ResolveSubcategoryItemCountResponse)(nil), // 123: productv1.ResolveSubcategoryItemCountResponse + (*Product)(nil), // 124: productv1.Product + (*Storage)(nil), // 125: productv1.Storage + (*Warehouse)(nil), // 126: productv1.Warehouse + (*User)(nil), // 127: productv1.User + (*NestedTypeA)(nil), // 128: productv1.NestedTypeA + (*RecursiveType)(nil), // 129: productv1.RecursiveType + (*TypeWithMultipleFilterFields)(nil), // 130: productv1.TypeWithMultipleFilterFields + (*FilterTypeInput)(nil), // 131: productv1.FilterTypeInput + (*ComplexFilterTypeInput)(nil), // 132: productv1.ComplexFilterTypeInput + (*TypeWithComplexFilterInput)(nil), // 133: productv1.TypeWithComplexFilterInput + (*OrderInput)(nil), // 134: productv1.OrderInput + (*Order)(nil), // 135: productv1.Order + (*Category)(nil), // 136: productv1.Category + (*CategoryFilter)(nil), // 137: productv1.CategoryFilter + (*Animal)(nil), // 138: productv1.Animal + (*SearchInput)(nil), // 139: productv1.SearchInput + (*SearchResult)(nil), // 140: productv1.SearchResult + (*NullableFieldsType)(nil), // 141: productv1.NullableFieldsType + (*NullableFieldsFilter)(nil), // 142: productv1.NullableFieldsFilter + (*BlogPost)(nil), // 143: productv1.BlogPost + (*BlogPostFilter)(nil), // 144: productv1.BlogPostFilter + (*Author)(nil), // 145: productv1.Author + (*AuthorFilter)(nil), // 146: productv1.AuthorFilter + (*UserInput)(nil), // 147: productv1.UserInput + (*ActionInput)(nil), // 148: productv1.ActionInput + (*ActionResult)(nil), // 149: productv1.ActionResult + (*NullableFieldsInput)(nil), // 150: productv1.NullableFieldsInput + (*BlogPostInput)(nil), // 151: productv1.BlogPostInput + (*AuthorInput)(nil), // 152: productv1.AuthorInput + (*NestedTypeB)(nil), // 153: productv1.NestedTypeB + (*NestedTypeC)(nil), // 154: productv1.NestedTypeC + (*FilterType)(nil), // 155: productv1.FilterType + (*Pagination)(nil), // 156: productv1.Pagination + (*OrderLineInput)(nil), // 157: productv1.OrderLineInput + (*OrderLine)(nil), // 158: productv1.OrderLine + (*Subcategory)(nil), // 159: productv1.Subcategory + (*Cat)(nil), // 160: productv1.Cat + (*Dog)(nil), // 161: productv1.Dog + (*ActionSuccess)(nil), // 162: productv1.ActionSuccess + (*ActionError)(nil), // 163: productv1.ActionError + (*CategoryInput)(nil), // 164: productv1.CategoryInput + (*ProductCountFilter)(nil), // 165: productv1.ProductCountFilter + (*SubcategoryItemFilter)(nil), // 166: productv1.SubcategoryItemFilter + (*ListOfAuthorFilter_List)(nil), // 167: productv1.ListOfAuthorFilter.List + (*ListOfAuthorInput_List)(nil), // 168: productv1.ListOfAuthorInput.List + (*ListOfBlogPost_List)(nil), // 169: productv1.ListOfBlogPost.List + (*ListOfBlogPostFilter_List)(nil), // 170: productv1.ListOfBlogPostFilter.List + (*ListOfBlogPostInput_List)(nil), // 171: productv1.ListOfBlogPostInput.List + (*ListOfBoolean_List)(nil), // 172: productv1.ListOfBoolean.List + (*ListOfCategory_List)(nil), // 173: productv1.ListOfCategory.List + (*ListOfCategoryInput_List)(nil), // 174: productv1.ListOfCategoryInput.List + (*ListOfFloat_List)(nil), // 175: productv1.ListOfFloat.List + (*ListOfListOfCategory_List)(nil), // 176: productv1.ListOfListOfCategory.List + (*ListOfListOfCategoryInput_List)(nil), // 177: productv1.ListOfListOfCategoryInput.List + (*ListOfListOfString_List)(nil), // 178: productv1.ListOfListOfString.List + (*ListOfListOfUser_List)(nil), // 179: productv1.ListOfListOfUser.List + (*ListOfListOfUserInput_List)(nil), // 180: productv1.ListOfListOfUserInput.List + (*ListOfOrderLine_List)(nil), // 181: productv1.ListOfOrderLine.List + (*ListOfProduct_List)(nil), // 182: productv1.ListOfProduct.List + (*ListOfString_List)(nil), // 183: productv1.ListOfString.List + (*ListOfSubcategory_List)(nil), // 184: productv1.ListOfSubcategory.List + (*ListOfUser_List)(nil), // 185: productv1.ListOfUser.List + (*ListOfUserInput_List)(nil), // 186: productv1.ListOfUserInput.List + (*wrapperspb.Int32Value)(nil), // 187: google.protobuf.Int32Value + (*wrapperspb.StringValue)(nil), // 188: google.protobuf.StringValue + (*wrapperspb.DoubleValue)(nil), // 189: google.protobuf.DoubleValue + (*wrapperspb.BoolValue)(nil), // 190: google.protobuf.BoolValue } var file_product_proto_depIdxs = []int32{ - 159, // 0: productv1.ListOfAuthorFilter.list:type_name -> productv1.ListOfAuthorFilter.List - 160, // 1: productv1.ListOfAuthorInput.list:type_name -> productv1.ListOfAuthorInput.List - 161, // 2: productv1.ListOfBlogPost.list:type_name -> productv1.ListOfBlogPost.List - 162, // 3: productv1.ListOfBlogPostFilter.list:type_name -> productv1.ListOfBlogPostFilter.List - 163, // 4: productv1.ListOfBlogPostInput.list:type_name -> productv1.ListOfBlogPostInput.List - 164, // 5: productv1.ListOfBoolean.list:type_name -> productv1.ListOfBoolean.List - 165, // 6: productv1.ListOfCategory.list:type_name -> productv1.ListOfCategory.List - 166, // 7: productv1.ListOfCategoryInput.list:type_name -> productv1.ListOfCategoryInput.List - 167, // 8: productv1.ListOfFloat.list:type_name -> productv1.ListOfFloat.List - 168, // 9: productv1.ListOfListOfCategory.list:type_name -> productv1.ListOfListOfCategory.List - 169, // 10: productv1.ListOfListOfCategoryInput.list:type_name -> productv1.ListOfListOfCategoryInput.List - 170, // 11: productv1.ListOfListOfString.list:type_name -> productv1.ListOfListOfString.List - 171, // 12: productv1.ListOfListOfUser.list:type_name -> productv1.ListOfListOfUser.List - 172, // 13: productv1.ListOfListOfUserInput.list:type_name -> productv1.ListOfListOfUserInput.List - 173, // 14: productv1.ListOfOrderLine.list:type_name -> productv1.ListOfOrderLine.List - 174, // 15: productv1.ListOfProduct.list:type_name -> productv1.ListOfProduct.List - 175, // 16: productv1.ListOfString.list:type_name -> productv1.ListOfString.List - 176, // 17: productv1.ListOfUser.list:type_name -> productv1.ListOfUser.List - 177, // 18: productv1.ListOfUserInput.list:type_name -> productv1.ListOfUserInput.List - 20, // 19: productv1.LookupProductByIdRequest.keys:type_name -> productv1.LookupProductByIdRequestKey - 118, // 20: productv1.LookupProductByIdResponse.result:type_name -> productv1.Product - 23, // 21: productv1.LookupStorageByIdRequest.keys:type_name -> productv1.LookupStorageByIdRequestKey - 119, // 22: productv1.LookupStorageByIdResponse.result:type_name -> productv1.Storage - 26, // 23: productv1.LookupWarehouseByIdRequest.keys:type_name -> productv1.LookupWarehouseByIdRequestKey - 120, // 24: productv1.LookupWarehouseByIdResponse.result:type_name -> productv1.Warehouse - 121, // 25: productv1.QueryUsersResponse.users:type_name -> productv1.User - 121, // 26: productv1.QueryUserResponse.user:type_name -> productv1.User - 122, // 27: productv1.QueryNestedTypeResponse.nested_type:type_name -> productv1.NestedTypeA - 123, // 28: productv1.QueryRecursiveTypeResponse.recursive_type:type_name -> productv1.RecursiveType - 124, // 29: productv1.QueryTypeFilterWithArgumentsResponse.type_filter_with_arguments:type_name -> productv1.TypeWithMultipleFilterFields - 125, // 30: productv1.QueryTypeWithMultipleFilterFieldsRequest.filter:type_name -> productv1.FilterTypeInput - 124, // 31: productv1.QueryTypeWithMultipleFilterFieldsResponse.type_with_multiple_filter_fields:type_name -> productv1.TypeWithMultipleFilterFields - 126, // 32: productv1.QueryComplexFilterTypeRequest.filter:type_name -> productv1.ComplexFilterTypeInput - 127, // 33: productv1.QueryComplexFilterTypeResponse.complex_filter_type:type_name -> productv1.TypeWithComplexFilterInput - 128, // 34: productv1.QueryCalculateTotalsRequest.orders:type_name -> productv1.OrderInput - 129, // 35: productv1.QueryCalculateTotalsResponse.calculate_totals:type_name -> productv1.Order - 130, // 36: productv1.QueryCategoriesResponse.categories:type_name -> productv1.Category - 0, // 37: productv1.QueryCategoriesByKindRequest.kind:type_name -> productv1.CategoryKind - 130, // 38: productv1.QueryCategoriesByKindResponse.categories_by_kind:type_name -> productv1.Category - 0, // 39: productv1.QueryCategoriesByKindsRequest.kinds:type_name -> productv1.CategoryKind - 130, // 40: productv1.QueryCategoriesByKindsResponse.categories_by_kinds:type_name -> productv1.Category - 131, // 41: productv1.QueryFilterCategoriesRequest.filter:type_name -> productv1.CategoryFilter - 130, // 42: productv1.QueryFilterCategoriesResponse.filter_categories:type_name -> productv1.Category - 132, // 43: productv1.QueryRandomPetResponse.random_pet:type_name -> productv1.Animal - 132, // 44: productv1.QueryAllPetsResponse.all_pets:type_name -> productv1.Animal - 133, // 45: productv1.QuerySearchRequest.input:type_name -> productv1.SearchInput - 134, // 46: productv1.QuerySearchResponse.search:type_name -> productv1.SearchResult - 134, // 47: productv1.QueryRandomSearchResultResponse.random_search_result:type_name -> productv1.SearchResult - 135, // 48: productv1.QueryNullableFieldsTypeResponse.nullable_fields_type:type_name -> productv1.NullableFieldsType - 135, // 49: productv1.QueryNullableFieldsTypeByIdResponse.nullable_fields_type_by_id:type_name -> productv1.NullableFieldsType - 136, // 50: productv1.QueryNullableFieldsTypeWithFilterRequest.filter:type_name -> productv1.NullableFieldsFilter - 135, // 51: productv1.QueryNullableFieldsTypeWithFilterResponse.nullable_fields_type_with_filter:type_name -> productv1.NullableFieldsType - 135, // 52: productv1.QueryAllNullableFieldsTypesResponse.all_nullable_fields_types:type_name -> productv1.NullableFieldsType - 137, // 53: productv1.QueryBlogPostResponse.blog_post:type_name -> productv1.BlogPost - 137, // 54: productv1.QueryBlogPostByIdResponse.blog_post_by_id:type_name -> productv1.BlogPost - 138, // 55: productv1.QueryBlogPostsWithFilterRequest.filter:type_name -> productv1.BlogPostFilter - 137, // 56: productv1.QueryBlogPostsWithFilterResponse.blog_posts_with_filter:type_name -> productv1.BlogPost - 137, // 57: productv1.QueryAllBlogPostsResponse.all_blog_posts:type_name -> productv1.BlogPost - 139, // 58: productv1.QueryAuthorResponse.author:type_name -> productv1.Author - 139, // 59: productv1.QueryAuthorByIdResponse.author_by_id:type_name -> productv1.Author - 140, // 60: productv1.QueryAuthorsWithFilterRequest.filter:type_name -> productv1.AuthorFilter - 139, // 61: productv1.QueryAuthorsWithFilterResponse.authors_with_filter:type_name -> productv1.Author - 139, // 62: productv1.QueryAllAuthorsResponse.all_authors:type_name -> productv1.Author - 1, // 63: productv1.QueryBulkSearchAuthorsRequest.filters:type_name -> productv1.ListOfAuthorFilter - 139, // 64: productv1.QueryBulkSearchAuthorsResponse.bulk_search_authors:type_name -> productv1.Author - 4, // 65: productv1.QueryBulkSearchBlogPostsRequest.filters:type_name -> productv1.ListOfBlogPostFilter - 137, // 66: productv1.QueryBulkSearchBlogPostsResponse.bulk_search_blog_posts:type_name -> productv1.BlogPost - 141, // 67: productv1.MutationCreateUserRequest.input:type_name -> productv1.UserInput - 121, // 68: productv1.MutationCreateUserResponse.create_user:type_name -> productv1.User - 142, // 69: productv1.MutationPerformActionRequest.input:type_name -> productv1.ActionInput - 143, // 70: productv1.MutationPerformActionResponse.perform_action:type_name -> productv1.ActionResult - 144, // 71: productv1.MutationCreateNullableFieldsTypeRequest.input:type_name -> productv1.NullableFieldsInput - 135, // 72: productv1.MutationCreateNullableFieldsTypeResponse.create_nullable_fields_type:type_name -> productv1.NullableFieldsType - 144, // 73: productv1.MutationUpdateNullableFieldsTypeRequest.input:type_name -> productv1.NullableFieldsInput - 135, // 74: productv1.MutationUpdateNullableFieldsTypeResponse.update_nullable_fields_type:type_name -> productv1.NullableFieldsType - 145, // 75: productv1.MutationCreateBlogPostRequest.input:type_name -> productv1.BlogPostInput - 137, // 76: productv1.MutationCreateBlogPostResponse.create_blog_post:type_name -> productv1.BlogPost - 145, // 77: productv1.MutationUpdateBlogPostRequest.input:type_name -> productv1.BlogPostInput - 137, // 78: productv1.MutationUpdateBlogPostResponse.update_blog_post:type_name -> productv1.BlogPost - 146, // 79: productv1.MutationCreateAuthorRequest.input:type_name -> productv1.AuthorInput - 139, // 80: productv1.MutationCreateAuthorResponse.create_author:type_name -> productv1.Author - 146, // 81: productv1.MutationUpdateAuthorRequest.input:type_name -> productv1.AuthorInput - 139, // 82: productv1.MutationUpdateAuthorResponse.update_author:type_name -> productv1.Author - 2, // 83: productv1.MutationBulkCreateAuthorsRequest.authors:type_name -> productv1.ListOfAuthorInput - 139, // 84: productv1.MutationBulkCreateAuthorsResponse.bulk_create_authors:type_name -> productv1.Author - 2, // 85: productv1.MutationBulkUpdateAuthorsRequest.authors:type_name -> productv1.ListOfAuthorInput - 139, // 86: productv1.MutationBulkUpdateAuthorsResponse.bulk_update_authors:type_name -> productv1.Author - 5, // 87: productv1.MutationBulkCreateBlogPostsRequest.blog_posts:type_name -> productv1.ListOfBlogPostInput - 137, // 88: productv1.MutationBulkCreateBlogPostsResponse.bulk_create_blog_posts:type_name -> productv1.BlogPost - 5, // 89: productv1.MutationBulkUpdateBlogPostsRequest.blog_posts:type_name -> productv1.ListOfBlogPostInput - 137, // 90: productv1.MutationBulkUpdateBlogPostsResponse.bulk_update_blog_posts:type_name -> productv1.BlogPost - 158, // 91: productv1.CategoryProductCountArgs.filters:type_name -> productv1.ProductCountFilter - 114, // 92: productv1.ResolveCategoryProductCountRequest.context:type_name -> productv1.CategoryProductCountContext - 113, // 93: productv1.ResolveCategoryProductCountRequest.field_args:type_name -> productv1.CategoryProductCountArgs - 116, // 94: productv1.ResolveCategoryProductCountResponse.result:type_name -> productv1.ResolveCategoryProductCountResponseResult - 147, // 95: productv1.NestedTypeA.b:type_name -> productv1.NestedTypeB - 123, // 96: productv1.RecursiveType.recursive_type:type_name -> productv1.RecursiveType - 149, // 97: productv1.ComplexFilterTypeInput.filter:type_name -> productv1.FilterType - 151, // 98: productv1.OrderInput.lines:type_name -> productv1.OrderLineInput - 15, // 99: productv1.Order.order_lines:type_name -> productv1.ListOfOrderLine - 0, // 100: productv1.Category.kind:type_name -> productv1.CategoryKind - 0, // 101: productv1.CategoryFilter.category:type_name -> productv1.CategoryKind - 150, // 102: productv1.CategoryFilter.pagination:type_name -> productv1.Pagination - 153, // 103: productv1.Animal.cat:type_name -> productv1.Cat - 154, // 104: productv1.Animal.dog:type_name -> productv1.Dog - 178, // 105: productv1.SearchInput.limit:type_name -> google.protobuf.Int32Value - 118, // 106: productv1.SearchResult.product:type_name -> productv1.Product - 121, // 107: productv1.SearchResult.user:type_name -> productv1.User - 130, // 108: productv1.SearchResult.category:type_name -> productv1.Category - 179, // 109: productv1.NullableFieldsType.optional_string:type_name -> google.protobuf.StringValue - 178, // 110: productv1.NullableFieldsType.optional_int:type_name -> google.protobuf.Int32Value - 180, // 111: productv1.NullableFieldsType.optional_float:type_name -> google.protobuf.DoubleValue - 181, // 112: productv1.NullableFieldsType.optional_boolean:type_name -> google.protobuf.BoolValue - 179, // 113: productv1.NullableFieldsFilter.name:type_name -> google.protobuf.StringValue - 179, // 114: productv1.NullableFieldsFilter.optional_string:type_name -> google.protobuf.StringValue - 181, // 115: productv1.NullableFieldsFilter.include_nulls:type_name -> google.protobuf.BoolValue - 17, // 116: productv1.BlogPost.optional_tags:type_name -> productv1.ListOfString - 17, // 117: productv1.BlogPost.keywords:type_name -> productv1.ListOfString - 9, // 118: productv1.BlogPost.ratings:type_name -> productv1.ListOfFloat - 6, // 119: productv1.BlogPost.is_published:type_name -> productv1.ListOfBoolean - 12, // 120: productv1.BlogPost.tag_groups:type_name -> productv1.ListOfListOfString - 12, // 121: productv1.BlogPost.related_topics:type_name -> productv1.ListOfListOfString - 12, // 122: productv1.BlogPost.comment_threads:type_name -> productv1.ListOfListOfString - 12, // 123: productv1.BlogPost.suggestions:type_name -> productv1.ListOfListOfString - 130, // 124: productv1.BlogPost.related_categories:type_name -> productv1.Category - 121, // 125: productv1.BlogPost.contributors:type_name -> productv1.User - 16, // 126: productv1.BlogPost.mentioned_products:type_name -> productv1.ListOfProduct - 18, // 127: productv1.BlogPost.mentioned_users:type_name -> productv1.ListOfUser - 10, // 128: productv1.BlogPost.category_groups:type_name -> productv1.ListOfListOfCategory - 13, // 129: productv1.BlogPost.contributor_teams:type_name -> productv1.ListOfListOfUser - 179, // 130: productv1.BlogPostFilter.title:type_name -> google.protobuf.StringValue - 181, // 131: productv1.BlogPostFilter.has_categories:type_name -> google.protobuf.BoolValue - 178, // 132: productv1.BlogPostFilter.min_tags:type_name -> google.protobuf.Int32Value - 179, // 133: productv1.Author.email:type_name -> google.protobuf.StringValue - 17, // 134: productv1.Author.social_links:type_name -> productv1.ListOfString - 12, // 135: productv1.Author.teams_by_project:type_name -> productv1.ListOfListOfString - 12, // 136: productv1.Author.collaborations:type_name -> productv1.ListOfListOfString - 3, // 137: productv1.Author.written_posts:type_name -> productv1.ListOfBlogPost - 130, // 138: productv1.Author.favorite_categories:type_name -> productv1.Category - 18, // 139: productv1.Author.related_authors:type_name -> productv1.ListOfUser - 16, // 140: productv1.Author.product_reviews:type_name -> productv1.ListOfProduct - 13, // 141: productv1.Author.author_groups:type_name -> productv1.ListOfListOfUser - 10, // 142: productv1.Author.category_preferences:type_name -> productv1.ListOfListOfCategory - 13, // 143: productv1.Author.project_teams:type_name -> productv1.ListOfListOfUser - 179, // 144: productv1.AuthorFilter.name:type_name -> google.protobuf.StringValue - 181, // 145: productv1.AuthorFilter.has_teams:type_name -> google.protobuf.BoolValue - 178, // 146: productv1.AuthorFilter.skill_count:type_name -> google.protobuf.Int32Value - 155, // 147: productv1.ActionResult.action_success:type_name -> productv1.ActionSuccess - 156, // 148: productv1.ActionResult.action_error:type_name -> productv1.ActionError - 179, // 149: productv1.NullableFieldsInput.optional_string:type_name -> google.protobuf.StringValue - 178, // 150: productv1.NullableFieldsInput.optional_int:type_name -> google.protobuf.Int32Value - 180, // 151: productv1.NullableFieldsInput.optional_float:type_name -> google.protobuf.DoubleValue - 181, // 152: productv1.NullableFieldsInput.optional_boolean:type_name -> google.protobuf.BoolValue - 17, // 153: productv1.BlogPostInput.optional_tags:type_name -> productv1.ListOfString - 17, // 154: productv1.BlogPostInput.keywords:type_name -> productv1.ListOfString - 9, // 155: productv1.BlogPostInput.ratings:type_name -> productv1.ListOfFloat - 6, // 156: productv1.BlogPostInput.is_published:type_name -> productv1.ListOfBoolean - 12, // 157: productv1.BlogPostInput.tag_groups:type_name -> productv1.ListOfListOfString - 12, // 158: productv1.BlogPostInput.related_topics:type_name -> productv1.ListOfListOfString - 12, // 159: productv1.BlogPostInput.comment_threads:type_name -> productv1.ListOfListOfString - 12, // 160: productv1.BlogPostInput.suggestions:type_name -> productv1.ListOfListOfString - 8, // 161: productv1.BlogPostInput.related_categories:type_name -> productv1.ListOfCategoryInput - 19, // 162: productv1.BlogPostInput.contributors:type_name -> productv1.ListOfUserInput - 11, // 163: productv1.BlogPostInput.category_groups:type_name -> productv1.ListOfListOfCategoryInput - 179, // 164: productv1.AuthorInput.email:type_name -> google.protobuf.StringValue - 17, // 165: productv1.AuthorInput.social_links:type_name -> productv1.ListOfString - 12, // 166: productv1.AuthorInput.teams_by_project:type_name -> productv1.ListOfListOfString - 12, // 167: productv1.AuthorInput.collaborations:type_name -> productv1.ListOfListOfString - 157, // 168: productv1.AuthorInput.favorite_categories:type_name -> productv1.CategoryInput - 14, // 169: productv1.AuthorInput.author_groups:type_name -> productv1.ListOfListOfUserInput - 14, // 170: productv1.AuthorInput.project_teams:type_name -> productv1.ListOfListOfUserInput - 148, // 171: productv1.NestedTypeB.c:type_name -> productv1.NestedTypeC - 150, // 172: productv1.FilterType.pagination:type_name -> productv1.Pagination - 17, // 173: productv1.OrderLineInput.modifiers:type_name -> productv1.ListOfString - 17, // 174: productv1.OrderLine.modifiers:type_name -> productv1.ListOfString - 0, // 175: productv1.CategoryInput.kind:type_name -> productv1.CategoryKind - 180, // 176: productv1.ProductCountFilter.min_price:type_name -> google.protobuf.DoubleValue - 180, // 177: productv1.ProductCountFilter.max_price:type_name -> google.protobuf.DoubleValue - 181, // 178: productv1.ProductCountFilter.in_stock:type_name -> google.protobuf.BoolValue - 179, // 179: productv1.ProductCountFilter.search_term:type_name -> google.protobuf.StringValue - 140, // 180: productv1.ListOfAuthorFilter.List.items:type_name -> productv1.AuthorFilter - 146, // 181: productv1.ListOfAuthorInput.List.items:type_name -> productv1.AuthorInput - 137, // 182: productv1.ListOfBlogPost.List.items:type_name -> productv1.BlogPost - 138, // 183: productv1.ListOfBlogPostFilter.List.items:type_name -> productv1.BlogPostFilter - 145, // 184: productv1.ListOfBlogPostInput.List.items:type_name -> productv1.BlogPostInput - 130, // 185: productv1.ListOfCategory.List.items:type_name -> productv1.Category - 157, // 186: productv1.ListOfCategoryInput.List.items:type_name -> productv1.CategoryInput - 7, // 187: productv1.ListOfListOfCategory.List.items:type_name -> productv1.ListOfCategory - 8, // 188: productv1.ListOfListOfCategoryInput.List.items:type_name -> productv1.ListOfCategoryInput - 17, // 189: productv1.ListOfListOfString.List.items:type_name -> productv1.ListOfString - 18, // 190: productv1.ListOfListOfUser.List.items:type_name -> productv1.ListOfUser - 19, // 191: productv1.ListOfListOfUserInput.List.items:type_name -> productv1.ListOfUserInput - 152, // 192: productv1.ListOfOrderLine.List.items:type_name -> productv1.OrderLine - 118, // 193: productv1.ListOfProduct.List.items:type_name -> productv1.Product - 121, // 194: productv1.ListOfUser.List.items:type_name -> productv1.User - 141, // 195: productv1.ListOfUserInput.List.items:type_name -> productv1.UserInput - 21, // 196: productv1.ProductService.LookupProductById:input_type -> productv1.LookupProductByIdRequest - 24, // 197: productv1.ProductService.LookupStorageById:input_type -> productv1.LookupStorageByIdRequest - 27, // 198: productv1.ProductService.LookupWarehouseById:input_type -> productv1.LookupWarehouseByIdRequest - 105, // 199: productv1.ProductService.MutationBulkCreateAuthors:input_type -> productv1.MutationBulkCreateAuthorsRequest - 109, // 200: productv1.ProductService.MutationBulkCreateBlogPosts:input_type -> productv1.MutationBulkCreateBlogPostsRequest - 107, // 201: productv1.ProductService.MutationBulkUpdateAuthors:input_type -> productv1.MutationBulkUpdateAuthorsRequest - 111, // 202: productv1.ProductService.MutationBulkUpdateBlogPosts:input_type -> productv1.MutationBulkUpdateBlogPostsRequest - 101, // 203: productv1.ProductService.MutationCreateAuthor:input_type -> productv1.MutationCreateAuthorRequest - 97, // 204: productv1.ProductService.MutationCreateBlogPost:input_type -> productv1.MutationCreateBlogPostRequest - 93, // 205: productv1.ProductService.MutationCreateNullableFieldsType:input_type -> productv1.MutationCreateNullableFieldsTypeRequest - 89, // 206: productv1.ProductService.MutationCreateUser:input_type -> productv1.MutationCreateUserRequest - 91, // 207: productv1.ProductService.MutationPerformAction:input_type -> productv1.MutationPerformActionRequest - 103, // 208: productv1.ProductService.MutationUpdateAuthor:input_type -> productv1.MutationUpdateAuthorRequest - 99, // 209: productv1.ProductService.MutationUpdateBlogPost:input_type -> productv1.MutationUpdateBlogPostRequest - 95, // 210: productv1.ProductService.MutationUpdateNullableFieldsType:input_type -> productv1.MutationUpdateNullableFieldsTypeRequest - 83, // 211: productv1.ProductService.QueryAllAuthors:input_type -> productv1.QueryAllAuthorsRequest - 75, // 212: productv1.ProductService.QueryAllBlogPosts:input_type -> productv1.QueryAllBlogPostsRequest - 67, // 213: productv1.ProductService.QueryAllNullableFieldsTypes:input_type -> productv1.QueryAllNullableFieldsTypesRequest - 55, // 214: productv1.ProductService.QueryAllPets:input_type -> productv1.QueryAllPetsRequest - 77, // 215: productv1.ProductService.QueryAuthor:input_type -> productv1.QueryAuthorRequest - 79, // 216: productv1.ProductService.QueryAuthorById:input_type -> productv1.QueryAuthorByIdRequest - 81, // 217: productv1.ProductService.QueryAuthorsWithFilter:input_type -> productv1.QueryAuthorsWithFilterRequest - 69, // 218: productv1.ProductService.QueryBlogPost:input_type -> productv1.QueryBlogPostRequest - 71, // 219: productv1.ProductService.QueryBlogPostById:input_type -> productv1.QueryBlogPostByIdRequest - 73, // 220: productv1.ProductService.QueryBlogPostsWithFilter:input_type -> productv1.QueryBlogPostsWithFilterRequest - 85, // 221: productv1.ProductService.QueryBulkSearchAuthors:input_type -> productv1.QueryBulkSearchAuthorsRequest - 87, // 222: productv1.ProductService.QueryBulkSearchBlogPosts:input_type -> productv1.QueryBulkSearchBlogPostsRequest - 43, // 223: productv1.ProductService.QueryCalculateTotals:input_type -> productv1.QueryCalculateTotalsRequest - 45, // 224: productv1.ProductService.QueryCategories:input_type -> productv1.QueryCategoriesRequest - 47, // 225: productv1.ProductService.QueryCategoriesByKind:input_type -> productv1.QueryCategoriesByKindRequest - 49, // 226: productv1.ProductService.QueryCategoriesByKinds:input_type -> productv1.QueryCategoriesByKindsRequest - 41, // 227: productv1.ProductService.QueryComplexFilterType:input_type -> productv1.QueryComplexFilterTypeRequest - 51, // 228: productv1.ProductService.QueryFilterCategories:input_type -> productv1.QueryFilterCategoriesRequest - 33, // 229: productv1.ProductService.QueryNestedType:input_type -> productv1.QueryNestedTypeRequest - 61, // 230: productv1.ProductService.QueryNullableFieldsType:input_type -> productv1.QueryNullableFieldsTypeRequest - 63, // 231: productv1.ProductService.QueryNullableFieldsTypeById:input_type -> productv1.QueryNullableFieldsTypeByIdRequest - 65, // 232: productv1.ProductService.QueryNullableFieldsTypeWithFilter:input_type -> productv1.QueryNullableFieldsTypeWithFilterRequest - 53, // 233: productv1.ProductService.QueryRandomPet:input_type -> productv1.QueryRandomPetRequest - 59, // 234: productv1.ProductService.QueryRandomSearchResult:input_type -> productv1.QueryRandomSearchResultRequest - 35, // 235: productv1.ProductService.QueryRecursiveType:input_type -> productv1.QueryRecursiveTypeRequest - 57, // 236: productv1.ProductService.QuerySearch:input_type -> productv1.QuerySearchRequest - 37, // 237: productv1.ProductService.QueryTypeFilterWithArguments:input_type -> productv1.QueryTypeFilterWithArgumentsRequest - 39, // 238: productv1.ProductService.QueryTypeWithMultipleFilterFields:input_type -> productv1.QueryTypeWithMultipleFilterFieldsRequest - 31, // 239: productv1.ProductService.QueryUser:input_type -> productv1.QueryUserRequest - 29, // 240: productv1.ProductService.QueryUsers:input_type -> productv1.QueryUsersRequest - 115, // 241: productv1.ProductService.ResolveCategoryProductCount:input_type -> productv1.ResolveCategoryProductCountRequest - 22, // 242: productv1.ProductService.LookupProductById:output_type -> productv1.LookupProductByIdResponse - 25, // 243: productv1.ProductService.LookupStorageById:output_type -> productv1.LookupStorageByIdResponse - 28, // 244: productv1.ProductService.LookupWarehouseById:output_type -> productv1.LookupWarehouseByIdResponse - 106, // 245: productv1.ProductService.MutationBulkCreateAuthors:output_type -> productv1.MutationBulkCreateAuthorsResponse - 110, // 246: productv1.ProductService.MutationBulkCreateBlogPosts:output_type -> productv1.MutationBulkCreateBlogPostsResponse - 108, // 247: productv1.ProductService.MutationBulkUpdateAuthors:output_type -> productv1.MutationBulkUpdateAuthorsResponse - 112, // 248: productv1.ProductService.MutationBulkUpdateBlogPosts:output_type -> productv1.MutationBulkUpdateBlogPostsResponse - 102, // 249: productv1.ProductService.MutationCreateAuthor:output_type -> productv1.MutationCreateAuthorResponse - 98, // 250: productv1.ProductService.MutationCreateBlogPost:output_type -> productv1.MutationCreateBlogPostResponse - 94, // 251: productv1.ProductService.MutationCreateNullableFieldsType:output_type -> productv1.MutationCreateNullableFieldsTypeResponse - 90, // 252: productv1.ProductService.MutationCreateUser:output_type -> productv1.MutationCreateUserResponse - 92, // 253: productv1.ProductService.MutationPerformAction:output_type -> productv1.MutationPerformActionResponse - 104, // 254: productv1.ProductService.MutationUpdateAuthor:output_type -> productv1.MutationUpdateAuthorResponse - 100, // 255: productv1.ProductService.MutationUpdateBlogPost:output_type -> productv1.MutationUpdateBlogPostResponse - 96, // 256: productv1.ProductService.MutationUpdateNullableFieldsType:output_type -> productv1.MutationUpdateNullableFieldsTypeResponse - 84, // 257: productv1.ProductService.QueryAllAuthors:output_type -> productv1.QueryAllAuthorsResponse - 76, // 258: productv1.ProductService.QueryAllBlogPosts:output_type -> productv1.QueryAllBlogPostsResponse - 68, // 259: productv1.ProductService.QueryAllNullableFieldsTypes:output_type -> productv1.QueryAllNullableFieldsTypesResponse - 56, // 260: productv1.ProductService.QueryAllPets:output_type -> productv1.QueryAllPetsResponse - 78, // 261: productv1.ProductService.QueryAuthor:output_type -> productv1.QueryAuthorResponse - 80, // 262: productv1.ProductService.QueryAuthorById:output_type -> productv1.QueryAuthorByIdResponse - 82, // 263: productv1.ProductService.QueryAuthorsWithFilter:output_type -> productv1.QueryAuthorsWithFilterResponse - 70, // 264: productv1.ProductService.QueryBlogPost:output_type -> productv1.QueryBlogPostResponse - 72, // 265: productv1.ProductService.QueryBlogPostById:output_type -> productv1.QueryBlogPostByIdResponse - 74, // 266: productv1.ProductService.QueryBlogPostsWithFilter:output_type -> productv1.QueryBlogPostsWithFilterResponse - 86, // 267: productv1.ProductService.QueryBulkSearchAuthors:output_type -> productv1.QueryBulkSearchAuthorsResponse - 88, // 268: productv1.ProductService.QueryBulkSearchBlogPosts:output_type -> productv1.QueryBulkSearchBlogPostsResponse - 44, // 269: productv1.ProductService.QueryCalculateTotals:output_type -> productv1.QueryCalculateTotalsResponse - 46, // 270: productv1.ProductService.QueryCategories:output_type -> productv1.QueryCategoriesResponse - 48, // 271: productv1.ProductService.QueryCategoriesByKind:output_type -> productv1.QueryCategoriesByKindResponse - 50, // 272: productv1.ProductService.QueryCategoriesByKinds:output_type -> productv1.QueryCategoriesByKindsResponse - 42, // 273: productv1.ProductService.QueryComplexFilterType:output_type -> productv1.QueryComplexFilterTypeResponse - 52, // 274: productv1.ProductService.QueryFilterCategories:output_type -> productv1.QueryFilterCategoriesResponse - 34, // 275: productv1.ProductService.QueryNestedType:output_type -> productv1.QueryNestedTypeResponse - 62, // 276: productv1.ProductService.QueryNullableFieldsType:output_type -> productv1.QueryNullableFieldsTypeResponse - 64, // 277: productv1.ProductService.QueryNullableFieldsTypeById:output_type -> productv1.QueryNullableFieldsTypeByIdResponse - 66, // 278: productv1.ProductService.QueryNullableFieldsTypeWithFilter:output_type -> productv1.QueryNullableFieldsTypeWithFilterResponse - 54, // 279: productv1.ProductService.QueryRandomPet:output_type -> productv1.QueryRandomPetResponse - 60, // 280: productv1.ProductService.QueryRandomSearchResult:output_type -> productv1.QueryRandomSearchResultResponse - 36, // 281: productv1.ProductService.QueryRecursiveType:output_type -> productv1.QueryRecursiveTypeResponse - 58, // 282: productv1.ProductService.QuerySearch:output_type -> productv1.QuerySearchResponse - 38, // 283: productv1.ProductService.QueryTypeFilterWithArguments:output_type -> productv1.QueryTypeFilterWithArgumentsResponse - 40, // 284: productv1.ProductService.QueryTypeWithMultipleFilterFields:output_type -> productv1.QueryTypeWithMultipleFilterFieldsResponse - 32, // 285: productv1.ProductService.QueryUser:output_type -> productv1.QueryUserResponse - 30, // 286: productv1.ProductService.QueryUsers:output_type -> productv1.QueryUsersResponse - 117, // 287: productv1.ProductService.ResolveCategoryProductCount:output_type -> productv1.ResolveCategoryProductCountResponse - 242, // [242:288] is the sub-list for method output_type - 196, // [196:242] is the sub-list for method input_type - 196, // [196:196] is the sub-list for extension type_name - 196, // [196:196] is the sub-list for extension extendee - 0, // [0:196] is the sub-list for field type_name + 167, // 0: productv1.ListOfAuthorFilter.list:type_name -> productv1.ListOfAuthorFilter.List + 168, // 1: productv1.ListOfAuthorInput.list:type_name -> productv1.ListOfAuthorInput.List + 169, // 2: productv1.ListOfBlogPost.list:type_name -> productv1.ListOfBlogPost.List + 170, // 3: productv1.ListOfBlogPostFilter.list:type_name -> productv1.ListOfBlogPostFilter.List + 171, // 4: productv1.ListOfBlogPostInput.list:type_name -> productv1.ListOfBlogPostInput.List + 172, // 5: productv1.ListOfBoolean.list:type_name -> productv1.ListOfBoolean.List + 173, // 6: productv1.ListOfCategory.list:type_name -> productv1.ListOfCategory.List + 174, // 7: productv1.ListOfCategoryInput.list:type_name -> productv1.ListOfCategoryInput.List + 175, // 8: productv1.ListOfFloat.list:type_name -> productv1.ListOfFloat.List + 176, // 9: productv1.ListOfListOfCategory.list:type_name -> productv1.ListOfListOfCategory.List + 177, // 10: productv1.ListOfListOfCategoryInput.list:type_name -> productv1.ListOfListOfCategoryInput.List + 178, // 11: productv1.ListOfListOfString.list:type_name -> productv1.ListOfListOfString.List + 179, // 12: productv1.ListOfListOfUser.list:type_name -> productv1.ListOfListOfUser.List + 180, // 13: productv1.ListOfListOfUserInput.list:type_name -> productv1.ListOfListOfUserInput.List + 181, // 14: productv1.ListOfOrderLine.list:type_name -> productv1.ListOfOrderLine.List + 182, // 15: productv1.ListOfProduct.list:type_name -> productv1.ListOfProduct.List + 183, // 16: productv1.ListOfString.list:type_name -> productv1.ListOfString.List + 184, // 17: productv1.ListOfSubcategory.list:type_name -> productv1.ListOfSubcategory.List + 185, // 18: productv1.ListOfUser.list:type_name -> productv1.ListOfUser.List + 186, // 19: productv1.ListOfUserInput.list:type_name -> productv1.ListOfUserInput.List + 21, // 20: productv1.LookupProductByIdRequest.keys:type_name -> productv1.LookupProductByIdRequestKey + 124, // 21: productv1.LookupProductByIdResponse.result:type_name -> productv1.Product + 24, // 22: productv1.LookupStorageByIdRequest.keys:type_name -> productv1.LookupStorageByIdRequestKey + 125, // 23: productv1.LookupStorageByIdResponse.result:type_name -> productv1.Storage + 27, // 24: productv1.LookupWarehouseByIdRequest.keys:type_name -> productv1.LookupWarehouseByIdRequestKey + 126, // 25: productv1.LookupWarehouseByIdResponse.result:type_name -> productv1.Warehouse + 127, // 26: productv1.QueryUsersResponse.users:type_name -> productv1.User + 127, // 27: productv1.QueryUserResponse.user:type_name -> productv1.User + 128, // 28: productv1.QueryNestedTypeResponse.nested_type:type_name -> productv1.NestedTypeA + 129, // 29: productv1.QueryRecursiveTypeResponse.recursive_type:type_name -> productv1.RecursiveType + 130, // 30: productv1.QueryTypeFilterWithArgumentsResponse.type_filter_with_arguments:type_name -> productv1.TypeWithMultipleFilterFields + 131, // 31: productv1.QueryTypeWithMultipleFilterFieldsRequest.filter:type_name -> productv1.FilterTypeInput + 130, // 32: productv1.QueryTypeWithMultipleFilterFieldsResponse.type_with_multiple_filter_fields:type_name -> productv1.TypeWithMultipleFilterFields + 132, // 33: productv1.QueryComplexFilterTypeRequest.filter:type_name -> productv1.ComplexFilterTypeInput + 133, // 34: productv1.QueryComplexFilterTypeResponse.complex_filter_type:type_name -> productv1.TypeWithComplexFilterInput + 134, // 35: productv1.QueryCalculateTotalsRequest.orders:type_name -> productv1.OrderInput + 135, // 36: productv1.QueryCalculateTotalsResponse.calculate_totals:type_name -> productv1.Order + 136, // 37: productv1.QueryCategoriesResponse.categories:type_name -> productv1.Category + 0, // 38: productv1.QueryCategoriesByKindRequest.kind:type_name -> productv1.CategoryKind + 136, // 39: productv1.QueryCategoriesByKindResponse.categories_by_kind:type_name -> productv1.Category + 0, // 40: productv1.QueryCategoriesByKindsRequest.kinds:type_name -> productv1.CategoryKind + 136, // 41: productv1.QueryCategoriesByKindsResponse.categories_by_kinds:type_name -> productv1.Category + 137, // 42: productv1.QueryFilterCategoriesRequest.filter:type_name -> productv1.CategoryFilter + 136, // 43: productv1.QueryFilterCategoriesResponse.filter_categories:type_name -> productv1.Category + 138, // 44: productv1.QueryRandomPetResponse.random_pet:type_name -> productv1.Animal + 138, // 45: productv1.QueryAllPetsResponse.all_pets:type_name -> productv1.Animal + 139, // 46: productv1.QuerySearchRequest.input:type_name -> productv1.SearchInput + 140, // 47: productv1.QuerySearchResponse.search:type_name -> productv1.SearchResult + 140, // 48: productv1.QueryRandomSearchResultResponse.random_search_result:type_name -> productv1.SearchResult + 141, // 49: productv1.QueryNullableFieldsTypeResponse.nullable_fields_type:type_name -> productv1.NullableFieldsType + 141, // 50: productv1.QueryNullableFieldsTypeByIdResponse.nullable_fields_type_by_id:type_name -> productv1.NullableFieldsType + 142, // 51: productv1.QueryNullableFieldsTypeWithFilterRequest.filter:type_name -> productv1.NullableFieldsFilter + 141, // 52: productv1.QueryNullableFieldsTypeWithFilterResponse.nullable_fields_type_with_filter:type_name -> productv1.NullableFieldsType + 141, // 53: productv1.QueryAllNullableFieldsTypesResponse.all_nullable_fields_types:type_name -> productv1.NullableFieldsType + 143, // 54: productv1.QueryBlogPostResponse.blog_post:type_name -> productv1.BlogPost + 143, // 55: productv1.QueryBlogPostByIdResponse.blog_post_by_id:type_name -> productv1.BlogPost + 144, // 56: productv1.QueryBlogPostsWithFilterRequest.filter:type_name -> productv1.BlogPostFilter + 143, // 57: productv1.QueryBlogPostsWithFilterResponse.blog_posts_with_filter:type_name -> productv1.BlogPost + 143, // 58: productv1.QueryAllBlogPostsResponse.all_blog_posts:type_name -> productv1.BlogPost + 145, // 59: productv1.QueryAuthorResponse.author:type_name -> productv1.Author + 145, // 60: productv1.QueryAuthorByIdResponse.author_by_id:type_name -> productv1.Author + 146, // 61: productv1.QueryAuthorsWithFilterRequest.filter:type_name -> productv1.AuthorFilter + 145, // 62: productv1.QueryAuthorsWithFilterResponse.authors_with_filter:type_name -> productv1.Author + 145, // 63: productv1.QueryAllAuthorsResponse.all_authors:type_name -> productv1.Author + 1, // 64: productv1.QueryBulkSearchAuthorsRequest.filters:type_name -> productv1.ListOfAuthorFilter + 145, // 65: productv1.QueryBulkSearchAuthorsResponse.bulk_search_authors:type_name -> productv1.Author + 4, // 66: productv1.QueryBulkSearchBlogPostsRequest.filters:type_name -> productv1.ListOfBlogPostFilter + 143, // 67: productv1.QueryBulkSearchBlogPostsResponse.bulk_search_blog_posts:type_name -> productv1.BlogPost + 147, // 68: productv1.MutationCreateUserRequest.input:type_name -> productv1.UserInput + 127, // 69: productv1.MutationCreateUserResponse.create_user:type_name -> productv1.User + 148, // 70: productv1.MutationPerformActionRequest.input:type_name -> productv1.ActionInput + 149, // 71: productv1.MutationPerformActionResponse.perform_action:type_name -> productv1.ActionResult + 150, // 72: productv1.MutationCreateNullableFieldsTypeRequest.input:type_name -> productv1.NullableFieldsInput + 141, // 73: productv1.MutationCreateNullableFieldsTypeResponse.create_nullable_fields_type:type_name -> productv1.NullableFieldsType + 150, // 74: productv1.MutationUpdateNullableFieldsTypeRequest.input:type_name -> productv1.NullableFieldsInput + 141, // 75: productv1.MutationUpdateNullableFieldsTypeResponse.update_nullable_fields_type:type_name -> productv1.NullableFieldsType + 151, // 76: productv1.MutationCreateBlogPostRequest.input:type_name -> productv1.BlogPostInput + 143, // 77: productv1.MutationCreateBlogPostResponse.create_blog_post:type_name -> productv1.BlogPost + 151, // 78: productv1.MutationUpdateBlogPostRequest.input:type_name -> productv1.BlogPostInput + 143, // 79: productv1.MutationUpdateBlogPostResponse.update_blog_post:type_name -> productv1.BlogPost + 152, // 80: productv1.MutationCreateAuthorRequest.input:type_name -> productv1.AuthorInput + 145, // 81: productv1.MutationCreateAuthorResponse.create_author:type_name -> productv1.Author + 152, // 82: productv1.MutationUpdateAuthorRequest.input:type_name -> productv1.AuthorInput + 145, // 83: productv1.MutationUpdateAuthorResponse.update_author:type_name -> productv1.Author + 2, // 84: productv1.MutationBulkCreateAuthorsRequest.authors:type_name -> productv1.ListOfAuthorInput + 145, // 85: productv1.MutationBulkCreateAuthorsResponse.bulk_create_authors:type_name -> productv1.Author + 2, // 86: productv1.MutationBulkUpdateAuthorsRequest.authors:type_name -> productv1.ListOfAuthorInput + 145, // 87: productv1.MutationBulkUpdateAuthorsResponse.bulk_update_authors:type_name -> productv1.Author + 5, // 88: productv1.MutationBulkCreateBlogPostsRequest.blog_posts:type_name -> productv1.ListOfBlogPostInput + 143, // 89: productv1.MutationBulkCreateBlogPostsResponse.bulk_create_blog_posts:type_name -> productv1.BlogPost + 5, // 90: productv1.MutationBulkUpdateBlogPostsRequest.blog_posts:type_name -> productv1.ListOfBlogPostInput + 143, // 91: productv1.MutationBulkUpdateBlogPostsResponse.bulk_update_blog_posts:type_name -> productv1.BlogPost + 165, // 92: productv1.ResolveCategoryProductCountArgs.filters:type_name -> productv1.ProductCountFilter + 115, // 93: productv1.ResolveCategoryProductCountRequest.context:type_name -> productv1.ResolveCategoryProductCountContext + 114, // 94: productv1.ResolveCategoryProductCountRequest.field_args:type_name -> productv1.ResolveCategoryProductCountArgs + 117, // 95: productv1.ResolveCategoryProductCountResponse.result:type_name -> productv1.ResolveCategoryProductCountResult + 166, // 96: productv1.ResolveSubcategoryItemCountArgs.filters:type_name -> productv1.SubcategoryItemFilter + 120, // 97: productv1.ResolveSubcategoryItemCountRequest.context:type_name -> productv1.ResolveSubcategoryItemCountContext + 119, // 98: productv1.ResolveSubcategoryItemCountRequest.field_args:type_name -> productv1.ResolveSubcategoryItemCountArgs + 122, // 99: productv1.ResolveSubcategoryItemCountResponse.result:type_name -> productv1.ResolveSubcategoryItemCountResult + 153, // 100: productv1.NestedTypeA.b:type_name -> productv1.NestedTypeB + 129, // 101: productv1.RecursiveType.recursive_type:type_name -> productv1.RecursiveType + 155, // 102: productv1.ComplexFilterTypeInput.filter:type_name -> productv1.FilterType + 157, // 103: productv1.OrderInput.lines:type_name -> productv1.OrderLineInput + 15, // 104: productv1.Order.order_lines:type_name -> productv1.ListOfOrderLine + 0, // 105: productv1.Category.kind:type_name -> productv1.CategoryKind + 18, // 106: productv1.Category.subcategories:type_name -> productv1.ListOfSubcategory + 0, // 107: productv1.CategoryFilter.category:type_name -> productv1.CategoryKind + 156, // 108: productv1.CategoryFilter.pagination:type_name -> productv1.Pagination + 160, // 109: productv1.Animal.cat:type_name -> productv1.Cat + 161, // 110: productv1.Animal.dog:type_name -> productv1.Dog + 187, // 111: productv1.SearchInput.limit:type_name -> google.protobuf.Int32Value + 124, // 112: productv1.SearchResult.product:type_name -> productv1.Product + 127, // 113: productv1.SearchResult.user:type_name -> productv1.User + 136, // 114: productv1.SearchResult.category:type_name -> productv1.Category + 188, // 115: productv1.NullableFieldsType.optional_string:type_name -> google.protobuf.StringValue + 187, // 116: productv1.NullableFieldsType.optional_int:type_name -> google.protobuf.Int32Value + 189, // 117: productv1.NullableFieldsType.optional_float:type_name -> google.protobuf.DoubleValue + 190, // 118: productv1.NullableFieldsType.optional_boolean:type_name -> google.protobuf.BoolValue + 188, // 119: productv1.NullableFieldsFilter.name:type_name -> google.protobuf.StringValue + 188, // 120: productv1.NullableFieldsFilter.optional_string:type_name -> google.protobuf.StringValue + 190, // 121: productv1.NullableFieldsFilter.include_nulls:type_name -> google.protobuf.BoolValue + 17, // 122: productv1.BlogPost.optional_tags:type_name -> productv1.ListOfString + 17, // 123: productv1.BlogPost.keywords:type_name -> productv1.ListOfString + 9, // 124: productv1.BlogPost.ratings:type_name -> productv1.ListOfFloat + 6, // 125: productv1.BlogPost.is_published:type_name -> productv1.ListOfBoolean + 12, // 126: productv1.BlogPost.tag_groups:type_name -> productv1.ListOfListOfString + 12, // 127: productv1.BlogPost.related_topics:type_name -> productv1.ListOfListOfString + 12, // 128: productv1.BlogPost.comment_threads:type_name -> productv1.ListOfListOfString + 12, // 129: productv1.BlogPost.suggestions:type_name -> productv1.ListOfListOfString + 136, // 130: productv1.BlogPost.related_categories:type_name -> productv1.Category + 127, // 131: productv1.BlogPost.contributors:type_name -> productv1.User + 16, // 132: productv1.BlogPost.mentioned_products:type_name -> productv1.ListOfProduct + 19, // 133: productv1.BlogPost.mentioned_users:type_name -> productv1.ListOfUser + 10, // 134: productv1.BlogPost.category_groups:type_name -> productv1.ListOfListOfCategory + 13, // 135: productv1.BlogPost.contributor_teams:type_name -> productv1.ListOfListOfUser + 188, // 136: productv1.BlogPostFilter.title:type_name -> google.protobuf.StringValue + 190, // 137: productv1.BlogPostFilter.has_categories:type_name -> google.protobuf.BoolValue + 187, // 138: productv1.BlogPostFilter.min_tags:type_name -> google.protobuf.Int32Value + 188, // 139: productv1.Author.email:type_name -> google.protobuf.StringValue + 17, // 140: productv1.Author.social_links:type_name -> productv1.ListOfString + 12, // 141: productv1.Author.teams_by_project:type_name -> productv1.ListOfListOfString + 12, // 142: productv1.Author.collaborations:type_name -> productv1.ListOfListOfString + 3, // 143: productv1.Author.written_posts:type_name -> productv1.ListOfBlogPost + 136, // 144: productv1.Author.favorite_categories:type_name -> productv1.Category + 19, // 145: productv1.Author.related_authors:type_name -> productv1.ListOfUser + 16, // 146: productv1.Author.product_reviews:type_name -> productv1.ListOfProduct + 13, // 147: productv1.Author.author_groups:type_name -> productv1.ListOfListOfUser + 10, // 148: productv1.Author.category_preferences:type_name -> productv1.ListOfListOfCategory + 13, // 149: productv1.Author.project_teams:type_name -> productv1.ListOfListOfUser + 188, // 150: productv1.AuthorFilter.name:type_name -> google.protobuf.StringValue + 190, // 151: productv1.AuthorFilter.has_teams:type_name -> google.protobuf.BoolValue + 187, // 152: productv1.AuthorFilter.skill_count:type_name -> google.protobuf.Int32Value + 162, // 153: productv1.ActionResult.action_success:type_name -> productv1.ActionSuccess + 163, // 154: productv1.ActionResult.action_error:type_name -> productv1.ActionError + 188, // 155: productv1.NullableFieldsInput.optional_string:type_name -> google.protobuf.StringValue + 187, // 156: productv1.NullableFieldsInput.optional_int:type_name -> google.protobuf.Int32Value + 189, // 157: productv1.NullableFieldsInput.optional_float:type_name -> google.protobuf.DoubleValue + 190, // 158: productv1.NullableFieldsInput.optional_boolean:type_name -> google.protobuf.BoolValue + 17, // 159: productv1.BlogPostInput.optional_tags:type_name -> productv1.ListOfString + 17, // 160: productv1.BlogPostInput.keywords:type_name -> productv1.ListOfString + 9, // 161: productv1.BlogPostInput.ratings:type_name -> productv1.ListOfFloat + 6, // 162: productv1.BlogPostInput.is_published:type_name -> productv1.ListOfBoolean + 12, // 163: productv1.BlogPostInput.tag_groups:type_name -> productv1.ListOfListOfString + 12, // 164: productv1.BlogPostInput.related_topics:type_name -> productv1.ListOfListOfString + 12, // 165: productv1.BlogPostInput.comment_threads:type_name -> productv1.ListOfListOfString + 12, // 166: productv1.BlogPostInput.suggestions:type_name -> productv1.ListOfListOfString + 8, // 167: productv1.BlogPostInput.related_categories:type_name -> productv1.ListOfCategoryInput + 20, // 168: productv1.BlogPostInput.contributors:type_name -> productv1.ListOfUserInput + 11, // 169: productv1.BlogPostInput.category_groups:type_name -> productv1.ListOfListOfCategoryInput + 188, // 170: productv1.AuthorInput.email:type_name -> google.protobuf.StringValue + 17, // 171: productv1.AuthorInput.social_links:type_name -> productv1.ListOfString + 12, // 172: productv1.AuthorInput.teams_by_project:type_name -> productv1.ListOfListOfString + 12, // 173: productv1.AuthorInput.collaborations:type_name -> productv1.ListOfListOfString + 164, // 174: productv1.AuthorInput.favorite_categories:type_name -> productv1.CategoryInput + 14, // 175: productv1.AuthorInput.author_groups:type_name -> productv1.ListOfListOfUserInput + 14, // 176: productv1.AuthorInput.project_teams:type_name -> productv1.ListOfListOfUserInput + 154, // 177: productv1.NestedTypeB.c:type_name -> productv1.NestedTypeC + 156, // 178: productv1.FilterType.pagination:type_name -> productv1.Pagination + 17, // 179: productv1.OrderLineInput.modifiers:type_name -> productv1.ListOfString + 17, // 180: productv1.OrderLine.modifiers:type_name -> productv1.ListOfString + 188, // 181: productv1.Subcategory.description:type_name -> google.protobuf.StringValue + 0, // 182: productv1.CategoryInput.kind:type_name -> productv1.CategoryKind + 189, // 183: productv1.ProductCountFilter.min_price:type_name -> google.protobuf.DoubleValue + 189, // 184: productv1.ProductCountFilter.max_price:type_name -> google.protobuf.DoubleValue + 190, // 185: productv1.ProductCountFilter.in_stock:type_name -> google.protobuf.BoolValue + 188, // 186: productv1.ProductCountFilter.search_term:type_name -> google.protobuf.StringValue + 189, // 187: productv1.SubcategoryItemFilter.min_price:type_name -> google.protobuf.DoubleValue + 189, // 188: productv1.SubcategoryItemFilter.max_price:type_name -> google.protobuf.DoubleValue + 190, // 189: productv1.SubcategoryItemFilter.in_stock:type_name -> google.protobuf.BoolValue + 190, // 190: productv1.SubcategoryItemFilter.is_active:type_name -> google.protobuf.BoolValue + 188, // 191: productv1.SubcategoryItemFilter.search_term:type_name -> google.protobuf.StringValue + 146, // 192: productv1.ListOfAuthorFilter.List.items:type_name -> productv1.AuthorFilter + 152, // 193: productv1.ListOfAuthorInput.List.items:type_name -> productv1.AuthorInput + 143, // 194: productv1.ListOfBlogPost.List.items:type_name -> productv1.BlogPost + 144, // 195: productv1.ListOfBlogPostFilter.List.items:type_name -> productv1.BlogPostFilter + 151, // 196: productv1.ListOfBlogPostInput.List.items:type_name -> productv1.BlogPostInput + 136, // 197: productv1.ListOfCategory.List.items:type_name -> productv1.Category + 164, // 198: productv1.ListOfCategoryInput.List.items:type_name -> productv1.CategoryInput + 7, // 199: productv1.ListOfListOfCategory.List.items:type_name -> productv1.ListOfCategory + 8, // 200: productv1.ListOfListOfCategoryInput.List.items:type_name -> productv1.ListOfCategoryInput + 17, // 201: productv1.ListOfListOfString.List.items:type_name -> productv1.ListOfString + 19, // 202: productv1.ListOfListOfUser.List.items:type_name -> productv1.ListOfUser + 20, // 203: productv1.ListOfListOfUserInput.List.items:type_name -> productv1.ListOfUserInput + 158, // 204: productv1.ListOfOrderLine.List.items:type_name -> productv1.OrderLine + 124, // 205: productv1.ListOfProduct.List.items:type_name -> productv1.Product + 159, // 206: productv1.ListOfSubcategory.List.items:type_name -> productv1.Subcategory + 127, // 207: productv1.ListOfUser.List.items:type_name -> productv1.User + 147, // 208: productv1.ListOfUserInput.List.items:type_name -> productv1.UserInput + 22, // 209: productv1.ProductService.LookupProductById:input_type -> productv1.LookupProductByIdRequest + 25, // 210: productv1.ProductService.LookupStorageById:input_type -> productv1.LookupStorageByIdRequest + 28, // 211: productv1.ProductService.LookupWarehouseById:input_type -> productv1.LookupWarehouseByIdRequest + 106, // 212: productv1.ProductService.MutationBulkCreateAuthors:input_type -> productv1.MutationBulkCreateAuthorsRequest + 110, // 213: productv1.ProductService.MutationBulkCreateBlogPosts:input_type -> productv1.MutationBulkCreateBlogPostsRequest + 108, // 214: productv1.ProductService.MutationBulkUpdateAuthors:input_type -> productv1.MutationBulkUpdateAuthorsRequest + 112, // 215: productv1.ProductService.MutationBulkUpdateBlogPosts:input_type -> productv1.MutationBulkUpdateBlogPostsRequest + 102, // 216: productv1.ProductService.MutationCreateAuthor:input_type -> productv1.MutationCreateAuthorRequest + 98, // 217: productv1.ProductService.MutationCreateBlogPost:input_type -> productv1.MutationCreateBlogPostRequest + 94, // 218: productv1.ProductService.MutationCreateNullableFieldsType:input_type -> productv1.MutationCreateNullableFieldsTypeRequest + 90, // 219: productv1.ProductService.MutationCreateUser:input_type -> productv1.MutationCreateUserRequest + 92, // 220: productv1.ProductService.MutationPerformAction:input_type -> productv1.MutationPerformActionRequest + 104, // 221: productv1.ProductService.MutationUpdateAuthor:input_type -> productv1.MutationUpdateAuthorRequest + 100, // 222: productv1.ProductService.MutationUpdateBlogPost:input_type -> productv1.MutationUpdateBlogPostRequest + 96, // 223: productv1.ProductService.MutationUpdateNullableFieldsType:input_type -> productv1.MutationUpdateNullableFieldsTypeRequest + 84, // 224: productv1.ProductService.QueryAllAuthors:input_type -> productv1.QueryAllAuthorsRequest + 76, // 225: productv1.ProductService.QueryAllBlogPosts:input_type -> productv1.QueryAllBlogPostsRequest + 68, // 226: productv1.ProductService.QueryAllNullableFieldsTypes:input_type -> productv1.QueryAllNullableFieldsTypesRequest + 56, // 227: productv1.ProductService.QueryAllPets:input_type -> productv1.QueryAllPetsRequest + 78, // 228: productv1.ProductService.QueryAuthor:input_type -> productv1.QueryAuthorRequest + 80, // 229: productv1.ProductService.QueryAuthorById:input_type -> productv1.QueryAuthorByIdRequest + 82, // 230: productv1.ProductService.QueryAuthorsWithFilter:input_type -> productv1.QueryAuthorsWithFilterRequest + 70, // 231: productv1.ProductService.QueryBlogPost:input_type -> productv1.QueryBlogPostRequest + 72, // 232: productv1.ProductService.QueryBlogPostById:input_type -> productv1.QueryBlogPostByIdRequest + 74, // 233: productv1.ProductService.QueryBlogPostsWithFilter:input_type -> productv1.QueryBlogPostsWithFilterRequest + 86, // 234: productv1.ProductService.QueryBulkSearchAuthors:input_type -> productv1.QueryBulkSearchAuthorsRequest + 88, // 235: productv1.ProductService.QueryBulkSearchBlogPosts:input_type -> productv1.QueryBulkSearchBlogPostsRequest + 44, // 236: productv1.ProductService.QueryCalculateTotals:input_type -> productv1.QueryCalculateTotalsRequest + 46, // 237: productv1.ProductService.QueryCategories:input_type -> productv1.QueryCategoriesRequest + 48, // 238: productv1.ProductService.QueryCategoriesByKind:input_type -> productv1.QueryCategoriesByKindRequest + 50, // 239: productv1.ProductService.QueryCategoriesByKinds:input_type -> productv1.QueryCategoriesByKindsRequest + 42, // 240: productv1.ProductService.QueryComplexFilterType:input_type -> productv1.QueryComplexFilterTypeRequest + 52, // 241: productv1.ProductService.QueryFilterCategories:input_type -> productv1.QueryFilterCategoriesRequest + 34, // 242: productv1.ProductService.QueryNestedType:input_type -> productv1.QueryNestedTypeRequest + 62, // 243: productv1.ProductService.QueryNullableFieldsType:input_type -> productv1.QueryNullableFieldsTypeRequest + 64, // 244: productv1.ProductService.QueryNullableFieldsTypeById:input_type -> productv1.QueryNullableFieldsTypeByIdRequest + 66, // 245: productv1.ProductService.QueryNullableFieldsTypeWithFilter:input_type -> productv1.QueryNullableFieldsTypeWithFilterRequest + 54, // 246: productv1.ProductService.QueryRandomPet:input_type -> productv1.QueryRandomPetRequest + 60, // 247: productv1.ProductService.QueryRandomSearchResult:input_type -> productv1.QueryRandomSearchResultRequest + 36, // 248: productv1.ProductService.QueryRecursiveType:input_type -> productv1.QueryRecursiveTypeRequest + 58, // 249: productv1.ProductService.QuerySearch:input_type -> productv1.QuerySearchRequest + 38, // 250: productv1.ProductService.QueryTypeFilterWithArguments:input_type -> productv1.QueryTypeFilterWithArgumentsRequest + 40, // 251: productv1.ProductService.QueryTypeWithMultipleFilterFields:input_type -> productv1.QueryTypeWithMultipleFilterFieldsRequest + 32, // 252: productv1.ProductService.QueryUser:input_type -> productv1.QueryUserRequest + 30, // 253: productv1.ProductService.QueryUsers:input_type -> productv1.QueryUsersRequest + 116, // 254: productv1.ProductService.ResolveCategoryProductCount:input_type -> productv1.ResolveCategoryProductCountRequest + 121, // 255: productv1.ProductService.ResolveSubcategoryItemCount:input_type -> productv1.ResolveSubcategoryItemCountRequest + 23, // 256: productv1.ProductService.LookupProductById:output_type -> productv1.LookupProductByIdResponse + 26, // 257: productv1.ProductService.LookupStorageById:output_type -> productv1.LookupStorageByIdResponse + 29, // 258: productv1.ProductService.LookupWarehouseById:output_type -> productv1.LookupWarehouseByIdResponse + 107, // 259: productv1.ProductService.MutationBulkCreateAuthors:output_type -> productv1.MutationBulkCreateAuthorsResponse + 111, // 260: productv1.ProductService.MutationBulkCreateBlogPosts:output_type -> productv1.MutationBulkCreateBlogPostsResponse + 109, // 261: productv1.ProductService.MutationBulkUpdateAuthors:output_type -> productv1.MutationBulkUpdateAuthorsResponse + 113, // 262: productv1.ProductService.MutationBulkUpdateBlogPosts:output_type -> productv1.MutationBulkUpdateBlogPostsResponse + 103, // 263: productv1.ProductService.MutationCreateAuthor:output_type -> productv1.MutationCreateAuthorResponse + 99, // 264: productv1.ProductService.MutationCreateBlogPost:output_type -> productv1.MutationCreateBlogPostResponse + 95, // 265: productv1.ProductService.MutationCreateNullableFieldsType:output_type -> productv1.MutationCreateNullableFieldsTypeResponse + 91, // 266: productv1.ProductService.MutationCreateUser:output_type -> productv1.MutationCreateUserResponse + 93, // 267: productv1.ProductService.MutationPerformAction:output_type -> productv1.MutationPerformActionResponse + 105, // 268: productv1.ProductService.MutationUpdateAuthor:output_type -> productv1.MutationUpdateAuthorResponse + 101, // 269: productv1.ProductService.MutationUpdateBlogPost:output_type -> productv1.MutationUpdateBlogPostResponse + 97, // 270: productv1.ProductService.MutationUpdateNullableFieldsType:output_type -> productv1.MutationUpdateNullableFieldsTypeResponse + 85, // 271: productv1.ProductService.QueryAllAuthors:output_type -> productv1.QueryAllAuthorsResponse + 77, // 272: productv1.ProductService.QueryAllBlogPosts:output_type -> productv1.QueryAllBlogPostsResponse + 69, // 273: productv1.ProductService.QueryAllNullableFieldsTypes:output_type -> productv1.QueryAllNullableFieldsTypesResponse + 57, // 274: productv1.ProductService.QueryAllPets:output_type -> productv1.QueryAllPetsResponse + 79, // 275: productv1.ProductService.QueryAuthor:output_type -> productv1.QueryAuthorResponse + 81, // 276: productv1.ProductService.QueryAuthorById:output_type -> productv1.QueryAuthorByIdResponse + 83, // 277: productv1.ProductService.QueryAuthorsWithFilter:output_type -> productv1.QueryAuthorsWithFilterResponse + 71, // 278: productv1.ProductService.QueryBlogPost:output_type -> productv1.QueryBlogPostResponse + 73, // 279: productv1.ProductService.QueryBlogPostById:output_type -> productv1.QueryBlogPostByIdResponse + 75, // 280: productv1.ProductService.QueryBlogPostsWithFilter:output_type -> productv1.QueryBlogPostsWithFilterResponse + 87, // 281: productv1.ProductService.QueryBulkSearchAuthors:output_type -> productv1.QueryBulkSearchAuthorsResponse + 89, // 282: productv1.ProductService.QueryBulkSearchBlogPosts:output_type -> productv1.QueryBulkSearchBlogPostsResponse + 45, // 283: productv1.ProductService.QueryCalculateTotals:output_type -> productv1.QueryCalculateTotalsResponse + 47, // 284: productv1.ProductService.QueryCategories:output_type -> productv1.QueryCategoriesResponse + 49, // 285: productv1.ProductService.QueryCategoriesByKind:output_type -> productv1.QueryCategoriesByKindResponse + 51, // 286: productv1.ProductService.QueryCategoriesByKinds:output_type -> productv1.QueryCategoriesByKindsResponse + 43, // 287: productv1.ProductService.QueryComplexFilterType:output_type -> productv1.QueryComplexFilterTypeResponse + 53, // 288: productv1.ProductService.QueryFilterCategories:output_type -> productv1.QueryFilterCategoriesResponse + 35, // 289: productv1.ProductService.QueryNestedType:output_type -> productv1.QueryNestedTypeResponse + 63, // 290: productv1.ProductService.QueryNullableFieldsType:output_type -> productv1.QueryNullableFieldsTypeResponse + 65, // 291: productv1.ProductService.QueryNullableFieldsTypeById:output_type -> productv1.QueryNullableFieldsTypeByIdResponse + 67, // 292: productv1.ProductService.QueryNullableFieldsTypeWithFilter:output_type -> productv1.QueryNullableFieldsTypeWithFilterResponse + 55, // 293: productv1.ProductService.QueryRandomPet:output_type -> productv1.QueryRandomPetResponse + 61, // 294: productv1.ProductService.QueryRandomSearchResult:output_type -> productv1.QueryRandomSearchResultResponse + 37, // 295: productv1.ProductService.QueryRecursiveType:output_type -> productv1.QueryRecursiveTypeResponse + 59, // 296: productv1.ProductService.QuerySearch:output_type -> productv1.QuerySearchResponse + 39, // 297: productv1.ProductService.QueryTypeFilterWithArguments:output_type -> productv1.QueryTypeFilterWithArgumentsResponse + 41, // 298: productv1.ProductService.QueryTypeWithMultipleFilterFields:output_type -> productv1.QueryTypeWithMultipleFilterFieldsResponse + 33, // 299: productv1.ProductService.QueryUser:output_type -> productv1.QueryUserResponse + 31, // 300: productv1.ProductService.QueryUsers:output_type -> productv1.QueryUsersResponse + 118, // 301: productv1.ProductService.ResolveCategoryProductCount:output_type -> productv1.ResolveCategoryProductCountResponse + 123, // 302: productv1.ProductService.ResolveSubcategoryItemCount:output_type -> productv1.ResolveSubcategoryItemCountResponse + 256, // [256:303] is the sub-list for method output_type + 209, // [209:256] is the sub-list for method input_type + 209, // [209:209] is the sub-list for extension type_name + 209, // [209:209] is the sub-list for extension extendee + 0, // [0:209] is the sub-list for field type_name } func init() { file_product_proto_init() } @@ -10178,16 +10704,16 @@ func file_product_proto_init() { if File_product_proto != nil { return } - file_product_proto_msgTypes[131].OneofWrappers = []any{ + file_product_proto_msgTypes[137].OneofWrappers = []any{ (*Animal_Cat)(nil), (*Animal_Dog)(nil), } - file_product_proto_msgTypes[133].OneofWrappers = []any{ + file_product_proto_msgTypes[139].OneofWrappers = []any{ (*SearchResult_Product)(nil), (*SearchResult_User)(nil), (*SearchResult_Category)(nil), } - file_product_proto_msgTypes[142].OneofWrappers = []any{ + file_product_proto_msgTypes[148].OneofWrappers = []any{ (*ActionResult_ActionSuccess)(nil), (*ActionResult_ActionError)(nil), } @@ -10197,7 +10723,7 @@ func file_product_proto_init() { GoPackagePath: reflect.TypeOf(x{}).PkgPath(), RawDescriptor: unsafe.Slice(unsafe.StringData(file_product_proto_rawDesc), len(file_product_proto_rawDesc)), NumEnums: 1, - NumMessages: 177, + NumMessages: 186, NumExtensions: 0, NumServices: 1, }, diff --git a/v2/pkg/grpctest/productv1/product_grpc.pb.go b/v2/pkg/grpctest/productv1/product_grpc.pb.go index 3821e954b..dff02dd32 100644 --- a/v2/pkg/grpctest/productv1/product_grpc.pb.go +++ b/v2/pkg/grpctest/productv1/product_grpc.pb.go @@ -65,6 +65,7 @@ const ( ProductService_QueryUser_FullMethodName = "/productv1.ProductService/QueryUser" ProductService_QueryUsers_FullMethodName = "/productv1.ProductService/QueryUsers" ProductService_ResolveCategoryProductCount_FullMethodName = "/productv1.ProductService/ResolveCategoryProductCount" + ProductService_ResolveSubcategoryItemCount_FullMethodName = "/productv1.ProductService/ResolveSubcategoryItemCount" ) // ProductServiceClient is the client API for ProductService service. @@ -122,6 +123,7 @@ type ProductServiceClient interface { QueryUser(ctx context.Context, in *QueryUserRequest, opts ...grpc.CallOption) (*QueryUserResponse, error) QueryUsers(ctx context.Context, in *QueryUsersRequest, opts ...grpc.CallOption) (*QueryUsersResponse, error) ResolveCategoryProductCount(ctx context.Context, in *ResolveCategoryProductCountRequest, opts ...grpc.CallOption) (*ResolveCategoryProductCountResponse, error) + ResolveSubcategoryItemCount(ctx context.Context, in *ResolveSubcategoryItemCountRequest, opts ...grpc.CallOption) (*ResolveSubcategoryItemCountResponse, error) } type productServiceClient struct { @@ -592,6 +594,16 @@ func (c *productServiceClient) ResolveCategoryProductCount(ctx context.Context, return out, nil } +func (c *productServiceClient) ResolveSubcategoryItemCount(ctx context.Context, in *ResolveSubcategoryItemCountRequest, opts ...grpc.CallOption) (*ResolveSubcategoryItemCountResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(ResolveSubcategoryItemCountResponse) + err := c.cc.Invoke(ctx, ProductService_ResolveSubcategoryItemCount_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + // ProductServiceServer is the server API for ProductService service. // All implementations must embed UnimplementedProductServiceServer // for forward compatibility. @@ -647,6 +659,7 @@ type ProductServiceServer interface { QueryUser(context.Context, *QueryUserRequest) (*QueryUserResponse, error) QueryUsers(context.Context, *QueryUsersRequest) (*QueryUsersResponse, error) ResolveCategoryProductCount(context.Context, *ResolveCategoryProductCountRequest) (*ResolveCategoryProductCountResponse, error) + ResolveSubcategoryItemCount(context.Context, *ResolveSubcategoryItemCountRequest) (*ResolveSubcategoryItemCountResponse, error) mustEmbedUnimplementedProductServiceServer() } @@ -795,6 +808,9 @@ func (UnimplementedProductServiceServer) QueryUsers(context.Context, *QueryUsers func (UnimplementedProductServiceServer) ResolveCategoryProductCount(context.Context, *ResolveCategoryProductCountRequest) (*ResolveCategoryProductCountResponse, error) { return nil, status.Errorf(codes.Unimplemented, "method ResolveCategoryProductCount not implemented") } +func (UnimplementedProductServiceServer) ResolveSubcategoryItemCount(context.Context, *ResolveSubcategoryItemCountRequest) (*ResolveSubcategoryItemCountResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method ResolveSubcategoryItemCount not implemented") +} func (UnimplementedProductServiceServer) mustEmbedUnimplementedProductServiceServer() {} func (UnimplementedProductServiceServer) testEmbeddedByValue() {} @@ -1644,6 +1660,24 @@ func _ProductService_ResolveCategoryProductCount_Handler(srv interface{}, ctx co return interceptor(ctx, in, info, handler) } +func _ProductService_ResolveSubcategoryItemCount_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(ResolveSubcategoryItemCountRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(ProductServiceServer).ResolveSubcategoryItemCount(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: ProductService_ResolveSubcategoryItemCount_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(ProductServiceServer).ResolveSubcategoryItemCount(ctx, req.(*ResolveSubcategoryItemCountRequest)) + } + return interceptor(ctx, in, info, handler) +} + // ProductService_ServiceDesc is the grpc.ServiceDesc for ProductService service. // It's only intended for direct use with grpc.RegisterService, // and not to be introspected or modified (even as a copy) @@ -1835,6 +1869,10 @@ var ProductService_ServiceDesc = grpc.ServiceDesc{ MethodName: "ResolveCategoryProductCount", Handler: _ProductService_ResolveCategoryProductCount_Handler, }, + { + MethodName: "ResolveSubcategoryItemCount", + Handler: _ProductService_ResolveSubcategoryItemCount_Handler, + }, }, Streams: []grpc.StreamDesc{}, Metadata: "product.proto", diff --git a/v2/pkg/grpctest/testdata/products.graphqls b/v2/pkg/grpctest/testdata/products.graphqls index a273796a4..19f11c258 100644 --- a/v2/pkg/grpctest/testdata/products.graphqls +++ b/v2/pkg/grpctest/testdata/products.graphqls @@ -117,6 +117,15 @@ type Category { name: String! kind: CategoryKind! productCount(filters: ProductCountFilter): Int! @resolved(context: "id name") + subcategories: [Subcategory!] +} + +type Subcategory { + id: ID! + name: String! + description: String + isActive: Boolean! + itemCount(filters: SubcategoryItemFilter): Int! @resolved(context: "id") } enum CategoryKind { @@ -325,6 +334,14 @@ input ProductCountFilter { searchTerm: String } +input SubcategoryItemFilter { + minPrice: Float + maxPrice: Float + inStock: Boolean + isActive: Boolean + searchTerm: String +} + type Query { _entities(representations: [_Any!]!): [_Entity!]! users: [User!]! From a84ccee0f697ec8bc969a7d146ab0b6a0d9072ff Mon Sep 17 00:00:00 2001 From: Ludwig Bedacht Date: Wed, 8 Oct 2025 15:51:55 +0200 Subject: [PATCH 05/72] chore: update planner to handle optional types --- .../grpc_datasource/execution_plan.go | 77 +- .../execution_plan_field_resolvers_test.go | 314 ++- .../grpc_datasource/execution_plan_visitor.go | 62 +- .../grpc_datasource/mapping_test_helper.go | 51 + .../required_fields_visitor.go | 6 +- v2/pkg/grpctest/cmd/mapping_helper/main.go | 46 +- v2/pkg/grpctest/mapping/mapping.go | 51 + v2/pkg/grpctest/mockservice.go | 61 + v2/pkg/grpctest/product.proto | 57 + v2/pkg/grpctest/productv1/product.pb.go | 1722 +++++++++++------ v2/pkg/grpctest/productv1/product_grpc.pb.go | 76 + v2/pkg/grpctest/testdata/products.graphqls | 10 + 12 files changed, 1924 insertions(+), 609 deletions(-) diff --git a/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go b/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go index dde7348ba..5471627c0 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go +++ b/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go @@ -832,10 +832,61 @@ type resolveRPCCallConfig struct { fieldArgsMessage *RPCMessage } -func (r *rpcPlanningContext) newResolveRPCCall(config resolveRPCCallConfig) RPCCall { +func (r *rpcPlanningContext) resolveRequiredFields(typeName, requiredFields string) (*RPCMessage, error) { + walker := astvisitor.WalkerFromPool() + defer walker.Release() + message := &RPCMessage{ + Name: typeName, + } + + rfv := newRequiredFieldsVisitor(walker, message, r) + if err := rfv.visitWithMemberTypes(r.definition, typeName, requiredFields, nil); err != nil { + return nil, err + } + return message, nil +} + +func (r *rpcPlanningContext) newResolveRPCCall(config resolveRPCCallConfig) (RPCCall, error) { resolveConfig := config.resolveConfig resolvedField := config.resolvedField + underlyingTypeRef := r.definition.ResolveUnderlyingType(resolvedField.fieldDefinitionTypeRef) + fieldTypeName := r.definition.ResolveTypeNameString(underlyingTypeRef) + dataType := r.toDataType(&r.definition.Types[underlyingTypeRef]) + + var responseFieldsMessage *RPCMessage + if dataType == DataTypeMessage { + var err error + responseFieldsMessage, err = r.resolveRequiredFields(fieldTypeName, resolvedField.requiredFields) + if err != nil { + return RPCCall{}, err + } + } + + response := RPCMessage{ + Name: resolveConfig.Response, + Fields: RPCFields{ + { + Name: "result", + TypeName: string(DataTypeMessage), + JSONPath: "result", + Repeated: true, + Message: &RPCMessage{ + Name: resolveConfig.RPC + "Result", + Fields: RPCFields{ + { + Name: resolveConfig.FieldMappingData.TargetName, + TypeName: string(dataType), + JSONPath: config.fieldName, + Message: responseFieldsMessage, + Optional: !r.definition.TypeIsNonNull(resolvedField.fieldDefinitionTypeRef), + }, + }, + }, + }, + }, + } + return RPCCall{ DependentCalls: []int{resolvedField.callerRef}, ResponsePath: resolvedField.responsePath, @@ -859,26 +910,6 @@ func (r *rpcPlanningContext) newResolveRPCCall(config resolveRPCCallConfig) RPCC }, }, }, - Response: RPCMessage{ - Name: resolveConfig.Response, - Fields: RPCFields{ - { - Name: "result", - TypeName: string(DataTypeMessage), - JSONPath: "result", - Repeated: true, - Message: &RPCMessage{ - Name: resolveConfig.RPC + "Result", - Fields: RPCFields{ - { - Name: resolveConfig.FieldMappingData.TargetName, - TypeName: string(DataTypeInt32), - JSONPath: config.fieldName, - }, - }, - }, - }, - }, - }, - } + Response: response, + }, nil } diff --git a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_field_resolvers_test.go b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_field_resolvers_test.go index f189a671e..60f8f48db 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_field_resolvers_test.go +++ b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_field_resolvers_test.go @@ -393,7 +393,7 @@ func TestExecutionPlanFieldResolvers(t *testing.T) { }, }, { - name: "Should create an execution plan for a query with nullable fields type", + name: "Should create an execution plan for a query with nullable lists type", query: "query SubcategoriesWithFieldResolvers($filter: SubcategoryItemFilter) { categories { id subcategories { id name description isActive itemCount(filters: $filter) } } }", expectedPlan: &RPCExecutionPlan{ Calls: []RPCCall{ @@ -567,6 +567,318 @@ func TestExecutionPlanFieldResolvers(t *testing.T) { }, }, }, + { + name: "Should create an execution plan for a query a field resolver with a message type", + query: "query CategoriesWithNullableTypes($metricType: String) { categories { categoryMetrics(metricType: $metricType) { id metricType value } } }", + expectedPlan: &RPCExecutionPlan{ + Calls: []RPCCall{ + { + ServiceName: "Products", + MethodName: "QueryCategories", + Request: RPCMessage{ + Name: "QueryCategoriesRequest", + }, + Response: RPCMessage{ + Name: "QueryCategoriesResponse", + Fields: []RPCField{ + { + Name: "categories", + TypeName: string(DataTypeMessage), + JSONPath: "categories", + Repeated: true, + Message: &RPCMessage{ + Name: "Category", + Fields: []RPCField{}, + }, + }, + }, + }, + }, + { + DependentCalls: []int{0}, + Kind: CallKindResolve, + ServiceName: "Products", + MethodName: "ResolveCategoryCategoryMetrics", + ResponsePath: buildPath("categories.categoryMetrics"), + Request: RPCMessage{ + Name: "ResolveCategoryCategoryMetricsRequest", + Fields: []RPCField{ + { + Name: "context", + TypeName: string(DataTypeMessage), + JSONPath: "", + Repeated: true, + Message: &RPCMessage{ + Name: "ResolveCategoryCategoryMetricsContext", + Fields: []RPCField{ + { + Name: "id", + TypeName: string(DataTypeString), + JSONPath: "id", + ResolvePath: buildPath("categories.id"), + }, + { + Name: "name", + TypeName: string(DataTypeString), + JSONPath: "name", + ResolvePath: buildPath("categories.name"), + }, + }, + }, + }, + { + Name: "field_args", + TypeName: string(DataTypeMessage), + JSONPath: "", + Message: &RPCMessage{ + Name: "ResolveCategoryCategoryMetricsArgs", + Fields: []RPCField{ + { + Name: "metric_type", + TypeName: string(DataTypeString), + JSONPath: "metricType", + Optional: false, + }, + }, + }, + }, + }, + }, + Response: RPCMessage{ + Name: "ResolveCategoryCategoryMetricsResponse", + Fields: []RPCField{ + { + Name: "result", + TypeName: string(DataTypeMessage), + JSONPath: "result", + Repeated: true, + Message: &RPCMessage{ + Name: "ResolveCategoryCategoryMetricsResult", + Fields: []RPCField{ + { + Name: "category_metrics", + TypeName: string(DataTypeMessage), + JSONPath: "categoryMetrics", + Optional: true, + Message: &RPCMessage{ + Name: "CategoryMetrics", + Fields: []RPCField{ + { + Name: "id", + TypeName: string(DataTypeString), + JSONPath: "id", + }, + { + Name: "metric_type", + TypeName: string(DataTypeString), + JSONPath: "metricType", + }, + { + Name: "value", + TypeName: string(DataTypeDouble), + JSONPath: "value", + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + { + name: "Should create an execution plan for a query with nullable types", + query: "query CategoriesWithNullableTypes($threshold: Int, $metricType: String) { categories { popularityScore(threshold: $threshold) categoryMetrics(metricType: $metricType) { id metricType value } } }", + expectedPlan: &RPCExecutionPlan{ + Calls: []RPCCall{ + { + ServiceName: "Products", + MethodName: "QueryCategories", + Request: RPCMessage{ + Name: "QueryCategoriesRequest", + }, + Response: RPCMessage{ + Name: "QueryCategoriesResponse", + Fields: []RPCField{ + { + Name: "categories", + TypeName: string(DataTypeMessage), + JSONPath: "categories", + Repeated: true, + Message: &RPCMessage{ + Name: "Category", + Fields: []RPCField{}, + }, + }, + }, + }, + }, + { + DependentCalls: []int{0}, + ServiceName: "Products", + MethodName: "ResolveCategoryPopularityScore", + Kind: CallKindResolve, + ResponsePath: buildPath("categories.popularityScore"), + Request: RPCMessage{ + Name: "ResolveCategoryPopularityScoreRequest", + Fields: []RPCField{ + { + Name: "context", + TypeName: string(DataTypeMessage), + JSONPath: "", + Repeated: true, + Message: &RPCMessage{ + Name: "ResolveCategoryPopularityScoreContext", + Fields: []RPCField{ + { + Name: "id", + TypeName: string(DataTypeString), + JSONPath: "id", + ResolvePath: buildPath("categories.id"), + }, + }, + }, + }, + { + Name: "field_args", + TypeName: string(DataTypeMessage), + JSONPath: "", + Message: &RPCMessage{ + Name: "ResolveCategoryPopularityScoreArgs", + Fields: []RPCField{ + { + Name: "threshold", + TypeName: string(DataTypeInt32), + JSONPath: "threshold", + Optional: true, + }, + }, + }, + }, + }, + }, + Response: RPCMessage{ + Name: "ResolveCategoryPopularityScoreResponse", + Fields: []RPCField{ + { + Name: "result", + TypeName: string(DataTypeMessage), + JSONPath: "result", + Repeated: true, + Message: &RPCMessage{ + Name: "ResolveCategoryPopularityScoreResult", + Fields: []RPCField{ + { + Name: "popularity_score", + TypeName: string(DataTypeInt32), + JSONPath: "popularityScore", + Optional: true, + }, + }, + }, + }, + }, + }, + }, + { + DependentCalls: []int{0}, + ServiceName: "Products", + MethodName: "ResolveCategoryCategoryMetrics", + Kind: CallKindResolve, + ResponsePath: buildPath("categories.categoryMetrics"), + Request: RPCMessage{ + Name: "ResolveCategoryCategoryMetricsRequest", + Fields: []RPCField{ + { + Name: "context", + TypeName: string(DataTypeMessage), + JSONPath: "", + Repeated: true, + Message: &RPCMessage{ + Name: "ResolveCategoryCategoryMetricsContext", + Fields: []RPCField{ + { + Name: "id", + TypeName: string(DataTypeString), + JSONPath: "id", + ResolvePath: buildPath("categories.id"), + }, + { + Name: "name", + TypeName: string(DataTypeString), + JSONPath: "name", + ResolvePath: buildPath("categories.name"), + }, + }, + }, + }, + { + Name: "field_args", + TypeName: string(DataTypeMessage), + JSONPath: "", + Message: &RPCMessage{ + Name: "ResolveCategoryCategoryMetricsArgs", + Fields: []RPCField{ + { + Name: "metric_type", + TypeName: string(DataTypeString), + JSONPath: "metricType", + }, + }, + }, + }, + }, + }, + Response: RPCMessage{ + Name: "ResolveCategoryCategoryMetricsResponse", + Fields: []RPCField{ + { + Name: "result", + TypeName: string(DataTypeMessage), + JSONPath: "result", + Repeated: true, + Message: &RPCMessage{ + Name: "ResolveCategoryCategoryMetricsResult", + Fields: []RPCField{ + { + Name: "category_metrics", + TypeName: string(DataTypeMessage), + JSONPath: "categoryMetrics", + Optional: true, + Message: &RPCMessage{ + Name: "CategoryMetrics", + Fields: []RPCField{ + { + Name: "id", + TypeName: string(DataTypeString), + JSONPath: "id", + }, + { + Name: "metric_type", + TypeName: string(DataTypeString), + JSONPath: "metricType", + }, + { + Name: "value", + TypeName: string(DataTypeDouble), + JSONPath: "value", + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, } for _, tt := range tests { diff --git a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor.go b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor.go index 27e530557..a8d50c3f5 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor.go +++ b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor.go @@ -11,6 +11,7 @@ import ( "github.com/wundergraph/graphql-go-tools/v2/pkg/ast" "github.com/wundergraph/graphql-go-tools/v2/pkg/astvisitor" "github.com/wundergraph/graphql-go-tools/v2/pkg/engine/plan" + "github.com/wundergraph/graphql-go-tools/v2/pkg/internal/unsafebytes" "github.com/wundergraph/graphql-go-tools/v2/pkg/operationreport" ) @@ -40,10 +41,12 @@ type fieldArgument struct { } type resolvedField struct { - callerRef int - parentTypeRef int - fieldRef int - responsePath ast.Path + callerRef int + parentTypeRef int + fieldRef int + fieldDefinitionTypeRef int + requiredFields string + responsePath ast.Path contextFields []contextField fieldArguments []fieldArgument @@ -64,8 +67,9 @@ type rpcPlanVisitor struct { currentCall *RPCCall currentCallID int - relatedCallID int - resolvedFields []resolvedField + relatedCallID int + resolvedFieldIndex int + resolvedFields []resolvedField fieldPath ast.Path } @@ -81,14 +85,15 @@ type rpcPlanVisitorConfig struct { func newRPCPlanVisitor(config rpcPlanVisitorConfig) *rpcPlanVisitor { walker := astvisitor.NewWalker(48) visitor := &rpcPlanVisitor{ - walker: &walker, - plan: &RPCExecutionPlan{}, - subgraphName: cases.Title(language.Und, cases.NoLower).String(config.subgraphName), - mapping: config.mapping, - operationFieldRef: -1, - resolvedFields: make([]resolvedField, 0), - relatedCallID: -1, - fieldPath: make(ast.Path, 0), + walker: &walker, + plan: &RPCExecutionPlan{}, + subgraphName: cases.Title(language.Und, cases.NoLower).String(config.subgraphName), + mapping: config.mapping, + operationFieldRef: ast.InvalidRef, + resolvedFields: make([]resolvedField, 0), + relatedCallID: ast.InvalidRef, + resolvedFieldIndex: ast.InvalidRef, + fieldPath: make(ast.Path, 0), } walker.RegisterDocumentVisitor(visitor) @@ -146,7 +151,7 @@ func (r *rpcPlanVisitor) LeaveDocument(_, _ *ast.Document) { Name: resolveConfig.RPC + "Args", } - call := r.planCtx.newResolveRPCCall(resolveRPCCallConfig{ + call, err := r.planCtx.newResolveRPCCall(resolveRPCCallConfig{ serviceName: r.subgraphName, typeName: r.definition.ResolveTypeNameString(resolvedField.parentTypeRef), fieldName: r.operation.FieldAliasOrNameString(resolvedField.fieldRef), @@ -156,6 +161,10 @@ func (r *rpcPlanVisitor) LeaveDocument(_, _ *ast.Document) { fieldArgsMessage: fieldArgsMessage, }) + if err != nil { + r.walker.StopWithInternalErr(err) + } + contextMessage.Fields = make(RPCFields, 0, len(resolvedField.contextFields)) for _, contextField := range resolvedField.contextFields { typeDefNode, found := r.definition.NodeByNameStr(r.definition.ResolveTypeNameString(resolvedField.parentTypeRef)) @@ -267,6 +276,15 @@ func (r *rpcPlanVisitor) EnterSelectionSet(ref int) { return } + if r.resolvedFieldIndex != ast.InvalidRef { + lbrace := r.operation.SelectionSets[ref].LBrace.CharEnd + rbrace := r.operation.SelectionSets[ref].RBrace.CharStart - 1 + + r.resolvedFields[r.resolvedFieldIndex].requiredFields = unsafebytes.BytesToString(r.operation.Input.RawBytes[lbrace:rbrace]) + r.walker.SkipNode() + return + } + if len(r.planInfo.currentResponseMessage.Fields) == 0 || len(r.planInfo.currentResponseMessage.Fields) <= r.planInfo.currentResponseFieldIndex { return } @@ -412,12 +430,14 @@ func (r *rpcPlanVisitor) EnterField(ref int) { // We need to make sure to handle a hierarchy of arguments in order to perform parallel calls in order to retrieve the data. // TODO: this needs to be available for both visitors and added to the plancontext if fieldArgs := r.operation.FieldArguments(ref); !inRootField && len(fieldArgs) > 0 { - r.relatedCallID++ + // We don't want to add fields from the selection set to the actual call + r.relatedCallID++ // TODO: handle this for multiple queries resolvedField := resolvedField{ - callerRef: r.relatedCallID, - parentTypeRef: r.walker.EnclosingTypeDefinition.Ref, - fieldRef: ref, - responsePath: r.walker.Path[1:].WithoutInlineFragmentNames().WithFieldNameItem(r.operation.FieldAliasOrNameBytes(ref)), + callerRef: r.relatedCallID, + parentTypeRef: r.walker.EnclosingTypeDefinition.Ref, + fieldRef: ref, + responsePath: r.walker.Path[1:].WithoutInlineFragmentNames().WithFieldNameItem(r.operation.FieldAliasOrNameBytes(ref)), + fieldDefinitionTypeRef: r.definition.FieldDefinitionType(fd), } contextFields, err := r.planCtx.resolveContextFields(r.walker, fd) @@ -444,6 +464,7 @@ func (r *rpcPlanVisitor) EnterField(ref int) { resolvedField.fieldArguments = fieldArguments r.resolvedFields = append(r.resolvedFields, resolvedField) + r.resolvedFieldIndex = len(r.resolvedFields) - 1 r.fieldPath = r.fieldPath.WithFieldNameItem(r.operation.FieldNameBytes(ref)) return } @@ -479,6 +500,7 @@ func (r *rpcPlanVisitor) EnterField(ref int) { // LeaveField implements astvisitor.FieldVisitor. func (r *rpcPlanVisitor) LeaveField(ref int) { r.fieldPath = r.fieldPath.RemoveLastItem() + r.resolvedFieldIndex = ast.InvalidRef // If we are not in the operation field, we can increment the response field index. if !r.walker.InRootField() { diff --git a/v2/pkg/engine/datasource/grpc_datasource/mapping_test_helper.go b/v2/pkg/engine/datasource/grpc_datasource/mapping_test_helper.go index 64d673cd6..96c511be0 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/mapping_test_helper.go +++ b/v2/pkg/engine/datasource/grpc_datasource/mapping_test_helper.go @@ -231,6 +231,28 @@ func testMapping() *GRPCMapping { Request: "ResolveCategoryProductCountRequest", Response: "ResolveCategoryProductCountResponse", }, + "popularityScore": { + FieldMappingData: FieldMapData{ + TargetName: "popularity_score", + ArgumentMappings: FieldArgumentMap{ + "threshold": "threshold", + }, + }, + RPC: "ResolveCategoryPopularityScore", + Request: "ResolveCategoryPopularityScoreRequest", + Response: "ResolveCategoryPopularityScoreResponse", + }, + "categoryMetrics": { + FieldMappingData: FieldMapData{ + TargetName: "category_metrics", + ArgumentMappings: FieldArgumentMap{ + "metricType": "metric_type", + }, + }, + RPC: "ResolveCategoryCategoryMetrics", + Request: "ResolveCategoryCategoryMetricsRequest", + Response: "ResolveCategoryCategoryMetricsResponse", + }, }, "Subcategory": { "itemCount": { @@ -721,6 +743,18 @@ func testMapping() *GRPCMapping { "subcategories": { TargetName: "subcategories", }, + "popularityScore": { + TargetName: "popularity_score", + ArgumentMappings: FieldArgumentMap{ + "threshold": "threshold", + }, + }, + "categoryMetrics": { + TargetName: "category_metrics", + ArgumentMappings: FieldArgumentMap{ + "metricType": "metric_type", + }, + }, }, "Subcategory": { "id": { @@ -742,6 +776,23 @@ func testMapping() *GRPCMapping { }, }, }, + "CategoryMetrics": { + "id": { + TargetName: "id", + }, + "metricType": { + TargetName: "metric_type", + }, + "value": { + TargetName: "value", + }, + "timestamp": { + TargetName: "timestamp", + }, + "categoryId": { + TargetName: "category_id", + }, + }, "Cat": { "id": { TargetName: "id", diff --git a/v2/pkg/engine/datasource/grpc_datasource/required_fields_visitor.go b/v2/pkg/engine/datasource/grpc_datasource/required_fields_visitor.go index 3271a624a..c78f4ea6a 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/required_fields_visitor.go +++ b/v2/pkg/engine/datasource/grpc_datasource/required_fields_visitor.go @@ -46,12 +46,16 @@ func newRequiredFieldsVisitor(walker *astvisitor.Walker, message *RPCMessage, pl // It creates a new document with the required fields and walks it. // To achieve that we create a fragment with the required fields and walk it. func (r *requiredFieldsVisitor) visitRequiredFields(definition *ast.Document, typeName, requiredFields string) error { + return r.visitWithMemberTypes(definition, typeName, requiredFields, []string{typeName}) +} + +func (r *requiredFieldsVisitor) visitWithMemberTypes(definition *ast.Document, typeName, requiredFields string, memberTypes []string) error { doc, report := plan.RequiredFieldsFragment(typeName, requiredFields, false) if report.HasErrors() { return report } - r.message.MemberTypes = []string{typeName} + r.message.MemberTypes = memberTypes r.walker.Walk(doc, definition, report) if report.HasErrors() { return report diff --git a/v2/pkg/grpctest/cmd/mapping_helper/main.go b/v2/pkg/grpctest/cmd/mapping_helper/main.go index 550af405c..927579f79 100644 --- a/v2/pkg/grpctest/cmd/mapping_helper/main.go +++ b/v2/pkg/grpctest/cmd/mapping_helper/main.go @@ -8,6 +8,11 @@ import ( "text/template" ) +type TemplateData struct { + JSONMapping + ResolveRPCs map[string][]ResolveRPC +} + type JSONMapping struct { Version int `json:"version"` Service string `json:"service"` @@ -35,6 +40,12 @@ type EntityMapping struct { Response string `json:"response"` } +type ResolveRPC struct { + LookupMapping LookupMapping + RPC string + Request string + Response string +} type ResolveMapping struct { Type string `json:"type"` LookupMapping LookupMapping `json:"lookupMapping"` @@ -120,9 +131,9 @@ func DefaultGRPCMapping() *grpcdatasource.GRPCMapping { {{- end }} }, ResolveRPCs: grpcdatasource.RPCConfigMap[grpcdatasource.ResolveRPCMapping]{ - {{- range $index, $resolve := .ResolveMappings}} - {{- if eq $resolve.Type "LOOKUP_TYPE_RESOLVE"}} - "{{$resolve.LookupMapping.Type}}": { + {{- range $type, $item := .ResolveRPCs}} + "{{$type}}": { + {{- range $index, $resolve := $item}} "{{$resolve.LookupMapping.FieldMapping.Original}}": { FieldMappingData: grpcdatasource.FieldMapData{ TargetName: "{{$resolve.LookupMapping.FieldMapping.Mapped}}", @@ -136,9 +147,9 @@ func DefaultGRPCMapping() *grpcdatasource.GRPCMapping { Request: "{{$resolve.Request}}", Response: "{{$resolve.Response}}", }, + {{- end }} }, {{- end }} - {{- end }} }, EntityRPCs: map[string][]grpcdatasource.EntityRPCConfig{ {{- range $index, $entity := .EntityMappings}} @@ -213,10 +224,35 @@ func main() { log.Fatal(err) } + data := TemplateData{ + JSONMapping: mapping, + ResolveRPCs: make(map[string][]ResolveRPC), + } + + for _, mapping := range mapping.ResolveMappings { + if mapping.Type != "LOOKUP_TYPE_RESOLVE" { + continue + } + t := mapping.LookupMapping.Type + item, ok := data.ResolveRPCs[t] + if !ok { + item = []ResolveRPC{} + } + + item = append(item, ResolveRPC{ + LookupMapping: mapping.LookupMapping, + RPC: mapping.RPC, + Request: mapping.Request, + Response: mapping.Response, + }) + + data.ResolveRPCs[t] = item + } + t := template.Must(template.New("mapping").Parse(tpl)) buf := &bytes.Buffer{} - if err := t.Execute(buf, mapping); err != nil { + if err := t.Execute(buf, data); err != nil { log.Fatal(err) } diff --git a/v2/pkg/grpctest/mapping/mapping.go b/v2/pkg/grpctest/mapping/mapping.go index 839545834..988f22c4e 100644 --- a/v2/pkg/grpctest/mapping/mapping.go +++ b/v2/pkg/grpctest/mapping/mapping.go @@ -238,6 +238,28 @@ func DefaultGRPCMapping() *grpcdatasource.GRPCMapping { Request: "ResolveCategoryProductCountRequest", Response: "ResolveCategoryProductCountResponse", }, + "popularityScore": { + FieldMappingData: grpcdatasource.FieldMapData{ + TargetName: "popularity_score", + ArgumentMappings: grpcdatasource.FieldArgumentMap{ + "threshold": "threshold", + }, + }, + RPC: "ResolveCategoryPopularityScore", + Request: "ResolveCategoryPopularityScoreRequest", + Response: "ResolveCategoryPopularityScoreResponse", + }, + "categoryMetrics": { + FieldMappingData: grpcdatasource.FieldMapData{ + TargetName: "category_metrics", + ArgumentMappings: grpcdatasource.FieldArgumentMap{ + "metricType": "metric_type", + }, + }, + RPC: "ResolveCategoryCategoryMetrics", + Request: "ResolveCategoryCategoryMetricsRequest", + Response: "ResolveCategoryCategoryMetricsResponse", + }, }, "Subcategory": { "itemCount": { @@ -728,6 +750,18 @@ func DefaultGRPCMapping() *grpcdatasource.GRPCMapping { "subcategories": { TargetName: "subcategories", }, + "popularityScore": { + TargetName: "popularity_score", + ArgumentMappings: grpcdatasource.FieldArgumentMap{ + "threshold": "threshold", + }, + }, + "categoryMetrics": { + TargetName: "category_metrics", + ArgumentMappings: grpcdatasource.FieldArgumentMap{ + "metricType": "metric_type", + }, + }, }, "Subcategory": { "id": { @@ -749,6 +783,23 @@ func DefaultGRPCMapping() *grpcdatasource.GRPCMapping { }, }, }, + "CategoryMetrics": { + "id": { + TargetName: "id", + }, + "metricType": { + TargetName: "metric_type", + }, + "value": { + TargetName: "value", + }, + "timestamp": { + TargetName: "timestamp", + }, + "categoryId": { + TargetName: "category_id", + }, + }, "Cat": { "id": { TargetName: "id", diff --git a/v2/pkg/grpctest/mockservice.go b/v2/pkg/grpctest/mockservice.go index 4132fda37..682db49a7 100644 --- a/v2/pkg/grpctest/mockservice.go +++ b/v2/pkg/grpctest/mockservice.go @@ -20,6 +20,67 @@ type MockService struct { productv1.UnimplementedProductServiceServer } +// ResolveCategoryCategoryMetrics implements productv1.ProductServiceServer. +func (s *MockService) ResolveCategoryCategoryMetrics(_ context.Context, req *productv1.ResolveCategoryCategoryMetricsRequest) (*productv1.ResolveCategoryCategoryMetricsResponse, error) { + results := make([]*productv1.ResolveCategoryCategoryMetricsResult, 0, len(req.GetContext())) + + metricType := "" + if req.GetFieldArgs() != nil { + metricType = req.GetFieldArgs().GetMetricType() + } + + for i, ctx := range req.GetContext() { + // Return nil for certain metric types to test optional return + if metricType == "unavailable" { + results = append(results, &productv1.ResolveCategoryCategoryMetricsResult{ + CategoryMetrics: nil, + }) + } else { + results = append(results, &productv1.ResolveCategoryCategoryMetricsResult{ + CategoryMetrics: &productv1.CategoryMetrics{ + Id: fmt.Sprintf("metrics-%s-%d", ctx.GetId(), i), + MetricType: metricType, + Value: float64(i*25 + 100), // Different values based on index + Timestamp: "2024-01-01T00:00:00Z", + CategoryId: ctx.GetId(), + }, + }) + } + } + + resp := &productv1.ResolveCategoryCategoryMetricsResponse{ + Result: results, + } + + return resp, nil +} + +// ResolveCategoryPopularityScore implements productv1.ProductServiceServer. +func (s *MockService) ResolveCategoryPopularityScore(_ context.Context, req *productv1.ResolveCategoryPopularityScoreRequest) (*productv1.ResolveCategoryPopularityScoreResponse, error) { + results := make([]*productv1.ResolveCategoryPopularityScoreResult, 0, len(req.GetContext())) + + threshold := req.GetFieldArgs().GetThreshold() + + baseScore := 50 + for range req.GetContext() { + if baseScore < int(threshold.GetValue()) { + results = append(results, &productv1.ResolveCategoryPopularityScoreResult{ + PopularityScore: nil, + }) + } else { + results = append(results, &productv1.ResolveCategoryPopularityScoreResult{ + PopularityScore: &wrapperspb.Int32Value{Value: int32(baseScore)}, + }) + } + } + + resp := &productv1.ResolveCategoryPopularityScoreResponse{ + Result: results, + } + + return resp, nil +} + // ResolveSubcategoryItemCount implements productv1.ProductServiceServer. func (s *MockService) ResolveSubcategoryItemCount(_ context.Context, req *productv1.ResolveSubcategoryItemCountRequest) (*productv1.ResolveSubcategoryItemCountResponse, error) { results := make([]*productv1.ResolveSubcategoryItemCountResult, 0, len(req.GetContext())) diff --git a/v2/pkg/grpctest/product.proto b/v2/pkg/grpctest/product.proto index 542fe4286..a577307b1 100644 --- a/v2/pkg/grpctest/product.proto +++ b/v2/pkg/grpctest/product.proto @@ -55,6 +55,8 @@ service ProductService { rpc QueryTypeWithMultipleFilterFields(QueryTypeWithMultipleFilterFieldsRequest) returns (QueryTypeWithMultipleFilterFieldsResponse) {} rpc QueryUser(QueryUserRequest) returns (QueryUserResponse) {} rpc QueryUsers(QueryUsersRequest) returns (QueryUsersResponse) {} + rpc ResolveCategoryCategoryMetrics(ResolveCategoryCategoryMetricsRequest) returns (ResolveCategoryCategoryMetricsResponse) {} + rpc ResolveCategoryPopularityScore(ResolveCategoryPopularityScoreRequest) returns (ResolveCategoryPopularityScoreResponse) {} rpc ResolveCategoryProductCount(ResolveCategoryProductCountRequest) returns (ResolveCategoryProductCountResponse) {} rpc ResolveSubcategoryItemCount(ResolveSubcategoryItemCountRequest) returns (ResolveSubcategoryItemCountResponse) {} } @@ -652,6 +654,53 @@ message ResolveCategoryProductCountResponse { repeated ResolveCategoryProductCountResult result = 1; } +message ResolveCategoryPopularityScoreArgs { + google.protobuf.Int32Value threshold = 1; +} + +message ResolveCategoryPopularityScoreContext { + string id = 1; +} + +message ResolveCategoryPopularityScoreRequest { + // context provides the resolver context for the field popularityScore of type Category. + repeated ResolveCategoryPopularityScoreContext context = 1; + // field_args provides the arguments for the resolver field popularityScore of type Category. + ResolveCategoryPopularityScoreArgs field_args = 2; +} + +message ResolveCategoryPopularityScoreResult { + google.protobuf.Int32Value popularity_score = 1; +} + +message ResolveCategoryPopularityScoreResponse { + repeated ResolveCategoryPopularityScoreResult result = 1; +} + +message ResolveCategoryCategoryMetricsArgs { + string metric_type = 1; +} + +message ResolveCategoryCategoryMetricsContext { + string id = 1; + string name = 2; +} + +message ResolveCategoryCategoryMetricsRequest { + // context provides the resolver context for the field categoryMetrics of type Category. + repeated ResolveCategoryCategoryMetricsContext context = 1; + // field_args provides the arguments for the resolver field categoryMetrics of type Category. + ResolveCategoryCategoryMetricsArgs field_args = 2; +} + +message ResolveCategoryCategoryMetricsResult { + CategoryMetrics category_metrics = 1; +} + +message ResolveCategoryCategoryMetricsResponse { + repeated ResolveCategoryCategoryMetricsResult result = 1; +} + message ResolveSubcategoryItemCountArgs { SubcategoryItemFilter filters = 1; } @@ -946,6 +995,14 @@ message Subcategory { bool is_active = 4; } +message CategoryMetrics { + string id = 1; + string metric_type = 2; + double value = 3; + string timestamp = 4; + string category_id = 5; +} + enum CategoryKind { CATEGORY_KIND_UNSPECIFIED = 0; CATEGORY_KIND_BOOK = 1; diff --git a/v2/pkg/grpctest/productv1/product.pb.go b/v2/pkg/grpctest/productv1/product.pb.go index b31f73663..6b79c603c 100644 --- a/v2/pkg/grpctest/productv1/product.pb.go +++ b/v2/pkg/grpctest/productv1/product.pb.go @@ -5376,6 +5376,474 @@ func (x *ResolveCategoryProductCountResponse) GetResult() []*ResolveCategoryProd return nil } +type ResolveCategoryPopularityScoreArgs struct { + state protoimpl.MessageState `protogen:"open.v1"` + Threshold *wrapperspb.Int32Value `protobuf:"bytes,1,opt,name=threshold,proto3" json:"threshold,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ResolveCategoryPopularityScoreArgs) Reset() { + *x = ResolveCategoryPopularityScoreArgs{} + mi := &file_product_proto_msgTypes[118] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ResolveCategoryPopularityScoreArgs) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ResolveCategoryPopularityScoreArgs) ProtoMessage() {} + +func (x *ResolveCategoryPopularityScoreArgs) ProtoReflect() protoreflect.Message { + mi := &file_product_proto_msgTypes[118] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ResolveCategoryPopularityScoreArgs.ProtoReflect.Descriptor instead. +func (*ResolveCategoryPopularityScoreArgs) Descriptor() ([]byte, []int) { + return file_product_proto_rawDescGZIP(), []int{118} +} + +func (x *ResolveCategoryPopularityScoreArgs) GetThreshold() *wrapperspb.Int32Value { + if x != nil { + return x.Threshold + } + return nil +} + +type ResolveCategoryPopularityScoreContext struct { + state protoimpl.MessageState `protogen:"open.v1"` + Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ResolveCategoryPopularityScoreContext) Reset() { + *x = ResolveCategoryPopularityScoreContext{} + mi := &file_product_proto_msgTypes[119] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ResolveCategoryPopularityScoreContext) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ResolveCategoryPopularityScoreContext) ProtoMessage() {} + +func (x *ResolveCategoryPopularityScoreContext) ProtoReflect() protoreflect.Message { + mi := &file_product_proto_msgTypes[119] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ResolveCategoryPopularityScoreContext.ProtoReflect.Descriptor instead. +func (*ResolveCategoryPopularityScoreContext) Descriptor() ([]byte, []int) { + return file_product_proto_rawDescGZIP(), []int{119} +} + +func (x *ResolveCategoryPopularityScoreContext) GetId() string { + if x != nil { + return x.Id + } + return "" +} + +type ResolveCategoryPopularityScoreRequest struct { + state protoimpl.MessageState `protogen:"open.v1"` + // context provides the resolver context for the field popularityScore of type Category. + Context []*ResolveCategoryPopularityScoreContext `protobuf:"bytes,1,rep,name=context,proto3" json:"context,omitempty"` + // field_args provides the arguments for the resolver field popularityScore of type Category. + FieldArgs *ResolveCategoryPopularityScoreArgs `protobuf:"bytes,2,opt,name=field_args,json=fieldArgs,proto3" json:"field_args,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ResolveCategoryPopularityScoreRequest) Reset() { + *x = ResolveCategoryPopularityScoreRequest{} + mi := &file_product_proto_msgTypes[120] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ResolveCategoryPopularityScoreRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ResolveCategoryPopularityScoreRequest) ProtoMessage() {} + +func (x *ResolveCategoryPopularityScoreRequest) ProtoReflect() protoreflect.Message { + mi := &file_product_proto_msgTypes[120] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ResolveCategoryPopularityScoreRequest.ProtoReflect.Descriptor instead. +func (*ResolveCategoryPopularityScoreRequest) Descriptor() ([]byte, []int) { + return file_product_proto_rawDescGZIP(), []int{120} +} + +func (x *ResolveCategoryPopularityScoreRequest) GetContext() []*ResolveCategoryPopularityScoreContext { + if x != nil { + return x.Context + } + return nil +} + +func (x *ResolveCategoryPopularityScoreRequest) GetFieldArgs() *ResolveCategoryPopularityScoreArgs { + if x != nil { + return x.FieldArgs + } + return nil +} + +type ResolveCategoryPopularityScoreResult struct { + state protoimpl.MessageState `protogen:"open.v1"` + PopularityScore *wrapperspb.Int32Value `protobuf:"bytes,1,opt,name=popularity_score,json=popularityScore,proto3" json:"popularity_score,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ResolveCategoryPopularityScoreResult) Reset() { + *x = ResolveCategoryPopularityScoreResult{} + mi := &file_product_proto_msgTypes[121] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ResolveCategoryPopularityScoreResult) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ResolveCategoryPopularityScoreResult) ProtoMessage() {} + +func (x *ResolveCategoryPopularityScoreResult) ProtoReflect() protoreflect.Message { + mi := &file_product_proto_msgTypes[121] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ResolveCategoryPopularityScoreResult.ProtoReflect.Descriptor instead. +func (*ResolveCategoryPopularityScoreResult) Descriptor() ([]byte, []int) { + return file_product_proto_rawDescGZIP(), []int{121} +} + +func (x *ResolveCategoryPopularityScoreResult) GetPopularityScore() *wrapperspb.Int32Value { + if x != nil { + return x.PopularityScore + } + return nil +} + +type ResolveCategoryPopularityScoreResponse struct { + state protoimpl.MessageState `protogen:"open.v1"` + Result []*ResolveCategoryPopularityScoreResult `protobuf:"bytes,1,rep,name=result,proto3" json:"result,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ResolveCategoryPopularityScoreResponse) Reset() { + *x = ResolveCategoryPopularityScoreResponse{} + mi := &file_product_proto_msgTypes[122] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ResolveCategoryPopularityScoreResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ResolveCategoryPopularityScoreResponse) ProtoMessage() {} + +func (x *ResolveCategoryPopularityScoreResponse) ProtoReflect() protoreflect.Message { + mi := &file_product_proto_msgTypes[122] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ResolveCategoryPopularityScoreResponse.ProtoReflect.Descriptor instead. +func (*ResolveCategoryPopularityScoreResponse) Descriptor() ([]byte, []int) { + return file_product_proto_rawDescGZIP(), []int{122} +} + +func (x *ResolveCategoryPopularityScoreResponse) GetResult() []*ResolveCategoryPopularityScoreResult { + if x != nil { + return x.Result + } + return nil +} + +type ResolveCategoryCategoryMetricsArgs struct { + state protoimpl.MessageState `protogen:"open.v1"` + MetricType string `protobuf:"bytes,1,opt,name=metric_type,json=metricType,proto3" json:"metric_type,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ResolveCategoryCategoryMetricsArgs) Reset() { + *x = ResolveCategoryCategoryMetricsArgs{} + mi := &file_product_proto_msgTypes[123] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ResolveCategoryCategoryMetricsArgs) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ResolveCategoryCategoryMetricsArgs) ProtoMessage() {} + +func (x *ResolveCategoryCategoryMetricsArgs) ProtoReflect() protoreflect.Message { + mi := &file_product_proto_msgTypes[123] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ResolveCategoryCategoryMetricsArgs.ProtoReflect.Descriptor instead. +func (*ResolveCategoryCategoryMetricsArgs) Descriptor() ([]byte, []int) { + return file_product_proto_rawDescGZIP(), []int{123} +} + +func (x *ResolveCategoryCategoryMetricsArgs) GetMetricType() string { + if x != nil { + return x.MetricType + } + return "" +} + +type ResolveCategoryCategoryMetricsContext struct { + state protoimpl.MessageState `protogen:"open.v1"` + Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` + Name string `protobuf:"bytes,2,opt,name=name,proto3" json:"name,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ResolveCategoryCategoryMetricsContext) Reset() { + *x = ResolveCategoryCategoryMetricsContext{} + mi := &file_product_proto_msgTypes[124] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ResolveCategoryCategoryMetricsContext) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ResolveCategoryCategoryMetricsContext) ProtoMessage() {} + +func (x *ResolveCategoryCategoryMetricsContext) ProtoReflect() protoreflect.Message { + mi := &file_product_proto_msgTypes[124] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ResolveCategoryCategoryMetricsContext.ProtoReflect.Descriptor instead. +func (*ResolveCategoryCategoryMetricsContext) Descriptor() ([]byte, []int) { + return file_product_proto_rawDescGZIP(), []int{124} +} + +func (x *ResolveCategoryCategoryMetricsContext) GetId() string { + if x != nil { + return x.Id + } + return "" +} + +func (x *ResolveCategoryCategoryMetricsContext) GetName() string { + if x != nil { + return x.Name + } + return "" +} + +type ResolveCategoryCategoryMetricsRequest struct { + state protoimpl.MessageState `protogen:"open.v1"` + // context provides the resolver context for the field categoryMetrics of type Category. + Context []*ResolveCategoryCategoryMetricsContext `protobuf:"bytes,1,rep,name=context,proto3" json:"context,omitempty"` + // field_args provides the arguments for the resolver field categoryMetrics of type Category. + FieldArgs *ResolveCategoryCategoryMetricsArgs `protobuf:"bytes,2,opt,name=field_args,json=fieldArgs,proto3" json:"field_args,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ResolveCategoryCategoryMetricsRequest) Reset() { + *x = ResolveCategoryCategoryMetricsRequest{} + mi := &file_product_proto_msgTypes[125] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ResolveCategoryCategoryMetricsRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ResolveCategoryCategoryMetricsRequest) ProtoMessage() {} + +func (x *ResolveCategoryCategoryMetricsRequest) ProtoReflect() protoreflect.Message { + mi := &file_product_proto_msgTypes[125] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ResolveCategoryCategoryMetricsRequest.ProtoReflect.Descriptor instead. +func (*ResolveCategoryCategoryMetricsRequest) Descriptor() ([]byte, []int) { + return file_product_proto_rawDescGZIP(), []int{125} +} + +func (x *ResolveCategoryCategoryMetricsRequest) GetContext() []*ResolveCategoryCategoryMetricsContext { + if x != nil { + return x.Context + } + return nil +} + +func (x *ResolveCategoryCategoryMetricsRequest) GetFieldArgs() *ResolveCategoryCategoryMetricsArgs { + if x != nil { + return x.FieldArgs + } + return nil +} + +type ResolveCategoryCategoryMetricsResult struct { + state protoimpl.MessageState `protogen:"open.v1"` + CategoryMetrics *CategoryMetrics `protobuf:"bytes,1,opt,name=category_metrics,json=categoryMetrics,proto3" json:"category_metrics,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ResolveCategoryCategoryMetricsResult) Reset() { + *x = ResolveCategoryCategoryMetricsResult{} + mi := &file_product_proto_msgTypes[126] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ResolveCategoryCategoryMetricsResult) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ResolveCategoryCategoryMetricsResult) ProtoMessage() {} + +func (x *ResolveCategoryCategoryMetricsResult) ProtoReflect() protoreflect.Message { + mi := &file_product_proto_msgTypes[126] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ResolveCategoryCategoryMetricsResult.ProtoReflect.Descriptor instead. +func (*ResolveCategoryCategoryMetricsResult) Descriptor() ([]byte, []int) { + return file_product_proto_rawDescGZIP(), []int{126} +} + +func (x *ResolveCategoryCategoryMetricsResult) GetCategoryMetrics() *CategoryMetrics { + if x != nil { + return x.CategoryMetrics + } + return nil +} + +type ResolveCategoryCategoryMetricsResponse struct { + state protoimpl.MessageState `protogen:"open.v1"` + Result []*ResolveCategoryCategoryMetricsResult `protobuf:"bytes,1,rep,name=result,proto3" json:"result,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ResolveCategoryCategoryMetricsResponse) Reset() { + *x = ResolveCategoryCategoryMetricsResponse{} + mi := &file_product_proto_msgTypes[127] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ResolveCategoryCategoryMetricsResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ResolveCategoryCategoryMetricsResponse) ProtoMessage() {} + +func (x *ResolveCategoryCategoryMetricsResponse) ProtoReflect() protoreflect.Message { + mi := &file_product_proto_msgTypes[127] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ResolveCategoryCategoryMetricsResponse.ProtoReflect.Descriptor instead. +func (*ResolveCategoryCategoryMetricsResponse) Descriptor() ([]byte, []int) { + return file_product_proto_rawDescGZIP(), []int{127} +} + +func (x *ResolveCategoryCategoryMetricsResponse) GetResult() []*ResolveCategoryCategoryMetricsResult { + if x != nil { + return x.Result + } + return nil +} + type ResolveSubcategoryItemCountArgs struct { state protoimpl.MessageState `protogen:"open.v1"` Filters *SubcategoryItemFilter `protobuf:"bytes,1,opt,name=filters,proto3" json:"filters,omitempty"` @@ -5385,7 +5853,7 @@ type ResolveSubcategoryItemCountArgs struct { func (x *ResolveSubcategoryItemCountArgs) Reset() { *x = ResolveSubcategoryItemCountArgs{} - mi := &file_product_proto_msgTypes[118] + mi := &file_product_proto_msgTypes[128] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5397,7 +5865,7 @@ func (x *ResolveSubcategoryItemCountArgs) String() string { func (*ResolveSubcategoryItemCountArgs) ProtoMessage() {} func (x *ResolveSubcategoryItemCountArgs) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[118] + mi := &file_product_proto_msgTypes[128] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5410,7 +5878,7 @@ func (x *ResolveSubcategoryItemCountArgs) ProtoReflect() protoreflect.Message { // Deprecated: Use ResolveSubcategoryItemCountArgs.ProtoReflect.Descriptor instead. func (*ResolveSubcategoryItemCountArgs) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{118} + return file_product_proto_rawDescGZIP(), []int{128} } func (x *ResolveSubcategoryItemCountArgs) GetFilters() *SubcategoryItemFilter { @@ -5429,7 +5897,7 @@ type ResolveSubcategoryItemCountContext struct { func (x *ResolveSubcategoryItemCountContext) Reset() { *x = ResolveSubcategoryItemCountContext{} - mi := &file_product_proto_msgTypes[119] + mi := &file_product_proto_msgTypes[129] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5441,7 +5909,7 @@ func (x *ResolveSubcategoryItemCountContext) String() string { func (*ResolveSubcategoryItemCountContext) ProtoMessage() {} func (x *ResolveSubcategoryItemCountContext) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[119] + mi := &file_product_proto_msgTypes[129] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5454,7 +5922,7 @@ func (x *ResolveSubcategoryItemCountContext) ProtoReflect() protoreflect.Message // Deprecated: Use ResolveSubcategoryItemCountContext.ProtoReflect.Descriptor instead. func (*ResolveSubcategoryItemCountContext) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{119} + return file_product_proto_rawDescGZIP(), []int{129} } func (x *ResolveSubcategoryItemCountContext) GetId() string { @@ -5476,7 +5944,7 @@ type ResolveSubcategoryItemCountRequest struct { func (x *ResolveSubcategoryItemCountRequest) Reset() { *x = ResolveSubcategoryItemCountRequest{} - mi := &file_product_proto_msgTypes[120] + mi := &file_product_proto_msgTypes[130] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5488,7 +5956,7 @@ func (x *ResolveSubcategoryItemCountRequest) String() string { func (*ResolveSubcategoryItemCountRequest) ProtoMessage() {} func (x *ResolveSubcategoryItemCountRequest) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[120] + mi := &file_product_proto_msgTypes[130] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5501,7 +5969,7 @@ func (x *ResolveSubcategoryItemCountRequest) ProtoReflect() protoreflect.Message // Deprecated: Use ResolveSubcategoryItemCountRequest.ProtoReflect.Descriptor instead. func (*ResolveSubcategoryItemCountRequest) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{120} + return file_product_proto_rawDescGZIP(), []int{130} } func (x *ResolveSubcategoryItemCountRequest) GetContext() []*ResolveSubcategoryItemCountContext { @@ -5527,7 +5995,7 @@ type ResolveSubcategoryItemCountResult struct { func (x *ResolveSubcategoryItemCountResult) Reset() { *x = ResolveSubcategoryItemCountResult{} - mi := &file_product_proto_msgTypes[121] + mi := &file_product_proto_msgTypes[131] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5539,7 +6007,7 @@ func (x *ResolveSubcategoryItemCountResult) String() string { func (*ResolveSubcategoryItemCountResult) ProtoMessage() {} func (x *ResolveSubcategoryItemCountResult) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[121] + mi := &file_product_proto_msgTypes[131] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5552,7 +6020,7 @@ func (x *ResolveSubcategoryItemCountResult) ProtoReflect() protoreflect.Message // Deprecated: Use ResolveSubcategoryItemCountResult.ProtoReflect.Descriptor instead. func (*ResolveSubcategoryItemCountResult) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{121} + return file_product_proto_rawDescGZIP(), []int{131} } func (x *ResolveSubcategoryItemCountResult) GetItemCount() int32 { @@ -5571,7 +6039,7 @@ type ResolveSubcategoryItemCountResponse struct { func (x *ResolveSubcategoryItemCountResponse) Reset() { *x = ResolveSubcategoryItemCountResponse{} - mi := &file_product_proto_msgTypes[122] + mi := &file_product_proto_msgTypes[132] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5583,7 +6051,7 @@ func (x *ResolveSubcategoryItemCountResponse) String() string { func (*ResolveSubcategoryItemCountResponse) ProtoMessage() {} func (x *ResolveSubcategoryItemCountResponse) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[122] + mi := &file_product_proto_msgTypes[132] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5596,7 +6064,7 @@ func (x *ResolveSubcategoryItemCountResponse) ProtoReflect() protoreflect.Messag // Deprecated: Use ResolveSubcategoryItemCountResponse.ProtoReflect.Descriptor instead. func (*ResolveSubcategoryItemCountResponse) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{122} + return file_product_proto_rawDescGZIP(), []int{132} } func (x *ResolveSubcategoryItemCountResponse) GetResult() []*ResolveSubcategoryItemCountResult { @@ -5617,7 +6085,7 @@ type Product struct { func (x *Product) Reset() { *x = Product{} - mi := &file_product_proto_msgTypes[123] + mi := &file_product_proto_msgTypes[133] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5629,7 +6097,7 @@ func (x *Product) String() string { func (*Product) ProtoMessage() {} func (x *Product) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[123] + mi := &file_product_proto_msgTypes[133] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5642,7 +6110,7 @@ func (x *Product) ProtoReflect() protoreflect.Message { // Deprecated: Use Product.ProtoReflect.Descriptor instead. func (*Product) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{123} + return file_product_proto_rawDescGZIP(), []int{133} } func (x *Product) GetId() string { @@ -5677,7 +6145,7 @@ type Storage struct { func (x *Storage) Reset() { *x = Storage{} - mi := &file_product_proto_msgTypes[124] + mi := &file_product_proto_msgTypes[134] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5689,7 +6157,7 @@ func (x *Storage) String() string { func (*Storage) ProtoMessage() {} func (x *Storage) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[124] + mi := &file_product_proto_msgTypes[134] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5702,7 +6170,7 @@ func (x *Storage) ProtoReflect() protoreflect.Message { // Deprecated: Use Storage.ProtoReflect.Descriptor instead. func (*Storage) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{124} + return file_product_proto_rawDescGZIP(), []int{134} } func (x *Storage) GetId() string { @@ -5737,7 +6205,7 @@ type Warehouse struct { func (x *Warehouse) Reset() { *x = Warehouse{} - mi := &file_product_proto_msgTypes[125] + mi := &file_product_proto_msgTypes[135] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5749,7 +6217,7 @@ func (x *Warehouse) String() string { func (*Warehouse) ProtoMessage() {} func (x *Warehouse) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[125] + mi := &file_product_proto_msgTypes[135] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5762,7 +6230,7 @@ func (x *Warehouse) ProtoReflect() protoreflect.Message { // Deprecated: Use Warehouse.ProtoReflect.Descriptor instead. func (*Warehouse) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{125} + return file_product_proto_rawDescGZIP(), []int{135} } func (x *Warehouse) GetId() string { @@ -5796,7 +6264,7 @@ type User struct { func (x *User) Reset() { *x = User{} - mi := &file_product_proto_msgTypes[126] + mi := &file_product_proto_msgTypes[136] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5808,7 +6276,7 @@ func (x *User) String() string { func (*User) ProtoMessage() {} func (x *User) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[126] + mi := &file_product_proto_msgTypes[136] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5821,7 +6289,7 @@ func (x *User) ProtoReflect() protoreflect.Message { // Deprecated: Use User.ProtoReflect.Descriptor instead. func (*User) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{126} + return file_product_proto_rawDescGZIP(), []int{136} } func (x *User) GetId() string { @@ -5849,7 +6317,7 @@ type NestedTypeA struct { func (x *NestedTypeA) Reset() { *x = NestedTypeA{} - mi := &file_product_proto_msgTypes[127] + mi := &file_product_proto_msgTypes[137] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5861,7 +6329,7 @@ func (x *NestedTypeA) String() string { func (*NestedTypeA) ProtoMessage() {} func (x *NestedTypeA) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[127] + mi := &file_product_proto_msgTypes[137] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5874,7 +6342,7 @@ func (x *NestedTypeA) ProtoReflect() protoreflect.Message { // Deprecated: Use NestedTypeA.ProtoReflect.Descriptor instead. func (*NestedTypeA) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{127} + return file_product_proto_rawDescGZIP(), []int{137} } func (x *NestedTypeA) GetId() string { @@ -5909,7 +6377,7 @@ type RecursiveType struct { func (x *RecursiveType) Reset() { *x = RecursiveType{} - mi := &file_product_proto_msgTypes[128] + mi := &file_product_proto_msgTypes[138] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5921,7 +6389,7 @@ func (x *RecursiveType) String() string { func (*RecursiveType) ProtoMessage() {} func (x *RecursiveType) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[128] + mi := &file_product_proto_msgTypes[138] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5934,7 +6402,7 @@ func (x *RecursiveType) ProtoReflect() protoreflect.Message { // Deprecated: Use RecursiveType.ProtoReflect.Descriptor instead. func (*RecursiveType) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{128} + return file_product_proto_rawDescGZIP(), []int{138} } func (x *RecursiveType) GetId() string { @@ -5970,7 +6438,7 @@ type TypeWithMultipleFilterFields struct { func (x *TypeWithMultipleFilterFields) Reset() { *x = TypeWithMultipleFilterFields{} - mi := &file_product_proto_msgTypes[129] + mi := &file_product_proto_msgTypes[139] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5982,7 +6450,7 @@ func (x *TypeWithMultipleFilterFields) String() string { func (*TypeWithMultipleFilterFields) ProtoMessage() {} func (x *TypeWithMultipleFilterFields) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[129] + mi := &file_product_proto_msgTypes[139] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5995,7 +6463,7 @@ func (x *TypeWithMultipleFilterFields) ProtoReflect() protoreflect.Message { // Deprecated: Use TypeWithMultipleFilterFields.ProtoReflect.Descriptor instead. func (*TypeWithMultipleFilterFields) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{129} + return file_product_proto_rawDescGZIP(), []int{139} } func (x *TypeWithMultipleFilterFields) GetId() string { @@ -6036,7 +6504,7 @@ type FilterTypeInput struct { func (x *FilterTypeInput) Reset() { *x = FilterTypeInput{} - mi := &file_product_proto_msgTypes[130] + mi := &file_product_proto_msgTypes[140] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6048,7 +6516,7 @@ func (x *FilterTypeInput) String() string { func (*FilterTypeInput) ProtoMessage() {} func (x *FilterTypeInput) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[130] + mi := &file_product_proto_msgTypes[140] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6061,7 +6529,7 @@ func (x *FilterTypeInput) ProtoReflect() protoreflect.Message { // Deprecated: Use FilterTypeInput.ProtoReflect.Descriptor instead. func (*FilterTypeInput) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{130} + return file_product_proto_rawDescGZIP(), []int{140} } func (x *FilterTypeInput) GetFilterField_1() string { @@ -6087,7 +6555,7 @@ type ComplexFilterTypeInput struct { func (x *ComplexFilterTypeInput) Reset() { *x = ComplexFilterTypeInput{} - mi := &file_product_proto_msgTypes[131] + mi := &file_product_proto_msgTypes[141] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6099,7 +6567,7 @@ func (x *ComplexFilterTypeInput) String() string { func (*ComplexFilterTypeInput) ProtoMessage() {} func (x *ComplexFilterTypeInput) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[131] + mi := &file_product_proto_msgTypes[141] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6112,7 +6580,7 @@ func (x *ComplexFilterTypeInput) ProtoReflect() protoreflect.Message { // Deprecated: Use ComplexFilterTypeInput.ProtoReflect.Descriptor instead. func (*ComplexFilterTypeInput) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{131} + return file_product_proto_rawDescGZIP(), []int{141} } func (x *ComplexFilterTypeInput) GetFilter() *FilterType { @@ -6132,7 +6600,7 @@ type TypeWithComplexFilterInput struct { func (x *TypeWithComplexFilterInput) Reset() { *x = TypeWithComplexFilterInput{} - mi := &file_product_proto_msgTypes[132] + mi := &file_product_proto_msgTypes[142] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6144,7 +6612,7 @@ func (x *TypeWithComplexFilterInput) String() string { func (*TypeWithComplexFilterInput) ProtoMessage() {} func (x *TypeWithComplexFilterInput) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[132] + mi := &file_product_proto_msgTypes[142] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6157,7 +6625,7 @@ func (x *TypeWithComplexFilterInput) ProtoReflect() protoreflect.Message { // Deprecated: Use TypeWithComplexFilterInput.ProtoReflect.Descriptor instead. func (*TypeWithComplexFilterInput) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{132} + return file_product_proto_rawDescGZIP(), []int{142} } func (x *TypeWithComplexFilterInput) GetId() string { @@ -6185,7 +6653,7 @@ type OrderInput struct { func (x *OrderInput) Reset() { *x = OrderInput{} - mi := &file_product_proto_msgTypes[133] + mi := &file_product_proto_msgTypes[143] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6197,7 +6665,7 @@ func (x *OrderInput) String() string { func (*OrderInput) ProtoMessage() {} func (x *OrderInput) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[133] + mi := &file_product_proto_msgTypes[143] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6210,7 +6678,7 @@ func (x *OrderInput) ProtoReflect() protoreflect.Message { // Deprecated: Use OrderInput.ProtoReflect.Descriptor instead. func (*OrderInput) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{133} + return file_product_proto_rawDescGZIP(), []int{143} } func (x *OrderInput) GetOrderId() string { @@ -6246,7 +6714,7 @@ type Order struct { func (x *Order) Reset() { *x = Order{} - mi := &file_product_proto_msgTypes[134] + mi := &file_product_proto_msgTypes[144] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6258,7 +6726,7 @@ func (x *Order) String() string { func (*Order) ProtoMessage() {} func (x *Order) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[134] + mi := &file_product_proto_msgTypes[144] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6271,7 +6739,7 @@ func (x *Order) ProtoReflect() protoreflect.Message { // Deprecated: Use Order.ProtoReflect.Descriptor instead. func (*Order) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{134} + return file_product_proto_rawDescGZIP(), []int{144} } func (x *Order) GetOrderId() string { @@ -6314,7 +6782,7 @@ type Category struct { func (x *Category) Reset() { *x = Category{} - mi := &file_product_proto_msgTypes[135] + mi := &file_product_proto_msgTypes[145] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6326,7 +6794,7 @@ func (x *Category) String() string { func (*Category) ProtoMessage() {} func (x *Category) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[135] + mi := &file_product_proto_msgTypes[145] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6339,7 +6807,7 @@ func (x *Category) ProtoReflect() protoreflect.Message { // Deprecated: Use Category.ProtoReflect.Descriptor instead. func (*Category) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{135} + return file_product_proto_rawDescGZIP(), []int{145} } func (x *Category) GetId() string { @@ -6380,7 +6848,7 @@ type CategoryFilter struct { func (x *CategoryFilter) Reset() { *x = CategoryFilter{} - mi := &file_product_proto_msgTypes[136] + mi := &file_product_proto_msgTypes[146] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6392,7 +6860,7 @@ func (x *CategoryFilter) String() string { func (*CategoryFilter) ProtoMessage() {} func (x *CategoryFilter) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[136] + mi := &file_product_proto_msgTypes[146] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6405,7 +6873,7 @@ func (x *CategoryFilter) ProtoReflect() protoreflect.Message { // Deprecated: Use CategoryFilter.ProtoReflect.Descriptor instead. func (*CategoryFilter) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{136} + return file_product_proto_rawDescGZIP(), []int{146} } func (x *CategoryFilter) GetCategory() CategoryKind { @@ -6435,7 +6903,7 @@ type Animal struct { func (x *Animal) Reset() { *x = Animal{} - mi := &file_product_proto_msgTypes[137] + mi := &file_product_proto_msgTypes[147] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6447,7 +6915,7 @@ func (x *Animal) String() string { func (*Animal) ProtoMessage() {} func (x *Animal) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[137] + mi := &file_product_proto_msgTypes[147] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6460,7 +6928,7 @@ func (x *Animal) ProtoReflect() protoreflect.Message { // Deprecated: Use Animal.ProtoReflect.Descriptor instead. func (*Animal) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{137} + return file_product_proto_rawDescGZIP(), []int{147} } func (x *Animal) GetInstance() isAnimal_Instance { @@ -6514,7 +6982,7 @@ type SearchInput struct { func (x *SearchInput) Reset() { *x = SearchInput{} - mi := &file_product_proto_msgTypes[138] + mi := &file_product_proto_msgTypes[148] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6526,7 +6994,7 @@ func (x *SearchInput) String() string { func (*SearchInput) ProtoMessage() {} func (x *SearchInput) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[138] + mi := &file_product_proto_msgTypes[148] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6539,7 +7007,7 @@ func (x *SearchInput) ProtoReflect() protoreflect.Message { // Deprecated: Use SearchInput.ProtoReflect.Descriptor instead. func (*SearchInput) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{138} + return file_product_proto_rawDescGZIP(), []int{148} } func (x *SearchInput) GetQuery() string { @@ -6570,7 +7038,7 @@ type SearchResult struct { func (x *SearchResult) Reset() { *x = SearchResult{} - mi := &file_product_proto_msgTypes[139] + mi := &file_product_proto_msgTypes[149] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6582,7 +7050,7 @@ func (x *SearchResult) String() string { func (*SearchResult) ProtoMessage() {} func (x *SearchResult) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[139] + mi := &file_product_proto_msgTypes[149] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6595,7 +7063,7 @@ func (x *SearchResult) ProtoReflect() protoreflect.Message { // Deprecated: Use SearchResult.ProtoReflect.Descriptor instead. func (*SearchResult) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{139} + return file_product_proto_rawDescGZIP(), []int{149} } func (x *SearchResult) GetValue() isSearchResult_Value { @@ -6670,7 +7138,7 @@ type NullableFieldsType struct { func (x *NullableFieldsType) Reset() { *x = NullableFieldsType{} - mi := &file_product_proto_msgTypes[140] + mi := &file_product_proto_msgTypes[150] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6682,7 +7150,7 @@ func (x *NullableFieldsType) String() string { func (*NullableFieldsType) ProtoMessage() {} func (x *NullableFieldsType) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[140] + mi := &file_product_proto_msgTypes[150] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6695,7 +7163,7 @@ func (x *NullableFieldsType) ProtoReflect() protoreflect.Message { // Deprecated: Use NullableFieldsType.ProtoReflect.Descriptor instead. func (*NullableFieldsType) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{140} + return file_product_proto_rawDescGZIP(), []int{150} } func (x *NullableFieldsType) GetId() string { @@ -6765,7 +7233,7 @@ type NullableFieldsFilter struct { func (x *NullableFieldsFilter) Reset() { *x = NullableFieldsFilter{} - mi := &file_product_proto_msgTypes[141] + mi := &file_product_proto_msgTypes[151] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6777,7 +7245,7 @@ func (x *NullableFieldsFilter) String() string { func (*NullableFieldsFilter) ProtoMessage() {} func (x *NullableFieldsFilter) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[141] + mi := &file_product_proto_msgTypes[151] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6790,7 +7258,7 @@ func (x *NullableFieldsFilter) ProtoReflect() protoreflect.Message { // Deprecated: Use NullableFieldsFilter.ProtoReflect.Descriptor instead. func (*NullableFieldsFilter) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{141} + return file_product_proto_rawDescGZIP(), []int{151} } func (x *NullableFieldsFilter) GetName() *wrapperspb.StringValue { @@ -6842,7 +7310,7 @@ type BlogPost struct { func (x *BlogPost) Reset() { *x = BlogPost{} - mi := &file_product_proto_msgTypes[142] + mi := &file_product_proto_msgTypes[152] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6854,7 +7322,7 @@ func (x *BlogPost) String() string { func (*BlogPost) ProtoMessage() {} func (x *BlogPost) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[142] + mi := &file_product_proto_msgTypes[152] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6867,7 +7335,7 @@ func (x *BlogPost) ProtoReflect() protoreflect.Message { // Deprecated: Use BlogPost.ProtoReflect.Descriptor instead. func (*BlogPost) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{142} + return file_product_proto_rawDescGZIP(), []int{152} } func (x *BlogPost) GetId() string { @@ -7021,7 +7489,7 @@ type BlogPostFilter struct { func (x *BlogPostFilter) Reset() { *x = BlogPostFilter{} - mi := &file_product_proto_msgTypes[143] + mi := &file_product_proto_msgTypes[153] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7033,7 +7501,7 @@ func (x *BlogPostFilter) String() string { func (*BlogPostFilter) ProtoMessage() {} func (x *BlogPostFilter) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[143] + mi := &file_product_proto_msgTypes[153] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7046,7 +7514,7 @@ func (x *BlogPostFilter) ProtoReflect() protoreflect.Message { // Deprecated: Use BlogPostFilter.ProtoReflect.Descriptor instead. func (*BlogPostFilter) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{143} + return file_product_proto_rawDescGZIP(), []int{153} } func (x *BlogPostFilter) GetTitle() *wrapperspb.StringValue { @@ -7093,7 +7561,7 @@ type Author struct { func (x *Author) Reset() { *x = Author{} - mi := &file_product_proto_msgTypes[144] + mi := &file_product_proto_msgTypes[154] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7105,7 +7573,7 @@ func (x *Author) String() string { func (*Author) ProtoMessage() {} func (x *Author) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[144] + mi := &file_product_proto_msgTypes[154] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7118,7 +7586,7 @@ func (x *Author) ProtoReflect() protoreflect.Message { // Deprecated: Use Author.ProtoReflect.Descriptor instead. func (*Author) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{144} + return file_product_proto_rawDescGZIP(), []int{154} } func (x *Author) GetId() string { @@ -7237,7 +7705,7 @@ type AuthorFilter struct { func (x *AuthorFilter) Reset() { *x = AuthorFilter{} - mi := &file_product_proto_msgTypes[145] + mi := &file_product_proto_msgTypes[155] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7249,7 +7717,7 @@ func (x *AuthorFilter) String() string { func (*AuthorFilter) ProtoMessage() {} func (x *AuthorFilter) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[145] + mi := &file_product_proto_msgTypes[155] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7262,7 +7730,7 @@ func (x *AuthorFilter) ProtoReflect() protoreflect.Message { // Deprecated: Use AuthorFilter.ProtoReflect.Descriptor instead. func (*AuthorFilter) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{145} + return file_product_proto_rawDescGZIP(), []int{155} } func (x *AuthorFilter) GetName() *wrapperspb.StringValue { @@ -7295,7 +7763,7 @@ type UserInput struct { func (x *UserInput) Reset() { *x = UserInput{} - mi := &file_product_proto_msgTypes[146] + mi := &file_product_proto_msgTypes[156] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7307,7 +7775,7 @@ func (x *UserInput) String() string { func (*UserInput) ProtoMessage() {} func (x *UserInput) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[146] + mi := &file_product_proto_msgTypes[156] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7320,7 +7788,7 @@ func (x *UserInput) ProtoReflect() protoreflect.Message { // Deprecated: Use UserInput.ProtoReflect.Descriptor instead. func (*UserInput) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{146} + return file_product_proto_rawDescGZIP(), []int{156} } func (x *UserInput) GetName() string { @@ -7340,7 +7808,7 @@ type ActionInput struct { func (x *ActionInput) Reset() { *x = ActionInput{} - mi := &file_product_proto_msgTypes[147] + mi := &file_product_proto_msgTypes[157] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7352,7 +7820,7 @@ func (x *ActionInput) String() string { func (*ActionInput) ProtoMessage() {} func (x *ActionInput) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[147] + mi := &file_product_proto_msgTypes[157] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7365,7 +7833,7 @@ func (x *ActionInput) ProtoReflect() protoreflect.Message { // Deprecated: Use ActionInput.ProtoReflect.Descriptor instead. func (*ActionInput) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{147} + return file_product_proto_rawDescGZIP(), []int{157} } func (x *ActionInput) GetType() string { @@ -7395,7 +7863,7 @@ type ActionResult struct { func (x *ActionResult) Reset() { *x = ActionResult{} - mi := &file_product_proto_msgTypes[148] + mi := &file_product_proto_msgTypes[158] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7407,7 +7875,7 @@ func (x *ActionResult) String() string { func (*ActionResult) ProtoMessage() {} func (x *ActionResult) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[148] + mi := &file_product_proto_msgTypes[158] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7420,7 +7888,7 @@ func (x *ActionResult) ProtoReflect() protoreflect.Message { // Deprecated: Use ActionResult.ProtoReflect.Descriptor instead. func (*ActionResult) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{148} + return file_product_proto_rawDescGZIP(), []int{158} } func (x *ActionResult) GetValue() isActionResult_Value { @@ -7479,7 +7947,7 @@ type NullableFieldsInput struct { func (x *NullableFieldsInput) Reset() { *x = NullableFieldsInput{} - mi := &file_product_proto_msgTypes[149] + mi := &file_product_proto_msgTypes[159] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7491,7 +7959,7 @@ func (x *NullableFieldsInput) String() string { func (*NullableFieldsInput) ProtoMessage() {} func (x *NullableFieldsInput) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[149] + mi := &file_product_proto_msgTypes[159] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7504,7 +7972,7 @@ func (x *NullableFieldsInput) ProtoReflect() protoreflect.Message { // Deprecated: Use NullableFieldsInput.ProtoReflect.Descriptor instead. func (*NullableFieldsInput) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{149} + return file_product_proto_rawDescGZIP(), []int{159} } func (x *NullableFieldsInput) GetName() string { @@ -7580,7 +8048,7 @@ type BlogPostInput struct { func (x *BlogPostInput) Reset() { *x = BlogPostInput{} - mi := &file_product_proto_msgTypes[150] + mi := &file_product_proto_msgTypes[160] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7592,7 +8060,7 @@ func (x *BlogPostInput) String() string { func (*BlogPostInput) ProtoMessage() {} func (x *BlogPostInput) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[150] + mi := &file_product_proto_msgTypes[160] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7605,7 +8073,7 @@ func (x *BlogPostInput) ProtoReflect() protoreflect.Message { // Deprecated: Use BlogPostInput.ProtoReflect.Descriptor instead. func (*BlogPostInput) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{150} + return file_product_proto_rawDescGZIP(), []int{160} } func (x *BlogPostInput) GetTitle() string { @@ -7738,7 +8206,7 @@ type AuthorInput struct { func (x *AuthorInput) Reset() { *x = AuthorInput{} - mi := &file_product_proto_msgTypes[151] + mi := &file_product_proto_msgTypes[161] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7750,7 +8218,7 @@ func (x *AuthorInput) String() string { func (*AuthorInput) ProtoMessage() {} func (x *AuthorInput) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[151] + mi := &file_product_proto_msgTypes[161] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7763,7 +8231,7 @@ func (x *AuthorInput) ProtoReflect() protoreflect.Message { // Deprecated: Use AuthorInput.ProtoReflect.Descriptor instead. func (*AuthorInput) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{151} + return file_product_proto_rawDescGZIP(), []int{161} } func (x *AuthorInput) GetName() string { @@ -7847,7 +8315,7 @@ type NestedTypeB struct { func (x *NestedTypeB) Reset() { *x = NestedTypeB{} - mi := &file_product_proto_msgTypes[152] + mi := &file_product_proto_msgTypes[162] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7859,7 +8327,7 @@ func (x *NestedTypeB) String() string { func (*NestedTypeB) ProtoMessage() {} func (x *NestedTypeB) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[152] + mi := &file_product_proto_msgTypes[162] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7872,7 +8340,7 @@ func (x *NestedTypeB) ProtoReflect() protoreflect.Message { // Deprecated: Use NestedTypeB.ProtoReflect.Descriptor instead. func (*NestedTypeB) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{152} + return file_product_proto_rawDescGZIP(), []int{162} } func (x *NestedTypeB) GetId() string { @@ -7906,7 +8374,7 @@ type NestedTypeC struct { func (x *NestedTypeC) Reset() { *x = NestedTypeC{} - mi := &file_product_proto_msgTypes[153] + mi := &file_product_proto_msgTypes[163] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7918,7 +8386,7 @@ func (x *NestedTypeC) String() string { func (*NestedTypeC) ProtoMessage() {} func (x *NestedTypeC) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[153] + mi := &file_product_proto_msgTypes[163] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7931,7 +8399,7 @@ func (x *NestedTypeC) ProtoReflect() protoreflect.Message { // Deprecated: Use NestedTypeC.ProtoReflect.Descriptor instead. func (*NestedTypeC) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{153} + return file_product_proto_rawDescGZIP(), []int{163} } func (x *NestedTypeC) GetId() string { @@ -7960,7 +8428,7 @@ type FilterType struct { func (x *FilterType) Reset() { *x = FilterType{} - mi := &file_product_proto_msgTypes[154] + mi := &file_product_proto_msgTypes[164] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7972,7 +8440,7 @@ func (x *FilterType) String() string { func (*FilterType) ProtoMessage() {} func (x *FilterType) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[154] + mi := &file_product_proto_msgTypes[164] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7985,7 +8453,7 @@ func (x *FilterType) ProtoReflect() protoreflect.Message { // Deprecated: Use FilterType.ProtoReflect.Descriptor instead. func (*FilterType) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{154} + return file_product_proto_rawDescGZIP(), []int{164} } func (x *FilterType) GetName() string { @@ -8026,7 +8494,7 @@ type Pagination struct { func (x *Pagination) Reset() { *x = Pagination{} - mi := &file_product_proto_msgTypes[155] + mi := &file_product_proto_msgTypes[165] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8038,7 +8506,7 @@ func (x *Pagination) String() string { func (*Pagination) ProtoMessage() {} func (x *Pagination) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[155] + mi := &file_product_proto_msgTypes[165] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8051,7 +8519,7 @@ func (x *Pagination) ProtoReflect() protoreflect.Message { // Deprecated: Use Pagination.ProtoReflect.Descriptor instead. func (*Pagination) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{155} + return file_product_proto_rawDescGZIP(), []int{165} } func (x *Pagination) GetPage() int32 { @@ -8079,7 +8547,7 @@ type OrderLineInput struct { func (x *OrderLineInput) Reset() { *x = OrderLineInput{} - mi := &file_product_proto_msgTypes[156] + mi := &file_product_proto_msgTypes[166] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8091,7 +8559,7 @@ func (x *OrderLineInput) String() string { func (*OrderLineInput) ProtoMessage() {} func (x *OrderLineInput) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[156] + mi := &file_product_proto_msgTypes[166] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8104,7 +8572,7 @@ func (x *OrderLineInput) ProtoReflect() protoreflect.Message { // Deprecated: Use OrderLineInput.ProtoReflect.Descriptor instead. func (*OrderLineInput) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{156} + return file_product_proto_rawDescGZIP(), []int{166} } func (x *OrderLineInput) GetProductId() string { @@ -8139,7 +8607,7 @@ type OrderLine struct { func (x *OrderLine) Reset() { *x = OrderLine{} - mi := &file_product_proto_msgTypes[157] + mi := &file_product_proto_msgTypes[167] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8151,7 +8619,7 @@ func (x *OrderLine) String() string { func (*OrderLine) ProtoMessage() {} func (x *OrderLine) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[157] + mi := &file_product_proto_msgTypes[167] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8164,7 +8632,7 @@ func (x *OrderLine) ProtoReflect() protoreflect.Message { // Deprecated: Use OrderLine.ProtoReflect.Descriptor instead. func (*OrderLine) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{157} + return file_product_proto_rawDescGZIP(), []int{167} } func (x *OrderLine) GetProductId() string { @@ -8200,7 +8668,7 @@ type Subcategory struct { func (x *Subcategory) Reset() { *x = Subcategory{} - mi := &file_product_proto_msgTypes[158] + mi := &file_product_proto_msgTypes[168] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8212,7 +8680,7 @@ func (x *Subcategory) String() string { func (*Subcategory) ProtoMessage() {} func (x *Subcategory) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[158] + mi := &file_product_proto_msgTypes[168] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8225,7 +8693,7 @@ func (x *Subcategory) ProtoReflect() protoreflect.Message { // Deprecated: Use Subcategory.ProtoReflect.Descriptor instead. func (*Subcategory) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{158} + return file_product_proto_rawDescGZIP(), []int{168} } func (x *Subcategory) GetId() string { @@ -8256,6 +8724,82 @@ func (x *Subcategory) GetIsActive() bool { return false } +type CategoryMetrics struct { + state protoimpl.MessageState `protogen:"open.v1"` + Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` + MetricType string `protobuf:"bytes,2,opt,name=metric_type,json=metricType,proto3" json:"metric_type,omitempty"` + Value float64 `protobuf:"fixed64,3,opt,name=value,proto3" json:"value,omitempty"` + Timestamp string `protobuf:"bytes,4,opt,name=timestamp,proto3" json:"timestamp,omitempty"` + CategoryId string `protobuf:"bytes,5,opt,name=category_id,json=categoryId,proto3" json:"category_id,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *CategoryMetrics) Reset() { + *x = CategoryMetrics{} + mi := &file_product_proto_msgTypes[169] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *CategoryMetrics) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*CategoryMetrics) ProtoMessage() {} + +func (x *CategoryMetrics) ProtoReflect() protoreflect.Message { + mi := &file_product_proto_msgTypes[169] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use CategoryMetrics.ProtoReflect.Descriptor instead. +func (*CategoryMetrics) Descriptor() ([]byte, []int) { + return file_product_proto_rawDescGZIP(), []int{169} +} + +func (x *CategoryMetrics) GetId() string { + if x != nil { + return x.Id + } + return "" +} + +func (x *CategoryMetrics) GetMetricType() string { + if x != nil { + return x.MetricType + } + return "" +} + +func (x *CategoryMetrics) GetValue() float64 { + if x != nil { + return x.Value + } + return 0 +} + +func (x *CategoryMetrics) GetTimestamp() string { + if x != nil { + return x.Timestamp + } + return "" +} + +func (x *CategoryMetrics) GetCategoryId() string { + if x != nil { + return x.CategoryId + } + return "" +} + type Cat struct { state protoimpl.MessageState `protogen:"open.v1"` Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` @@ -8268,7 +8812,7 @@ type Cat struct { func (x *Cat) Reset() { *x = Cat{} - mi := &file_product_proto_msgTypes[159] + mi := &file_product_proto_msgTypes[170] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8280,7 +8824,7 @@ func (x *Cat) String() string { func (*Cat) ProtoMessage() {} func (x *Cat) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[159] + mi := &file_product_proto_msgTypes[170] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8293,7 +8837,7 @@ func (x *Cat) ProtoReflect() protoreflect.Message { // Deprecated: Use Cat.ProtoReflect.Descriptor instead. func (*Cat) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{159} + return file_product_proto_rawDescGZIP(), []int{170} } func (x *Cat) GetId() string { @@ -8336,7 +8880,7 @@ type Dog struct { func (x *Dog) Reset() { *x = Dog{} - mi := &file_product_proto_msgTypes[160] + mi := &file_product_proto_msgTypes[171] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8348,7 +8892,7 @@ func (x *Dog) String() string { func (*Dog) ProtoMessage() {} func (x *Dog) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[160] + mi := &file_product_proto_msgTypes[171] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8361,7 +8905,7 @@ func (x *Dog) ProtoReflect() protoreflect.Message { // Deprecated: Use Dog.ProtoReflect.Descriptor instead. func (*Dog) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{160} + return file_product_proto_rawDescGZIP(), []int{171} } func (x *Dog) GetId() string { @@ -8402,7 +8946,7 @@ type ActionSuccess struct { func (x *ActionSuccess) Reset() { *x = ActionSuccess{} - mi := &file_product_proto_msgTypes[161] + mi := &file_product_proto_msgTypes[172] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8414,7 +8958,7 @@ func (x *ActionSuccess) String() string { func (*ActionSuccess) ProtoMessage() {} func (x *ActionSuccess) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[161] + mi := &file_product_proto_msgTypes[172] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8427,7 +8971,7 @@ func (x *ActionSuccess) ProtoReflect() protoreflect.Message { // Deprecated: Use ActionSuccess.ProtoReflect.Descriptor instead. func (*ActionSuccess) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{161} + return file_product_proto_rawDescGZIP(), []int{172} } func (x *ActionSuccess) GetMessage() string { @@ -8454,7 +8998,7 @@ type ActionError struct { func (x *ActionError) Reset() { *x = ActionError{} - mi := &file_product_proto_msgTypes[162] + mi := &file_product_proto_msgTypes[173] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8466,7 +9010,7 @@ func (x *ActionError) String() string { func (*ActionError) ProtoMessage() {} func (x *ActionError) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[162] + mi := &file_product_proto_msgTypes[173] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8479,7 +9023,7 @@ func (x *ActionError) ProtoReflect() protoreflect.Message { // Deprecated: Use ActionError.ProtoReflect.Descriptor instead. func (*ActionError) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{162} + return file_product_proto_rawDescGZIP(), []int{173} } func (x *ActionError) GetMessage() string { @@ -8506,7 +9050,7 @@ type CategoryInput struct { func (x *CategoryInput) Reset() { *x = CategoryInput{} - mi := &file_product_proto_msgTypes[163] + mi := &file_product_proto_msgTypes[174] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8518,7 +9062,7 @@ func (x *CategoryInput) String() string { func (*CategoryInput) ProtoMessage() {} func (x *CategoryInput) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[163] + mi := &file_product_proto_msgTypes[174] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8531,7 +9075,7 @@ func (x *CategoryInput) ProtoReflect() protoreflect.Message { // Deprecated: Use CategoryInput.ProtoReflect.Descriptor instead. func (*CategoryInput) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{163} + return file_product_proto_rawDescGZIP(), []int{174} } func (x *CategoryInput) GetName() string { @@ -8560,7 +9104,7 @@ type ProductCountFilter struct { func (x *ProductCountFilter) Reset() { *x = ProductCountFilter{} - mi := &file_product_proto_msgTypes[164] + mi := &file_product_proto_msgTypes[175] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8572,7 +9116,7 @@ func (x *ProductCountFilter) String() string { func (*ProductCountFilter) ProtoMessage() {} func (x *ProductCountFilter) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[164] + mi := &file_product_proto_msgTypes[175] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8585,7 +9129,7 @@ func (x *ProductCountFilter) ProtoReflect() protoreflect.Message { // Deprecated: Use ProductCountFilter.ProtoReflect.Descriptor instead. func (*ProductCountFilter) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{164} + return file_product_proto_rawDescGZIP(), []int{175} } func (x *ProductCountFilter) GetMinPrice() *wrapperspb.DoubleValue { @@ -8629,7 +9173,7 @@ type SubcategoryItemFilter struct { func (x *SubcategoryItemFilter) Reset() { *x = SubcategoryItemFilter{} - mi := &file_product_proto_msgTypes[165] + mi := &file_product_proto_msgTypes[176] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8641,7 +9185,7 @@ func (x *SubcategoryItemFilter) String() string { func (*SubcategoryItemFilter) ProtoMessage() {} func (x *SubcategoryItemFilter) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[165] + mi := &file_product_proto_msgTypes[176] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8654,7 +9198,7 @@ func (x *SubcategoryItemFilter) ProtoReflect() protoreflect.Message { // Deprecated: Use SubcategoryItemFilter.ProtoReflect.Descriptor instead. func (*SubcategoryItemFilter) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{165} + return file_product_proto_rawDescGZIP(), []int{176} } func (x *SubcategoryItemFilter) GetMinPrice() *wrapperspb.DoubleValue { @@ -8701,7 +9245,7 @@ type ListOfAuthorFilter_List struct { func (x *ListOfAuthorFilter_List) Reset() { *x = ListOfAuthorFilter_List{} - mi := &file_product_proto_msgTypes[166] + mi := &file_product_proto_msgTypes[177] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8713,7 +9257,7 @@ func (x *ListOfAuthorFilter_List) String() string { func (*ListOfAuthorFilter_List) ProtoMessage() {} func (x *ListOfAuthorFilter_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[166] + mi := &file_product_proto_msgTypes[177] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8745,7 +9289,7 @@ type ListOfAuthorInput_List struct { func (x *ListOfAuthorInput_List) Reset() { *x = ListOfAuthorInput_List{} - mi := &file_product_proto_msgTypes[167] + mi := &file_product_proto_msgTypes[178] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8757,7 +9301,7 @@ func (x *ListOfAuthorInput_List) String() string { func (*ListOfAuthorInput_List) ProtoMessage() {} func (x *ListOfAuthorInput_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[167] + mi := &file_product_proto_msgTypes[178] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8789,7 +9333,7 @@ type ListOfBlogPost_List struct { func (x *ListOfBlogPost_List) Reset() { *x = ListOfBlogPost_List{} - mi := &file_product_proto_msgTypes[168] + mi := &file_product_proto_msgTypes[179] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8801,7 +9345,7 @@ func (x *ListOfBlogPost_List) String() string { func (*ListOfBlogPost_List) ProtoMessage() {} func (x *ListOfBlogPost_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[168] + mi := &file_product_proto_msgTypes[179] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8833,7 +9377,7 @@ type ListOfBlogPostFilter_List struct { func (x *ListOfBlogPostFilter_List) Reset() { *x = ListOfBlogPostFilter_List{} - mi := &file_product_proto_msgTypes[169] + mi := &file_product_proto_msgTypes[180] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8845,7 +9389,7 @@ func (x *ListOfBlogPostFilter_List) String() string { func (*ListOfBlogPostFilter_List) ProtoMessage() {} func (x *ListOfBlogPostFilter_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[169] + mi := &file_product_proto_msgTypes[180] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8877,7 +9421,7 @@ type ListOfBlogPostInput_List struct { func (x *ListOfBlogPostInput_List) Reset() { *x = ListOfBlogPostInput_List{} - mi := &file_product_proto_msgTypes[170] + mi := &file_product_proto_msgTypes[181] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8889,7 +9433,7 @@ func (x *ListOfBlogPostInput_List) String() string { func (*ListOfBlogPostInput_List) ProtoMessage() {} func (x *ListOfBlogPostInput_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[170] + mi := &file_product_proto_msgTypes[181] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8921,7 +9465,7 @@ type ListOfBoolean_List struct { func (x *ListOfBoolean_List) Reset() { *x = ListOfBoolean_List{} - mi := &file_product_proto_msgTypes[171] + mi := &file_product_proto_msgTypes[182] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8933,7 +9477,7 @@ func (x *ListOfBoolean_List) String() string { func (*ListOfBoolean_List) ProtoMessage() {} func (x *ListOfBoolean_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[171] + mi := &file_product_proto_msgTypes[182] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8965,7 +9509,7 @@ type ListOfCategory_List struct { func (x *ListOfCategory_List) Reset() { *x = ListOfCategory_List{} - mi := &file_product_proto_msgTypes[172] + mi := &file_product_proto_msgTypes[183] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8977,7 +9521,7 @@ func (x *ListOfCategory_List) String() string { func (*ListOfCategory_List) ProtoMessage() {} func (x *ListOfCategory_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[172] + mi := &file_product_proto_msgTypes[183] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9009,7 +9553,7 @@ type ListOfCategoryInput_List struct { func (x *ListOfCategoryInput_List) Reset() { *x = ListOfCategoryInput_List{} - mi := &file_product_proto_msgTypes[173] + mi := &file_product_proto_msgTypes[184] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9021,7 +9565,7 @@ func (x *ListOfCategoryInput_List) String() string { func (*ListOfCategoryInput_List) ProtoMessage() {} func (x *ListOfCategoryInput_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[173] + mi := &file_product_proto_msgTypes[184] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9053,7 +9597,7 @@ type ListOfFloat_List struct { func (x *ListOfFloat_List) Reset() { *x = ListOfFloat_List{} - mi := &file_product_proto_msgTypes[174] + mi := &file_product_proto_msgTypes[185] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9065,7 +9609,7 @@ func (x *ListOfFloat_List) String() string { func (*ListOfFloat_List) ProtoMessage() {} func (x *ListOfFloat_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[174] + mi := &file_product_proto_msgTypes[185] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9097,7 +9641,7 @@ type ListOfListOfCategory_List struct { func (x *ListOfListOfCategory_List) Reset() { *x = ListOfListOfCategory_List{} - mi := &file_product_proto_msgTypes[175] + mi := &file_product_proto_msgTypes[186] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9109,7 +9653,7 @@ func (x *ListOfListOfCategory_List) String() string { func (*ListOfListOfCategory_List) ProtoMessage() {} func (x *ListOfListOfCategory_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[175] + mi := &file_product_proto_msgTypes[186] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9141,7 +9685,7 @@ type ListOfListOfCategoryInput_List struct { func (x *ListOfListOfCategoryInput_List) Reset() { *x = ListOfListOfCategoryInput_List{} - mi := &file_product_proto_msgTypes[176] + mi := &file_product_proto_msgTypes[187] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9153,7 +9697,7 @@ func (x *ListOfListOfCategoryInput_List) String() string { func (*ListOfListOfCategoryInput_List) ProtoMessage() {} func (x *ListOfListOfCategoryInput_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[176] + mi := &file_product_proto_msgTypes[187] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9185,7 +9729,7 @@ type ListOfListOfString_List struct { func (x *ListOfListOfString_List) Reset() { *x = ListOfListOfString_List{} - mi := &file_product_proto_msgTypes[177] + mi := &file_product_proto_msgTypes[188] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9197,7 +9741,7 @@ func (x *ListOfListOfString_List) String() string { func (*ListOfListOfString_List) ProtoMessage() {} func (x *ListOfListOfString_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[177] + mi := &file_product_proto_msgTypes[188] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9229,7 +9773,7 @@ type ListOfListOfUser_List struct { func (x *ListOfListOfUser_List) Reset() { *x = ListOfListOfUser_List{} - mi := &file_product_proto_msgTypes[178] + mi := &file_product_proto_msgTypes[189] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9241,7 +9785,7 @@ func (x *ListOfListOfUser_List) String() string { func (*ListOfListOfUser_List) ProtoMessage() {} func (x *ListOfListOfUser_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[178] + mi := &file_product_proto_msgTypes[189] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9273,7 +9817,7 @@ type ListOfListOfUserInput_List struct { func (x *ListOfListOfUserInput_List) Reset() { *x = ListOfListOfUserInput_List{} - mi := &file_product_proto_msgTypes[179] + mi := &file_product_proto_msgTypes[190] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9285,7 +9829,7 @@ func (x *ListOfListOfUserInput_List) String() string { func (*ListOfListOfUserInput_List) ProtoMessage() {} func (x *ListOfListOfUserInput_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[179] + mi := &file_product_proto_msgTypes[190] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9317,7 +9861,7 @@ type ListOfOrderLine_List struct { func (x *ListOfOrderLine_List) Reset() { *x = ListOfOrderLine_List{} - mi := &file_product_proto_msgTypes[180] + mi := &file_product_proto_msgTypes[191] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9329,7 +9873,7 @@ func (x *ListOfOrderLine_List) String() string { func (*ListOfOrderLine_List) ProtoMessage() {} func (x *ListOfOrderLine_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[180] + mi := &file_product_proto_msgTypes[191] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9361,7 +9905,7 @@ type ListOfProduct_List struct { func (x *ListOfProduct_List) Reset() { *x = ListOfProduct_List{} - mi := &file_product_proto_msgTypes[181] + mi := &file_product_proto_msgTypes[192] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9373,7 +9917,7 @@ func (x *ListOfProduct_List) String() string { func (*ListOfProduct_List) ProtoMessage() {} func (x *ListOfProduct_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[181] + mi := &file_product_proto_msgTypes[192] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9405,7 +9949,7 @@ type ListOfString_List struct { func (x *ListOfString_List) Reset() { *x = ListOfString_List{} - mi := &file_product_proto_msgTypes[182] + mi := &file_product_proto_msgTypes[193] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9417,7 +9961,7 @@ func (x *ListOfString_List) String() string { func (*ListOfString_List) ProtoMessage() {} func (x *ListOfString_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[182] + mi := &file_product_proto_msgTypes[193] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9449,7 +9993,7 @@ type ListOfSubcategory_List struct { func (x *ListOfSubcategory_List) Reset() { *x = ListOfSubcategory_List{} - mi := &file_product_proto_msgTypes[183] + mi := &file_product_proto_msgTypes[194] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9461,7 +10005,7 @@ func (x *ListOfSubcategory_List) String() string { func (*ListOfSubcategory_List) ProtoMessage() {} func (x *ListOfSubcategory_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[183] + mi := &file_product_proto_msgTypes[194] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9493,7 +10037,7 @@ type ListOfUser_List struct { func (x *ListOfUser_List) Reset() { *x = ListOfUser_List{} - mi := &file_product_proto_msgTypes[184] + mi := &file_product_proto_msgTypes[195] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9505,7 +10049,7 @@ func (x *ListOfUser_List) String() string { func (*ListOfUser_List) ProtoMessage() {} func (x *ListOfUser_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[184] + mi := &file_product_proto_msgTypes[195] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9537,7 +10081,7 @@ type ListOfUserInput_List struct { func (x *ListOfUserInput_List) Reset() { *x = ListOfUserInput_List{} - mi := &file_product_proto_msgTypes[185] + mi := &file_product_proto_msgTypes[196] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9549,7 +10093,7 @@ func (x *ListOfUserInput_List) String() string { func (*ListOfUserInput_List) ProtoMessage() {} func (x *ListOfUserInput_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[185] + mi := &file_product_proto_msgTypes[196] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9856,7 +10400,33 @@ const file_product_proto_rawDesc = "" + "!ResolveCategoryProductCountResult\x12#\n" + "\rproduct_count\x18\x01 \x01(\x05R\fproductCount\"k\n" + "#ResolveCategoryProductCountResponse\x12D\n" + - "\x06result\x18\x01 \x03(\v2,.productv1.ResolveCategoryProductCountResultR\x06result\"]\n" + + "\x06result\x18\x01 \x03(\v2,.productv1.ResolveCategoryProductCountResultR\x06result\"_\n" + + "\"ResolveCategoryPopularityScoreArgs\x129\n" + + "\tthreshold\x18\x01 \x01(\v2\x1b.google.protobuf.Int32ValueR\tthreshold\"7\n" + + "%ResolveCategoryPopularityScoreContext\x12\x0e\n" + + "\x02id\x18\x01 \x01(\tR\x02id\"\xc1\x01\n" + + "%ResolveCategoryPopularityScoreRequest\x12J\n" + + "\acontext\x18\x01 \x03(\v20.productv1.ResolveCategoryPopularityScoreContextR\acontext\x12L\n" + + "\n" + + "field_args\x18\x02 \x01(\v2-.productv1.ResolveCategoryPopularityScoreArgsR\tfieldArgs\"n\n" + + "$ResolveCategoryPopularityScoreResult\x12F\n" + + "\x10popularity_score\x18\x01 \x01(\v2\x1b.google.protobuf.Int32ValueR\x0fpopularityScore\"q\n" + + "&ResolveCategoryPopularityScoreResponse\x12G\n" + + "\x06result\x18\x01 \x03(\v2/.productv1.ResolveCategoryPopularityScoreResultR\x06result\"E\n" + + "\"ResolveCategoryCategoryMetricsArgs\x12\x1f\n" + + "\vmetric_type\x18\x01 \x01(\tR\n" + + "metricType\"K\n" + + "%ResolveCategoryCategoryMetricsContext\x12\x0e\n" + + "\x02id\x18\x01 \x01(\tR\x02id\x12\x12\n" + + "\x04name\x18\x02 \x01(\tR\x04name\"\xc1\x01\n" + + "%ResolveCategoryCategoryMetricsRequest\x12J\n" + + "\acontext\x18\x01 \x03(\v20.productv1.ResolveCategoryCategoryMetricsContextR\acontext\x12L\n" + + "\n" + + "field_args\x18\x02 \x01(\v2-.productv1.ResolveCategoryCategoryMetricsArgsR\tfieldArgs\"m\n" + + "$ResolveCategoryCategoryMetricsResult\x12E\n" + + "\x10category_metrics\x18\x01 \x01(\v2\x1a.productv1.CategoryMetricsR\x0fcategoryMetrics\"q\n" + + "&ResolveCategoryCategoryMetricsResponse\x12G\n" + + "\x06result\x18\x01 \x03(\v2/.productv1.ResolveCategoryCategoryMetricsResultR\x06result\"]\n" + "\x1fResolveSubcategoryItemCountArgs\x12:\n" + "\afilters\x18\x01 \x01(\v2 .productv1.SubcategoryItemFilterR\afilters\"4\n" + "\"ResolveSubcategoryItemCountContext\x12\x0e\n" + @@ -10090,7 +10660,15 @@ const file_product_proto_rawDesc = "" + "\x02id\x18\x01 \x01(\tR\x02id\x12\x12\n" + "\x04name\x18\x02 \x01(\tR\x04name\x12>\n" + "\vdescription\x18\x03 \x01(\v2\x1c.google.protobuf.StringValueR\vdescription\x12\x1b\n" + - "\tis_active\x18\x04 \x01(\bR\bisActive\"^\n" + + "\tis_active\x18\x04 \x01(\bR\bisActive\"\x97\x01\n" + + "\x0fCategoryMetrics\x12\x0e\n" + + "\x02id\x18\x01 \x01(\tR\x02id\x12\x1f\n" + + "\vmetric_type\x18\x02 \x01(\tR\n" + + "metricType\x12\x14\n" + + "\x05value\x18\x03 \x01(\x01R\x05value\x12\x1c\n" + + "\ttimestamp\x18\x04 \x01(\tR\ttimestamp\x12\x1f\n" + + "\vcategory_id\x18\x05 \x01(\tR\n" + + "categoryId\"^\n" + "\x03Cat\x12\x0e\n" + "\x02id\x18\x01 \x01(\tR\x02id\x12\x12\n" + "\x04name\x18\x02 \x01(\tR\x04name\x12\x12\n" + @@ -10130,7 +10708,7 @@ const file_product_proto_rawDesc = "" + "\x12CATEGORY_KIND_BOOK\x10\x01\x12\x1d\n" + "\x19CATEGORY_KIND_ELECTRONICS\x10\x02\x12\x1b\n" + "\x17CATEGORY_KIND_FURNITURE\x10\x03\x12\x17\n" + - "\x13CATEGORY_KIND_OTHER\x10\x042\xb8(\n" + + "\x13CATEGORY_KIND_OTHER\x10\x042\xcc*\n" + "\x0eProductService\x12`\n" + "\x11LookupProductById\x12#.productv1.LookupProductByIdRequest\x1a$.productv1.LookupProductByIdResponse\"\x00\x12`\n" + "\x11LookupStorageById\x12#.productv1.LookupStorageByIdRequest\x1a$.productv1.LookupStorageByIdResponse\"\x00\x12f\n" + @@ -10177,7 +10755,9 @@ const file_product_proto_rawDesc = "" + "!QueryTypeWithMultipleFilterFields\x123.productv1.QueryTypeWithMultipleFilterFieldsRequest\x1a4.productv1.QueryTypeWithMultipleFilterFieldsResponse\"\x00\x12H\n" + "\tQueryUser\x12\x1b.productv1.QueryUserRequest\x1a\x1c.productv1.QueryUserResponse\"\x00\x12K\n" + "\n" + - "QueryUsers\x12\x1c.productv1.QueryUsersRequest\x1a\x1d.productv1.QueryUsersResponse\"\x00\x12~\n" + + "QueryUsers\x12\x1c.productv1.QueryUsersRequest\x1a\x1d.productv1.QueryUsersResponse\"\x00\x12\x87\x01\n" + + "\x1eResolveCategoryCategoryMetrics\x120.productv1.ResolveCategoryCategoryMetricsRequest\x1a1.productv1.ResolveCategoryCategoryMetricsResponse\"\x00\x12\x87\x01\n" + + "\x1eResolveCategoryPopularityScore\x120.productv1.ResolveCategoryPopularityScoreRequest\x1a1.productv1.ResolveCategoryPopularityScoreResponse\"\x00\x12~\n" + "\x1bResolveCategoryProductCount\x12-.productv1.ResolveCategoryProductCountRequest\x1a..productv1.ResolveCategoryProductCountResponse\"\x00\x12~\n" + "\x1bResolveSubcategoryItemCount\x12-.productv1.ResolveSubcategoryItemCountRequest\x1a..productv1.ResolveSubcategoryItemCountResponse\"\x00B%Z#cosmo/pkg/proto/productv1;productv1b\x06proto3" @@ -10194,7 +10774,7 @@ func file_product_proto_rawDescGZIP() []byte { } var file_product_proto_enumTypes = make([]protoimpl.EnumInfo, 1) -var file_product_proto_msgTypes = make([]protoimpl.MessageInfo, 186) +var file_product_proto_msgTypes = make([]protoimpl.MessageInfo, 197) var file_product_proto_goTypes = []any{ (CategoryKind)(0), // 0: productv1.CategoryKind (*ListOfAuthorFilter)(nil), // 1: productv1.ListOfAuthorFilter @@ -10315,388 +10895,412 @@ var file_product_proto_goTypes = []any{ (*ResolveCategoryProductCountRequest)(nil), // 116: productv1.ResolveCategoryProductCountRequest (*ResolveCategoryProductCountResult)(nil), // 117: productv1.ResolveCategoryProductCountResult (*ResolveCategoryProductCountResponse)(nil), // 118: productv1.ResolveCategoryProductCountResponse - (*ResolveSubcategoryItemCountArgs)(nil), // 119: productv1.ResolveSubcategoryItemCountArgs - (*ResolveSubcategoryItemCountContext)(nil), // 120: productv1.ResolveSubcategoryItemCountContext - (*ResolveSubcategoryItemCountRequest)(nil), // 121: productv1.ResolveSubcategoryItemCountRequest - (*ResolveSubcategoryItemCountResult)(nil), // 122: productv1.ResolveSubcategoryItemCountResult - (*ResolveSubcategoryItemCountResponse)(nil), // 123: productv1.ResolveSubcategoryItemCountResponse - (*Product)(nil), // 124: productv1.Product - (*Storage)(nil), // 125: productv1.Storage - (*Warehouse)(nil), // 126: productv1.Warehouse - (*User)(nil), // 127: productv1.User - (*NestedTypeA)(nil), // 128: productv1.NestedTypeA - (*RecursiveType)(nil), // 129: productv1.RecursiveType - (*TypeWithMultipleFilterFields)(nil), // 130: productv1.TypeWithMultipleFilterFields - (*FilterTypeInput)(nil), // 131: productv1.FilterTypeInput - (*ComplexFilterTypeInput)(nil), // 132: productv1.ComplexFilterTypeInput - (*TypeWithComplexFilterInput)(nil), // 133: productv1.TypeWithComplexFilterInput - (*OrderInput)(nil), // 134: productv1.OrderInput - (*Order)(nil), // 135: productv1.Order - (*Category)(nil), // 136: productv1.Category - (*CategoryFilter)(nil), // 137: productv1.CategoryFilter - (*Animal)(nil), // 138: productv1.Animal - (*SearchInput)(nil), // 139: productv1.SearchInput - (*SearchResult)(nil), // 140: productv1.SearchResult - (*NullableFieldsType)(nil), // 141: productv1.NullableFieldsType - (*NullableFieldsFilter)(nil), // 142: productv1.NullableFieldsFilter - (*BlogPost)(nil), // 143: productv1.BlogPost - (*BlogPostFilter)(nil), // 144: productv1.BlogPostFilter - (*Author)(nil), // 145: productv1.Author - (*AuthorFilter)(nil), // 146: productv1.AuthorFilter - (*UserInput)(nil), // 147: productv1.UserInput - (*ActionInput)(nil), // 148: productv1.ActionInput - (*ActionResult)(nil), // 149: productv1.ActionResult - (*NullableFieldsInput)(nil), // 150: productv1.NullableFieldsInput - (*BlogPostInput)(nil), // 151: productv1.BlogPostInput - (*AuthorInput)(nil), // 152: productv1.AuthorInput - (*NestedTypeB)(nil), // 153: productv1.NestedTypeB - (*NestedTypeC)(nil), // 154: productv1.NestedTypeC - (*FilterType)(nil), // 155: productv1.FilterType - (*Pagination)(nil), // 156: productv1.Pagination - (*OrderLineInput)(nil), // 157: productv1.OrderLineInput - (*OrderLine)(nil), // 158: productv1.OrderLine - (*Subcategory)(nil), // 159: productv1.Subcategory - (*Cat)(nil), // 160: productv1.Cat - (*Dog)(nil), // 161: productv1.Dog - (*ActionSuccess)(nil), // 162: productv1.ActionSuccess - (*ActionError)(nil), // 163: productv1.ActionError - (*CategoryInput)(nil), // 164: productv1.CategoryInput - (*ProductCountFilter)(nil), // 165: productv1.ProductCountFilter - (*SubcategoryItemFilter)(nil), // 166: productv1.SubcategoryItemFilter - (*ListOfAuthorFilter_List)(nil), // 167: productv1.ListOfAuthorFilter.List - (*ListOfAuthorInput_List)(nil), // 168: productv1.ListOfAuthorInput.List - (*ListOfBlogPost_List)(nil), // 169: productv1.ListOfBlogPost.List - (*ListOfBlogPostFilter_List)(nil), // 170: productv1.ListOfBlogPostFilter.List - (*ListOfBlogPostInput_List)(nil), // 171: productv1.ListOfBlogPostInput.List - (*ListOfBoolean_List)(nil), // 172: productv1.ListOfBoolean.List - (*ListOfCategory_List)(nil), // 173: productv1.ListOfCategory.List - (*ListOfCategoryInput_List)(nil), // 174: productv1.ListOfCategoryInput.List - (*ListOfFloat_List)(nil), // 175: productv1.ListOfFloat.List - (*ListOfListOfCategory_List)(nil), // 176: productv1.ListOfListOfCategory.List - (*ListOfListOfCategoryInput_List)(nil), // 177: productv1.ListOfListOfCategoryInput.List - (*ListOfListOfString_List)(nil), // 178: productv1.ListOfListOfString.List - (*ListOfListOfUser_List)(nil), // 179: productv1.ListOfListOfUser.List - (*ListOfListOfUserInput_List)(nil), // 180: productv1.ListOfListOfUserInput.List - (*ListOfOrderLine_List)(nil), // 181: productv1.ListOfOrderLine.List - (*ListOfProduct_List)(nil), // 182: productv1.ListOfProduct.List - (*ListOfString_List)(nil), // 183: productv1.ListOfString.List - (*ListOfSubcategory_List)(nil), // 184: productv1.ListOfSubcategory.List - (*ListOfUser_List)(nil), // 185: productv1.ListOfUser.List - (*ListOfUserInput_List)(nil), // 186: productv1.ListOfUserInput.List - (*wrapperspb.Int32Value)(nil), // 187: google.protobuf.Int32Value - (*wrapperspb.StringValue)(nil), // 188: google.protobuf.StringValue - (*wrapperspb.DoubleValue)(nil), // 189: google.protobuf.DoubleValue - (*wrapperspb.BoolValue)(nil), // 190: google.protobuf.BoolValue + (*ResolveCategoryPopularityScoreArgs)(nil), // 119: productv1.ResolveCategoryPopularityScoreArgs + (*ResolveCategoryPopularityScoreContext)(nil), // 120: productv1.ResolveCategoryPopularityScoreContext + (*ResolveCategoryPopularityScoreRequest)(nil), // 121: productv1.ResolveCategoryPopularityScoreRequest + (*ResolveCategoryPopularityScoreResult)(nil), // 122: productv1.ResolveCategoryPopularityScoreResult + (*ResolveCategoryPopularityScoreResponse)(nil), // 123: productv1.ResolveCategoryPopularityScoreResponse + (*ResolveCategoryCategoryMetricsArgs)(nil), // 124: productv1.ResolveCategoryCategoryMetricsArgs + (*ResolveCategoryCategoryMetricsContext)(nil), // 125: productv1.ResolveCategoryCategoryMetricsContext + (*ResolveCategoryCategoryMetricsRequest)(nil), // 126: productv1.ResolveCategoryCategoryMetricsRequest + (*ResolveCategoryCategoryMetricsResult)(nil), // 127: productv1.ResolveCategoryCategoryMetricsResult + (*ResolveCategoryCategoryMetricsResponse)(nil), // 128: productv1.ResolveCategoryCategoryMetricsResponse + (*ResolveSubcategoryItemCountArgs)(nil), // 129: productv1.ResolveSubcategoryItemCountArgs + (*ResolveSubcategoryItemCountContext)(nil), // 130: productv1.ResolveSubcategoryItemCountContext + (*ResolveSubcategoryItemCountRequest)(nil), // 131: productv1.ResolveSubcategoryItemCountRequest + (*ResolveSubcategoryItemCountResult)(nil), // 132: productv1.ResolveSubcategoryItemCountResult + (*ResolveSubcategoryItemCountResponse)(nil), // 133: productv1.ResolveSubcategoryItemCountResponse + (*Product)(nil), // 134: productv1.Product + (*Storage)(nil), // 135: productv1.Storage + (*Warehouse)(nil), // 136: productv1.Warehouse + (*User)(nil), // 137: productv1.User + (*NestedTypeA)(nil), // 138: productv1.NestedTypeA + (*RecursiveType)(nil), // 139: productv1.RecursiveType + (*TypeWithMultipleFilterFields)(nil), // 140: productv1.TypeWithMultipleFilterFields + (*FilterTypeInput)(nil), // 141: productv1.FilterTypeInput + (*ComplexFilterTypeInput)(nil), // 142: productv1.ComplexFilterTypeInput + (*TypeWithComplexFilterInput)(nil), // 143: productv1.TypeWithComplexFilterInput + (*OrderInput)(nil), // 144: productv1.OrderInput + (*Order)(nil), // 145: productv1.Order + (*Category)(nil), // 146: productv1.Category + (*CategoryFilter)(nil), // 147: productv1.CategoryFilter + (*Animal)(nil), // 148: productv1.Animal + (*SearchInput)(nil), // 149: productv1.SearchInput + (*SearchResult)(nil), // 150: productv1.SearchResult + (*NullableFieldsType)(nil), // 151: productv1.NullableFieldsType + (*NullableFieldsFilter)(nil), // 152: productv1.NullableFieldsFilter + (*BlogPost)(nil), // 153: productv1.BlogPost + (*BlogPostFilter)(nil), // 154: productv1.BlogPostFilter + (*Author)(nil), // 155: productv1.Author + (*AuthorFilter)(nil), // 156: productv1.AuthorFilter + (*UserInput)(nil), // 157: productv1.UserInput + (*ActionInput)(nil), // 158: productv1.ActionInput + (*ActionResult)(nil), // 159: productv1.ActionResult + (*NullableFieldsInput)(nil), // 160: productv1.NullableFieldsInput + (*BlogPostInput)(nil), // 161: productv1.BlogPostInput + (*AuthorInput)(nil), // 162: productv1.AuthorInput + (*NestedTypeB)(nil), // 163: productv1.NestedTypeB + (*NestedTypeC)(nil), // 164: productv1.NestedTypeC + (*FilterType)(nil), // 165: productv1.FilterType + (*Pagination)(nil), // 166: productv1.Pagination + (*OrderLineInput)(nil), // 167: productv1.OrderLineInput + (*OrderLine)(nil), // 168: productv1.OrderLine + (*Subcategory)(nil), // 169: productv1.Subcategory + (*CategoryMetrics)(nil), // 170: productv1.CategoryMetrics + (*Cat)(nil), // 171: productv1.Cat + (*Dog)(nil), // 172: productv1.Dog + (*ActionSuccess)(nil), // 173: productv1.ActionSuccess + (*ActionError)(nil), // 174: productv1.ActionError + (*CategoryInput)(nil), // 175: productv1.CategoryInput + (*ProductCountFilter)(nil), // 176: productv1.ProductCountFilter + (*SubcategoryItemFilter)(nil), // 177: productv1.SubcategoryItemFilter + (*ListOfAuthorFilter_List)(nil), // 178: productv1.ListOfAuthorFilter.List + (*ListOfAuthorInput_List)(nil), // 179: productv1.ListOfAuthorInput.List + (*ListOfBlogPost_List)(nil), // 180: productv1.ListOfBlogPost.List + (*ListOfBlogPostFilter_List)(nil), // 181: productv1.ListOfBlogPostFilter.List + (*ListOfBlogPostInput_List)(nil), // 182: productv1.ListOfBlogPostInput.List + (*ListOfBoolean_List)(nil), // 183: productv1.ListOfBoolean.List + (*ListOfCategory_List)(nil), // 184: productv1.ListOfCategory.List + (*ListOfCategoryInput_List)(nil), // 185: productv1.ListOfCategoryInput.List + (*ListOfFloat_List)(nil), // 186: productv1.ListOfFloat.List + (*ListOfListOfCategory_List)(nil), // 187: productv1.ListOfListOfCategory.List + (*ListOfListOfCategoryInput_List)(nil), // 188: productv1.ListOfListOfCategoryInput.List + (*ListOfListOfString_List)(nil), // 189: productv1.ListOfListOfString.List + (*ListOfListOfUser_List)(nil), // 190: productv1.ListOfListOfUser.List + (*ListOfListOfUserInput_List)(nil), // 191: productv1.ListOfListOfUserInput.List + (*ListOfOrderLine_List)(nil), // 192: productv1.ListOfOrderLine.List + (*ListOfProduct_List)(nil), // 193: productv1.ListOfProduct.List + (*ListOfString_List)(nil), // 194: productv1.ListOfString.List + (*ListOfSubcategory_List)(nil), // 195: productv1.ListOfSubcategory.List + (*ListOfUser_List)(nil), // 196: productv1.ListOfUser.List + (*ListOfUserInput_List)(nil), // 197: productv1.ListOfUserInput.List + (*wrapperspb.Int32Value)(nil), // 198: google.protobuf.Int32Value + (*wrapperspb.StringValue)(nil), // 199: google.protobuf.StringValue + (*wrapperspb.DoubleValue)(nil), // 200: google.protobuf.DoubleValue + (*wrapperspb.BoolValue)(nil), // 201: google.protobuf.BoolValue } var file_product_proto_depIdxs = []int32{ - 167, // 0: productv1.ListOfAuthorFilter.list:type_name -> productv1.ListOfAuthorFilter.List - 168, // 1: productv1.ListOfAuthorInput.list:type_name -> productv1.ListOfAuthorInput.List - 169, // 2: productv1.ListOfBlogPost.list:type_name -> productv1.ListOfBlogPost.List - 170, // 3: productv1.ListOfBlogPostFilter.list:type_name -> productv1.ListOfBlogPostFilter.List - 171, // 4: productv1.ListOfBlogPostInput.list:type_name -> productv1.ListOfBlogPostInput.List - 172, // 5: productv1.ListOfBoolean.list:type_name -> productv1.ListOfBoolean.List - 173, // 6: productv1.ListOfCategory.list:type_name -> productv1.ListOfCategory.List - 174, // 7: productv1.ListOfCategoryInput.list:type_name -> productv1.ListOfCategoryInput.List - 175, // 8: productv1.ListOfFloat.list:type_name -> productv1.ListOfFloat.List - 176, // 9: productv1.ListOfListOfCategory.list:type_name -> productv1.ListOfListOfCategory.List - 177, // 10: productv1.ListOfListOfCategoryInput.list:type_name -> productv1.ListOfListOfCategoryInput.List - 178, // 11: productv1.ListOfListOfString.list:type_name -> productv1.ListOfListOfString.List - 179, // 12: productv1.ListOfListOfUser.list:type_name -> productv1.ListOfListOfUser.List - 180, // 13: productv1.ListOfListOfUserInput.list:type_name -> productv1.ListOfListOfUserInput.List - 181, // 14: productv1.ListOfOrderLine.list:type_name -> productv1.ListOfOrderLine.List - 182, // 15: productv1.ListOfProduct.list:type_name -> productv1.ListOfProduct.List - 183, // 16: productv1.ListOfString.list:type_name -> productv1.ListOfString.List - 184, // 17: productv1.ListOfSubcategory.list:type_name -> productv1.ListOfSubcategory.List - 185, // 18: productv1.ListOfUser.list:type_name -> productv1.ListOfUser.List - 186, // 19: productv1.ListOfUserInput.list:type_name -> productv1.ListOfUserInput.List + 178, // 0: productv1.ListOfAuthorFilter.list:type_name -> productv1.ListOfAuthorFilter.List + 179, // 1: productv1.ListOfAuthorInput.list:type_name -> productv1.ListOfAuthorInput.List + 180, // 2: productv1.ListOfBlogPost.list:type_name -> productv1.ListOfBlogPost.List + 181, // 3: productv1.ListOfBlogPostFilter.list:type_name -> productv1.ListOfBlogPostFilter.List + 182, // 4: productv1.ListOfBlogPostInput.list:type_name -> productv1.ListOfBlogPostInput.List + 183, // 5: productv1.ListOfBoolean.list:type_name -> productv1.ListOfBoolean.List + 184, // 6: productv1.ListOfCategory.list:type_name -> productv1.ListOfCategory.List + 185, // 7: productv1.ListOfCategoryInput.list:type_name -> productv1.ListOfCategoryInput.List + 186, // 8: productv1.ListOfFloat.list:type_name -> productv1.ListOfFloat.List + 187, // 9: productv1.ListOfListOfCategory.list:type_name -> productv1.ListOfListOfCategory.List + 188, // 10: productv1.ListOfListOfCategoryInput.list:type_name -> productv1.ListOfListOfCategoryInput.List + 189, // 11: productv1.ListOfListOfString.list:type_name -> productv1.ListOfListOfString.List + 190, // 12: productv1.ListOfListOfUser.list:type_name -> productv1.ListOfListOfUser.List + 191, // 13: productv1.ListOfListOfUserInput.list:type_name -> productv1.ListOfListOfUserInput.List + 192, // 14: productv1.ListOfOrderLine.list:type_name -> productv1.ListOfOrderLine.List + 193, // 15: productv1.ListOfProduct.list:type_name -> productv1.ListOfProduct.List + 194, // 16: productv1.ListOfString.list:type_name -> productv1.ListOfString.List + 195, // 17: productv1.ListOfSubcategory.list:type_name -> productv1.ListOfSubcategory.List + 196, // 18: productv1.ListOfUser.list:type_name -> productv1.ListOfUser.List + 197, // 19: productv1.ListOfUserInput.list:type_name -> productv1.ListOfUserInput.List 21, // 20: productv1.LookupProductByIdRequest.keys:type_name -> productv1.LookupProductByIdRequestKey - 124, // 21: productv1.LookupProductByIdResponse.result:type_name -> productv1.Product + 134, // 21: productv1.LookupProductByIdResponse.result:type_name -> productv1.Product 24, // 22: productv1.LookupStorageByIdRequest.keys:type_name -> productv1.LookupStorageByIdRequestKey - 125, // 23: productv1.LookupStorageByIdResponse.result:type_name -> productv1.Storage + 135, // 23: productv1.LookupStorageByIdResponse.result:type_name -> productv1.Storage 27, // 24: productv1.LookupWarehouseByIdRequest.keys:type_name -> productv1.LookupWarehouseByIdRequestKey - 126, // 25: productv1.LookupWarehouseByIdResponse.result:type_name -> productv1.Warehouse - 127, // 26: productv1.QueryUsersResponse.users:type_name -> productv1.User - 127, // 27: productv1.QueryUserResponse.user:type_name -> productv1.User - 128, // 28: productv1.QueryNestedTypeResponse.nested_type:type_name -> productv1.NestedTypeA - 129, // 29: productv1.QueryRecursiveTypeResponse.recursive_type:type_name -> productv1.RecursiveType - 130, // 30: productv1.QueryTypeFilterWithArgumentsResponse.type_filter_with_arguments:type_name -> productv1.TypeWithMultipleFilterFields - 131, // 31: productv1.QueryTypeWithMultipleFilterFieldsRequest.filter:type_name -> productv1.FilterTypeInput - 130, // 32: productv1.QueryTypeWithMultipleFilterFieldsResponse.type_with_multiple_filter_fields:type_name -> productv1.TypeWithMultipleFilterFields - 132, // 33: productv1.QueryComplexFilterTypeRequest.filter:type_name -> productv1.ComplexFilterTypeInput - 133, // 34: productv1.QueryComplexFilterTypeResponse.complex_filter_type:type_name -> productv1.TypeWithComplexFilterInput - 134, // 35: productv1.QueryCalculateTotalsRequest.orders:type_name -> productv1.OrderInput - 135, // 36: productv1.QueryCalculateTotalsResponse.calculate_totals:type_name -> productv1.Order - 136, // 37: productv1.QueryCategoriesResponse.categories:type_name -> productv1.Category + 136, // 25: productv1.LookupWarehouseByIdResponse.result:type_name -> productv1.Warehouse + 137, // 26: productv1.QueryUsersResponse.users:type_name -> productv1.User + 137, // 27: productv1.QueryUserResponse.user:type_name -> productv1.User + 138, // 28: productv1.QueryNestedTypeResponse.nested_type:type_name -> productv1.NestedTypeA + 139, // 29: productv1.QueryRecursiveTypeResponse.recursive_type:type_name -> productv1.RecursiveType + 140, // 30: productv1.QueryTypeFilterWithArgumentsResponse.type_filter_with_arguments:type_name -> productv1.TypeWithMultipleFilterFields + 141, // 31: productv1.QueryTypeWithMultipleFilterFieldsRequest.filter:type_name -> productv1.FilterTypeInput + 140, // 32: productv1.QueryTypeWithMultipleFilterFieldsResponse.type_with_multiple_filter_fields:type_name -> productv1.TypeWithMultipleFilterFields + 142, // 33: productv1.QueryComplexFilterTypeRequest.filter:type_name -> productv1.ComplexFilterTypeInput + 143, // 34: productv1.QueryComplexFilterTypeResponse.complex_filter_type:type_name -> productv1.TypeWithComplexFilterInput + 144, // 35: productv1.QueryCalculateTotalsRequest.orders:type_name -> productv1.OrderInput + 145, // 36: productv1.QueryCalculateTotalsResponse.calculate_totals:type_name -> productv1.Order + 146, // 37: productv1.QueryCategoriesResponse.categories:type_name -> productv1.Category 0, // 38: productv1.QueryCategoriesByKindRequest.kind:type_name -> productv1.CategoryKind - 136, // 39: productv1.QueryCategoriesByKindResponse.categories_by_kind:type_name -> productv1.Category + 146, // 39: productv1.QueryCategoriesByKindResponse.categories_by_kind:type_name -> productv1.Category 0, // 40: productv1.QueryCategoriesByKindsRequest.kinds:type_name -> productv1.CategoryKind - 136, // 41: productv1.QueryCategoriesByKindsResponse.categories_by_kinds:type_name -> productv1.Category - 137, // 42: productv1.QueryFilterCategoriesRequest.filter:type_name -> productv1.CategoryFilter - 136, // 43: productv1.QueryFilterCategoriesResponse.filter_categories:type_name -> productv1.Category - 138, // 44: productv1.QueryRandomPetResponse.random_pet:type_name -> productv1.Animal - 138, // 45: productv1.QueryAllPetsResponse.all_pets:type_name -> productv1.Animal - 139, // 46: productv1.QuerySearchRequest.input:type_name -> productv1.SearchInput - 140, // 47: productv1.QuerySearchResponse.search:type_name -> productv1.SearchResult - 140, // 48: productv1.QueryRandomSearchResultResponse.random_search_result:type_name -> productv1.SearchResult - 141, // 49: productv1.QueryNullableFieldsTypeResponse.nullable_fields_type:type_name -> productv1.NullableFieldsType - 141, // 50: productv1.QueryNullableFieldsTypeByIdResponse.nullable_fields_type_by_id:type_name -> productv1.NullableFieldsType - 142, // 51: productv1.QueryNullableFieldsTypeWithFilterRequest.filter:type_name -> productv1.NullableFieldsFilter - 141, // 52: productv1.QueryNullableFieldsTypeWithFilterResponse.nullable_fields_type_with_filter:type_name -> productv1.NullableFieldsType - 141, // 53: productv1.QueryAllNullableFieldsTypesResponse.all_nullable_fields_types:type_name -> productv1.NullableFieldsType - 143, // 54: productv1.QueryBlogPostResponse.blog_post:type_name -> productv1.BlogPost - 143, // 55: productv1.QueryBlogPostByIdResponse.blog_post_by_id:type_name -> productv1.BlogPost - 144, // 56: productv1.QueryBlogPostsWithFilterRequest.filter:type_name -> productv1.BlogPostFilter - 143, // 57: productv1.QueryBlogPostsWithFilterResponse.blog_posts_with_filter:type_name -> productv1.BlogPost - 143, // 58: productv1.QueryAllBlogPostsResponse.all_blog_posts:type_name -> productv1.BlogPost - 145, // 59: productv1.QueryAuthorResponse.author:type_name -> productv1.Author - 145, // 60: productv1.QueryAuthorByIdResponse.author_by_id:type_name -> productv1.Author - 146, // 61: productv1.QueryAuthorsWithFilterRequest.filter:type_name -> productv1.AuthorFilter - 145, // 62: productv1.QueryAuthorsWithFilterResponse.authors_with_filter:type_name -> productv1.Author - 145, // 63: productv1.QueryAllAuthorsResponse.all_authors:type_name -> productv1.Author + 146, // 41: productv1.QueryCategoriesByKindsResponse.categories_by_kinds:type_name -> productv1.Category + 147, // 42: productv1.QueryFilterCategoriesRequest.filter:type_name -> productv1.CategoryFilter + 146, // 43: productv1.QueryFilterCategoriesResponse.filter_categories:type_name -> productv1.Category + 148, // 44: productv1.QueryRandomPetResponse.random_pet:type_name -> productv1.Animal + 148, // 45: productv1.QueryAllPetsResponse.all_pets:type_name -> productv1.Animal + 149, // 46: productv1.QuerySearchRequest.input:type_name -> productv1.SearchInput + 150, // 47: productv1.QuerySearchResponse.search:type_name -> productv1.SearchResult + 150, // 48: productv1.QueryRandomSearchResultResponse.random_search_result:type_name -> productv1.SearchResult + 151, // 49: productv1.QueryNullableFieldsTypeResponse.nullable_fields_type:type_name -> productv1.NullableFieldsType + 151, // 50: productv1.QueryNullableFieldsTypeByIdResponse.nullable_fields_type_by_id:type_name -> productv1.NullableFieldsType + 152, // 51: productv1.QueryNullableFieldsTypeWithFilterRequest.filter:type_name -> productv1.NullableFieldsFilter + 151, // 52: productv1.QueryNullableFieldsTypeWithFilterResponse.nullable_fields_type_with_filter:type_name -> productv1.NullableFieldsType + 151, // 53: productv1.QueryAllNullableFieldsTypesResponse.all_nullable_fields_types:type_name -> productv1.NullableFieldsType + 153, // 54: productv1.QueryBlogPostResponse.blog_post:type_name -> productv1.BlogPost + 153, // 55: productv1.QueryBlogPostByIdResponse.blog_post_by_id:type_name -> productv1.BlogPost + 154, // 56: productv1.QueryBlogPostsWithFilterRequest.filter:type_name -> productv1.BlogPostFilter + 153, // 57: productv1.QueryBlogPostsWithFilterResponse.blog_posts_with_filter:type_name -> productv1.BlogPost + 153, // 58: productv1.QueryAllBlogPostsResponse.all_blog_posts:type_name -> productv1.BlogPost + 155, // 59: productv1.QueryAuthorResponse.author:type_name -> productv1.Author + 155, // 60: productv1.QueryAuthorByIdResponse.author_by_id:type_name -> productv1.Author + 156, // 61: productv1.QueryAuthorsWithFilterRequest.filter:type_name -> productv1.AuthorFilter + 155, // 62: productv1.QueryAuthorsWithFilterResponse.authors_with_filter:type_name -> productv1.Author + 155, // 63: productv1.QueryAllAuthorsResponse.all_authors:type_name -> productv1.Author 1, // 64: productv1.QueryBulkSearchAuthorsRequest.filters:type_name -> productv1.ListOfAuthorFilter - 145, // 65: productv1.QueryBulkSearchAuthorsResponse.bulk_search_authors:type_name -> productv1.Author + 155, // 65: productv1.QueryBulkSearchAuthorsResponse.bulk_search_authors:type_name -> productv1.Author 4, // 66: productv1.QueryBulkSearchBlogPostsRequest.filters:type_name -> productv1.ListOfBlogPostFilter - 143, // 67: productv1.QueryBulkSearchBlogPostsResponse.bulk_search_blog_posts:type_name -> productv1.BlogPost - 147, // 68: productv1.MutationCreateUserRequest.input:type_name -> productv1.UserInput - 127, // 69: productv1.MutationCreateUserResponse.create_user:type_name -> productv1.User - 148, // 70: productv1.MutationPerformActionRequest.input:type_name -> productv1.ActionInput - 149, // 71: productv1.MutationPerformActionResponse.perform_action:type_name -> productv1.ActionResult - 150, // 72: productv1.MutationCreateNullableFieldsTypeRequest.input:type_name -> productv1.NullableFieldsInput - 141, // 73: productv1.MutationCreateNullableFieldsTypeResponse.create_nullable_fields_type:type_name -> productv1.NullableFieldsType - 150, // 74: productv1.MutationUpdateNullableFieldsTypeRequest.input:type_name -> productv1.NullableFieldsInput - 141, // 75: productv1.MutationUpdateNullableFieldsTypeResponse.update_nullable_fields_type:type_name -> productv1.NullableFieldsType - 151, // 76: productv1.MutationCreateBlogPostRequest.input:type_name -> productv1.BlogPostInput - 143, // 77: productv1.MutationCreateBlogPostResponse.create_blog_post:type_name -> productv1.BlogPost - 151, // 78: productv1.MutationUpdateBlogPostRequest.input:type_name -> productv1.BlogPostInput - 143, // 79: productv1.MutationUpdateBlogPostResponse.update_blog_post:type_name -> productv1.BlogPost - 152, // 80: productv1.MutationCreateAuthorRequest.input:type_name -> productv1.AuthorInput - 145, // 81: productv1.MutationCreateAuthorResponse.create_author:type_name -> productv1.Author - 152, // 82: productv1.MutationUpdateAuthorRequest.input:type_name -> productv1.AuthorInput - 145, // 83: productv1.MutationUpdateAuthorResponse.update_author:type_name -> productv1.Author + 153, // 67: productv1.QueryBulkSearchBlogPostsResponse.bulk_search_blog_posts:type_name -> productv1.BlogPost + 157, // 68: productv1.MutationCreateUserRequest.input:type_name -> productv1.UserInput + 137, // 69: productv1.MutationCreateUserResponse.create_user:type_name -> productv1.User + 158, // 70: productv1.MutationPerformActionRequest.input:type_name -> productv1.ActionInput + 159, // 71: productv1.MutationPerformActionResponse.perform_action:type_name -> productv1.ActionResult + 160, // 72: productv1.MutationCreateNullableFieldsTypeRequest.input:type_name -> productv1.NullableFieldsInput + 151, // 73: productv1.MutationCreateNullableFieldsTypeResponse.create_nullable_fields_type:type_name -> productv1.NullableFieldsType + 160, // 74: productv1.MutationUpdateNullableFieldsTypeRequest.input:type_name -> productv1.NullableFieldsInput + 151, // 75: productv1.MutationUpdateNullableFieldsTypeResponse.update_nullable_fields_type:type_name -> productv1.NullableFieldsType + 161, // 76: productv1.MutationCreateBlogPostRequest.input:type_name -> productv1.BlogPostInput + 153, // 77: productv1.MutationCreateBlogPostResponse.create_blog_post:type_name -> productv1.BlogPost + 161, // 78: productv1.MutationUpdateBlogPostRequest.input:type_name -> productv1.BlogPostInput + 153, // 79: productv1.MutationUpdateBlogPostResponse.update_blog_post:type_name -> productv1.BlogPost + 162, // 80: productv1.MutationCreateAuthorRequest.input:type_name -> productv1.AuthorInput + 155, // 81: productv1.MutationCreateAuthorResponse.create_author:type_name -> productv1.Author + 162, // 82: productv1.MutationUpdateAuthorRequest.input:type_name -> productv1.AuthorInput + 155, // 83: productv1.MutationUpdateAuthorResponse.update_author:type_name -> productv1.Author 2, // 84: productv1.MutationBulkCreateAuthorsRequest.authors:type_name -> productv1.ListOfAuthorInput - 145, // 85: productv1.MutationBulkCreateAuthorsResponse.bulk_create_authors:type_name -> productv1.Author + 155, // 85: productv1.MutationBulkCreateAuthorsResponse.bulk_create_authors:type_name -> productv1.Author 2, // 86: productv1.MutationBulkUpdateAuthorsRequest.authors:type_name -> productv1.ListOfAuthorInput - 145, // 87: productv1.MutationBulkUpdateAuthorsResponse.bulk_update_authors:type_name -> productv1.Author + 155, // 87: productv1.MutationBulkUpdateAuthorsResponse.bulk_update_authors:type_name -> productv1.Author 5, // 88: productv1.MutationBulkCreateBlogPostsRequest.blog_posts:type_name -> productv1.ListOfBlogPostInput - 143, // 89: productv1.MutationBulkCreateBlogPostsResponse.bulk_create_blog_posts:type_name -> productv1.BlogPost + 153, // 89: productv1.MutationBulkCreateBlogPostsResponse.bulk_create_blog_posts:type_name -> productv1.BlogPost 5, // 90: productv1.MutationBulkUpdateBlogPostsRequest.blog_posts:type_name -> productv1.ListOfBlogPostInput - 143, // 91: productv1.MutationBulkUpdateBlogPostsResponse.bulk_update_blog_posts:type_name -> productv1.BlogPost - 165, // 92: productv1.ResolveCategoryProductCountArgs.filters:type_name -> productv1.ProductCountFilter + 153, // 91: productv1.MutationBulkUpdateBlogPostsResponse.bulk_update_blog_posts:type_name -> productv1.BlogPost + 176, // 92: productv1.ResolveCategoryProductCountArgs.filters:type_name -> productv1.ProductCountFilter 115, // 93: productv1.ResolveCategoryProductCountRequest.context:type_name -> productv1.ResolveCategoryProductCountContext 114, // 94: productv1.ResolveCategoryProductCountRequest.field_args:type_name -> productv1.ResolveCategoryProductCountArgs 117, // 95: productv1.ResolveCategoryProductCountResponse.result:type_name -> productv1.ResolveCategoryProductCountResult - 166, // 96: productv1.ResolveSubcategoryItemCountArgs.filters:type_name -> productv1.SubcategoryItemFilter - 120, // 97: productv1.ResolveSubcategoryItemCountRequest.context:type_name -> productv1.ResolveSubcategoryItemCountContext - 119, // 98: productv1.ResolveSubcategoryItemCountRequest.field_args:type_name -> productv1.ResolveSubcategoryItemCountArgs - 122, // 99: productv1.ResolveSubcategoryItemCountResponse.result:type_name -> productv1.ResolveSubcategoryItemCountResult - 153, // 100: productv1.NestedTypeA.b:type_name -> productv1.NestedTypeB - 129, // 101: productv1.RecursiveType.recursive_type:type_name -> productv1.RecursiveType - 155, // 102: productv1.ComplexFilterTypeInput.filter:type_name -> productv1.FilterType - 157, // 103: productv1.OrderInput.lines:type_name -> productv1.OrderLineInput - 15, // 104: productv1.Order.order_lines:type_name -> productv1.ListOfOrderLine - 0, // 105: productv1.Category.kind:type_name -> productv1.CategoryKind - 18, // 106: productv1.Category.subcategories:type_name -> productv1.ListOfSubcategory - 0, // 107: productv1.CategoryFilter.category:type_name -> productv1.CategoryKind - 156, // 108: productv1.CategoryFilter.pagination:type_name -> productv1.Pagination - 160, // 109: productv1.Animal.cat:type_name -> productv1.Cat - 161, // 110: productv1.Animal.dog:type_name -> productv1.Dog - 187, // 111: productv1.SearchInput.limit:type_name -> google.protobuf.Int32Value - 124, // 112: productv1.SearchResult.product:type_name -> productv1.Product - 127, // 113: productv1.SearchResult.user:type_name -> productv1.User - 136, // 114: productv1.SearchResult.category:type_name -> productv1.Category - 188, // 115: productv1.NullableFieldsType.optional_string:type_name -> google.protobuf.StringValue - 187, // 116: productv1.NullableFieldsType.optional_int:type_name -> google.protobuf.Int32Value - 189, // 117: productv1.NullableFieldsType.optional_float:type_name -> google.protobuf.DoubleValue - 190, // 118: productv1.NullableFieldsType.optional_boolean:type_name -> google.protobuf.BoolValue - 188, // 119: productv1.NullableFieldsFilter.name:type_name -> google.protobuf.StringValue - 188, // 120: productv1.NullableFieldsFilter.optional_string:type_name -> google.protobuf.StringValue - 190, // 121: productv1.NullableFieldsFilter.include_nulls:type_name -> google.protobuf.BoolValue - 17, // 122: productv1.BlogPost.optional_tags:type_name -> productv1.ListOfString - 17, // 123: productv1.BlogPost.keywords:type_name -> productv1.ListOfString - 9, // 124: productv1.BlogPost.ratings:type_name -> productv1.ListOfFloat - 6, // 125: productv1.BlogPost.is_published:type_name -> productv1.ListOfBoolean - 12, // 126: productv1.BlogPost.tag_groups:type_name -> productv1.ListOfListOfString - 12, // 127: productv1.BlogPost.related_topics:type_name -> productv1.ListOfListOfString - 12, // 128: productv1.BlogPost.comment_threads:type_name -> productv1.ListOfListOfString - 12, // 129: productv1.BlogPost.suggestions:type_name -> productv1.ListOfListOfString - 136, // 130: productv1.BlogPost.related_categories:type_name -> productv1.Category - 127, // 131: productv1.BlogPost.contributors:type_name -> productv1.User - 16, // 132: productv1.BlogPost.mentioned_products:type_name -> productv1.ListOfProduct - 19, // 133: productv1.BlogPost.mentioned_users:type_name -> productv1.ListOfUser - 10, // 134: productv1.BlogPost.category_groups:type_name -> productv1.ListOfListOfCategory - 13, // 135: productv1.BlogPost.contributor_teams:type_name -> productv1.ListOfListOfUser - 188, // 136: productv1.BlogPostFilter.title:type_name -> google.protobuf.StringValue - 190, // 137: productv1.BlogPostFilter.has_categories:type_name -> google.protobuf.BoolValue - 187, // 138: productv1.BlogPostFilter.min_tags:type_name -> google.protobuf.Int32Value - 188, // 139: productv1.Author.email:type_name -> google.protobuf.StringValue - 17, // 140: productv1.Author.social_links:type_name -> productv1.ListOfString - 12, // 141: productv1.Author.teams_by_project:type_name -> productv1.ListOfListOfString - 12, // 142: productv1.Author.collaborations:type_name -> productv1.ListOfListOfString - 3, // 143: productv1.Author.written_posts:type_name -> productv1.ListOfBlogPost - 136, // 144: productv1.Author.favorite_categories:type_name -> productv1.Category - 19, // 145: productv1.Author.related_authors:type_name -> productv1.ListOfUser - 16, // 146: productv1.Author.product_reviews:type_name -> productv1.ListOfProduct - 13, // 147: productv1.Author.author_groups:type_name -> productv1.ListOfListOfUser - 10, // 148: productv1.Author.category_preferences:type_name -> productv1.ListOfListOfCategory - 13, // 149: productv1.Author.project_teams:type_name -> productv1.ListOfListOfUser - 188, // 150: productv1.AuthorFilter.name:type_name -> google.protobuf.StringValue - 190, // 151: productv1.AuthorFilter.has_teams:type_name -> google.protobuf.BoolValue - 187, // 152: productv1.AuthorFilter.skill_count:type_name -> google.protobuf.Int32Value - 162, // 153: productv1.ActionResult.action_success:type_name -> productv1.ActionSuccess - 163, // 154: productv1.ActionResult.action_error:type_name -> productv1.ActionError - 188, // 155: productv1.NullableFieldsInput.optional_string:type_name -> google.protobuf.StringValue - 187, // 156: productv1.NullableFieldsInput.optional_int:type_name -> google.protobuf.Int32Value - 189, // 157: productv1.NullableFieldsInput.optional_float:type_name -> google.protobuf.DoubleValue - 190, // 158: productv1.NullableFieldsInput.optional_boolean:type_name -> google.protobuf.BoolValue - 17, // 159: productv1.BlogPostInput.optional_tags:type_name -> productv1.ListOfString - 17, // 160: productv1.BlogPostInput.keywords:type_name -> productv1.ListOfString - 9, // 161: productv1.BlogPostInput.ratings:type_name -> productv1.ListOfFloat - 6, // 162: productv1.BlogPostInput.is_published:type_name -> productv1.ListOfBoolean - 12, // 163: productv1.BlogPostInput.tag_groups:type_name -> productv1.ListOfListOfString - 12, // 164: productv1.BlogPostInput.related_topics:type_name -> productv1.ListOfListOfString - 12, // 165: productv1.BlogPostInput.comment_threads:type_name -> productv1.ListOfListOfString - 12, // 166: productv1.BlogPostInput.suggestions:type_name -> productv1.ListOfListOfString - 8, // 167: productv1.BlogPostInput.related_categories:type_name -> productv1.ListOfCategoryInput - 20, // 168: productv1.BlogPostInput.contributors:type_name -> productv1.ListOfUserInput - 11, // 169: productv1.BlogPostInput.category_groups:type_name -> productv1.ListOfListOfCategoryInput - 188, // 170: productv1.AuthorInput.email:type_name -> google.protobuf.StringValue - 17, // 171: productv1.AuthorInput.social_links:type_name -> productv1.ListOfString - 12, // 172: productv1.AuthorInput.teams_by_project:type_name -> productv1.ListOfListOfString - 12, // 173: productv1.AuthorInput.collaborations:type_name -> productv1.ListOfListOfString - 164, // 174: productv1.AuthorInput.favorite_categories:type_name -> productv1.CategoryInput - 14, // 175: productv1.AuthorInput.author_groups:type_name -> productv1.ListOfListOfUserInput - 14, // 176: productv1.AuthorInput.project_teams:type_name -> productv1.ListOfListOfUserInput - 154, // 177: productv1.NestedTypeB.c:type_name -> productv1.NestedTypeC - 156, // 178: productv1.FilterType.pagination:type_name -> productv1.Pagination - 17, // 179: productv1.OrderLineInput.modifiers:type_name -> productv1.ListOfString - 17, // 180: productv1.OrderLine.modifiers:type_name -> productv1.ListOfString - 188, // 181: productv1.Subcategory.description:type_name -> google.protobuf.StringValue - 0, // 182: productv1.CategoryInput.kind:type_name -> productv1.CategoryKind - 189, // 183: productv1.ProductCountFilter.min_price:type_name -> google.protobuf.DoubleValue - 189, // 184: productv1.ProductCountFilter.max_price:type_name -> google.protobuf.DoubleValue - 190, // 185: productv1.ProductCountFilter.in_stock:type_name -> google.protobuf.BoolValue - 188, // 186: productv1.ProductCountFilter.search_term:type_name -> google.protobuf.StringValue - 189, // 187: productv1.SubcategoryItemFilter.min_price:type_name -> google.protobuf.DoubleValue - 189, // 188: productv1.SubcategoryItemFilter.max_price:type_name -> google.protobuf.DoubleValue - 190, // 189: productv1.SubcategoryItemFilter.in_stock:type_name -> google.protobuf.BoolValue - 190, // 190: productv1.SubcategoryItemFilter.is_active:type_name -> google.protobuf.BoolValue - 188, // 191: productv1.SubcategoryItemFilter.search_term:type_name -> google.protobuf.StringValue - 146, // 192: productv1.ListOfAuthorFilter.List.items:type_name -> productv1.AuthorFilter - 152, // 193: productv1.ListOfAuthorInput.List.items:type_name -> productv1.AuthorInput - 143, // 194: productv1.ListOfBlogPost.List.items:type_name -> productv1.BlogPost - 144, // 195: productv1.ListOfBlogPostFilter.List.items:type_name -> productv1.BlogPostFilter - 151, // 196: productv1.ListOfBlogPostInput.List.items:type_name -> productv1.BlogPostInput - 136, // 197: productv1.ListOfCategory.List.items:type_name -> productv1.Category - 164, // 198: productv1.ListOfCategoryInput.List.items:type_name -> productv1.CategoryInput - 7, // 199: productv1.ListOfListOfCategory.List.items:type_name -> productv1.ListOfCategory - 8, // 200: productv1.ListOfListOfCategoryInput.List.items:type_name -> productv1.ListOfCategoryInput - 17, // 201: productv1.ListOfListOfString.List.items:type_name -> productv1.ListOfString - 19, // 202: productv1.ListOfListOfUser.List.items:type_name -> productv1.ListOfUser - 20, // 203: productv1.ListOfListOfUserInput.List.items:type_name -> productv1.ListOfUserInput - 158, // 204: productv1.ListOfOrderLine.List.items:type_name -> productv1.OrderLine - 124, // 205: productv1.ListOfProduct.List.items:type_name -> productv1.Product - 159, // 206: productv1.ListOfSubcategory.List.items:type_name -> productv1.Subcategory - 127, // 207: productv1.ListOfUser.List.items:type_name -> productv1.User - 147, // 208: productv1.ListOfUserInput.List.items:type_name -> productv1.UserInput - 22, // 209: productv1.ProductService.LookupProductById:input_type -> productv1.LookupProductByIdRequest - 25, // 210: productv1.ProductService.LookupStorageById:input_type -> productv1.LookupStorageByIdRequest - 28, // 211: productv1.ProductService.LookupWarehouseById:input_type -> productv1.LookupWarehouseByIdRequest - 106, // 212: productv1.ProductService.MutationBulkCreateAuthors:input_type -> productv1.MutationBulkCreateAuthorsRequest - 110, // 213: productv1.ProductService.MutationBulkCreateBlogPosts:input_type -> productv1.MutationBulkCreateBlogPostsRequest - 108, // 214: productv1.ProductService.MutationBulkUpdateAuthors:input_type -> productv1.MutationBulkUpdateAuthorsRequest - 112, // 215: productv1.ProductService.MutationBulkUpdateBlogPosts:input_type -> productv1.MutationBulkUpdateBlogPostsRequest - 102, // 216: productv1.ProductService.MutationCreateAuthor:input_type -> productv1.MutationCreateAuthorRequest - 98, // 217: productv1.ProductService.MutationCreateBlogPost:input_type -> productv1.MutationCreateBlogPostRequest - 94, // 218: productv1.ProductService.MutationCreateNullableFieldsType:input_type -> productv1.MutationCreateNullableFieldsTypeRequest - 90, // 219: productv1.ProductService.MutationCreateUser:input_type -> productv1.MutationCreateUserRequest - 92, // 220: productv1.ProductService.MutationPerformAction:input_type -> productv1.MutationPerformActionRequest - 104, // 221: productv1.ProductService.MutationUpdateAuthor:input_type -> productv1.MutationUpdateAuthorRequest - 100, // 222: productv1.ProductService.MutationUpdateBlogPost:input_type -> productv1.MutationUpdateBlogPostRequest - 96, // 223: productv1.ProductService.MutationUpdateNullableFieldsType:input_type -> productv1.MutationUpdateNullableFieldsTypeRequest - 84, // 224: productv1.ProductService.QueryAllAuthors:input_type -> productv1.QueryAllAuthorsRequest - 76, // 225: productv1.ProductService.QueryAllBlogPosts:input_type -> productv1.QueryAllBlogPostsRequest - 68, // 226: productv1.ProductService.QueryAllNullableFieldsTypes:input_type -> productv1.QueryAllNullableFieldsTypesRequest - 56, // 227: productv1.ProductService.QueryAllPets:input_type -> productv1.QueryAllPetsRequest - 78, // 228: productv1.ProductService.QueryAuthor:input_type -> productv1.QueryAuthorRequest - 80, // 229: productv1.ProductService.QueryAuthorById:input_type -> productv1.QueryAuthorByIdRequest - 82, // 230: productv1.ProductService.QueryAuthorsWithFilter:input_type -> productv1.QueryAuthorsWithFilterRequest - 70, // 231: productv1.ProductService.QueryBlogPost:input_type -> productv1.QueryBlogPostRequest - 72, // 232: productv1.ProductService.QueryBlogPostById:input_type -> productv1.QueryBlogPostByIdRequest - 74, // 233: productv1.ProductService.QueryBlogPostsWithFilter:input_type -> productv1.QueryBlogPostsWithFilterRequest - 86, // 234: productv1.ProductService.QueryBulkSearchAuthors:input_type -> productv1.QueryBulkSearchAuthorsRequest - 88, // 235: productv1.ProductService.QueryBulkSearchBlogPosts:input_type -> productv1.QueryBulkSearchBlogPostsRequest - 44, // 236: productv1.ProductService.QueryCalculateTotals:input_type -> productv1.QueryCalculateTotalsRequest - 46, // 237: productv1.ProductService.QueryCategories:input_type -> productv1.QueryCategoriesRequest - 48, // 238: productv1.ProductService.QueryCategoriesByKind:input_type -> productv1.QueryCategoriesByKindRequest - 50, // 239: productv1.ProductService.QueryCategoriesByKinds:input_type -> productv1.QueryCategoriesByKindsRequest - 42, // 240: productv1.ProductService.QueryComplexFilterType:input_type -> productv1.QueryComplexFilterTypeRequest - 52, // 241: productv1.ProductService.QueryFilterCategories:input_type -> productv1.QueryFilterCategoriesRequest - 34, // 242: productv1.ProductService.QueryNestedType:input_type -> productv1.QueryNestedTypeRequest - 62, // 243: productv1.ProductService.QueryNullableFieldsType:input_type -> productv1.QueryNullableFieldsTypeRequest - 64, // 244: productv1.ProductService.QueryNullableFieldsTypeById:input_type -> productv1.QueryNullableFieldsTypeByIdRequest - 66, // 245: productv1.ProductService.QueryNullableFieldsTypeWithFilter:input_type -> productv1.QueryNullableFieldsTypeWithFilterRequest - 54, // 246: productv1.ProductService.QueryRandomPet:input_type -> productv1.QueryRandomPetRequest - 60, // 247: productv1.ProductService.QueryRandomSearchResult:input_type -> productv1.QueryRandomSearchResultRequest - 36, // 248: productv1.ProductService.QueryRecursiveType:input_type -> productv1.QueryRecursiveTypeRequest - 58, // 249: productv1.ProductService.QuerySearch:input_type -> productv1.QuerySearchRequest - 38, // 250: productv1.ProductService.QueryTypeFilterWithArguments:input_type -> productv1.QueryTypeFilterWithArgumentsRequest - 40, // 251: productv1.ProductService.QueryTypeWithMultipleFilterFields:input_type -> productv1.QueryTypeWithMultipleFilterFieldsRequest - 32, // 252: productv1.ProductService.QueryUser:input_type -> productv1.QueryUserRequest - 30, // 253: productv1.ProductService.QueryUsers:input_type -> productv1.QueryUsersRequest - 116, // 254: productv1.ProductService.ResolveCategoryProductCount:input_type -> productv1.ResolveCategoryProductCountRequest - 121, // 255: productv1.ProductService.ResolveSubcategoryItemCount:input_type -> productv1.ResolveSubcategoryItemCountRequest - 23, // 256: productv1.ProductService.LookupProductById:output_type -> productv1.LookupProductByIdResponse - 26, // 257: productv1.ProductService.LookupStorageById:output_type -> productv1.LookupStorageByIdResponse - 29, // 258: productv1.ProductService.LookupWarehouseById:output_type -> productv1.LookupWarehouseByIdResponse - 107, // 259: productv1.ProductService.MutationBulkCreateAuthors:output_type -> productv1.MutationBulkCreateAuthorsResponse - 111, // 260: productv1.ProductService.MutationBulkCreateBlogPosts:output_type -> productv1.MutationBulkCreateBlogPostsResponse - 109, // 261: productv1.ProductService.MutationBulkUpdateAuthors:output_type -> productv1.MutationBulkUpdateAuthorsResponse - 113, // 262: productv1.ProductService.MutationBulkUpdateBlogPosts:output_type -> productv1.MutationBulkUpdateBlogPostsResponse - 103, // 263: productv1.ProductService.MutationCreateAuthor:output_type -> productv1.MutationCreateAuthorResponse - 99, // 264: productv1.ProductService.MutationCreateBlogPost:output_type -> productv1.MutationCreateBlogPostResponse - 95, // 265: productv1.ProductService.MutationCreateNullableFieldsType:output_type -> productv1.MutationCreateNullableFieldsTypeResponse - 91, // 266: productv1.ProductService.MutationCreateUser:output_type -> productv1.MutationCreateUserResponse - 93, // 267: productv1.ProductService.MutationPerformAction:output_type -> productv1.MutationPerformActionResponse - 105, // 268: productv1.ProductService.MutationUpdateAuthor:output_type -> productv1.MutationUpdateAuthorResponse - 101, // 269: productv1.ProductService.MutationUpdateBlogPost:output_type -> productv1.MutationUpdateBlogPostResponse - 97, // 270: productv1.ProductService.MutationUpdateNullableFieldsType:output_type -> productv1.MutationUpdateNullableFieldsTypeResponse - 85, // 271: productv1.ProductService.QueryAllAuthors:output_type -> productv1.QueryAllAuthorsResponse - 77, // 272: productv1.ProductService.QueryAllBlogPosts:output_type -> productv1.QueryAllBlogPostsResponse - 69, // 273: productv1.ProductService.QueryAllNullableFieldsTypes:output_type -> productv1.QueryAllNullableFieldsTypesResponse - 57, // 274: productv1.ProductService.QueryAllPets:output_type -> productv1.QueryAllPetsResponse - 79, // 275: productv1.ProductService.QueryAuthor:output_type -> productv1.QueryAuthorResponse - 81, // 276: productv1.ProductService.QueryAuthorById:output_type -> productv1.QueryAuthorByIdResponse - 83, // 277: productv1.ProductService.QueryAuthorsWithFilter:output_type -> productv1.QueryAuthorsWithFilterResponse - 71, // 278: productv1.ProductService.QueryBlogPost:output_type -> productv1.QueryBlogPostResponse - 73, // 279: productv1.ProductService.QueryBlogPostById:output_type -> productv1.QueryBlogPostByIdResponse - 75, // 280: productv1.ProductService.QueryBlogPostsWithFilter:output_type -> productv1.QueryBlogPostsWithFilterResponse - 87, // 281: productv1.ProductService.QueryBulkSearchAuthors:output_type -> productv1.QueryBulkSearchAuthorsResponse - 89, // 282: productv1.ProductService.QueryBulkSearchBlogPosts:output_type -> productv1.QueryBulkSearchBlogPostsResponse - 45, // 283: productv1.ProductService.QueryCalculateTotals:output_type -> productv1.QueryCalculateTotalsResponse - 47, // 284: productv1.ProductService.QueryCategories:output_type -> productv1.QueryCategoriesResponse - 49, // 285: productv1.ProductService.QueryCategoriesByKind:output_type -> productv1.QueryCategoriesByKindResponse - 51, // 286: productv1.ProductService.QueryCategoriesByKinds:output_type -> productv1.QueryCategoriesByKindsResponse - 43, // 287: productv1.ProductService.QueryComplexFilterType:output_type -> productv1.QueryComplexFilterTypeResponse - 53, // 288: productv1.ProductService.QueryFilterCategories:output_type -> productv1.QueryFilterCategoriesResponse - 35, // 289: productv1.ProductService.QueryNestedType:output_type -> productv1.QueryNestedTypeResponse - 63, // 290: productv1.ProductService.QueryNullableFieldsType:output_type -> productv1.QueryNullableFieldsTypeResponse - 65, // 291: productv1.ProductService.QueryNullableFieldsTypeById:output_type -> productv1.QueryNullableFieldsTypeByIdResponse - 67, // 292: productv1.ProductService.QueryNullableFieldsTypeWithFilter:output_type -> productv1.QueryNullableFieldsTypeWithFilterResponse - 55, // 293: productv1.ProductService.QueryRandomPet:output_type -> productv1.QueryRandomPetResponse - 61, // 294: productv1.ProductService.QueryRandomSearchResult:output_type -> productv1.QueryRandomSearchResultResponse - 37, // 295: productv1.ProductService.QueryRecursiveType:output_type -> productv1.QueryRecursiveTypeResponse - 59, // 296: productv1.ProductService.QuerySearch:output_type -> productv1.QuerySearchResponse - 39, // 297: productv1.ProductService.QueryTypeFilterWithArguments:output_type -> productv1.QueryTypeFilterWithArgumentsResponse - 41, // 298: productv1.ProductService.QueryTypeWithMultipleFilterFields:output_type -> productv1.QueryTypeWithMultipleFilterFieldsResponse - 33, // 299: productv1.ProductService.QueryUser:output_type -> productv1.QueryUserResponse - 31, // 300: productv1.ProductService.QueryUsers:output_type -> productv1.QueryUsersResponse - 118, // 301: productv1.ProductService.ResolveCategoryProductCount:output_type -> productv1.ResolveCategoryProductCountResponse - 123, // 302: productv1.ProductService.ResolveSubcategoryItemCount:output_type -> productv1.ResolveSubcategoryItemCountResponse - 256, // [256:303] is the sub-list for method output_type - 209, // [209:256] is the sub-list for method input_type - 209, // [209:209] is the sub-list for extension type_name - 209, // [209:209] is the sub-list for extension extendee - 0, // [0:209] is the sub-list for field type_name + 198, // 96: productv1.ResolveCategoryPopularityScoreArgs.threshold:type_name -> google.protobuf.Int32Value + 120, // 97: productv1.ResolveCategoryPopularityScoreRequest.context:type_name -> productv1.ResolveCategoryPopularityScoreContext + 119, // 98: productv1.ResolveCategoryPopularityScoreRequest.field_args:type_name -> productv1.ResolveCategoryPopularityScoreArgs + 198, // 99: productv1.ResolveCategoryPopularityScoreResult.popularity_score:type_name -> google.protobuf.Int32Value + 122, // 100: productv1.ResolveCategoryPopularityScoreResponse.result:type_name -> productv1.ResolveCategoryPopularityScoreResult + 125, // 101: productv1.ResolveCategoryCategoryMetricsRequest.context:type_name -> productv1.ResolveCategoryCategoryMetricsContext + 124, // 102: productv1.ResolveCategoryCategoryMetricsRequest.field_args:type_name -> productv1.ResolveCategoryCategoryMetricsArgs + 170, // 103: productv1.ResolveCategoryCategoryMetricsResult.category_metrics:type_name -> productv1.CategoryMetrics + 127, // 104: productv1.ResolveCategoryCategoryMetricsResponse.result:type_name -> productv1.ResolveCategoryCategoryMetricsResult + 177, // 105: productv1.ResolveSubcategoryItemCountArgs.filters:type_name -> productv1.SubcategoryItemFilter + 130, // 106: productv1.ResolveSubcategoryItemCountRequest.context:type_name -> productv1.ResolveSubcategoryItemCountContext + 129, // 107: productv1.ResolveSubcategoryItemCountRequest.field_args:type_name -> productv1.ResolveSubcategoryItemCountArgs + 132, // 108: productv1.ResolveSubcategoryItemCountResponse.result:type_name -> productv1.ResolveSubcategoryItemCountResult + 163, // 109: productv1.NestedTypeA.b:type_name -> productv1.NestedTypeB + 139, // 110: productv1.RecursiveType.recursive_type:type_name -> productv1.RecursiveType + 165, // 111: productv1.ComplexFilterTypeInput.filter:type_name -> productv1.FilterType + 167, // 112: productv1.OrderInput.lines:type_name -> productv1.OrderLineInput + 15, // 113: productv1.Order.order_lines:type_name -> productv1.ListOfOrderLine + 0, // 114: productv1.Category.kind:type_name -> productv1.CategoryKind + 18, // 115: productv1.Category.subcategories:type_name -> productv1.ListOfSubcategory + 0, // 116: productv1.CategoryFilter.category:type_name -> productv1.CategoryKind + 166, // 117: productv1.CategoryFilter.pagination:type_name -> productv1.Pagination + 171, // 118: productv1.Animal.cat:type_name -> productv1.Cat + 172, // 119: productv1.Animal.dog:type_name -> productv1.Dog + 198, // 120: productv1.SearchInput.limit:type_name -> google.protobuf.Int32Value + 134, // 121: productv1.SearchResult.product:type_name -> productv1.Product + 137, // 122: productv1.SearchResult.user:type_name -> productv1.User + 146, // 123: productv1.SearchResult.category:type_name -> productv1.Category + 199, // 124: productv1.NullableFieldsType.optional_string:type_name -> google.protobuf.StringValue + 198, // 125: productv1.NullableFieldsType.optional_int:type_name -> google.protobuf.Int32Value + 200, // 126: productv1.NullableFieldsType.optional_float:type_name -> google.protobuf.DoubleValue + 201, // 127: productv1.NullableFieldsType.optional_boolean:type_name -> google.protobuf.BoolValue + 199, // 128: productv1.NullableFieldsFilter.name:type_name -> google.protobuf.StringValue + 199, // 129: productv1.NullableFieldsFilter.optional_string:type_name -> google.protobuf.StringValue + 201, // 130: productv1.NullableFieldsFilter.include_nulls:type_name -> google.protobuf.BoolValue + 17, // 131: productv1.BlogPost.optional_tags:type_name -> productv1.ListOfString + 17, // 132: productv1.BlogPost.keywords:type_name -> productv1.ListOfString + 9, // 133: productv1.BlogPost.ratings:type_name -> productv1.ListOfFloat + 6, // 134: productv1.BlogPost.is_published:type_name -> productv1.ListOfBoolean + 12, // 135: productv1.BlogPost.tag_groups:type_name -> productv1.ListOfListOfString + 12, // 136: productv1.BlogPost.related_topics:type_name -> productv1.ListOfListOfString + 12, // 137: productv1.BlogPost.comment_threads:type_name -> productv1.ListOfListOfString + 12, // 138: productv1.BlogPost.suggestions:type_name -> productv1.ListOfListOfString + 146, // 139: productv1.BlogPost.related_categories:type_name -> productv1.Category + 137, // 140: productv1.BlogPost.contributors:type_name -> productv1.User + 16, // 141: productv1.BlogPost.mentioned_products:type_name -> productv1.ListOfProduct + 19, // 142: productv1.BlogPost.mentioned_users:type_name -> productv1.ListOfUser + 10, // 143: productv1.BlogPost.category_groups:type_name -> productv1.ListOfListOfCategory + 13, // 144: productv1.BlogPost.contributor_teams:type_name -> productv1.ListOfListOfUser + 199, // 145: productv1.BlogPostFilter.title:type_name -> google.protobuf.StringValue + 201, // 146: productv1.BlogPostFilter.has_categories:type_name -> google.protobuf.BoolValue + 198, // 147: productv1.BlogPostFilter.min_tags:type_name -> google.protobuf.Int32Value + 199, // 148: productv1.Author.email:type_name -> google.protobuf.StringValue + 17, // 149: productv1.Author.social_links:type_name -> productv1.ListOfString + 12, // 150: productv1.Author.teams_by_project:type_name -> productv1.ListOfListOfString + 12, // 151: productv1.Author.collaborations:type_name -> productv1.ListOfListOfString + 3, // 152: productv1.Author.written_posts:type_name -> productv1.ListOfBlogPost + 146, // 153: productv1.Author.favorite_categories:type_name -> productv1.Category + 19, // 154: productv1.Author.related_authors:type_name -> productv1.ListOfUser + 16, // 155: productv1.Author.product_reviews:type_name -> productv1.ListOfProduct + 13, // 156: productv1.Author.author_groups:type_name -> productv1.ListOfListOfUser + 10, // 157: productv1.Author.category_preferences:type_name -> productv1.ListOfListOfCategory + 13, // 158: productv1.Author.project_teams:type_name -> productv1.ListOfListOfUser + 199, // 159: productv1.AuthorFilter.name:type_name -> google.protobuf.StringValue + 201, // 160: productv1.AuthorFilter.has_teams:type_name -> google.protobuf.BoolValue + 198, // 161: productv1.AuthorFilter.skill_count:type_name -> google.protobuf.Int32Value + 173, // 162: productv1.ActionResult.action_success:type_name -> productv1.ActionSuccess + 174, // 163: productv1.ActionResult.action_error:type_name -> productv1.ActionError + 199, // 164: productv1.NullableFieldsInput.optional_string:type_name -> google.protobuf.StringValue + 198, // 165: productv1.NullableFieldsInput.optional_int:type_name -> google.protobuf.Int32Value + 200, // 166: productv1.NullableFieldsInput.optional_float:type_name -> google.protobuf.DoubleValue + 201, // 167: productv1.NullableFieldsInput.optional_boolean:type_name -> google.protobuf.BoolValue + 17, // 168: productv1.BlogPostInput.optional_tags:type_name -> productv1.ListOfString + 17, // 169: productv1.BlogPostInput.keywords:type_name -> productv1.ListOfString + 9, // 170: productv1.BlogPostInput.ratings:type_name -> productv1.ListOfFloat + 6, // 171: productv1.BlogPostInput.is_published:type_name -> productv1.ListOfBoolean + 12, // 172: productv1.BlogPostInput.tag_groups:type_name -> productv1.ListOfListOfString + 12, // 173: productv1.BlogPostInput.related_topics:type_name -> productv1.ListOfListOfString + 12, // 174: productv1.BlogPostInput.comment_threads:type_name -> productv1.ListOfListOfString + 12, // 175: productv1.BlogPostInput.suggestions:type_name -> productv1.ListOfListOfString + 8, // 176: productv1.BlogPostInput.related_categories:type_name -> productv1.ListOfCategoryInput + 20, // 177: productv1.BlogPostInput.contributors:type_name -> productv1.ListOfUserInput + 11, // 178: productv1.BlogPostInput.category_groups:type_name -> productv1.ListOfListOfCategoryInput + 199, // 179: productv1.AuthorInput.email:type_name -> google.protobuf.StringValue + 17, // 180: productv1.AuthorInput.social_links:type_name -> productv1.ListOfString + 12, // 181: productv1.AuthorInput.teams_by_project:type_name -> productv1.ListOfListOfString + 12, // 182: productv1.AuthorInput.collaborations:type_name -> productv1.ListOfListOfString + 175, // 183: productv1.AuthorInput.favorite_categories:type_name -> productv1.CategoryInput + 14, // 184: productv1.AuthorInput.author_groups:type_name -> productv1.ListOfListOfUserInput + 14, // 185: productv1.AuthorInput.project_teams:type_name -> productv1.ListOfListOfUserInput + 164, // 186: productv1.NestedTypeB.c:type_name -> productv1.NestedTypeC + 166, // 187: productv1.FilterType.pagination:type_name -> productv1.Pagination + 17, // 188: productv1.OrderLineInput.modifiers:type_name -> productv1.ListOfString + 17, // 189: productv1.OrderLine.modifiers:type_name -> productv1.ListOfString + 199, // 190: productv1.Subcategory.description:type_name -> google.protobuf.StringValue + 0, // 191: productv1.CategoryInput.kind:type_name -> productv1.CategoryKind + 200, // 192: productv1.ProductCountFilter.min_price:type_name -> google.protobuf.DoubleValue + 200, // 193: productv1.ProductCountFilter.max_price:type_name -> google.protobuf.DoubleValue + 201, // 194: productv1.ProductCountFilter.in_stock:type_name -> google.protobuf.BoolValue + 199, // 195: productv1.ProductCountFilter.search_term:type_name -> google.protobuf.StringValue + 200, // 196: productv1.SubcategoryItemFilter.min_price:type_name -> google.protobuf.DoubleValue + 200, // 197: productv1.SubcategoryItemFilter.max_price:type_name -> google.protobuf.DoubleValue + 201, // 198: productv1.SubcategoryItemFilter.in_stock:type_name -> google.protobuf.BoolValue + 201, // 199: productv1.SubcategoryItemFilter.is_active:type_name -> google.protobuf.BoolValue + 199, // 200: productv1.SubcategoryItemFilter.search_term:type_name -> google.protobuf.StringValue + 156, // 201: productv1.ListOfAuthorFilter.List.items:type_name -> productv1.AuthorFilter + 162, // 202: productv1.ListOfAuthorInput.List.items:type_name -> productv1.AuthorInput + 153, // 203: productv1.ListOfBlogPost.List.items:type_name -> productv1.BlogPost + 154, // 204: productv1.ListOfBlogPostFilter.List.items:type_name -> productv1.BlogPostFilter + 161, // 205: productv1.ListOfBlogPostInput.List.items:type_name -> productv1.BlogPostInput + 146, // 206: productv1.ListOfCategory.List.items:type_name -> productv1.Category + 175, // 207: productv1.ListOfCategoryInput.List.items:type_name -> productv1.CategoryInput + 7, // 208: productv1.ListOfListOfCategory.List.items:type_name -> productv1.ListOfCategory + 8, // 209: productv1.ListOfListOfCategoryInput.List.items:type_name -> productv1.ListOfCategoryInput + 17, // 210: productv1.ListOfListOfString.List.items:type_name -> productv1.ListOfString + 19, // 211: productv1.ListOfListOfUser.List.items:type_name -> productv1.ListOfUser + 20, // 212: productv1.ListOfListOfUserInput.List.items:type_name -> productv1.ListOfUserInput + 168, // 213: productv1.ListOfOrderLine.List.items:type_name -> productv1.OrderLine + 134, // 214: productv1.ListOfProduct.List.items:type_name -> productv1.Product + 169, // 215: productv1.ListOfSubcategory.List.items:type_name -> productv1.Subcategory + 137, // 216: productv1.ListOfUser.List.items:type_name -> productv1.User + 157, // 217: productv1.ListOfUserInput.List.items:type_name -> productv1.UserInput + 22, // 218: productv1.ProductService.LookupProductById:input_type -> productv1.LookupProductByIdRequest + 25, // 219: productv1.ProductService.LookupStorageById:input_type -> productv1.LookupStorageByIdRequest + 28, // 220: productv1.ProductService.LookupWarehouseById:input_type -> productv1.LookupWarehouseByIdRequest + 106, // 221: productv1.ProductService.MutationBulkCreateAuthors:input_type -> productv1.MutationBulkCreateAuthorsRequest + 110, // 222: productv1.ProductService.MutationBulkCreateBlogPosts:input_type -> productv1.MutationBulkCreateBlogPostsRequest + 108, // 223: productv1.ProductService.MutationBulkUpdateAuthors:input_type -> productv1.MutationBulkUpdateAuthorsRequest + 112, // 224: productv1.ProductService.MutationBulkUpdateBlogPosts:input_type -> productv1.MutationBulkUpdateBlogPostsRequest + 102, // 225: productv1.ProductService.MutationCreateAuthor:input_type -> productv1.MutationCreateAuthorRequest + 98, // 226: productv1.ProductService.MutationCreateBlogPost:input_type -> productv1.MutationCreateBlogPostRequest + 94, // 227: productv1.ProductService.MutationCreateNullableFieldsType:input_type -> productv1.MutationCreateNullableFieldsTypeRequest + 90, // 228: productv1.ProductService.MutationCreateUser:input_type -> productv1.MutationCreateUserRequest + 92, // 229: productv1.ProductService.MutationPerformAction:input_type -> productv1.MutationPerformActionRequest + 104, // 230: productv1.ProductService.MutationUpdateAuthor:input_type -> productv1.MutationUpdateAuthorRequest + 100, // 231: productv1.ProductService.MutationUpdateBlogPost:input_type -> productv1.MutationUpdateBlogPostRequest + 96, // 232: productv1.ProductService.MutationUpdateNullableFieldsType:input_type -> productv1.MutationUpdateNullableFieldsTypeRequest + 84, // 233: productv1.ProductService.QueryAllAuthors:input_type -> productv1.QueryAllAuthorsRequest + 76, // 234: productv1.ProductService.QueryAllBlogPosts:input_type -> productv1.QueryAllBlogPostsRequest + 68, // 235: productv1.ProductService.QueryAllNullableFieldsTypes:input_type -> productv1.QueryAllNullableFieldsTypesRequest + 56, // 236: productv1.ProductService.QueryAllPets:input_type -> productv1.QueryAllPetsRequest + 78, // 237: productv1.ProductService.QueryAuthor:input_type -> productv1.QueryAuthorRequest + 80, // 238: productv1.ProductService.QueryAuthorById:input_type -> productv1.QueryAuthorByIdRequest + 82, // 239: productv1.ProductService.QueryAuthorsWithFilter:input_type -> productv1.QueryAuthorsWithFilterRequest + 70, // 240: productv1.ProductService.QueryBlogPost:input_type -> productv1.QueryBlogPostRequest + 72, // 241: productv1.ProductService.QueryBlogPostById:input_type -> productv1.QueryBlogPostByIdRequest + 74, // 242: productv1.ProductService.QueryBlogPostsWithFilter:input_type -> productv1.QueryBlogPostsWithFilterRequest + 86, // 243: productv1.ProductService.QueryBulkSearchAuthors:input_type -> productv1.QueryBulkSearchAuthorsRequest + 88, // 244: productv1.ProductService.QueryBulkSearchBlogPosts:input_type -> productv1.QueryBulkSearchBlogPostsRequest + 44, // 245: productv1.ProductService.QueryCalculateTotals:input_type -> productv1.QueryCalculateTotalsRequest + 46, // 246: productv1.ProductService.QueryCategories:input_type -> productv1.QueryCategoriesRequest + 48, // 247: productv1.ProductService.QueryCategoriesByKind:input_type -> productv1.QueryCategoriesByKindRequest + 50, // 248: productv1.ProductService.QueryCategoriesByKinds:input_type -> productv1.QueryCategoriesByKindsRequest + 42, // 249: productv1.ProductService.QueryComplexFilterType:input_type -> productv1.QueryComplexFilterTypeRequest + 52, // 250: productv1.ProductService.QueryFilterCategories:input_type -> productv1.QueryFilterCategoriesRequest + 34, // 251: productv1.ProductService.QueryNestedType:input_type -> productv1.QueryNestedTypeRequest + 62, // 252: productv1.ProductService.QueryNullableFieldsType:input_type -> productv1.QueryNullableFieldsTypeRequest + 64, // 253: productv1.ProductService.QueryNullableFieldsTypeById:input_type -> productv1.QueryNullableFieldsTypeByIdRequest + 66, // 254: productv1.ProductService.QueryNullableFieldsTypeWithFilter:input_type -> productv1.QueryNullableFieldsTypeWithFilterRequest + 54, // 255: productv1.ProductService.QueryRandomPet:input_type -> productv1.QueryRandomPetRequest + 60, // 256: productv1.ProductService.QueryRandomSearchResult:input_type -> productv1.QueryRandomSearchResultRequest + 36, // 257: productv1.ProductService.QueryRecursiveType:input_type -> productv1.QueryRecursiveTypeRequest + 58, // 258: productv1.ProductService.QuerySearch:input_type -> productv1.QuerySearchRequest + 38, // 259: productv1.ProductService.QueryTypeFilterWithArguments:input_type -> productv1.QueryTypeFilterWithArgumentsRequest + 40, // 260: productv1.ProductService.QueryTypeWithMultipleFilterFields:input_type -> productv1.QueryTypeWithMultipleFilterFieldsRequest + 32, // 261: productv1.ProductService.QueryUser:input_type -> productv1.QueryUserRequest + 30, // 262: productv1.ProductService.QueryUsers:input_type -> productv1.QueryUsersRequest + 126, // 263: productv1.ProductService.ResolveCategoryCategoryMetrics:input_type -> productv1.ResolveCategoryCategoryMetricsRequest + 121, // 264: productv1.ProductService.ResolveCategoryPopularityScore:input_type -> productv1.ResolveCategoryPopularityScoreRequest + 116, // 265: productv1.ProductService.ResolveCategoryProductCount:input_type -> productv1.ResolveCategoryProductCountRequest + 131, // 266: productv1.ProductService.ResolveSubcategoryItemCount:input_type -> productv1.ResolveSubcategoryItemCountRequest + 23, // 267: productv1.ProductService.LookupProductById:output_type -> productv1.LookupProductByIdResponse + 26, // 268: productv1.ProductService.LookupStorageById:output_type -> productv1.LookupStorageByIdResponse + 29, // 269: productv1.ProductService.LookupWarehouseById:output_type -> productv1.LookupWarehouseByIdResponse + 107, // 270: productv1.ProductService.MutationBulkCreateAuthors:output_type -> productv1.MutationBulkCreateAuthorsResponse + 111, // 271: productv1.ProductService.MutationBulkCreateBlogPosts:output_type -> productv1.MutationBulkCreateBlogPostsResponse + 109, // 272: productv1.ProductService.MutationBulkUpdateAuthors:output_type -> productv1.MutationBulkUpdateAuthorsResponse + 113, // 273: productv1.ProductService.MutationBulkUpdateBlogPosts:output_type -> productv1.MutationBulkUpdateBlogPostsResponse + 103, // 274: productv1.ProductService.MutationCreateAuthor:output_type -> productv1.MutationCreateAuthorResponse + 99, // 275: productv1.ProductService.MutationCreateBlogPost:output_type -> productv1.MutationCreateBlogPostResponse + 95, // 276: productv1.ProductService.MutationCreateNullableFieldsType:output_type -> productv1.MutationCreateNullableFieldsTypeResponse + 91, // 277: productv1.ProductService.MutationCreateUser:output_type -> productv1.MutationCreateUserResponse + 93, // 278: productv1.ProductService.MutationPerformAction:output_type -> productv1.MutationPerformActionResponse + 105, // 279: productv1.ProductService.MutationUpdateAuthor:output_type -> productv1.MutationUpdateAuthorResponse + 101, // 280: productv1.ProductService.MutationUpdateBlogPost:output_type -> productv1.MutationUpdateBlogPostResponse + 97, // 281: productv1.ProductService.MutationUpdateNullableFieldsType:output_type -> productv1.MutationUpdateNullableFieldsTypeResponse + 85, // 282: productv1.ProductService.QueryAllAuthors:output_type -> productv1.QueryAllAuthorsResponse + 77, // 283: productv1.ProductService.QueryAllBlogPosts:output_type -> productv1.QueryAllBlogPostsResponse + 69, // 284: productv1.ProductService.QueryAllNullableFieldsTypes:output_type -> productv1.QueryAllNullableFieldsTypesResponse + 57, // 285: productv1.ProductService.QueryAllPets:output_type -> productv1.QueryAllPetsResponse + 79, // 286: productv1.ProductService.QueryAuthor:output_type -> productv1.QueryAuthorResponse + 81, // 287: productv1.ProductService.QueryAuthorById:output_type -> productv1.QueryAuthorByIdResponse + 83, // 288: productv1.ProductService.QueryAuthorsWithFilter:output_type -> productv1.QueryAuthorsWithFilterResponse + 71, // 289: productv1.ProductService.QueryBlogPost:output_type -> productv1.QueryBlogPostResponse + 73, // 290: productv1.ProductService.QueryBlogPostById:output_type -> productv1.QueryBlogPostByIdResponse + 75, // 291: productv1.ProductService.QueryBlogPostsWithFilter:output_type -> productv1.QueryBlogPostsWithFilterResponse + 87, // 292: productv1.ProductService.QueryBulkSearchAuthors:output_type -> productv1.QueryBulkSearchAuthorsResponse + 89, // 293: productv1.ProductService.QueryBulkSearchBlogPosts:output_type -> productv1.QueryBulkSearchBlogPostsResponse + 45, // 294: productv1.ProductService.QueryCalculateTotals:output_type -> productv1.QueryCalculateTotalsResponse + 47, // 295: productv1.ProductService.QueryCategories:output_type -> productv1.QueryCategoriesResponse + 49, // 296: productv1.ProductService.QueryCategoriesByKind:output_type -> productv1.QueryCategoriesByKindResponse + 51, // 297: productv1.ProductService.QueryCategoriesByKinds:output_type -> productv1.QueryCategoriesByKindsResponse + 43, // 298: productv1.ProductService.QueryComplexFilterType:output_type -> productv1.QueryComplexFilterTypeResponse + 53, // 299: productv1.ProductService.QueryFilterCategories:output_type -> productv1.QueryFilterCategoriesResponse + 35, // 300: productv1.ProductService.QueryNestedType:output_type -> productv1.QueryNestedTypeResponse + 63, // 301: productv1.ProductService.QueryNullableFieldsType:output_type -> productv1.QueryNullableFieldsTypeResponse + 65, // 302: productv1.ProductService.QueryNullableFieldsTypeById:output_type -> productv1.QueryNullableFieldsTypeByIdResponse + 67, // 303: productv1.ProductService.QueryNullableFieldsTypeWithFilter:output_type -> productv1.QueryNullableFieldsTypeWithFilterResponse + 55, // 304: productv1.ProductService.QueryRandomPet:output_type -> productv1.QueryRandomPetResponse + 61, // 305: productv1.ProductService.QueryRandomSearchResult:output_type -> productv1.QueryRandomSearchResultResponse + 37, // 306: productv1.ProductService.QueryRecursiveType:output_type -> productv1.QueryRecursiveTypeResponse + 59, // 307: productv1.ProductService.QuerySearch:output_type -> productv1.QuerySearchResponse + 39, // 308: productv1.ProductService.QueryTypeFilterWithArguments:output_type -> productv1.QueryTypeFilterWithArgumentsResponse + 41, // 309: productv1.ProductService.QueryTypeWithMultipleFilterFields:output_type -> productv1.QueryTypeWithMultipleFilterFieldsResponse + 33, // 310: productv1.ProductService.QueryUser:output_type -> productv1.QueryUserResponse + 31, // 311: productv1.ProductService.QueryUsers:output_type -> productv1.QueryUsersResponse + 128, // 312: productv1.ProductService.ResolveCategoryCategoryMetrics:output_type -> productv1.ResolveCategoryCategoryMetricsResponse + 123, // 313: productv1.ProductService.ResolveCategoryPopularityScore:output_type -> productv1.ResolveCategoryPopularityScoreResponse + 118, // 314: productv1.ProductService.ResolveCategoryProductCount:output_type -> productv1.ResolveCategoryProductCountResponse + 133, // 315: productv1.ProductService.ResolveSubcategoryItemCount:output_type -> productv1.ResolveSubcategoryItemCountResponse + 267, // [267:316] is the sub-list for method output_type + 218, // [218:267] is the sub-list for method input_type + 218, // [218:218] is the sub-list for extension type_name + 218, // [218:218] is the sub-list for extension extendee + 0, // [0:218] is the sub-list for field type_name } func init() { file_product_proto_init() } @@ -10704,16 +11308,16 @@ func file_product_proto_init() { if File_product_proto != nil { return } - file_product_proto_msgTypes[137].OneofWrappers = []any{ + file_product_proto_msgTypes[147].OneofWrappers = []any{ (*Animal_Cat)(nil), (*Animal_Dog)(nil), } - file_product_proto_msgTypes[139].OneofWrappers = []any{ + file_product_proto_msgTypes[149].OneofWrappers = []any{ (*SearchResult_Product)(nil), (*SearchResult_User)(nil), (*SearchResult_Category)(nil), } - file_product_proto_msgTypes[148].OneofWrappers = []any{ + file_product_proto_msgTypes[158].OneofWrappers = []any{ (*ActionResult_ActionSuccess)(nil), (*ActionResult_ActionError)(nil), } @@ -10723,7 +11327,7 @@ func file_product_proto_init() { GoPackagePath: reflect.TypeOf(x{}).PkgPath(), RawDescriptor: unsafe.Slice(unsafe.StringData(file_product_proto_rawDesc), len(file_product_proto_rawDesc)), NumEnums: 1, - NumMessages: 186, + NumMessages: 197, NumExtensions: 0, NumServices: 1, }, diff --git a/v2/pkg/grpctest/productv1/product_grpc.pb.go b/v2/pkg/grpctest/productv1/product_grpc.pb.go index dff02dd32..8e27362a2 100644 --- a/v2/pkg/grpctest/productv1/product_grpc.pb.go +++ b/v2/pkg/grpctest/productv1/product_grpc.pb.go @@ -64,6 +64,8 @@ const ( ProductService_QueryTypeWithMultipleFilterFields_FullMethodName = "/productv1.ProductService/QueryTypeWithMultipleFilterFields" ProductService_QueryUser_FullMethodName = "/productv1.ProductService/QueryUser" ProductService_QueryUsers_FullMethodName = "/productv1.ProductService/QueryUsers" + ProductService_ResolveCategoryCategoryMetrics_FullMethodName = "/productv1.ProductService/ResolveCategoryCategoryMetrics" + ProductService_ResolveCategoryPopularityScore_FullMethodName = "/productv1.ProductService/ResolveCategoryPopularityScore" ProductService_ResolveCategoryProductCount_FullMethodName = "/productv1.ProductService/ResolveCategoryProductCount" ProductService_ResolveSubcategoryItemCount_FullMethodName = "/productv1.ProductService/ResolveSubcategoryItemCount" ) @@ -122,6 +124,8 @@ type ProductServiceClient interface { QueryTypeWithMultipleFilterFields(ctx context.Context, in *QueryTypeWithMultipleFilterFieldsRequest, opts ...grpc.CallOption) (*QueryTypeWithMultipleFilterFieldsResponse, error) QueryUser(ctx context.Context, in *QueryUserRequest, opts ...grpc.CallOption) (*QueryUserResponse, error) QueryUsers(ctx context.Context, in *QueryUsersRequest, opts ...grpc.CallOption) (*QueryUsersResponse, error) + ResolveCategoryCategoryMetrics(ctx context.Context, in *ResolveCategoryCategoryMetricsRequest, opts ...grpc.CallOption) (*ResolveCategoryCategoryMetricsResponse, error) + ResolveCategoryPopularityScore(ctx context.Context, in *ResolveCategoryPopularityScoreRequest, opts ...grpc.CallOption) (*ResolveCategoryPopularityScoreResponse, error) ResolveCategoryProductCount(ctx context.Context, in *ResolveCategoryProductCountRequest, opts ...grpc.CallOption) (*ResolveCategoryProductCountResponse, error) ResolveSubcategoryItemCount(ctx context.Context, in *ResolveSubcategoryItemCountRequest, opts ...grpc.CallOption) (*ResolveSubcategoryItemCountResponse, error) } @@ -584,6 +588,26 @@ func (c *productServiceClient) QueryUsers(ctx context.Context, in *QueryUsersReq return out, nil } +func (c *productServiceClient) ResolveCategoryCategoryMetrics(ctx context.Context, in *ResolveCategoryCategoryMetricsRequest, opts ...grpc.CallOption) (*ResolveCategoryCategoryMetricsResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(ResolveCategoryCategoryMetricsResponse) + err := c.cc.Invoke(ctx, ProductService_ResolveCategoryCategoryMetrics_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *productServiceClient) ResolveCategoryPopularityScore(ctx context.Context, in *ResolveCategoryPopularityScoreRequest, opts ...grpc.CallOption) (*ResolveCategoryPopularityScoreResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(ResolveCategoryPopularityScoreResponse) + err := c.cc.Invoke(ctx, ProductService_ResolveCategoryPopularityScore_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + func (c *productServiceClient) ResolveCategoryProductCount(ctx context.Context, in *ResolveCategoryProductCountRequest, opts ...grpc.CallOption) (*ResolveCategoryProductCountResponse, error) { cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) out := new(ResolveCategoryProductCountResponse) @@ -658,6 +682,8 @@ type ProductServiceServer interface { QueryTypeWithMultipleFilterFields(context.Context, *QueryTypeWithMultipleFilterFieldsRequest) (*QueryTypeWithMultipleFilterFieldsResponse, error) QueryUser(context.Context, *QueryUserRequest) (*QueryUserResponse, error) QueryUsers(context.Context, *QueryUsersRequest) (*QueryUsersResponse, error) + ResolveCategoryCategoryMetrics(context.Context, *ResolveCategoryCategoryMetricsRequest) (*ResolveCategoryCategoryMetricsResponse, error) + ResolveCategoryPopularityScore(context.Context, *ResolveCategoryPopularityScoreRequest) (*ResolveCategoryPopularityScoreResponse, error) ResolveCategoryProductCount(context.Context, *ResolveCategoryProductCountRequest) (*ResolveCategoryProductCountResponse, error) ResolveSubcategoryItemCount(context.Context, *ResolveSubcategoryItemCountRequest) (*ResolveSubcategoryItemCountResponse, error) mustEmbedUnimplementedProductServiceServer() @@ -805,6 +831,12 @@ func (UnimplementedProductServiceServer) QueryUser(context.Context, *QueryUserRe func (UnimplementedProductServiceServer) QueryUsers(context.Context, *QueryUsersRequest) (*QueryUsersResponse, error) { return nil, status.Errorf(codes.Unimplemented, "method QueryUsers not implemented") } +func (UnimplementedProductServiceServer) ResolveCategoryCategoryMetrics(context.Context, *ResolveCategoryCategoryMetricsRequest) (*ResolveCategoryCategoryMetricsResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method ResolveCategoryCategoryMetrics not implemented") +} +func (UnimplementedProductServiceServer) ResolveCategoryPopularityScore(context.Context, *ResolveCategoryPopularityScoreRequest) (*ResolveCategoryPopularityScoreResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method ResolveCategoryPopularityScore not implemented") +} func (UnimplementedProductServiceServer) ResolveCategoryProductCount(context.Context, *ResolveCategoryProductCountRequest) (*ResolveCategoryProductCountResponse, error) { return nil, status.Errorf(codes.Unimplemented, "method ResolveCategoryProductCount not implemented") } @@ -1642,6 +1674,42 @@ func _ProductService_QueryUsers_Handler(srv interface{}, ctx context.Context, de return interceptor(ctx, in, info, handler) } +func _ProductService_ResolveCategoryCategoryMetrics_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(ResolveCategoryCategoryMetricsRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(ProductServiceServer).ResolveCategoryCategoryMetrics(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: ProductService_ResolveCategoryCategoryMetrics_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(ProductServiceServer).ResolveCategoryCategoryMetrics(ctx, req.(*ResolveCategoryCategoryMetricsRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _ProductService_ResolveCategoryPopularityScore_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(ResolveCategoryPopularityScoreRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(ProductServiceServer).ResolveCategoryPopularityScore(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: ProductService_ResolveCategoryPopularityScore_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(ProductServiceServer).ResolveCategoryPopularityScore(ctx, req.(*ResolveCategoryPopularityScoreRequest)) + } + return interceptor(ctx, in, info, handler) +} + func _ProductService_ResolveCategoryProductCount_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { in := new(ResolveCategoryProductCountRequest) if err := dec(in); err != nil { @@ -1865,6 +1933,14 @@ var ProductService_ServiceDesc = grpc.ServiceDesc{ MethodName: "QueryUsers", Handler: _ProductService_QueryUsers_Handler, }, + { + MethodName: "ResolveCategoryCategoryMetrics", + Handler: _ProductService_ResolveCategoryCategoryMetrics_Handler, + }, + { + MethodName: "ResolveCategoryPopularityScore", + Handler: _ProductService_ResolveCategoryPopularityScore_Handler, + }, { MethodName: "ResolveCategoryProductCount", Handler: _ProductService_ResolveCategoryProductCount_Handler, diff --git a/v2/pkg/grpctest/testdata/products.graphqls b/v2/pkg/grpctest/testdata/products.graphqls index 19f11c258..d7555b5d9 100644 --- a/v2/pkg/grpctest/testdata/products.graphqls +++ b/v2/pkg/grpctest/testdata/products.graphqls @@ -118,6 +118,8 @@ type Category { kind: CategoryKind! productCount(filters: ProductCountFilter): Int! @resolved(context: "id name") subcategories: [Subcategory!] + popularityScore(threshold: Int): Int @resolved(context: "id") + categoryMetrics(metricType: String!): CategoryMetrics @resolved(context: "id name") } type Subcategory { @@ -128,6 +130,14 @@ type Subcategory { itemCount(filters: SubcategoryItemFilter): Int! @resolved(context: "id") } +type CategoryMetrics { + id: ID! + metricType: String! + value: Float! + timestamp: String! + categoryId: ID! +} + enum CategoryKind { BOOK ELECTRONICS From 28ab022634a642fac9fc762e5572e9598f5f2664 Mon Sep 17 00:00:00 2001 From: Ludwig Bedacht Date: Thu, 9 Oct 2025 14:44:06 +0200 Subject: [PATCH 06/72] chore: add tests for nullable and message types --- .../grpc_datasource/grpc_datasource_test.go | 138 ++++++++++++------ v2/pkg/grpctest/mockservice.go | 2 +- 2 files changed, 97 insertions(+), 43 deletions(-) diff --git a/v2/pkg/engine/datasource/grpc_datasource/grpc_datasource_test.go b/v2/pkg/engine/datasource/grpc_datasource/grpc_datasource_test.go index 2f56967fd..67a2e46ea 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/grpc_datasource_test.go +++ b/v2/pkg/engine/datasource/grpc_datasource/grpc_datasource_test.go @@ -3657,48 +3657,48 @@ func Test_Datasource_Load_WithFieldResolvers(t *testing.T) { validate func(t *testing.T, data map[string]interface{}) validateError func(t *testing.T, errData []graphqlError) }{ - // { - // name: "Query with field resolvers", - // query: `query CategoriesWithFieldResolvers($filters: ProductCountFilter) { categories { id name kind productCount(filters: $filters) } }`, - // vars: `{"variables":{"filters":{"minPrice":100}}}`, - // validate: func(t *testing.T, data map[string]interface{}) { - // require.NotEmpty(t, data) - - // categories, ok := data["categories"].([]interface{}) - // require.True(t, ok, "categories should be an array") - // require.NotEmpty(t, categories, "categories should not be empty") - // require.Len(t, categories, 4, "Should return 1 category") - - // for productCount, category := range categories { - // category, ok := category.(map[string]interface{}) - // require.True(t, ok, "category should be an object") - // require.NotEmpty(t, category["id"]) - // require.NotEmpty(t, category["name"]) - // require.NotEmpty(t, category["kind"]) - // require.Equal(t, float64(productCount), category["productCount"]) - // } - - // }, - // validateError: func(t *testing.T, errData []graphqlError) { - // require.Empty(t, errData) - // }, - // }, - // { - // name: "Query with field resolvers and nullable lists", - // query: "query SubcategoriesWithFieldResolvers($filter: SubcategoryItemFilter) { categories { id subcategories { id name description isActive itemCount(filters: $filter) } } }", - // vars: `{"variables":{"filter":{"isActive":true}}}`, - // validate: func(t *testing.T, data map[string]interface{}) { - // require.NotEmpty(t, data) - - // categories, ok := data["categories"].([]interface{}) - // require.True(t, ok, "categories should be an array") - // require.NotEmpty(t, categories, "categories should not be empty") - // require.Len(t, categories, 4, "Should return 1 category") - // }, - // validateError: func(t *testing.T, errData []graphqlError) { - // require.Empty(t, errData) - // }, - // }, + { + name: "Query with field resolvers", + query: `query CategoriesWithFieldResolvers($filters: ProductCountFilter) { categories { id name kind productCount(filters: $filters) } }`, + vars: `{"variables":{"filters":{"minPrice":100}}}`, + validate: func(t *testing.T, data map[string]interface{}) { + require.NotEmpty(t, data) + + categories, ok := data["categories"].([]interface{}) + require.True(t, ok, "categories should be an array") + require.NotEmpty(t, categories, "categories should not be empty") + require.Len(t, categories, 4, "Should return 1 category") + + for productCount, category := range categories { + category, ok := category.(map[string]interface{}) + require.True(t, ok, "category should be an object") + require.NotEmpty(t, category["id"]) + require.NotEmpty(t, category["name"]) + require.NotEmpty(t, category["kind"]) + require.Equal(t, float64(productCount), category["productCount"]) + } + + }, + validateError: func(t *testing.T, errData []graphqlError) { + require.Empty(t, errData) + }, + }, + { + name: "Query with field resolvers and nullable lists", + query: "query SubcategoriesWithFieldResolvers($filter: SubcategoryItemFilter) { categories { id subcategories { id name description isActive itemCount(filters: $filter) } } }", + vars: `{"variables":{"filter":{"isActive":true}}}`, + validate: func(t *testing.T, data map[string]interface{}) { + require.NotEmpty(t, data) + + categories, ok := data["categories"].([]interface{}) + require.True(t, ok, "categories should be an array") + require.NotEmpty(t, categories, "categories should not be empty") + require.Len(t, categories, 4, "Should return 1 category") + }, + validateError: func(t *testing.T, errData []graphqlError) { + require.Empty(t, errData) + }, + }, { name: "Query with field resolvers and aliases", query: "query CategoriesWithFieldResolversAndAliases($filter1: ProductCountFilter, $filter2: ProductCountFilter) { categories { productCount1: productCount(filters: $filter1) productCount2: productCount(filters: $filter2) } }", @@ -3723,6 +3723,60 @@ func Test_Datasource_Load_WithFieldResolvers(t *testing.T) { require.Empty(t, errData) }, }, + { + name: "Query with field resolvers and message type", + query: "query CategoriesWithNullableTypes($nullType: String, $valueType: String) { categories { nullMetrics: categoryMetrics(metricType: $nullType) { id metricType value } valueMetrics: categoryMetrics(metricType: $valueType) { id metricType value } } }", + vars: `{"variables":{"nullType":"unavailable","valueType":"popularity_score"}}`, + validate: func(t *testing.T, data map[string]interface{}) { + require.NotEmpty(t, data) + + categories, ok := data["categories"].([]interface{}) + require.True(t, ok, "categories should be an array") + require.NotEmpty(t, categories, "categories should not be empty") + require.Len(t, categories, 4, "Should return 1 category") + + for _, category := range categories { + category, ok := category.(map[string]interface{}) + require.True(t, ok, "category should be an object") + require.NotEmpty(t, category["valueMetrics"]) + valueMetrics, ok := category["valueMetrics"].(map[string]interface{}) + require.True(t, ok, "categoryMetrics should be an object") + require.NotEmpty(t, valueMetrics, "valueMetrics should not be empty") + require.Len(t, valueMetrics, 3, "Should return 1 valueMetrics") + require.NotEmpty(t, valueMetrics["id"]) + require.NotEmpty(t, valueMetrics["metricType"]) + require.NotEmpty(t, valueMetrics["value"]) + + require.Empty(t, category["nullMetrics"], "nullMetrics should be empty") + } + }, + validateError: func(t *testing.T, errData []graphqlError) { + require.Empty(t, errData) + }, + }, + { + name: "Query with field resolvers and null fields", + query: "query CategoriesWithNullableTypes($threshold1: Int, $threshold2: Int) { categories { nullScore: popularityScore(threshold: $threshold1) valueScore: popularityScore(threshold: $threshold2) } }", + vars: `{"variables":{"threshold1":100, "threshold2":50}}`, // Threshold above 50 should return null + validate: func(t *testing.T, data map[string]interface{}) { + require.NotEmpty(t, data) + + categories, ok := data["categories"].([]interface{}) + require.True(t, ok, "categories should be an array") + require.NotEmpty(t, categories, "categories should not be empty") + require.Len(t, categories, 4, "Should return 1 category") + + for _, category := range categories { + category, ok := category.(map[string]interface{}) + require.True(t, ok, "category should be an object") + require.NotEmpty(t, category["valueScore"]) + require.Empty(t, category["nullScore"]) + } + }, + validateError: func(t *testing.T, errData []graphqlError) { + require.Empty(t, errData) + }, + }, } for _, tc := range testCases { diff --git a/v2/pkg/grpctest/mockservice.go b/v2/pkg/grpctest/mockservice.go index 682db49a7..992089b5c 100644 --- a/v2/pkg/grpctest/mockservice.go +++ b/v2/pkg/grpctest/mockservice.go @@ -63,7 +63,7 @@ func (s *MockService) ResolveCategoryPopularityScore(_ context.Context, req *pro baseScore := 50 for range req.GetContext() { - if baseScore < int(threshold.GetValue()) { + if int(threshold.GetValue()) > baseScore { results = append(results, &productv1.ResolveCategoryPopularityScoreResult{ PopularityScore: nil, }) From ed61f4623ad931d09854dc3c95f0acd6dfc9ec1a Mon Sep 17 00:00:00 2001 From: Ludwig Bedacht Date: Thu, 9 Oct 2025 14:45:30 +0200 Subject: [PATCH 07/72] chore: rename function --- v2/pkg/ast/path.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/v2/pkg/ast/path.go b/v2/pkg/ast/path.go index 28f66fa7a..19493ec60 100644 --- a/v2/pkg/ast/path.go +++ b/v2/pkg/ast/path.go @@ -81,7 +81,7 @@ func (p Path) WithoutInlineFragmentNames() Path { return out } -func (p Path) WithPathElement(element PathItem) Path { +func (p Path) WithPathItem(element PathItem) Path { res := make(Path, len(p)+1) copy(res, p) res[len(res)-1] = element @@ -89,7 +89,7 @@ func (p Path) WithPathElement(element PathItem) Path { } func (p Path) WithFieldNameItem(fieldName []byte) Path { - return p.WithPathElement(PathItem{ + return p.WithPathItem(PathItem{ Kind: FieldName, FieldName: fieldName, }) From a0f6523d4761e56335a7ff734b5743ad0e77522e Mon Sep 17 00:00:00 2001 From: Ludwig Bedacht Date: Fri, 10 Oct 2025 12:45:06 +0200 Subject: [PATCH 08/72] chore: rename directive --- .../datasource/grpc_datasource/compiler.go | 4 - .../grpc_datasource/execution_plan.go | 5 +- .../grpc_datasource/grpc_datasource_test.go | 80 ++++++++++++++++++- v2/pkg/grpctest/schema.go | 4 +- v2/pkg/grpctest/testdata/products.graphqls | 10 +-- 5 files changed, 90 insertions(+), 13 deletions(-) diff --git a/v2/pkg/engine/datasource/grpc_datasource/compiler.go b/v2/pkg/engine/datasource/grpc_datasource/compiler.go index 76f061f51..6a5cdda75 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/compiler.go +++ b/v2/pkg/engine/datasource/grpc_datasource/compiler.go @@ -592,10 +592,6 @@ func (p *RPCCompiler) newEmptyMessage(message Message) *dynamicpb.Message { // repeated CategoryProductCountContext context = 1; // CategoryProductCountArgs field_args = 2; // } -// -// message ResolveCategoryProductCountRequestKey { - -// } func (p *RPCCompiler) buildProtoMessageWithContext(inputMessage Message, rpcMessage *RPCMessage, data gjson.Result, context []FetchItem) *dynamicpb.Message { if rpcMessage == nil { return nil diff --git a/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go b/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go index 5471627c0..2651acf4b 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go +++ b/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go @@ -14,6 +14,9 @@ const ( // knownTypeOptionalFieldValueName is the name of the field that is used to wrap optional scalar values // in a message as protobuf scalar types are not nullable. knownTypeOptionalFieldValueName = "value" + + // resolverContextDirectiveName is the name of the directive that is used to configure the resolver context. + resolverContextDirectiveName = "configureResolver" ) // OneOfType represents the type of a oneof field in a protobuf message. @@ -709,7 +712,7 @@ func (r *rpcPlanningContext) resolveServiceName(subgraphName string) string { } func (r *rpcPlanningContext) resolveContextFields(walker *astvisitor.Walker, fd int) ([]int, error) { - contextDirectiveRef, exists := r.definition.FieldDefinitionDirectiveByName(fd, ast.ByteSlice("resolved")) + contextDirectiveRef, exists := r.definition.FieldDefinitionDirectiveByName(fd, ast.ByteSlice(resolverContextDirectiveName)) if exists { fields, err := r.getFieldsFromContext(walker.EnclosingTypeDefinition, contextDirectiveRef) if err != nil { diff --git a/v2/pkg/engine/datasource/grpc_datasource/grpc_datasource_test.go b/v2/pkg/engine/datasource/grpc_datasource/grpc_datasource_test.go index 67a2e46ea..5b682f7d7 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/grpc_datasource_test.go +++ b/v2/pkg/engine/datasource/grpc_datasource/grpc_datasource_test.go @@ -27,6 +27,79 @@ import ( "github.com/wundergraph/graphql-go-tools/v2/pkg/grpctest/productv1" ) +func Benchmark_DataSource_Load(b *testing.B) { + conn, cleanup := setupTestGRPCServer(b) + b.Cleanup(cleanup) + + schemaDoc := grpctest.MustGraphQLSchema(b) + + query := `query ComplexFilterTypeQuery($filter: ComplexFilterTypeInput!) { complexFilterType(filter: $filter) { id name } }` + variables := `{"variables":{"filter":{"name":"test","filterField1":"test","filterField2":"test"}}}` + + // Parse the GraphQL query + queryDoc, report := astparser.ParseGraphqlDocumentString(query) + if report.HasErrors() { + b.Fatalf("failed to parse query: %s", report.Error()) + } + + compiler, err := NewProtoCompiler(grpctest.MustProtoSchema(b), testMapping()) + require.NoError(b, err) + + ds, err := NewDataSource(conn, DataSourceConfig{ + Operation: &queryDoc, + Definition: &schemaDoc, + SubgraphName: "Products", + Compiler: compiler, + Mapping: testMapping(), + }) + require.NoError(b, err) + + b.ReportAllocs() + b.ResetTimer() + for b.Loop() { + output := new(bytes.Buffer) + err = ds.Load(context.Background(), []byte(`{"query":"`+query+`","body":`+variables+`}`), output) + require.NoError(b, err) + } +} + +func Benchmark_DataSource_Load_WithFieldArguments(b *testing.B) { + conn, cleanup := setupTestGRPCServer(b) + b.Cleanup(cleanup) + + schemaDoc := grpctest.MustGraphQLSchema(b) + + query := `query CategoriesWithNullableTypes($nullType: String, $valueType: String) { categories { nullMetrics: categoryMetrics(metricType: $nullType) { id metricType value } valueMetrics: categoryMetrics(metricType: $valueType) { id metricType value } } }` + variables := `{"variables":{"nullType":"unavailable","valueType":"popularity_score"}}` + + // Parse the GraphQL query + queryDoc, report := astparser.ParseGraphqlDocumentString(query) + if report.HasErrors() { + b.Fatalf("failed to parse query: %s", report.Error()) + } + + compiler, err := NewProtoCompiler(grpctest.MustProtoSchema(b), testMapping()) + require.NoError(b, err) + + b.ReportAllocs() + b.ResetTimer() + + for b.Loop() { + ds, err := NewDataSource(conn, DataSourceConfig{ + Operation: &queryDoc, + Definition: &schemaDoc, + SubgraphName: "Products", + Compiler: compiler, + Mapping: testMapping(), + }) + require.NoError(b, err) + + output := new(bytes.Buffer) + err = ds.Load(context.Background(), []byte(`{"query":"`+query+`","body":`+variables+`}`), output) + require.NoError(b, err) + } +} + // mockInterface provides a simple implementation of grpc.ClientConnInterface for testing type mockInterface struct { } @@ -58,7 +131,7 @@ func (m mockInterface) NewStream(ctx context.Context, desc *grpc.StreamDesc, met var _ grpc.ClientConnInterface = (*mockInterface)(nil) -func setupTestGRPCServer(t *testing.T) (conn *grpc.ClientConn, cleanup func()) { +func setupTestGRPCServer(t testing.TB) (conn *grpc.ClientConn, cleanup func()) { t.Helper() // Set up the bufconn listener @@ -3812,6 +3885,11 @@ func Test_Datasource_Load_WithFieldResolvers(t *testing.T) { err = ds.Load(context.Background(), []byte(input), output) require.NoError(t, err) + pretty := new(bytes.Buffer) + err = json.Indent(pretty, output.Bytes(), "", " ") + require.NoError(t, err) + fmt.Println(pretty.String()) + // Parse the response var resp graphqlResponse diff --git a/v2/pkg/grpctest/schema.go b/v2/pkg/grpctest/schema.go index 72c74e381..163b188f3 100644 --- a/v2/pkg/grpctest/schema.go +++ b/v2/pkg/grpctest/schema.go @@ -72,7 +72,7 @@ func GraphQLSchemaWithoutBaseDefinitions() (ast.Document, error) { return doc, nil } -func MustGraphQLSchema(t *testing.T) ast.Document { +func MustGraphQLSchema(t testing.TB) ast.Document { schemaBytes, err := getSchemaBytes() require.NoError(t, err) doc := unsafeparser.ParseGraphqlDocumentStringWithBaseSchema(string(schemaBytes)) @@ -95,7 +95,7 @@ func ProtoSchema() (string, error) { return string(protoBytes), nil } -func MustProtoSchema(t *testing.T) string { +func MustProtoSchema(t testing.TB) string { schema, err := ProtoSchema() require.NoError(t, err) return schema diff --git a/v2/pkg/grpctest/testdata/products.graphqls b/v2/pkg/grpctest/testdata/products.graphqls index d7555b5d9..43547d36a 100644 --- a/v2/pkg/grpctest/testdata/products.graphqls +++ b/v2/pkg/grpctest/testdata/products.graphqls @@ -1,5 +1,5 @@ # Directive for gRPC field resolvers -directive @resolved(context: openfed__FieldSet!) on FIELD_DEFINITION +directive @configureResolver(context: openfed__FieldSet!) on FIELD_DEFINITION type Product @key(fields: "id") { id: ID! @@ -116,10 +116,10 @@ type Category { id: ID! name: String! kind: CategoryKind! - productCount(filters: ProductCountFilter): Int! @resolved(context: "id name") + productCount(filters: ProductCountFilter): Int! @configureResolver(context: "id name") subcategories: [Subcategory!] - popularityScore(threshold: Int): Int @resolved(context: "id") - categoryMetrics(metricType: String!): CategoryMetrics @resolved(context: "id name") + popularityScore(threshold: Int): Int @configureResolver(context: "id") + categoryMetrics(metricType: String!): CategoryMetrics @configureResolver(context: "id name") } type Subcategory { @@ -127,7 +127,7 @@ type Subcategory { name: String! description: String isActive: Boolean! - itemCount(filters: SubcategoryItemFilter): Int! @resolved(context: "id") + itemCount(filters: SubcategoryItemFilter): Int! @configureResolver(context: "id") } type CategoryMetrics { From 598aafe54c1add9be95557cafe78414420b1845b Mon Sep 17 00:00:00 2001 From: Ludwig Bedacht Date: Wed, 15 Oct 2025 16:02:20 +0200 Subject: [PATCH 09/72] feat: add field resolver support for entity calls --- .../grpc_datasource/execution_plan.go | 184 +- .../execution_plan_federation_test.go | 368 +- .../grpc_datasource/execution_plan_visitor.go | 111 +- .../execution_plan_visitor_federation.go | 90 +- .../grpc_datasource/grpc_datasource.go | 17 +- .../grpc_datasource/grpc_datasource_test.go | 47 + .../grpc_datasource/mapping_test_helper.go | 53 +- v2/pkg/grpctest/Makefile | 26 +- v2/pkg/grpctest/Readme.md | 160 + v2/pkg/grpctest/cmd/mapping_helper/main.go | 122 +- .../templates/grpctest_mapping.tmpl | 116 + v2/pkg/grpctest/mapping/mapping.go | 52 + v2/pkg/grpctest/mockservice.go | 89 + v2/pkg/grpctest/product.proto | 64 + v2/pkg/grpctest/productv1/product.pb.go | 2436 ++-- v2/pkg/grpctest/productv1/product_grpc.pb.go | 76 + .../grpctest/productv1/testdata/service.pb.go | 12002 ++++++++++++++++ .../productv1/testdata/service_grpc.pb.go | 2031 +++ v2/pkg/grpctest/testdata/products.graphqls | 15 +- 19 files changed, 16709 insertions(+), 1350 deletions(-) create mode 100644 v2/pkg/grpctest/Readme.md create mode 100644 v2/pkg/grpctest/cmd/mapping_helper/templates/grpctest_mapping.tmpl create mode 100644 v2/pkg/grpctest/productv1/testdata/service.pb.go create mode 100644 v2/pkg/grpctest/productv1/testdata/service_grpc.pb.go diff --git a/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go b/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go index 2651acf4b..ae8e8714a 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go +++ b/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go @@ -711,6 +711,43 @@ func (r *rpcPlanningContext) resolveServiceName(subgraphName string) string { return r.mapping.Service } +type resolvedField struct { + callerRef int + parentTypeRef int + fieldRef int + fieldDefinitionTypeRef int + requiredFields string + responsePath ast.Path + + contextFields []contextField + fieldArguments []fieldArgument +} + +func (r *rpcPlanningContext) setResolvedField(walker *astvisitor.Walker, fd int, fieldArgs []int, fieldPath ast.Path, resolvedField *resolvedField) error { + contextFields, err := r.resolveContextFields(walker, fd) + if err != nil { + return err + } + + for _, contextFieldRef := range contextFields { + contextFieldName := r.definition.FieldDefinitionNameBytes(contextFieldRef) + resolvedPath := fieldPath.WithFieldNameItem(contextFieldName) + + resolvedField.contextFields = append(resolvedField.contextFields, contextField{ + fieldRef: contextFieldRef, + resolvePath: resolvedPath, + }) + } + + fieldArguments, err := r.parseFieldArguments(walker, fd, fieldArgs) + if err != nil { + return err + } + + resolvedField.fieldArguments = fieldArguments + return nil +} + func (r *rpcPlanningContext) resolveContextFields(walker *astvisitor.Walker, fd int) ([]int, error) { contextDirectiveRef, exists := r.definition.FieldDefinitionDirectiveByName(fd, ast.ByteSlice(resolverContextDirectiveName)) if exists { @@ -726,6 +763,41 @@ func (r *rpcPlanningContext) resolveContextFields(walker *astvisitor.Walker, fd return []int{idFieldRef}, err } +func (r *rpcPlanningContext) parseFieldArguments(walker *astvisitor.Walker, fd int, fieldArgs []int) ([]fieldArgument, error) { + result := make([]fieldArgument, 0, len(fieldArgs)) + for _, fieldArgRef := range fieldArgs { + arg := r.operation.Arguments[fieldArgRef] + fieldArg := r.operation.ArgumentNameString(fieldArgRef) + fieldType := arg.Value.Kind + + argDefRef := r.definition.NodeFieldDefinitionArgumentDefinitionByName( + walker.EnclosingTypeDefinition, + r.definition.FieldDefinitionNameBytes(fd), + r.operation.ArgumentNameBytes(fieldArgRef), + ) + + if argDefRef == ast.InvalidRef { + return nil, fmt.Errorf("unable to resolve argument input value definition for argument %s", fieldArg) + } + + jsonValue := fieldArg + if fieldType == ast.ValueKindVariable { + jsonValue = r.operation.Input.ByteSliceString(r.operation.VariableValues[arg.Value.Ref].Name) + } + + result = append(result, fieldArgument{ + fieldDefinitionRef: fd, + argumentDefinitionRef: argDefRef, + parentTypeNode: walker.EnclosingTypeDefinition, + jsonPath: jsonValue, + }) + + } + + return result, nil + +} + func (r *rpcPlanningContext) getFieldsFromContext(parentNode ast.Node, contextRef int) ([]int, error) { val, exists := r.definition.DirectiveArgumentValueByName(contextRef, []byte("context")) if !exists { @@ -784,41 +856,6 @@ func (r *rpcPlanningContext) filterIDFieldsFunc(o ast.ObjectTypeDefinition, fd i } } -func (r *rpcPlanningContext) parseFieldArguments(walker *astvisitor.Walker, fd int, fieldArgs []int) ([]fieldArgument, error) { - result := make([]fieldArgument, 0, len(fieldArgs)) - for _, fieldArgRef := range fieldArgs { - arg := r.operation.Arguments[fieldArgRef] - fieldArg := r.operation.ArgumentNameString(fieldArgRef) - fieldType := arg.Value.Kind - - argDefRef := r.definition.NodeFieldDefinitionArgumentDefinitionByName( - walker.EnclosingTypeDefinition, - r.definition.FieldDefinitionNameBytes(fd), - r.operation.ArgumentNameBytes(fieldArgRef), - ) - - if argDefRef == ast.InvalidRef { - return nil, fmt.Errorf("unable to resolve argument input value definition for argument %s", fieldArg) - } - - jsonValue := fieldArg - if fieldType == ast.ValueKindVariable { - jsonValue = r.operation.Input.ByteSliceString(r.operation.VariableValues[arg.Value.Ref].Name) - } - - result = append(result, fieldArgument{ - fieldDefinitionRef: fd, - argumentDefinitionRef: argDefRef, - parentTypeNode: walker.EnclosingTypeDefinition, - jsonPath: jsonValue, - }) - - } - - return result, nil - -} - // nodeByTypeRef is a helper function to resolve the underlying type node for a given type reference. func (r *rpcPlanningContext) nodeByTypeRef(typeRef int) (ast.Node, bool) { underlyingTypeName := r.definition.ResolveTypeNameString(typeRef) @@ -849,6 +886,83 @@ func (r *rpcPlanningContext) resolveRequiredFields(typeName, requiredFields stri return message, nil } +// createResolverRPCCalls creates a new call for each resolved field. +func (r *rpcPlanningContext) createResolverRPCCalls(subgraphName string, resolvedFields []resolvedField) ([]RPCCall, error) { + // We need to create a new call for each resolved field. + calls := make([]RPCCall, 0, len(resolvedFields)) + + for _, resolvedField := range resolvedFields { + resolveConfig, exists := r.mapping.FindResolveTypeFieldMapping( + r.definition.ObjectTypeDefinitionNameString(resolvedField.parentTypeRef), + r.operation.FieldNameString(resolvedField.fieldRef), + ) + + if !exists { + return nil, fmt.Errorf("resolve config not found for type: %s, field: %s", r.definition.ResolveTypeNameString(resolvedField.parentTypeRef), r.operation.FieldAliasString(resolvedField.fieldRef)) + } + + contextMessage := &RPCMessage{ + Name: resolveConfig.RPC + "Context", + } + + fieldArgsMessage := &RPCMessage{ + Name: resolveConfig.RPC + "Args", + } + + call, err := r.newResolveRPCCall(resolveRPCCallConfig{ + serviceName: subgraphName, + typeName: r.definition.ResolveTypeNameString(resolvedField.parentTypeRef), + fieldName: r.operation.FieldAliasOrNameString(resolvedField.fieldRef), + resolveConfig: resolveConfig, + resolvedField: resolvedField, + contextMessage: contextMessage, + fieldArgsMessage: fieldArgsMessage, + }) + + if err != nil { + return nil, err + } + + contextMessage.Fields = make(RPCFields, 0, len(resolvedField.contextFields)) + for _, contextField := range resolvedField.contextFields { + typeDefNode, found := r.definition.NodeByNameStr(r.definition.ResolveTypeNameString(resolvedField.parentTypeRef)) + if !found { + return nil, fmt.Errorf("type definition node not found for type: %s", r.definition.ResolveTypeNameString(resolvedField.parentTypeRef)) + } + + field, err := r.buildField( + typeDefNode, + contextField.fieldRef, + r.definition.FieldDefinitionNameString(contextField.fieldRef), + "", + ) + + field.ResolvePath = contextField.resolvePath + + if err != nil { + return nil, err + } + + contextMessage.Fields = append(contextMessage.Fields, field) + } + + fieldArgsMessage.Fields = make(RPCFields, 0, len(resolvedField.fieldArguments)) + for _, fieldArgument := range resolvedField.fieldArguments { + field, err := r.createRPCFieldFromFieldArgument(fieldArgument) + + if err != nil { + return nil, err + } + + fieldArgsMessage.Fields = append(fieldArgsMessage.Fields, field) + } + + calls = append(calls, call) + } + + return calls, nil +} + func (r *rpcPlanningContext) newResolveRPCCall(config resolveRPCCallConfig) (RPCCall, error) { resolveConfig := config.resolveConfig resolvedField := config.resolvedField diff --git a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_federation_test.go b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_federation_test.go index b87a14694..7e636d0ab 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_federation_test.go +++ b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_federation_test.go @@ -54,6 +54,7 @@ func TestEntityLookup(t *testing.T) { { ServiceName: "Products", MethodName: "LookupProductById", + Kind: CallKindEntity, // Define the structure of the request message Request: RPCMessage{ Name: "LookupProductByIdRequest", @@ -138,6 +139,7 @@ func TestEntityLookup(t *testing.T) { { ServiceName: "Products", MethodName: "LookupProductById", + Kind: CallKindEntity, Request: RPCMessage{ Name: "LookupProductByIdRequest", Fields: []RPCField{ @@ -202,6 +204,7 @@ func TestEntityLookup(t *testing.T) { { ServiceName: "Products", MethodName: "LookupStorageById", + Kind: CallKindEntity, Request: RPCMessage{ Name: "LookupStorageByIdRequest", Fields: []RPCField{ @@ -265,195 +268,179 @@ func TestEntityLookup(t *testing.T) { }, }, }, + { + name: "Should create an execution plan for an entity lookup with a field resolver", + query: `query EntityLookup($representations: [_Any!]!, $input: ShippingEstimateInput!) { _entities(representations: $representations) { ... on Product { __typename id name price shippingEstimate(input: $input) } } }`, + mapping: testMapping(), + federationConfigs: plan.FederationFieldConfigurations{ + { + TypeName: "Product", + SelectionSet: "id", + }, + }, + expectedPlan: &RPCExecutionPlan{ + Calls: []RPCCall{ + { + ServiceName: "Products", + MethodName: "LookupProductById", + Kind: CallKindEntity, + Request: RPCMessage{ + Name: "LookupProductByIdRequest", + Fields: []RPCField{ + { + Name: "keys", + TypeName: string(DataTypeMessage), + Repeated: true, + JSONPath: "representations", + Message: &RPCMessage{ + Name: "LookupProductByIdKey", + MemberTypes: []string{"Product"}, + Fields: []RPCField{ + { + Name: "id", + TypeName: string(DataTypeString), + JSONPath: "id", + }, + }, + }, + }, + }, + }, + Response: RPCMessage{ + Name: "LookupProductByIdResponse", + Fields: []RPCField{ + { + Name: "result", + TypeName: string(DataTypeMessage), + Repeated: true, + JSONPath: "_entities", + Message: &RPCMessage{ + Name: "Product", + Fields: []RPCField{ + { + Name: "__typename", + TypeName: string(DataTypeString), + JSONPath: "__typename", + StaticValue: "Product", + }, + { + Name: "id", + TypeName: string(DataTypeString), + JSONPath: "id", + }, + { + Name: "name", + TypeName: string(DataTypeString), + JSONPath: "name", + }, - // TODO implement multiple entity lookup types - // { - // name: "Should create an execution plan for an entity lookup multiple types", - // query: ` - // query EntityLookup($representations: [_Any!]!) { - // _entities(representations: $representations) { - // ... on Product { - // id - // name - // price - // } - // ... on Storage { - // id - // name - // location - // } - // } - // } - // `, - // expectedPlan: &RPCExecutionPlan{ - // Groups: []RPCCallGroup{ - // { - // Calls: []RPCCall{ - // { - // ServiceName: "Products", - // MethodName: "LookupProductById", - // // Define the structure of the request message - // Request: RPCMessage{ - // Name: "LookupProductByIdRequest", - // Fields: []RPCField{ - // { - // Name: "inputs", - // TypeName: string(DataTypeMessage), - // Repeated: true, - // JSONPath: "representations", // Path to extract data from GraphQL variables - // - - // Message: &RPCMessage{ - // Name: "LookupProductByIdInput", - // Fields: []RPCField{ - // { - // Name: "key", - // TypeName: string(DataTypeMessage), - // - - // Message: &RPCMessage{ - // Name: "ProductByIdKey", - // Fields: []RPCField{ - // { - // Name: "id", - // TypeName: string(DataTypeString), - // JSONPath: "id", // Extract 'id' from each representation - // - - // }, - // }, - // }, - // }, - // }, - // }, - // }, - // }, - // }, - // // Define the structure of the response message - // Response: RPCMessage{ - // Name: "LookupProductByIdResponse", - // Fields: []RPCField{ - // { - // Name: "results", - // TypeName: string(DataTypeMessage), - // Repeated: true, - // - - // JSONPath: "results", - // Message: &RPCMessage{ - // Name: "LookupProductByIdResult", - // Fields: []RPCField{ - // { - // Name: "product", - // TypeName: string(DataTypeMessage), - // - - // Message: &RPCMessage{ - // Name: "Product", - // Fields: []RPCField{ - // { - // Name: "id", - // TypeName: string(DataTypeString), - // JSONPath: "id", - // }, - // { - // Name: "name", - // TypeName: string(DataTypeString), - // JSONPath: "name", - // }, - // { - // Name: "price", - // TypeName: string(DataTypeFloat), - // JSONPath: "price", - // }, - // }, - // }, - // }, - // }, - // }, - // }, - // }, - // }, - // }, - // { - // ServiceName: "Products", - // MethodName: "LookupStorageById", - // Request: RPCMessage{ - // Name: "LookupStorageByIdRequest", - // Fields: []RPCField{ - // { - // Name: "inputs", - // TypeName: string(DataTypeMessage), - // Repeated: true, - // JSONPath: "representations", - // Message: &RPCMessage{ - // Name: "LookupStorageByIdInput", - // Fields: []RPCField{ - // { - // Name: "key", - // TypeName: string(DataTypeMessage), - // Message: &RPCMessage{ - // Name: "StorageByIdKey", - // Fields: []RPCField{ - // { - // Name: "id", - // TypeName: string(DataTypeString), - // JSONPath: "id", - // }, - // }, - // }, - // }, - // }, - // }, - // }, - // }, - // }, - // Response: RPCMessage{ - // Name: "LookupStorageByIdResponse", - // Fields: []RPCField{ - // { - // Name: "results", - // TypeName: string(DataTypeMessage), - // Repeated: true, - // JSONPath: "results", - // Message: &RPCMessage{ - // Name: "LookupStorageByIdResult", - // Fields: []RPCField{ - // { - // Name: "storage", - // TypeName: string(DataTypeMessage), - // Message: &RPCMessage{ - // Name: "Storage", - // Fields: []RPCField{ - // { - // Name: "id", - // TypeName: string(DataTypeString), - // JSONPath: "id", - // }, - // { - // Name: "name", - // TypeName: string(DataTypeString), - // JSONPath: "name", - // }, - // { - // Name: "location", - // TypeName: string(DataTypeString), - // JSONPath: "location", - // }, - // }, - // }, - // }, - // }, - // }, - // }, - // }, - // }, - // }, - // }, - // }, - // }, - // }, - // }, + { + Name: "price", + TypeName: string(DataTypeDouble), + JSONPath: "price", + }, + }, + }, + }, + }, + }, + }, + { + ServiceName: "Products", + MethodName: "ResolveProductShippingEstimate", + Kind: CallKindResolve, + DependentCalls: []int{0}, + ResponsePath: buildPath("_entities.shippingEstimate"), + Request: RPCMessage{ + Name: "ResolveProductShippingEstimateRequest", + Fields: []RPCField{ + { + Name: "context", + TypeName: string(DataTypeMessage), + JSONPath: "", + Repeated: true, + Message: &RPCMessage{ + Name: "ResolveProductShippingEstimateContext", + Fields: []RPCField{ + { + Name: "id", + TypeName: string(DataTypeString), + JSONPath: "id", + ResolvePath: buildPath("result.id"), + }, + { + Name: "price", + TypeName: string(DataTypeDouble), + JSONPath: "price", + ResolvePath: buildPath("result.price"), + }, + }, + }, + }, + { + Name: "field_args", + TypeName: string(DataTypeMessage), + JSONPath: "", + Message: &RPCMessage{ + Name: "ResolveProductShippingEstimateArgs", + Fields: []RPCField{ + { + Name: "input", + TypeName: string(DataTypeMessage), + JSONPath: "input", + Message: &RPCMessage{ + Name: "ShippingEstimateInput", + Fields: []RPCField{ + { + Name: "destination", + TypeName: string(DataTypeEnum), + JSONPath: "destination", + EnumName: "ShippingDestination", + }, + { + Name: "weight", + TypeName: string(DataTypeDouble), + JSONPath: "weight", + }, + { + Name: "expedited", + TypeName: string(DataTypeBool), + JSONPath: "expedited", + Optional: true, + }, + }, + }, + }, + }, + }, + }, + }, + }, + Response: RPCMessage{ + Name: "ResolveProductShippingEstimateResponse", + Fields: []RPCField{ + { + Name: "result", + TypeName: string(DataTypeMessage), + JSONPath: "result", + Repeated: true, + Message: &RPCMessage{ + Name: "ResolveProductShippingEstimateResult", + Fields: []RPCField{ + { + Name: "shipping_estimate", + TypeName: string(DataTypeDouble), + JSONPath: "shippingEstimate", + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, } for _, tt := range tests { @@ -531,6 +518,7 @@ func TestEntityKeys(t *testing.T) { { ServiceName: "Products", MethodName: "LookupUserById", + Kind: CallKindEntity, // Define the structure of the request message Request: RPCMessage{ Name: "LookupUserByIdRequest", @@ -640,6 +628,7 @@ func TestEntityKeys(t *testing.T) { { ServiceName: "Products", MethodName: "LookupUserByIdAndAddress", + Kind: CallKindEntity, // Define the structure of the request message Request: RPCMessage{ Name: "LookupUserByIdAndAddressRequest", @@ -753,6 +742,7 @@ func TestEntityKeys(t *testing.T) { { ServiceName: "Products", MethodName: "LookupUserByIdAndName", + Kind: CallKindEntity, Request: RPCMessage{ Name: "LookupUserByIdAndNameRequest", Fields: []RPCField{ @@ -854,6 +844,7 @@ func TestEntityKeys(t *testing.T) { { ServiceName: "Products", MethodName: "LookupUserByIdAndName", + Kind: CallKindEntity, Request: RPCMessage{ Name: "LookupUserByIdAndNameRequest", Fields: []RPCField{ @@ -961,6 +952,7 @@ func TestEntityKeys(t *testing.T) { { ServiceName: "Products", MethodName: "LookupUserByIdAndNameAndAddress", + Kind: CallKindEntity, Request: RPCMessage{ Name: "LookupUserByIdAndNameAndAddressRequest", Fields: []RPCField{ diff --git a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor.go b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor.go index a8d50c3f5..6f4ca132f 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor.go +++ b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor.go @@ -40,18 +40,6 @@ type fieldArgument struct { argumentDefinitionRef int } -type resolvedField struct { - callerRef int - parentTypeRef int - fieldRef int - fieldDefinitionTypeRef int - requiredFields string - responsePath ast.Path - - contextFields []contextField - fieldArguments []fieldArgument -} - type rpcPlanVisitor struct { walker *astvisitor.Walker operation *ast.Document @@ -129,80 +117,10 @@ func (r *rpcPlanVisitor) LeaveDocument(_, _ *ast.Document) { return } - // We need to create a new call for each resolved field. - calls := make([]RPCCall, 0, len(r.resolvedFields)) - - for _, resolvedField := range r.resolvedFields { - resolveConfig, exists := r.mapping.FindResolveTypeFieldMapping( - r.definition.ObjectTypeDefinitionNameString(resolvedField.parentTypeRef), - r.operation.FieldNameString(resolvedField.fieldRef), - ) - - if !exists { - r.walker.StopWithInternalErr(fmt.Errorf("resolve config not found for type: %s, field: %s", r.definition.ResolveTypeNameString(resolvedField.parentTypeRef), r.operation.FieldAliasString(resolvedField.fieldRef))) - return - } - - contextMessage := &RPCMessage{ - Name: resolveConfig.RPC + "Context", - } - - fieldArgsMessage := &RPCMessage{ - Name: resolveConfig.RPC + "Args", - } - - call, err := r.planCtx.newResolveRPCCall(resolveRPCCallConfig{ - serviceName: r.subgraphName, - typeName: r.definition.ResolveTypeNameString(resolvedField.parentTypeRef), - fieldName: r.operation.FieldAliasOrNameString(resolvedField.fieldRef), - resolveConfig: resolveConfig, - resolvedField: resolvedField, - contextMessage: contextMessage, - fieldArgsMessage: fieldArgsMessage, - }) - - if err != nil { - r.walker.StopWithInternalErr(err) - } - - contextMessage.Fields = make(RPCFields, 0, len(resolvedField.contextFields)) - for _, contextField := range resolvedField.contextFields { - typeDefNode, found := r.definition.NodeByNameStr(r.definition.ResolveTypeNameString(resolvedField.parentTypeRef)) - if !found { - r.walker.StopWithInternalErr(fmt.Errorf("type definition node not found for type: %s", r.definition.ResolveTypeNameString(resolvedField.parentTypeRef))) - return - } - - field, err := r.planCtx.buildField( - typeDefNode, - contextField.fieldRef, - r.definition.FieldDefinitionNameString(contextField.fieldRef), - "", - ) - - field.ResolvePath = contextField.resolvePath - - if err != nil { - r.walker.StopWithInternalErr(err) - return - } - - contextMessage.Fields = append(contextMessage.Fields, field) - } - - fieldArgsMessage.Fields = make(RPCFields, 0, len(resolvedField.fieldArguments)) - for _, fieldArgument := range resolvedField.fieldArguments { - field, err := r.planCtx.createRPCFieldFromFieldArgument(fieldArgument) - - if err != nil { - r.walker.StopWithInternalErr(err) - return - } - - fieldArgsMessage.Fields = append(fieldArgsMessage.Fields, field) - } - - calls = append(calls, call) + calls, err := r.planCtx.createResolverRPCCalls(r.subgraphName, r.resolvedFields) + if err != nil { + r.walker.StopWithInternalErr(err) + return } r.plan.Calls = append(r.plan.Calls, calls...) @@ -276,6 +194,7 @@ func (r *rpcPlanVisitor) EnterSelectionSet(ref int) { return } + // If we are inside of a resolved field that selects multiple fields, we get all the fields from the input and pass them to the required fields visitor. if r.resolvedFieldIndex != ast.InvalidRef { lbrace := r.operation.SelectionSets[ref].LBrace.CharEnd rbrace := r.operation.SelectionSets[ref].RBrace.CharStart - 1 @@ -440,29 +359,11 @@ func (r *rpcPlanVisitor) EnterField(ref int) { fieldDefinitionTypeRef: r.definition.FieldDefinitionType(fd), } - contextFields, err := r.planCtx.resolveContextFields(r.walker, fd) - if err != nil { - r.walker.StopWithInternalErr(err) - return - } - - for _, contextFieldRef := range contextFields { - contextFieldName := r.definition.FieldDefinitionNameBytes(contextFieldRef) // TODO handle aliases - resolvedPath := r.fieldPath.WithFieldNameItem(contextFieldName) - - resolvedField.contextFields = append(resolvedField.contextFields, contextField{ - fieldRef: contextFieldRef, - resolvePath: resolvedPath, - }) - } - - fieldArguments, err := r.planCtx.parseFieldArguments(r.walker, fd, fieldArgs) - if err != nil { + if err := r.planCtx.setResolvedField(r.walker, fd, fieldArgs, r.fieldPath, &resolvedField); err != nil { r.walker.StopWithInternalErr(err) return } - resolvedField.fieldArguments = fieldArguments r.resolvedFields = append(r.resolvedFields, resolvedField) r.resolvedFieldIndex = len(r.resolvedFields) - 1 r.fieldPath = r.fieldPath.WithFieldNameItem(r.operation.FieldNameBytes(ref)) diff --git a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor_federation.go b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor_federation.go index 303158135..8ebc790e6 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor_federation.go +++ b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor_federation.go @@ -3,6 +3,7 @@ package grpcdatasource import ( "errors" "fmt" + "strings" "golang.org/x/text/cases" "golang.org/x/text/language" @@ -10,6 +11,7 @@ import ( "github.com/wundergraph/graphql-go-tools/v2/pkg/ast" "github.com/wundergraph/graphql-go-tools/v2/pkg/astvisitor" "github.com/wundergraph/graphql-go-tools/v2/pkg/engine/plan" + "github.com/wundergraph/graphql-go-tools/v2/pkg/internal/unsafebytes" "github.com/wundergraph/graphql-go-tools/v2/pkg/operationreport" ) @@ -49,6 +51,12 @@ type rpcPlanVisitorFederation struct { subgraphName string currentCall *RPCCall currentCallIndex int + + relatedCallID int + resolvedFieldIndex int + resolvedFields []resolvedField + + fieldPath ast.Path } func newRPCPlanVisitorFederation(config rpcPlanVisitorConfig) *rpcPlanVisitorFederation { @@ -63,9 +71,13 @@ func newRPCPlanVisitorFederation(config rpcPlanVisitorConfig) *rpcPlanVisitorFed entityInlineFragmentRef: ast.InvalidRef, }, federationConfigData: parseFederationConfigData(config.federationConfigs), + resolvedFields: make([]resolvedField, 0), + resolvedFieldIndex: ast.InvalidRef, + relatedCallID: ast.InvalidRef, + fieldPath: ast.Path{}.WithFieldNameItem([]byte("result")), } - walker.RegisterEnterDocumentVisitor(visitor) + walker.RegisterDocumentVisitor(visitor) walker.RegisterEnterOperationVisitor(visitor) walker.RegisterInlineFragmentVisitor(visitor) walker.RegisterSelectionSetVisitor(visitor) @@ -92,6 +104,23 @@ func (r *rpcPlanVisitorFederation) EnterDocument(operation *ast.Document, defini r.planCtx = newRPCPlanningContext(operation, definition, r.mapping) } +// LeaveDocument implements astvisitor.DocumentVisitor. +func (r *rpcPlanVisitorFederation) LeaveDocument(_, _ *ast.Document) { + if len(r.resolvedFields) == 0 { + return + } + + calls, err := r.planCtx.createResolverRPCCalls(r.subgraphName, r.resolvedFields) + if err != nil { + r.walker.StopWithInternalErr(err) + return + } + + r.plan.Calls = append(r.plan.Calls, calls...) + r.resolvedFields = nil + +} + // EnterOperationDefinition implements astvisitor.EnterOperationDefinitionVisitor. func (r *rpcPlanVisitorFederation) EnterOperationDefinition(ref int) { if r.operation.OperationDefinitions[ref].OperationType != ast.OperationTypeQuery { @@ -112,8 +141,11 @@ func (r *rpcPlanVisitorFederation) EnterInlineFragment(ref int) { r.currentCall = &RPCCall{ ServiceName: r.planCtx.resolveServiceName(r.subgraphName), + Kind: CallKindEntity, } + r.relatedCallID = len(r.plan.Calls) + r.planInfo.currentRequestMessage = &r.currentCall.Request r.planInfo.currentResponseMessage = &r.currentCall.Response @@ -136,7 +168,6 @@ func (r *rpcPlanVisitorFederation) LeaveInlineFragment(ref int) { r.plan.Calls = append(r.plan.Calls, *r.currentCall) r.currentCall = &RPCCall{} - r.currentCallIndex++ r.planInfo = planningInfo{ operationType: r.planInfo.operationType, @@ -157,6 +188,16 @@ func (r *rpcPlanVisitorFederation) EnterSelectionSet(ref int) { return } + // If we are inside of a resolved field that selects multiple fields, we get all the fields from the input and pass them to the required fields visitor. + if r.resolvedFieldIndex != ast.InvalidRef { + lbrace := r.operation.SelectionSets[ref].LBrace.CharEnd + rbrace := r.operation.SelectionSets[ref].RBrace.CharStart - 1 + + r.resolvedFields[r.resolvedFieldIndex].requiredFields = unsafebytes.BytesToString(r.operation.Input.RawBytes[lbrace:rbrace]) + r.walker.SkipNode() + return + } + if r.planInfo.currentRequestMessage == nil || len(r.planInfo.currentResponseMessage.Fields) == 0 || len(r.planInfo.currentResponseMessage.Fields) <= r.planInfo.currentResponseFieldIndex { return } @@ -247,7 +288,8 @@ func (r *rpcPlanVisitorFederation) LeaveSelectionSet(ref int) { // EnterField implements astvisitor.FieldVisitor. func (r *rpcPlanVisitorFederation) EnterField(ref int) { fieldName := r.operation.FieldNameString(ref) - if r.walker.InRootField() { + inRootField := r.walker.InRootField() + if inRootField { r.planInfo.operationFieldName = r.operation.FieldNameString(ref) } @@ -268,6 +310,7 @@ func (r *rpcPlanVisitorFederation) EnterField(ref int) { // prevent duplicate fields fieldAlias := r.operation.FieldAliasString(ref) if r.planInfo.currentResponseMessage.Fields.Exists(fieldName, fieldAlias) { + r.fieldPath = r.fieldPath.WithFieldNameItem([]byte{}) return } @@ -279,12 +322,44 @@ func (r *rpcPlanVisitorFederation) EnterField(ref int) { return } + if fieldArgs := r.operation.FieldArguments(ref); !inRootField && len(fieldArgs) > 0 { + // We don't want to add fields from the selection set to the actual call + resolvedField := resolvedField{ + callerRef: r.relatedCallID, + parentTypeRef: r.walker.EnclosingTypeDefinition.Ref, + fieldRef: ref, + responsePath: r.walker.Path[1:].WithoutInlineFragmentNames().WithFieldNameItem(r.operation.FieldAliasOrNameBytes(ref)), + fieldDefinitionTypeRef: r.definition.FieldDefinitionType(fd), + } + + if err := r.planCtx.setResolvedField(r.walker, fd, fieldArgs, r.fieldPath, &resolvedField); err != nil { + r.walker.StopWithInternalErr(err) + return + } + + r.resolvedFields = append(r.resolvedFields, resolvedField) + r.resolvedFieldIndex = len(r.resolvedFields) - 1 + r.fieldPath = r.fieldPath.WithFieldNameItem(r.operation.FieldNameBytes(ref)) + + // In case of nested fields with arguments, we need to increment the related call ID. + r.relatedCallID++ + return + } + field, err := r.planCtx.buildField(r.walker.EnclosingTypeDefinition, fd, fieldName, fieldAlias) if err != nil { r.walker.StopWithInternalErr(err) return } + // If we have a nested or nullable list, we add a @ prefix to indicate the nesting level. + prefix := "" + if field.ListMetadata != nil { + prefix = strings.Repeat("@", field.ListMetadata.NestingLevel) + } + + r.fieldPath = r.fieldPath.WithFieldNameItem([]byte(prefix + field.Name)) + // check if we are inside of an inline fragment and not the entity inline fragment if ref, ok := r.walker.ResolveInlineFragment(); ok && r.entityInfo.entityInlineFragmentRef != ref { if r.planInfo.currentResponseMessage.FieldSelectionSet == nil { @@ -301,8 +376,17 @@ func (r *rpcPlanVisitorFederation) EnterField(ref int) { // LeaveField implements astvisitor.FieldVisitor. func (r *rpcPlanVisitorFederation) LeaveField(ref int) { + r.fieldPath = r.fieldPath.RemoveLastItem() + r.resolvedFieldIndex = ast.InvalidRef // If we are not in the operation field, we can increment the response field index. if !r.walker.InRootField() { + // If the field has arguments, we need to decrement the related call ID. + // This is because we can also have nested arguments, which require the underlying field to be resolved + // by values provided by the parent call. + if r.operation.FieldHasArguments(ref) { + r.relatedCallID-- + } + r.planInfo.currentResponseFieldIndex++ return } diff --git a/v2/pkg/engine/datasource/grpc_datasource/grpc_datasource.go b/v2/pkg/engine/datasource/grpc_datasource/grpc_datasource.go index 1ae2b4688..02a0a1246 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/grpc_datasource.go +++ b/v2/pkg/engine/datasource/grpc_datasource/grpc_datasource.go @@ -22,6 +22,7 @@ import ( "github.com/wundergraph/graphql-go-tools/v2/pkg/engine/datasource/httpclient" "github.com/wundergraph/graphql-go-tools/v2/pkg/engine/plan" "github.com/wundergraph/graphql-go-tools/v2/pkg/engine/resolve" + "github.com/wundergraph/graphql-go-tools/v2/pkg/internal/unsafebytes" ) type resultData struct { @@ -87,7 +88,7 @@ func NewDataSource(client grpc.ClientConnInterface, config DataSourceConfig) (*D // a gRPC call, including service name, method name, and request data. func (d *DataSource) Load(ctx context.Context, input []byte, out *bytes.Buffer) (err error) { // get variables from input - variables := gjson.Parse(string(input)).Get("body.variables") + variables := gjson.Parse(unsafebytes.BytesToString(input)).Get("body.variables") builder := newJSONBuilder(d.mapping, variables) if d.disabled { @@ -127,18 +128,14 @@ func (d *DataSource) Load(ctx context.Context, input []byte, out *bytes.Buffer) return nil } - // if serviceCall.RPC.Kind == CallKindResolve { - // return builder.mergeWithPath(root, response, serviceCall.RPC.ResponsePath) - // } - // In case of a federated response, we need to ensure that the response is valid. // The number of entities per type must match the number of lookup keys in the variablese - // if serviceCall.RPC.Kind == CallKindEntity { - err = builder.validateFederatedResponse(response) - if err != nil { - return err + if serviceCall.RPC.Kind == CallKindEntity { + err = builder.validateFederatedResponse(response) + if err != nil { + return err + } } - // } mu.Lock() results[index] = resultData{ diff --git a/v2/pkg/engine/datasource/grpc_datasource/grpc_datasource_test.go b/v2/pkg/engine/datasource/grpc_datasource/grpc_datasource_test.go index 5b682f7d7..a69bd5f52 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/grpc_datasource_test.go +++ b/v2/pkg/engine/datasource/grpc_datasource/grpc_datasource_test.go @@ -3663,6 +3663,53 @@ func Test_DataSource_Load_WithEntity_Calls(t *testing.T) { require.Equal(t, "entity type Warehouse received 3 entities in the subgraph response, but 4 are expected", errorData[0].Message) }, }, + { + name: "Query Product with field resolvers", + query: `query($representations: [_Any!]!, $input: ShippingEstimateInput!) { _entities(representations: $representations) { ...on Product { id name price shippingEstimate(input: $input) } } }`, + vars: ` + { + "variables": + { + "representations":[ + {"__typename":"Product","id":"1"}, + {"__typename":"Product","id":"2"}, + {"__typename":"Product","id":"3"} + ], + "input":{ + "destination":"INTERNATIONAL", + "weight":10.0, + "expedited":true + } + }`, + federationConfigs: plan.FederationFieldConfigurations{ + { + TypeName: "Product", + SelectionSet: "id", + }, + }, + validate: func(t *testing.T, data map[string]interface{}) { + require.NotEmpty(t, data) + + entities, ok := data["_entities"].([]interface{}) + require.True(t, ok, "_entities should be an array") + require.NotEmpty(t, entities, "_entities should not be empty") + require.Len(t, entities, 3, "Should return 3 entities") + for index, entity := range entities { + entity, ok := entity.(map[string]interface{}) + require.True(t, ok, "entity should be an object") + productID := index + 1 + + require.Equal(t, fmt.Sprintf("%d", productID), entity["id"]) + require.Equal(t, fmt.Sprintf("Product %d", productID), entity["name"]) + require.InDelta(t, float64(99.99), entity["price"], 0.01) + require.InDelta(t, float64(77.49), entity["shippingEstimate"], 0.01) + } + + }, + validateError: func(t *testing.T, errorData []graphqlError) { + require.Empty(t, errorData) + }, + }, } for _, tc := range testCases { diff --git a/v2/pkg/engine/datasource/grpc_datasource/mapping_test_helper.go b/v2/pkg/engine/datasource/grpc_datasource/mapping_test_helper.go index 96c511be0..148cd51dc 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/mapping_test_helper.go +++ b/v2/pkg/engine/datasource/grpc_datasource/mapping_test_helper.go @@ -254,6 +254,30 @@ func testMapping() *GRPCMapping { Response: "ResolveCategoryCategoryMetricsResponse", }, }, + "Product": { + "shippingEstimate": { + FieldMappingData: FieldMapData{ + TargetName: "shipping_estimate", + ArgumentMappings: FieldArgumentMap{ + "input": "input", + }, + }, + RPC: "ResolveProductShippingEstimate", + Request: "ResolveProductShippingEstimateRequest", + Response: "ResolveProductShippingEstimateResponse", + }, + "recommendedCategory": { + FieldMappingData: FieldMapData{ + TargetName: "recommended_category", + ArgumentMappings: FieldArgumentMap{ + "maxPrice": "max_price", + }, + }, + RPC: "ResolveProductRecommendedCategory", + Request: "ResolveProductRecommendedCategoryRequest", + Response: "ResolveProductRecommendedCategoryResponse", + }, + }, "Subcategory": { "itemCount": { FieldMappingData: FieldMapData{ @@ -307,6 +331,11 @@ func testMapping() *GRPCMapping { {Value: "FURNITURE", TargetValue: "CATEGORY_KIND_FURNITURE"}, {Value: "OTHER", TargetValue: "CATEGORY_KIND_OTHER"}, }, + "ShippingDestination": { + {Value: "DOMESTIC", TargetValue: "SHIPPING_DESTINATION_DOMESTIC"}, + {Value: "EXPRESS", TargetValue: "SHIPPING_DESTINATION_EXPRESS"}, + {Value: "INTERNATIONAL", TargetValue: "SHIPPING_DESTINATION_INTERNATIONAL"}, + }, }, Fields: map[string]FieldMap{ "Query": { @@ -540,6 +569,18 @@ func testMapping() *GRPCMapping { "price": { TargetName: "price", }, + "shippingEstimate": { + TargetName: "shipping_estimate", + ArgumentMappings: FieldArgumentMap{ + "input": "input", + }, + }, + "recommendedCategory": { + TargetName: "recommended_category", + ArgumentMappings: FieldArgumentMap{ + "maxPrice": "max_price", + }, + }, }, "Storage": { "id": { @@ -1165,6 +1206,17 @@ func testMapping() *GRPCMapping { TargetName: "search_term", }, }, + "ShippingEstimateInput": { + "destination": { + TargetName: "destination", + }, + "weight": { + TargetName: "weight", + }, + "expedited": { + TargetName: "expedited", + }, + }, "UserInput": { "name": { TargetName: "name", @@ -1172,5 +1224,4 @@ func testMapping() *GRPCMapping { }, }, } - } diff --git a/v2/pkg/grpctest/Makefile b/v2/pkg/grpctest/Makefile index e7ea6737d..4fda449fd 100644 --- a/v2/pkg/grpctest/Makefile +++ b/v2/pkg/grpctest/Makefile @@ -1,3 +1,5 @@ +mkfile_dir := $(shell dirname $(abspath $(lastword $(MAKEFILE_LIST)))) + .PHONY: install-protoc install-protoc: go install google.golang.org/protobuf/cmd/protoc-gen-go@latest @@ -8,6 +10,8 @@ generate-proto: install-protoc protoc --go_out=productv1 --go_opt=paths=source_relative \ --go-grpc_out=productv1 --go-grpc_opt=paths=source_relative \ product.proto + mv $(mkfile_dir)/testdata/service.proto $(mkfile_dir)/product.proto + rm -f $(mkfile_dir)/testdata/service.proto.lock.json .PHONY: build-plugin build-plugin: @@ -17,6 +21,26 @@ build-plugin: regenerate-proto: pnpx wgc@latest grpc-service generate -i testdata/products.graphqls -o testdata/ -p productv1 -g "cosmo/pkg/proto/productv1;productv1" Product +.PHONY: generate-mapping-code +generate-mapping-code: regenerate-proto + go run $(mkfile_dir)/cmd/mapping_helper/main.go $(mkfile_dir)/testdata/mapping.json $(mkfile_dir)/mapping/mapping.go + go fmt $(mkfile_dir)/mapping/mapping.go + rm -f $(mkfile_dir)/testdata/mapping.json + + .PHONY: regenerate-proto-local regenerate-proto-local: - pnpx tsx --env-file ../../../../cosmo/cli/.env ../../../../cosmo/cli/src/index.ts grpc-service generate -i testdata/products.graphqls -o testdata/ -p productv1 -g "cosmo/pkg/proto/productv1;productv1" Product \ No newline at end of file + pnpx tsx --env-file ../../../../cosmo/cli/.env ../../../../cosmo/cli/src/index.ts grpc-service generate -i testdata/products.graphqls -o testdata/ -p productv1 -g "cosmo/pkg/proto/productv1;productv1" Product + +.PHONY: generate-mapping-code-local +generate-mapping-code-local: regenerate-proto-local + go run $(mkfile_dir)/cmd/mapping_helper/main.go $(mkfile_dir)/testdata/mapping.json $(mkfile_dir)/mapping/mapping.go + go fmt $(mkfile_dir)/mapping/mapping.go + rm -f $(mkfile_dir)/testdata/mapping.json + + +.PHONY: generate-all-local +generate-all-local: generate-mapping-code-local generate-proto + +.PHONY: generate-all +generate-all: generate-mapping-code generate-proto \ No newline at end of file diff --git a/v2/pkg/grpctest/Readme.md b/v2/pkg/grpctest/Readme.md new file mode 100644 index 000000000..e49201c4a --- /dev/null +++ b/v2/pkg/grpctest/Readme.md @@ -0,0 +1,160 @@ +# grpctest Package + +The `grpctest` package provides a comprehensive mock gRPC service implementation for testing the `grpc_datasource` functionality in graphql-go-tools. It includes a complete GraphQL schema, Protocol Buffers definitions, and mock service implementations to facilitate testing of gRPC-to-GraphQL integration. + +## Overview + +This package contains: + +- **Mock gRPC Service**: A fully functional mock implementation of a product service with various data types and operations +- **GraphQL Schema**: A comprehensive schema covering various GraphQL features including queries, mutations, unions, interfaces, and complex nested types +- **Protocol Buffers**: Complete `.proto` definitions with generated Go code +- **Field Mappings**: Detailed mappings between GraphQL fields and gRPC message fields +- **Test Utilities**: Helper functions for schema loading, validation, and test data generation + +## Key Components + +### MockService (`mockservice.go`) + +The core mock gRPC service implementation that provides: + +- **Entity Lookups**: Product, Storage, and Warehouse entity resolution by ID +- **Query Operations**: Various query types including filtering, pagination, and complex nested queries +- **Mutation Operations**: Create and update operations for different entity types +- **Field Resolvers**: Custom field resolution for computed fields like `productCount`, `popularityScore`, etc. +- **Union Types**: Support for GraphQL unions (e.g., `SearchResult`, `ActionResult`, `Animal`) +- **Nullable Fields**: Comprehensive testing of optional/nullable field handling +- **Complex Lists**: Nested list structures and bulk operations + +### Schema Management (`schema.go`) + +Provides utilities for: + +- Loading and parsing GraphQL schemas from embedded files +- Schema validation and transformation +- Protocol buffer schema access +- Test-friendly schema loading functions + +### Field Mappings (`mapping/mapping.go`) + +Defines the complete mapping between GraphQL and gRPC: + +- **Query/Mutation RPCs**: Maps GraphQL operations to gRPC service methods +- **Field Mappings**: Maps GraphQL field names to protobuf field names +- **Argument Mappings**: Maps GraphQL arguments to gRPC request parameters +- **Entity Lookups**: Configuration for entity resolution patterns +- **Enum Mappings**: GraphQL enum to protobuf enum value mappings + +## Code Generation + +The package includes tools for regenerating Protocol Buffer definitions: + +```bash +# Generate Go code from .proto files +make generate-proto + +# Regenerate .proto from GraphQL schema (requires wgc CLI) +make regenerate-proto + +# Regenerate .proto from GraphQL schema locally (requires cosmo to be checked out in the same parent directory as graphql-go-tools) +make regenerate-proto-local + +# Build plugin service +make build-plugin +``` + +## File Structure + +``` +grpctest/ +├── Readme.md # This documentation +├── Makefile # Build and generation commands +├── mockservice.go # Main mock service implementation +├── schema.go # Schema loading utilities +├── product.proto # Protocol buffer definitions +├── mapping/ +│ └── mapping.go # GraphQL-to-gRPC field mappings +├── productv1/ # Generated protobuf Go code +│ ├── product.pb.go +│ └── product_grpc.pb.go +├── testdata/ # Test schemas and data +│ ├── products.graphqls # GraphQL schema +├── plugin/ # Plugin service implementation +└── cmd/ # Command-line utilities +``` + +## Contributing + +When adding new test cases or extending the mock service: + +1. Update the GraphQL schema in `testdata/products.graphqls` +2. Regenerate the Protocol Buffer definitions using `make regenerate-proto` +3. Update the mock service implementation in `mockservice.go` +4. Add corresponding field mappings in `mapping/mapping.go` +5. Update field configurations in `schema.go` if needed + +## Integration with grpc_datasource + +This package is specifically designed to test the `grpc_datasource` implementation and provides comprehensive coverage of: + +- Request/response mapping +- Field resolution patterns +- Error handling scenarios +- Performance characteristics +- Edge cases and boundary conditions + +The mock service implements realistic data patterns and edge cases that help ensure the gRPC data source implementation is robust and handles real-world scenarios correctly. + +## Regenerate all files + +To make generating super simple, you can run the following command: + +```bash +# Generate all files +make generate-all + +# Generate all files using the local wgc cli +make generate-all-local +``` + +This will regenerate the .proto file, generate the mapping.go file, and regenerate the protobuf Go code. + +## Generating the mapping.go file + +The mapping.go file is generated using the `mapping_helper` command. This command takes a mapping.json file and generates the mapping.go file. + +You can either run the `mapping_helper` command manually or use the `make generate-mapping-code` command. +The recommended way is to use the Makefile. + +### Use Makefile + +```bash +# Regenerate .proto and mapping files +make regenerate-proto +# or make regenerate-proto-local (if you need to test local changes in protographic) + +# Generate the mapping.go file +make generate-mapping-code +``` + +This will run the `generate-mapping-code` command with the correct arguments. + +### Setup a Run Configuration in VSCode + +This is mostly useful for debugging the mapping helper. + +```json +{ + "name": "Launch mapping helper", + "type": "go", + "request": "launch", + "mode": "debug", + "program": "${workspaceFolder}/v2/pkg/grpctest/cmd/mapping_helper/main.go", + "args": [ + "${workspaceFolder}/v2/pkg/grpctest/testdata/mapping.json", + "${workspaceFolder}/v2/pkg/grpctest/mapping/mapping.go" + ], + "cwd": "${workspaceFolder}/v2/pkg/grpctest/cmd/mapping_helper" +} +``` + diff --git a/v2/pkg/grpctest/cmd/mapping_helper/main.go b/v2/pkg/grpctest/cmd/mapping_helper/main.go index 927579f79..9280120f4 100644 --- a/v2/pkg/grpctest/cmd/mapping_helper/main.go +++ b/v2/pkg/grpctest/cmd/mapping_helper/main.go @@ -2,6 +2,7 @@ package main import ( "bytes" + _ "embed" "encoding/json" "log" "os" @@ -85,126 +86,11 @@ type EnumValue struct { Mapped string `json:"mapped"` } -const tpl = `package mapping - -import ( - "testing" - - grpcdatasource "github.com/wundergraph/graphql-go-tools/v2/pkg/engine/datasource/grpc_datasource" +var ( + //go:embed templates/grpctest_mapping.tmpl + tpl string ) -// DefaultGRPCMapping returns a hardcoded default mapping between GraphQL and Protobuf -func DefaultGRPCMapping() *grpcdatasource.GRPCMapping { - return &grpcdatasource.GRPCMapping{ - Service: "{{.Service}}", - QueryRPCs: grpcdatasource.RPCConfigMap[grpcdatasource.RPCConfig]{ - {{- range $index, $operation := .OperationMappings}} - {{- if eq $operation.Type "OPERATION_TYPE_QUERY"}} - "{{$operation.Original}}": { - RPC: "{{$operation.Mapped}}", - Request: "{{$operation.Request}}", - Response: "{{$operation.Response}}", - }, - {{- end }} - {{- end }} - }, - MutationRPCs: grpcdatasource.RPCConfigMap[grpcdatasource.RPCConfig]{ - {{- range $index, $operation := .OperationMappings}} - {{- if eq $operation.Type "OPERATION_TYPE_MUTATION"}} - "{{$operation.Original}}": { - RPC: "{{$operation.Mapped}}", - Request: "{{$operation.Request}}", - Response: "{{$operation.Response}}", - }, - {{- end }} - {{- end }} - }, - SubscriptionRPCs: grpcdatasource.RPCConfigMap[grpcdatasource.RPCConfig]{ - {{- range $index, $operation := .OperationMappings}} - {{- if eq $operation.Type "OPERATION_TYPE_SUBSCRIPTION"}} - "{{$operation.Original}}": { - RPC: "{{$operation.Mapped}}", - Request: "{{$operation.Request}}", - Response: "{{$operation.Response}}", - }, - {{- end }} - {{- end }} - }, - ResolveRPCs: grpcdatasource.RPCConfigMap[grpcdatasource.ResolveRPCMapping]{ - {{- range $type, $item := .ResolveRPCs}} - "{{$type}}": { - {{- range $index, $resolve := $item}} - "{{$resolve.LookupMapping.FieldMapping.Original}}": { - FieldMappingData: grpcdatasource.FieldMapData{ - TargetName: "{{$resolve.LookupMapping.FieldMapping.Mapped}}", - ArgumentMappings: grpcdatasource.FieldArgumentMap{ - {{- range $index, $argument := $resolve.LookupMapping.FieldMapping.ArgumentMappings}} - "{{$argument.Original}}": "{{$argument.Mapped}}", - {{- end }} - }, - }, - RPC: "{{$resolve.RPC}}", - Request: "{{$resolve.Request}}", - Response: "{{$resolve.Response}}", - }, - {{- end }} - }, - {{- end }} - }, - EntityRPCs: map[string][]grpcdatasource.EntityRPCConfig{ - {{- range $index, $entity := .EntityMappings}} - "{{$entity.TypeName}}": { - { - Key: "{{$entity.Key}}", - RPCConfig: grpcdatasource.RPCConfig{ - RPC: "{{$entity.RPC}}", - Request: "{{$entity.Request}}", - Response: "{{$entity.Response}}", - }, - }, - }, - {{- end }} - }, - EnumValues: map[string][]grpcdatasource.EnumValueMapping{ - {{- range $index, $enum := .EnumMappings}} - "{{$enum.Type}}": { - {{- range $index, $value := .Values}} - {Value: "{{$value.Original}}", TargetValue: "{{$value.Mapped}}"}, - {{- end }} - }, - {{- end }} - }, - Fields: map[string]grpcdatasource.FieldMap{ - {{- range $index, $typeField := .TypeFieldMappings}} - "{{$typeField.Type}}": { - {{- range $index, $field := $typeField.FieldMappings}} - "{{$field.Original}}": { - TargetName: "{{$field.Mapped}}", - {{- if (gt (len $field.ArgumentMappings) 0)}} - ArgumentMappings: grpcdatasource.FieldArgumentMap{ - {{- range $index, $argument := $field.ArgumentMappings}} - "{{$argument.Original}}": "{{$argument.Mapped}}", - {{- end }} - }, - {{- end }} - }, - {{- end}} - }, - {{- end}} - }, - } -} - - -// MustDefaultGRPCMapping returns the default GRPC mapping -func MustDefaultGRPCMapping(t *testing.T) *grpcdatasource.GRPCMapping { - mapping := DefaultGRPCMapping() - return mapping -} - - -` - func main() { args := os.Args[1:] if len(args) < 2 { diff --git a/v2/pkg/grpctest/cmd/mapping_helper/templates/grpctest_mapping.tmpl b/v2/pkg/grpctest/cmd/mapping_helper/templates/grpctest_mapping.tmpl new file mode 100644 index 000000000..ec8f37cce --- /dev/null +++ b/v2/pkg/grpctest/cmd/mapping_helper/templates/grpctest_mapping.tmpl @@ -0,0 +1,116 @@ +package mapping + +import ( + "testing" + + grpcdatasource "github.com/wundergraph/graphql-go-tools/v2/pkg/engine/datasource/grpc_datasource" +) + +// DefaultGRPCMapping returns a hardcoded default mapping between GraphQL and Protobuf +func DefaultGRPCMapping() *grpcdatasource.GRPCMapping { + return &grpcdatasource.GRPCMapping{ + Service: "Products", + QueryRPCs: grpcdatasource.RPCConfigMap[grpcdatasource.RPCConfig]{ + {{- range $index, $operation := .OperationMappings}} + {{- if eq $operation.Type "OPERATION_TYPE_QUERY"}} + "{{$operation.Original}}": { + RPC: "{{$operation.Mapped}}", + Request: "{{$operation.Request}}", + Response: "{{$operation.Response}}", + }, + {{- end }} + {{- end }} + }, + MutationRPCs: grpcdatasource.RPCConfigMap[grpcdatasource.RPCConfig]{ + {{- range $index, $operation := .OperationMappings}} + {{- if eq $operation.Type "OPERATION_TYPE_MUTATION"}} + "{{$operation.Original}}": { + RPC: "{{$operation.Mapped}}", + Request: "{{$operation.Request}}", + Response: "{{$operation.Response}}", + }, + {{- end }} + {{- end }} + }, + SubscriptionRPCs: grpcdatasource.RPCConfigMap[grpcdatasource.RPCConfig]{ + {{- range $index, $operation := .OperationMappings}} + {{- if eq $operation.Type "OPERATION_TYPE_SUBSCRIPTION"}} + "{{$operation.Original}}": { + RPC: "{{$operation.Mapped}}", + Request: "{{$operation.Request}}", + Response: "{{$operation.Response}}", + }, + {{- end }} + {{- end }} + }, + ResolveRPCs: grpcdatasource.RPCConfigMap[grpcdatasource.ResolveRPCMapping]{ + {{- range $type, $item := .ResolveRPCs}} + "{{$type}}": { + {{- range $index, $resolve := $item}} + "{{$resolve.LookupMapping.FieldMapping.Original}}": { + FieldMappingData: grpcdatasource.FieldMapData{ + TargetName: "{{$resolve.LookupMapping.FieldMapping.Mapped}}", + ArgumentMappings: grpcdatasource.FieldArgumentMap{ + {{- range $index, $argument := $resolve.LookupMapping.FieldMapping.ArgumentMappings}} + "{{$argument.Original}}": "{{$argument.Mapped}}", + {{- end }} + }, + }, + RPC: "{{$resolve.RPC}}", + Request: "{{$resolve.Request}}", + Response: "{{$resolve.Response}}", + }, + {{- end }} + }, + {{- end }} + }, + EntityRPCs: map[string][]grpcdatasource.EntityRPCConfig{ + {{- range $index, $entity := .EntityMappings}} + "{{$entity.TypeName}}": { + { + Key: "{{$entity.Key}}", + RPCConfig: grpcdatasource.RPCConfig{ + RPC: "{{$entity.RPC}}", + Request: "{{$entity.Request}}", + Response: "{{$entity.Response}}", + }, + }, + }, + {{- end }} + }, + EnumValues: map[string][]grpcdatasource.EnumValueMapping{ + {{- range $index, $enum := .EnumMappings}} + "{{$enum.Type}}": { + {{- range $index, $value := .Values}} + {Value: "{{$value.Original}}", TargetValue: "{{$value.Mapped}}"}, + {{- end }} + }, + {{- end }} + }, + Fields: map[string]grpcdatasource.FieldMap{ + {{- range $index, $typeField := .TypeFieldMappings}} + "{{$typeField.Type}}": { + {{- range $index, $field := $typeField.FieldMappings}} + "{{$field.Original}}": { + TargetName: "{{$field.Mapped}}", + {{- if (gt (len $field.ArgumentMappings) 0)}} + ArgumentMappings: grpcdatasource.FieldArgumentMap{ + {{- range $index, $argument := $field.ArgumentMappings}} + "{{$argument.Original}}": "{{$argument.Mapped}}", + {{- end }} + }, + {{- end }} + }, + {{- end}} + }, + {{- end}} + }, + } +} + + +// MustDefaultGRPCMapping returns the default GRPC mapping +func MustDefaultGRPCMapping(t *testing.T) *grpcdatasource.GRPCMapping { + mapping := DefaultGRPCMapping() + return mapping +} diff --git a/v2/pkg/grpctest/mapping/mapping.go b/v2/pkg/grpctest/mapping/mapping.go index 988f22c4e..6397adc31 100644 --- a/v2/pkg/grpctest/mapping/mapping.go +++ b/v2/pkg/grpctest/mapping/mapping.go @@ -261,6 +261,30 @@ func DefaultGRPCMapping() *grpcdatasource.GRPCMapping { Response: "ResolveCategoryCategoryMetricsResponse", }, }, + "Product": { + "shippingEstimate": { + FieldMappingData: grpcdatasource.FieldMapData{ + TargetName: "shipping_estimate", + ArgumentMappings: grpcdatasource.FieldArgumentMap{ + "input": "input", + }, + }, + RPC: "ResolveProductShippingEstimate", + Request: "ResolveProductShippingEstimateRequest", + Response: "ResolveProductShippingEstimateResponse", + }, + "recommendedCategory": { + FieldMappingData: grpcdatasource.FieldMapData{ + TargetName: "recommended_category", + ArgumentMappings: grpcdatasource.FieldArgumentMap{ + "maxPrice": "max_price", + }, + }, + RPC: "ResolveProductRecommendedCategory", + Request: "ResolveProductRecommendedCategoryRequest", + Response: "ResolveProductRecommendedCategoryResponse", + }, + }, "Subcategory": { "itemCount": { FieldMappingData: grpcdatasource.FieldMapData{ @@ -314,6 +338,11 @@ func DefaultGRPCMapping() *grpcdatasource.GRPCMapping { {Value: "FURNITURE", TargetValue: "CATEGORY_KIND_FURNITURE"}, {Value: "OTHER", TargetValue: "CATEGORY_KIND_OTHER"}, }, + "ShippingDestination": { + {Value: "DOMESTIC", TargetValue: "SHIPPING_DESTINATION_DOMESTIC"}, + {Value: "EXPRESS", TargetValue: "SHIPPING_DESTINATION_EXPRESS"}, + {Value: "INTERNATIONAL", TargetValue: "SHIPPING_DESTINATION_INTERNATIONAL"}, + }, }, Fields: map[string]grpcdatasource.FieldMap{ "Query": { @@ -547,6 +576,18 @@ func DefaultGRPCMapping() *grpcdatasource.GRPCMapping { "price": { TargetName: "price", }, + "shippingEstimate": { + TargetName: "shipping_estimate", + ArgumentMappings: grpcdatasource.FieldArgumentMap{ + "input": "input", + }, + }, + "recommendedCategory": { + TargetName: "recommended_category", + ArgumentMappings: grpcdatasource.FieldArgumentMap{ + "maxPrice": "max_price", + }, + }, }, "Storage": { "id": { @@ -1172,6 +1213,17 @@ func DefaultGRPCMapping() *grpcdatasource.GRPCMapping { TargetName: "search_term", }, }, + "ShippingEstimateInput": { + "destination": { + TargetName: "destination", + }, + "weight": { + TargetName: "weight", + }, + "expedited": { + TargetName: "expedited", + }, + }, "UserInput": { "name": { TargetName: "name", diff --git a/v2/pkg/grpctest/mockservice.go b/v2/pkg/grpctest/mockservice.go index 992089b5c..5e504a2f3 100644 --- a/v2/pkg/grpctest/mockservice.go +++ b/v2/pkg/grpctest/mockservice.go @@ -20,6 +20,95 @@ type MockService struct { productv1.UnimplementedProductServiceServer } +// ResolveProductRecommendedCategory implements productv1.ProductServiceServer. +func (s *MockService) ResolveProductRecommendedCategory(_ context.Context, req *productv1.ResolveProductRecommendedCategoryRequest) (*productv1.ResolveProductRecommendedCategoryResponse, error) { + results := make([]*productv1.ResolveProductRecommendedCategoryResult, 0, len(req.GetContext())) + + maxPrice := int32(0) + if req.GetFieldArgs() != nil { + maxPrice = req.GetFieldArgs().GetMaxPrice() + } + + for _, ctx := range req.GetContext() { + // Return nil for products with high price to test optional return + if maxPrice > 0 && ctx.GetPrice() > float64(maxPrice) { + results = append(results, &productv1.ResolveProductRecommendedCategoryResult{ + RecommendedCategory: nil, + }) + } else { + // Create a recommended category based on product context + categoryKind := productv1.CategoryKind_CATEGORY_KIND_OTHER + if ctx.GetPrice() < 50 { + categoryKind = productv1.CategoryKind_CATEGORY_KIND_BOOK + } else if ctx.GetPrice() < 200 { + categoryKind = productv1.CategoryKind_CATEGORY_KIND_ELECTRONICS + } else { + categoryKind = productv1.CategoryKind_CATEGORY_KIND_FURNITURE + } + + results = append(results, &productv1.ResolveProductRecommendedCategoryResult{ + RecommendedCategory: &productv1.Category{ + Id: fmt.Sprintf("recommended-cat-%s", ctx.GetId()), + Name: fmt.Sprintf("Recommended for %s", ctx.GetName()), + Kind: categoryKind, + Subcategories: createSubcategories(fmt.Sprintf("recommended-cat-%s", ctx.GetId()), categoryKind, 2), + }, + }) + } + } + + resp := &productv1.ResolveProductRecommendedCategoryResponse{ + Result: results, + } + + return resp, nil +} + +// ResolveProductShippingEstimate implements productv1.ProductServiceServer. +func (s *MockService) ResolveProductShippingEstimate(_ context.Context, req *productv1.ResolveProductShippingEstimateRequest) (*productv1.ResolveProductShippingEstimateResponse, error) { + results := make([]*productv1.ResolveProductShippingEstimateResult, 0, len(req.GetContext())) + + for _, ctx := range req.GetContext() { + // Base shipping cost calculation + baseCost := ctx.GetPrice() * 0.1 // 10% of product price + + // Add weight-based cost if input provided + if req.GetFieldArgs() != nil && req.GetFieldArgs().GetInput() != nil { + input := req.GetFieldArgs().GetInput() + + // Add weight cost + weightCost := float64(input.GetWeight()) * 2.5 + baseCost += weightCost + + // Add expedited shipping cost + if input.GetExpedited() != nil && input.GetExpedited().GetValue() { + baseCost *= 1.5 // 50% surcharge for expedited + } + + // Add destination-based cost + destination := input.GetDestination() + switch destination { + case productv1.ShippingDestination_SHIPPING_DESTINATION_INTERNATIONAL: + baseCost += 25.0 + case productv1.ShippingDestination_SHIPPING_DESTINATION_EXPRESS: + baseCost += 10.0 + case productv1.ShippingDestination_SHIPPING_DESTINATION_DOMESTIC: + // No additional cost for domestic shipping + } + } + + results = append(results, &productv1.ResolveProductShippingEstimateResult{ + ShippingEstimate: baseCost, + }) + } + + resp := &productv1.ResolveProductShippingEstimateResponse{ + Result: results, + } + + return resp, nil +} + // ResolveCategoryCategoryMetrics implements productv1.ProductServiceServer. func (s *MockService) ResolveCategoryCategoryMetrics(_ context.Context, req *productv1.ResolveCategoryCategoryMetricsRequest) (*productv1.ResolveCategoryCategoryMetricsResponse, error) { results := make([]*productv1.ResolveCategoryCategoryMetricsResult, 0, len(req.GetContext())) diff --git a/v2/pkg/grpctest/product.proto b/v2/pkg/grpctest/product.proto index a577307b1..ac6ec65b2 100644 --- a/v2/pkg/grpctest/product.proto +++ b/v2/pkg/grpctest/product.proto @@ -58,6 +58,8 @@ service ProductService { rpc ResolveCategoryCategoryMetrics(ResolveCategoryCategoryMetricsRequest) returns (ResolveCategoryCategoryMetricsResponse) {} rpc ResolveCategoryPopularityScore(ResolveCategoryPopularityScoreRequest) returns (ResolveCategoryPopularityScoreResponse) {} rpc ResolveCategoryProductCount(ResolveCategoryProductCountRequest) returns (ResolveCategoryProductCountResponse) {} + rpc ResolveProductRecommendedCategory(ResolveProductRecommendedCategoryRequest) returns (ResolveProductRecommendedCategoryResponse) {} + rpc ResolveProductShippingEstimate(ResolveProductShippingEstimateRequest) returns (ResolveProductShippingEstimateResponse) {} rpc ResolveSubcategoryItemCount(ResolveSubcategoryItemCountRequest) returns (ResolveSubcategoryItemCountResponse) {} } @@ -630,6 +632,55 @@ message MutationBulkUpdateBlogPostsRequest { message MutationBulkUpdateBlogPostsResponse { repeated BlogPost bulk_update_blog_posts = 1; } +message ResolveProductShippingEstimateArgs { + ShippingEstimateInput input = 1; +} + +message ResolveProductShippingEstimateContext { + string id = 1; + double price = 2; +} + +message ResolveProductShippingEstimateRequest { + // context provides the resolver context for the field shippingEstimate of type Product. + repeated ResolveProductShippingEstimateContext context = 1; + // field_args provides the arguments for the resolver field shippingEstimate of type Product. + ResolveProductShippingEstimateArgs field_args = 2; +} + +message ResolveProductShippingEstimateResult { + double shipping_estimate = 1; +} + +message ResolveProductShippingEstimateResponse { + repeated ResolveProductShippingEstimateResult result = 1; +} + +message ResolveProductRecommendedCategoryArgs { + int32 max_price = 1; +} + +message ResolveProductRecommendedCategoryContext { + string id = 1; + string name = 2; + double price = 3; +} + +message ResolveProductRecommendedCategoryRequest { + // context provides the resolver context for the field recommendedCategory of type Product. + repeated ResolveProductRecommendedCategoryContext context = 1; + // field_args provides the arguments for the resolver field recommendedCategory of type Product. + ResolveProductRecommendedCategoryArgs field_args = 2; +} + +message ResolveProductRecommendedCategoryResult { + Category recommended_category = 1; +} + +message ResolveProductRecommendedCategoryResponse { + repeated ResolveProductRecommendedCategoryResult result = 1; +} + message ResolveCategoryProductCountArgs { ProductCountFilter filters = 1; } @@ -1053,4 +1104,17 @@ message SubcategoryItemFilter { google.protobuf.BoolValue in_stock = 3; google.protobuf.BoolValue is_active = 4; google.protobuf.StringValue search_term = 5; +} + +enum ShippingDestination { + SHIPPING_DESTINATION_UNSPECIFIED = 0; + SHIPPING_DESTINATION_DOMESTIC = 1; + SHIPPING_DESTINATION_EXPRESS = 2; + SHIPPING_DESTINATION_INTERNATIONAL = 3; +} + +message ShippingEstimateInput { + ShippingDestination destination = 1; + double weight = 2; + google.protobuf.BoolValue expedited = 3; } \ No newline at end of file diff --git a/v2/pkg/grpctest/productv1/product.pb.go b/v2/pkg/grpctest/productv1/product.pb.go index 6b79c603c..ed265f796 100644 --- a/v2/pkg/grpctest/productv1/product.pb.go +++ b/v2/pkg/grpctest/productv1/product.pb.go @@ -77,6 +77,58 @@ func (CategoryKind) EnumDescriptor() ([]byte, []int) { return file_product_proto_rawDescGZIP(), []int{0} } +type ShippingDestination int32 + +const ( + ShippingDestination_SHIPPING_DESTINATION_UNSPECIFIED ShippingDestination = 0 + ShippingDestination_SHIPPING_DESTINATION_DOMESTIC ShippingDestination = 1 + ShippingDestination_SHIPPING_DESTINATION_EXPRESS ShippingDestination = 2 + ShippingDestination_SHIPPING_DESTINATION_INTERNATIONAL ShippingDestination = 3 +) + +// Enum value maps for ShippingDestination. +var ( + ShippingDestination_name = map[int32]string{ + 0: "SHIPPING_DESTINATION_UNSPECIFIED", + 1: "SHIPPING_DESTINATION_DOMESTIC", + 2: "SHIPPING_DESTINATION_EXPRESS", + 3: "SHIPPING_DESTINATION_INTERNATIONAL", + } + ShippingDestination_value = map[string]int32{ + "SHIPPING_DESTINATION_UNSPECIFIED": 0, + "SHIPPING_DESTINATION_DOMESTIC": 1, + "SHIPPING_DESTINATION_EXPRESS": 2, + "SHIPPING_DESTINATION_INTERNATIONAL": 3, + } +) + +func (x ShippingDestination) Enum() *ShippingDestination { + p := new(ShippingDestination) + *p = x + return p +} + +func (x ShippingDestination) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (ShippingDestination) Descriptor() protoreflect.EnumDescriptor { + return file_product_proto_enumTypes[1].Descriptor() +} + +func (ShippingDestination) Type() protoreflect.EnumType { + return &file_product_proto_enumTypes[1] +} + +func (x ShippingDestination) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use ShippingDestination.Descriptor instead. +func (ShippingDestination) EnumDescriptor() ([]byte, []int) { + return file_product_proto_rawDescGZIP(), []int{1} +} + // Wrapper message for a list of AuthorFilter. type ListOfAuthorFilter struct { state protoimpl.MessageState `protogen:"open.v1"` @@ -5138,27 +5190,27 @@ func (x *MutationBulkUpdateBlogPostsResponse) GetBulkUpdateBlogPosts() []*BlogPo return nil } -type ResolveCategoryProductCountArgs struct { +type ResolveProductShippingEstimateArgs struct { state protoimpl.MessageState `protogen:"open.v1"` - Filters *ProductCountFilter `protobuf:"bytes,1,opt,name=filters,proto3" json:"filters,omitempty"` + Input *ShippingEstimateInput `protobuf:"bytes,1,opt,name=input,proto3" json:"input,omitempty"` unknownFields protoimpl.UnknownFields sizeCache protoimpl.SizeCache } -func (x *ResolveCategoryProductCountArgs) Reset() { - *x = ResolveCategoryProductCountArgs{} +func (x *ResolveProductShippingEstimateArgs) Reset() { + *x = ResolveProductShippingEstimateArgs{} mi := &file_product_proto_msgTypes[113] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } -func (x *ResolveCategoryProductCountArgs) String() string { +func (x *ResolveProductShippingEstimateArgs) String() string { return protoimpl.X.MessageStringOf(x) } -func (*ResolveCategoryProductCountArgs) ProtoMessage() {} +func (*ResolveProductShippingEstimateArgs) ProtoMessage() {} -func (x *ResolveCategoryProductCountArgs) ProtoReflect() protoreflect.Message { +func (x *ResolveProductShippingEstimateArgs) ProtoReflect() protoreflect.Message { mi := &file_product_proto_msgTypes[113] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) @@ -5170,40 +5222,40 @@ func (x *ResolveCategoryProductCountArgs) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use ResolveCategoryProductCountArgs.ProtoReflect.Descriptor instead. -func (*ResolveCategoryProductCountArgs) Descriptor() ([]byte, []int) { +// Deprecated: Use ResolveProductShippingEstimateArgs.ProtoReflect.Descriptor instead. +func (*ResolveProductShippingEstimateArgs) Descriptor() ([]byte, []int) { return file_product_proto_rawDescGZIP(), []int{113} } -func (x *ResolveCategoryProductCountArgs) GetFilters() *ProductCountFilter { +func (x *ResolveProductShippingEstimateArgs) GetInput() *ShippingEstimateInput { if x != nil { - return x.Filters + return x.Input } return nil } -type ResolveCategoryProductCountContext struct { +type ResolveProductShippingEstimateContext struct { state protoimpl.MessageState `protogen:"open.v1"` Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` - Name string `protobuf:"bytes,2,opt,name=name,proto3" json:"name,omitempty"` + Price float64 `protobuf:"fixed64,2,opt,name=price,proto3" json:"price,omitempty"` unknownFields protoimpl.UnknownFields sizeCache protoimpl.SizeCache } -func (x *ResolveCategoryProductCountContext) Reset() { - *x = ResolveCategoryProductCountContext{} +func (x *ResolveProductShippingEstimateContext) Reset() { + *x = ResolveProductShippingEstimateContext{} mi := &file_product_proto_msgTypes[114] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } -func (x *ResolveCategoryProductCountContext) String() string { +func (x *ResolveProductShippingEstimateContext) String() string { return protoimpl.X.MessageStringOf(x) } -func (*ResolveCategoryProductCountContext) ProtoMessage() {} +func (*ResolveProductShippingEstimateContext) ProtoMessage() {} -func (x *ResolveCategoryProductCountContext) ProtoReflect() protoreflect.Message { +func (x *ResolveProductShippingEstimateContext) ProtoReflect() protoreflect.Message { mi := &file_product_proto_msgTypes[114] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) @@ -5215,49 +5267,49 @@ func (x *ResolveCategoryProductCountContext) ProtoReflect() protoreflect.Message return mi.MessageOf(x) } -// Deprecated: Use ResolveCategoryProductCountContext.ProtoReflect.Descriptor instead. -func (*ResolveCategoryProductCountContext) Descriptor() ([]byte, []int) { +// Deprecated: Use ResolveProductShippingEstimateContext.ProtoReflect.Descriptor instead. +func (*ResolveProductShippingEstimateContext) Descriptor() ([]byte, []int) { return file_product_proto_rawDescGZIP(), []int{114} } -func (x *ResolveCategoryProductCountContext) GetId() string { +func (x *ResolveProductShippingEstimateContext) GetId() string { if x != nil { return x.Id } return "" } -func (x *ResolveCategoryProductCountContext) GetName() string { +func (x *ResolveProductShippingEstimateContext) GetPrice() float64 { if x != nil { - return x.Name + return x.Price } - return "" + return 0 } -type ResolveCategoryProductCountRequest struct { +type ResolveProductShippingEstimateRequest struct { state protoimpl.MessageState `protogen:"open.v1"` - // context provides the resolver context for the field productCount of type Category. - Context []*ResolveCategoryProductCountContext `protobuf:"bytes,1,rep,name=context,proto3" json:"context,omitempty"` - // field_args provides the arguments for the resolver field productCount of type Category. - FieldArgs *ResolveCategoryProductCountArgs `protobuf:"bytes,2,opt,name=field_args,json=fieldArgs,proto3" json:"field_args,omitempty"` + // context provides the resolver context for the field shippingEstimate of type Product. + Context []*ResolveProductShippingEstimateContext `protobuf:"bytes,1,rep,name=context,proto3" json:"context,omitempty"` + // field_args provides the arguments for the resolver field shippingEstimate of type Product. + FieldArgs *ResolveProductShippingEstimateArgs `protobuf:"bytes,2,opt,name=field_args,json=fieldArgs,proto3" json:"field_args,omitempty"` unknownFields protoimpl.UnknownFields sizeCache protoimpl.SizeCache } -func (x *ResolveCategoryProductCountRequest) Reset() { - *x = ResolveCategoryProductCountRequest{} +func (x *ResolveProductShippingEstimateRequest) Reset() { + *x = ResolveProductShippingEstimateRequest{} mi := &file_product_proto_msgTypes[115] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } -func (x *ResolveCategoryProductCountRequest) String() string { +func (x *ResolveProductShippingEstimateRequest) String() string { return protoimpl.X.MessageStringOf(x) } -func (*ResolveCategoryProductCountRequest) ProtoMessage() {} +func (*ResolveProductShippingEstimateRequest) ProtoMessage() {} -func (x *ResolveCategoryProductCountRequest) ProtoReflect() protoreflect.Message { +func (x *ResolveProductShippingEstimateRequest) ProtoReflect() protoreflect.Message { mi := &file_product_proto_msgTypes[115] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) @@ -5269,46 +5321,46 @@ func (x *ResolveCategoryProductCountRequest) ProtoReflect() protoreflect.Message return mi.MessageOf(x) } -// Deprecated: Use ResolveCategoryProductCountRequest.ProtoReflect.Descriptor instead. -func (*ResolveCategoryProductCountRequest) Descriptor() ([]byte, []int) { +// Deprecated: Use ResolveProductShippingEstimateRequest.ProtoReflect.Descriptor instead. +func (*ResolveProductShippingEstimateRequest) Descriptor() ([]byte, []int) { return file_product_proto_rawDescGZIP(), []int{115} } -func (x *ResolveCategoryProductCountRequest) GetContext() []*ResolveCategoryProductCountContext { +func (x *ResolveProductShippingEstimateRequest) GetContext() []*ResolveProductShippingEstimateContext { if x != nil { return x.Context } return nil } -func (x *ResolveCategoryProductCountRequest) GetFieldArgs() *ResolveCategoryProductCountArgs { +func (x *ResolveProductShippingEstimateRequest) GetFieldArgs() *ResolveProductShippingEstimateArgs { if x != nil { return x.FieldArgs } return nil } -type ResolveCategoryProductCountResult struct { - state protoimpl.MessageState `protogen:"open.v1"` - ProductCount int32 `protobuf:"varint,1,opt,name=product_count,json=productCount,proto3" json:"product_count,omitempty"` - unknownFields protoimpl.UnknownFields - sizeCache protoimpl.SizeCache +type ResolveProductShippingEstimateResult struct { + state protoimpl.MessageState `protogen:"open.v1"` + ShippingEstimate float64 `protobuf:"fixed64,1,opt,name=shipping_estimate,json=shippingEstimate,proto3" json:"shipping_estimate,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } -func (x *ResolveCategoryProductCountResult) Reset() { - *x = ResolveCategoryProductCountResult{} +func (x *ResolveProductShippingEstimateResult) Reset() { + *x = ResolveProductShippingEstimateResult{} mi := &file_product_proto_msgTypes[116] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } -func (x *ResolveCategoryProductCountResult) String() string { +func (x *ResolveProductShippingEstimateResult) String() string { return protoimpl.X.MessageStringOf(x) } -func (*ResolveCategoryProductCountResult) ProtoMessage() {} +func (*ResolveProductShippingEstimateResult) ProtoMessage() {} -func (x *ResolveCategoryProductCountResult) ProtoReflect() protoreflect.Message { +func (x *ResolveProductShippingEstimateResult) ProtoReflect() protoreflect.Message { mi := &file_product_proto_msgTypes[116] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) @@ -5320,39 +5372,39 @@ func (x *ResolveCategoryProductCountResult) ProtoReflect() protoreflect.Message return mi.MessageOf(x) } -// Deprecated: Use ResolveCategoryProductCountResult.ProtoReflect.Descriptor instead. -func (*ResolveCategoryProductCountResult) Descriptor() ([]byte, []int) { +// Deprecated: Use ResolveProductShippingEstimateResult.ProtoReflect.Descriptor instead. +func (*ResolveProductShippingEstimateResult) Descriptor() ([]byte, []int) { return file_product_proto_rawDescGZIP(), []int{116} } -func (x *ResolveCategoryProductCountResult) GetProductCount() int32 { +func (x *ResolveProductShippingEstimateResult) GetShippingEstimate() float64 { if x != nil { - return x.ProductCount + return x.ShippingEstimate } return 0 } -type ResolveCategoryProductCountResponse struct { - state protoimpl.MessageState `protogen:"open.v1"` - Result []*ResolveCategoryProductCountResult `protobuf:"bytes,1,rep,name=result,proto3" json:"result,omitempty"` +type ResolveProductShippingEstimateResponse struct { + state protoimpl.MessageState `protogen:"open.v1"` + Result []*ResolveProductShippingEstimateResult `protobuf:"bytes,1,rep,name=result,proto3" json:"result,omitempty"` unknownFields protoimpl.UnknownFields sizeCache protoimpl.SizeCache } -func (x *ResolveCategoryProductCountResponse) Reset() { - *x = ResolveCategoryProductCountResponse{} +func (x *ResolveProductShippingEstimateResponse) Reset() { + *x = ResolveProductShippingEstimateResponse{} mi := &file_product_proto_msgTypes[117] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } -func (x *ResolveCategoryProductCountResponse) String() string { +func (x *ResolveProductShippingEstimateResponse) String() string { return protoimpl.X.MessageStringOf(x) } -func (*ResolveCategoryProductCountResponse) ProtoMessage() {} +func (*ResolveProductShippingEstimateResponse) ProtoMessage() {} -func (x *ResolveCategoryProductCountResponse) ProtoReflect() protoreflect.Message { +func (x *ResolveProductShippingEstimateResponse) ProtoReflect() protoreflect.Message { mi := &file_product_proto_msgTypes[117] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) @@ -5364,39 +5416,39 @@ func (x *ResolveCategoryProductCountResponse) ProtoReflect() protoreflect.Messag return mi.MessageOf(x) } -// Deprecated: Use ResolveCategoryProductCountResponse.ProtoReflect.Descriptor instead. -func (*ResolveCategoryProductCountResponse) Descriptor() ([]byte, []int) { +// Deprecated: Use ResolveProductShippingEstimateResponse.ProtoReflect.Descriptor instead. +func (*ResolveProductShippingEstimateResponse) Descriptor() ([]byte, []int) { return file_product_proto_rawDescGZIP(), []int{117} } -func (x *ResolveCategoryProductCountResponse) GetResult() []*ResolveCategoryProductCountResult { +func (x *ResolveProductShippingEstimateResponse) GetResult() []*ResolveProductShippingEstimateResult { if x != nil { return x.Result } return nil } -type ResolveCategoryPopularityScoreArgs struct { +type ResolveProductRecommendedCategoryArgs struct { state protoimpl.MessageState `protogen:"open.v1"` - Threshold *wrapperspb.Int32Value `protobuf:"bytes,1,opt,name=threshold,proto3" json:"threshold,omitempty"` + MaxPrice int32 `protobuf:"varint,1,opt,name=max_price,json=maxPrice,proto3" json:"max_price,omitempty"` unknownFields protoimpl.UnknownFields sizeCache protoimpl.SizeCache } -func (x *ResolveCategoryPopularityScoreArgs) Reset() { - *x = ResolveCategoryPopularityScoreArgs{} +func (x *ResolveProductRecommendedCategoryArgs) Reset() { + *x = ResolveProductRecommendedCategoryArgs{} mi := &file_product_proto_msgTypes[118] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } -func (x *ResolveCategoryPopularityScoreArgs) String() string { +func (x *ResolveProductRecommendedCategoryArgs) String() string { return protoimpl.X.MessageStringOf(x) } -func (*ResolveCategoryPopularityScoreArgs) ProtoMessage() {} +func (*ResolveProductRecommendedCategoryArgs) ProtoMessage() {} -func (x *ResolveCategoryPopularityScoreArgs) ProtoReflect() protoreflect.Message { +func (x *ResolveProductRecommendedCategoryArgs) ProtoReflect() protoreflect.Message { mi := &file_product_proto_msgTypes[118] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) @@ -5408,39 +5460,41 @@ func (x *ResolveCategoryPopularityScoreArgs) ProtoReflect() protoreflect.Message return mi.MessageOf(x) } -// Deprecated: Use ResolveCategoryPopularityScoreArgs.ProtoReflect.Descriptor instead. -func (*ResolveCategoryPopularityScoreArgs) Descriptor() ([]byte, []int) { +// Deprecated: Use ResolveProductRecommendedCategoryArgs.ProtoReflect.Descriptor instead. +func (*ResolveProductRecommendedCategoryArgs) Descriptor() ([]byte, []int) { return file_product_proto_rawDescGZIP(), []int{118} } -func (x *ResolveCategoryPopularityScoreArgs) GetThreshold() *wrapperspb.Int32Value { +func (x *ResolveProductRecommendedCategoryArgs) GetMaxPrice() int32 { if x != nil { - return x.Threshold + return x.MaxPrice } - return nil + return 0 } -type ResolveCategoryPopularityScoreContext struct { +type ResolveProductRecommendedCategoryContext struct { state protoimpl.MessageState `protogen:"open.v1"` Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` + Name string `protobuf:"bytes,2,opt,name=name,proto3" json:"name,omitempty"` + Price float64 `protobuf:"fixed64,3,opt,name=price,proto3" json:"price,omitempty"` unknownFields protoimpl.UnknownFields sizeCache protoimpl.SizeCache } -func (x *ResolveCategoryPopularityScoreContext) Reset() { - *x = ResolveCategoryPopularityScoreContext{} +func (x *ResolveProductRecommendedCategoryContext) Reset() { + *x = ResolveProductRecommendedCategoryContext{} mi := &file_product_proto_msgTypes[119] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } -func (x *ResolveCategoryPopularityScoreContext) String() string { +func (x *ResolveProductRecommendedCategoryContext) String() string { return protoimpl.X.MessageStringOf(x) } -func (*ResolveCategoryPopularityScoreContext) ProtoMessage() {} +func (*ResolveProductRecommendedCategoryContext) ProtoMessage() {} -func (x *ResolveCategoryPopularityScoreContext) ProtoReflect() protoreflect.Message { +func (x *ResolveProductRecommendedCategoryContext) ProtoReflect() protoreflect.Message { mi := &file_product_proto_msgTypes[119] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) @@ -5452,42 +5506,56 @@ func (x *ResolveCategoryPopularityScoreContext) ProtoReflect() protoreflect.Mess return mi.MessageOf(x) } -// Deprecated: Use ResolveCategoryPopularityScoreContext.ProtoReflect.Descriptor instead. -func (*ResolveCategoryPopularityScoreContext) Descriptor() ([]byte, []int) { +// Deprecated: Use ResolveProductRecommendedCategoryContext.ProtoReflect.Descriptor instead. +func (*ResolveProductRecommendedCategoryContext) Descriptor() ([]byte, []int) { return file_product_proto_rawDescGZIP(), []int{119} } -func (x *ResolveCategoryPopularityScoreContext) GetId() string { +func (x *ResolveProductRecommendedCategoryContext) GetId() string { if x != nil { return x.Id } return "" } -type ResolveCategoryPopularityScoreRequest struct { +func (x *ResolveProductRecommendedCategoryContext) GetName() string { + if x != nil { + return x.Name + } + return "" +} + +func (x *ResolveProductRecommendedCategoryContext) GetPrice() float64 { + if x != nil { + return x.Price + } + return 0 +} + +type ResolveProductRecommendedCategoryRequest struct { state protoimpl.MessageState `protogen:"open.v1"` - // context provides the resolver context for the field popularityScore of type Category. - Context []*ResolveCategoryPopularityScoreContext `protobuf:"bytes,1,rep,name=context,proto3" json:"context,omitempty"` - // field_args provides the arguments for the resolver field popularityScore of type Category. - FieldArgs *ResolveCategoryPopularityScoreArgs `protobuf:"bytes,2,opt,name=field_args,json=fieldArgs,proto3" json:"field_args,omitempty"` + // context provides the resolver context for the field recommendedCategory of type Product. + Context []*ResolveProductRecommendedCategoryContext `protobuf:"bytes,1,rep,name=context,proto3" json:"context,omitempty"` + // field_args provides the arguments for the resolver field recommendedCategory of type Product. + FieldArgs *ResolveProductRecommendedCategoryArgs `protobuf:"bytes,2,opt,name=field_args,json=fieldArgs,proto3" json:"field_args,omitempty"` unknownFields protoimpl.UnknownFields sizeCache protoimpl.SizeCache } -func (x *ResolveCategoryPopularityScoreRequest) Reset() { - *x = ResolveCategoryPopularityScoreRequest{} +func (x *ResolveProductRecommendedCategoryRequest) Reset() { + *x = ResolveProductRecommendedCategoryRequest{} mi := &file_product_proto_msgTypes[120] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } -func (x *ResolveCategoryPopularityScoreRequest) String() string { +func (x *ResolveProductRecommendedCategoryRequest) String() string { return protoimpl.X.MessageStringOf(x) } -func (*ResolveCategoryPopularityScoreRequest) ProtoMessage() {} +func (*ResolveProductRecommendedCategoryRequest) ProtoMessage() {} -func (x *ResolveCategoryPopularityScoreRequest) ProtoReflect() protoreflect.Message { +func (x *ResolveProductRecommendedCategoryRequest) ProtoReflect() protoreflect.Message { mi := &file_product_proto_msgTypes[120] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) @@ -5499,46 +5567,46 @@ func (x *ResolveCategoryPopularityScoreRequest) ProtoReflect() protoreflect.Mess return mi.MessageOf(x) } -// Deprecated: Use ResolveCategoryPopularityScoreRequest.ProtoReflect.Descriptor instead. -func (*ResolveCategoryPopularityScoreRequest) Descriptor() ([]byte, []int) { +// Deprecated: Use ResolveProductRecommendedCategoryRequest.ProtoReflect.Descriptor instead. +func (*ResolveProductRecommendedCategoryRequest) Descriptor() ([]byte, []int) { return file_product_proto_rawDescGZIP(), []int{120} } -func (x *ResolveCategoryPopularityScoreRequest) GetContext() []*ResolveCategoryPopularityScoreContext { +func (x *ResolveProductRecommendedCategoryRequest) GetContext() []*ResolveProductRecommendedCategoryContext { if x != nil { return x.Context } return nil } -func (x *ResolveCategoryPopularityScoreRequest) GetFieldArgs() *ResolveCategoryPopularityScoreArgs { +func (x *ResolveProductRecommendedCategoryRequest) GetFieldArgs() *ResolveProductRecommendedCategoryArgs { if x != nil { return x.FieldArgs } return nil } -type ResolveCategoryPopularityScoreResult struct { - state protoimpl.MessageState `protogen:"open.v1"` - PopularityScore *wrapperspb.Int32Value `protobuf:"bytes,1,opt,name=popularity_score,json=popularityScore,proto3" json:"popularity_score,omitempty"` - unknownFields protoimpl.UnknownFields - sizeCache protoimpl.SizeCache +type ResolveProductRecommendedCategoryResult struct { + state protoimpl.MessageState `protogen:"open.v1"` + RecommendedCategory *Category `protobuf:"bytes,1,opt,name=recommended_category,json=recommendedCategory,proto3" json:"recommended_category,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } -func (x *ResolveCategoryPopularityScoreResult) Reset() { - *x = ResolveCategoryPopularityScoreResult{} +func (x *ResolveProductRecommendedCategoryResult) Reset() { + *x = ResolveProductRecommendedCategoryResult{} mi := &file_product_proto_msgTypes[121] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } -func (x *ResolveCategoryPopularityScoreResult) String() string { +func (x *ResolveProductRecommendedCategoryResult) String() string { return protoimpl.X.MessageStringOf(x) } -func (*ResolveCategoryPopularityScoreResult) ProtoMessage() {} +func (*ResolveProductRecommendedCategoryResult) ProtoMessage() {} -func (x *ResolveCategoryPopularityScoreResult) ProtoReflect() protoreflect.Message { +func (x *ResolveProductRecommendedCategoryResult) ProtoReflect() protoreflect.Message { mi := &file_product_proto_msgTypes[121] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) @@ -5550,39 +5618,39 @@ func (x *ResolveCategoryPopularityScoreResult) ProtoReflect() protoreflect.Messa return mi.MessageOf(x) } -// Deprecated: Use ResolveCategoryPopularityScoreResult.ProtoReflect.Descriptor instead. -func (*ResolveCategoryPopularityScoreResult) Descriptor() ([]byte, []int) { +// Deprecated: Use ResolveProductRecommendedCategoryResult.ProtoReflect.Descriptor instead. +func (*ResolveProductRecommendedCategoryResult) Descriptor() ([]byte, []int) { return file_product_proto_rawDescGZIP(), []int{121} } -func (x *ResolveCategoryPopularityScoreResult) GetPopularityScore() *wrapperspb.Int32Value { +func (x *ResolveProductRecommendedCategoryResult) GetRecommendedCategory() *Category { if x != nil { - return x.PopularityScore + return x.RecommendedCategory } return nil } -type ResolveCategoryPopularityScoreResponse struct { - state protoimpl.MessageState `protogen:"open.v1"` - Result []*ResolveCategoryPopularityScoreResult `protobuf:"bytes,1,rep,name=result,proto3" json:"result,omitempty"` +type ResolveProductRecommendedCategoryResponse struct { + state protoimpl.MessageState `protogen:"open.v1"` + Result []*ResolveProductRecommendedCategoryResult `protobuf:"bytes,1,rep,name=result,proto3" json:"result,omitempty"` unknownFields protoimpl.UnknownFields sizeCache protoimpl.SizeCache } -func (x *ResolveCategoryPopularityScoreResponse) Reset() { - *x = ResolveCategoryPopularityScoreResponse{} +func (x *ResolveProductRecommendedCategoryResponse) Reset() { + *x = ResolveProductRecommendedCategoryResponse{} mi := &file_product_proto_msgTypes[122] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } -func (x *ResolveCategoryPopularityScoreResponse) String() string { +func (x *ResolveProductRecommendedCategoryResponse) String() string { return protoimpl.X.MessageStringOf(x) } -func (*ResolveCategoryPopularityScoreResponse) ProtoMessage() {} +func (*ResolveProductRecommendedCategoryResponse) ProtoMessage() {} -func (x *ResolveCategoryPopularityScoreResponse) ProtoReflect() protoreflect.Message { +func (x *ResolveProductRecommendedCategoryResponse) ProtoReflect() protoreflect.Message { mi := &file_product_proto_msgTypes[122] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) @@ -5594,39 +5662,39 @@ func (x *ResolveCategoryPopularityScoreResponse) ProtoReflect() protoreflect.Mes return mi.MessageOf(x) } -// Deprecated: Use ResolveCategoryPopularityScoreResponse.ProtoReflect.Descriptor instead. -func (*ResolveCategoryPopularityScoreResponse) Descriptor() ([]byte, []int) { +// Deprecated: Use ResolveProductRecommendedCategoryResponse.ProtoReflect.Descriptor instead. +func (*ResolveProductRecommendedCategoryResponse) Descriptor() ([]byte, []int) { return file_product_proto_rawDescGZIP(), []int{122} } -func (x *ResolveCategoryPopularityScoreResponse) GetResult() []*ResolveCategoryPopularityScoreResult { +func (x *ResolveProductRecommendedCategoryResponse) GetResult() []*ResolveProductRecommendedCategoryResult { if x != nil { return x.Result } return nil } -type ResolveCategoryCategoryMetricsArgs struct { +type ResolveCategoryProductCountArgs struct { state protoimpl.MessageState `protogen:"open.v1"` - MetricType string `protobuf:"bytes,1,opt,name=metric_type,json=metricType,proto3" json:"metric_type,omitempty"` + Filters *ProductCountFilter `protobuf:"bytes,1,opt,name=filters,proto3" json:"filters,omitempty"` unknownFields protoimpl.UnknownFields sizeCache protoimpl.SizeCache } -func (x *ResolveCategoryCategoryMetricsArgs) Reset() { - *x = ResolveCategoryCategoryMetricsArgs{} +func (x *ResolveCategoryProductCountArgs) Reset() { + *x = ResolveCategoryProductCountArgs{} mi := &file_product_proto_msgTypes[123] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } -func (x *ResolveCategoryCategoryMetricsArgs) String() string { +func (x *ResolveCategoryProductCountArgs) String() string { return protoimpl.X.MessageStringOf(x) } -func (*ResolveCategoryCategoryMetricsArgs) ProtoMessage() {} +func (*ResolveCategoryProductCountArgs) ProtoMessage() {} -func (x *ResolveCategoryCategoryMetricsArgs) ProtoReflect() protoreflect.Message { +func (x *ResolveCategoryProductCountArgs) ProtoReflect() protoreflect.Message { mi := &file_product_proto_msgTypes[123] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) @@ -5638,19 +5706,19 @@ func (x *ResolveCategoryCategoryMetricsArgs) ProtoReflect() protoreflect.Message return mi.MessageOf(x) } -// Deprecated: Use ResolveCategoryCategoryMetricsArgs.ProtoReflect.Descriptor instead. -func (*ResolveCategoryCategoryMetricsArgs) Descriptor() ([]byte, []int) { +// Deprecated: Use ResolveCategoryProductCountArgs.ProtoReflect.Descriptor instead. +func (*ResolveCategoryProductCountArgs) Descriptor() ([]byte, []int) { return file_product_proto_rawDescGZIP(), []int{123} } -func (x *ResolveCategoryCategoryMetricsArgs) GetMetricType() string { +func (x *ResolveCategoryProductCountArgs) GetFilters() *ProductCountFilter { if x != nil { - return x.MetricType + return x.Filters } - return "" + return nil } -type ResolveCategoryCategoryMetricsContext struct { +type ResolveCategoryProductCountContext struct { state protoimpl.MessageState `protogen:"open.v1"` Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` Name string `protobuf:"bytes,2,opt,name=name,proto3" json:"name,omitempty"` @@ -5658,20 +5726,20 @@ type ResolveCategoryCategoryMetricsContext struct { sizeCache protoimpl.SizeCache } -func (x *ResolveCategoryCategoryMetricsContext) Reset() { - *x = ResolveCategoryCategoryMetricsContext{} +func (x *ResolveCategoryProductCountContext) Reset() { + *x = ResolveCategoryProductCountContext{} mi := &file_product_proto_msgTypes[124] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } -func (x *ResolveCategoryCategoryMetricsContext) String() string { +func (x *ResolveCategoryProductCountContext) String() string { return protoimpl.X.MessageStringOf(x) } -func (*ResolveCategoryCategoryMetricsContext) ProtoMessage() {} +func (*ResolveCategoryProductCountContext) ProtoMessage() {} -func (x *ResolveCategoryCategoryMetricsContext) ProtoReflect() protoreflect.Message { +func (x *ResolveCategoryProductCountContext) ProtoReflect() protoreflect.Message { mi := &file_product_proto_msgTypes[124] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) @@ -5683,49 +5751,49 @@ func (x *ResolveCategoryCategoryMetricsContext) ProtoReflect() protoreflect.Mess return mi.MessageOf(x) } -// Deprecated: Use ResolveCategoryCategoryMetricsContext.ProtoReflect.Descriptor instead. -func (*ResolveCategoryCategoryMetricsContext) Descriptor() ([]byte, []int) { +// Deprecated: Use ResolveCategoryProductCountContext.ProtoReflect.Descriptor instead. +func (*ResolveCategoryProductCountContext) Descriptor() ([]byte, []int) { return file_product_proto_rawDescGZIP(), []int{124} } -func (x *ResolveCategoryCategoryMetricsContext) GetId() string { +func (x *ResolveCategoryProductCountContext) GetId() string { if x != nil { return x.Id } return "" } -func (x *ResolveCategoryCategoryMetricsContext) GetName() string { +func (x *ResolveCategoryProductCountContext) GetName() string { if x != nil { return x.Name } return "" } -type ResolveCategoryCategoryMetricsRequest struct { +type ResolveCategoryProductCountRequest struct { state protoimpl.MessageState `protogen:"open.v1"` - // context provides the resolver context for the field categoryMetrics of type Category. - Context []*ResolveCategoryCategoryMetricsContext `protobuf:"bytes,1,rep,name=context,proto3" json:"context,omitempty"` - // field_args provides the arguments for the resolver field categoryMetrics of type Category. - FieldArgs *ResolveCategoryCategoryMetricsArgs `protobuf:"bytes,2,opt,name=field_args,json=fieldArgs,proto3" json:"field_args,omitempty"` + // context provides the resolver context for the field productCount of type Category. + Context []*ResolveCategoryProductCountContext `protobuf:"bytes,1,rep,name=context,proto3" json:"context,omitempty"` + // field_args provides the arguments for the resolver field productCount of type Category. + FieldArgs *ResolveCategoryProductCountArgs `protobuf:"bytes,2,opt,name=field_args,json=fieldArgs,proto3" json:"field_args,omitempty"` unknownFields protoimpl.UnknownFields sizeCache protoimpl.SizeCache } -func (x *ResolveCategoryCategoryMetricsRequest) Reset() { - *x = ResolveCategoryCategoryMetricsRequest{} +func (x *ResolveCategoryProductCountRequest) Reset() { + *x = ResolveCategoryProductCountRequest{} mi := &file_product_proto_msgTypes[125] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } -func (x *ResolveCategoryCategoryMetricsRequest) String() string { +func (x *ResolveCategoryProductCountRequest) String() string { return protoimpl.X.MessageStringOf(x) } -func (*ResolveCategoryCategoryMetricsRequest) ProtoMessage() {} +func (*ResolveCategoryProductCountRequest) ProtoMessage() {} -func (x *ResolveCategoryCategoryMetricsRequest) ProtoReflect() protoreflect.Message { +func (x *ResolveCategoryProductCountRequest) ProtoReflect() protoreflect.Message { mi := &file_product_proto_msgTypes[125] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) @@ -5737,35 +5805,503 @@ func (x *ResolveCategoryCategoryMetricsRequest) ProtoReflect() protoreflect.Mess return mi.MessageOf(x) } -// Deprecated: Use ResolveCategoryCategoryMetricsRequest.ProtoReflect.Descriptor instead. -func (*ResolveCategoryCategoryMetricsRequest) Descriptor() ([]byte, []int) { +// Deprecated: Use ResolveCategoryProductCountRequest.ProtoReflect.Descriptor instead. +func (*ResolveCategoryProductCountRequest) Descriptor() ([]byte, []int) { return file_product_proto_rawDescGZIP(), []int{125} } -func (x *ResolveCategoryCategoryMetricsRequest) GetContext() []*ResolveCategoryCategoryMetricsContext { +func (x *ResolveCategoryProductCountRequest) GetContext() []*ResolveCategoryProductCountContext { if x != nil { return x.Context } return nil } -func (x *ResolveCategoryCategoryMetricsRequest) GetFieldArgs() *ResolveCategoryCategoryMetricsArgs { +func (x *ResolveCategoryProductCountRequest) GetFieldArgs() *ResolveCategoryProductCountArgs { if x != nil { return x.FieldArgs } return nil } -type ResolveCategoryCategoryMetricsResult struct { - state protoimpl.MessageState `protogen:"open.v1"` - CategoryMetrics *CategoryMetrics `protobuf:"bytes,1,opt,name=category_metrics,json=categoryMetrics,proto3" json:"category_metrics,omitempty"` - unknownFields protoimpl.UnknownFields - sizeCache protoimpl.SizeCache +type ResolveCategoryProductCountResult struct { + state protoimpl.MessageState `protogen:"open.v1"` + ProductCount int32 `protobuf:"varint,1,opt,name=product_count,json=productCount,proto3" json:"product_count,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ResolveCategoryProductCountResult) Reset() { + *x = ResolveCategoryProductCountResult{} + mi := &file_product_proto_msgTypes[126] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ResolveCategoryProductCountResult) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ResolveCategoryProductCountResult) ProtoMessage() {} + +func (x *ResolveCategoryProductCountResult) ProtoReflect() protoreflect.Message { + mi := &file_product_proto_msgTypes[126] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ResolveCategoryProductCountResult.ProtoReflect.Descriptor instead. +func (*ResolveCategoryProductCountResult) Descriptor() ([]byte, []int) { + return file_product_proto_rawDescGZIP(), []int{126} +} + +func (x *ResolveCategoryProductCountResult) GetProductCount() int32 { + if x != nil { + return x.ProductCount + } + return 0 +} + +type ResolveCategoryProductCountResponse struct { + state protoimpl.MessageState `protogen:"open.v1"` + Result []*ResolveCategoryProductCountResult `protobuf:"bytes,1,rep,name=result,proto3" json:"result,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ResolveCategoryProductCountResponse) Reset() { + *x = ResolveCategoryProductCountResponse{} + mi := &file_product_proto_msgTypes[127] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ResolveCategoryProductCountResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ResolveCategoryProductCountResponse) ProtoMessage() {} + +func (x *ResolveCategoryProductCountResponse) ProtoReflect() protoreflect.Message { + mi := &file_product_proto_msgTypes[127] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ResolveCategoryProductCountResponse.ProtoReflect.Descriptor instead. +func (*ResolveCategoryProductCountResponse) Descriptor() ([]byte, []int) { + return file_product_proto_rawDescGZIP(), []int{127} +} + +func (x *ResolveCategoryProductCountResponse) GetResult() []*ResolveCategoryProductCountResult { + if x != nil { + return x.Result + } + return nil +} + +type ResolveCategoryPopularityScoreArgs struct { + state protoimpl.MessageState `protogen:"open.v1"` + Threshold *wrapperspb.Int32Value `protobuf:"bytes,1,opt,name=threshold,proto3" json:"threshold,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ResolveCategoryPopularityScoreArgs) Reset() { + *x = ResolveCategoryPopularityScoreArgs{} + mi := &file_product_proto_msgTypes[128] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ResolveCategoryPopularityScoreArgs) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ResolveCategoryPopularityScoreArgs) ProtoMessage() {} + +func (x *ResolveCategoryPopularityScoreArgs) ProtoReflect() protoreflect.Message { + mi := &file_product_proto_msgTypes[128] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ResolveCategoryPopularityScoreArgs.ProtoReflect.Descriptor instead. +func (*ResolveCategoryPopularityScoreArgs) Descriptor() ([]byte, []int) { + return file_product_proto_rawDescGZIP(), []int{128} +} + +func (x *ResolveCategoryPopularityScoreArgs) GetThreshold() *wrapperspb.Int32Value { + if x != nil { + return x.Threshold + } + return nil +} + +type ResolveCategoryPopularityScoreContext struct { + state protoimpl.MessageState `protogen:"open.v1"` + Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ResolveCategoryPopularityScoreContext) Reset() { + *x = ResolveCategoryPopularityScoreContext{} + mi := &file_product_proto_msgTypes[129] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ResolveCategoryPopularityScoreContext) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ResolveCategoryPopularityScoreContext) ProtoMessage() {} + +func (x *ResolveCategoryPopularityScoreContext) ProtoReflect() protoreflect.Message { + mi := &file_product_proto_msgTypes[129] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ResolveCategoryPopularityScoreContext.ProtoReflect.Descriptor instead. +func (*ResolveCategoryPopularityScoreContext) Descriptor() ([]byte, []int) { + return file_product_proto_rawDescGZIP(), []int{129} +} + +func (x *ResolveCategoryPopularityScoreContext) GetId() string { + if x != nil { + return x.Id + } + return "" +} + +type ResolveCategoryPopularityScoreRequest struct { + state protoimpl.MessageState `protogen:"open.v1"` + // context provides the resolver context for the field popularityScore of type Category. + Context []*ResolveCategoryPopularityScoreContext `protobuf:"bytes,1,rep,name=context,proto3" json:"context,omitempty"` + // field_args provides the arguments for the resolver field popularityScore of type Category. + FieldArgs *ResolveCategoryPopularityScoreArgs `protobuf:"bytes,2,opt,name=field_args,json=fieldArgs,proto3" json:"field_args,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ResolveCategoryPopularityScoreRequest) Reset() { + *x = ResolveCategoryPopularityScoreRequest{} + mi := &file_product_proto_msgTypes[130] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ResolveCategoryPopularityScoreRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ResolveCategoryPopularityScoreRequest) ProtoMessage() {} + +func (x *ResolveCategoryPopularityScoreRequest) ProtoReflect() protoreflect.Message { + mi := &file_product_proto_msgTypes[130] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ResolveCategoryPopularityScoreRequest.ProtoReflect.Descriptor instead. +func (*ResolveCategoryPopularityScoreRequest) Descriptor() ([]byte, []int) { + return file_product_proto_rawDescGZIP(), []int{130} +} + +func (x *ResolveCategoryPopularityScoreRequest) GetContext() []*ResolveCategoryPopularityScoreContext { + if x != nil { + return x.Context + } + return nil +} + +func (x *ResolveCategoryPopularityScoreRequest) GetFieldArgs() *ResolveCategoryPopularityScoreArgs { + if x != nil { + return x.FieldArgs + } + return nil +} + +type ResolveCategoryPopularityScoreResult struct { + state protoimpl.MessageState `protogen:"open.v1"` + PopularityScore *wrapperspb.Int32Value `protobuf:"bytes,1,opt,name=popularity_score,json=popularityScore,proto3" json:"popularity_score,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ResolveCategoryPopularityScoreResult) Reset() { + *x = ResolveCategoryPopularityScoreResult{} + mi := &file_product_proto_msgTypes[131] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ResolveCategoryPopularityScoreResult) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ResolveCategoryPopularityScoreResult) ProtoMessage() {} + +func (x *ResolveCategoryPopularityScoreResult) ProtoReflect() protoreflect.Message { + mi := &file_product_proto_msgTypes[131] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ResolveCategoryPopularityScoreResult.ProtoReflect.Descriptor instead. +func (*ResolveCategoryPopularityScoreResult) Descriptor() ([]byte, []int) { + return file_product_proto_rawDescGZIP(), []int{131} +} + +func (x *ResolveCategoryPopularityScoreResult) GetPopularityScore() *wrapperspb.Int32Value { + if x != nil { + return x.PopularityScore + } + return nil +} + +type ResolveCategoryPopularityScoreResponse struct { + state protoimpl.MessageState `protogen:"open.v1"` + Result []*ResolveCategoryPopularityScoreResult `protobuf:"bytes,1,rep,name=result,proto3" json:"result,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ResolveCategoryPopularityScoreResponse) Reset() { + *x = ResolveCategoryPopularityScoreResponse{} + mi := &file_product_proto_msgTypes[132] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ResolveCategoryPopularityScoreResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ResolveCategoryPopularityScoreResponse) ProtoMessage() {} + +func (x *ResolveCategoryPopularityScoreResponse) ProtoReflect() protoreflect.Message { + mi := &file_product_proto_msgTypes[132] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ResolveCategoryPopularityScoreResponse.ProtoReflect.Descriptor instead. +func (*ResolveCategoryPopularityScoreResponse) Descriptor() ([]byte, []int) { + return file_product_proto_rawDescGZIP(), []int{132} +} + +func (x *ResolveCategoryPopularityScoreResponse) GetResult() []*ResolveCategoryPopularityScoreResult { + if x != nil { + return x.Result + } + return nil +} + +type ResolveCategoryCategoryMetricsArgs struct { + state protoimpl.MessageState `protogen:"open.v1"` + MetricType string `protobuf:"bytes,1,opt,name=metric_type,json=metricType,proto3" json:"metric_type,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ResolveCategoryCategoryMetricsArgs) Reset() { + *x = ResolveCategoryCategoryMetricsArgs{} + mi := &file_product_proto_msgTypes[133] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ResolveCategoryCategoryMetricsArgs) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ResolveCategoryCategoryMetricsArgs) ProtoMessage() {} + +func (x *ResolveCategoryCategoryMetricsArgs) ProtoReflect() protoreflect.Message { + mi := &file_product_proto_msgTypes[133] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ResolveCategoryCategoryMetricsArgs.ProtoReflect.Descriptor instead. +func (*ResolveCategoryCategoryMetricsArgs) Descriptor() ([]byte, []int) { + return file_product_proto_rawDescGZIP(), []int{133} +} + +func (x *ResolveCategoryCategoryMetricsArgs) GetMetricType() string { + if x != nil { + return x.MetricType + } + return "" +} + +type ResolveCategoryCategoryMetricsContext struct { + state protoimpl.MessageState `protogen:"open.v1"` + Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` + Name string `protobuf:"bytes,2,opt,name=name,proto3" json:"name,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ResolveCategoryCategoryMetricsContext) Reset() { + *x = ResolveCategoryCategoryMetricsContext{} + mi := &file_product_proto_msgTypes[134] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ResolveCategoryCategoryMetricsContext) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ResolveCategoryCategoryMetricsContext) ProtoMessage() {} + +func (x *ResolveCategoryCategoryMetricsContext) ProtoReflect() protoreflect.Message { + mi := &file_product_proto_msgTypes[134] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ResolveCategoryCategoryMetricsContext.ProtoReflect.Descriptor instead. +func (*ResolveCategoryCategoryMetricsContext) Descriptor() ([]byte, []int) { + return file_product_proto_rawDescGZIP(), []int{134} +} + +func (x *ResolveCategoryCategoryMetricsContext) GetId() string { + if x != nil { + return x.Id + } + return "" +} + +func (x *ResolveCategoryCategoryMetricsContext) GetName() string { + if x != nil { + return x.Name + } + return "" +} + +type ResolveCategoryCategoryMetricsRequest struct { + state protoimpl.MessageState `protogen:"open.v1"` + // context provides the resolver context for the field categoryMetrics of type Category. + Context []*ResolveCategoryCategoryMetricsContext `protobuf:"bytes,1,rep,name=context,proto3" json:"context,omitempty"` + // field_args provides the arguments for the resolver field categoryMetrics of type Category. + FieldArgs *ResolveCategoryCategoryMetricsArgs `protobuf:"bytes,2,opt,name=field_args,json=fieldArgs,proto3" json:"field_args,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ResolveCategoryCategoryMetricsRequest) Reset() { + *x = ResolveCategoryCategoryMetricsRequest{} + mi := &file_product_proto_msgTypes[135] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ResolveCategoryCategoryMetricsRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ResolveCategoryCategoryMetricsRequest) ProtoMessage() {} + +func (x *ResolveCategoryCategoryMetricsRequest) ProtoReflect() protoreflect.Message { + mi := &file_product_proto_msgTypes[135] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ResolveCategoryCategoryMetricsRequest.ProtoReflect.Descriptor instead. +func (*ResolveCategoryCategoryMetricsRequest) Descriptor() ([]byte, []int) { + return file_product_proto_rawDescGZIP(), []int{135} +} + +func (x *ResolveCategoryCategoryMetricsRequest) GetContext() []*ResolveCategoryCategoryMetricsContext { + if x != nil { + return x.Context + } + return nil +} + +func (x *ResolveCategoryCategoryMetricsRequest) GetFieldArgs() *ResolveCategoryCategoryMetricsArgs { + if x != nil { + return x.FieldArgs + } + return nil +} + +type ResolveCategoryCategoryMetricsResult struct { + state protoimpl.MessageState `protogen:"open.v1"` + CategoryMetrics *CategoryMetrics `protobuf:"bytes,1,opt,name=category_metrics,json=categoryMetrics,proto3" json:"category_metrics,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *ResolveCategoryCategoryMetricsResult) Reset() { *x = ResolveCategoryCategoryMetricsResult{} - mi := &file_product_proto_msgTypes[126] + mi := &file_product_proto_msgTypes[136] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5777,7 +6313,7 @@ func (x *ResolveCategoryCategoryMetricsResult) String() string { func (*ResolveCategoryCategoryMetricsResult) ProtoMessage() {} func (x *ResolveCategoryCategoryMetricsResult) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[126] + mi := &file_product_proto_msgTypes[136] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5790,7 +6326,7 @@ func (x *ResolveCategoryCategoryMetricsResult) ProtoReflect() protoreflect.Messa // Deprecated: Use ResolveCategoryCategoryMetricsResult.ProtoReflect.Descriptor instead. func (*ResolveCategoryCategoryMetricsResult) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{126} + return file_product_proto_rawDescGZIP(), []int{136} } func (x *ResolveCategoryCategoryMetricsResult) GetCategoryMetrics() *CategoryMetrics { @@ -5809,7 +6345,7 @@ type ResolveCategoryCategoryMetricsResponse struct { func (x *ResolveCategoryCategoryMetricsResponse) Reset() { *x = ResolveCategoryCategoryMetricsResponse{} - mi := &file_product_proto_msgTypes[127] + mi := &file_product_proto_msgTypes[137] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5821,7 +6357,7 @@ func (x *ResolveCategoryCategoryMetricsResponse) String() string { func (*ResolveCategoryCategoryMetricsResponse) ProtoMessage() {} func (x *ResolveCategoryCategoryMetricsResponse) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[127] + mi := &file_product_proto_msgTypes[137] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5834,7 +6370,7 @@ func (x *ResolveCategoryCategoryMetricsResponse) ProtoReflect() protoreflect.Mes // Deprecated: Use ResolveCategoryCategoryMetricsResponse.ProtoReflect.Descriptor instead. func (*ResolveCategoryCategoryMetricsResponse) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{127} + return file_product_proto_rawDescGZIP(), []int{137} } func (x *ResolveCategoryCategoryMetricsResponse) GetResult() []*ResolveCategoryCategoryMetricsResult { @@ -5853,7 +6389,7 @@ type ResolveSubcategoryItemCountArgs struct { func (x *ResolveSubcategoryItemCountArgs) Reset() { *x = ResolveSubcategoryItemCountArgs{} - mi := &file_product_proto_msgTypes[128] + mi := &file_product_proto_msgTypes[138] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5865,7 +6401,7 @@ func (x *ResolveSubcategoryItemCountArgs) String() string { func (*ResolveSubcategoryItemCountArgs) ProtoMessage() {} func (x *ResolveSubcategoryItemCountArgs) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[128] + mi := &file_product_proto_msgTypes[138] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5878,7 +6414,7 @@ func (x *ResolveSubcategoryItemCountArgs) ProtoReflect() protoreflect.Message { // Deprecated: Use ResolveSubcategoryItemCountArgs.ProtoReflect.Descriptor instead. func (*ResolveSubcategoryItemCountArgs) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{128} + return file_product_proto_rawDescGZIP(), []int{138} } func (x *ResolveSubcategoryItemCountArgs) GetFilters() *SubcategoryItemFilter { @@ -5897,7 +6433,7 @@ type ResolveSubcategoryItemCountContext struct { func (x *ResolveSubcategoryItemCountContext) Reset() { *x = ResolveSubcategoryItemCountContext{} - mi := &file_product_proto_msgTypes[129] + mi := &file_product_proto_msgTypes[139] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5909,7 +6445,7 @@ func (x *ResolveSubcategoryItemCountContext) String() string { func (*ResolveSubcategoryItemCountContext) ProtoMessage() {} func (x *ResolveSubcategoryItemCountContext) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[129] + mi := &file_product_proto_msgTypes[139] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5922,7 +6458,7 @@ func (x *ResolveSubcategoryItemCountContext) ProtoReflect() protoreflect.Message // Deprecated: Use ResolveSubcategoryItemCountContext.ProtoReflect.Descriptor instead. func (*ResolveSubcategoryItemCountContext) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{129} + return file_product_proto_rawDescGZIP(), []int{139} } func (x *ResolveSubcategoryItemCountContext) GetId() string { @@ -5944,7 +6480,7 @@ type ResolveSubcategoryItemCountRequest struct { func (x *ResolveSubcategoryItemCountRequest) Reset() { *x = ResolveSubcategoryItemCountRequest{} - mi := &file_product_proto_msgTypes[130] + mi := &file_product_proto_msgTypes[140] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5956,7 +6492,7 @@ func (x *ResolveSubcategoryItemCountRequest) String() string { func (*ResolveSubcategoryItemCountRequest) ProtoMessage() {} func (x *ResolveSubcategoryItemCountRequest) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[130] + mi := &file_product_proto_msgTypes[140] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5969,7 +6505,7 @@ func (x *ResolveSubcategoryItemCountRequest) ProtoReflect() protoreflect.Message // Deprecated: Use ResolveSubcategoryItemCountRequest.ProtoReflect.Descriptor instead. func (*ResolveSubcategoryItemCountRequest) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{130} + return file_product_proto_rawDescGZIP(), []int{140} } func (x *ResolveSubcategoryItemCountRequest) GetContext() []*ResolveSubcategoryItemCountContext { @@ -5995,7 +6531,7 @@ type ResolveSubcategoryItemCountResult struct { func (x *ResolveSubcategoryItemCountResult) Reset() { *x = ResolveSubcategoryItemCountResult{} - mi := &file_product_proto_msgTypes[131] + mi := &file_product_proto_msgTypes[141] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6007,7 +6543,7 @@ func (x *ResolveSubcategoryItemCountResult) String() string { func (*ResolveSubcategoryItemCountResult) ProtoMessage() {} func (x *ResolveSubcategoryItemCountResult) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[131] + mi := &file_product_proto_msgTypes[141] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6020,7 +6556,7 @@ func (x *ResolveSubcategoryItemCountResult) ProtoReflect() protoreflect.Message // Deprecated: Use ResolveSubcategoryItemCountResult.ProtoReflect.Descriptor instead. func (*ResolveSubcategoryItemCountResult) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{131} + return file_product_proto_rawDescGZIP(), []int{141} } func (x *ResolveSubcategoryItemCountResult) GetItemCount() int32 { @@ -6039,7 +6575,7 @@ type ResolveSubcategoryItemCountResponse struct { func (x *ResolveSubcategoryItemCountResponse) Reset() { *x = ResolveSubcategoryItemCountResponse{} - mi := &file_product_proto_msgTypes[132] + mi := &file_product_proto_msgTypes[142] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6051,7 +6587,7 @@ func (x *ResolveSubcategoryItemCountResponse) String() string { func (*ResolveSubcategoryItemCountResponse) ProtoMessage() {} func (x *ResolveSubcategoryItemCountResponse) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[132] + mi := &file_product_proto_msgTypes[142] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6064,7 +6600,7 @@ func (x *ResolveSubcategoryItemCountResponse) ProtoReflect() protoreflect.Messag // Deprecated: Use ResolveSubcategoryItemCountResponse.ProtoReflect.Descriptor instead. func (*ResolveSubcategoryItemCountResponse) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{132} + return file_product_proto_rawDescGZIP(), []int{142} } func (x *ResolveSubcategoryItemCountResponse) GetResult() []*ResolveSubcategoryItemCountResult { @@ -6085,7 +6621,7 @@ type Product struct { func (x *Product) Reset() { *x = Product{} - mi := &file_product_proto_msgTypes[133] + mi := &file_product_proto_msgTypes[143] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6097,7 +6633,7 @@ func (x *Product) String() string { func (*Product) ProtoMessage() {} func (x *Product) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[133] + mi := &file_product_proto_msgTypes[143] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6110,7 +6646,7 @@ func (x *Product) ProtoReflect() protoreflect.Message { // Deprecated: Use Product.ProtoReflect.Descriptor instead. func (*Product) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{133} + return file_product_proto_rawDescGZIP(), []int{143} } func (x *Product) GetId() string { @@ -6145,7 +6681,7 @@ type Storage struct { func (x *Storage) Reset() { *x = Storage{} - mi := &file_product_proto_msgTypes[134] + mi := &file_product_proto_msgTypes[144] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6157,7 +6693,7 @@ func (x *Storage) String() string { func (*Storage) ProtoMessage() {} func (x *Storage) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[134] + mi := &file_product_proto_msgTypes[144] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6170,7 +6706,7 @@ func (x *Storage) ProtoReflect() protoreflect.Message { // Deprecated: Use Storage.ProtoReflect.Descriptor instead. func (*Storage) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{134} + return file_product_proto_rawDescGZIP(), []int{144} } func (x *Storage) GetId() string { @@ -6205,7 +6741,7 @@ type Warehouse struct { func (x *Warehouse) Reset() { *x = Warehouse{} - mi := &file_product_proto_msgTypes[135] + mi := &file_product_proto_msgTypes[145] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6217,7 +6753,7 @@ func (x *Warehouse) String() string { func (*Warehouse) ProtoMessage() {} func (x *Warehouse) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[135] + mi := &file_product_proto_msgTypes[145] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6230,7 +6766,7 @@ func (x *Warehouse) ProtoReflect() protoreflect.Message { // Deprecated: Use Warehouse.ProtoReflect.Descriptor instead. func (*Warehouse) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{135} + return file_product_proto_rawDescGZIP(), []int{145} } func (x *Warehouse) GetId() string { @@ -6264,7 +6800,7 @@ type User struct { func (x *User) Reset() { *x = User{} - mi := &file_product_proto_msgTypes[136] + mi := &file_product_proto_msgTypes[146] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6276,7 +6812,7 @@ func (x *User) String() string { func (*User) ProtoMessage() {} func (x *User) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[136] + mi := &file_product_proto_msgTypes[146] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6289,7 +6825,7 @@ func (x *User) ProtoReflect() protoreflect.Message { // Deprecated: Use User.ProtoReflect.Descriptor instead. func (*User) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{136} + return file_product_proto_rawDescGZIP(), []int{146} } func (x *User) GetId() string { @@ -6317,7 +6853,7 @@ type NestedTypeA struct { func (x *NestedTypeA) Reset() { *x = NestedTypeA{} - mi := &file_product_proto_msgTypes[137] + mi := &file_product_proto_msgTypes[147] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6329,7 +6865,7 @@ func (x *NestedTypeA) String() string { func (*NestedTypeA) ProtoMessage() {} func (x *NestedTypeA) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[137] + mi := &file_product_proto_msgTypes[147] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6342,7 +6878,7 @@ func (x *NestedTypeA) ProtoReflect() protoreflect.Message { // Deprecated: Use NestedTypeA.ProtoReflect.Descriptor instead. func (*NestedTypeA) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{137} + return file_product_proto_rawDescGZIP(), []int{147} } func (x *NestedTypeA) GetId() string { @@ -6377,7 +6913,7 @@ type RecursiveType struct { func (x *RecursiveType) Reset() { *x = RecursiveType{} - mi := &file_product_proto_msgTypes[138] + mi := &file_product_proto_msgTypes[148] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6389,7 +6925,7 @@ func (x *RecursiveType) String() string { func (*RecursiveType) ProtoMessage() {} func (x *RecursiveType) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[138] + mi := &file_product_proto_msgTypes[148] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6402,7 +6938,7 @@ func (x *RecursiveType) ProtoReflect() protoreflect.Message { // Deprecated: Use RecursiveType.ProtoReflect.Descriptor instead. func (*RecursiveType) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{138} + return file_product_proto_rawDescGZIP(), []int{148} } func (x *RecursiveType) GetId() string { @@ -6438,7 +6974,7 @@ type TypeWithMultipleFilterFields struct { func (x *TypeWithMultipleFilterFields) Reset() { *x = TypeWithMultipleFilterFields{} - mi := &file_product_proto_msgTypes[139] + mi := &file_product_proto_msgTypes[149] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6450,7 +6986,7 @@ func (x *TypeWithMultipleFilterFields) String() string { func (*TypeWithMultipleFilterFields) ProtoMessage() {} func (x *TypeWithMultipleFilterFields) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[139] + mi := &file_product_proto_msgTypes[149] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6463,7 +6999,7 @@ func (x *TypeWithMultipleFilterFields) ProtoReflect() protoreflect.Message { // Deprecated: Use TypeWithMultipleFilterFields.ProtoReflect.Descriptor instead. func (*TypeWithMultipleFilterFields) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{139} + return file_product_proto_rawDescGZIP(), []int{149} } func (x *TypeWithMultipleFilterFields) GetId() string { @@ -6504,7 +7040,7 @@ type FilterTypeInput struct { func (x *FilterTypeInput) Reset() { *x = FilterTypeInput{} - mi := &file_product_proto_msgTypes[140] + mi := &file_product_proto_msgTypes[150] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6516,7 +7052,7 @@ func (x *FilterTypeInput) String() string { func (*FilterTypeInput) ProtoMessage() {} func (x *FilterTypeInput) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[140] + mi := &file_product_proto_msgTypes[150] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6529,7 +7065,7 @@ func (x *FilterTypeInput) ProtoReflect() protoreflect.Message { // Deprecated: Use FilterTypeInput.ProtoReflect.Descriptor instead. func (*FilterTypeInput) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{140} + return file_product_proto_rawDescGZIP(), []int{150} } func (x *FilterTypeInput) GetFilterField_1() string { @@ -6555,7 +7091,7 @@ type ComplexFilterTypeInput struct { func (x *ComplexFilterTypeInput) Reset() { *x = ComplexFilterTypeInput{} - mi := &file_product_proto_msgTypes[141] + mi := &file_product_proto_msgTypes[151] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6567,7 +7103,7 @@ func (x *ComplexFilterTypeInput) String() string { func (*ComplexFilterTypeInput) ProtoMessage() {} func (x *ComplexFilterTypeInput) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[141] + mi := &file_product_proto_msgTypes[151] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6580,7 +7116,7 @@ func (x *ComplexFilterTypeInput) ProtoReflect() protoreflect.Message { // Deprecated: Use ComplexFilterTypeInput.ProtoReflect.Descriptor instead. func (*ComplexFilterTypeInput) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{141} + return file_product_proto_rawDescGZIP(), []int{151} } func (x *ComplexFilterTypeInput) GetFilter() *FilterType { @@ -6600,7 +7136,7 @@ type TypeWithComplexFilterInput struct { func (x *TypeWithComplexFilterInput) Reset() { *x = TypeWithComplexFilterInput{} - mi := &file_product_proto_msgTypes[142] + mi := &file_product_proto_msgTypes[152] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6612,7 +7148,7 @@ func (x *TypeWithComplexFilterInput) String() string { func (*TypeWithComplexFilterInput) ProtoMessage() {} func (x *TypeWithComplexFilterInput) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[142] + mi := &file_product_proto_msgTypes[152] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6625,7 +7161,7 @@ func (x *TypeWithComplexFilterInput) ProtoReflect() protoreflect.Message { // Deprecated: Use TypeWithComplexFilterInput.ProtoReflect.Descriptor instead. func (*TypeWithComplexFilterInput) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{142} + return file_product_proto_rawDescGZIP(), []int{152} } func (x *TypeWithComplexFilterInput) GetId() string { @@ -6653,7 +7189,7 @@ type OrderInput struct { func (x *OrderInput) Reset() { *x = OrderInput{} - mi := &file_product_proto_msgTypes[143] + mi := &file_product_proto_msgTypes[153] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6665,7 +7201,7 @@ func (x *OrderInput) String() string { func (*OrderInput) ProtoMessage() {} func (x *OrderInput) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[143] + mi := &file_product_proto_msgTypes[153] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6678,7 +7214,7 @@ func (x *OrderInput) ProtoReflect() protoreflect.Message { // Deprecated: Use OrderInput.ProtoReflect.Descriptor instead. func (*OrderInput) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{143} + return file_product_proto_rawDescGZIP(), []int{153} } func (x *OrderInput) GetOrderId() string { @@ -6714,7 +7250,7 @@ type Order struct { func (x *Order) Reset() { *x = Order{} - mi := &file_product_proto_msgTypes[144] + mi := &file_product_proto_msgTypes[154] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6726,7 +7262,7 @@ func (x *Order) String() string { func (*Order) ProtoMessage() {} func (x *Order) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[144] + mi := &file_product_proto_msgTypes[154] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6739,7 +7275,7 @@ func (x *Order) ProtoReflect() protoreflect.Message { // Deprecated: Use Order.ProtoReflect.Descriptor instead. func (*Order) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{144} + return file_product_proto_rawDescGZIP(), []int{154} } func (x *Order) GetOrderId() string { @@ -6782,7 +7318,7 @@ type Category struct { func (x *Category) Reset() { *x = Category{} - mi := &file_product_proto_msgTypes[145] + mi := &file_product_proto_msgTypes[155] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6794,7 +7330,7 @@ func (x *Category) String() string { func (*Category) ProtoMessage() {} func (x *Category) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[145] + mi := &file_product_proto_msgTypes[155] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6807,7 +7343,7 @@ func (x *Category) ProtoReflect() protoreflect.Message { // Deprecated: Use Category.ProtoReflect.Descriptor instead. func (*Category) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{145} + return file_product_proto_rawDescGZIP(), []int{155} } func (x *Category) GetId() string { @@ -6848,7 +7384,7 @@ type CategoryFilter struct { func (x *CategoryFilter) Reset() { *x = CategoryFilter{} - mi := &file_product_proto_msgTypes[146] + mi := &file_product_proto_msgTypes[156] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6860,7 +7396,7 @@ func (x *CategoryFilter) String() string { func (*CategoryFilter) ProtoMessage() {} func (x *CategoryFilter) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[146] + mi := &file_product_proto_msgTypes[156] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6873,7 +7409,7 @@ func (x *CategoryFilter) ProtoReflect() protoreflect.Message { // Deprecated: Use CategoryFilter.ProtoReflect.Descriptor instead. func (*CategoryFilter) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{146} + return file_product_proto_rawDescGZIP(), []int{156} } func (x *CategoryFilter) GetCategory() CategoryKind { @@ -6903,7 +7439,7 @@ type Animal struct { func (x *Animal) Reset() { *x = Animal{} - mi := &file_product_proto_msgTypes[147] + mi := &file_product_proto_msgTypes[157] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6915,7 +7451,7 @@ func (x *Animal) String() string { func (*Animal) ProtoMessage() {} func (x *Animal) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[147] + mi := &file_product_proto_msgTypes[157] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6928,7 +7464,7 @@ func (x *Animal) ProtoReflect() protoreflect.Message { // Deprecated: Use Animal.ProtoReflect.Descriptor instead. func (*Animal) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{147} + return file_product_proto_rawDescGZIP(), []int{157} } func (x *Animal) GetInstance() isAnimal_Instance { @@ -6982,7 +7518,7 @@ type SearchInput struct { func (x *SearchInput) Reset() { *x = SearchInput{} - mi := &file_product_proto_msgTypes[148] + mi := &file_product_proto_msgTypes[158] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6994,7 +7530,7 @@ func (x *SearchInput) String() string { func (*SearchInput) ProtoMessage() {} func (x *SearchInput) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[148] + mi := &file_product_proto_msgTypes[158] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7007,7 +7543,7 @@ func (x *SearchInput) ProtoReflect() protoreflect.Message { // Deprecated: Use SearchInput.ProtoReflect.Descriptor instead. func (*SearchInput) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{148} + return file_product_proto_rawDescGZIP(), []int{158} } func (x *SearchInput) GetQuery() string { @@ -7038,7 +7574,7 @@ type SearchResult struct { func (x *SearchResult) Reset() { *x = SearchResult{} - mi := &file_product_proto_msgTypes[149] + mi := &file_product_proto_msgTypes[159] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7050,7 +7586,7 @@ func (x *SearchResult) String() string { func (*SearchResult) ProtoMessage() {} func (x *SearchResult) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[149] + mi := &file_product_proto_msgTypes[159] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7063,7 +7599,7 @@ func (x *SearchResult) ProtoReflect() protoreflect.Message { // Deprecated: Use SearchResult.ProtoReflect.Descriptor instead. func (*SearchResult) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{149} + return file_product_proto_rawDescGZIP(), []int{159} } func (x *SearchResult) GetValue() isSearchResult_Value { @@ -7138,7 +7674,7 @@ type NullableFieldsType struct { func (x *NullableFieldsType) Reset() { *x = NullableFieldsType{} - mi := &file_product_proto_msgTypes[150] + mi := &file_product_proto_msgTypes[160] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7150,7 +7686,7 @@ func (x *NullableFieldsType) String() string { func (*NullableFieldsType) ProtoMessage() {} func (x *NullableFieldsType) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[150] + mi := &file_product_proto_msgTypes[160] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7163,7 +7699,7 @@ func (x *NullableFieldsType) ProtoReflect() protoreflect.Message { // Deprecated: Use NullableFieldsType.ProtoReflect.Descriptor instead. func (*NullableFieldsType) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{150} + return file_product_proto_rawDescGZIP(), []int{160} } func (x *NullableFieldsType) GetId() string { @@ -7233,7 +7769,7 @@ type NullableFieldsFilter struct { func (x *NullableFieldsFilter) Reset() { *x = NullableFieldsFilter{} - mi := &file_product_proto_msgTypes[151] + mi := &file_product_proto_msgTypes[161] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7245,7 +7781,7 @@ func (x *NullableFieldsFilter) String() string { func (*NullableFieldsFilter) ProtoMessage() {} func (x *NullableFieldsFilter) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[151] + mi := &file_product_proto_msgTypes[161] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7258,7 +7794,7 @@ func (x *NullableFieldsFilter) ProtoReflect() protoreflect.Message { // Deprecated: Use NullableFieldsFilter.ProtoReflect.Descriptor instead. func (*NullableFieldsFilter) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{151} + return file_product_proto_rawDescGZIP(), []int{161} } func (x *NullableFieldsFilter) GetName() *wrapperspb.StringValue { @@ -7310,7 +7846,7 @@ type BlogPost struct { func (x *BlogPost) Reset() { *x = BlogPost{} - mi := &file_product_proto_msgTypes[152] + mi := &file_product_proto_msgTypes[162] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7322,7 +7858,7 @@ func (x *BlogPost) String() string { func (*BlogPost) ProtoMessage() {} func (x *BlogPost) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[152] + mi := &file_product_proto_msgTypes[162] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7335,7 +7871,7 @@ func (x *BlogPost) ProtoReflect() protoreflect.Message { // Deprecated: Use BlogPost.ProtoReflect.Descriptor instead. func (*BlogPost) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{152} + return file_product_proto_rawDescGZIP(), []int{162} } func (x *BlogPost) GetId() string { @@ -7489,7 +8025,7 @@ type BlogPostFilter struct { func (x *BlogPostFilter) Reset() { *x = BlogPostFilter{} - mi := &file_product_proto_msgTypes[153] + mi := &file_product_proto_msgTypes[163] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7501,7 +8037,7 @@ func (x *BlogPostFilter) String() string { func (*BlogPostFilter) ProtoMessage() {} func (x *BlogPostFilter) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[153] + mi := &file_product_proto_msgTypes[163] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7514,7 +8050,7 @@ func (x *BlogPostFilter) ProtoReflect() protoreflect.Message { // Deprecated: Use BlogPostFilter.ProtoReflect.Descriptor instead. func (*BlogPostFilter) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{153} + return file_product_proto_rawDescGZIP(), []int{163} } func (x *BlogPostFilter) GetTitle() *wrapperspb.StringValue { @@ -7561,7 +8097,7 @@ type Author struct { func (x *Author) Reset() { *x = Author{} - mi := &file_product_proto_msgTypes[154] + mi := &file_product_proto_msgTypes[164] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7573,7 +8109,7 @@ func (x *Author) String() string { func (*Author) ProtoMessage() {} func (x *Author) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[154] + mi := &file_product_proto_msgTypes[164] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7586,7 +8122,7 @@ func (x *Author) ProtoReflect() protoreflect.Message { // Deprecated: Use Author.ProtoReflect.Descriptor instead. func (*Author) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{154} + return file_product_proto_rawDescGZIP(), []int{164} } func (x *Author) GetId() string { @@ -7705,7 +8241,7 @@ type AuthorFilter struct { func (x *AuthorFilter) Reset() { *x = AuthorFilter{} - mi := &file_product_proto_msgTypes[155] + mi := &file_product_proto_msgTypes[165] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7717,7 +8253,7 @@ func (x *AuthorFilter) String() string { func (*AuthorFilter) ProtoMessage() {} func (x *AuthorFilter) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[155] + mi := &file_product_proto_msgTypes[165] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7730,7 +8266,7 @@ func (x *AuthorFilter) ProtoReflect() protoreflect.Message { // Deprecated: Use AuthorFilter.ProtoReflect.Descriptor instead. func (*AuthorFilter) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{155} + return file_product_proto_rawDescGZIP(), []int{165} } func (x *AuthorFilter) GetName() *wrapperspb.StringValue { @@ -7763,7 +8299,7 @@ type UserInput struct { func (x *UserInput) Reset() { *x = UserInput{} - mi := &file_product_proto_msgTypes[156] + mi := &file_product_proto_msgTypes[166] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7775,7 +8311,7 @@ func (x *UserInput) String() string { func (*UserInput) ProtoMessage() {} func (x *UserInput) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[156] + mi := &file_product_proto_msgTypes[166] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7788,7 +8324,7 @@ func (x *UserInput) ProtoReflect() protoreflect.Message { // Deprecated: Use UserInput.ProtoReflect.Descriptor instead. func (*UserInput) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{156} + return file_product_proto_rawDescGZIP(), []int{166} } func (x *UserInput) GetName() string { @@ -7808,7 +8344,7 @@ type ActionInput struct { func (x *ActionInput) Reset() { *x = ActionInput{} - mi := &file_product_proto_msgTypes[157] + mi := &file_product_proto_msgTypes[167] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7820,7 +8356,7 @@ func (x *ActionInput) String() string { func (*ActionInput) ProtoMessage() {} func (x *ActionInput) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[157] + mi := &file_product_proto_msgTypes[167] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7833,7 +8369,7 @@ func (x *ActionInput) ProtoReflect() protoreflect.Message { // Deprecated: Use ActionInput.ProtoReflect.Descriptor instead. func (*ActionInput) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{157} + return file_product_proto_rawDescGZIP(), []int{167} } func (x *ActionInput) GetType() string { @@ -7863,7 +8399,7 @@ type ActionResult struct { func (x *ActionResult) Reset() { *x = ActionResult{} - mi := &file_product_proto_msgTypes[158] + mi := &file_product_proto_msgTypes[168] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7875,7 +8411,7 @@ func (x *ActionResult) String() string { func (*ActionResult) ProtoMessage() {} func (x *ActionResult) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[158] + mi := &file_product_proto_msgTypes[168] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7888,7 +8424,7 @@ func (x *ActionResult) ProtoReflect() protoreflect.Message { // Deprecated: Use ActionResult.ProtoReflect.Descriptor instead. func (*ActionResult) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{158} + return file_product_proto_rawDescGZIP(), []int{168} } func (x *ActionResult) GetValue() isActionResult_Value { @@ -7947,7 +8483,7 @@ type NullableFieldsInput struct { func (x *NullableFieldsInput) Reset() { *x = NullableFieldsInput{} - mi := &file_product_proto_msgTypes[159] + mi := &file_product_proto_msgTypes[169] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7959,7 +8495,7 @@ func (x *NullableFieldsInput) String() string { func (*NullableFieldsInput) ProtoMessage() {} func (x *NullableFieldsInput) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[159] + mi := &file_product_proto_msgTypes[169] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7972,7 +8508,7 @@ func (x *NullableFieldsInput) ProtoReflect() protoreflect.Message { // Deprecated: Use NullableFieldsInput.ProtoReflect.Descriptor instead. func (*NullableFieldsInput) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{159} + return file_product_proto_rawDescGZIP(), []int{169} } func (x *NullableFieldsInput) GetName() string { @@ -8048,7 +8584,7 @@ type BlogPostInput struct { func (x *BlogPostInput) Reset() { *x = BlogPostInput{} - mi := &file_product_proto_msgTypes[160] + mi := &file_product_proto_msgTypes[170] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8060,7 +8596,7 @@ func (x *BlogPostInput) String() string { func (*BlogPostInput) ProtoMessage() {} func (x *BlogPostInput) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[160] + mi := &file_product_proto_msgTypes[170] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8073,7 +8609,7 @@ func (x *BlogPostInput) ProtoReflect() protoreflect.Message { // Deprecated: Use BlogPostInput.ProtoReflect.Descriptor instead. func (*BlogPostInput) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{160} + return file_product_proto_rawDescGZIP(), []int{170} } func (x *BlogPostInput) GetTitle() string { @@ -8206,7 +8742,7 @@ type AuthorInput struct { func (x *AuthorInput) Reset() { *x = AuthorInput{} - mi := &file_product_proto_msgTypes[161] + mi := &file_product_proto_msgTypes[171] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8218,7 +8754,7 @@ func (x *AuthorInput) String() string { func (*AuthorInput) ProtoMessage() {} func (x *AuthorInput) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[161] + mi := &file_product_proto_msgTypes[171] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8231,7 +8767,7 @@ func (x *AuthorInput) ProtoReflect() protoreflect.Message { // Deprecated: Use AuthorInput.ProtoReflect.Descriptor instead. func (*AuthorInput) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{161} + return file_product_proto_rawDescGZIP(), []int{171} } func (x *AuthorInput) GetName() string { @@ -8315,7 +8851,7 @@ type NestedTypeB struct { func (x *NestedTypeB) Reset() { *x = NestedTypeB{} - mi := &file_product_proto_msgTypes[162] + mi := &file_product_proto_msgTypes[172] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8327,7 +8863,7 @@ func (x *NestedTypeB) String() string { func (*NestedTypeB) ProtoMessage() {} func (x *NestedTypeB) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[162] + mi := &file_product_proto_msgTypes[172] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8340,7 +8876,7 @@ func (x *NestedTypeB) ProtoReflect() protoreflect.Message { // Deprecated: Use NestedTypeB.ProtoReflect.Descriptor instead. func (*NestedTypeB) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{162} + return file_product_proto_rawDescGZIP(), []int{172} } func (x *NestedTypeB) GetId() string { @@ -8374,7 +8910,7 @@ type NestedTypeC struct { func (x *NestedTypeC) Reset() { *x = NestedTypeC{} - mi := &file_product_proto_msgTypes[163] + mi := &file_product_proto_msgTypes[173] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8386,7 +8922,7 @@ func (x *NestedTypeC) String() string { func (*NestedTypeC) ProtoMessage() {} func (x *NestedTypeC) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[163] + mi := &file_product_proto_msgTypes[173] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8399,7 +8935,7 @@ func (x *NestedTypeC) ProtoReflect() protoreflect.Message { // Deprecated: Use NestedTypeC.ProtoReflect.Descriptor instead. func (*NestedTypeC) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{163} + return file_product_proto_rawDescGZIP(), []int{173} } func (x *NestedTypeC) GetId() string { @@ -8428,7 +8964,7 @@ type FilterType struct { func (x *FilterType) Reset() { *x = FilterType{} - mi := &file_product_proto_msgTypes[164] + mi := &file_product_proto_msgTypes[174] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8440,7 +8976,7 @@ func (x *FilterType) String() string { func (*FilterType) ProtoMessage() {} func (x *FilterType) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[164] + mi := &file_product_proto_msgTypes[174] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8453,7 +8989,7 @@ func (x *FilterType) ProtoReflect() protoreflect.Message { // Deprecated: Use FilterType.ProtoReflect.Descriptor instead. func (*FilterType) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{164} + return file_product_proto_rawDescGZIP(), []int{174} } func (x *FilterType) GetName() string { @@ -8494,7 +9030,7 @@ type Pagination struct { func (x *Pagination) Reset() { *x = Pagination{} - mi := &file_product_proto_msgTypes[165] + mi := &file_product_proto_msgTypes[175] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8506,7 +9042,7 @@ func (x *Pagination) String() string { func (*Pagination) ProtoMessage() {} func (x *Pagination) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[165] + mi := &file_product_proto_msgTypes[175] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8519,7 +9055,7 @@ func (x *Pagination) ProtoReflect() protoreflect.Message { // Deprecated: Use Pagination.ProtoReflect.Descriptor instead. func (*Pagination) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{165} + return file_product_proto_rawDescGZIP(), []int{175} } func (x *Pagination) GetPage() int32 { @@ -8547,7 +9083,7 @@ type OrderLineInput struct { func (x *OrderLineInput) Reset() { *x = OrderLineInput{} - mi := &file_product_proto_msgTypes[166] + mi := &file_product_proto_msgTypes[176] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8559,7 +9095,7 @@ func (x *OrderLineInput) String() string { func (*OrderLineInput) ProtoMessage() {} func (x *OrderLineInput) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[166] + mi := &file_product_proto_msgTypes[176] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8572,7 +9108,7 @@ func (x *OrderLineInput) ProtoReflect() protoreflect.Message { // Deprecated: Use OrderLineInput.ProtoReflect.Descriptor instead. func (*OrderLineInput) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{166} + return file_product_proto_rawDescGZIP(), []int{176} } func (x *OrderLineInput) GetProductId() string { @@ -8607,7 +9143,7 @@ type OrderLine struct { func (x *OrderLine) Reset() { *x = OrderLine{} - mi := &file_product_proto_msgTypes[167] + mi := &file_product_proto_msgTypes[177] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8619,7 +9155,7 @@ func (x *OrderLine) String() string { func (*OrderLine) ProtoMessage() {} func (x *OrderLine) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[167] + mi := &file_product_proto_msgTypes[177] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8632,7 +9168,7 @@ func (x *OrderLine) ProtoReflect() protoreflect.Message { // Deprecated: Use OrderLine.ProtoReflect.Descriptor instead. func (*OrderLine) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{167} + return file_product_proto_rawDescGZIP(), []int{177} } func (x *OrderLine) GetProductId() string { @@ -8668,7 +9204,7 @@ type Subcategory struct { func (x *Subcategory) Reset() { *x = Subcategory{} - mi := &file_product_proto_msgTypes[168] + mi := &file_product_proto_msgTypes[178] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8680,7 +9216,7 @@ func (x *Subcategory) String() string { func (*Subcategory) ProtoMessage() {} func (x *Subcategory) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[168] + mi := &file_product_proto_msgTypes[178] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8693,7 +9229,7 @@ func (x *Subcategory) ProtoReflect() protoreflect.Message { // Deprecated: Use Subcategory.ProtoReflect.Descriptor instead. func (*Subcategory) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{168} + return file_product_proto_rawDescGZIP(), []int{178} } func (x *Subcategory) GetId() string { @@ -8737,7 +9273,7 @@ type CategoryMetrics struct { func (x *CategoryMetrics) Reset() { *x = CategoryMetrics{} - mi := &file_product_proto_msgTypes[169] + mi := &file_product_proto_msgTypes[179] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8749,7 +9285,7 @@ func (x *CategoryMetrics) String() string { func (*CategoryMetrics) ProtoMessage() {} func (x *CategoryMetrics) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[169] + mi := &file_product_proto_msgTypes[179] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8762,7 +9298,7 @@ func (x *CategoryMetrics) ProtoReflect() protoreflect.Message { // Deprecated: Use CategoryMetrics.ProtoReflect.Descriptor instead. func (*CategoryMetrics) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{169} + return file_product_proto_rawDescGZIP(), []int{179} } func (x *CategoryMetrics) GetId() string { @@ -8812,7 +9348,7 @@ type Cat struct { func (x *Cat) Reset() { *x = Cat{} - mi := &file_product_proto_msgTypes[170] + mi := &file_product_proto_msgTypes[180] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8824,7 +9360,7 @@ func (x *Cat) String() string { func (*Cat) ProtoMessage() {} func (x *Cat) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[170] + mi := &file_product_proto_msgTypes[180] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8837,7 +9373,7 @@ func (x *Cat) ProtoReflect() protoreflect.Message { // Deprecated: Use Cat.ProtoReflect.Descriptor instead. func (*Cat) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{170} + return file_product_proto_rawDescGZIP(), []int{180} } func (x *Cat) GetId() string { @@ -8880,7 +9416,7 @@ type Dog struct { func (x *Dog) Reset() { *x = Dog{} - mi := &file_product_proto_msgTypes[171] + mi := &file_product_proto_msgTypes[181] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8892,7 +9428,7 @@ func (x *Dog) String() string { func (*Dog) ProtoMessage() {} func (x *Dog) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[171] + mi := &file_product_proto_msgTypes[181] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8905,7 +9441,7 @@ func (x *Dog) ProtoReflect() protoreflect.Message { // Deprecated: Use Dog.ProtoReflect.Descriptor instead. func (*Dog) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{171} + return file_product_proto_rawDescGZIP(), []int{181} } func (x *Dog) GetId() string { @@ -8946,7 +9482,7 @@ type ActionSuccess struct { func (x *ActionSuccess) Reset() { *x = ActionSuccess{} - mi := &file_product_proto_msgTypes[172] + mi := &file_product_proto_msgTypes[182] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8958,7 +9494,7 @@ func (x *ActionSuccess) String() string { func (*ActionSuccess) ProtoMessage() {} func (x *ActionSuccess) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[172] + mi := &file_product_proto_msgTypes[182] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8971,7 +9507,7 @@ func (x *ActionSuccess) ProtoReflect() protoreflect.Message { // Deprecated: Use ActionSuccess.ProtoReflect.Descriptor instead. func (*ActionSuccess) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{172} + return file_product_proto_rawDescGZIP(), []int{182} } func (x *ActionSuccess) GetMessage() string { @@ -8998,7 +9534,7 @@ type ActionError struct { func (x *ActionError) Reset() { *x = ActionError{} - mi := &file_product_proto_msgTypes[173] + mi := &file_product_proto_msgTypes[183] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9010,7 +9546,7 @@ func (x *ActionError) String() string { func (*ActionError) ProtoMessage() {} func (x *ActionError) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[173] + mi := &file_product_proto_msgTypes[183] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9023,7 +9559,7 @@ func (x *ActionError) ProtoReflect() protoreflect.Message { // Deprecated: Use ActionError.ProtoReflect.Descriptor instead. func (*ActionError) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{173} + return file_product_proto_rawDescGZIP(), []int{183} } func (x *ActionError) GetMessage() string { @@ -9050,7 +9586,7 @@ type CategoryInput struct { func (x *CategoryInput) Reset() { *x = CategoryInput{} - mi := &file_product_proto_msgTypes[174] + mi := &file_product_proto_msgTypes[184] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9062,7 +9598,7 @@ func (x *CategoryInput) String() string { func (*CategoryInput) ProtoMessage() {} func (x *CategoryInput) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[174] + mi := &file_product_proto_msgTypes[184] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9075,7 +9611,7 @@ func (x *CategoryInput) ProtoReflect() protoreflect.Message { // Deprecated: Use CategoryInput.ProtoReflect.Descriptor instead. func (*CategoryInput) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{174} + return file_product_proto_rawDescGZIP(), []int{184} } func (x *CategoryInput) GetName() string { @@ -9104,7 +9640,7 @@ type ProductCountFilter struct { func (x *ProductCountFilter) Reset() { *x = ProductCountFilter{} - mi := &file_product_proto_msgTypes[175] + mi := &file_product_proto_msgTypes[185] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9116,7 +9652,7 @@ func (x *ProductCountFilter) String() string { func (*ProductCountFilter) ProtoMessage() {} func (x *ProductCountFilter) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[175] + mi := &file_product_proto_msgTypes[185] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9129,7 +9665,7 @@ func (x *ProductCountFilter) ProtoReflect() protoreflect.Message { // Deprecated: Use ProductCountFilter.ProtoReflect.Descriptor instead. func (*ProductCountFilter) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{175} + return file_product_proto_rawDescGZIP(), []int{185} } func (x *ProductCountFilter) GetMinPrice() *wrapperspb.DoubleValue { @@ -9173,7 +9709,7 @@ type SubcategoryItemFilter struct { func (x *SubcategoryItemFilter) Reset() { *x = SubcategoryItemFilter{} - mi := &file_product_proto_msgTypes[176] + mi := &file_product_proto_msgTypes[186] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9185,7 +9721,7 @@ func (x *SubcategoryItemFilter) String() string { func (*SubcategoryItemFilter) ProtoMessage() {} func (x *SubcategoryItemFilter) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[176] + mi := &file_product_proto_msgTypes[186] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9198,7 +9734,7 @@ func (x *SubcategoryItemFilter) ProtoReflect() protoreflect.Message { // Deprecated: Use SubcategoryItemFilter.ProtoReflect.Descriptor instead. func (*SubcategoryItemFilter) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{176} + return file_product_proto_rawDescGZIP(), []int{186} } func (x *SubcategoryItemFilter) GetMinPrice() *wrapperspb.DoubleValue { @@ -9236,6 +9772,66 @@ func (x *SubcategoryItemFilter) GetSearchTerm() *wrapperspb.StringValue { return nil } +type ShippingEstimateInput struct { + state protoimpl.MessageState `protogen:"open.v1"` + Destination ShippingDestination `protobuf:"varint,1,opt,name=destination,proto3,enum=productv1.ShippingDestination" json:"destination,omitempty"` + Weight float64 `protobuf:"fixed64,2,opt,name=weight,proto3" json:"weight,omitempty"` + Expedited *wrapperspb.BoolValue `protobuf:"bytes,3,opt,name=expedited,proto3" json:"expedited,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ShippingEstimateInput) Reset() { + *x = ShippingEstimateInput{} + mi := &file_product_proto_msgTypes[187] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ShippingEstimateInput) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ShippingEstimateInput) ProtoMessage() {} + +func (x *ShippingEstimateInput) ProtoReflect() protoreflect.Message { + mi := &file_product_proto_msgTypes[187] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ShippingEstimateInput.ProtoReflect.Descriptor instead. +func (*ShippingEstimateInput) Descriptor() ([]byte, []int) { + return file_product_proto_rawDescGZIP(), []int{187} +} + +func (x *ShippingEstimateInput) GetDestination() ShippingDestination { + if x != nil { + return x.Destination + } + return ShippingDestination_SHIPPING_DESTINATION_UNSPECIFIED +} + +func (x *ShippingEstimateInput) GetWeight() float64 { + if x != nil { + return x.Weight + } + return 0 +} + +func (x *ShippingEstimateInput) GetExpedited() *wrapperspb.BoolValue { + if x != nil { + return x.Expedited + } + return nil +} + type ListOfAuthorFilter_List struct { state protoimpl.MessageState `protogen:"open.v1"` Items []*AuthorFilter `protobuf:"bytes,1,rep,name=items,proto3" json:"items,omitempty"` @@ -9245,7 +9841,7 @@ type ListOfAuthorFilter_List struct { func (x *ListOfAuthorFilter_List) Reset() { *x = ListOfAuthorFilter_List{} - mi := &file_product_proto_msgTypes[177] + mi := &file_product_proto_msgTypes[188] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9257,7 +9853,7 @@ func (x *ListOfAuthorFilter_List) String() string { func (*ListOfAuthorFilter_List) ProtoMessage() {} func (x *ListOfAuthorFilter_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[177] + mi := &file_product_proto_msgTypes[188] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9289,7 +9885,7 @@ type ListOfAuthorInput_List struct { func (x *ListOfAuthorInput_List) Reset() { *x = ListOfAuthorInput_List{} - mi := &file_product_proto_msgTypes[178] + mi := &file_product_proto_msgTypes[189] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9301,7 +9897,7 @@ func (x *ListOfAuthorInput_List) String() string { func (*ListOfAuthorInput_List) ProtoMessage() {} func (x *ListOfAuthorInput_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[178] + mi := &file_product_proto_msgTypes[189] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9333,7 +9929,7 @@ type ListOfBlogPost_List struct { func (x *ListOfBlogPost_List) Reset() { *x = ListOfBlogPost_List{} - mi := &file_product_proto_msgTypes[179] + mi := &file_product_proto_msgTypes[190] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9345,7 +9941,7 @@ func (x *ListOfBlogPost_List) String() string { func (*ListOfBlogPost_List) ProtoMessage() {} func (x *ListOfBlogPost_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[179] + mi := &file_product_proto_msgTypes[190] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9377,7 +9973,7 @@ type ListOfBlogPostFilter_List struct { func (x *ListOfBlogPostFilter_List) Reset() { *x = ListOfBlogPostFilter_List{} - mi := &file_product_proto_msgTypes[180] + mi := &file_product_proto_msgTypes[191] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9389,7 +9985,7 @@ func (x *ListOfBlogPostFilter_List) String() string { func (*ListOfBlogPostFilter_List) ProtoMessage() {} func (x *ListOfBlogPostFilter_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[180] + mi := &file_product_proto_msgTypes[191] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9421,7 +10017,7 @@ type ListOfBlogPostInput_List struct { func (x *ListOfBlogPostInput_List) Reset() { *x = ListOfBlogPostInput_List{} - mi := &file_product_proto_msgTypes[181] + mi := &file_product_proto_msgTypes[192] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9433,7 +10029,7 @@ func (x *ListOfBlogPostInput_List) String() string { func (*ListOfBlogPostInput_List) ProtoMessage() {} func (x *ListOfBlogPostInput_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[181] + mi := &file_product_proto_msgTypes[192] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9465,7 +10061,7 @@ type ListOfBoolean_List struct { func (x *ListOfBoolean_List) Reset() { *x = ListOfBoolean_List{} - mi := &file_product_proto_msgTypes[182] + mi := &file_product_proto_msgTypes[193] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9477,7 +10073,7 @@ func (x *ListOfBoolean_List) String() string { func (*ListOfBoolean_List) ProtoMessage() {} func (x *ListOfBoolean_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[182] + mi := &file_product_proto_msgTypes[193] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9509,7 +10105,7 @@ type ListOfCategory_List struct { func (x *ListOfCategory_List) Reset() { *x = ListOfCategory_List{} - mi := &file_product_proto_msgTypes[183] + mi := &file_product_proto_msgTypes[194] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9521,7 +10117,7 @@ func (x *ListOfCategory_List) String() string { func (*ListOfCategory_List) ProtoMessage() {} func (x *ListOfCategory_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[183] + mi := &file_product_proto_msgTypes[194] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9553,7 +10149,7 @@ type ListOfCategoryInput_List struct { func (x *ListOfCategoryInput_List) Reset() { *x = ListOfCategoryInput_List{} - mi := &file_product_proto_msgTypes[184] + mi := &file_product_proto_msgTypes[195] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9565,7 +10161,7 @@ func (x *ListOfCategoryInput_List) String() string { func (*ListOfCategoryInput_List) ProtoMessage() {} func (x *ListOfCategoryInput_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[184] + mi := &file_product_proto_msgTypes[195] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9597,7 +10193,7 @@ type ListOfFloat_List struct { func (x *ListOfFloat_List) Reset() { *x = ListOfFloat_List{} - mi := &file_product_proto_msgTypes[185] + mi := &file_product_proto_msgTypes[196] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9609,7 +10205,7 @@ func (x *ListOfFloat_List) String() string { func (*ListOfFloat_List) ProtoMessage() {} func (x *ListOfFloat_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[185] + mi := &file_product_proto_msgTypes[196] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9641,7 +10237,7 @@ type ListOfListOfCategory_List struct { func (x *ListOfListOfCategory_List) Reset() { *x = ListOfListOfCategory_List{} - mi := &file_product_proto_msgTypes[186] + mi := &file_product_proto_msgTypes[197] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9653,7 +10249,7 @@ func (x *ListOfListOfCategory_List) String() string { func (*ListOfListOfCategory_List) ProtoMessage() {} func (x *ListOfListOfCategory_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[186] + mi := &file_product_proto_msgTypes[197] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9685,7 +10281,7 @@ type ListOfListOfCategoryInput_List struct { func (x *ListOfListOfCategoryInput_List) Reset() { *x = ListOfListOfCategoryInput_List{} - mi := &file_product_proto_msgTypes[187] + mi := &file_product_proto_msgTypes[198] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9697,7 +10293,7 @@ func (x *ListOfListOfCategoryInput_List) String() string { func (*ListOfListOfCategoryInput_List) ProtoMessage() {} func (x *ListOfListOfCategoryInput_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[187] + mi := &file_product_proto_msgTypes[198] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9729,7 +10325,7 @@ type ListOfListOfString_List struct { func (x *ListOfListOfString_List) Reset() { *x = ListOfListOfString_List{} - mi := &file_product_proto_msgTypes[188] + mi := &file_product_proto_msgTypes[199] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9741,7 +10337,7 @@ func (x *ListOfListOfString_List) String() string { func (*ListOfListOfString_List) ProtoMessage() {} func (x *ListOfListOfString_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[188] + mi := &file_product_proto_msgTypes[199] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9773,7 +10369,7 @@ type ListOfListOfUser_List struct { func (x *ListOfListOfUser_List) Reset() { *x = ListOfListOfUser_List{} - mi := &file_product_proto_msgTypes[189] + mi := &file_product_proto_msgTypes[200] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9785,7 +10381,7 @@ func (x *ListOfListOfUser_List) String() string { func (*ListOfListOfUser_List) ProtoMessage() {} func (x *ListOfListOfUser_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[189] + mi := &file_product_proto_msgTypes[200] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9817,7 +10413,7 @@ type ListOfListOfUserInput_List struct { func (x *ListOfListOfUserInput_List) Reset() { *x = ListOfListOfUserInput_List{} - mi := &file_product_proto_msgTypes[190] + mi := &file_product_proto_msgTypes[201] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9829,7 +10425,7 @@ func (x *ListOfListOfUserInput_List) String() string { func (*ListOfListOfUserInput_List) ProtoMessage() {} func (x *ListOfListOfUserInput_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[190] + mi := &file_product_proto_msgTypes[201] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9861,7 +10457,7 @@ type ListOfOrderLine_List struct { func (x *ListOfOrderLine_List) Reset() { *x = ListOfOrderLine_List{} - mi := &file_product_proto_msgTypes[191] + mi := &file_product_proto_msgTypes[202] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9873,7 +10469,7 @@ func (x *ListOfOrderLine_List) String() string { func (*ListOfOrderLine_List) ProtoMessage() {} func (x *ListOfOrderLine_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[191] + mi := &file_product_proto_msgTypes[202] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9905,7 +10501,7 @@ type ListOfProduct_List struct { func (x *ListOfProduct_List) Reset() { *x = ListOfProduct_List{} - mi := &file_product_proto_msgTypes[192] + mi := &file_product_proto_msgTypes[203] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9917,7 +10513,7 @@ func (x *ListOfProduct_List) String() string { func (*ListOfProduct_List) ProtoMessage() {} func (x *ListOfProduct_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[192] + mi := &file_product_proto_msgTypes[203] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9949,7 +10545,7 @@ type ListOfString_List struct { func (x *ListOfString_List) Reset() { *x = ListOfString_List{} - mi := &file_product_proto_msgTypes[193] + mi := &file_product_proto_msgTypes[204] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9961,7 +10557,7 @@ func (x *ListOfString_List) String() string { func (*ListOfString_List) ProtoMessage() {} func (x *ListOfString_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[193] + mi := &file_product_proto_msgTypes[204] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9993,7 +10589,7 @@ type ListOfSubcategory_List struct { func (x *ListOfSubcategory_List) Reset() { *x = ListOfSubcategory_List{} - mi := &file_product_proto_msgTypes[194] + mi := &file_product_proto_msgTypes[205] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -10005,7 +10601,7 @@ func (x *ListOfSubcategory_List) String() string { func (*ListOfSubcategory_List) ProtoMessage() {} func (x *ListOfSubcategory_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[194] + mi := &file_product_proto_msgTypes[205] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -10037,7 +10633,7 @@ type ListOfUser_List struct { func (x *ListOfUser_List) Reset() { *x = ListOfUser_List{} - mi := &file_product_proto_msgTypes[195] + mi := &file_product_proto_msgTypes[206] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -10049,7 +10645,7 @@ func (x *ListOfUser_List) String() string { func (*ListOfUser_List) ProtoMessage() {} func (x *ListOfUser_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[195] + mi := &file_product_proto_msgTypes[206] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -10081,7 +10677,7 @@ type ListOfUserInput_List struct { func (x *ListOfUserInput_List) Reset() { *x = ListOfUserInput_List{} - mi := &file_product_proto_msgTypes[196] + mi := &file_product_proto_msgTypes[207] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -10093,7 +10689,7 @@ func (x *ListOfUserInput_List) String() string { func (*ListOfUserInput_List) ProtoMessage() {} func (x *ListOfUserInput_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[196] + mi := &file_product_proto_msgTypes[207] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -10387,7 +10983,34 @@ const file_product_proto_rawDesc = "" + "\n" + "blog_posts\x18\x01 \x01(\v2\x1e.productv1.ListOfBlogPostInputR\tblogPosts\"o\n" + "#MutationBulkUpdateBlogPostsResponse\x12H\n" + - "\x16bulk_update_blog_posts\x18\x01 \x03(\v2\x13.productv1.BlogPostR\x13bulkUpdateBlogPosts\"Z\n" + + "\x16bulk_update_blog_posts\x18\x01 \x03(\v2\x13.productv1.BlogPostR\x13bulkUpdateBlogPosts\"\\\n" + + "\"ResolveProductShippingEstimateArgs\x126\n" + + "\x05input\x18\x01 \x01(\v2 .productv1.ShippingEstimateInputR\x05input\"M\n" + + "%ResolveProductShippingEstimateContext\x12\x0e\n" + + "\x02id\x18\x01 \x01(\tR\x02id\x12\x14\n" + + "\x05price\x18\x02 \x01(\x01R\x05price\"\xc1\x01\n" + + "%ResolveProductShippingEstimateRequest\x12J\n" + + "\acontext\x18\x01 \x03(\v20.productv1.ResolveProductShippingEstimateContextR\acontext\x12L\n" + + "\n" + + "field_args\x18\x02 \x01(\v2-.productv1.ResolveProductShippingEstimateArgsR\tfieldArgs\"S\n" + + "$ResolveProductShippingEstimateResult\x12+\n" + + "\x11shipping_estimate\x18\x01 \x01(\x01R\x10shippingEstimate\"q\n" + + "&ResolveProductShippingEstimateResponse\x12G\n" + + "\x06result\x18\x01 \x03(\v2/.productv1.ResolveProductShippingEstimateResultR\x06result\"D\n" + + "%ResolveProductRecommendedCategoryArgs\x12\x1b\n" + + "\tmax_price\x18\x01 \x01(\x05R\bmaxPrice\"d\n" + + "(ResolveProductRecommendedCategoryContext\x12\x0e\n" + + "\x02id\x18\x01 \x01(\tR\x02id\x12\x12\n" + + "\x04name\x18\x02 \x01(\tR\x04name\x12\x14\n" + + "\x05price\x18\x03 \x01(\x01R\x05price\"\xca\x01\n" + + "(ResolveProductRecommendedCategoryRequest\x12M\n" + + "\acontext\x18\x01 \x03(\v23.productv1.ResolveProductRecommendedCategoryContextR\acontext\x12O\n" + + "\n" + + "field_args\x18\x02 \x01(\v20.productv1.ResolveProductRecommendedCategoryArgsR\tfieldArgs\"q\n" + + "'ResolveProductRecommendedCategoryResult\x12F\n" + + "\x14recommended_category\x18\x01 \x01(\v2\x13.productv1.CategoryR\x13recommendedCategory\"w\n" + + ")ResolveProductRecommendedCategoryResponse\x12J\n" + + "\x06result\x18\x01 \x03(\v22.productv1.ResolveProductRecommendedCategoryResultR\x06result\"Z\n" + "\x1fResolveCategoryProductCountArgs\x127\n" + "\afilters\x18\x01 \x01(\v2\x1d.productv1.ProductCountFilterR\afilters\"H\n" + "\"ResolveCategoryProductCountContext\x12\x0e\n" + @@ -10702,13 +11325,22 @@ const file_product_proto_rawDesc = "" + "\bin_stock\x18\x03 \x01(\v2\x1a.google.protobuf.BoolValueR\ainStock\x127\n" + "\tis_active\x18\x04 \x01(\v2\x1a.google.protobuf.BoolValueR\bisActive\x12=\n" + "\vsearch_term\x18\x05 \x01(\v2\x1c.google.protobuf.StringValueR\n" + - "searchTerm*\x9a\x01\n" + + "searchTerm\"\xab\x01\n" + + "\x15ShippingEstimateInput\x12@\n" + + "\vdestination\x18\x01 \x01(\x0e2\x1e.productv1.ShippingDestinationR\vdestination\x12\x16\n" + + "\x06weight\x18\x02 \x01(\x01R\x06weight\x128\n" + + "\texpedited\x18\x03 \x01(\v2\x1a.google.protobuf.BoolValueR\texpedited*\x9a\x01\n" + "\fCategoryKind\x12\x1d\n" + "\x19CATEGORY_KIND_UNSPECIFIED\x10\x00\x12\x16\n" + "\x12CATEGORY_KIND_BOOK\x10\x01\x12\x1d\n" + "\x19CATEGORY_KIND_ELECTRONICS\x10\x02\x12\x1b\n" + "\x17CATEGORY_KIND_FURNITURE\x10\x03\x12\x17\n" + - "\x13CATEGORY_KIND_OTHER\x10\x042\xcc*\n" + + "\x13CATEGORY_KIND_OTHER\x10\x04*\xa8\x01\n" + + "\x13ShippingDestination\x12$\n" + + " SHIPPING_DESTINATION_UNSPECIFIED\x10\x00\x12!\n" + + "\x1dSHIPPING_DESTINATION_DOMESTIC\x10\x01\x12 \n" + + "\x1cSHIPPING_DESTINATION_EXPRESS\x10\x02\x12&\n" + + "\"SHIPPING_DESTINATION_INTERNATIONAL\x10\x032\xe9,\n" + "\x0eProductService\x12`\n" + "\x11LookupProductById\x12#.productv1.LookupProductByIdRequest\x1a$.productv1.LookupProductByIdResponse\"\x00\x12`\n" + "\x11LookupStorageById\x12#.productv1.LookupStorageByIdRequest\x1a$.productv1.LookupStorageByIdResponse\"\x00\x12f\n" + @@ -10758,7 +11390,9 @@ const file_product_proto_rawDesc = "" + "QueryUsers\x12\x1c.productv1.QueryUsersRequest\x1a\x1d.productv1.QueryUsersResponse\"\x00\x12\x87\x01\n" + "\x1eResolveCategoryCategoryMetrics\x120.productv1.ResolveCategoryCategoryMetricsRequest\x1a1.productv1.ResolveCategoryCategoryMetricsResponse\"\x00\x12\x87\x01\n" + "\x1eResolveCategoryPopularityScore\x120.productv1.ResolveCategoryPopularityScoreRequest\x1a1.productv1.ResolveCategoryPopularityScoreResponse\"\x00\x12~\n" + - "\x1bResolveCategoryProductCount\x12-.productv1.ResolveCategoryProductCountRequest\x1a..productv1.ResolveCategoryProductCountResponse\"\x00\x12~\n" + + "\x1bResolveCategoryProductCount\x12-.productv1.ResolveCategoryProductCountRequest\x1a..productv1.ResolveCategoryProductCountResponse\"\x00\x12\x90\x01\n" + + "!ResolveProductRecommendedCategory\x123.productv1.ResolveProductRecommendedCategoryRequest\x1a4.productv1.ResolveProductRecommendedCategoryResponse\"\x00\x12\x87\x01\n" + + "\x1eResolveProductShippingEstimate\x120.productv1.ResolveProductShippingEstimateRequest\x1a1.productv1.ResolveProductShippingEstimateResponse\"\x00\x12~\n" + "\x1bResolveSubcategoryItemCount\x12-.productv1.ResolveSubcategoryItemCountRequest\x1a..productv1.ResolveSubcategoryItemCountResponse\"\x00B%Z#cosmo/pkg/proto/productv1;productv1b\x06proto3" var ( @@ -10773,534 +11407,560 @@ func file_product_proto_rawDescGZIP() []byte { return file_product_proto_rawDescData } -var file_product_proto_enumTypes = make([]protoimpl.EnumInfo, 1) -var file_product_proto_msgTypes = make([]protoimpl.MessageInfo, 197) +var file_product_proto_enumTypes = make([]protoimpl.EnumInfo, 2) +var file_product_proto_msgTypes = make([]protoimpl.MessageInfo, 208) var file_product_proto_goTypes = []any{ (CategoryKind)(0), // 0: productv1.CategoryKind - (*ListOfAuthorFilter)(nil), // 1: productv1.ListOfAuthorFilter - (*ListOfAuthorInput)(nil), // 2: productv1.ListOfAuthorInput - (*ListOfBlogPost)(nil), // 3: productv1.ListOfBlogPost - (*ListOfBlogPostFilter)(nil), // 4: productv1.ListOfBlogPostFilter - (*ListOfBlogPostInput)(nil), // 5: productv1.ListOfBlogPostInput - (*ListOfBoolean)(nil), // 6: productv1.ListOfBoolean - (*ListOfCategory)(nil), // 7: productv1.ListOfCategory - (*ListOfCategoryInput)(nil), // 8: productv1.ListOfCategoryInput - (*ListOfFloat)(nil), // 9: productv1.ListOfFloat - (*ListOfListOfCategory)(nil), // 10: productv1.ListOfListOfCategory - (*ListOfListOfCategoryInput)(nil), // 11: productv1.ListOfListOfCategoryInput - (*ListOfListOfString)(nil), // 12: productv1.ListOfListOfString - (*ListOfListOfUser)(nil), // 13: productv1.ListOfListOfUser - (*ListOfListOfUserInput)(nil), // 14: productv1.ListOfListOfUserInput - (*ListOfOrderLine)(nil), // 15: productv1.ListOfOrderLine - (*ListOfProduct)(nil), // 16: productv1.ListOfProduct - (*ListOfString)(nil), // 17: productv1.ListOfString - (*ListOfSubcategory)(nil), // 18: productv1.ListOfSubcategory - (*ListOfUser)(nil), // 19: productv1.ListOfUser - (*ListOfUserInput)(nil), // 20: productv1.ListOfUserInput - (*LookupProductByIdRequestKey)(nil), // 21: productv1.LookupProductByIdRequestKey - (*LookupProductByIdRequest)(nil), // 22: productv1.LookupProductByIdRequest - (*LookupProductByIdResponse)(nil), // 23: productv1.LookupProductByIdResponse - (*LookupStorageByIdRequestKey)(nil), // 24: productv1.LookupStorageByIdRequestKey - (*LookupStorageByIdRequest)(nil), // 25: productv1.LookupStorageByIdRequest - (*LookupStorageByIdResponse)(nil), // 26: productv1.LookupStorageByIdResponse - (*LookupWarehouseByIdRequestKey)(nil), // 27: productv1.LookupWarehouseByIdRequestKey - (*LookupWarehouseByIdRequest)(nil), // 28: productv1.LookupWarehouseByIdRequest - (*LookupWarehouseByIdResponse)(nil), // 29: productv1.LookupWarehouseByIdResponse - (*QueryUsersRequest)(nil), // 30: productv1.QueryUsersRequest - (*QueryUsersResponse)(nil), // 31: productv1.QueryUsersResponse - (*QueryUserRequest)(nil), // 32: productv1.QueryUserRequest - (*QueryUserResponse)(nil), // 33: productv1.QueryUserResponse - (*QueryNestedTypeRequest)(nil), // 34: productv1.QueryNestedTypeRequest - (*QueryNestedTypeResponse)(nil), // 35: productv1.QueryNestedTypeResponse - (*QueryRecursiveTypeRequest)(nil), // 36: productv1.QueryRecursiveTypeRequest - (*QueryRecursiveTypeResponse)(nil), // 37: productv1.QueryRecursiveTypeResponse - (*QueryTypeFilterWithArgumentsRequest)(nil), // 38: productv1.QueryTypeFilterWithArgumentsRequest - (*QueryTypeFilterWithArgumentsResponse)(nil), // 39: productv1.QueryTypeFilterWithArgumentsResponse - (*QueryTypeWithMultipleFilterFieldsRequest)(nil), // 40: productv1.QueryTypeWithMultipleFilterFieldsRequest - (*QueryTypeWithMultipleFilterFieldsResponse)(nil), // 41: productv1.QueryTypeWithMultipleFilterFieldsResponse - (*QueryComplexFilterTypeRequest)(nil), // 42: productv1.QueryComplexFilterTypeRequest - (*QueryComplexFilterTypeResponse)(nil), // 43: productv1.QueryComplexFilterTypeResponse - (*QueryCalculateTotalsRequest)(nil), // 44: productv1.QueryCalculateTotalsRequest - (*QueryCalculateTotalsResponse)(nil), // 45: productv1.QueryCalculateTotalsResponse - (*QueryCategoriesRequest)(nil), // 46: productv1.QueryCategoriesRequest - (*QueryCategoriesResponse)(nil), // 47: productv1.QueryCategoriesResponse - (*QueryCategoriesByKindRequest)(nil), // 48: productv1.QueryCategoriesByKindRequest - (*QueryCategoriesByKindResponse)(nil), // 49: productv1.QueryCategoriesByKindResponse - (*QueryCategoriesByKindsRequest)(nil), // 50: productv1.QueryCategoriesByKindsRequest - (*QueryCategoriesByKindsResponse)(nil), // 51: productv1.QueryCategoriesByKindsResponse - (*QueryFilterCategoriesRequest)(nil), // 52: productv1.QueryFilterCategoriesRequest - (*QueryFilterCategoriesResponse)(nil), // 53: productv1.QueryFilterCategoriesResponse - (*QueryRandomPetRequest)(nil), // 54: productv1.QueryRandomPetRequest - (*QueryRandomPetResponse)(nil), // 55: productv1.QueryRandomPetResponse - (*QueryAllPetsRequest)(nil), // 56: productv1.QueryAllPetsRequest - (*QueryAllPetsResponse)(nil), // 57: productv1.QueryAllPetsResponse - (*QuerySearchRequest)(nil), // 58: productv1.QuerySearchRequest - (*QuerySearchResponse)(nil), // 59: productv1.QuerySearchResponse - (*QueryRandomSearchResultRequest)(nil), // 60: productv1.QueryRandomSearchResultRequest - (*QueryRandomSearchResultResponse)(nil), // 61: productv1.QueryRandomSearchResultResponse - (*QueryNullableFieldsTypeRequest)(nil), // 62: productv1.QueryNullableFieldsTypeRequest - (*QueryNullableFieldsTypeResponse)(nil), // 63: productv1.QueryNullableFieldsTypeResponse - (*QueryNullableFieldsTypeByIdRequest)(nil), // 64: productv1.QueryNullableFieldsTypeByIdRequest - (*QueryNullableFieldsTypeByIdResponse)(nil), // 65: productv1.QueryNullableFieldsTypeByIdResponse - (*QueryNullableFieldsTypeWithFilterRequest)(nil), // 66: productv1.QueryNullableFieldsTypeWithFilterRequest - (*QueryNullableFieldsTypeWithFilterResponse)(nil), // 67: productv1.QueryNullableFieldsTypeWithFilterResponse - (*QueryAllNullableFieldsTypesRequest)(nil), // 68: productv1.QueryAllNullableFieldsTypesRequest - (*QueryAllNullableFieldsTypesResponse)(nil), // 69: productv1.QueryAllNullableFieldsTypesResponse - (*QueryBlogPostRequest)(nil), // 70: productv1.QueryBlogPostRequest - (*QueryBlogPostResponse)(nil), // 71: productv1.QueryBlogPostResponse - (*QueryBlogPostByIdRequest)(nil), // 72: productv1.QueryBlogPostByIdRequest - (*QueryBlogPostByIdResponse)(nil), // 73: productv1.QueryBlogPostByIdResponse - (*QueryBlogPostsWithFilterRequest)(nil), // 74: productv1.QueryBlogPostsWithFilterRequest - (*QueryBlogPostsWithFilterResponse)(nil), // 75: productv1.QueryBlogPostsWithFilterResponse - (*QueryAllBlogPostsRequest)(nil), // 76: productv1.QueryAllBlogPostsRequest - (*QueryAllBlogPostsResponse)(nil), // 77: productv1.QueryAllBlogPostsResponse - (*QueryAuthorRequest)(nil), // 78: productv1.QueryAuthorRequest - (*QueryAuthorResponse)(nil), // 79: productv1.QueryAuthorResponse - (*QueryAuthorByIdRequest)(nil), // 80: productv1.QueryAuthorByIdRequest - (*QueryAuthorByIdResponse)(nil), // 81: productv1.QueryAuthorByIdResponse - (*QueryAuthorsWithFilterRequest)(nil), // 82: productv1.QueryAuthorsWithFilterRequest - (*QueryAuthorsWithFilterResponse)(nil), // 83: productv1.QueryAuthorsWithFilterResponse - (*QueryAllAuthorsRequest)(nil), // 84: productv1.QueryAllAuthorsRequest - (*QueryAllAuthorsResponse)(nil), // 85: productv1.QueryAllAuthorsResponse - (*QueryBulkSearchAuthorsRequest)(nil), // 86: productv1.QueryBulkSearchAuthorsRequest - (*QueryBulkSearchAuthorsResponse)(nil), // 87: productv1.QueryBulkSearchAuthorsResponse - (*QueryBulkSearchBlogPostsRequest)(nil), // 88: productv1.QueryBulkSearchBlogPostsRequest - (*QueryBulkSearchBlogPostsResponse)(nil), // 89: productv1.QueryBulkSearchBlogPostsResponse - (*MutationCreateUserRequest)(nil), // 90: productv1.MutationCreateUserRequest - (*MutationCreateUserResponse)(nil), // 91: productv1.MutationCreateUserResponse - (*MutationPerformActionRequest)(nil), // 92: productv1.MutationPerformActionRequest - (*MutationPerformActionResponse)(nil), // 93: productv1.MutationPerformActionResponse - (*MutationCreateNullableFieldsTypeRequest)(nil), // 94: productv1.MutationCreateNullableFieldsTypeRequest - (*MutationCreateNullableFieldsTypeResponse)(nil), // 95: productv1.MutationCreateNullableFieldsTypeResponse - (*MutationUpdateNullableFieldsTypeRequest)(nil), // 96: productv1.MutationUpdateNullableFieldsTypeRequest - (*MutationUpdateNullableFieldsTypeResponse)(nil), // 97: productv1.MutationUpdateNullableFieldsTypeResponse - (*MutationCreateBlogPostRequest)(nil), // 98: productv1.MutationCreateBlogPostRequest - (*MutationCreateBlogPostResponse)(nil), // 99: productv1.MutationCreateBlogPostResponse - (*MutationUpdateBlogPostRequest)(nil), // 100: productv1.MutationUpdateBlogPostRequest - (*MutationUpdateBlogPostResponse)(nil), // 101: productv1.MutationUpdateBlogPostResponse - (*MutationCreateAuthorRequest)(nil), // 102: productv1.MutationCreateAuthorRequest - (*MutationCreateAuthorResponse)(nil), // 103: productv1.MutationCreateAuthorResponse - (*MutationUpdateAuthorRequest)(nil), // 104: productv1.MutationUpdateAuthorRequest - (*MutationUpdateAuthorResponse)(nil), // 105: productv1.MutationUpdateAuthorResponse - (*MutationBulkCreateAuthorsRequest)(nil), // 106: productv1.MutationBulkCreateAuthorsRequest - (*MutationBulkCreateAuthorsResponse)(nil), // 107: productv1.MutationBulkCreateAuthorsResponse - (*MutationBulkUpdateAuthorsRequest)(nil), // 108: productv1.MutationBulkUpdateAuthorsRequest - (*MutationBulkUpdateAuthorsResponse)(nil), // 109: productv1.MutationBulkUpdateAuthorsResponse - (*MutationBulkCreateBlogPostsRequest)(nil), // 110: productv1.MutationBulkCreateBlogPostsRequest - (*MutationBulkCreateBlogPostsResponse)(nil), // 111: productv1.MutationBulkCreateBlogPostsResponse - (*MutationBulkUpdateBlogPostsRequest)(nil), // 112: productv1.MutationBulkUpdateBlogPostsRequest - (*MutationBulkUpdateBlogPostsResponse)(nil), // 113: productv1.MutationBulkUpdateBlogPostsResponse - (*ResolveCategoryProductCountArgs)(nil), // 114: productv1.ResolveCategoryProductCountArgs - (*ResolveCategoryProductCountContext)(nil), // 115: productv1.ResolveCategoryProductCountContext - (*ResolveCategoryProductCountRequest)(nil), // 116: productv1.ResolveCategoryProductCountRequest - (*ResolveCategoryProductCountResult)(nil), // 117: productv1.ResolveCategoryProductCountResult - (*ResolveCategoryProductCountResponse)(nil), // 118: productv1.ResolveCategoryProductCountResponse - (*ResolveCategoryPopularityScoreArgs)(nil), // 119: productv1.ResolveCategoryPopularityScoreArgs - (*ResolveCategoryPopularityScoreContext)(nil), // 120: productv1.ResolveCategoryPopularityScoreContext - (*ResolveCategoryPopularityScoreRequest)(nil), // 121: productv1.ResolveCategoryPopularityScoreRequest - (*ResolveCategoryPopularityScoreResult)(nil), // 122: productv1.ResolveCategoryPopularityScoreResult - (*ResolveCategoryPopularityScoreResponse)(nil), // 123: productv1.ResolveCategoryPopularityScoreResponse - (*ResolveCategoryCategoryMetricsArgs)(nil), // 124: productv1.ResolveCategoryCategoryMetricsArgs - (*ResolveCategoryCategoryMetricsContext)(nil), // 125: productv1.ResolveCategoryCategoryMetricsContext - (*ResolveCategoryCategoryMetricsRequest)(nil), // 126: productv1.ResolveCategoryCategoryMetricsRequest - (*ResolveCategoryCategoryMetricsResult)(nil), // 127: productv1.ResolveCategoryCategoryMetricsResult - (*ResolveCategoryCategoryMetricsResponse)(nil), // 128: productv1.ResolveCategoryCategoryMetricsResponse - (*ResolveSubcategoryItemCountArgs)(nil), // 129: productv1.ResolveSubcategoryItemCountArgs - (*ResolveSubcategoryItemCountContext)(nil), // 130: productv1.ResolveSubcategoryItemCountContext - (*ResolveSubcategoryItemCountRequest)(nil), // 131: productv1.ResolveSubcategoryItemCountRequest - (*ResolveSubcategoryItemCountResult)(nil), // 132: productv1.ResolveSubcategoryItemCountResult - (*ResolveSubcategoryItemCountResponse)(nil), // 133: productv1.ResolveSubcategoryItemCountResponse - (*Product)(nil), // 134: productv1.Product - (*Storage)(nil), // 135: productv1.Storage - (*Warehouse)(nil), // 136: productv1.Warehouse - (*User)(nil), // 137: productv1.User - (*NestedTypeA)(nil), // 138: productv1.NestedTypeA - (*RecursiveType)(nil), // 139: productv1.RecursiveType - (*TypeWithMultipleFilterFields)(nil), // 140: productv1.TypeWithMultipleFilterFields - (*FilterTypeInput)(nil), // 141: productv1.FilterTypeInput - (*ComplexFilterTypeInput)(nil), // 142: productv1.ComplexFilterTypeInput - (*TypeWithComplexFilterInput)(nil), // 143: productv1.TypeWithComplexFilterInput - (*OrderInput)(nil), // 144: productv1.OrderInput - (*Order)(nil), // 145: productv1.Order - (*Category)(nil), // 146: productv1.Category - (*CategoryFilter)(nil), // 147: productv1.CategoryFilter - (*Animal)(nil), // 148: productv1.Animal - (*SearchInput)(nil), // 149: productv1.SearchInput - (*SearchResult)(nil), // 150: productv1.SearchResult - (*NullableFieldsType)(nil), // 151: productv1.NullableFieldsType - (*NullableFieldsFilter)(nil), // 152: productv1.NullableFieldsFilter - (*BlogPost)(nil), // 153: productv1.BlogPost - (*BlogPostFilter)(nil), // 154: productv1.BlogPostFilter - (*Author)(nil), // 155: productv1.Author - (*AuthorFilter)(nil), // 156: productv1.AuthorFilter - (*UserInput)(nil), // 157: productv1.UserInput - (*ActionInput)(nil), // 158: productv1.ActionInput - (*ActionResult)(nil), // 159: productv1.ActionResult - (*NullableFieldsInput)(nil), // 160: productv1.NullableFieldsInput - (*BlogPostInput)(nil), // 161: productv1.BlogPostInput - (*AuthorInput)(nil), // 162: productv1.AuthorInput - (*NestedTypeB)(nil), // 163: productv1.NestedTypeB - (*NestedTypeC)(nil), // 164: productv1.NestedTypeC - (*FilterType)(nil), // 165: productv1.FilterType - (*Pagination)(nil), // 166: productv1.Pagination - (*OrderLineInput)(nil), // 167: productv1.OrderLineInput - (*OrderLine)(nil), // 168: productv1.OrderLine - (*Subcategory)(nil), // 169: productv1.Subcategory - (*CategoryMetrics)(nil), // 170: productv1.CategoryMetrics - (*Cat)(nil), // 171: productv1.Cat - (*Dog)(nil), // 172: productv1.Dog - (*ActionSuccess)(nil), // 173: productv1.ActionSuccess - (*ActionError)(nil), // 174: productv1.ActionError - (*CategoryInput)(nil), // 175: productv1.CategoryInput - (*ProductCountFilter)(nil), // 176: productv1.ProductCountFilter - (*SubcategoryItemFilter)(nil), // 177: productv1.SubcategoryItemFilter - (*ListOfAuthorFilter_List)(nil), // 178: productv1.ListOfAuthorFilter.List - (*ListOfAuthorInput_List)(nil), // 179: productv1.ListOfAuthorInput.List - (*ListOfBlogPost_List)(nil), // 180: productv1.ListOfBlogPost.List - (*ListOfBlogPostFilter_List)(nil), // 181: productv1.ListOfBlogPostFilter.List - (*ListOfBlogPostInput_List)(nil), // 182: productv1.ListOfBlogPostInput.List - (*ListOfBoolean_List)(nil), // 183: productv1.ListOfBoolean.List - (*ListOfCategory_List)(nil), // 184: productv1.ListOfCategory.List - (*ListOfCategoryInput_List)(nil), // 185: productv1.ListOfCategoryInput.List - (*ListOfFloat_List)(nil), // 186: productv1.ListOfFloat.List - (*ListOfListOfCategory_List)(nil), // 187: productv1.ListOfListOfCategory.List - (*ListOfListOfCategoryInput_List)(nil), // 188: productv1.ListOfListOfCategoryInput.List - (*ListOfListOfString_List)(nil), // 189: productv1.ListOfListOfString.List - (*ListOfListOfUser_List)(nil), // 190: productv1.ListOfListOfUser.List - (*ListOfListOfUserInput_List)(nil), // 191: productv1.ListOfListOfUserInput.List - (*ListOfOrderLine_List)(nil), // 192: productv1.ListOfOrderLine.List - (*ListOfProduct_List)(nil), // 193: productv1.ListOfProduct.List - (*ListOfString_List)(nil), // 194: productv1.ListOfString.List - (*ListOfSubcategory_List)(nil), // 195: productv1.ListOfSubcategory.List - (*ListOfUser_List)(nil), // 196: productv1.ListOfUser.List - (*ListOfUserInput_List)(nil), // 197: productv1.ListOfUserInput.List - (*wrapperspb.Int32Value)(nil), // 198: google.protobuf.Int32Value - (*wrapperspb.StringValue)(nil), // 199: google.protobuf.StringValue - (*wrapperspb.DoubleValue)(nil), // 200: google.protobuf.DoubleValue - (*wrapperspb.BoolValue)(nil), // 201: google.protobuf.BoolValue + (ShippingDestination)(0), // 1: productv1.ShippingDestination + (*ListOfAuthorFilter)(nil), // 2: productv1.ListOfAuthorFilter + (*ListOfAuthorInput)(nil), // 3: productv1.ListOfAuthorInput + (*ListOfBlogPost)(nil), // 4: productv1.ListOfBlogPost + (*ListOfBlogPostFilter)(nil), // 5: productv1.ListOfBlogPostFilter + (*ListOfBlogPostInput)(nil), // 6: productv1.ListOfBlogPostInput + (*ListOfBoolean)(nil), // 7: productv1.ListOfBoolean + (*ListOfCategory)(nil), // 8: productv1.ListOfCategory + (*ListOfCategoryInput)(nil), // 9: productv1.ListOfCategoryInput + (*ListOfFloat)(nil), // 10: productv1.ListOfFloat + (*ListOfListOfCategory)(nil), // 11: productv1.ListOfListOfCategory + (*ListOfListOfCategoryInput)(nil), // 12: productv1.ListOfListOfCategoryInput + (*ListOfListOfString)(nil), // 13: productv1.ListOfListOfString + (*ListOfListOfUser)(nil), // 14: productv1.ListOfListOfUser + (*ListOfListOfUserInput)(nil), // 15: productv1.ListOfListOfUserInput + (*ListOfOrderLine)(nil), // 16: productv1.ListOfOrderLine + (*ListOfProduct)(nil), // 17: productv1.ListOfProduct + (*ListOfString)(nil), // 18: productv1.ListOfString + (*ListOfSubcategory)(nil), // 19: productv1.ListOfSubcategory + (*ListOfUser)(nil), // 20: productv1.ListOfUser + (*ListOfUserInput)(nil), // 21: productv1.ListOfUserInput + (*LookupProductByIdRequestKey)(nil), // 22: productv1.LookupProductByIdRequestKey + (*LookupProductByIdRequest)(nil), // 23: productv1.LookupProductByIdRequest + (*LookupProductByIdResponse)(nil), // 24: productv1.LookupProductByIdResponse + (*LookupStorageByIdRequestKey)(nil), // 25: productv1.LookupStorageByIdRequestKey + (*LookupStorageByIdRequest)(nil), // 26: productv1.LookupStorageByIdRequest + (*LookupStorageByIdResponse)(nil), // 27: productv1.LookupStorageByIdResponse + (*LookupWarehouseByIdRequestKey)(nil), // 28: productv1.LookupWarehouseByIdRequestKey + (*LookupWarehouseByIdRequest)(nil), // 29: productv1.LookupWarehouseByIdRequest + (*LookupWarehouseByIdResponse)(nil), // 30: productv1.LookupWarehouseByIdResponse + (*QueryUsersRequest)(nil), // 31: productv1.QueryUsersRequest + (*QueryUsersResponse)(nil), // 32: productv1.QueryUsersResponse + (*QueryUserRequest)(nil), // 33: productv1.QueryUserRequest + (*QueryUserResponse)(nil), // 34: productv1.QueryUserResponse + (*QueryNestedTypeRequest)(nil), // 35: productv1.QueryNestedTypeRequest + (*QueryNestedTypeResponse)(nil), // 36: productv1.QueryNestedTypeResponse + (*QueryRecursiveTypeRequest)(nil), // 37: productv1.QueryRecursiveTypeRequest + (*QueryRecursiveTypeResponse)(nil), // 38: productv1.QueryRecursiveTypeResponse + (*QueryTypeFilterWithArgumentsRequest)(nil), // 39: productv1.QueryTypeFilterWithArgumentsRequest + (*QueryTypeFilterWithArgumentsResponse)(nil), // 40: productv1.QueryTypeFilterWithArgumentsResponse + (*QueryTypeWithMultipleFilterFieldsRequest)(nil), // 41: productv1.QueryTypeWithMultipleFilterFieldsRequest + (*QueryTypeWithMultipleFilterFieldsResponse)(nil), // 42: productv1.QueryTypeWithMultipleFilterFieldsResponse + (*QueryComplexFilterTypeRequest)(nil), // 43: productv1.QueryComplexFilterTypeRequest + (*QueryComplexFilterTypeResponse)(nil), // 44: productv1.QueryComplexFilterTypeResponse + (*QueryCalculateTotalsRequest)(nil), // 45: productv1.QueryCalculateTotalsRequest + (*QueryCalculateTotalsResponse)(nil), // 46: productv1.QueryCalculateTotalsResponse + (*QueryCategoriesRequest)(nil), // 47: productv1.QueryCategoriesRequest + (*QueryCategoriesResponse)(nil), // 48: productv1.QueryCategoriesResponse + (*QueryCategoriesByKindRequest)(nil), // 49: productv1.QueryCategoriesByKindRequest + (*QueryCategoriesByKindResponse)(nil), // 50: productv1.QueryCategoriesByKindResponse + (*QueryCategoriesByKindsRequest)(nil), // 51: productv1.QueryCategoriesByKindsRequest + (*QueryCategoriesByKindsResponse)(nil), // 52: productv1.QueryCategoriesByKindsResponse + (*QueryFilterCategoriesRequest)(nil), // 53: productv1.QueryFilterCategoriesRequest + (*QueryFilterCategoriesResponse)(nil), // 54: productv1.QueryFilterCategoriesResponse + (*QueryRandomPetRequest)(nil), // 55: productv1.QueryRandomPetRequest + (*QueryRandomPetResponse)(nil), // 56: productv1.QueryRandomPetResponse + (*QueryAllPetsRequest)(nil), // 57: productv1.QueryAllPetsRequest + (*QueryAllPetsResponse)(nil), // 58: productv1.QueryAllPetsResponse + (*QuerySearchRequest)(nil), // 59: productv1.QuerySearchRequest + (*QuerySearchResponse)(nil), // 60: productv1.QuerySearchResponse + (*QueryRandomSearchResultRequest)(nil), // 61: productv1.QueryRandomSearchResultRequest + (*QueryRandomSearchResultResponse)(nil), // 62: productv1.QueryRandomSearchResultResponse + (*QueryNullableFieldsTypeRequest)(nil), // 63: productv1.QueryNullableFieldsTypeRequest + (*QueryNullableFieldsTypeResponse)(nil), // 64: productv1.QueryNullableFieldsTypeResponse + (*QueryNullableFieldsTypeByIdRequest)(nil), // 65: productv1.QueryNullableFieldsTypeByIdRequest + (*QueryNullableFieldsTypeByIdResponse)(nil), // 66: productv1.QueryNullableFieldsTypeByIdResponse + (*QueryNullableFieldsTypeWithFilterRequest)(nil), // 67: productv1.QueryNullableFieldsTypeWithFilterRequest + (*QueryNullableFieldsTypeWithFilterResponse)(nil), // 68: productv1.QueryNullableFieldsTypeWithFilterResponse + (*QueryAllNullableFieldsTypesRequest)(nil), // 69: productv1.QueryAllNullableFieldsTypesRequest + (*QueryAllNullableFieldsTypesResponse)(nil), // 70: productv1.QueryAllNullableFieldsTypesResponse + (*QueryBlogPostRequest)(nil), // 71: productv1.QueryBlogPostRequest + (*QueryBlogPostResponse)(nil), // 72: productv1.QueryBlogPostResponse + (*QueryBlogPostByIdRequest)(nil), // 73: productv1.QueryBlogPostByIdRequest + (*QueryBlogPostByIdResponse)(nil), // 74: productv1.QueryBlogPostByIdResponse + (*QueryBlogPostsWithFilterRequest)(nil), // 75: productv1.QueryBlogPostsWithFilterRequest + (*QueryBlogPostsWithFilterResponse)(nil), // 76: productv1.QueryBlogPostsWithFilterResponse + (*QueryAllBlogPostsRequest)(nil), // 77: productv1.QueryAllBlogPostsRequest + (*QueryAllBlogPostsResponse)(nil), // 78: productv1.QueryAllBlogPostsResponse + (*QueryAuthorRequest)(nil), // 79: productv1.QueryAuthorRequest + (*QueryAuthorResponse)(nil), // 80: productv1.QueryAuthorResponse + (*QueryAuthorByIdRequest)(nil), // 81: productv1.QueryAuthorByIdRequest + (*QueryAuthorByIdResponse)(nil), // 82: productv1.QueryAuthorByIdResponse + (*QueryAuthorsWithFilterRequest)(nil), // 83: productv1.QueryAuthorsWithFilterRequest + (*QueryAuthorsWithFilterResponse)(nil), // 84: productv1.QueryAuthorsWithFilterResponse + (*QueryAllAuthorsRequest)(nil), // 85: productv1.QueryAllAuthorsRequest + (*QueryAllAuthorsResponse)(nil), // 86: productv1.QueryAllAuthorsResponse + (*QueryBulkSearchAuthorsRequest)(nil), // 87: productv1.QueryBulkSearchAuthorsRequest + (*QueryBulkSearchAuthorsResponse)(nil), // 88: productv1.QueryBulkSearchAuthorsResponse + (*QueryBulkSearchBlogPostsRequest)(nil), // 89: productv1.QueryBulkSearchBlogPostsRequest + (*QueryBulkSearchBlogPostsResponse)(nil), // 90: productv1.QueryBulkSearchBlogPostsResponse + (*MutationCreateUserRequest)(nil), // 91: productv1.MutationCreateUserRequest + (*MutationCreateUserResponse)(nil), // 92: productv1.MutationCreateUserResponse + (*MutationPerformActionRequest)(nil), // 93: productv1.MutationPerformActionRequest + (*MutationPerformActionResponse)(nil), // 94: productv1.MutationPerformActionResponse + (*MutationCreateNullableFieldsTypeRequest)(nil), // 95: productv1.MutationCreateNullableFieldsTypeRequest + (*MutationCreateNullableFieldsTypeResponse)(nil), // 96: productv1.MutationCreateNullableFieldsTypeResponse + (*MutationUpdateNullableFieldsTypeRequest)(nil), // 97: productv1.MutationUpdateNullableFieldsTypeRequest + (*MutationUpdateNullableFieldsTypeResponse)(nil), // 98: productv1.MutationUpdateNullableFieldsTypeResponse + (*MutationCreateBlogPostRequest)(nil), // 99: productv1.MutationCreateBlogPostRequest + (*MutationCreateBlogPostResponse)(nil), // 100: productv1.MutationCreateBlogPostResponse + (*MutationUpdateBlogPostRequest)(nil), // 101: productv1.MutationUpdateBlogPostRequest + (*MutationUpdateBlogPostResponse)(nil), // 102: productv1.MutationUpdateBlogPostResponse + (*MutationCreateAuthorRequest)(nil), // 103: productv1.MutationCreateAuthorRequest + (*MutationCreateAuthorResponse)(nil), // 104: productv1.MutationCreateAuthorResponse + (*MutationUpdateAuthorRequest)(nil), // 105: productv1.MutationUpdateAuthorRequest + (*MutationUpdateAuthorResponse)(nil), // 106: productv1.MutationUpdateAuthorResponse + (*MutationBulkCreateAuthorsRequest)(nil), // 107: productv1.MutationBulkCreateAuthorsRequest + (*MutationBulkCreateAuthorsResponse)(nil), // 108: productv1.MutationBulkCreateAuthorsResponse + (*MutationBulkUpdateAuthorsRequest)(nil), // 109: productv1.MutationBulkUpdateAuthorsRequest + (*MutationBulkUpdateAuthorsResponse)(nil), // 110: productv1.MutationBulkUpdateAuthorsResponse + (*MutationBulkCreateBlogPostsRequest)(nil), // 111: productv1.MutationBulkCreateBlogPostsRequest + (*MutationBulkCreateBlogPostsResponse)(nil), // 112: productv1.MutationBulkCreateBlogPostsResponse + (*MutationBulkUpdateBlogPostsRequest)(nil), // 113: productv1.MutationBulkUpdateBlogPostsRequest + (*MutationBulkUpdateBlogPostsResponse)(nil), // 114: productv1.MutationBulkUpdateBlogPostsResponse + (*ResolveProductShippingEstimateArgs)(nil), // 115: productv1.ResolveProductShippingEstimateArgs + (*ResolveProductShippingEstimateContext)(nil), // 116: productv1.ResolveProductShippingEstimateContext + (*ResolveProductShippingEstimateRequest)(nil), // 117: productv1.ResolveProductShippingEstimateRequest + (*ResolveProductShippingEstimateResult)(nil), // 118: productv1.ResolveProductShippingEstimateResult + (*ResolveProductShippingEstimateResponse)(nil), // 119: productv1.ResolveProductShippingEstimateResponse + (*ResolveProductRecommendedCategoryArgs)(nil), // 120: productv1.ResolveProductRecommendedCategoryArgs + (*ResolveProductRecommendedCategoryContext)(nil), // 121: productv1.ResolveProductRecommendedCategoryContext + (*ResolveProductRecommendedCategoryRequest)(nil), // 122: productv1.ResolveProductRecommendedCategoryRequest + (*ResolveProductRecommendedCategoryResult)(nil), // 123: productv1.ResolveProductRecommendedCategoryResult + (*ResolveProductRecommendedCategoryResponse)(nil), // 124: productv1.ResolveProductRecommendedCategoryResponse + (*ResolveCategoryProductCountArgs)(nil), // 125: productv1.ResolveCategoryProductCountArgs + (*ResolveCategoryProductCountContext)(nil), // 126: productv1.ResolveCategoryProductCountContext + (*ResolveCategoryProductCountRequest)(nil), // 127: productv1.ResolveCategoryProductCountRequest + (*ResolveCategoryProductCountResult)(nil), // 128: productv1.ResolveCategoryProductCountResult + (*ResolveCategoryProductCountResponse)(nil), // 129: productv1.ResolveCategoryProductCountResponse + (*ResolveCategoryPopularityScoreArgs)(nil), // 130: productv1.ResolveCategoryPopularityScoreArgs + (*ResolveCategoryPopularityScoreContext)(nil), // 131: productv1.ResolveCategoryPopularityScoreContext + (*ResolveCategoryPopularityScoreRequest)(nil), // 132: productv1.ResolveCategoryPopularityScoreRequest + (*ResolveCategoryPopularityScoreResult)(nil), // 133: productv1.ResolveCategoryPopularityScoreResult + (*ResolveCategoryPopularityScoreResponse)(nil), // 134: productv1.ResolveCategoryPopularityScoreResponse + (*ResolveCategoryCategoryMetricsArgs)(nil), // 135: productv1.ResolveCategoryCategoryMetricsArgs + (*ResolveCategoryCategoryMetricsContext)(nil), // 136: productv1.ResolveCategoryCategoryMetricsContext + (*ResolveCategoryCategoryMetricsRequest)(nil), // 137: productv1.ResolveCategoryCategoryMetricsRequest + (*ResolveCategoryCategoryMetricsResult)(nil), // 138: productv1.ResolveCategoryCategoryMetricsResult + (*ResolveCategoryCategoryMetricsResponse)(nil), // 139: productv1.ResolveCategoryCategoryMetricsResponse + (*ResolveSubcategoryItemCountArgs)(nil), // 140: productv1.ResolveSubcategoryItemCountArgs + (*ResolveSubcategoryItemCountContext)(nil), // 141: productv1.ResolveSubcategoryItemCountContext + (*ResolveSubcategoryItemCountRequest)(nil), // 142: productv1.ResolveSubcategoryItemCountRequest + (*ResolveSubcategoryItemCountResult)(nil), // 143: productv1.ResolveSubcategoryItemCountResult + (*ResolveSubcategoryItemCountResponse)(nil), // 144: productv1.ResolveSubcategoryItemCountResponse + (*Product)(nil), // 145: productv1.Product + (*Storage)(nil), // 146: productv1.Storage + (*Warehouse)(nil), // 147: productv1.Warehouse + (*User)(nil), // 148: productv1.User + (*NestedTypeA)(nil), // 149: productv1.NestedTypeA + (*RecursiveType)(nil), // 150: productv1.RecursiveType + (*TypeWithMultipleFilterFields)(nil), // 151: productv1.TypeWithMultipleFilterFields + (*FilterTypeInput)(nil), // 152: productv1.FilterTypeInput + (*ComplexFilterTypeInput)(nil), // 153: productv1.ComplexFilterTypeInput + (*TypeWithComplexFilterInput)(nil), // 154: productv1.TypeWithComplexFilterInput + (*OrderInput)(nil), // 155: productv1.OrderInput + (*Order)(nil), // 156: productv1.Order + (*Category)(nil), // 157: productv1.Category + (*CategoryFilter)(nil), // 158: productv1.CategoryFilter + (*Animal)(nil), // 159: productv1.Animal + (*SearchInput)(nil), // 160: productv1.SearchInput + (*SearchResult)(nil), // 161: productv1.SearchResult + (*NullableFieldsType)(nil), // 162: productv1.NullableFieldsType + (*NullableFieldsFilter)(nil), // 163: productv1.NullableFieldsFilter + (*BlogPost)(nil), // 164: productv1.BlogPost + (*BlogPostFilter)(nil), // 165: productv1.BlogPostFilter + (*Author)(nil), // 166: productv1.Author + (*AuthorFilter)(nil), // 167: productv1.AuthorFilter + (*UserInput)(nil), // 168: productv1.UserInput + (*ActionInput)(nil), // 169: productv1.ActionInput + (*ActionResult)(nil), // 170: productv1.ActionResult + (*NullableFieldsInput)(nil), // 171: productv1.NullableFieldsInput + (*BlogPostInput)(nil), // 172: productv1.BlogPostInput + (*AuthorInput)(nil), // 173: productv1.AuthorInput + (*NestedTypeB)(nil), // 174: productv1.NestedTypeB + (*NestedTypeC)(nil), // 175: productv1.NestedTypeC + (*FilterType)(nil), // 176: productv1.FilterType + (*Pagination)(nil), // 177: productv1.Pagination + (*OrderLineInput)(nil), // 178: productv1.OrderLineInput + (*OrderLine)(nil), // 179: productv1.OrderLine + (*Subcategory)(nil), // 180: productv1.Subcategory + (*CategoryMetrics)(nil), // 181: productv1.CategoryMetrics + (*Cat)(nil), // 182: productv1.Cat + (*Dog)(nil), // 183: productv1.Dog + (*ActionSuccess)(nil), // 184: productv1.ActionSuccess + (*ActionError)(nil), // 185: productv1.ActionError + (*CategoryInput)(nil), // 186: productv1.CategoryInput + (*ProductCountFilter)(nil), // 187: productv1.ProductCountFilter + (*SubcategoryItemFilter)(nil), // 188: productv1.SubcategoryItemFilter + (*ShippingEstimateInput)(nil), // 189: productv1.ShippingEstimateInput + (*ListOfAuthorFilter_List)(nil), // 190: productv1.ListOfAuthorFilter.List + (*ListOfAuthorInput_List)(nil), // 191: productv1.ListOfAuthorInput.List + (*ListOfBlogPost_List)(nil), // 192: productv1.ListOfBlogPost.List + (*ListOfBlogPostFilter_List)(nil), // 193: productv1.ListOfBlogPostFilter.List + (*ListOfBlogPostInput_List)(nil), // 194: productv1.ListOfBlogPostInput.List + (*ListOfBoolean_List)(nil), // 195: productv1.ListOfBoolean.List + (*ListOfCategory_List)(nil), // 196: productv1.ListOfCategory.List + (*ListOfCategoryInput_List)(nil), // 197: productv1.ListOfCategoryInput.List + (*ListOfFloat_List)(nil), // 198: productv1.ListOfFloat.List + (*ListOfListOfCategory_List)(nil), // 199: productv1.ListOfListOfCategory.List + (*ListOfListOfCategoryInput_List)(nil), // 200: productv1.ListOfListOfCategoryInput.List + (*ListOfListOfString_List)(nil), // 201: productv1.ListOfListOfString.List + (*ListOfListOfUser_List)(nil), // 202: productv1.ListOfListOfUser.List + (*ListOfListOfUserInput_List)(nil), // 203: productv1.ListOfListOfUserInput.List + (*ListOfOrderLine_List)(nil), // 204: productv1.ListOfOrderLine.List + (*ListOfProduct_List)(nil), // 205: productv1.ListOfProduct.List + (*ListOfString_List)(nil), // 206: productv1.ListOfString.List + (*ListOfSubcategory_List)(nil), // 207: productv1.ListOfSubcategory.List + (*ListOfUser_List)(nil), // 208: productv1.ListOfUser.List + (*ListOfUserInput_List)(nil), // 209: productv1.ListOfUserInput.List + (*wrapperspb.Int32Value)(nil), // 210: google.protobuf.Int32Value + (*wrapperspb.StringValue)(nil), // 211: google.protobuf.StringValue + (*wrapperspb.DoubleValue)(nil), // 212: google.protobuf.DoubleValue + (*wrapperspb.BoolValue)(nil), // 213: google.protobuf.BoolValue } var file_product_proto_depIdxs = []int32{ - 178, // 0: productv1.ListOfAuthorFilter.list:type_name -> productv1.ListOfAuthorFilter.List - 179, // 1: productv1.ListOfAuthorInput.list:type_name -> productv1.ListOfAuthorInput.List - 180, // 2: productv1.ListOfBlogPost.list:type_name -> productv1.ListOfBlogPost.List - 181, // 3: productv1.ListOfBlogPostFilter.list:type_name -> productv1.ListOfBlogPostFilter.List - 182, // 4: productv1.ListOfBlogPostInput.list:type_name -> productv1.ListOfBlogPostInput.List - 183, // 5: productv1.ListOfBoolean.list:type_name -> productv1.ListOfBoolean.List - 184, // 6: productv1.ListOfCategory.list:type_name -> productv1.ListOfCategory.List - 185, // 7: productv1.ListOfCategoryInput.list:type_name -> productv1.ListOfCategoryInput.List - 186, // 8: productv1.ListOfFloat.list:type_name -> productv1.ListOfFloat.List - 187, // 9: productv1.ListOfListOfCategory.list:type_name -> productv1.ListOfListOfCategory.List - 188, // 10: productv1.ListOfListOfCategoryInput.list:type_name -> productv1.ListOfListOfCategoryInput.List - 189, // 11: productv1.ListOfListOfString.list:type_name -> productv1.ListOfListOfString.List - 190, // 12: productv1.ListOfListOfUser.list:type_name -> productv1.ListOfListOfUser.List - 191, // 13: productv1.ListOfListOfUserInput.list:type_name -> productv1.ListOfListOfUserInput.List - 192, // 14: productv1.ListOfOrderLine.list:type_name -> productv1.ListOfOrderLine.List - 193, // 15: productv1.ListOfProduct.list:type_name -> productv1.ListOfProduct.List - 194, // 16: productv1.ListOfString.list:type_name -> productv1.ListOfString.List - 195, // 17: productv1.ListOfSubcategory.list:type_name -> productv1.ListOfSubcategory.List - 196, // 18: productv1.ListOfUser.list:type_name -> productv1.ListOfUser.List - 197, // 19: productv1.ListOfUserInput.list:type_name -> productv1.ListOfUserInput.List - 21, // 20: productv1.LookupProductByIdRequest.keys:type_name -> productv1.LookupProductByIdRequestKey - 134, // 21: productv1.LookupProductByIdResponse.result:type_name -> productv1.Product - 24, // 22: productv1.LookupStorageByIdRequest.keys:type_name -> productv1.LookupStorageByIdRequestKey - 135, // 23: productv1.LookupStorageByIdResponse.result:type_name -> productv1.Storage - 27, // 24: productv1.LookupWarehouseByIdRequest.keys:type_name -> productv1.LookupWarehouseByIdRequestKey - 136, // 25: productv1.LookupWarehouseByIdResponse.result:type_name -> productv1.Warehouse - 137, // 26: productv1.QueryUsersResponse.users:type_name -> productv1.User - 137, // 27: productv1.QueryUserResponse.user:type_name -> productv1.User - 138, // 28: productv1.QueryNestedTypeResponse.nested_type:type_name -> productv1.NestedTypeA - 139, // 29: productv1.QueryRecursiveTypeResponse.recursive_type:type_name -> productv1.RecursiveType - 140, // 30: productv1.QueryTypeFilterWithArgumentsResponse.type_filter_with_arguments:type_name -> productv1.TypeWithMultipleFilterFields - 141, // 31: productv1.QueryTypeWithMultipleFilterFieldsRequest.filter:type_name -> productv1.FilterTypeInput - 140, // 32: productv1.QueryTypeWithMultipleFilterFieldsResponse.type_with_multiple_filter_fields:type_name -> productv1.TypeWithMultipleFilterFields - 142, // 33: productv1.QueryComplexFilterTypeRequest.filter:type_name -> productv1.ComplexFilterTypeInput - 143, // 34: productv1.QueryComplexFilterTypeResponse.complex_filter_type:type_name -> productv1.TypeWithComplexFilterInput - 144, // 35: productv1.QueryCalculateTotalsRequest.orders:type_name -> productv1.OrderInput - 145, // 36: productv1.QueryCalculateTotalsResponse.calculate_totals:type_name -> productv1.Order - 146, // 37: productv1.QueryCategoriesResponse.categories:type_name -> productv1.Category + 190, // 0: productv1.ListOfAuthorFilter.list:type_name -> productv1.ListOfAuthorFilter.List + 191, // 1: productv1.ListOfAuthorInput.list:type_name -> productv1.ListOfAuthorInput.List + 192, // 2: productv1.ListOfBlogPost.list:type_name -> productv1.ListOfBlogPost.List + 193, // 3: productv1.ListOfBlogPostFilter.list:type_name -> productv1.ListOfBlogPostFilter.List + 194, // 4: productv1.ListOfBlogPostInput.list:type_name -> productv1.ListOfBlogPostInput.List + 195, // 5: productv1.ListOfBoolean.list:type_name -> productv1.ListOfBoolean.List + 196, // 6: productv1.ListOfCategory.list:type_name -> productv1.ListOfCategory.List + 197, // 7: productv1.ListOfCategoryInput.list:type_name -> productv1.ListOfCategoryInput.List + 198, // 8: productv1.ListOfFloat.list:type_name -> productv1.ListOfFloat.List + 199, // 9: productv1.ListOfListOfCategory.list:type_name -> productv1.ListOfListOfCategory.List + 200, // 10: productv1.ListOfListOfCategoryInput.list:type_name -> productv1.ListOfListOfCategoryInput.List + 201, // 11: productv1.ListOfListOfString.list:type_name -> productv1.ListOfListOfString.List + 202, // 12: productv1.ListOfListOfUser.list:type_name -> productv1.ListOfListOfUser.List + 203, // 13: productv1.ListOfListOfUserInput.list:type_name -> productv1.ListOfListOfUserInput.List + 204, // 14: productv1.ListOfOrderLine.list:type_name -> productv1.ListOfOrderLine.List + 205, // 15: productv1.ListOfProduct.list:type_name -> productv1.ListOfProduct.List + 206, // 16: productv1.ListOfString.list:type_name -> productv1.ListOfString.List + 207, // 17: productv1.ListOfSubcategory.list:type_name -> productv1.ListOfSubcategory.List + 208, // 18: productv1.ListOfUser.list:type_name -> productv1.ListOfUser.List + 209, // 19: productv1.ListOfUserInput.list:type_name -> productv1.ListOfUserInput.List + 22, // 20: productv1.LookupProductByIdRequest.keys:type_name -> productv1.LookupProductByIdRequestKey + 145, // 21: productv1.LookupProductByIdResponse.result:type_name -> productv1.Product + 25, // 22: productv1.LookupStorageByIdRequest.keys:type_name -> productv1.LookupStorageByIdRequestKey + 146, // 23: productv1.LookupStorageByIdResponse.result:type_name -> productv1.Storage + 28, // 24: productv1.LookupWarehouseByIdRequest.keys:type_name -> productv1.LookupWarehouseByIdRequestKey + 147, // 25: productv1.LookupWarehouseByIdResponse.result:type_name -> productv1.Warehouse + 148, // 26: productv1.QueryUsersResponse.users:type_name -> productv1.User + 148, // 27: productv1.QueryUserResponse.user:type_name -> productv1.User + 149, // 28: productv1.QueryNestedTypeResponse.nested_type:type_name -> productv1.NestedTypeA + 150, // 29: productv1.QueryRecursiveTypeResponse.recursive_type:type_name -> productv1.RecursiveType + 151, // 30: productv1.QueryTypeFilterWithArgumentsResponse.type_filter_with_arguments:type_name -> productv1.TypeWithMultipleFilterFields + 152, // 31: productv1.QueryTypeWithMultipleFilterFieldsRequest.filter:type_name -> productv1.FilterTypeInput + 151, // 32: productv1.QueryTypeWithMultipleFilterFieldsResponse.type_with_multiple_filter_fields:type_name -> productv1.TypeWithMultipleFilterFields + 153, // 33: productv1.QueryComplexFilterTypeRequest.filter:type_name -> productv1.ComplexFilterTypeInput + 154, // 34: productv1.QueryComplexFilterTypeResponse.complex_filter_type:type_name -> productv1.TypeWithComplexFilterInput + 155, // 35: productv1.QueryCalculateTotalsRequest.orders:type_name -> productv1.OrderInput + 156, // 36: productv1.QueryCalculateTotalsResponse.calculate_totals:type_name -> productv1.Order + 157, // 37: productv1.QueryCategoriesResponse.categories:type_name -> productv1.Category 0, // 38: productv1.QueryCategoriesByKindRequest.kind:type_name -> productv1.CategoryKind - 146, // 39: productv1.QueryCategoriesByKindResponse.categories_by_kind:type_name -> productv1.Category + 157, // 39: productv1.QueryCategoriesByKindResponse.categories_by_kind:type_name -> productv1.Category 0, // 40: productv1.QueryCategoriesByKindsRequest.kinds:type_name -> productv1.CategoryKind - 146, // 41: productv1.QueryCategoriesByKindsResponse.categories_by_kinds:type_name -> productv1.Category - 147, // 42: productv1.QueryFilterCategoriesRequest.filter:type_name -> productv1.CategoryFilter - 146, // 43: productv1.QueryFilterCategoriesResponse.filter_categories:type_name -> productv1.Category - 148, // 44: productv1.QueryRandomPetResponse.random_pet:type_name -> productv1.Animal - 148, // 45: productv1.QueryAllPetsResponse.all_pets:type_name -> productv1.Animal - 149, // 46: productv1.QuerySearchRequest.input:type_name -> productv1.SearchInput - 150, // 47: productv1.QuerySearchResponse.search:type_name -> productv1.SearchResult - 150, // 48: productv1.QueryRandomSearchResultResponse.random_search_result:type_name -> productv1.SearchResult - 151, // 49: productv1.QueryNullableFieldsTypeResponse.nullable_fields_type:type_name -> productv1.NullableFieldsType - 151, // 50: productv1.QueryNullableFieldsTypeByIdResponse.nullable_fields_type_by_id:type_name -> productv1.NullableFieldsType - 152, // 51: productv1.QueryNullableFieldsTypeWithFilterRequest.filter:type_name -> productv1.NullableFieldsFilter - 151, // 52: productv1.QueryNullableFieldsTypeWithFilterResponse.nullable_fields_type_with_filter:type_name -> productv1.NullableFieldsType - 151, // 53: productv1.QueryAllNullableFieldsTypesResponse.all_nullable_fields_types:type_name -> productv1.NullableFieldsType - 153, // 54: productv1.QueryBlogPostResponse.blog_post:type_name -> productv1.BlogPost - 153, // 55: productv1.QueryBlogPostByIdResponse.blog_post_by_id:type_name -> productv1.BlogPost - 154, // 56: productv1.QueryBlogPostsWithFilterRequest.filter:type_name -> productv1.BlogPostFilter - 153, // 57: productv1.QueryBlogPostsWithFilterResponse.blog_posts_with_filter:type_name -> productv1.BlogPost - 153, // 58: productv1.QueryAllBlogPostsResponse.all_blog_posts:type_name -> productv1.BlogPost - 155, // 59: productv1.QueryAuthorResponse.author:type_name -> productv1.Author - 155, // 60: productv1.QueryAuthorByIdResponse.author_by_id:type_name -> productv1.Author - 156, // 61: productv1.QueryAuthorsWithFilterRequest.filter:type_name -> productv1.AuthorFilter - 155, // 62: productv1.QueryAuthorsWithFilterResponse.authors_with_filter:type_name -> productv1.Author - 155, // 63: productv1.QueryAllAuthorsResponse.all_authors:type_name -> productv1.Author - 1, // 64: productv1.QueryBulkSearchAuthorsRequest.filters:type_name -> productv1.ListOfAuthorFilter - 155, // 65: productv1.QueryBulkSearchAuthorsResponse.bulk_search_authors:type_name -> productv1.Author - 4, // 66: productv1.QueryBulkSearchBlogPostsRequest.filters:type_name -> productv1.ListOfBlogPostFilter - 153, // 67: productv1.QueryBulkSearchBlogPostsResponse.bulk_search_blog_posts:type_name -> productv1.BlogPost - 157, // 68: productv1.MutationCreateUserRequest.input:type_name -> productv1.UserInput - 137, // 69: productv1.MutationCreateUserResponse.create_user:type_name -> productv1.User - 158, // 70: productv1.MutationPerformActionRequest.input:type_name -> productv1.ActionInput - 159, // 71: productv1.MutationPerformActionResponse.perform_action:type_name -> productv1.ActionResult - 160, // 72: productv1.MutationCreateNullableFieldsTypeRequest.input:type_name -> productv1.NullableFieldsInput - 151, // 73: productv1.MutationCreateNullableFieldsTypeResponse.create_nullable_fields_type:type_name -> productv1.NullableFieldsType - 160, // 74: productv1.MutationUpdateNullableFieldsTypeRequest.input:type_name -> productv1.NullableFieldsInput - 151, // 75: productv1.MutationUpdateNullableFieldsTypeResponse.update_nullable_fields_type:type_name -> productv1.NullableFieldsType - 161, // 76: productv1.MutationCreateBlogPostRequest.input:type_name -> productv1.BlogPostInput - 153, // 77: productv1.MutationCreateBlogPostResponse.create_blog_post:type_name -> productv1.BlogPost - 161, // 78: productv1.MutationUpdateBlogPostRequest.input:type_name -> productv1.BlogPostInput - 153, // 79: productv1.MutationUpdateBlogPostResponse.update_blog_post:type_name -> productv1.BlogPost - 162, // 80: productv1.MutationCreateAuthorRequest.input:type_name -> productv1.AuthorInput - 155, // 81: productv1.MutationCreateAuthorResponse.create_author:type_name -> productv1.Author - 162, // 82: productv1.MutationUpdateAuthorRequest.input:type_name -> productv1.AuthorInput - 155, // 83: productv1.MutationUpdateAuthorResponse.update_author:type_name -> productv1.Author - 2, // 84: productv1.MutationBulkCreateAuthorsRequest.authors:type_name -> productv1.ListOfAuthorInput - 155, // 85: productv1.MutationBulkCreateAuthorsResponse.bulk_create_authors:type_name -> productv1.Author - 2, // 86: productv1.MutationBulkUpdateAuthorsRequest.authors:type_name -> productv1.ListOfAuthorInput - 155, // 87: productv1.MutationBulkUpdateAuthorsResponse.bulk_update_authors:type_name -> productv1.Author - 5, // 88: productv1.MutationBulkCreateBlogPostsRequest.blog_posts:type_name -> productv1.ListOfBlogPostInput - 153, // 89: productv1.MutationBulkCreateBlogPostsResponse.bulk_create_blog_posts:type_name -> productv1.BlogPost - 5, // 90: productv1.MutationBulkUpdateBlogPostsRequest.blog_posts:type_name -> productv1.ListOfBlogPostInput - 153, // 91: productv1.MutationBulkUpdateBlogPostsResponse.bulk_update_blog_posts:type_name -> productv1.BlogPost - 176, // 92: productv1.ResolveCategoryProductCountArgs.filters:type_name -> productv1.ProductCountFilter - 115, // 93: productv1.ResolveCategoryProductCountRequest.context:type_name -> productv1.ResolveCategoryProductCountContext - 114, // 94: productv1.ResolveCategoryProductCountRequest.field_args:type_name -> productv1.ResolveCategoryProductCountArgs - 117, // 95: productv1.ResolveCategoryProductCountResponse.result:type_name -> productv1.ResolveCategoryProductCountResult - 198, // 96: productv1.ResolveCategoryPopularityScoreArgs.threshold:type_name -> google.protobuf.Int32Value - 120, // 97: productv1.ResolveCategoryPopularityScoreRequest.context:type_name -> productv1.ResolveCategoryPopularityScoreContext - 119, // 98: productv1.ResolveCategoryPopularityScoreRequest.field_args:type_name -> productv1.ResolveCategoryPopularityScoreArgs - 198, // 99: productv1.ResolveCategoryPopularityScoreResult.popularity_score:type_name -> google.protobuf.Int32Value - 122, // 100: productv1.ResolveCategoryPopularityScoreResponse.result:type_name -> productv1.ResolveCategoryPopularityScoreResult - 125, // 101: productv1.ResolveCategoryCategoryMetricsRequest.context:type_name -> productv1.ResolveCategoryCategoryMetricsContext - 124, // 102: productv1.ResolveCategoryCategoryMetricsRequest.field_args:type_name -> productv1.ResolveCategoryCategoryMetricsArgs - 170, // 103: productv1.ResolveCategoryCategoryMetricsResult.category_metrics:type_name -> productv1.CategoryMetrics - 127, // 104: productv1.ResolveCategoryCategoryMetricsResponse.result:type_name -> productv1.ResolveCategoryCategoryMetricsResult - 177, // 105: productv1.ResolveSubcategoryItemCountArgs.filters:type_name -> productv1.SubcategoryItemFilter - 130, // 106: productv1.ResolveSubcategoryItemCountRequest.context:type_name -> productv1.ResolveSubcategoryItemCountContext - 129, // 107: productv1.ResolveSubcategoryItemCountRequest.field_args:type_name -> productv1.ResolveSubcategoryItemCountArgs - 132, // 108: productv1.ResolveSubcategoryItemCountResponse.result:type_name -> productv1.ResolveSubcategoryItemCountResult - 163, // 109: productv1.NestedTypeA.b:type_name -> productv1.NestedTypeB - 139, // 110: productv1.RecursiveType.recursive_type:type_name -> productv1.RecursiveType - 165, // 111: productv1.ComplexFilterTypeInput.filter:type_name -> productv1.FilterType - 167, // 112: productv1.OrderInput.lines:type_name -> productv1.OrderLineInput - 15, // 113: productv1.Order.order_lines:type_name -> productv1.ListOfOrderLine - 0, // 114: productv1.Category.kind:type_name -> productv1.CategoryKind - 18, // 115: productv1.Category.subcategories:type_name -> productv1.ListOfSubcategory - 0, // 116: productv1.CategoryFilter.category:type_name -> productv1.CategoryKind - 166, // 117: productv1.CategoryFilter.pagination:type_name -> productv1.Pagination - 171, // 118: productv1.Animal.cat:type_name -> productv1.Cat - 172, // 119: productv1.Animal.dog:type_name -> productv1.Dog - 198, // 120: productv1.SearchInput.limit:type_name -> google.protobuf.Int32Value - 134, // 121: productv1.SearchResult.product:type_name -> productv1.Product - 137, // 122: productv1.SearchResult.user:type_name -> productv1.User - 146, // 123: productv1.SearchResult.category:type_name -> productv1.Category - 199, // 124: productv1.NullableFieldsType.optional_string:type_name -> google.protobuf.StringValue - 198, // 125: productv1.NullableFieldsType.optional_int:type_name -> google.protobuf.Int32Value - 200, // 126: productv1.NullableFieldsType.optional_float:type_name -> google.protobuf.DoubleValue - 201, // 127: productv1.NullableFieldsType.optional_boolean:type_name -> google.protobuf.BoolValue - 199, // 128: productv1.NullableFieldsFilter.name:type_name -> google.protobuf.StringValue - 199, // 129: productv1.NullableFieldsFilter.optional_string:type_name -> google.protobuf.StringValue - 201, // 130: productv1.NullableFieldsFilter.include_nulls:type_name -> google.protobuf.BoolValue - 17, // 131: productv1.BlogPost.optional_tags:type_name -> productv1.ListOfString - 17, // 132: productv1.BlogPost.keywords:type_name -> productv1.ListOfString - 9, // 133: productv1.BlogPost.ratings:type_name -> productv1.ListOfFloat - 6, // 134: productv1.BlogPost.is_published:type_name -> productv1.ListOfBoolean - 12, // 135: productv1.BlogPost.tag_groups:type_name -> productv1.ListOfListOfString - 12, // 136: productv1.BlogPost.related_topics:type_name -> productv1.ListOfListOfString - 12, // 137: productv1.BlogPost.comment_threads:type_name -> productv1.ListOfListOfString - 12, // 138: productv1.BlogPost.suggestions:type_name -> productv1.ListOfListOfString - 146, // 139: productv1.BlogPost.related_categories:type_name -> productv1.Category - 137, // 140: productv1.BlogPost.contributors:type_name -> productv1.User - 16, // 141: productv1.BlogPost.mentioned_products:type_name -> productv1.ListOfProduct - 19, // 142: productv1.BlogPost.mentioned_users:type_name -> productv1.ListOfUser - 10, // 143: productv1.BlogPost.category_groups:type_name -> productv1.ListOfListOfCategory - 13, // 144: productv1.BlogPost.contributor_teams:type_name -> productv1.ListOfListOfUser - 199, // 145: productv1.BlogPostFilter.title:type_name -> google.protobuf.StringValue - 201, // 146: productv1.BlogPostFilter.has_categories:type_name -> google.protobuf.BoolValue - 198, // 147: productv1.BlogPostFilter.min_tags:type_name -> google.protobuf.Int32Value - 199, // 148: productv1.Author.email:type_name -> google.protobuf.StringValue - 17, // 149: productv1.Author.social_links:type_name -> productv1.ListOfString - 12, // 150: productv1.Author.teams_by_project:type_name -> productv1.ListOfListOfString - 12, // 151: productv1.Author.collaborations:type_name -> productv1.ListOfListOfString - 3, // 152: productv1.Author.written_posts:type_name -> productv1.ListOfBlogPost - 146, // 153: productv1.Author.favorite_categories:type_name -> productv1.Category - 19, // 154: productv1.Author.related_authors:type_name -> productv1.ListOfUser - 16, // 155: productv1.Author.product_reviews:type_name -> productv1.ListOfProduct - 13, // 156: productv1.Author.author_groups:type_name -> productv1.ListOfListOfUser - 10, // 157: productv1.Author.category_preferences:type_name -> productv1.ListOfListOfCategory - 13, // 158: productv1.Author.project_teams:type_name -> productv1.ListOfListOfUser - 199, // 159: productv1.AuthorFilter.name:type_name -> google.protobuf.StringValue - 201, // 160: productv1.AuthorFilter.has_teams:type_name -> google.protobuf.BoolValue - 198, // 161: productv1.AuthorFilter.skill_count:type_name -> google.protobuf.Int32Value - 173, // 162: productv1.ActionResult.action_success:type_name -> productv1.ActionSuccess - 174, // 163: productv1.ActionResult.action_error:type_name -> productv1.ActionError - 199, // 164: productv1.NullableFieldsInput.optional_string:type_name -> google.protobuf.StringValue - 198, // 165: productv1.NullableFieldsInput.optional_int:type_name -> google.protobuf.Int32Value - 200, // 166: productv1.NullableFieldsInput.optional_float:type_name -> google.protobuf.DoubleValue - 201, // 167: productv1.NullableFieldsInput.optional_boolean:type_name -> google.protobuf.BoolValue - 17, // 168: productv1.BlogPostInput.optional_tags:type_name -> productv1.ListOfString - 17, // 169: productv1.BlogPostInput.keywords:type_name -> productv1.ListOfString - 9, // 170: productv1.BlogPostInput.ratings:type_name -> productv1.ListOfFloat - 6, // 171: productv1.BlogPostInput.is_published:type_name -> productv1.ListOfBoolean - 12, // 172: productv1.BlogPostInput.tag_groups:type_name -> productv1.ListOfListOfString - 12, // 173: productv1.BlogPostInput.related_topics:type_name -> productv1.ListOfListOfString - 12, // 174: productv1.BlogPostInput.comment_threads:type_name -> productv1.ListOfListOfString - 12, // 175: productv1.BlogPostInput.suggestions:type_name -> productv1.ListOfListOfString - 8, // 176: productv1.BlogPostInput.related_categories:type_name -> productv1.ListOfCategoryInput - 20, // 177: productv1.BlogPostInput.contributors:type_name -> productv1.ListOfUserInput - 11, // 178: productv1.BlogPostInput.category_groups:type_name -> productv1.ListOfListOfCategoryInput - 199, // 179: productv1.AuthorInput.email:type_name -> google.protobuf.StringValue - 17, // 180: productv1.AuthorInput.social_links:type_name -> productv1.ListOfString - 12, // 181: productv1.AuthorInput.teams_by_project:type_name -> productv1.ListOfListOfString - 12, // 182: productv1.AuthorInput.collaborations:type_name -> productv1.ListOfListOfString - 175, // 183: productv1.AuthorInput.favorite_categories:type_name -> productv1.CategoryInput - 14, // 184: productv1.AuthorInput.author_groups:type_name -> productv1.ListOfListOfUserInput - 14, // 185: productv1.AuthorInput.project_teams:type_name -> productv1.ListOfListOfUserInput - 164, // 186: productv1.NestedTypeB.c:type_name -> productv1.NestedTypeC - 166, // 187: productv1.FilterType.pagination:type_name -> productv1.Pagination - 17, // 188: productv1.OrderLineInput.modifiers:type_name -> productv1.ListOfString - 17, // 189: productv1.OrderLine.modifiers:type_name -> productv1.ListOfString - 199, // 190: productv1.Subcategory.description:type_name -> google.protobuf.StringValue - 0, // 191: productv1.CategoryInput.kind:type_name -> productv1.CategoryKind - 200, // 192: productv1.ProductCountFilter.min_price:type_name -> google.protobuf.DoubleValue - 200, // 193: productv1.ProductCountFilter.max_price:type_name -> google.protobuf.DoubleValue - 201, // 194: productv1.ProductCountFilter.in_stock:type_name -> google.protobuf.BoolValue - 199, // 195: productv1.ProductCountFilter.search_term:type_name -> google.protobuf.StringValue - 200, // 196: productv1.SubcategoryItemFilter.min_price:type_name -> google.protobuf.DoubleValue - 200, // 197: productv1.SubcategoryItemFilter.max_price:type_name -> google.protobuf.DoubleValue - 201, // 198: productv1.SubcategoryItemFilter.in_stock:type_name -> google.protobuf.BoolValue - 201, // 199: productv1.SubcategoryItemFilter.is_active:type_name -> google.protobuf.BoolValue - 199, // 200: productv1.SubcategoryItemFilter.search_term:type_name -> google.protobuf.StringValue - 156, // 201: productv1.ListOfAuthorFilter.List.items:type_name -> productv1.AuthorFilter - 162, // 202: productv1.ListOfAuthorInput.List.items:type_name -> productv1.AuthorInput - 153, // 203: productv1.ListOfBlogPost.List.items:type_name -> productv1.BlogPost - 154, // 204: productv1.ListOfBlogPostFilter.List.items:type_name -> productv1.BlogPostFilter - 161, // 205: productv1.ListOfBlogPostInput.List.items:type_name -> productv1.BlogPostInput - 146, // 206: productv1.ListOfCategory.List.items:type_name -> productv1.Category - 175, // 207: productv1.ListOfCategoryInput.List.items:type_name -> productv1.CategoryInput - 7, // 208: productv1.ListOfListOfCategory.List.items:type_name -> productv1.ListOfCategory - 8, // 209: productv1.ListOfListOfCategoryInput.List.items:type_name -> productv1.ListOfCategoryInput - 17, // 210: productv1.ListOfListOfString.List.items:type_name -> productv1.ListOfString - 19, // 211: productv1.ListOfListOfUser.List.items:type_name -> productv1.ListOfUser - 20, // 212: productv1.ListOfListOfUserInput.List.items:type_name -> productv1.ListOfUserInput - 168, // 213: productv1.ListOfOrderLine.List.items:type_name -> productv1.OrderLine - 134, // 214: productv1.ListOfProduct.List.items:type_name -> productv1.Product - 169, // 215: productv1.ListOfSubcategory.List.items:type_name -> productv1.Subcategory - 137, // 216: productv1.ListOfUser.List.items:type_name -> productv1.User - 157, // 217: productv1.ListOfUserInput.List.items:type_name -> productv1.UserInput - 22, // 218: productv1.ProductService.LookupProductById:input_type -> productv1.LookupProductByIdRequest - 25, // 219: productv1.ProductService.LookupStorageById:input_type -> productv1.LookupStorageByIdRequest - 28, // 220: productv1.ProductService.LookupWarehouseById:input_type -> productv1.LookupWarehouseByIdRequest - 106, // 221: productv1.ProductService.MutationBulkCreateAuthors:input_type -> productv1.MutationBulkCreateAuthorsRequest - 110, // 222: productv1.ProductService.MutationBulkCreateBlogPosts:input_type -> productv1.MutationBulkCreateBlogPostsRequest - 108, // 223: productv1.ProductService.MutationBulkUpdateAuthors:input_type -> productv1.MutationBulkUpdateAuthorsRequest - 112, // 224: productv1.ProductService.MutationBulkUpdateBlogPosts:input_type -> productv1.MutationBulkUpdateBlogPostsRequest - 102, // 225: productv1.ProductService.MutationCreateAuthor:input_type -> productv1.MutationCreateAuthorRequest - 98, // 226: productv1.ProductService.MutationCreateBlogPost:input_type -> productv1.MutationCreateBlogPostRequest - 94, // 227: productv1.ProductService.MutationCreateNullableFieldsType:input_type -> productv1.MutationCreateNullableFieldsTypeRequest - 90, // 228: productv1.ProductService.MutationCreateUser:input_type -> productv1.MutationCreateUserRequest - 92, // 229: productv1.ProductService.MutationPerformAction:input_type -> productv1.MutationPerformActionRequest - 104, // 230: productv1.ProductService.MutationUpdateAuthor:input_type -> productv1.MutationUpdateAuthorRequest - 100, // 231: productv1.ProductService.MutationUpdateBlogPost:input_type -> productv1.MutationUpdateBlogPostRequest - 96, // 232: productv1.ProductService.MutationUpdateNullableFieldsType:input_type -> productv1.MutationUpdateNullableFieldsTypeRequest - 84, // 233: productv1.ProductService.QueryAllAuthors:input_type -> productv1.QueryAllAuthorsRequest - 76, // 234: productv1.ProductService.QueryAllBlogPosts:input_type -> productv1.QueryAllBlogPostsRequest - 68, // 235: productv1.ProductService.QueryAllNullableFieldsTypes:input_type -> productv1.QueryAllNullableFieldsTypesRequest - 56, // 236: productv1.ProductService.QueryAllPets:input_type -> productv1.QueryAllPetsRequest - 78, // 237: productv1.ProductService.QueryAuthor:input_type -> productv1.QueryAuthorRequest - 80, // 238: productv1.ProductService.QueryAuthorById:input_type -> productv1.QueryAuthorByIdRequest - 82, // 239: productv1.ProductService.QueryAuthorsWithFilter:input_type -> productv1.QueryAuthorsWithFilterRequest - 70, // 240: productv1.ProductService.QueryBlogPost:input_type -> productv1.QueryBlogPostRequest - 72, // 241: productv1.ProductService.QueryBlogPostById:input_type -> productv1.QueryBlogPostByIdRequest - 74, // 242: productv1.ProductService.QueryBlogPostsWithFilter:input_type -> productv1.QueryBlogPostsWithFilterRequest - 86, // 243: productv1.ProductService.QueryBulkSearchAuthors:input_type -> productv1.QueryBulkSearchAuthorsRequest - 88, // 244: productv1.ProductService.QueryBulkSearchBlogPosts:input_type -> productv1.QueryBulkSearchBlogPostsRequest - 44, // 245: productv1.ProductService.QueryCalculateTotals:input_type -> productv1.QueryCalculateTotalsRequest - 46, // 246: productv1.ProductService.QueryCategories:input_type -> productv1.QueryCategoriesRequest - 48, // 247: productv1.ProductService.QueryCategoriesByKind:input_type -> productv1.QueryCategoriesByKindRequest - 50, // 248: productv1.ProductService.QueryCategoriesByKinds:input_type -> productv1.QueryCategoriesByKindsRequest - 42, // 249: productv1.ProductService.QueryComplexFilterType:input_type -> productv1.QueryComplexFilterTypeRequest - 52, // 250: productv1.ProductService.QueryFilterCategories:input_type -> productv1.QueryFilterCategoriesRequest - 34, // 251: productv1.ProductService.QueryNestedType:input_type -> productv1.QueryNestedTypeRequest - 62, // 252: productv1.ProductService.QueryNullableFieldsType:input_type -> productv1.QueryNullableFieldsTypeRequest - 64, // 253: productv1.ProductService.QueryNullableFieldsTypeById:input_type -> productv1.QueryNullableFieldsTypeByIdRequest - 66, // 254: productv1.ProductService.QueryNullableFieldsTypeWithFilter:input_type -> productv1.QueryNullableFieldsTypeWithFilterRequest - 54, // 255: productv1.ProductService.QueryRandomPet:input_type -> productv1.QueryRandomPetRequest - 60, // 256: productv1.ProductService.QueryRandomSearchResult:input_type -> productv1.QueryRandomSearchResultRequest - 36, // 257: productv1.ProductService.QueryRecursiveType:input_type -> productv1.QueryRecursiveTypeRequest - 58, // 258: productv1.ProductService.QuerySearch:input_type -> productv1.QuerySearchRequest - 38, // 259: productv1.ProductService.QueryTypeFilterWithArguments:input_type -> productv1.QueryTypeFilterWithArgumentsRequest - 40, // 260: productv1.ProductService.QueryTypeWithMultipleFilterFields:input_type -> productv1.QueryTypeWithMultipleFilterFieldsRequest - 32, // 261: productv1.ProductService.QueryUser:input_type -> productv1.QueryUserRequest - 30, // 262: productv1.ProductService.QueryUsers:input_type -> productv1.QueryUsersRequest - 126, // 263: productv1.ProductService.ResolveCategoryCategoryMetrics:input_type -> productv1.ResolveCategoryCategoryMetricsRequest - 121, // 264: productv1.ProductService.ResolveCategoryPopularityScore:input_type -> productv1.ResolveCategoryPopularityScoreRequest - 116, // 265: productv1.ProductService.ResolveCategoryProductCount:input_type -> productv1.ResolveCategoryProductCountRequest - 131, // 266: productv1.ProductService.ResolveSubcategoryItemCount:input_type -> productv1.ResolveSubcategoryItemCountRequest - 23, // 267: productv1.ProductService.LookupProductById:output_type -> productv1.LookupProductByIdResponse - 26, // 268: productv1.ProductService.LookupStorageById:output_type -> productv1.LookupStorageByIdResponse - 29, // 269: productv1.ProductService.LookupWarehouseById:output_type -> productv1.LookupWarehouseByIdResponse - 107, // 270: productv1.ProductService.MutationBulkCreateAuthors:output_type -> productv1.MutationBulkCreateAuthorsResponse - 111, // 271: productv1.ProductService.MutationBulkCreateBlogPosts:output_type -> productv1.MutationBulkCreateBlogPostsResponse - 109, // 272: productv1.ProductService.MutationBulkUpdateAuthors:output_type -> productv1.MutationBulkUpdateAuthorsResponse - 113, // 273: productv1.ProductService.MutationBulkUpdateBlogPosts:output_type -> productv1.MutationBulkUpdateBlogPostsResponse - 103, // 274: productv1.ProductService.MutationCreateAuthor:output_type -> productv1.MutationCreateAuthorResponse - 99, // 275: productv1.ProductService.MutationCreateBlogPost:output_type -> productv1.MutationCreateBlogPostResponse - 95, // 276: productv1.ProductService.MutationCreateNullableFieldsType:output_type -> productv1.MutationCreateNullableFieldsTypeResponse - 91, // 277: productv1.ProductService.MutationCreateUser:output_type -> productv1.MutationCreateUserResponse - 93, // 278: productv1.ProductService.MutationPerformAction:output_type -> productv1.MutationPerformActionResponse - 105, // 279: productv1.ProductService.MutationUpdateAuthor:output_type -> productv1.MutationUpdateAuthorResponse - 101, // 280: productv1.ProductService.MutationUpdateBlogPost:output_type -> productv1.MutationUpdateBlogPostResponse - 97, // 281: productv1.ProductService.MutationUpdateNullableFieldsType:output_type -> productv1.MutationUpdateNullableFieldsTypeResponse - 85, // 282: productv1.ProductService.QueryAllAuthors:output_type -> productv1.QueryAllAuthorsResponse - 77, // 283: productv1.ProductService.QueryAllBlogPosts:output_type -> productv1.QueryAllBlogPostsResponse - 69, // 284: productv1.ProductService.QueryAllNullableFieldsTypes:output_type -> productv1.QueryAllNullableFieldsTypesResponse - 57, // 285: productv1.ProductService.QueryAllPets:output_type -> productv1.QueryAllPetsResponse - 79, // 286: productv1.ProductService.QueryAuthor:output_type -> productv1.QueryAuthorResponse - 81, // 287: productv1.ProductService.QueryAuthorById:output_type -> productv1.QueryAuthorByIdResponse - 83, // 288: productv1.ProductService.QueryAuthorsWithFilter:output_type -> productv1.QueryAuthorsWithFilterResponse - 71, // 289: productv1.ProductService.QueryBlogPost:output_type -> productv1.QueryBlogPostResponse - 73, // 290: productv1.ProductService.QueryBlogPostById:output_type -> productv1.QueryBlogPostByIdResponse - 75, // 291: productv1.ProductService.QueryBlogPostsWithFilter:output_type -> productv1.QueryBlogPostsWithFilterResponse - 87, // 292: productv1.ProductService.QueryBulkSearchAuthors:output_type -> productv1.QueryBulkSearchAuthorsResponse - 89, // 293: productv1.ProductService.QueryBulkSearchBlogPosts:output_type -> productv1.QueryBulkSearchBlogPostsResponse - 45, // 294: productv1.ProductService.QueryCalculateTotals:output_type -> productv1.QueryCalculateTotalsResponse - 47, // 295: productv1.ProductService.QueryCategories:output_type -> productv1.QueryCategoriesResponse - 49, // 296: productv1.ProductService.QueryCategoriesByKind:output_type -> productv1.QueryCategoriesByKindResponse - 51, // 297: productv1.ProductService.QueryCategoriesByKinds:output_type -> productv1.QueryCategoriesByKindsResponse - 43, // 298: productv1.ProductService.QueryComplexFilterType:output_type -> productv1.QueryComplexFilterTypeResponse - 53, // 299: productv1.ProductService.QueryFilterCategories:output_type -> productv1.QueryFilterCategoriesResponse - 35, // 300: productv1.ProductService.QueryNestedType:output_type -> productv1.QueryNestedTypeResponse - 63, // 301: productv1.ProductService.QueryNullableFieldsType:output_type -> productv1.QueryNullableFieldsTypeResponse - 65, // 302: productv1.ProductService.QueryNullableFieldsTypeById:output_type -> productv1.QueryNullableFieldsTypeByIdResponse - 67, // 303: productv1.ProductService.QueryNullableFieldsTypeWithFilter:output_type -> productv1.QueryNullableFieldsTypeWithFilterResponse - 55, // 304: productv1.ProductService.QueryRandomPet:output_type -> productv1.QueryRandomPetResponse - 61, // 305: productv1.ProductService.QueryRandomSearchResult:output_type -> productv1.QueryRandomSearchResultResponse - 37, // 306: productv1.ProductService.QueryRecursiveType:output_type -> productv1.QueryRecursiveTypeResponse - 59, // 307: productv1.ProductService.QuerySearch:output_type -> productv1.QuerySearchResponse - 39, // 308: productv1.ProductService.QueryTypeFilterWithArguments:output_type -> productv1.QueryTypeFilterWithArgumentsResponse - 41, // 309: productv1.ProductService.QueryTypeWithMultipleFilterFields:output_type -> productv1.QueryTypeWithMultipleFilterFieldsResponse - 33, // 310: productv1.ProductService.QueryUser:output_type -> productv1.QueryUserResponse - 31, // 311: productv1.ProductService.QueryUsers:output_type -> productv1.QueryUsersResponse - 128, // 312: productv1.ProductService.ResolveCategoryCategoryMetrics:output_type -> productv1.ResolveCategoryCategoryMetricsResponse - 123, // 313: productv1.ProductService.ResolveCategoryPopularityScore:output_type -> productv1.ResolveCategoryPopularityScoreResponse - 118, // 314: productv1.ProductService.ResolveCategoryProductCount:output_type -> productv1.ResolveCategoryProductCountResponse - 133, // 315: productv1.ProductService.ResolveSubcategoryItemCount:output_type -> productv1.ResolveSubcategoryItemCountResponse - 267, // [267:316] is the sub-list for method output_type - 218, // [218:267] is the sub-list for method input_type - 218, // [218:218] is the sub-list for extension type_name - 218, // [218:218] is the sub-list for extension extendee - 0, // [0:218] is the sub-list for field type_name + 157, // 41: productv1.QueryCategoriesByKindsResponse.categories_by_kinds:type_name -> productv1.Category + 158, // 42: productv1.QueryFilterCategoriesRequest.filter:type_name -> productv1.CategoryFilter + 157, // 43: productv1.QueryFilterCategoriesResponse.filter_categories:type_name -> productv1.Category + 159, // 44: productv1.QueryRandomPetResponse.random_pet:type_name -> productv1.Animal + 159, // 45: productv1.QueryAllPetsResponse.all_pets:type_name -> productv1.Animal + 160, // 46: productv1.QuerySearchRequest.input:type_name -> productv1.SearchInput + 161, // 47: productv1.QuerySearchResponse.search:type_name -> productv1.SearchResult + 161, // 48: productv1.QueryRandomSearchResultResponse.random_search_result:type_name -> productv1.SearchResult + 162, // 49: productv1.QueryNullableFieldsTypeResponse.nullable_fields_type:type_name -> productv1.NullableFieldsType + 162, // 50: productv1.QueryNullableFieldsTypeByIdResponse.nullable_fields_type_by_id:type_name -> productv1.NullableFieldsType + 163, // 51: productv1.QueryNullableFieldsTypeWithFilterRequest.filter:type_name -> productv1.NullableFieldsFilter + 162, // 52: productv1.QueryNullableFieldsTypeWithFilterResponse.nullable_fields_type_with_filter:type_name -> productv1.NullableFieldsType + 162, // 53: productv1.QueryAllNullableFieldsTypesResponse.all_nullable_fields_types:type_name -> productv1.NullableFieldsType + 164, // 54: productv1.QueryBlogPostResponse.blog_post:type_name -> productv1.BlogPost + 164, // 55: productv1.QueryBlogPostByIdResponse.blog_post_by_id:type_name -> productv1.BlogPost + 165, // 56: productv1.QueryBlogPostsWithFilterRequest.filter:type_name -> productv1.BlogPostFilter + 164, // 57: productv1.QueryBlogPostsWithFilterResponse.blog_posts_with_filter:type_name -> productv1.BlogPost + 164, // 58: productv1.QueryAllBlogPostsResponse.all_blog_posts:type_name -> productv1.BlogPost + 166, // 59: productv1.QueryAuthorResponse.author:type_name -> productv1.Author + 166, // 60: productv1.QueryAuthorByIdResponse.author_by_id:type_name -> productv1.Author + 167, // 61: productv1.QueryAuthorsWithFilterRequest.filter:type_name -> productv1.AuthorFilter + 166, // 62: productv1.QueryAuthorsWithFilterResponse.authors_with_filter:type_name -> productv1.Author + 166, // 63: productv1.QueryAllAuthorsResponse.all_authors:type_name -> productv1.Author + 2, // 64: productv1.QueryBulkSearchAuthorsRequest.filters:type_name -> productv1.ListOfAuthorFilter + 166, // 65: productv1.QueryBulkSearchAuthorsResponse.bulk_search_authors:type_name -> productv1.Author + 5, // 66: productv1.QueryBulkSearchBlogPostsRequest.filters:type_name -> productv1.ListOfBlogPostFilter + 164, // 67: productv1.QueryBulkSearchBlogPostsResponse.bulk_search_blog_posts:type_name -> productv1.BlogPost + 168, // 68: productv1.MutationCreateUserRequest.input:type_name -> productv1.UserInput + 148, // 69: productv1.MutationCreateUserResponse.create_user:type_name -> productv1.User + 169, // 70: productv1.MutationPerformActionRequest.input:type_name -> productv1.ActionInput + 170, // 71: productv1.MutationPerformActionResponse.perform_action:type_name -> productv1.ActionResult + 171, // 72: productv1.MutationCreateNullableFieldsTypeRequest.input:type_name -> productv1.NullableFieldsInput + 162, // 73: productv1.MutationCreateNullableFieldsTypeResponse.create_nullable_fields_type:type_name -> productv1.NullableFieldsType + 171, // 74: productv1.MutationUpdateNullableFieldsTypeRequest.input:type_name -> productv1.NullableFieldsInput + 162, // 75: productv1.MutationUpdateNullableFieldsTypeResponse.update_nullable_fields_type:type_name -> productv1.NullableFieldsType + 172, // 76: productv1.MutationCreateBlogPostRequest.input:type_name -> productv1.BlogPostInput + 164, // 77: productv1.MutationCreateBlogPostResponse.create_blog_post:type_name -> productv1.BlogPost + 172, // 78: productv1.MutationUpdateBlogPostRequest.input:type_name -> productv1.BlogPostInput + 164, // 79: productv1.MutationUpdateBlogPostResponse.update_blog_post:type_name -> productv1.BlogPost + 173, // 80: productv1.MutationCreateAuthorRequest.input:type_name -> productv1.AuthorInput + 166, // 81: productv1.MutationCreateAuthorResponse.create_author:type_name -> productv1.Author + 173, // 82: productv1.MutationUpdateAuthorRequest.input:type_name -> productv1.AuthorInput + 166, // 83: productv1.MutationUpdateAuthorResponse.update_author:type_name -> productv1.Author + 3, // 84: productv1.MutationBulkCreateAuthorsRequest.authors:type_name -> productv1.ListOfAuthorInput + 166, // 85: productv1.MutationBulkCreateAuthorsResponse.bulk_create_authors:type_name -> productv1.Author + 3, // 86: productv1.MutationBulkUpdateAuthorsRequest.authors:type_name -> productv1.ListOfAuthorInput + 166, // 87: productv1.MutationBulkUpdateAuthorsResponse.bulk_update_authors:type_name -> productv1.Author + 6, // 88: productv1.MutationBulkCreateBlogPostsRequest.blog_posts:type_name -> productv1.ListOfBlogPostInput + 164, // 89: productv1.MutationBulkCreateBlogPostsResponse.bulk_create_blog_posts:type_name -> productv1.BlogPost + 6, // 90: productv1.MutationBulkUpdateBlogPostsRequest.blog_posts:type_name -> productv1.ListOfBlogPostInput + 164, // 91: productv1.MutationBulkUpdateBlogPostsResponse.bulk_update_blog_posts:type_name -> productv1.BlogPost + 189, // 92: productv1.ResolveProductShippingEstimateArgs.input:type_name -> productv1.ShippingEstimateInput + 116, // 93: productv1.ResolveProductShippingEstimateRequest.context:type_name -> productv1.ResolveProductShippingEstimateContext + 115, // 94: productv1.ResolveProductShippingEstimateRequest.field_args:type_name -> productv1.ResolveProductShippingEstimateArgs + 118, // 95: productv1.ResolveProductShippingEstimateResponse.result:type_name -> productv1.ResolveProductShippingEstimateResult + 121, // 96: productv1.ResolveProductRecommendedCategoryRequest.context:type_name -> productv1.ResolveProductRecommendedCategoryContext + 120, // 97: productv1.ResolveProductRecommendedCategoryRequest.field_args:type_name -> productv1.ResolveProductRecommendedCategoryArgs + 157, // 98: productv1.ResolveProductRecommendedCategoryResult.recommended_category:type_name -> productv1.Category + 123, // 99: productv1.ResolveProductRecommendedCategoryResponse.result:type_name -> productv1.ResolveProductRecommendedCategoryResult + 187, // 100: productv1.ResolveCategoryProductCountArgs.filters:type_name -> productv1.ProductCountFilter + 126, // 101: productv1.ResolveCategoryProductCountRequest.context:type_name -> productv1.ResolveCategoryProductCountContext + 125, // 102: productv1.ResolveCategoryProductCountRequest.field_args:type_name -> productv1.ResolveCategoryProductCountArgs + 128, // 103: productv1.ResolveCategoryProductCountResponse.result:type_name -> productv1.ResolveCategoryProductCountResult + 210, // 104: productv1.ResolveCategoryPopularityScoreArgs.threshold:type_name -> google.protobuf.Int32Value + 131, // 105: productv1.ResolveCategoryPopularityScoreRequest.context:type_name -> productv1.ResolveCategoryPopularityScoreContext + 130, // 106: productv1.ResolveCategoryPopularityScoreRequest.field_args:type_name -> productv1.ResolveCategoryPopularityScoreArgs + 210, // 107: productv1.ResolveCategoryPopularityScoreResult.popularity_score:type_name -> google.protobuf.Int32Value + 133, // 108: productv1.ResolveCategoryPopularityScoreResponse.result:type_name -> productv1.ResolveCategoryPopularityScoreResult + 136, // 109: productv1.ResolveCategoryCategoryMetricsRequest.context:type_name -> productv1.ResolveCategoryCategoryMetricsContext + 135, // 110: productv1.ResolveCategoryCategoryMetricsRequest.field_args:type_name -> productv1.ResolveCategoryCategoryMetricsArgs + 181, // 111: productv1.ResolveCategoryCategoryMetricsResult.category_metrics:type_name -> productv1.CategoryMetrics + 138, // 112: productv1.ResolveCategoryCategoryMetricsResponse.result:type_name -> productv1.ResolveCategoryCategoryMetricsResult + 188, // 113: productv1.ResolveSubcategoryItemCountArgs.filters:type_name -> productv1.SubcategoryItemFilter + 141, // 114: productv1.ResolveSubcategoryItemCountRequest.context:type_name -> productv1.ResolveSubcategoryItemCountContext + 140, // 115: productv1.ResolveSubcategoryItemCountRequest.field_args:type_name -> productv1.ResolveSubcategoryItemCountArgs + 143, // 116: productv1.ResolveSubcategoryItemCountResponse.result:type_name -> productv1.ResolveSubcategoryItemCountResult + 174, // 117: productv1.NestedTypeA.b:type_name -> productv1.NestedTypeB + 150, // 118: productv1.RecursiveType.recursive_type:type_name -> productv1.RecursiveType + 176, // 119: productv1.ComplexFilterTypeInput.filter:type_name -> productv1.FilterType + 178, // 120: productv1.OrderInput.lines:type_name -> productv1.OrderLineInput + 16, // 121: productv1.Order.order_lines:type_name -> productv1.ListOfOrderLine + 0, // 122: productv1.Category.kind:type_name -> productv1.CategoryKind + 19, // 123: productv1.Category.subcategories:type_name -> productv1.ListOfSubcategory + 0, // 124: productv1.CategoryFilter.category:type_name -> productv1.CategoryKind + 177, // 125: productv1.CategoryFilter.pagination:type_name -> productv1.Pagination + 182, // 126: productv1.Animal.cat:type_name -> productv1.Cat + 183, // 127: productv1.Animal.dog:type_name -> productv1.Dog + 210, // 128: productv1.SearchInput.limit:type_name -> google.protobuf.Int32Value + 145, // 129: productv1.SearchResult.product:type_name -> productv1.Product + 148, // 130: productv1.SearchResult.user:type_name -> productv1.User + 157, // 131: productv1.SearchResult.category:type_name -> productv1.Category + 211, // 132: productv1.NullableFieldsType.optional_string:type_name -> google.protobuf.StringValue + 210, // 133: productv1.NullableFieldsType.optional_int:type_name -> google.protobuf.Int32Value + 212, // 134: productv1.NullableFieldsType.optional_float:type_name -> google.protobuf.DoubleValue + 213, // 135: productv1.NullableFieldsType.optional_boolean:type_name -> google.protobuf.BoolValue + 211, // 136: productv1.NullableFieldsFilter.name:type_name -> google.protobuf.StringValue + 211, // 137: productv1.NullableFieldsFilter.optional_string:type_name -> google.protobuf.StringValue + 213, // 138: productv1.NullableFieldsFilter.include_nulls:type_name -> google.protobuf.BoolValue + 18, // 139: productv1.BlogPost.optional_tags:type_name -> productv1.ListOfString + 18, // 140: productv1.BlogPost.keywords:type_name -> productv1.ListOfString + 10, // 141: productv1.BlogPost.ratings:type_name -> productv1.ListOfFloat + 7, // 142: productv1.BlogPost.is_published:type_name -> productv1.ListOfBoolean + 13, // 143: productv1.BlogPost.tag_groups:type_name -> productv1.ListOfListOfString + 13, // 144: productv1.BlogPost.related_topics:type_name -> productv1.ListOfListOfString + 13, // 145: productv1.BlogPost.comment_threads:type_name -> productv1.ListOfListOfString + 13, // 146: productv1.BlogPost.suggestions:type_name -> productv1.ListOfListOfString + 157, // 147: productv1.BlogPost.related_categories:type_name -> productv1.Category + 148, // 148: productv1.BlogPost.contributors:type_name -> productv1.User + 17, // 149: productv1.BlogPost.mentioned_products:type_name -> productv1.ListOfProduct + 20, // 150: productv1.BlogPost.mentioned_users:type_name -> productv1.ListOfUser + 11, // 151: productv1.BlogPost.category_groups:type_name -> productv1.ListOfListOfCategory + 14, // 152: productv1.BlogPost.contributor_teams:type_name -> productv1.ListOfListOfUser + 211, // 153: productv1.BlogPostFilter.title:type_name -> google.protobuf.StringValue + 213, // 154: productv1.BlogPostFilter.has_categories:type_name -> google.protobuf.BoolValue + 210, // 155: productv1.BlogPostFilter.min_tags:type_name -> google.protobuf.Int32Value + 211, // 156: productv1.Author.email:type_name -> google.protobuf.StringValue + 18, // 157: productv1.Author.social_links:type_name -> productv1.ListOfString + 13, // 158: productv1.Author.teams_by_project:type_name -> productv1.ListOfListOfString + 13, // 159: productv1.Author.collaborations:type_name -> productv1.ListOfListOfString + 4, // 160: productv1.Author.written_posts:type_name -> productv1.ListOfBlogPost + 157, // 161: productv1.Author.favorite_categories:type_name -> productv1.Category + 20, // 162: productv1.Author.related_authors:type_name -> productv1.ListOfUser + 17, // 163: productv1.Author.product_reviews:type_name -> productv1.ListOfProduct + 14, // 164: productv1.Author.author_groups:type_name -> productv1.ListOfListOfUser + 11, // 165: productv1.Author.category_preferences:type_name -> productv1.ListOfListOfCategory + 14, // 166: productv1.Author.project_teams:type_name -> productv1.ListOfListOfUser + 211, // 167: productv1.AuthorFilter.name:type_name -> google.protobuf.StringValue + 213, // 168: productv1.AuthorFilter.has_teams:type_name -> google.protobuf.BoolValue + 210, // 169: productv1.AuthorFilter.skill_count:type_name -> google.protobuf.Int32Value + 184, // 170: productv1.ActionResult.action_success:type_name -> productv1.ActionSuccess + 185, // 171: productv1.ActionResult.action_error:type_name -> productv1.ActionError + 211, // 172: productv1.NullableFieldsInput.optional_string:type_name -> google.protobuf.StringValue + 210, // 173: productv1.NullableFieldsInput.optional_int:type_name -> google.protobuf.Int32Value + 212, // 174: productv1.NullableFieldsInput.optional_float:type_name -> google.protobuf.DoubleValue + 213, // 175: productv1.NullableFieldsInput.optional_boolean:type_name -> google.protobuf.BoolValue + 18, // 176: productv1.BlogPostInput.optional_tags:type_name -> productv1.ListOfString + 18, // 177: productv1.BlogPostInput.keywords:type_name -> productv1.ListOfString + 10, // 178: productv1.BlogPostInput.ratings:type_name -> productv1.ListOfFloat + 7, // 179: productv1.BlogPostInput.is_published:type_name -> productv1.ListOfBoolean + 13, // 180: productv1.BlogPostInput.tag_groups:type_name -> productv1.ListOfListOfString + 13, // 181: productv1.BlogPostInput.related_topics:type_name -> productv1.ListOfListOfString + 13, // 182: productv1.BlogPostInput.comment_threads:type_name -> productv1.ListOfListOfString + 13, // 183: productv1.BlogPostInput.suggestions:type_name -> productv1.ListOfListOfString + 9, // 184: productv1.BlogPostInput.related_categories:type_name -> productv1.ListOfCategoryInput + 21, // 185: productv1.BlogPostInput.contributors:type_name -> productv1.ListOfUserInput + 12, // 186: productv1.BlogPostInput.category_groups:type_name -> productv1.ListOfListOfCategoryInput + 211, // 187: productv1.AuthorInput.email:type_name -> google.protobuf.StringValue + 18, // 188: productv1.AuthorInput.social_links:type_name -> productv1.ListOfString + 13, // 189: productv1.AuthorInput.teams_by_project:type_name -> productv1.ListOfListOfString + 13, // 190: productv1.AuthorInput.collaborations:type_name -> productv1.ListOfListOfString + 186, // 191: productv1.AuthorInput.favorite_categories:type_name -> productv1.CategoryInput + 15, // 192: productv1.AuthorInput.author_groups:type_name -> productv1.ListOfListOfUserInput + 15, // 193: productv1.AuthorInput.project_teams:type_name -> productv1.ListOfListOfUserInput + 175, // 194: productv1.NestedTypeB.c:type_name -> productv1.NestedTypeC + 177, // 195: productv1.FilterType.pagination:type_name -> productv1.Pagination + 18, // 196: productv1.OrderLineInput.modifiers:type_name -> productv1.ListOfString + 18, // 197: productv1.OrderLine.modifiers:type_name -> productv1.ListOfString + 211, // 198: productv1.Subcategory.description:type_name -> google.protobuf.StringValue + 0, // 199: productv1.CategoryInput.kind:type_name -> productv1.CategoryKind + 212, // 200: productv1.ProductCountFilter.min_price:type_name -> google.protobuf.DoubleValue + 212, // 201: productv1.ProductCountFilter.max_price:type_name -> google.protobuf.DoubleValue + 213, // 202: productv1.ProductCountFilter.in_stock:type_name -> google.protobuf.BoolValue + 211, // 203: productv1.ProductCountFilter.search_term:type_name -> google.protobuf.StringValue + 212, // 204: productv1.SubcategoryItemFilter.min_price:type_name -> google.protobuf.DoubleValue + 212, // 205: productv1.SubcategoryItemFilter.max_price:type_name -> google.protobuf.DoubleValue + 213, // 206: productv1.SubcategoryItemFilter.in_stock:type_name -> google.protobuf.BoolValue + 213, // 207: productv1.SubcategoryItemFilter.is_active:type_name -> google.protobuf.BoolValue + 211, // 208: productv1.SubcategoryItemFilter.search_term:type_name -> google.protobuf.StringValue + 1, // 209: productv1.ShippingEstimateInput.destination:type_name -> productv1.ShippingDestination + 213, // 210: productv1.ShippingEstimateInput.expedited:type_name -> google.protobuf.BoolValue + 167, // 211: productv1.ListOfAuthorFilter.List.items:type_name -> productv1.AuthorFilter + 173, // 212: productv1.ListOfAuthorInput.List.items:type_name -> productv1.AuthorInput + 164, // 213: productv1.ListOfBlogPost.List.items:type_name -> productv1.BlogPost + 165, // 214: productv1.ListOfBlogPostFilter.List.items:type_name -> productv1.BlogPostFilter + 172, // 215: productv1.ListOfBlogPostInput.List.items:type_name -> productv1.BlogPostInput + 157, // 216: productv1.ListOfCategory.List.items:type_name -> productv1.Category + 186, // 217: productv1.ListOfCategoryInput.List.items:type_name -> productv1.CategoryInput + 8, // 218: productv1.ListOfListOfCategory.List.items:type_name -> productv1.ListOfCategory + 9, // 219: productv1.ListOfListOfCategoryInput.List.items:type_name -> productv1.ListOfCategoryInput + 18, // 220: productv1.ListOfListOfString.List.items:type_name -> productv1.ListOfString + 20, // 221: productv1.ListOfListOfUser.List.items:type_name -> productv1.ListOfUser + 21, // 222: productv1.ListOfListOfUserInput.List.items:type_name -> productv1.ListOfUserInput + 179, // 223: productv1.ListOfOrderLine.List.items:type_name -> productv1.OrderLine + 145, // 224: productv1.ListOfProduct.List.items:type_name -> productv1.Product + 180, // 225: productv1.ListOfSubcategory.List.items:type_name -> productv1.Subcategory + 148, // 226: productv1.ListOfUser.List.items:type_name -> productv1.User + 168, // 227: productv1.ListOfUserInput.List.items:type_name -> productv1.UserInput + 23, // 228: productv1.ProductService.LookupProductById:input_type -> productv1.LookupProductByIdRequest + 26, // 229: productv1.ProductService.LookupStorageById:input_type -> productv1.LookupStorageByIdRequest + 29, // 230: productv1.ProductService.LookupWarehouseById:input_type -> productv1.LookupWarehouseByIdRequest + 107, // 231: productv1.ProductService.MutationBulkCreateAuthors:input_type -> productv1.MutationBulkCreateAuthorsRequest + 111, // 232: productv1.ProductService.MutationBulkCreateBlogPosts:input_type -> productv1.MutationBulkCreateBlogPostsRequest + 109, // 233: productv1.ProductService.MutationBulkUpdateAuthors:input_type -> productv1.MutationBulkUpdateAuthorsRequest + 113, // 234: productv1.ProductService.MutationBulkUpdateBlogPosts:input_type -> productv1.MutationBulkUpdateBlogPostsRequest + 103, // 235: productv1.ProductService.MutationCreateAuthor:input_type -> productv1.MutationCreateAuthorRequest + 99, // 236: productv1.ProductService.MutationCreateBlogPost:input_type -> productv1.MutationCreateBlogPostRequest + 95, // 237: productv1.ProductService.MutationCreateNullableFieldsType:input_type -> productv1.MutationCreateNullableFieldsTypeRequest + 91, // 238: productv1.ProductService.MutationCreateUser:input_type -> productv1.MutationCreateUserRequest + 93, // 239: productv1.ProductService.MutationPerformAction:input_type -> productv1.MutationPerformActionRequest + 105, // 240: productv1.ProductService.MutationUpdateAuthor:input_type -> productv1.MutationUpdateAuthorRequest + 101, // 241: productv1.ProductService.MutationUpdateBlogPost:input_type -> productv1.MutationUpdateBlogPostRequest + 97, // 242: productv1.ProductService.MutationUpdateNullableFieldsType:input_type -> productv1.MutationUpdateNullableFieldsTypeRequest + 85, // 243: productv1.ProductService.QueryAllAuthors:input_type -> productv1.QueryAllAuthorsRequest + 77, // 244: productv1.ProductService.QueryAllBlogPosts:input_type -> productv1.QueryAllBlogPostsRequest + 69, // 245: productv1.ProductService.QueryAllNullableFieldsTypes:input_type -> productv1.QueryAllNullableFieldsTypesRequest + 57, // 246: productv1.ProductService.QueryAllPets:input_type -> productv1.QueryAllPetsRequest + 79, // 247: productv1.ProductService.QueryAuthor:input_type -> productv1.QueryAuthorRequest + 81, // 248: productv1.ProductService.QueryAuthorById:input_type -> productv1.QueryAuthorByIdRequest + 83, // 249: productv1.ProductService.QueryAuthorsWithFilter:input_type -> productv1.QueryAuthorsWithFilterRequest + 71, // 250: productv1.ProductService.QueryBlogPost:input_type -> productv1.QueryBlogPostRequest + 73, // 251: productv1.ProductService.QueryBlogPostById:input_type -> productv1.QueryBlogPostByIdRequest + 75, // 252: productv1.ProductService.QueryBlogPostsWithFilter:input_type -> productv1.QueryBlogPostsWithFilterRequest + 87, // 253: productv1.ProductService.QueryBulkSearchAuthors:input_type -> productv1.QueryBulkSearchAuthorsRequest + 89, // 254: productv1.ProductService.QueryBulkSearchBlogPosts:input_type -> productv1.QueryBulkSearchBlogPostsRequest + 45, // 255: productv1.ProductService.QueryCalculateTotals:input_type -> productv1.QueryCalculateTotalsRequest + 47, // 256: productv1.ProductService.QueryCategories:input_type -> productv1.QueryCategoriesRequest + 49, // 257: productv1.ProductService.QueryCategoriesByKind:input_type -> productv1.QueryCategoriesByKindRequest + 51, // 258: productv1.ProductService.QueryCategoriesByKinds:input_type -> productv1.QueryCategoriesByKindsRequest + 43, // 259: productv1.ProductService.QueryComplexFilterType:input_type -> productv1.QueryComplexFilterTypeRequest + 53, // 260: productv1.ProductService.QueryFilterCategories:input_type -> productv1.QueryFilterCategoriesRequest + 35, // 261: productv1.ProductService.QueryNestedType:input_type -> productv1.QueryNestedTypeRequest + 63, // 262: productv1.ProductService.QueryNullableFieldsType:input_type -> productv1.QueryNullableFieldsTypeRequest + 65, // 263: productv1.ProductService.QueryNullableFieldsTypeById:input_type -> productv1.QueryNullableFieldsTypeByIdRequest + 67, // 264: productv1.ProductService.QueryNullableFieldsTypeWithFilter:input_type -> productv1.QueryNullableFieldsTypeWithFilterRequest + 55, // 265: productv1.ProductService.QueryRandomPet:input_type -> productv1.QueryRandomPetRequest + 61, // 266: productv1.ProductService.QueryRandomSearchResult:input_type -> productv1.QueryRandomSearchResultRequest + 37, // 267: productv1.ProductService.QueryRecursiveType:input_type -> productv1.QueryRecursiveTypeRequest + 59, // 268: productv1.ProductService.QuerySearch:input_type -> productv1.QuerySearchRequest + 39, // 269: productv1.ProductService.QueryTypeFilterWithArguments:input_type -> productv1.QueryTypeFilterWithArgumentsRequest + 41, // 270: productv1.ProductService.QueryTypeWithMultipleFilterFields:input_type -> productv1.QueryTypeWithMultipleFilterFieldsRequest + 33, // 271: productv1.ProductService.QueryUser:input_type -> productv1.QueryUserRequest + 31, // 272: productv1.ProductService.QueryUsers:input_type -> productv1.QueryUsersRequest + 137, // 273: productv1.ProductService.ResolveCategoryCategoryMetrics:input_type -> productv1.ResolveCategoryCategoryMetricsRequest + 132, // 274: productv1.ProductService.ResolveCategoryPopularityScore:input_type -> productv1.ResolveCategoryPopularityScoreRequest + 127, // 275: productv1.ProductService.ResolveCategoryProductCount:input_type -> productv1.ResolveCategoryProductCountRequest + 122, // 276: productv1.ProductService.ResolveProductRecommendedCategory:input_type -> productv1.ResolveProductRecommendedCategoryRequest + 117, // 277: productv1.ProductService.ResolveProductShippingEstimate:input_type -> productv1.ResolveProductShippingEstimateRequest + 142, // 278: productv1.ProductService.ResolveSubcategoryItemCount:input_type -> productv1.ResolveSubcategoryItemCountRequest + 24, // 279: productv1.ProductService.LookupProductById:output_type -> productv1.LookupProductByIdResponse + 27, // 280: productv1.ProductService.LookupStorageById:output_type -> productv1.LookupStorageByIdResponse + 30, // 281: productv1.ProductService.LookupWarehouseById:output_type -> productv1.LookupWarehouseByIdResponse + 108, // 282: productv1.ProductService.MutationBulkCreateAuthors:output_type -> productv1.MutationBulkCreateAuthorsResponse + 112, // 283: productv1.ProductService.MutationBulkCreateBlogPosts:output_type -> productv1.MutationBulkCreateBlogPostsResponse + 110, // 284: productv1.ProductService.MutationBulkUpdateAuthors:output_type -> productv1.MutationBulkUpdateAuthorsResponse + 114, // 285: productv1.ProductService.MutationBulkUpdateBlogPosts:output_type -> productv1.MutationBulkUpdateBlogPostsResponse + 104, // 286: productv1.ProductService.MutationCreateAuthor:output_type -> productv1.MutationCreateAuthorResponse + 100, // 287: productv1.ProductService.MutationCreateBlogPost:output_type -> productv1.MutationCreateBlogPostResponse + 96, // 288: productv1.ProductService.MutationCreateNullableFieldsType:output_type -> productv1.MutationCreateNullableFieldsTypeResponse + 92, // 289: productv1.ProductService.MutationCreateUser:output_type -> productv1.MutationCreateUserResponse + 94, // 290: productv1.ProductService.MutationPerformAction:output_type -> productv1.MutationPerformActionResponse + 106, // 291: productv1.ProductService.MutationUpdateAuthor:output_type -> productv1.MutationUpdateAuthorResponse + 102, // 292: productv1.ProductService.MutationUpdateBlogPost:output_type -> productv1.MutationUpdateBlogPostResponse + 98, // 293: productv1.ProductService.MutationUpdateNullableFieldsType:output_type -> productv1.MutationUpdateNullableFieldsTypeResponse + 86, // 294: productv1.ProductService.QueryAllAuthors:output_type -> productv1.QueryAllAuthorsResponse + 78, // 295: productv1.ProductService.QueryAllBlogPosts:output_type -> productv1.QueryAllBlogPostsResponse + 70, // 296: productv1.ProductService.QueryAllNullableFieldsTypes:output_type -> productv1.QueryAllNullableFieldsTypesResponse + 58, // 297: productv1.ProductService.QueryAllPets:output_type -> productv1.QueryAllPetsResponse + 80, // 298: productv1.ProductService.QueryAuthor:output_type -> productv1.QueryAuthorResponse + 82, // 299: productv1.ProductService.QueryAuthorById:output_type -> productv1.QueryAuthorByIdResponse + 84, // 300: productv1.ProductService.QueryAuthorsWithFilter:output_type -> productv1.QueryAuthorsWithFilterResponse + 72, // 301: productv1.ProductService.QueryBlogPost:output_type -> productv1.QueryBlogPostResponse + 74, // 302: productv1.ProductService.QueryBlogPostById:output_type -> productv1.QueryBlogPostByIdResponse + 76, // 303: productv1.ProductService.QueryBlogPostsWithFilter:output_type -> productv1.QueryBlogPostsWithFilterResponse + 88, // 304: productv1.ProductService.QueryBulkSearchAuthors:output_type -> productv1.QueryBulkSearchAuthorsResponse + 90, // 305: productv1.ProductService.QueryBulkSearchBlogPosts:output_type -> productv1.QueryBulkSearchBlogPostsResponse + 46, // 306: productv1.ProductService.QueryCalculateTotals:output_type -> productv1.QueryCalculateTotalsResponse + 48, // 307: productv1.ProductService.QueryCategories:output_type -> productv1.QueryCategoriesResponse + 50, // 308: productv1.ProductService.QueryCategoriesByKind:output_type -> productv1.QueryCategoriesByKindResponse + 52, // 309: productv1.ProductService.QueryCategoriesByKinds:output_type -> productv1.QueryCategoriesByKindsResponse + 44, // 310: productv1.ProductService.QueryComplexFilterType:output_type -> productv1.QueryComplexFilterTypeResponse + 54, // 311: productv1.ProductService.QueryFilterCategories:output_type -> productv1.QueryFilterCategoriesResponse + 36, // 312: productv1.ProductService.QueryNestedType:output_type -> productv1.QueryNestedTypeResponse + 64, // 313: productv1.ProductService.QueryNullableFieldsType:output_type -> productv1.QueryNullableFieldsTypeResponse + 66, // 314: productv1.ProductService.QueryNullableFieldsTypeById:output_type -> productv1.QueryNullableFieldsTypeByIdResponse + 68, // 315: productv1.ProductService.QueryNullableFieldsTypeWithFilter:output_type -> productv1.QueryNullableFieldsTypeWithFilterResponse + 56, // 316: productv1.ProductService.QueryRandomPet:output_type -> productv1.QueryRandomPetResponse + 62, // 317: productv1.ProductService.QueryRandomSearchResult:output_type -> productv1.QueryRandomSearchResultResponse + 38, // 318: productv1.ProductService.QueryRecursiveType:output_type -> productv1.QueryRecursiveTypeResponse + 60, // 319: productv1.ProductService.QuerySearch:output_type -> productv1.QuerySearchResponse + 40, // 320: productv1.ProductService.QueryTypeFilterWithArguments:output_type -> productv1.QueryTypeFilterWithArgumentsResponse + 42, // 321: productv1.ProductService.QueryTypeWithMultipleFilterFields:output_type -> productv1.QueryTypeWithMultipleFilterFieldsResponse + 34, // 322: productv1.ProductService.QueryUser:output_type -> productv1.QueryUserResponse + 32, // 323: productv1.ProductService.QueryUsers:output_type -> productv1.QueryUsersResponse + 139, // 324: productv1.ProductService.ResolveCategoryCategoryMetrics:output_type -> productv1.ResolveCategoryCategoryMetricsResponse + 134, // 325: productv1.ProductService.ResolveCategoryPopularityScore:output_type -> productv1.ResolveCategoryPopularityScoreResponse + 129, // 326: productv1.ProductService.ResolveCategoryProductCount:output_type -> productv1.ResolveCategoryProductCountResponse + 124, // 327: productv1.ProductService.ResolveProductRecommendedCategory:output_type -> productv1.ResolveProductRecommendedCategoryResponse + 119, // 328: productv1.ProductService.ResolveProductShippingEstimate:output_type -> productv1.ResolveProductShippingEstimateResponse + 144, // 329: productv1.ProductService.ResolveSubcategoryItemCount:output_type -> productv1.ResolveSubcategoryItemCountResponse + 279, // [279:330] is the sub-list for method output_type + 228, // [228:279] is the sub-list for method input_type + 228, // [228:228] is the sub-list for extension type_name + 228, // [228:228] is the sub-list for extension extendee + 0, // [0:228] is the sub-list for field type_name } func init() { file_product_proto_init() } @@ -11308,16 +11968,16 @@ func file_product_proto_init() { if File_product_proto != nil { return } - file_product_proto_msgTypes[147].OneofWrappers = []any{ + file_product_proto_msgTypes[157].OneofWrappers = []any{ (*Animal_Cat)(nil), (*Animal_Dog)(nil), } - file_product_proto_msgTypes[149].OneofWrappers = []any{ + file_product_proto_msgTypes[159].OneofWrappers = []any{ (*SearchResult_Product)(nil), (*SearchResult_User)(nil), (*SearchResult_Category)(nil), } - file_product_proto_msgTypes[158].OneofWrappers = []any{ + file_product_proto_msgTypes[168].OneofWrappers = []any{ (*ActionResult_ActionSuccess)(nil), (*ActionResult_ActionError)(nil), } @@ -11326,8 +11986,8 @@ func file_product_proto_init() { File: protoimpl.DescBuilder{ GoPackagePath: reflect.TypeOf(x{}).PkgPath(), RawDescriptor: unsafe.Slice(unsafe.StringData(file_product_proto_rawDesc), len(file_product_proto_rawDesc)), - NumEnums: 1, - NumMessages: 197, + NumEnums: 2, + NumMessages: 208, NumExtensions: 0, NumServices: 1, }, diff --git a/v2/pkg/grpctest/productv1/product_grpc.pb.go b/v2/pkg/grpctest/productv1/product_grpc.pb.go index 8e27362a2..9f856d858 100644 --- a/v2/pkg/grpctest/productv1/product_grpc.pb.go +++ b/v2/pkg/grpctest/productv1/product_grpc.pb.go @@ -67,6 +67,8 @@ const ( ProductService_ResolveCategoryCategoryMetrics_FullMethodName = "/productv1.ProductService/ResolveCategoryCategoryMetrics" ProductService_ResolveCategoryPopularityScore_FullMethodName = "/productv1.ProductService/ResolveCategoryPopularityScore" ProductService_ResolveCategoryProductCount_FullMethodName = "/productv1.ProductService/ResolveCategoryProductCount" + ProductService_ResolveProductRecommendedCategory_FullMethodName = "/productv1.ProductService/ResolveProductRecommendedCategory" + ProductService_ResolveProductShippingEstimate_FullMethodName = "/productv1.ProductService/ResolveProductShippingEstimate" ProductService_ResolveSubcategoryItemCount_FullMethodName = "/productv1.ProductService/ResolveSubcategoryItemCount" ) @@ -127,6 +129,8 @@ type ProductServiceClient interface { ResolveCategoryCategoryMetrics(ctx context.Context, in *ResolveCategoryCategoryMetricsRequest, opts ...grpc.CallOption) (*ResolveCategoryCategoryMetricsResponse, error) ResolveCategoryPopularityScore(ctx context.Context, in *ResolveCategoryPopularityScoreRequest, opts ...grpc.CallOption) (*ResolveCategoryPopularityScoreResponse, error) ResolveCategoryProductCount(ctx context.Context, in *ResolveCategoryProductCountRequest, opts ...grpc.CallOption) (*ResolveCategoryProductCountResponse, error) + ResolveProductRecommendedCategory(ctx context.Context, in *ResolveProductRecommendedCategoryRequest, opts ...grpc.CallOption) (*ResolveProductRecommendedCategoryResponse, error) + ResolveProductShippingEstimate(ctx context.Context, in *ResolveProductShippingEstimateRequest, opts ...grpc.CallOption) (*ResolveProductShippingEstimateResponse, error) ResolveSubcategoryItemCount(ctx context.Context, in *ResolveSubcategoryItemCountRequest, opts ...grpc.CallOption) (*ResolveSubcategoryItemCountResponse, error) } @@ -618,6 +622,26 @@ func (c *productServiceClient) ResolveCategoryProductCount(ctx context.Context, return out, nil } +func (c *productServiceClient) ResolveProductRecommendedCategory(ctx context.Context, in *ResolveProductRecommendedCategoryRequest, opts ...grpc.CallOption) (*ResolveProductRecommendedCategoryResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(ResolveProductRecommendedCategoryResponse) + err := c.cc.Invoke(ctx, ProductService_ResolveProductRecommendedCategory_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *productServiceClient) ResolveProductShippingEstimate(ctx context.Context, in *ResolveProductShippingEstimateRequest, opts ...grpc.CallOption) (*ResolveProductShippingEstimateResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(ResolveProductShippingEstimateResponse) + err := c.cc.Invoke(ctx, ProductService_ResolveProductShippingEstimate_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + func (c *productServiceClient) ResolveSubcategoryItemCount(ctx context.Context, in *ResolveSubcategoryItemCountRequest, opts ...grpc.CallOption) (*ResolveSubcategoryItemCountResponse, error) { cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) out := new(ResolveSubcategoryItemCountResponse) @@ -685,6 +709,8 @@ type ProductServiceServer interface { ResolveCategoryCategoryMetrics(context.Context, *ResolveCategoryCategoryMetricsRequest) (*ResolveCategoryCategoryMetricsResponse, error) ResolveCategoryPopularityScore(context.Context, *ResolveCategoryPopularityScoreRequest) (*ResolveCategoryPopularityScoreResponse, error) ResolveCategoryProductCount(context.Context, *ResolveCategoryProductCountRequest) (*ResolveCategoryProductCountResponse, error) + ResolveProductRecommendedCategory(context.Context, *ResolveProductRecommendedCategoryRequest) (*ResolveProductRecommendedCategoryResponse, error) + ResolveProductShippingEstimate(context.Context, *ResolveProductShippingEstimateRequest) (*ResolveProductShippingEstimateResponse, error) ResolveSubcategoryItemCount(context.Context, *ResolveSubcategoryItemCountRequest) (*ResolveSubcategoryItemCountResponse, error) mustEmbedUnimplementedProductServiceServer() } @@ -840,6 +866,12 @@ func (UnimplementedProductServiceServer) ResolveCategoryPopularityScore(context. func (UnimplementedProductServiceServer) ResolveCategoryProductCount(context.Context, *ResolveCategoryProductCountRequest) (*ResolveCategoryProductCountResponse, error) { return nil, status.Errorf(codes.Unimplemented, "method ResolveCategoryProductCount not implemented") } +func (UnimplementedProductServiceServer) ResolveProductRecommendedCategory(context.Context, *ResolveProductRecommendedCategoryRequest) (*ResolveProductRecommendedCategoryResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method ResolveProductRecommendedCategory not implemented") +} +func (UnimplementedProductServiceServer) ResolveProductShippingEstimate(context.Context, *ResolveProductShippingEstimateRequest) (*ResolveProductShippingEstimateResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method ResolveProductShippingEstimate not implemented") +} func (UnimplementedProductServiceServer) ResolveSubcategoryItemCount(context.Context, *ResolveSubcategoryItemCountRequest) (*ResolveSubcategoryItemCountResponse, error) { return nil, status.Errorf(codes.Unimplemented, "method ResolveSubcategoryItemCount not implemented") } @@ -1728,6 +1760,42 @@ func _ProductService_ResolveCategoryProductCount_Handler(srv interface{}, ctx co return interceptor(ctx, in, info, handler) } +func _ProductService_ResolveProductRecommendedCategory_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(ResolveProductRecommendedCategoryRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(ProductServiceServer).ResolveProductRecommendedCategory(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: ProductService_ResolveProductRecommendedCategory_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(ProductServiceServer).ResolveProductRecommendedCategory(ctx, req.(*ResolveProductRecommendedCategoryRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _ProductService_ResolveProductShippingEstimate_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(ResolveProductShippingEstimateRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(ProductServiceServer).ResolveProductShippingEstimate(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: ProductService_ResolveProductShippingEstimate_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(ProductServiceServer).ResolveProductShippingEstimate(ctx, req.(*ResolveProductShippingEstimateRequest)) + } + return interceptor(ctx, in, info, handler) +} + func _ProductService_ResolveSubcategoryItemCount_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { in := new(ResolveSubcategoryItemCountRequest) if err := dec(in); err != nil { @@ -1945,6 +2013,14 @@ var ProductService_ServiceDesc = grpc.ServiceDesc{ MethodName: "ResolveCategoryProductCount", Handler: _ProductService_ResolveCategoryProductCount_Handler, }, + { + MethodName: "ResolveProductRecommendedCategory", + Handler: _ProductService_ResolveProductRecommendedCategory_Handler, + }, + { + MethodName: "ResolveProductShippingEstimate", + Handler: _ProductService_ResolveProductShippingEstimate_Handler, + }, { MethodName: "ResolveSubcategoryItemCount", Handler: _ProductService_ResolveSubcategoryItemCount_Handler, diff --git a/v2/pkg/grpctest/productv1/testdata/service.pb.go b/v2/pkg/grpctest/productv1/testdata/service.pb.go new file mode 100644 index 000000000..2d182f99d --- /dev/null +++ b/v2/pkg/grpctest/productv1/testdata/service.pb.go @@ -0,0 +1,12002 @@ +// Code generated by protoc-gen-go. DO NOT EDIT. +// versions: +// protoc-gen-go v1.36.10 +// protoc v6.32.0 +// source: testdata/service.proto + +package productv1 + +import ( + protoreflect "google.golang.org/protobuf/reflect/protoreflect" + protoimpl "google.golang.org/protobuf/runtime/protoimpl" + wrapperspb "google.golang.org/protobuf/types/known/wrapperspb" + reflect "reflect" + sync "sync" + unsafe "unsafe" +) + +const ( + // Verify that this generated code is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion) + // Verify that runtime/protoimpl is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20) +) + +type CategoryKind int32 + +const ( + CategoryKind_CATEGORY_KIND_UNSPECIFIED CategoryKind = 0 + CategoryKind_CATEGORY_KIND_BOOK CategoryKind = 1 + CategoryKind_CATEGORY_KIND_ELECTRONICS CategoryKind = 2 + CategoryKind_CATEGORY_KIND_FURNITURE CategoryKind = 3 + CategoryKind_CATEGORY_KIND_OTHER CategoryKind = 4 +) + +// Enum value maps for CategoryKind. +var ( + CategoryKind_name = map[int32]string{ + 0: "CATEGORY_KIND_UNSPECIFIED", + 1: "CATEGORY_KIND_BOOK", + 2: "CATEGORY_KIND_ELECTRONICS", + 3: "CATEGORY_KIND_FURNITURE", + 4: "CATEGORY_KIND_OTHER", + } + CategoryKind_value = map[string]int32{ + "CATEGORY_KIND_UNSPECIFIED": 0, + "CATEGORY_KIND_BOOK": 1, + "CATEGORY_KIND_ELECTRONICS": 2, + "CATEGORY_KIND_FURNITURE": 3, + "CATEGORY_KIND_OTHER": 4, + } +) + +func (x CategoryKind) Enum() *CategoryKind { + p := new(CategoryKind) + *p = x + return p +} + +func (x CategoryKind) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (CategoryKind) Descriptor() protoreflect.EnumDescriptor { + return file_testdata_service_proto_enumTypes[0].Descriptor() +} + +func (CategoryKind) Type() protoreflect.EnumType { + return &file_testdata_service_proto_enumTypes[0] +} + +func (x CategoryKind) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use CategoryKind.Descriptor instead. +func (CategoryKind) EnumDescriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{0} +} + +type ShippingDestination int32 + +const ( + ShippingDestination_SHIPPING_DESTINATION_UNSPECIFIED ShippingDestination = 0 + ShippingDestination_SHIPPING_DESTINATION_DOMESTIC ShippingDestination = 1 + ShippingDestination_SHIPPING_DESTINATION_EXPRESS ShippingDestination = 2 + ShippingDestination_SHIPPING_DESTINATION_INTERNATIONAL ShippingDestination = 3 +) + +// Enum value maps for ShippingDestination. +var ( + ShippingDestination_name = map[int32]string{ + 0: "SHIPPING_DESTINATION_UNSPECIFIED", + 1: "SHIPPING_DESTINATION_DOMESTIC", + 2: "SHIPPING_DESTINATION_EXPRESS", + 3: "SHIPPING_DESTINATION_INTERNATIONAL", + } + ShippingDestination_value = map[string]int32{ + "SHIPPING_DESTINATION_UNSPECIFIED": 0, + "SHIPPING_DESTINATION_DOMESTIC": 1, + "SHIPPING_DESTINATION_EXPRESS": 2, + "SHIPPING_DESTINATION_INTERNATIONAL": 3, + } +) + +func (x ShippingDestination) Enum() *ShippingDestination { + p := new(ShippingDestination) + *p = x + return p +} + +func (x ShippingDestination) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (ShippingDestination) Descriptor() protoreflect.EnumDescriptor { + return file_testdata_service_proto_enumTypes[1].Descriptor() +} + +func (ShippingDestination) Type() protoreflect.EnumType { + return &file_testdata_service_proto_enumTypes[1] +} + +func (x ShippingDestination) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use ShippingDestination.Descriptor instead. +func (ShippingDestination) EnumDescriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{1} +} + +// Wrapper message for a list of AuthorFilter. +type ListOfAuthorFilter struct { + state protoimpl.MessageState `protogen:"open.v1"` + List *ListOfAuthorFilter_List `protobuf:"bytes,1,opt,name=list,proto3" json:"list,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ListOfAuthorFilter) Reset() { + *x = ListOfAuthorFilter{} + mi := &file_testdata_service_proto_msgTypes[0] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ListOfAuthorFilter) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ListOfAuthorFilter) ProtoMessage() {} + +func (x *ListOfAuthorFilter) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[0] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ListOfAuthorFilter.ProtoReflect.Descriptor instead. +func (*ListOfAuthorFilter) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{0} +} + +func (x *ListOfAuthorFilter) GetList() *ListOfAuthorFilter_List { + if x != nil { + return x.List + } + return nil +} + +// Wrapper message for a list of AuthorInput. +type ListOfAuthorInput struct { + state protoimpl.MessageState `protogen:"open.v1"` + List *ListOfAuthorInput_List `protobuf:"bytes,1,opt,name=list,proto3" json:"list,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ListOfAuthorInput) Reset() { + *x = ListOfAuthorInput{} + mi := &file_testdata_service_proto_msgTypes[1] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ListOfAuthorInput) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ListOfAuthorInput) ProtoMessage() {} + +func (x *ListOfAuthorInput) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[1] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ListOfAuthorInput.ProtoReflect.Descriptor instead. +func (*ListOfAuthorInput) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{1} +} + +func (x *ListOfAuthorInput) GetList() *ListOfAuthorInput_List { + if x != nil { + return x.List + } + return nil +} + +// Wrapper message for a list of BlogPost. +type ListOfBlogPost struct { + state protoimpl.MessageState `protogen:"open.v1"` + List *ListOfBlogPost_List `protobuf:"bytes,1,opt,name=list,proto3" json:"list,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ListOfBlogPost) Reset() { + *x = ListOfBlogPost{} + mi := &file_testdata_service_proto_msgTypes[2] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ListOfBlogPost) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ListOfBlogPost) ProtoMessage() {} + +func (x *ListOfBlogPost) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[2] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ListOfBlogPost.ProtoReflect.Descriptor instead. +func (*ListOfBlogPost) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{2} +} + +func (x *ListOfBlogPost) GetList() *ListOfBlogPost_List { + if x != nil { + return x.List + } + return nil +} + +// Wrapper message for a list of BlogPostFilter. +type ListOfBlogPostFilter struct { + state protoimpl.MessageState `protogen:"open.v1"` + List *ListOfBlogPostFilter_List `protobuf:"bytes,1,opt,name=list,proto3" json:"list,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ListOfBlogPostFilter) Reset() { + *x = ListOfBlogPostFilter{} + mi := &file_testdata_service_proto_msgTypes[3] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ListOfBlogPostFilter) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ListOfBlogPostFilter) ProtoMessage() {} + +func (x *ListOfBlogPostFilter) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[3] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ListOfBlogPostFilter.ProtoReflect.Descriptor instead. +func (*ListOfBlogPostFilter) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{3} +} + +func (x *ListOfBlogPostFilter) GetList() *ListOfBlogPostFilter_List { + if x != nil { + return x.List + } + return nil +} + +// Wrapper message for a list of BlogPostInput. +type ListOfBlogPostInput struct { + state protoimpl.MessageState `protogen:"open.v1"` + List *ListOfBlogPostInput_List `protobuf:"bytes,1,opt,name=list,proto3" json:"list,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ListOfBlogPostInput) Reset() { + *x = ListOfBlogPostInput{} + mi := &file_testdata_service_proto_msgTypes[4] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ListOfBlogPostInput) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ListOfBlogPostInput) ProtoMessage() {} + +func (x *ListOfBlogPostInput) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[4] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ListOfBlogPostInput.ProtoReflect.Descriptor instead. +func (*ListOfBlogPostInput) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{4} +} + +func (x *ListOfBlogPostInput) GetList() *ListOfBlogPostInput_List { + if x != nil { + return x.List + } + return nil +} + +// Wrapper message for a list of Boolean. +type ListOfBoolean struct { + state protoimpl.MessageState `protogen:"open.v1"` + List *ListOfBoolean_List `protobuf:"bytes,1,opt,name=list,proto3" json:"list,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ListOfBoolean) Reset() { + *x = ListOfBoolean{} + mi := &file_testdata_service_proto_msgTypes[5] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ListOfBoolean) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ListOfBoolean) ProtoMessage() {} + +func (x *ListOfBoolean) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[5] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ListOfBoolean.ProtoReflect.Descriptor instead. +func (*ListOfBoolean) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{5} +} + +func (x *ListOfBoolean) GetList() *ListOfBoolean_List { + if x != nil { + return x.List + } + return nil +} + +// Wrapper message for a list of Category. +type ListOfCategory struct { + state protoimpl.MessageState `protogen:"open.v1"` + List *ListOfCategory_List `protobuf:"bytes,1,opt,name=list,proto3" json:"list,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ListOfCategory) Reset() { + *x = ListOfCategory{} + mi := &file_testdata_service_proto_msgTypes[6] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ListOfCategory) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ListOfCategory) ProtoMessage() {} + +func (x *ListOfCategory) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[6] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ListOfCategory.ProtoReflect.Descriptor instead. +func (*ListOfCategory) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{6} +} + +func (x *ListOfCategory) GetList() *ListOfCategory_List { + if x != nil { + return x.List + } + return nil +} + +// Wrapper message for a list of CategoryInput. +type ListOfCategoryInput struct { + state protoimpl.MessageState `protogen:"open.v1"` + List *ListOfCategoryInput_List `protobuf:"bytes,1,opt,name=list,proto3" json:"list,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ListOfCategoryInput) Reset() { + *x = ListOfCategoryInput{} + mi := &file_testdata_service_proto_msgTypes[7] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ListOfCategoryInput) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ListOfCategoryInput) ProtoMessage() {} + +func (x *ListOfCategoryInput) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[7] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ListOfCategoryInput.ProtoReflect.Descriptor instead. +func (*ListOfCategoryInput) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{7} +} + +func (x *ListOfCategoryInput) GetList() *ListOfCategoryInput_List { + if x != nil { + return x.List + } + return nil +} + +// Wrapper message for a list of Float. +type ListOfFloat struct { + state protoimpl.MessageState `protogen:"open.v1"` + List *ListOfFloat_List `protobuf:"bytes,1,opt,name=list,proto3" json:"list,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ListOfFloat) Reset() { + *x = ListOfFloat{} + mi := &file_testdata_service_proto_msgTypes[8] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ListOfFloat) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ListOfFloat) ProtoMessage() {} + +func (x *ListOfFloat) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[8] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ListOfFloat.ProtoReflect.Descriptor instead. +func (*ListOfFloat) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{8} +} + +func (x *ListOfFloat) GetList() *ListOfFloat_List { + if x != nil { + return x.List + } + return nil +} + +// Wrapper message for a list of Category. +type ListOfListOfCategory struct { + state protoimpl.MessageState `protogen:"open.v1"` + List *ListOfListOfCategory_List `protobuf:"bytes,1,opt,name=list,proto3" json:"list,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ListOfListOfCategory) Reset() { + *x = ListOfListOfCategory{} + mi := &file_testdata_service_proto_msgTypes[9] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ListOfListOfCategory) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ListOfListOfCategory) ProtoMessage() {} + +func (x *ListOfListOfCategory) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[9] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ListOfListOfCategory.ProtoReflect.Descriptor instead. +func (*ListOfListOfCategory) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{9} +} + +func (x *ListOfListOfCategory) GetList() *ListOfListOfCategory_List { + if x != nil { + return x.List + } + return nil +} + +// Wrapper message for a list of CategoryInput. +type ListOfListOfCategoryInput struct { + state protoimpl.MessageState `protogen:"open.v1"` + List *ListOfListOfCategoryInput_List `protobuf:"bytes,1,opt,name=list,proto3" json:"list,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ListOfListOfCategoryInput) Reset() { + *x = ListOfListOfCategoryInput{} + mi := &file_testdata_service_proto_msgTypes[10] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ListOfListOfCategoryInput) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ListOfListOfCategoryInput) ProtoMessage() {} + +func (x *ListOfListOfCategoryInput) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[10] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ListOfListOfCategoryInput.ProtoReflect.Descriptor instead. +func (*ListOfListOfCategoryInput) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{10} +} + +func (x *ListOfListOfCategoryInput) GetList() *ListOfListOfCategoryInput_List { + if x != nil { + return x.List + } + return nil +} + +// Wrapper message for a list of String. +type ListOfListOfString struct { + state protoimpl.MessageState `protogen:"open.v1"` + List *ListOfListOfString_List `protobuf:"bytes,1,opt,name=list,proto3" json:"list,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ListOfListOfString) Reset() { + *x = ListOfListOfString{} + mi := &file_testdata_service_proto_msgTypes[11] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ListOfListOfString) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ListOfListOfString) ProtoMessage() {} + +func (x *ListOfListOfString) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[11] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ListOfListOfString.ProtoReflect.Descriptor instead. +func (*ListOfListOfString) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{11} +} + +func (x *ListOfListOfString) GetList() *ListOfListOfString_List { + if x != nil { + return x.List + } + return nil +} + +// Wrapper message for a list of User. +type ListOfListOfUser struct { + state protoimpl.MessageState `protogen:"open.v1"` + List *ListOfListOfUser_List `protobuf:"bytes,1,opt,name=list,proto3" json:"list,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ListOfListOfUser) Reset() { + *x = ListOfListOfUser{} + mi := &file_testdata_service_proto_msgTypes[12] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ListOfListOfUser) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ListOfListOfUser) ProtoMessage() {} + +func (x *ListOfListOfUser) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[12] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ListOfListOfUser.ProtoReflect.Descriptor instead. +func (*ListOfListOfUser) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{12} +} + +func (x *ListOfListOfUser) GetList() *ListOfListOfUser_List { + if x != nil { + return x.List + } + return nil +} + +// Wrapper message for a list of UserInput. +type ListOfListOfUserInput struct { + state protoimpl.MessageState `protogen:"open.v1"` + List *ListOfListOfUserInput_List `protobuf:"bytes,1,opt,name=list,proto3" json:"list,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ListOfListOfUserInput) Reset() { + *x = ListOfListOfUserInput{} + mi := &file_testdata_service_proto_msgTypes[13] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ListOfListOfUserInput) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ListOfListOfUserInput) ProtoMessage() {} + +func (x *ListOfListOfUserInput) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[13] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ListOfListOfUserInput.ProtoReflect.Descriptor instead. +func (*ListOfListOfUserInput) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{13} +} + +func (x *ListOfListOfUserInput) GetList() *ListOfListOfUserInput_List { + if x != nil { + return x.List + } + return nil +} + +// Wrapper message for a list of OrderLine. +type ListOfOrderLine struct { + state protoimpl.MessageState `protogen:"open.v1"` + List *ListOfOrderLine_List `protobuf:"bytes,1,opt,name=list,proto3" json:"list,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ListOfOrderLine) Reset() { + *x = ListOfOrderLine{} + mi := &file_testdata_service_proto_msgTypes[14] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ListOfOrderLine) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ListOfOrderLine) ProtoMessage() {} + +func (x *ListOfOrderLine) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[14] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ListOfOrderLine.ProtoReflect.Descriptor instead. +func (*ListOfOrderLine) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{14} +} + +func (x *ListOfOrderLine) GetList() *ListOfOrderLine_List { + if x != nil { + return x.List + } + return nil +} + +// Wrapper message for a list of Product. +type ListOfProduct struct { + state protoimpl.MessageState `protogen:"open.v1"` + List *ListOfProduct_List `protobuf:"bytes,1,opt,name=list,proto3" json:"list,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ListOfProduct) Reset() { + *x = ListOfProduct{} + mi := &file_testdata_service_proto_msgTypes[15] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ListOfProduct) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ListOfProduct) ProtoMessage() {} + +func (x *ListOfProduct) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[15] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ListOfProduct.ProtoReflect.Descriptor instead. +func (*ListOfProduct) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{15} +} + +func (x *ListOfProduct) GetList() *ListOfProduct_List { + if x != nil { + return x.List + } + return nil +} + +// Wrapper message for a list of String. +type ListOfString struct { + state protoimpl.MessageState `protogen:"open.v1"` + List *ListOfString_List `protobuf:"bytes,1,opt,name=list,proto3" json:"list,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ListOfString) Reset() { + *x = ListOfString{} + mi := &file_testdata_service_proto_msgTypes[16] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ListOfString) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ListOfString) ProtoMessage() {} + +func (x *ListOfString) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[16] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ListOfString.ProtoReflect.Descriptor instead. +func (*ListOfString) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{16} +} + +func (x *ListOfString) GetList() *ListOfString_List { + if x != nil { + return x.List + } + return nil +} + +// Wrapper message for a list of Subcategory. +type ListOfSubcategory struct { + state protoimpl.MessageState `protogen:"open.v1"` + List *ListOfSubcategory_List `protobuf:"bytes,1,opt,name=list,proto3" json:"list,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ListOfSubcategory) Reset() { + *x = ListOfSubcategory{} + mi := &file_testdata_service_proto_msgTypes[17] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ListOfSubcategory) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ListOfSubcategory) ProtoMessage() {} + +func (x *ListOfSubcategory) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[17] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ListOfSubcategory.ProtoReflect.Descriptor instead. +func (*ListOfSubcategory) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{17} +} + +func (x *ListOfSubcategory) GetList() *ListOfSubcategory_List { + if x != nil { + return x.List + } + return nil +} + +// Wrapper message for a list of User. +type ListOfUser struct { + state protoimpl.MessageState `protogen:"open.v1"` + List *ListOfUser_List `protobuf:"bytes,1,opt,name=list,proto3" json:"list,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ListOfUser) Reset() { + *x = ListOfUser{} + mi := &file_testdata_service_proto_msgTypes[18] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ListOfUser) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ListOfUser) ProtoMessage() {} + +func (x *ListOfUser) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[18] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ListOfUser.ProtoReflect.Descriptor instead. +func (*ListOfUser) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{18} +} + +func (x *ListOfUser) GetList() *ListOfUser_List { + if x != nil { + return x.List + } + return nil +} + +// Wrapper message for a list of UserInput. +type ListOfUserInput struct { + state protoimpl.MessageState `protogen:"open.v1"` + List *ListOfUserInput_List `protobuf:"bytes,1,opt,name=list,proto3" json:"list,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ListOfUserInput) Reset() { + *x = ListOfUserInput{} + mi := &file_testdata_service_proto_msgTypes[19] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ListOfUserInput) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ListOfUserInput) ProtoMessage() {} + +func (x *ListOfUserInput) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[19] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ListOfUserInput.ProtoReflect.Descriptor instead. +func (*ListOfUserInput) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{19} +} + +func (x *ListOfUserInput) GetList() *ListOfUserInput_List { + if x != nil { + return x.List + } + return nil +} + +// Key message for Product entity lookup +type LookupProductByIdRequestKey struct { + state protoimpl.MessageState `protogen:"open.v1"` + // Key field for Product entity lookup. + Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *LookupProductByIdRequestKey) Reset() { + *x = LookupProductByIdRequestKey{} + mi := &file_testdata_service_proto_msgTypes[20] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *LookupProductByIdRequestKey) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*LookupProductByIdRequestKey) ProtoMessage() {} + +func (x *LookupProductByIdRequestKey) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[20] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use LookupProductByIdRequestKey.ProtoReflect.Descriptor instead. +func (*LookupProductByIdRequestKey) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{20} +} + +func (x *LookupProductByIdRequestKey) GetId() string { + if x != nil { + return x.Id + } + return "" +} + +// Request message for Product entity lookup. +type LookupProductByIdRequest struct { + state protoimpl.MessageState `protogen:"open.v1"` + // List of keys to look up Product entities. + // Order matters - each key maps to one entity in LookupProductByIdResponse. + Keys []*LookupProductByIdRequestKey `protobuf:"bytes,1,rep,name=keys,proto3" json:"keys,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *LookupProductByIdRequest) Reset() { + *x = LookupProductByIdRequest{} + mi := &file_testdata_service_proto_msgTypes[21] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *LookupProductByIdRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*LookupProductByIdRequest) ProtoMessage() {} + +func (x *LookupProductByIdRequest) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[21] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use LookupProductByIdRequest.ProtoReflect.Descriptor instead. +func (*LookupProductByIdRequest) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{21} +} + +func (x *LookupProductByIdRequest) GetKeys() []*LookupProductByIdRequestKey { + if x != nil { + return x.Keys + } + return nil +} + +// Response message for Product entity lookup. +type LookupProductByIdResponse struct { + state protoimpl.MessageState `protogen:"open.v1"` + // List of Product entities in the same order as the keys in LookupProductByIdRequest. + // Always return the same number of entities as keys. Use null for entities that cannot be found. + // + // Example: + // + // LookupUserByIdRequest: + // keys: + // - id: 1 + // - id: 2 + // LookupUserByIdResponse: + // result: + // - id: 1 # User with id 1 found + // - null # User with id 2 not found + Result []*Product `protobuf:"bytes,1,rep,name=result,proto3" json:"result,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *LookupProductByIdResponse) Reset() { + *x = LookupProductByIdResponse{} + mi := &file_testdata_service_proto_msgTypes[22] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *LookupProductByIdResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*LookupProductByIdResponse) ProtoMessage() {} + +func (x *LookupProductByIdResponse) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[22] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use LookupProductByIdResponse.ProtoReflect.Descriptor instead. +func (*LookupProductByIdResponse) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{22} +} + +func (x *LookupProductByIdResponse) GetResult() []*Product { + if x != nil { + return x.Result + } + return nil +} + +// Key message for Storage entity lookup +type LookupStorageByIdRequestKey struct { + state protoimpl.MessageState `protogen:"open.v1"` + // Key field for Storage entity lookup. + Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *LookupStorageByIdRequestKey) Reset() { + *x = LookupStorageByIdRequestKey{} + mi := &file_testdata_service_proto_msgTypes[23] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *LookupStorageByIdRequestKey) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*LookupStorageByIdRequestKey) ProtoMessage() {} + +func (x *LookupStorageByIdRequestKey) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[23] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use LookupStorageByIdRequestKey.ProtoReflect.Descriptor instead. +func (*LookupStorageByIdRequestKey) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{23} +} + +func (x *LookupStorageByIdRequestKey) GetId() string { + if x != nil { + return x.Id + } + return "" +} + +// Request message for Storage entity lookup. +type LookupStorageByIdRequest struct { + state protoimpl.MessageState `protogen:"open.v1"` + // List of keys to look up Storage entities. + // Order matters - each key maps to one entity in LookupStorageByIdResponse. + Keys []*LookupStorageByIdRequestKey `protobuf:"bytes,1,rep,name=keys,proto3" json:"keys,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *LookupStorageByIdRequest) Reset() { + *x = LookupStorageByIdRequest{} + mi := &file_testdata_service_proto_msgTypes[24] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *LookupStorageByIdRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*LookupStorageByIdRequest) ProtoMessage() {} + +func (x *LookupStorageByIdRequest) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[24] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use LookupStorageByIdRequest.ProtoReflect.Descriptor instead. +func (*LookupStorageByIdRequest) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{24} +} + +func (x *LookupStorageByIdRequest) GetKeys() []*LookupStorageByIdRequestKey { + if x != nil { + return x.Keys + } + return nil +} + +// Response message for Storage entity lookup. +type LookupStorageByIdResponse struct { + state protoimpl.MessageState `protogen:"open.v1"` + // List of Storage entities in the same order as the keys in LookupStorageByIdRequest. + // Always return the same number of entities as keys. Use null for entities that cannot be found. + // + // Example: + // + // LookupUserByIdRequest: + // keys: + // - id: 1 + // - id: 2 + // LookupUserByIdResponse: + // result: + // - id: 1 # User with id 1 found + // - null # User with id 2 not found + Result []*Storage `protobuf:"bytes,1,rep,name=result,proto3" json:"result,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *LookupStorageByIdResponse) Reset() { + *x = LookupStorageByIdResponse{} + mi := &file_testdata_service_proto_msgTypes[25] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *LookupStorageByIdResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*LookupStorageByIdResponse) ProtoMessage() {} + +func (x *LookupStorageByIdResponse) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[25] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use LookupStorageByIdResponse.ProtoReflect.Descriptor instead. +func (*LookupStorageByIdResponse) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{25} +} + +func (x *LookupStorageByIdResponse) GetResult() []*Storage { + if x != nil { + return x.Result + } + return nil +} + +// Key message for Warehouse entity lookup +type LookupWarehouseByIdRequestKey struct { + state protoimpl.MessageState `protogen:"open.v1"` + // Key field for Warehouse entity lookup. + Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *LookupWarehouseByIdRequestKey) Reset() { + *x = LookupWarehouseByIdRequestKey{} + mi := &file_testdata_service_proto_msgTypes[26] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *LookupWarehouseByIdRequestKey) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*LookupWarehouseByIdRequestKey) ProtoMessage() {} + +func (x *LookupWarehouseByIdRequestKey) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[26] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use LookupWarehouseByIdRequestKey.ProtoReflect.Descriptor instead. +func (*LookupWarehouseByIdRequestKey) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{26} +} + +func (x *LookupWarehouseByIdRequestKey) GetId() string { + if x != nil { + return x.Id + } + return "" +} + +// Request message for Warehouse entity lookup. +type LookupWarehouseByIdRequest struct { + state protoimpl.MessageState `protogen:"open.v1"` + // List of keys to look up Warehouse entities. + // Order matters - each key maps to one entity in LookupWarehouseByIdResponse. + Keys []*LookupWarehouseByIdRequestKey `protobuf:"bytes,1,rep,name=keys,proto3" json:"keys,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *LookupWarehouseByIdRequest) Reset() { + *x = LookupWarehouseByIdRequest{} + mi := &file_testdata_service_proto_msgTypes[27] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *LookupWarehouseByIdRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*LookupWarehouseByIdRequest) ProtoMessage() {} + +func (x *LookupWarehouseByIdRequest) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[27] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use LookupWarehouseByIdRequest.ProtoReflect.Descriptor instead. +func (*LookupWarehouseByIdRequest) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{27} +} + +func (x *LookupWarehouseByIdRequest) GetKeys() []*LookupWarehouseByIdRequestKey { + if x != nil { + return x.Keys + } + return nil +} + +// Response message for Warehouse entity lookup. +type LookupWarehouseByIdResponse struct { + state protoimpl.MessageState `protogen:"open.v1"` + // List of Warehouse entities in the same order as the keys in LookupWarehouseByIdRequest. + // Always return the same number of entities as keys. Use null for entities that cannot be found. + // + // Example: + // + // LookupUserByIdRequest: + // keys: + // - id: 1 + // - id: 2 + // LookupUserByIdResponse: + // result: + // - id: 1 # User with id 1 found + // - null # User with id 2 not found + Result []*Warehouse `protobuf:"bytes,1,rep,name=result,proto3" json:"result,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *LookupWarehouseByIdResponse) Reset() { + *x = LookupWarehouseByIdResponse{} + mi := &file_testdata_service_proto_msgTypes[28] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *LookupWarehouseByIdResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*LookupWarehouseByIdResponse) ProtoMessage() {} + +func (x *LookupWarehouseByIdResponse) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[28] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use LookupWarehouseByIdResponse.ProtoReflect.Descriptor instead. +func (*LookupWarehouseByIdResponse) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{28} +} + +func (x *LookupWarehouseByIdResponse) GetResult() []*Warehouse { + if x != nil { + return x.Result + } + return nil +} + +// Request message for users operation. +type QueryUsersRequest struct { + state protoimpl.MessageState `protogen:"open.v1"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *QueryUsersRequest) Reset() { + *x = QueryUsersRequest{} + mi := &file_testdata_service_proto_msgTypes[29] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *QueryUsersRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*QueryUsersRequest) ProtoMessage() {} + +func (x *QueryUsersRequest) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[29] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use QueryUsersRequest.ProtoReflect.Descriptor instead. +func (*QueryUsersRequest) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{29} +} + +// Response message for users operation. +type QueryUsersResponse struct { + state protoimpl.MessageState `protogen:"open.v1"` + Users []*User `protobuf:"bytes,1,rep,name=users,proto3" json:"users,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *QueryUsersResponse) Reset() { + *x = QueryUsersResponse{} + mi := &file_testdata_service_proto_msgTypes[30] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *QueryUsersResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*QueryUsersResponse) ProtoMessage() {} + +func (x *QueryUsersResponse) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[30] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use QueryUsersResponse.ProtoReflect.Descriptor instead. +func (*QueryUsersResponse) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{30} +} + +func (x *QueryUsersResponse) GetUsers() []*User { + if x != nil { + return x.Users + } + return nil +} + +// Request message for user operation. +type QueryUserRequest struct { + state protoimpl.MessageState `protogen:"open.v1"` + Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *QueryUserRequest) Reset() { + *x = QueryUserRequest{} + mi := &file_testdata_service_proto_msgTypes[31] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *QueryUserRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*QueryUserRequest) ProtoMessage() {} + +func (x *QueryUserRequest) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[31] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use QueryUserRequest.ProtoReflect.Descriptor instead. +func (*QueryUserRequest) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{31} +} + +func (x *QueryUserRequest) GetId() string { + if x != nil { + return x.Id + } + return "" +} + +// Response message for user operation. +type QueryUserResponse struct { + state protoimpl.MessageState `protogen:"open.v1"` + User *User `protobuf:"bytes,1,opt,name=user,proto3" json:"user,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *QueryUserResponse) Reset() { + *x = QueryUserResponse{} + mi := &file_testdata_service_proto_msgTypes[32] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *QueryUserResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*QueryUserResponse) ProtoMessage() {} + +func (x *QueryUserResponse) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[32] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use QueryUserResponse.ProtoReflect.Descriptor instead. +func (*QueryUserResponse) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{32} +} + +func (x *QueryUserResponse) GetUser() *User { + if x != nil { + return x.User + } + return nil +} + +// Request message for nestedType operation. +type QueryNestedTypeRequest struct { + state protoimpl.MessageState `protogen:"open.v1"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *QueryNestedTypeRequest) Reset() { + *x = QueryNestedTypeRequest{} + mi := &file_testdata_service_proto_msgTypes[33] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *QueryNestedTypeRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*QueryNestedTypeRequest) ProtoMessage() {} + +func (x *QueryNestedTypeRequest) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[33] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use QueryNestedTypeRequest.ProtoReflect.Descriptor instead. +func (*QueryNestedTypeRequest) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{33} +} + +// Response message for nestedType operation. +type QueryNestedTypeResponse struct { + state protoimpl.MessageState `protogen:"open.v1"` + NestedType []*NestedTypeA `protobuf:"bytes,1,rep,name=nested_type,json=nestedType,proto3" json:"nested_type,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *QueryNestedTypeResponse) Reset() { + *x = QueryNestedTypeResponse{} + mi := &file_testdata_service_proto_msgTypes[34] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *QueryNestedTypeResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*QueryNestedTypeResponse) ProtoMessage() {} + +func (x *QueryNestedTypeResponse) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[34] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use QueryNestedTypeResponse.ProtoReflect.Descriptor instead. +func (*QueryNestedTypeResponse) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{34} +} + +func (x *QueryNestedTypeResponse) GetNestedType() []*NestedTypeA { + if x != nil { + return x.NestedType + } + return nil +} + +// Request message for recursiveType operation. +type QueryRecursiveTypeRequest struct { + state protoimpl.MessageState `protogen:"open.v1"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *QueryRecursiveTypeRequest) Reset() { + *x = QueryRecursiveTypeRequest{} + mi := &file_testdata_service_proto_msgTypes[35] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *QueryRecursiveTypeRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*QueryRecursiveTypeRequest) ProtoMessage() {} + +func (x *QueryRecursiveTypeRequest) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[35] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use QueryRecursiveTypeRequest.ProtoReflect.Descriptor instead. +func (*QueryRecursiveTypeRequest) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{35} +} + +// Response message for recursiveType operation. +type QueryRecursiveTypeResponse struct { + state protoimpl.MessageState `protogen:"open.v1"` + RecursiveType *RecursiveType `protobuf:"bytes,1,opt,name=recursive_type,json=recursiveType,proto3" json:"recursive_type,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *QueryRecursiveTypeResponse) Reset() { + *x = QueryRecursiveTypeResponse{} + mi := &file_testdata_service_proto_msgTypes[36] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *QueryRecursiveTypeResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*QueryRecursiveTypeResponse) ProtoMessage() {} + +func (x *QueryRecursiveTypeResponse) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[36] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use QueryRecursiveTypeResponse.ProtoReflect.Descriptor instead. +func (*QueryRecursiveTypeResponse) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{36} +} + +func (x *QueryRecursiveTypeResponse) GetRecursiveType() *RecursiveType { + if x != nil { + return x.RecursiveType + } + return nil +} + +// Request message for typeFilterWithArguments operation. +type QueryTypeFilterWithArgumentsRequest struct { + state protoimpl.MessageState `protogen:"open.v1"` + FilterField_1 string `protobuf:"bytes,1,opt,name=filter_field_1,json=filterField1,proto3" json:"filter_field_1,omitempty"` + FilterField_2 string `protobuf:"bytes,2,opt,name=filter_field_2,json=filterField2,proto3" json:"filter_field_2,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *QueryTypeFilterWithArgumentsRequest) Reset() { + *x = QueryTypeFilterWithArgumentsRequest{} + mi := &file_testdata_service_proto_msgTypes[37] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *QueryTypeFilterWithArgumentsRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*QueryTypeFilterWithArgumentsRequest) ProtoMessage() {} + +func (x *QueryTypeFilterWithArgumentsRequest) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[37] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use QueryTypeFilterWithArgumentsRequest.ProtoReflect.Descriptor instead. +func (*QueryTypeFilterWithArgumentsRequest) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{37} +} + +func (x *QueryTypeFilterWithArgumentsRequest) GetFilterField_1() string { + if x != nil { + return x.FilterField_1 + } + return "" +} + +func (x *QueryTypeFilterWithArgumentsRequest) GetFilterField_2() string { + if x != nil { + return x.FilterField_2 + } + return "" +} + +// Response message for typeFilterWithArguments operation. +type QueryTypeFilterWithArgumentsResponse struct { + state protoimpl.MessageState `protogen:"open.v1"` + TypeFilterWithArguments []*TypeWithMultipleFilterFields `protobuf:"bytes,1,rep,name=type_filter_with_arguments,json=typeFilterWithArguments,proto3" json:"type_filter_with_arguments,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *QueryTypeFilterWithArgumentsResponse) Reset() { + *x = QueryTypeFilterWithArgumentsResponse{} + mi := &file_testdata_service_proto_msgTypes[38] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *QueryTypeFilterWithArgumentsResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*QueryTypeFilterWithArgumentsResponse) ProtoMessage() {} + +func (x *QueryTypeFilterWithArgumentsResponse) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[38] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use QueryTypeFilterWithArgumentsResponse.ProtoReflect.Descriptor instead. +func (*QueryTypeFilterWithArgumentsResponse) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{38} +} + +func (x *QueryTypeFilterWithArgumentsResponse) GetTypeFilterWithArguments() []*TypeWithMultipleFilterFields { + if x != nil { + return x.TypeFilterWithArguments + } + return nil +} + +// Request message for typeWithMultipleFilterFields operation. +type QueryTypeWithMultipleFilterFieldsRequest struct { + state protoimpl.MessageState `protogen:"open.v1"` + Filter *FilterTypeInput `protobuf:"bytes,1,opt,name=filter,proto3" json:"filter,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *QueryTypeWithMultipleFilterFieldsRequest) Reset() { + *x = QueryTypeWithMultipleFilterFieldsRequest{} + mi := &file_testdata_service_proto_msgTypes[39] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *QueryTypeWithMultipleFilterFieldsRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*QueryTypeWithMultipleFilterFieldsRequest) ProtoMessage() {} + +func (x *QueryTypeWithMultipleFilterFieldsRequest) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[39] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use QueryTypeWithMultipleFilterFieldsRequest.ProtoReflect.Descriptor instead. +func (*QueryTypeWithMultipleFilterFieldsRequest) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{39} +} + +func (x *QueryTypeWithMultipleFilterFieldsRequest) GetFilter() *FilterTypeInput { + if x != nil { + return x.Filter + } + return nil +} + +// Response message for typeWithMultipleFilterFields operation. +type QueryTypeWithMultipleFilterFieldsResponse struct { + state protoimpl.MessageState `protogen:"open.v1"` + TypeWithMultipleFilterFields []*TypeWithMultipleFilterFields `protobuf:"bytes,1,rep,name=type_with_multiple_filter_fields,json=typeWithMultipleFilterFields,proto3" json:"type_with_multiple_filter_fields,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *QueryTypeWithMultipleFilterFieldsResponse) Reset() { + *x = QueryTypeWithMultipleFilterFieldsResponse{} + mi := &file_testdata_service_proto_msgTypes[40] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *QueryTypeWithMultipleFilterFieldsResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*QueryTypeWithMultipleFilterFieldsResponse) ProtoMessage() {} + +func (x *QueryTypeWithMultipleFilterFieldsResponse) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[40] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use QueryTypeWithMultipleFilterFieldsResponse.ProtoReflect.Descriptor instead. +func (*QueryTypeWithMultipleFilterFieldsResponse) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{40} +} + +func (x *QueryTypeWithMultipleFilterFieldsResponse) GetTypeWithMultipleFilterFields() []*TypeWithMultipleFilterFields { + if x != nil { + return x.TypeWithMultipleFilterFields + } + return nil +} + +// Request message for complexFilterType operation. +type QueryComplexFilterTypeRequest struct { + state protoimpl.MessageState `protogen:"open.v1"` + Filter *ComplexFilterTypeInput `protobuf:"bytes,1,opt,name=filter,proto3" json:"filter,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *QueryComplexFilterTypeRequest) Reset() { + *x = QueryComplexFilterTypeRequest{} + mi := &file_testdata_service_proto_msgTypes[41] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *QueryComplexFilterTypeRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*QueryComplexFilterTypeRequest) ProtoMessage() {} + +func (x *QueryComplexFilterTypeRequest) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[41] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use QueryComplexFilterTypeRequest.ProtoReflect.Descriptor instead. +func (*QueryComplexFilterTypeRequest) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{41} +} + +func (x *QueryComplexFilterTypeRequest) GetFilter() *ComplexFilterTypeInput { + if x != nil { + return x.Filter + } + return nil +} + +// Response message for complexFilterType operation. +type QueryComplexFilterTypeResponse struct { + state protoimpl.MessageState `protogen:"open.v1"` + ComplexFilterType []*TypeWithComplexFilterInput `protobuf:"bytes,1,rep,name=complex_filter_type,json=complexFilterType,proto3" json:"complex_filter_type,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *QueryComplexFilterTypeResponse) Reset() { + *x = QueryComplexFilterTypeResponse{} + mi := &file_testdata_service_proto_msgTypes[42] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *QueryComplexFilterTypeResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*QueryComplexFilterTypeResponse) ProtoMessage() {} + +func (x *QueryComplexFilterTypeResponse) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[42] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use QueryComplexFilterTypeResponse.ProtoReflect.Descriptor instead. +func (*QueryComplexFilterTypeResponse) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{42} +} + +func (x *QueryComplexFilterTypeResponse) GetComplexFilterType() []*TypeWithComplexFilterInput { + if x != nil { + return x.ComplexFilterType + } + return nil +} + +// Request message for calculateTotals operation. +type QueryCalculateTotalsRequest struct { + state protoimpl.MessageState `protogen:"open.v1"` + Orders []*OrderInput `protobuf:"bytes,1,rep,name=orders,proto3" json:"orders,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *QueryCalculateTotalsRequest) Reset() { + *x = QueryCalculateTotalsRequest{} + mi := &file_testdata_service_proto_msgTypes[43] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *QueryCalculateTotalsRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*QueryCalculateTotalsRequest) ProtoMessage() {} + +func (x *QueryCalculateTotalsRequest) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[43] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use QueryCalculateTotalsRequest.ProtoReflect.Descriptor instead. +func (*QueryCalculateTotalsRequest) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{43} +} + +func (x *QueryCalculateTotalsRequest) GetOrders() []*OrderInput { + if x != nil { + return x.Orders + } + return nil +} + +// Response message for calculateTotals operation. +type QueryCalculateTotalsResponse struct { + state protoimpl.MessageState `protogen:"open.v1"` + CalculateTotals []*Order `protobuf:"bytes,1,rep,name=calculate_totals,json=calculateTotals,proto3" json:"calculate_totals,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *QueryCalculateTotalsResponse) Reset() { + *x = QueryCalculateTotalsResponse{} + mi := &file_testdata_service_proto_msgTypes[44] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *QueryCalculateTotalsResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*QueryCalculateTotalsResponse) ProtoMessage() {} + +func (x *QueryCalculateTotalsResponse) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[44] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use QueryCalculateTotalsResponse.ProtoReflect.Descriptor instead. +func (*QueryCalculateTotalsResponse) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{44} +} + +func (x *QueryCalculateTotalsResponse) GetCalculateTotals() []*Order { + if x != nil { + return x.CalculateTotals + } + return nil +} + +// Request message for categories operation. +type QueryCategoriesRequest struct { + state protoimpl.MessageState `protogen:"open.v1"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *QueryCategoriesRequest) Reset() { + *x = QueryCategoriesRequest{} + mi := &file_testdata_service_proto_msgTypes[45] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *QueryCategoriesRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*QueryCategoriesRequest) ProtoMessage() {} + +func (x *QueryCategoriesRequest) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[45] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use QueryCategoriesRequest.ProtoReflect.Descriptor instead. +func (*QueryCategoriesRequest) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{45} +} + +// Response message for categories operation. +type QueryCategoriesResponse struct { + state protoimpl.MessageState `protogen:"open.v1"` + Categories []*Category `protobuf:"bytes,1,rep,name=categories,proto3" json:"categories,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *QueryCategoriesResponse) Reset() { + *x = QueryCategoriesResponse{} + mi := &file_testdata_service_proto_msgTypes[46] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *QueryCategoriesResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*QueryCategoriesResponse) ProtoMessage() {} + +func (x *QueryCategoriesResponse) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[46] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use QueryCategoriesResponse.ProtoReflect.Descriptor instead. +func (*QueryCategoriesResponse) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{46} +} + +func (x *QueryCategoriesResponse) GetCategories() []*Category { + if x != nil { + return x.Categories + } + return nil +} + +// Request message for categoriesByKind operation. +type QueryCategoriesByKindRequest struct { + state protoimpl.MessageState `protogen:"open.v1"` + Kind CategoryKind `protobuf:"varint,1,opt,name=kind,proto3,enum=productv1.CategoryKind" json:"kind,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *QueryCategoriesByKindRequest) Reset() { + *x = QueryCategoriesByKindRequest{} + mi := &file_testdata_service_proto_msgTypes[47] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *QueryCategoriesByKindRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*QueryCategoriesByKindRequest) ProtoMessage() {} + +func (x *QueryCategoriesByKindRequest) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[47] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use QueryCategoriesByKindRequest.ProtoReflect.Descriptor instead. +func (*QueryCategoriesByKindRequest) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{47} +} + +func (x *QueryCategoriesByKindRequest) GetKind() CategoryKind { + if x != nil { + return x.Kind + } + return CategoryKind_CATEGORY_KIND_UNSPECIFIED +} + +// Response message for categoriesByKind operation. +type QueryCategoriesByKindResponse struct { + state protoimpl.MessageState `protogen:"open.v1"` + CategoriesByKind []*Category `protobuf:"bytes,1,rep,name=categories_by_kind,json=categoriesByKind,proto3" json:"categories_by_kind,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *QueryCategoriesByKindResponse) Reset() { + *x = QueryCategoriesByKindResponse{} + mi := &file_testdata_service_proto_msgTypes[48] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *QueryCategoriesByKindResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*QueryCategoriesByKindResponse) ProtoMessage() {} + +func (x *QueryCategoriesByKindResponse) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[48] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use QueryCategoriesByKindResponse.ProtoReflect.Descriptor instead. +func (*QueryCategoriesByKindResponse) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{48} +} + +func (x *QueryCategoriesByKindResponse) GetCategoriesByKind() []*Category { + if x != nil { + return x.CategoriesByKind + } + return nil +} + +// Request message for categoriesByKinds operation. +type QueryCategoriesByKindsRequest struct { + state protoimpl.MessageState `protogen:"open.v1"` + Kinds []CategoryKind `protobuf:"varint,1,rep,packed,name=kinds,proto3,enum=productv1.CategoryKind" json:"kinds,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *QueryCategoriesByKindsRequest) Reset() { + *x = QueryCategoriesByKindsRequest{} + mi := &file_testdata_service_proto_msgTypes[49] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *QueryCategoriesByKindsRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*QueryCategoriesByKindsRequest) ProtoMessage() {} + +func (x *QueryCategoriesByKindsRequest) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[49] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use QueryCategoriesByKindsRequest.ProtoReflect.Descriptor instead. +func (*QueryCategoriesByKindsRequest) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{49} +} + +func (x *QueryCategoriesByKindsRequest) GetKinds() []CategoryKind { + if x != nil { + return x.Kinds + } + return nil +} + +// Response message for categoriesByKinds operation. +type QueryCategoriesByKindsResponse struct { + state protoimpl.MessageState `protogen:"open.v1"` + CategoriesByKinds []*Category `protobuf:"bytes,1,rep,name=categories_by_kinds,json=categoriesByKinds,proto3" json:"categories_by_kinds,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *QueryCategoriesByKindsResponse) Reset() { + *x = QueryCategoriesByKindsResponse{} + mi := &file_testdata_service_proto_msgTypes[50] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *QueryCategoriesByKindsResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*QueryCategoriesByKindsResponse) ProtoMessage() {} + +func (x *QueryCategoriesByKindsResponse) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[50] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use QueryCategoriesByKindsResponse.ProtoReflect.Descriptor instead. +func (*QueryCategoriesByKindsResponse) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{50} +} + +func (x *QueryCategoriesByKindsResponse) GetCategoriesByKinds() []*Category { + if x != nil { + return x.CategoriesByKinds + } + return nil +} + +// Request message for filterCategories operation. +type QueryFilterCategoriesRequest struct { + state protoimpl.MessageState `protogen:"open.v1"` + Filter *CategoryFilter `protobuf:"bytes,1,opt,name=filter,proto3" json:"filter,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *QueryFilterCategoriesRequest) Reset() { + *x = QueryFilterCategoriesRequest{} + mi := &file_testdata_service_proto_msgTypes[51] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *QueryFilterCategoriesRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*QueryFilterCategoriesRequest) ProtoMessage() {} + +func (x *QueryFilterCategoriesRequest) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[51] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use QueryFilterCategoriesRequest.ProtoReflect.Descriptor instead. +func (*QueryFilterCategoriesRequest) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{51} +} + +func (x *QueryFilterCategoriesRequest) GetFilter() *CategoryFilter { + if x != nil { + return x.Filter + } + return nil +} + +// Response message for filterCategories operation. +type QueryFilterCategoriesResponse struct { + state protoimpl.MessageState `protogen:"open.v1"` + FilterCategories []*Category `protobuf:"bytes,1,rep,name=filter_categories,json=filterCategories,proto3" json:"filter_categories,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *QueryFilterCategoriesResponse) Reset() { + *x = QueryFilterCategoriesResponse{} + mi := &file_testdata_service_proto_msgTypes[52] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *QueryFilterCategoriesResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*QueryFilterCategoriesResponse) ProtoMessage() {} + +func (x *QueryFilterCategoriesResponse) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[52] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use QueryFilterCategoriesResponse.ProtoReflect.Descriptor instead. +func (*QueryFilterCategoriesResponse) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{52} +} + +func (x *QueryFilterCategoriesResponse) GetFilterCategories() []*Category { + if x != nil { + return x.FilterCategories + } + return nil +} + +// Request message for randomPet operation. +type QueryRandomPetRequest struct { + state protoimpl.MessageState `protogen:"open.v1"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *QueryRandomPetRequest) Reset() { + *x = QueryRandomPetRequest{} + mi := &file_testdata_service_proto_msgTypes[53] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *QueryRandomPetRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*QueryRandomPetRequest) ProtoMessage() {} + +func (x *QueryRandomPetRequest) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[53] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use QueryRandomPetRequest.ProtoReflect.Descriptor instead. +func (*QueryRandomPetRequest) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{53} +} + +// Response message for randomPet operation. +type QueryRandomPetResponse struct { + state protoimpl.MessageState `protogen:"open.v1"` + RandomPet *Animal `protobuf:"bytes,1,opt,name=random_pet,json=randomPet,proto3" json:"random_pet,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *QueryRandomPetResponse) Reset() { + *x = QueryRandomPetResponse{} + mi := &file_testdata_service_proto_msgTypes[54] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *QueryRandomPetResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*QueryRandomPetResponse) ProtoMessage() {} + +func (x *QueryRandomPetResponse) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[54] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use QueryRandomPetResponse.ProtoReflect.Descriptor instead. +func (*QueryRandomPetResponse) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{54} +} + +func (x *QueryRandomPetResponse) GetRandomPet() *Animal { + if x != nil { + return x.RandomPet + } + return nil +} + +// Request message for allPets operation. +type QueryAllPetsRequest struct { + state protoimpl.MessageState `protogen:"open.v1"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *QueryAllPetsRequest) Reset() { + *x = QueryAllPetsRequest{} + mi := &file_testdata_service_proto_msgTypes[55] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *QueryAllPetsRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*QueryAllPetsRequest) ProtoMessage() {} + +func (x *QueryAllPetsRequest) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[55] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use QueryAllPetsRequest.ProtoReflect.Descriptor instead. +func (*QueryAllPetsRequest) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{55} +} + +// Response message for allPets operation. +type QueryAllPetsResponse struct { + state protoimpl.MessageState `protogen:"open.v1"` + AllPets []*Animal `protobuf:"bytes,1,rep,name=all_pets,json=allPets,proto3" json:"all_pets,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *QueryAllPetsResponse) Reset() { + *x = QueryAllPetsResponse{} + mi := &file_testdata_service_proto_msgTypes[56] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *QueryAllPetsResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*QueryAllPetsResponse) ProtoMessage() {} + +func (x *QueryAllPetsResponse) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[56] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use QueryAllPetsResponse.ProtoReflect.Descriptor instead. +func (*QueryAllPetsResponse) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{56} +} + +func (x *QueryAllPetsResponse) GetAllPets() []*Animal { + if x != nil { + return x.AllPets + } + return nil +} + +// Request message for search operation. +type QuerySearchRequest struct { + state protoimpl.MessageState `protogen:"open.v1"` + Input *SearchInput `protobuf:"bytes,1,opt,name=input,proto3" json:"input,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *QuerySearchRequest) Reset() { + *x = QuerySearchRequest{} + mi := &file_testdata_service_proto_msgTypes[57] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *QuerySearchRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*QuerySearchRequest) ProtoMessage() {} + +func (x *QuerySearchRequest) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[57] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use QuerySearchRequest.ProtoReflect.Descriptor instead. +func (*QuerySearchRequest) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{57} +} + +func (x *QuerySearchRequest) GetInput() *SearchInput { + if x != nil { + return x.Input + } + return nil +} + +// Response message for search operation. +type QuerySearchResponse struct { + state protoimpl.MessageState `protogen:"open.v1"` + Search []*SearchResult `protobuf:"bytes,1,rep,name=search,proto3" json:"search,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *QuerySearchResponse) Reset() { + *x = QuerySearchResponse{} + mi := &file_testdata_service_proto_msgTypes[58] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *QuerySearchResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*QuerySearchResponse) ProtoMessage() {} + +func (x *QuerySearchResponse) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[58] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use QuerySearchResponse.ProtoReflect.Descriptor instead. +func (*QuerySearchResponse) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{58} +} + +func (x *QuerySearchResponse) GetSearch() []*SearchResult { + if x != nil { + return x.Search + } + return nil +} + +// Request message for randomSearchResult operation. +type QueryRandomSearchResultRequest struct { + state protoimpl.MessageState `protogen:"open.v1"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *QueryRandomSearchResultRequest) Reset() { + *x = QueryRandomSearchResultRequest{} + mi := &file_testdata_service_proto_msgTypes[59] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *QueryRandomSearchResultRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*QueryRandomSearchResultRequest) ProtoMessage() {} + +func (x *QueryRandomSearchResultRequest) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[59] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use QueryRandomSearchResultRequest.ProtoReflect.Descriptor instead. +func (*QueryRandomSearchResultRequest) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{59} +} + +// Response message for randomSearchResult operation. +type QueryRandomSearchResultResponse struct { + state protoimpl.MessageState `protogen:"open.v1"` + RandomSearchResult *SearchResult `protobuf:"bytes,1,opt,name=random_search_result,json=randomSearchResult,proto3" json:"random_search_result,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *QueryRandomSearchResultResponse) Reset() { + *x = QueryRandomSearchResultResponse{} + mi := &file_testdata_service_proto_msgTypes[60] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *QueryRandomSearchResultResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*QueryRandomSearchResultResponse) ProtoMessage() {} + +func (x *QueryRandomSearchResultResponse) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[60] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use QueryRandomSearchResultResponse.ProtoReflect.Descriptor instead. +func (*QueryRandomSearchResultResponse) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{60} +} + +func (x *QueryRandomSearchResultResponse) GetRandomSearchResult() *SearchResult { + if x != nil { + return x.RandomSearchResult + } + return nil +} + +// Request message for nullableFieldsType operation. +type QueryNullableFieldsTypeRequest struct { + state protoimpl.MessageState `protogen:"open.v1"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *QueryNullableFieldsTypeRequest) Reset() { + *x = QueryNullableFieldsTypeRequest{} + mi := &file_testdata_service_proto_msgTypes[61] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *QueryNullableFieldsTypeRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*QueryNullableFieldsTypeRequest) ProtoMessage() {} + +func (x *QueryNullableFieldsTypeRequest) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[61] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use QueryNullableFieldsTypeRequest.ProtoReflect.Descriptor instead. +func (*QueryNullableFieldsTypeRequest) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{61} +} + +// Response message for nullableFieldsType operation. +type QueryNullableFieldsTypeResponse struct { + state protoimpl.MessageState `protogen:"open.v1"` + NullableFieldsType *NullableFieldsType `protobuf:"bytes,1,opt,name=nullable_fields_type,json=nullableFieldsType,proto3" json:"nullable_fields_type,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *QueryNullableFieldsTypeResponse) Reset() { + *x = QueryNullableFieldsTypeResponse{} + mi := &file_testdata_service_proto_msgTypes[62] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *QueryNullableFieldsTypeResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*QueryNullableFieldsTypeResponse) ProtoMessage() {} + +func (x *QueryNullableFieldsTypeResponse) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[62] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use QueryNullableFieldsTypeResponse.ProtoReflect.Descriptor instead. +func (*QueryNullableFieldsTypeResponse) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{62} +} + +func (x *QueryNullableFieldsTypeResponse) GetNullableFieldsType() *NullableFieldsType { + if x != nil { + return x.NullableFieldsType + } + return nil +} + +// Request message for nullableFieldsTypeById operation. +type QueryNullableFieldsTypeByIdRequest struct { + state protoimpl.MessageState `protogen:"open.v1"` + Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *QueryNullableFieldsTypeByIdRequest) Reset() { + *x = QueryNullableFieldsTypeByIdRequest{} + mi := &file_testdata_service_proto_msgTypes[63] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *QueryNullableFieldsTypeByIdRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*QueryNullableFieldsTypeByIdRequest) ProtoMessage() {} + +func (x *QueryNullableFieldsTypeByIdRequest) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[63] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use QueryNullableFieldsTypeByIdRequest.ProtoReflect.Descriptor instead. +func (*QueryNullableFieldsTypeByIdRequest) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{63} +} + +func (x *QueryNullableFieldsTypeByIdRequest) GetId() string { + if x != nil { + return x.Id + } + return "" +} + +// Response message for nullableFieldsTypeById operation. +type QueryNullableFieldsTypeByIdResponse struct { + state protoimpl.MessageState `protogen:"open.v1"` + NullableFieldsTypeById *NullableFieldsType `protobuf:"bytes,1,opt,name=nullable_fields_type_by_id,json=nullableFieldsTypeById,proto3" json:"nullable_fields_type_by_id,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *QueryNullableFieldsTypeByIdResponse) Reset() { + *x = QueryNullableFieldsTypeByIdResponse{} + mi := &file_testdata_service_proto_msgTypes[64] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *QueryNullableFieldsTypeByIdResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*QueryNullableFieldsTypeByIdResponse) ProtoMessage() {} + +func (x *QueryNullableFieldsTypeByIdResponse) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[64] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use QueryNullableFieldsTypeByIdResponse.ProtoReflect.Descriptor instead. +func (*QueryNullableFieldsTypeByIdResponse) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{64} +} + +func (x *QueryNullableFieldsTypeByIdResponse) GetNullableFieldsTypeById() *NullableFieldsType { + if x != nil { + return x.NullableFieldsTypeById + } + return nil +} + +// Request message for nullableFieldsTypeWithFilter operation. +type QueryNullableFieldsTypeWithFilterRequest struct { + state protoimpl.MessageState `protogen:"open.v1"` + Filter *NullableFieldsFilter `protobuf:"bytes,1,opt,name=filter,proto3" json:"filter,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *QueryNullableFieldsTypeWithFilterRequest) Reset() { + *x = QueryNullableFieldsTypeWithFilterRequest{} + mi := &file_testdata_service_proto_msgTypes[65] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *QueryNullableFieldsTypeWithFilterRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*QueryNullableFieldsTypeWithFilterRequest) ProtoMessage() {} + +func (x *QueryNullableFieldsTypeWithFilterRequest) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[65] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use QueryNullableFieldsTypeWithFilterRequest.ProtoReflect.Descriptor instead. +func (*QueryNullableFieldsTypeWithFilterRequest) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{65} +} + +func (x *QueryNullableFieldsTypeWithFilterRequest) GetFilter() *NullableFieldsFilter { + if x != nil { + return x.Filter + } + return nil +} + +// Response message for nullableFieldsTypeWithFilter operation. +type QueryNullableFieldsTypeWithFilterResponse struct { + state protoimpl.MessageState `protogen:"open.v1"` + NullableFieldsTypeWithFilter []*NullableFieldsType `protobuf:"bytes,1,rep,name=nullable_fields_type_with_filter,json=nullableFieldsTypeWithFilter,proto3" json:"nullable_fields_type_with_filter,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *QueryNullableFieldsTypeWithFilterResponse) Reset() { + *x = QueryNullableFieldsTypeWithFilterResponse{} + mi := &file_testdata_service_proto_msgTypes[66] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *QueryNullableFieldsTypeWithFilterResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*QueryNullableFieldsTypeWithFilterResponse) ProtoMessage() {} + +func (x *QueryNullableFieldsTypeWithFilterResponse) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[66] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use QueryNullableFieldsTypeWithFilterResponse.ProtoReflect.Descriptor instead. +func (*QueryNullableFieldsTypeWithFilterResponse) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{66} +} + +func (x *QueryNullableFieldsTypeWithFilterResponse) GetNullableFieldsTypeWithFilter() []*NullableFieldsType { + if x != nil { + return x.NullableFieldsTypeWithFilter + } + return nil +} + +// Request message for allNullableFieldsTypes operation. +type QueryAllNullableFieldsTypesRequest struct { + state protoimpl.MessageState `protogen:"open.v1"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *QueryAllNullableFieldsTypesRequest) Reset() { + *x = QueryAllNullableFieldsTypesRequest{} + mi := &file_testdata_service_proto_msgTypes[67] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *QueryAllNullableFieldsTypesRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*QueryAllNullableFieldsTypesRequest) ProtoMessage() {} + +func (x *QueryAllNullableFieldsTypesRequest) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[67] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use QueryAllNullableFieldsTypesRequest.ProtoReflect.Descriptor instead. +func (*QueryAllNullableFieldsTypesRequest) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{67} +} + +// Response message for allNullableFieldsTypes operation. +type QueryAllNullableFieldsTypesResponse struct { + state protoimpl.MessageState `protogen:"open.v1"` + AllNullableFieldsTypes []*NullableFieldsType `protobuf:"bytes,1,rep,name=all_nullable_fields_types,json=allNullableFieldsTypes,proto3" json:"all_nullable_fields_types,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *QueryAllNullableFieldsTypesResponse) Reset() { + *x = QueryAllNullableFieldsTypesResponse{} + mi := &file_testdata_service_proto_msgTypes[68] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *QueryAllNullableFieldsTypesResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*QueryAllNullableFieldsTypesResponse) ProtoMessage() {} + +func (x *QueryAllNullableFieldsTypesResponse) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[68] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use QueryAllNullableFieldsTypesResponse.ProtoReflect.Descriptor instead. +func (*QueryAllNullableFieldsTypesResponse) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{68} +} + +func (x *QueryAllNullableFieldsTypesResponse) GetAllNullableFieldsTypes() []*NullableFieldsType { + if x != nil { + return x.AllNullableFieldsTypes + } + return nil +} + +// Request message for blogPost operation. +type QueryBlogPostRequest struct { + state protoimpl.MessageState `protogen:"open.v1"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *QueryBlogPostRequest) Reset() { + *x = QueryBlogPostRequest{} + mi := &file_testdata_service_proto_msgTypes[69] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *QueryBlogPostRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*QueryBlogPostRequest) ProtoMessage() {} + +func (x *QueryBlogPostRequest) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[69] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use QueryBlogPostRequest.ProtoReflect.Descriptor instead. +func (*QueryBlogPostRequest) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{69} +} + +// Response message for blogPost operation. +type QueryBlogPostResponse struct { + state protoimpl.MessageState `protogen:"open.v1"` + BlogPost *BlogPost `protobuf:"bytes,1,opt,name=blog_post,json=blogPost,proto3" json:"blog_post,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *QueryBlogPostResponse) Reset() { + *x = QueryBlogPostResponse{} + mi := &file_testdata_service_proto_msgTypes[70] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *QueryBlogPostResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*QueryBlogPostResponse) ProtoMessage() {} + +func (x *QueryBlogPostResponse) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[70] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use QueryBlogPostResponse.ProtoReflect.Descriptor instead. +func (*QueryBlogPostResponse) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{70} +} + +func (x *QueryBlogPostResponse) GetBlogPost() *BlogPost { + if x != nil { + return x.BlogPost + } + return nil +} + +// Request message for blogPostById operation. +type QueryBlogPostByIdRequest struct { + state protoimpl.MessageState `protogen:"open.v1"` + Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *QueryBlogPostByIdRequest) Reset() { + *x = QueryBlogPostByIdRequest{} + mi := &file_testdata_service_proto_msgTypes[71] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *QueryBlogPostByIdRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*QueryBlogPostByIdRequest) ProtoMessage() {} + +func (x *QueryBlogPostByIdRequest) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[71] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use QueryBlogPostByIdRequest.ProtoReflect.Descriptor instead. +func (*QueryBlogPostByIdRequest) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{71} +} + +func (x *QueryBlogPostByIdRequest) GetId() string { + if x != nil { + return x.Id + } + return "" +} + +// Response message for blogPostById operation. +type QueryBlogPostByIdResponse struct { + state protoimpl.MessageState `protogen:"open.v1"` + BlogPostById *BlogPost `protobuf:"bytes,1,opt,name=blog_post_by_id,json=blogPostById,proto3" json:"blog_post_by_id,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *QueryBlogPostByIdResponse) Reset() { + *x = QueryBlogPostByIdResponse{} + mi := &file_testdata_service_proto_msgTypes[72] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *QueryBlogPostByIdResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*QueryBlogPostByIdResponse) ProtoMessage() {} + +func (x *QueryBlogPostByIdResponse) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[72] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use QueryBlogPostByIdResponse.ProtoReflect.Descriptor instead. +func (*QueryBlogPostByIdResponse) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{72} +} + +func (x *QueryBlogPostByIdResponse) GetBlogPostById() *BlogPost { + if x != nil { + return x.BlogPostById + } + return nil +} + +// Request message for blogPostsWithFilter operation. +type QueryBlogPostsWithFilterRequest struct { + state protoimpl.MessageState `protogen:"open.v1"` + Filter *BlogPostFilter `protobuf:"bytes,1,opt,name=filter,proto3" json:"filter,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *QueryBlogPostsWithFilterRequest) Reset() { + *x = QueryBlogPostsWithFilterRequest{} + mi := &file_testdata_service_proto_msgTypes[73] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *QueryBlogPostsWithFilterRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*QueryBlogPostsWithFilterRequest) ProtoMessage() {} + +func (x *QueryBlogPostsWithFilterRequest) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[73] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use QueryBlogPostsWithFilterRequest.ProtoReflect.Descriptor instead. +func (*QueryBlogPostsWithFilterRequest) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{73} +} + +func (x *QueryBlogPostsWithFilterRequest) GetFilter() *BlogPostFilter { + if x != nil { + return x.Filter + } + return nil +} + +// Response message for blogPostsWithFilter operation. +type QueryBlogPostsWithFilterResponse struct { + state protoimpl.MessageState `protogen:"open.v1"` + BlogPostsWithFilter []*BlogPost `protobuf:"bytes,1,rep,name=blog_posts_with_filter,json=blogPostsWithFilter,proto3" json:"blog_posts_with_filter,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *QueryBlogPostsWithFilterResponse) Reset() { + *x = QueryBlogPostsWithFilterResponse{} + mi := &file_testdata_service_proto_msgTypes[74] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *QueryBlogPostsWithFilterResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*QueryBlogPostsWithFilterResponse) ProtoMessage() {} + +func (x *QueryBlogPostsWithFilterResponse) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[74] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use QueryBlogPostsWithFilterResponse.ProtoReflect.Descriptor instead. +func (*QueryBlogPostsWithFilterResponse) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{74} +} + +func (x *QueryBlogPostsWithFilterResponse) GetBlogPostsWithFilter() []*BlogPost { + if x != nil { + return x.BlogPostsWithFilter + } + return nil +} + +// Request message for allBlogPosts operation. +type QueryAllBlogPostsRequest struct { + state protoimpl.MessageState `protogen:"open.v1"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *QueryAllBlogPostsRequest) Reset() { + *x = QueryAllBlogPostsRequest{} + mi := &file_testdata_service_proto_msgTypes[75] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *QueryAllBlogPostsRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*QueryAllBlogPostsRequest) ProtoMessage() {} + +func (x *QueryAllBlogPostsRequest) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[75] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use QueryAllBlogPostsRequest.ProtoReflect.Descriptor instead. +func (*QueryAllBlogPostsRequest) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{75} +} + +// Response message for allBlogPosts operation. +type QueryAllBlogPostsResponse struct { + state protoimpl.MessageState `protogen:"open.v1"` + AllBlogPosts []*BlogPost `protobuf:"bytes,1,rep,name=all_blog_posts,json=allBlogPosts,proto3" json:"all_blog_posts,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *QueryAllBlogPostsResponse) Reset() { + *x = QueryAllBlogPostsResponse{} + mi := &file_testdata_service_proto_msgTypes[76] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *QueryAllBlogPostsResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*QueryAllBlogPostsResponse) ProtoMessage() {} + +func (x *QueryAllBlogPostsResponse) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[76] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use QueryAllBlogPostsResponse.ProtoReflect.Descriptor instead. +func (*QueryAllBlogPostsResponse) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{76} +} + +func (x *QueryAllBlogPostsResponse) GetAllBlogPosts() []*BlogPost { + if x != nil { + return x.AllBlogPosts + } + return nil +} + +// Request message for author operation. +type QueryAuthorRequest struct { + state protoimpl.MessageState `protogen:"open.v1"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *QueryAuthorRequest) Reset() { + *x = QueryAuthorRequest{} + mi := &file_testdata_service_proto_msgTypes[77] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *QueryAuthorRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*QueryAuthorRequest) ProtoMessage() {} + +func (x *QueryAuthorRequest) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[77] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use QueryAuthorRequest.ProtoReflect.Descriptor instead. +func (*QueryAuthorRequest) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{77} +} + +// Response message for author operation. +type QueryAuthorResponse struct { + state protoimpl.MessageState `protogen:"open.v1"` + Author *Author `protobuf:"bytes,1,opt,name=author,proto3" json:"author,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *QueryAuthorResponse) Reset() { + *x = QueryAuthorResponse{} + mi := &file_testdata_service_proto_msgTypes[78] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *QueryAuthorResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*QueryAuthorResponse) ProtoMessage() {} + +func (x *QueryAuthorResponse) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[78] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use QueryAuthorResponse.ProtoReflect.Descriptor instead. +func (*QueryAuthorResponse) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{78} +} + +func (x *QueryAuthorResponse) GetAuthor() *Author { + if x != nil { + return x.Author + } + return nil +} + +// Request message for authorById operation. +type QueryAuthorByIdRequest struct { + state protoimpl.MessageState `protogen:"open.v1"` + Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *QueryAuthorByIdRequest) Reset() { + *x = QueryAuthorByIdRequest{} + mi := &file_testdata_service_proto_msgTypes[79] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *QueryAuthorByIdRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*QueryAuthorByIdRequest) ProtoMessage() {} + +func (x *QueryAuthorByIdRequest) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[79] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use QueryAuthorByIdRequest.ProtoReflect.Descriptor instead. +func (*QueryAuthorByIdRequest) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{79} +} + +func (x *QueryAuthorByIdRequest) GetId() string { + if x != nil { + return x.Id + } + return "" +} + +// Response message for authorById operation. +type QueryAuthorByIdResponse struct { + state protoimpl.MessageState `protogen:"open.v1"` + AuthorById *Author `protobuf:"bytes,1,opt,name=author_by_id,json=authorById,proto3" json:"author_by_id,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *QueryAuthorByIdResponse) Reset() { + *x = QueryAuthorByIdResponse{} + mi := &file_testdata_service_proto_msgTypes[80] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *QueryAuthorByIdResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*QueryAuthorByIdResponse) ProtoMessage() {} + +func (x *QueryAuthorByIdResponse) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[80] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use QueryAuthorByIdResponse.ProtoReflect.Descriptor instead. +func (*QueryAuthorByIdResponse) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{80} +} + +func (x *QueryAuthorByIdResponse) GetAuthorById() *Author { + if x != nil { + return x.AuthorById + } + return nil +} + +// Request message for authorsWithFilter operation. +type QueryAuthorsWithFilterRequest struct { + state protoimpl.MessageState `protogen:"open.v1"` + Filter *AuthorFilter `protobuf:"bytes,1,opt,name=filter,proto3" json:"filter,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *QueryAuthorsWithFilterRequest) Reset() { + *x = QueryAuthorsWithFilterRequest{} + mi := &file_testdata_service_proto_msgTypes[81] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *QueryAuthorsWithFilterRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*QueryAuthorsWithFilterRequest) ProtoMessage() {} + +func (x *QueryAuthorsWithFilterRequest) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[81] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use QueryAuthorsWithFilterRequest.ProtoReflect.Descriptor instead. +func (*QueryAuthorsWithFilterRequest) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{81} +} + +func (x *QueryAuthorsWithFilterRequest) GetFilter() *AuthorFilter { + if x != nil { + return x.Filter + } + return nil +} + +// Response message for authorsWithFilter operation. +type QueryAuthorsWithFilterResponse struct { + state protoimpl.MessageState `protogen:"open.v1"` + AuthorsWithFilter []*Author `protobuf:"bytes,1,rep,name=authors_with_filter,json=authorsWithFilter,proto3" json:"authors_with_filter,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *QueryAuthorsWithFilterResponse) Reset() { + *x = QueryAuthorsWithFilterResponse{} + mi := &file_testdata_service_proto_msgTypes[82] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *QueryAuthorsWithFilterResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*QueryAuthorsWithFilterResponse) ProtoMessage() {} + +func (x *QueryAuthorsWithFilterResponse) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[82] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use QueryAuthorsWithFilterResponse.ProtoReflect.Descriptor instead. +func (*QueryAuthorsWithFilterResponse) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{82} +} + +func (x *QueryAuthorsWithFilterResponse) GetAuthorsWithFilter() []*Author { + if x != nil { + return x.AuthorsWithFilter + } + return nil +} + +// Request message for allAuthors operation. +type QueryAllAuthorsRequest struct { + state protoimpl.MessageState `protogen:"open.v1"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *QueryAllAuthorsRequest) Reset() { + *x = QueryAllAuthorsRequest{} + mi := &file_testdata_service_proto_msgTypes[83] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *QueryAllAuthorsRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*QueryAllAuthorsRequest) ProtoMessage() {} + +func (x *QueryAllAuthorsRequest) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[83] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use QueryAllAuthorsRequest.ProtoReflect.Descriptor instead. +func (*QueryAllAuthorsRequest) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{83} +} + +// Response message for allAuthors operation. +type QueryAllAuthorsResponse struct { + state protoimpl.MessageState `protogen:"open.v1"` + AllAuthors []*Author `protobuf:"bytes,1,rep,name=all_authors,json=allAuthors,proto3" json:"all_authors,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *QueryAllAuthorsResponse) Reset() { + *x = QueryAllAuthorsResponse{} + mi := &file_testdata_service_proto_msgTypes[84] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *QueryAllAuthorsResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*QueryAllAuthorsResponse) ProtoMessage() {} + +func (x *QueryAllAuthorsResponse) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[84] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use QueryAllAuthorsResponse.ProtoReflect.Descriptor instead. +func (*QueryAllAuthorsResponse) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{84} +} + +func (x *QueryAllAuthorsResponse) GetAllAuthors() []*Author { + if x != nil { + return x.AllAuthors + } + return nil +} + +// Request message for bulkSearchAuthors operation. +type QueryBulkSearchAuthorsRequest struct { + state protoimpl.MessageState `protogen:"open.v1"` + Filters *ListOfAuthorFilter `protobuf:"bytes,1,opt,name=filters,proto3" json:"filters,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *QueryBulkSearchAuthorsRequest) Reset() { + *x = QueryBulkSearchAuthorsRequest{} + mi := &file_testdata_service_proto_msgTypes[85] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *QueryBulkSearchAuthorsRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*QueryBulkSearchAuthorsRequest) ProtoMessage() {} + +func (x *QueryBulkSearchAuthorsRequest) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[85] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use QueryBulkSearchAuthorsRequest.ProtoReflect.Descriptor instead. +func (*QueryBulkSearchAuthorsRequest) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{85} +} + +func (x *QueryBulkSearchAuthorsRequest) GetFilters() *ListOfAuthorFilter { + if x != nil { + return x.Filters + } + return nil +} + +// Response message for bulkSearchAuthors operation. +type QueryBulkSearchAuthorsResponse struct { + state protoimpl.MessageState `protogen:"open.v1"` + BulkSearchAuthors []*Author `protobuf:"bytes,1,rep,name=bulk_search_authors,json=bulkSearchAuthors,proto3" json:"bulk_search_authors,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *QueryBulkSearchAuthorsResponse) Reset() { + *x = QueryBulkSearchAuthorsResponse{} + mi := &file_testdata_service_proto_msgTypes[86] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *QueryBulkSearchAuthorsResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*QueryBulkSearchAuthorsResponse) ProtoMessage() {} + +func (x *QueryBulkSearchAuthorsResponse) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[86] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use QueryBulkSearchAuthorsResponse.ProtoReflect.Descriptor instead. +func (*QueryBulkSearchAuthorsResponse) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{86} +} + +func (x *QueryBulkSearchAuthorsResponse) GetBulkSearchAuthors() []*Author { + if x != nil { + return x.BulkSearchAuthors + } + return nil +} + +// Request message for bulkSearchBlogPosts operation. +type QueryBulkSearchBlogPostsRequest struct { + state protoimpl.MessageState `protogen:"open.v1"` + Filters *ListOfBlogPostFilter `protobuf:"bytes,1,opt,name=filters,proto3" json:"filters,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *QueryBulkSearchBlogPostsRequest) Reset() { + *x = QueryBulkSearchBlogPostsRequest{} + mi := &file_testdata_service_proto_msgTypes[87] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *QueryBulkSearchBlogPostsRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*QueryBulkSearchBlogPostsRequest) ProtoMessage() {} + +func (x *QueryBulkSearchBlogPostsRequest) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[87] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use QueryBulkSearchBlogPostsRequest.ProtoReflect.Descriptor instead. +func (*QueryBulkSearchBlogPostsRequest) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{87} +} + +func (x *QueryBulkSearchBlogPostsRequest) GetFilters() *ListOfBlogPostFilter { + if x != nil { + return x.Filters + } + return nil +} + +// Response message for bulkSearchBlogPosts operation. +type QueryBulkSearchBlogPostsResponse struct { + state protoimpl.MessageState `protogen:"open.v1"` + BulkSearchBlogPosts []*BlogPost `protobuf:"bytes,1,rep,name=bulk_search_blog_posts,json=bulkSearchBlogPosts,proto3" json:"bulk_search_blog_posts,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *QueryBulkSearchBlogPostsResponse) Reset() { + *x = QueryBulkSearchBlogPostsResponse{} + mi := &file_testdata_service_proto_msgTypes[88] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *QueryBulkSearchBlogPostsResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*QueryBulkSearchBlogPostsResponse) ProtoMessage() {} + +func (x *QueryBulkSearchBlogPostsResponse) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[88] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use QueryBulkSearchBlogPostsResponse.ProtoReflect.Descriptor instead. +func (*QueryBulkSearchBlogPostsResponse) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{88} +} + +func (x *QueryBulkSearchBlogPostsResponse) GetBulkSearchBlogPosts() []*BlogPost { + if x != nil { + return x.BulkSearchBlogPosts + } + return nil +} + +// Request message for createUser operation. +type MutationCreateUserRequest struct { + state protoimpl.MessageState `protogen:"open.v1"` + Input *UserInput `protobuf:"bytes,1,opt,name=input,proto3" json:"input,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *MutationCreateUserRequest) Reset() { + *x = MutationCreateUserRequest{} + mi := &file_testdata_service_proto_msgTypes[89] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *MutationCreateUserRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*MutationCreateUserRequest) ProtoMessage() {} + +func (x *MutationCreateUserRequest) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[89] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use MutationCreateUserRequest.ProtoReflect.Descriptor instead. +func (*MutationCreateUserRequest) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{89} +} + +func (x *MutationCreateUserRequest) GetInput() *UserInput { + if x != nil { + return x.Input + } + return nil +} + +// Response message for createUser operation. +type MutationCreateUserResponse struct { + state protoimpl.MessageState `protogen:"open.v1"` + CreateUser *User `protobuf:"bytes,1,opt,name=create_user,json=createUser,proto3" json:"create_user,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *MutationCreateUserResponse) Reset() { + *x = MutationCreateUserResponse{} + mi := &file_testdata_service_proto_msgTypes[90] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *MutationCreateUserResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*MutationCreateUserResponse) ProtoMessage() {} + +func (x *MutationCreateUserResponse) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[90] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use MutationCreateUserResponse.ProtoReflect.Descriptor instead. +func (*MutationCreateUserResponse) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{90} +} + +func (x *MutationCreateUserResponse) GetCreateUser() *User { + if x != nil { + return x.CreateUser + } + return nil +} + +// Request message for performAction operation. +type MutationPerformActionRequest struct { + state protoimpl.MessageState `protogen:"open.v1"` + Input *ActionInput `protobuf:"bytes,1,opt,name=input,proto3" json:"input,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *MutationPerformActionRequest) Reset() { + *x = MutationPerformActionRequest{} + mi := &file_testdata_service_proto_msgTypes[91] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *MutationPerformActionRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*MutationPerformActionRequest) ProtoMessage() {} + +func (x *MutationPerformActionRequest) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[91] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use MutationPerformActionRequest.ProtoReflect.Descriptor instead. +func (*MutationPerformActionRequest) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{91} +} + +func (x *MutationPerformActionRequest) GetInput() *ActionInput { + if x != nil { + return x.Input + } + return nil +} + +// Response message for performAction operation. +type MutationPerformActionResponse struct { + state protoimpl.MessageState `protogen:"open.v1"` + PerformAction *ActionResult `protobuf:"bytes,1,opt,name=perform_action,json=performAction,proto3" json:"perform_action,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *MutationPerformActionResponse) Reset() { + *x = MutationPerformActionResponse{} + mi := &file_testdata_service_proto_msgTypes[92] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *MutationPerformActionResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*MutationPerformActionResponse) ProtoMessage() {} + +func (x *MutationPerformActionResponse) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[92] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use MutationPerformActionResponse.ProtoReflect.Descriptor instead. +func (*MutationPerformActionResponse) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{92} +} + +func (x *MutationPerformActionResponse) GetPerformAction() *ActionResult { + if x != nil { + return x.PerformAction + } + return nil +} + +// Request message for createNullableFieldsType operation. +type MutationCreateNullableFieldsTypeRequest struct { + state protoimpl.MessageState `protogen:"open.v1"` + Input *NullableFieldsInput `protobuf:"bytes,1,opt,name=input,proto3" json:"input,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *MutationCreateNullableFieldsTypeRequest) Reset() { + *x = MutationCreateNullableFieldsTypeRequest{} + mi := &file_testdata_service_proto_msgTypes[93] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *MutationCreateNullableFieldsTypeRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*MutationCreateNullableFieldsTypeRequest) ProtoMessage() {} + +func (x *MutationCreateNullableFieldsTypeRequest) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[93] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use MutationCreateNullableFieldsTypeRequest.ProtoReflect.Descriptor instead. +func (*MutationCreateNullableFieldsTypeRequest) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{93} +} + +func (x *MutationCreateNullableFieldsTypeRequest) GetInput() *NullableFieldsInput { + if x != nil { + return x.Input + } + return nil +} + +// Response message for createNullableFieldsType operation. +type MutationCreateNullableFieldsTypeResponse struct { + state protoimpl.MessageState `protogen:"open.v1"` + CreateNullableFieldsType *NullableFieldsType `protobuf:"bytes,1,opt,name=create_nullable_fields_type,json=createNullableFieldsType,proto3" json:"create_nullable_fields_type,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *MutationCreateNullableFieldsTypeResponse) Reset() { + *x = MutationCreateNullableFieldsTypeResponse{} + mi := &file_testdata_service_proto_msgTypes[94] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *MutationCreateNullableFieldsTypeResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*MutationCreateNullableFieldsTypeResponse) ProtoMessage() {} + +func (x *MutationCreateNullableFieldsTypeResponse) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[94] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use MutationCreateNullableFieldsTypeResponse.ProtoReflect.Descriptor instead. +func (*MutationCreateNullableFieldsTypeResponse) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{94} +} + +func (x *MutationCreateNullableFieldsTypeResponse) GetCreateNullableFieldsType() *NullableFieldsType { + if x != nil { + return x.CreateNullableFieldsType + } + return nil +} + +// Request message for updateNullableFieldsType operation. +type MutationUpdateNullableFieldsTypeRequest struct { + state protoimpl.MessageState `protogen:"open.v1"` + Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` + Input *NullableFieldsInput `protobuf:"bytes,2,opt,name=input,proto3" json:"input,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *MutationUpdateNullableFieldsTypeRequest) Reset() { + *x = MutationUpdateNullableFieldsTypeRequest{} + mi := &file_testdata_service_proto_msgTypes[95] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *MutationUpdateNullableFieldsTypeRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*MutationUpdateNullableFieldsTypeRequest) ProtoMessage() {} + +func (x *MutationUpdateNullableFieldsTypeRequest) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[95] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use MutationUpdateNullableFieldsTypeRequest.ProtoReflect.Descriptor instead. +func (*MutationUpdateNullableFieldsTypeRequest) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{95} +} + +func (x *MutationUpdateNullableFieldsTypeRequest) GetId() string { + if x != nil { + return x.Id + } + return "" +} + +func (x *MutationUpdateNullableFieldsTypeRequest) GetInput() *NullableFieldsInput { + if x != nil { + return x.Input + } + return nil +} + +// Response message for updateNullableFieldsType operation. +type MutationUpdateNullableFieldsTypeResponse struct { + state protoimpl.MessageState `protogen:"open.v1"` + UpdateNullableFieldsType *NullableFieldsType `protobuf:"bytes,1,opt,name=update_nullable_fields_type,json=updateNullableFieldsType,proto3" json:"update_nullable_fields_type,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *MutationUpdateNullableFieldsTypeResponse) Reset() { + *x = MutationUpdateNullableFieldsTypeResponse{} + mi := &file_testdata_service_proto_msgTypes[96] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *MutationUpdateNullableFieldsTypeResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*MutationUpdateNullableFieldsTypeResponse) ProtoMessage() {} + +func (x *MutationUpdateNullableFieldsTypeResponse) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[96] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use MutationUpdateNullableFieldsTypeResponse.ProtoReflect.Descriptor instead. +func (*MutationUpdateNullableFieldsTypeResponse) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{96} +} + +func (x *MutationUpdateNullableFieldsTypeResponse) GetUpdateNullableFieldsType() *NullableFieldsType { + if x != nil { + return x.UpdateNullableFieldsType + } + return nil +} + +// Request message for createBlogPost operation. +type MutationCreateBlogPostRequest struct { + state protoimpl.MessageState `protogen:"open.v1"` + Input *BlogPostInput `protobuf:"bytes,1,opt,name=input,proto3" json:"input,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *MutationCreateBlogPostRequest) Reset() { + *x = MutationCreateBlogPostRequest{} + mi := &file_testdata_service_proto_msgTypes[97] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *MutationCreateBlogPostRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*MutationCreateBlogPostRequest) ProtoMessage() {} + +func (x *MutationCreateBlogPostRequest) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[97] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use MutationCreateBlogPostRequest.ProtoReflect.Descriptor instead. +func (*MutationCreateBlogPostRequest) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{97} +} + +func (x *MutationCreateBlogPostRequest) GetInput() *BlogPostInput { + if x != nil { + return x.Input + } + return nil +} + +// Response message for createBlogPost operation. +type MutationCreateBlogPostResponse struct { + state protoimpl.MessageState `protogen:"open.v1"` + CreateBlogPost *BlogPost `protobuf:"bytes,1,opt,name=create_blog_post,json=createBlogPost,proto3" json:"create_blog_post,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *MutationCreateBlogPostResponse) Reset() { + *x = MutationCreateBlogPostResponse{} + mi := &file_testdata_service_proto_msgTypes[98] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *MutationCreateBlogPostResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*MutationCreateBlogPostResponse) ProtoMessage() {} + +func (x *MutationCreateBlogPostResponse) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[98] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use MutationCreateBlogPostResponse.ProtoReflect.Descriptor instead. +func (*MutationCreateBlogPostResponse) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{98} +} + +func (x *MutationCreateBlogPostResponse) GetCreateBlogPost() *BlogPost { + if x != nil { + return x.CreateBlogPost + } + return nil +} + +// Request message for updateBlogPost operation. +type MutationUpdateBlogPostRequest struct { + state protoimpl.MessageState `protogen:"open.v1"` + Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` + Input *BlogPostInput `protobuf:"bytes,2,opt,name=input,proto3" json:"input,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *MutationUpdateBlogPostRequest) Reset() { + *x = MutationUpdateBlogPostRequest{} + mi := &file_testdata_service_proto_msgTypes[99] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *MutationUpdateBlogPostRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*MutationUpdateBlogPostRequest) ProtoMessage() {} + +func (x *MutationUpdateBlogPostRequest) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[99] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use MutationUpdateBlogPostRequest.ProtoReflect.Descriptor instead. +func (*MutationUpdateBlogPostRequest) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{99} +} + +func (x *MutationUpdateBlogPostRequest) GetId() string { + if x != nil { + return x.Id + } + return "" +} + +func (x *MutationUpdateBlogPostRequest) GetInput() *BlogPostInput { + if x != nil { + return x.Input + } + return nil +} + +// Response message for updateBlogPost operation. +type MutationUpdateBlogPostResponse struct { + state protoimpl.MessageState `protogen:"open.v1"` + UpdateBlogPost *BlogPost `protobuf:"bytes,1,opt,name=update_blog_post,json=updateBlogPost,proto3" json:"update_blog_post,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *MutationUpdateBlogPostResponse) Reset() { + *x = MutationUpdateBlogPostResponse{} + mi := &file_testdata_service_proto_msgTypes[100] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *MutationUpdateBlogPostResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*MutationUpdateBlogPostResponse) ProtoMessage() {} + +func (x *MutationUpdateBlogPostResponse) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[100] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use MutationUpdateBlogPostResponse.ProtoReflect.Descriptor instead. +func (*MutationUpdateBlogPostResponse) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{100} +} + +func (x *MutationUpdateBlogPostResponse) GetUpdateBlogPost() *BlogPost { + if x != nil { + return x.UpdateBlogPost + } + return nil +} + +// Request message for createAuthor operation. +type MutationCreateAuthorRequest struct { + state protoimpl.MessageState `protogen:"open.v1"` + Input *AuthorInput `protobuf:"bytes,1,opt,name=input,proto3" json:"input,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *MutationCreateAuthorRequest) Reset() { + *x = MutationCreateAuthorRequest{} + mi := &file_testdata_service_proto_msgTypes[101] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *MutationCreateAuthorRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*MutationCreateAuthorRequest) ProtoMessage() {} + +func (x *MutationCreateAuthorRequest) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[101] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use MutationCreateAuthorRequest.ProtoReflect.Descriptor instead. +func (*MutationCreateAuthorRequest) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{101} +} + +func (x *MutationCreateAuthorRequest) GetInput() *AuthorInput { + if x != nil { + return x.Input + } + return nil +} + +// Response message for createAuthor operation. +type MutationCreateAuthorResponse struct { + state protoimpl.MessageState `protogen:"open.v1"` + CreateAuthor *Author `protobuf:"bytes,1,opt,name=create_author,json=createAuthor,proto3" json:"create_author,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *MutationCreateAuthorResponse) Reset() { + *x = MutationCreateAuthorResponse{} + mi := &file_testdata_service_proto_msgTypes[102] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *MutationCreateAuthorResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*MutationCreateAuthorResponse) ProtoMessage() {} + +func (x *MutationCreateAuthorResponse) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[102] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use MutationCreateAuthorResponse.ProtoReflect.Descriptor instead. +func (*MutationCreateAuthorResponse) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{102} +} + +func (x *MutationCreateAuthorResponse) GetCreateAuthor() *Author { + if x != nil { + return x.CreateAuthor + } + return nil +} + +// Request message for updateAuthor operation. +type MutationUpdateAuthorRequest struct { + state protoimpl.MessageState `protogen:"open.v1"` + Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` + Input *AuthorInput `protobuf:"bytes,2,opt,name=input,proto3" json:"input,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *MutationUpdateAuthorRequest) Reset() { + *x = MutationUpdateAuthorRequest{} + mi := &file_testdata_service_proto_msgTypes[103] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *MutationUpdateAuthorRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*MutationUpdateAuthorRequest) ProtoMessage() {} + +func (x *MutationUpdateAuthorRequest) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[103] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use MutationUpdateAuthorRequest.ProtoReflect.Descriptor instead. +func (*MutationUpdateAuthorRequest) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{103} +} + +func (x *MutationUpdateAuthorRequest) GetId() string { + if x != nil { + return x.Id + } + return "" +} + +func (x *MutationUpdateAuthorRequest) GetInput() *AuthorInput { + if x != nil { + return x.Input + } + return nil +} + +// Response message for updateAuthor operation. +type MutationUpdateAuthorResponse struct { + state protoimpl.MessageState `protogen:"open.v1"` + UpdateAuthor *Author `protobuf:"bytes,1,opt,name=update_author,json=updateAuthor,proto3" json:"update_author,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *MutationUpdateAuthorResponse) Reset() { + *x = MutationUpdateAuthorResponse{} + mi := &file_testdata_service_proto_msgTypes[104] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *MutationUpdateAuthorResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*MutationUpdateAuthorResponse) ProtoMessage() {} + +func (x *MutationUpdateAuthorResponse) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[104] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use MutationUpdateAuthorResponse.ProtoReflect.Descriptor instead. +func (*MutationUpdateAuthorResponse) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{104} +} + +func (x *MutationUpdateAuthorResponse) GetUpdateAuthor() *Author { + if x != nil { + return x.UpdateAuthor + } + return nil +} + +// Request message for bulkCreateAuthors operation. +type MutationBulkCreateAuthorsRequest struct { + state protoimpl.MessageState `protogen:"open.v1"` + Authors *ListOfAuthorInput `protobuf:"bytes,1,opt,name=authors,proto3" json:"authors,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *MutationBulkCreateAuthorsRequest) Reset() { + *x = MutationBulkCreateAuthorsRequest{} + mi := &file_testdata_service_proto_msgTypes[105] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *MutationBulkCreateAuthorsRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*MutationBulkCreateAuthorsRequest) ProtoMessage() {} + +func (x *MutationBulkCreateAuthorsRequest) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[105] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use MutationBulkCreateAuthorsRequest.ProtoReflect.Descriptor instead. +func (*MutationBulkCreateAuthorsRequest) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{105} +} + +func (x *MutationBulkCreateAuthorsRequest) GetAuthors() *ListOfAuthorInput { + if x != nil { + return x.Authors + } + return nil +} + +// Response message for bulkCreateAuthors operation. +type MutationBulkCreateAuthorsResponse struct { + state protoimpl.MessageState `protogen:"open.v1"` + BulkCreateAuthors []*Author `protobuf:"bytes,1,rep,name=bulk_create_authors,json=bulkCreateAuthors,proto3" json:"bulk_create_authors,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *MutationBulkCreateAuthorsResponse) Reset() { + *x = MutationBulkCreateAuthorsResponse{} + mi := &file_testdata_service_proto_msgTypes[106] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *MutationBulkCreateAuthorsResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*MutationBulkCreateAuthorsResponse) ProtoMessage() {} + +func (x *MutationBulkCreateAuthorsResponse) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[106] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use MutationBulkCreateAuthorsResponse.ProtoReflect.Descriptor instead. +func (*MutationBulkCreateAuthorsResponse) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{106} +} + +func (x *MutationBulkCreateAuthorsResponse) GetBulkCreateAuthors() []*Author { + if x != nil { + return x.BulkCreateAuthors + } + return nil +} + +// Request message for bulkUpdateAuthors operation. +type MutationBulkUpdateAuthorsRequest struct { + state protoimpl.MessageState `protogen:"open.v1"` + Authors *ListOfAuthorInput `protobuf:"bytes,1,opt,name=authors,proto3" json:"authors,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *MutationBulkUpdateAuthorsRequest) Reset() { + *x = MutationBulkUpdateAuthorsRequest{} + mi := &file_testdata_service_proto_msgTypes[107] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *MutationBulkUpdateAuthorsRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*MutationBulkUpdateAuthorsRequest) ProtoMessage() {} + +func (x *MutationBulkUpdateAuthorsRequest) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[107] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use MutationBulkUpdateAuthorsRequest.ProtoReflect.Descriptor instead. +func (*MutationBulkUpdateAuthorsRequest) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{107} +} + +func (x *MutationBulkUpdateAuthorsRequest) GetAuthors() *ListOfAuthorInput { + if x != nil { + return x.Authors + } + return nil +} + +// Response message for bulkUpdateAuthors operation. +type MutationBulkUpdateAuthorsResponse struct { + state protoimpl.MessageState `protogen:"open.v1"` + BulkUpdateAuthors []*Author `protobuf:"bytes,1,rep,name=bulk_update_authors,json=bulkUpdateAuthors,proto3" json:"bulk_update_authors,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *MutationBulkUpdateAuthorsResponse) Reset() { + *x = MutationBulkUpdateAuthorsResponse{} + mi := &file_testdata_service_proto_msgTypes[108] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *MutationBulkUpdateAuthorsResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*MutationBulkUpdateAuthorsResponse) ProtoMessage() {} + +func (x *MutationBulkUpdateAuthorsResponse) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[108] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use MutationBulkUpdateAuthorsResponse.ProtoReflect.Descriptor instead. +func (*MutationBulkUpdateAuthorsResponse) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{108} +} + +func (x *MutationBulkUpdateAuthorsResponse) GetBulkUpdateAuthors() []*Author { + if x != nil { + return x.BulkUpdateAuthors + } + return nil +} + +// Request message for bulkCreateBlogPosts operation. +type MutationBulkCreateBlogPostsRequest struct { + state protoimpl.MessageState `protogen:"open.v1"` + BlogPosts *ListOfBlogPostInput `protobuf:"bytes,1,opt,name=blog_posts,json=blogPosts,proto3" json:"blog_posts,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *MutationBulkCreateBlogPostsRequest) Reset() { + *x = MutationBulkCreateBlogPostsRequest{} + mi := &file_testdata_service_proto_msgTypes[109] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *MutationBulkCreateBlogPostsRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*MutationBulkCreateBlogPostsRequest) ProtoMessage() {} + +func (x *MutationBulkCreateBlogPostsRequest) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[109] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use MutationBulkCreateBlogPostsRequest.ProtoReflect.Descriptor instead. +func (*MutationBulkCreateBlogPostsRequest) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{109} +} + +func (x *MutationBulkCreateBlogPostsRequest) GetBlogPosts() *ListOfBlogPostInput { + if x != nil { + return x.BlogPosts + } + return nil +} + +// Response message for bulkCreateBlogPosts operation. +type MutationBulkCreateBlogPostsResponse struct { + state protoimpl.MessageState `protogen:"open.v1"` + BulkCreateBlogPosts []*BlogPost `protobuf:"bytes,1,rep,name=bulk_create_blog_posts,json=bulkCreateBlogPosts,proto3" json:"bulk_create_blog_posts,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *MutationBulkCreateBlogPostsResponse) Reset() { + *x = MutationBulkCreateBlogPostsResponse{} + mi := &file_testdata_service_proto_msgTypes[110] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *MutationBulkCreateBlogPostsResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*MutationBulkCreateBlogPostsResponse) ProtoMessage() {} + +func (x *MutationBulkCreateBlogPostsResponse) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[110] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use MutationBulkCreateBlogPostsResponse.ProtoReflect.Descriptor instead. +func (*MutationBulkCreateBlogPostsResponse) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{110} +} + +func (x *MutationBulkCreateBlogPostsResponse) GetBulkCreateBlogPosts() []*BlogPost { + if x != nil { + return x.BulkCreateBlogPosts + } + return nil +} + +// Request message for bulkUpdateBlogPosts operation. +type MutationBulkUpdateBlogPostsRequest struct { + state protoimpl.MessageState `protogen:"open.v1"` + BlogPosts *ListOfBlogPostInput `protobuf:"bytes,1,opt,name=blog_posts,json=blogPosts,proto3" json:"blog_posts,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *MutationBulkUpdateBlogPostsRequest) Reset() { + *x = MutationBulkUpdateBlogPostsRequest{} + mi := &file_testdata_service_proto_msgTypes[111] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *MutationBulkUpdateBlogPostsRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*MutationBulkUpdateBlogPostsRequest) ProtoMessage() {} + +func (x *MutationBulkUpdateBlogPostsRequest) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[111] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use MutationBulkUpdateBlogPostsRequest.ProtoReflect.Descriptor instead. +func (*MutationBulkUpdateBlogPostsRequest) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{111} +} + +func (x *MutationBulkUpdateBlogPostsRequest) GetBlogPosts() *ListOfBlogPostInput { + if x != nil { + return x.BlogPosts + } + return nil +} + +// Response message for bulkUpdateBlogPosts operation. +type MutationBulkUpdateBlogPostsResponse struct { + state protoimpl.MessageState `protogen:"open.v1"` + BulkUpdateBlogPosts []*BlogPost `protobuf:"bytes,1,rep,name=bulk_update_blog_posts,json=bulkUpdateBlogPosts,proto3" json:"bulk_update_blog_posts,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *MutationBulkUpdateBlogPostsResponse) Reset() { + *x = MutationBulkUpdateBlogPostsResponse{} + mi := &file_testdata_service_proto_msgTypes[112] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *MutationBulkUpdateBlogPostsResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*MutationBulkUpdateBlogPostsResponse) ProtoMessage() {} + +func (x *MutationBulkUpdateBlogPostsResponse) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[112] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use MutationBulkUpdateBlogPostsResponse.ProtoReflect.Descriptor instead. +func (*MutationBulkUpdateBlogPostsResponse) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{112} +} + +func (x *MutationBulkUpdateBlogPostsResponse) GetBulkUpdateBlogPosts() []*BlogPost { + if x != nil { + return x.BulkUpdateBlogPosts + } + return nil +} + +type ResolveProductShippingEstimateArgs struct { + state protoimpl.MessageState `protogen:"open.v1"` + Input *ShippingEstimateInput `protobuf:"bytes,1,opt,name=input,proto3" json:"input,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ResolveProductShippingEstimateArgs) Reset() { + *x = ResolveProductShippingEstimateArgs{} + mi := &file_testdata_service_proto_msgTypes[113] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ResolveProductShippingEstimateArgs) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ResolveProductShippingEstimateArgs) ProtoMessage() {} + +func (x *ResolveProductShippingEstimateArgs) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[113] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ResolveProductShippingEstimateArgs.ProtoReflect.Descriptor instead. +func (*ResolveProductShippingEstimateArgs) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{113} +} + +func (x *ResolveProductShippingEstimateArgs) GetInput() *ShippingEstimateInput { + if x != nil { + return x.Input + } + return nil +} + +type ResolveProductShippingEstimateContext struct { + state protoimpl.MessageState `protogen:"open.v1"` + Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` + Price float64 `protobuf:"fixed64,2,opt,name=price,proto3" json:"price,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ResolveProductShippingEstimateContext) Reset() { + *x = ResolveProductShippingEstimateContext{} + mi := &file_testdata_service_proto_msgTypes[114] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ResolveProductShippingEstimateContext) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ResolveProductShippingEstimateContext) ProtoMessage() {} + +func (x *ResolveProductShippingEstimateContext) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[114] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ResolveProductShippingEstimateContext.ProtoReflect.Descriptor instead. +func (*ResolveProductShippingEstimateContext) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{114} +} + +func (x *ResolveProductShippingEstimateContext) GetId() string { + if x != nil { + return x.Id + } + return "" +} + +func (x *ResolveProductShippingEstimateContext) GetPrice() float64 { + if x != nil { + return x.Price + } + return 0 +} + +type ResolveProductShippingEstimateRequest struct { + state protoimpl.MessageState `protogen:"open.v1"` + // context provides the resolver context for the field shippingEstimate of type Product. + Context []*ResolveProductShippingEstimateContext `protobuf:"bytes,1,rep,name=context,proto3" json:"context,omitempty"` + // field_args provides the arguments for the resolver field shippingEstimate of type Product. + FieldArgs *ResolveProductShippingEstimateArgs `protobuf:"bytes,2,opt,name=field_args,json=fieldArgs,proto3" json:"field_args,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ResolveProductShippingEstimateRequest) Reset() { + *x = ResolveProductShippingEstimateRequest{} + mi := &file_testdata_service_proto_msgTypes[115] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ResolveProductShippingEstimateRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ResolveProductShippingEstimateRequest) ProtoMessage() {} + +func (x *ResolveProductShippingEstimateRequest) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[115] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ResolveProductShippingEstimateRequest.ProtoReflect.Descriptor instead. +func (*ResolveProductShippingEstimateRequest) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{115} +} + +func (x *ResolveProductShippingEstimateRequest) GetContext() []*ResolveProductShippingEstimateContext { + if x != nil { + return x.Context + } + return nil +} + +func (x *ResolveProductShippingEstimateRequest) GetFieldArgs() *ResolveProductShippingEstimateArgs { + if x != nil { + return x.FieldArgs + } + return nil +} + +type ResolveProductShippingEstimateResult struct { + state protoimpl.MessageState `protogen:"open.v1"` + ShippingEstimate float64 `protobuf:"fixed64,1,opt,name=shipping_estimate,json=shippingEstimate,proto3" json:"shipping_estimate,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ResolveProductShippingEstimateResult) Reset() { + *x = ResolveProductShippingEstimateResult{} + mi := &file_testdata_service_proto_msgTypes[116] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ResolveProductShippingEstimateResult) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ResolveProductShippingEstimateResult) ProtoMessage() {} + +func (x *ResolveProductShippingEstimateResult) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[116] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ResolveProductShippingEstimateResult.ProtoReflect.Descriptor instead. +func (*ResolveProductShippingEstimateResult) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{116} +} + +func (x *ResolveProductShippingEstimateResult) GetShippingEstimate() float64 { + if x != nil { + return x.ShippingEstimate + } + return 0 +} + +type ResolveProductShippingEstimateResponse struct { + state protoimpl.MessageState `protogen:"open.v1"` + Result []*ResolveProductShippingEstimateResult `protobuf:"bytes,1,rep,name=result,proto3" json:"result,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ResolveProductShippingEstimateResponse) Reset() { + *x = ResolveProductShippingEstimateResponse{} + mi := &file_testdata_service_proto_msgTypes[117] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ResolveProductShippingEstimateResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ResolveProductShippingEstimateResponse) ProtoMessage() {} + +func (x *ResolveProductShippingEstimateResponse) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[117] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ResolveProductShippingEstimateResponse.ProtoReflect.Descriptor instead. +func (*ResolveProductShippingEstimateResponse) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{117} +} + +func (x *ResolveProductShippingEstimateResponse) GetResult() []*ResolveProductShippingEstimateResult { + if x != nil { + return x.Result + } + return nil +} + +type ResolveProductRecommendedCategoryArgs struct { + state protoimpl.MessageState `protogen:"open.v1"` + MaxPrice int32 `protobuf:"varint,1,opt,name=max_price,json=maxPrice,proto3" json:"max_price,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ResolveProductRecommendedCategoryArgs) Reset() { + *x = ResolveProductRecommendedCategoryArgs{} + mi := &file_testdata_service_proto_msgTypes[118] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ResolveProductRecommendedCategoryArgs) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ResolveProductRecommendedCategoryArgs) ProtoMessage() {} + +func (x *ResolveProductRecommendedCategoryArgs) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[118] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ResolveProductRecommendedCategoryArgs.ProtoReflect.Descriptor instead. +func (*ResolveProductRecommendedCategoryArgs) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{118} +} + +func (x *ResolveProductRecommendedCategoryArgs) GetMaxPrice() int32 { + if x != nil { + return x.MaxPrice + } + return 0 +} + +type ResolveProductRecommendedCategoryContext struct { + state protoimpl.MessageState `protogen:"open.v1"` + Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` + Name string `protobuf:"bytes,2,opt,name=name,proto3" json:"name,omitempty"` + Price float64 `protobuf:"fixed64,3,opt,name=price,proto3" json:"price,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ResolveProductRecommendedCategoryContext) Reset() { + *x = ResolveProductRecommendedCategoryContext{} + mi := &file_testdata_service_proto_msgTypes[119] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ResolveProductRecommendedCategoryContext) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ResolveProductRecommendedCategoryContext) ProtoMessage() {} + +func (x *ResolveProductRecommendedCategoryContext) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[119] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ResolveProductRecommendedCategoryContext.ProtoReflect.Descriptor instead. +func (*ResolveProductRecommendedCategoryContext) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{119} +} + +func (x *ResolveProductRecommendedCategoryContext) GetId() string { + if x != nil { + return x.Id + } + return "" +} + +func (x *ResolveProductRecommendedCategoryContext) GetName() string { + if x != nil { + return x.Name + } + return "" +} + +func (x *ResolveProductRecommendedCategoryContext) GetPrice() float64 { + if x != nil { + return x.Price + } + return 0 +} + +type ResolveProductRecommendedCategoryRequest struct { + state protoimpl.MessageState `protogen:"open.v1"` + // context provides the resolver context for the field recommendedCategory of type Product. + Context []*ResolveProductRecommendedCategoryContext `protobuf:"bytes,1,rep,name=context,proto3" json:"context,omitempty"` + // field_args provides the arguments for the resolver field recommendedCategory of type Product. + FieldArgs *ResolveProductRecommendedCategoryArgs `protobuf:"bytes,2,opt,name=field_args,json=fieldArgs,proto3" json:"field_args,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ResolveProductRecommendedCategoryRequest) Reset() { + *x = ResolveProductRecommendedCategoryRequest{} + mi := &file_testdata_service_proto_msgTypes[120] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ResolveProductRecommendedCategoryRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ResolveProductRecommendedCategoryRequest) ProtoMessage() {} + +func (x *ResolveProductRecommendedCategoryRequest) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[120] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ResolveProductRecommendedCategoryRequest.ProtoReflect.Descriptor instead. +func (*ResolveProductRecommendedCategoryRequest) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{120} +} + +func (x *ResolveProductRecommendedCategoryRequest) GetContext() []*ResolveProductRecommendedCategoryContext { + if x != nil { + return x.Context + } + return nil +} + +func (x *ResolveProductRecommendedCategoryRequest) GetFieldArgs() *ResolveProductRecommendedCategoryArgs { + if x != nil { + return x.FieldArgs + } + return nil +} + +type ResolveProductRecommendedCategoryResult struct { + state protoimpl.MessageState `protogen:"open.v1"` + RecommendedCategory *Category `protobuf:"bytes,1,opt,name=recommended_category,json=recommendedCategory,proto3" json:"recommended_category,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ResolveProductRecommendedCategoryResult) Reset() { + *x = ResolveProductRecommendedCategoryResult{} + mi := &file_testdata_service_proto_msgTypes[121] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ResolveProductRecommendedCategoryResult) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ResolveProductRecommendedCategoryResult) ProtoMessage() {} + +func (x *ResolveProductRecommendedCategoryResult) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[121] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ResolveProductRecommendedCategoryResult.ProtoReflect.Descriptor instead. +func (*ResolveProductRecommendedCategoryResult) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{121} +} + +func (x *ResolveProductRecommendedCategoryResult) GetRecommendedCategory() *Category { + if x != nil { + return x.RecommendedCategory + } + return nil +} + +type ResolveProductRecommendedCategoryResponse struct { + state protoimpl.MessageState `protogen:"open.v1"` + Result []*ResolveProductRecommendedCategoryResult `protobuf:"bytes,1,rep,name=result,proto3" json:"result,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ResolveProductRecommendedCategoryResponse) Reset() { + *x = ResolveProductRecommendedCategoryResponse{} + mi := &file_testdata_service_proto_msgTypes[122] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ResolveProductRecommendedCategoryResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ResolveProductRecommendedCategoryResponse) ProtoMessage() {} + +func (x *ResolveProductRecommendedCategoryResponse) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[122] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ResolveProductRecommendedCategoryResponse.ProtoReflect.Descriptor instead. +func (*ResolveProductRecommendedCategoryResponse) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{122} +} + +func (x *ResolveProductRecommendedCategoryResponse) GetResult() []*ResolveProductRecommendedCategoryResult { + if x != nil { + return x.Result + } + return nil +} + +type ResolveCategoryProductCountArgs struct { + state protoimpl.MessageState `protogen:"open.v1"` + Filters *ProductCountFilter `protobuf:"bytes,1,opt,name=filters,proto3" json:"filters,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ResolveCategoryProductCountArgs) Reset() { + *x = ResolveCategoryProductCountArgs{} + mi := &file_testdata_service_proto_msgTypes[123] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ResolveCategoryProductCountArgs) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ResolveCategoryProductCountArgs) ProtoMessage() {} + +func (x *ResolveCategoryProductCountArgs) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[123] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ResolveCategoryProductCountArgs.ProtoReflect.Descriptor instead. +func (*ResolveCategoryProductCountArgs) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{123} +} + +func (x *ResolveCategoryProductCountArgs) GetFilters() *ProductCountFilter { + if x != nil { + return x.Filters + } + return nil +} + +type ResolveCategoryProductCountContext struct { + state protoimpl.MessageState `protogen:"open.v1"` + Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` + Name string `protobuf:"bytes,2,opt,name=name,proto3" json:"name,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ResolveCategoryProductCountContext) Reset() { + *x = ResolveCategoryProductCountContext{} + mi := &file_testdata_service_proto_msgTypes[124] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ResolveCategoryProductCountContext) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ResolveCategoryProductCountContext) ProtoMessage() {} + +func (x *ResolveCategoryProductCountContext) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[124] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ResolveCategoryProductCountContext.ProtoReflect.Descriptor instead. +func (*ResolveCategoryProductCountContext) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{124} +} + +func (x *ResolveCategoryProductCountContext) GetId() string { + if x != nil { + return x.Id + } + return "" +} + +func (x *ResolveCategoryProductCountContext) GetName() string { + if x != nil { + return x.Name + } + return "" +} + +type ResolveCategoryProductCountRequest struct { + state protoimpl.MessageState `protogen:"open.v1"` + // context provides the resolver context for the field productCount of type Category. + Context []*ResolveCategoryProductCountContext `protobuf:"bytes,1,rep,name=context,proto3" json:"context,omitempty"` + // field_args provides the arguments for the resolver field productCount of type Category. + FieldArgs *ResolveCategoryProductCountArgs `protobuf:"bytes,2,opt,name=field_args,json=fieldArgs,proto3" json:"field_args,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ResolveCategoryProductCountRequest) Reset() { + *x = ResolveCategoryProductCountRequest{} + mi := &file_testdata_service_proto_msgTypes[125] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ResolveCategoryProductCountRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ResolveCategoryProductCountRequest) ProtoMessage() {} + +func (x *ResolveCategoryProductCountRequest) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[125] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ResolveCategoryProductCountRequest.ProtoReflect.Descriptor instead. +func (*ResolveCategoryProductCountRequest) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{125} +} + +func (x *ResolveCategoryProductCountRequest) GetContext() []*ResolveCategoryProductCountContext { + if x != nil { + return x.Context + } + return nil +} + +func (x *ResolveCategoryProductCountRequest) GetFieldArgs() *ResolveCategoryProductCountArgs { + if x != nil { + return x.FieldArgs + } + return nil +} + +type ResolveCategoryProductCountResult struct { + state protoimpl.MessageState `protogen:"open.v1"` + ProductCount int32 `protobuf:"varint,1,opt,name=product_count,json=productCount,proto3" json:"product_count,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ResolveCategoryProductCountResult) Reset() { + *x = ResolveCategoryProductCountResult{} + mi := &file_testdata_service_proto_msgTypes[126] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ResolveCategoryProductCountResult) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ResolveCategoryProductCountResult) ProtoMessage() {} + +func (x *ResolveCategoryProductCountResult) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[126] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ResolveCategoryProductCountResult.ProtoReflect.Descriptor instead. +func (*ResolveCategoryProductCountResult) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{126} +} + +func (x *ResolveCategoryProductCountResult) GetProductCount() int32 { + if x != nil { + return x.ProductCount + } + return 0 +} + +type ResolveCategoryProductCountResponse struct { + state protoimpl.MessageState `protogen:"open.v1"` + Result []*ResolveCategoryProductCountResult `protobuf:"bytes,1,rep,name=result,proto3" json:"result,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ResolveCategoryProductCountResponse) Reset() { + *x = ResolveCategoryProductCountResponse{} + mi := &file_testdata_service_proto_msgTypes[127] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ResolveCategoryProductCountResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ResolveCategoryProductCountResponse) ProtoMessage() {} + +func (x *ResolveCategoryProductCountResponse) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[127] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ResolveCategoryProductCountResponse.ProtoReflect.Descriptor instead. +func (*ResolveCategoryProductCountResponse) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{127} +} + +func (x *ResolveCategoryProductCountResponse) GetResult() []*ResolveCategoryProductCountResult { + if x != nil { + return x.Result + } + return nil +} + +type ResolveCategoryPopularityScoreArgs struct { + state protoimpl.MessageState `protogen:"open.v1"` + Threshold *wrapperspb.Int32Value `protobuf:"bytes,1,opt,name=threshold,proto3" json:"threshold,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ResolveCategoryPopularityScoreArgs) Reset() { + *x = ResolveCategoryPopularityScoreArgs{} + mi := &file_testdata_service_proto_msgTypes[128] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ResolveCategoryPopularityScoreArgs) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ResolveCategoryPopularityScoreArgs) ProtoMessage() {} + +func (x *ResolveCategoryPopularityScoreArgs) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[128] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ResolveCategoryPopularityScoreArgs.ProtoReflect.Descriptor instead. +func (*ResolveCategoryPopularityScoreArgs) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{128} +} + +func (x *ResolveCategoryPopularityScoreArgs) GetThreshold() *wrapperspb.Int32Value { + if x != nil { + return x.Threshold + } + return nil +} + +type ResolveCategoryPopularityScoreContext struct { + state protoimpl.MessageState `protogen:"open.v1"` + Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ResolveCategoryPopularityScoreContext) Reset() { + *x = ResolveCategoryPopularityScoreContext{} + mi := &file_testdata_service_proto_msgTypes[129] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ResolveCategoryPopularityScoreContext) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ResolveCategoryPopularityScoreContext) ProtoMessage() {} + +func (x *ResolveCategoryPopularityScoreContext) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[129] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ResolveCategoryPopularityScoreContext.ProtoReflect.Descriptor instead. +func (*ResolveCategoryPopularityScoreContext) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{129} +} + +func (x *ResolveCategoryPopularityScoreContext) GetId() string { + if x != nil { + return x.Id + } + return "" +} + +type ResolveCategoryPopularityScoreRequest struct { + state protoimpl.MessageState `protogen:"open.v1"` + // context provides the resolver context for the field popularityScore of type Category. + Context []*ResolveCategoryPopularityScoreContext `protobuf:"bytes,1,rep,name=context,proto3" json:"context,omitempty"` + // field_args provides the arguments for the resolver field popularityScore of type Category. + FieldArgs *ResolveCategoryPopularityScoreArgs `protobuf:"bytes,2,opt,name=field_args,json=fieldArgs,proto3" json:"field_args,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ResolveCategoryPopularityScoreRequest) Reset() { + *x = ResolveCategoryPopularityScoreRequest{} + mi := &file_testdata_service_proto_msgTypes[130] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ResolveCategoryPopularityScoreRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ResolveCategoryPopularityScoreRequest) ProtoMessage() {} + +func (x *ResolveCategoryPopularityScoreRequest) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[130] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ResolveCategoryPopularityScoreRequest.ProtoReflect.Descriptor instead. +func (*ResolveCategoryPopularityScoreRequest) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{130} +} + +func (x *ResolveCategoryPopularityScoreRequest) GetContext() []*ResolveCategoryPopularityScoreContext { + if x != nil { + return x.Context + } + return nil +} + +func (x *ResolveCategoryPopularityScoreRequest) GetFieldArgs() *ResolveCategoryPopularityScoreArgs { + if x != nil { + return x.FieldArgs + } + return nil +} + +type ResolveCategoryPopularityScoreResult struct { + state protoimpl.MessageState `protogen:"open.v1"` + PopularityScore *wrapperspb.Int32Value `protobuf:"bytes,1,opt,name=popularity_score,json=popularityScore,proto3" json:"popularity_score,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ResolveCategoryPopularityScoreResult) Reset() { + *x = ResolveCategoryPopularityScoreResult{} + mi := &file_testdata_service_proto_msgTypes[131] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ResolveCategoryPopularityScoreResult) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ResolveCategoryPopularityScoreResult) ProtoMessage() {} + +func (x *ResolveCategoryPopularityScoreResult) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[131] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ResolveCategoryPopularityScoreResult.ProtoReflect.Descriptor instead. +func (*ResolveCategoryPopularityScoreResult) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{131} +} + +func (x *ResolveCategoryPopularityScoreResult) GetPopularityScore() *wrapperspb.Int32Value { + if x != nil { + return x.PopularityScore + } + return nil +} + +type ResolveCategoryPopularityScoreResponse struct { + state protoimpl.MessageState `protogen:"open.v1"` + Result []*ResolveCategoryPopularityScoreResult `protobuf:"bytes,1,rep,name=result,proto3" json:"result,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ResolveCategoryPopularityScoreResponse) Reset() { + *x = ResolveCategoryPopularityScoreResponse{} + mi := &file_testdata_service_proto_msgTypes[132] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ResolveCategoryPopularityScoreResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ResolveCategoryPopularityScoreResponse) ProtoMessage() {} + +func (x *ResolveCategoryPopularityScoreResponse) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[132] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ResolveCategoryPopularityScoreResponse.ProtoReflect.Descriptor instead. +func (*ResolveCategoryPopularityScoreResponse) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{132} +} + +func (x *ResolveCategoryPopularityScoreResponse) GetResult() []*ResolveCategoryPopularityScoreResult { + if x != nil { + return x.Result + } + return nil +} + +type ResolveCategoryCategoryMetricsArgs struct { + state protoimpl.MessageState `protogen:"open.v1"` + MetricType string `protobuf:"bytes,1,opt,name=metric_type,json=metricType,proto3" json:"metric_type,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ResolveCategoryCategoryMetricsArgs) Reset() { + *x = ResolveCategoryCategoryMetricsArgs{} + mi := &file_testdata_service_proto_msgTypes[133] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ResolveCategoryCategoryMetricsArgs) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ResolveCategoryCategoryMetricsArgs) ProtoMessage() {} + +func (x *ResolveCategoryCategoryMetricsArgs) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[133] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ResolveCategoryCategoryMetricsArgs.ProtoReflect.Descriptor instead. +func (*ResolveCategoryCategoryMetricsArgs) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{133} +} + +func (x *ResolveCategoryCategoryMetricsArgs) GetMetricType() string { + if x != nil { + return x.MetricType + } + return "" +} + +type ResolveCategoryCategoryMetricsContext struct { + state protoimpl.MessageState `protogen:"open.v1"` + Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` + Name string `protobuf:"bytes,2,opt,name=name,proto3" json:"name,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ResolveCategoryCategoryMetricsContext) Reset() { + *x = ResolveCategoryCategoryMetricsContext{} + mi := &file_testdata_service_proto_msgTypes[134] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ResolveCategoryCategoryMetricsContext) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ResolveCategoryCategoryMetricsContext) ProtoMessage() {} + +func (x *ResolveCategoryCategoryMetricsContext) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[134] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ResolveCategoryCategoryMetricsContext.ProtoReflect.Descriptor instead. +func (*ResolveCategoryCategoryMetricsContext) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{134} +} + +func (x *ResolveCategoryCategoryMetricsContext) GetId() string { + if x != nil { + return x.Id + } + return "" +} + +func (x *ResolveCategoryCategoryMetricsContext) GetName() string { + if x != nil { + return x.Name + } + return "" +} + +type ResolveCategoryCategoryMetricsRequest struct { + state protoimpl.MessageState `protogen:"open.v1"` + // context provides the resolver context for the field categoryMetrics of type Category. + Context []*ResolveCategoryCategoryMetricsContext `protobuf:"bytes,1,rep,name=context,proto3" json:"context,omitempty"` + // field_args provides the arguments for the resolver field categoryMetrics of type Category. + FieldArgs *ResolveCategoryCategoryMetricsArgs `protobuf:"bytes,2,opt,name=field_args,json=fieldArgs,proto3" json:"field_args,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ResolveCategoryCategoryMetricsRequest) Reset() { + *x = ResolveCategoryCategoryMetricsRequest{} + mi := &file_testdata_service_proto_msgTypes[135] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ResolveCategoryCategoryMetricsRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ResolveCategoryCategoryMetricsRequest) ProtoMessage() {} + +func (x *ResolveCategoryCategoryMetricsRequest) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[135] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ResolveCategoryCategoryMetricsRequest.ProtoReflect.Descriptor instead. +func (*ResolveCategoryCategoryMetricsRequest) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{135} +} + +func (x *ResolveCategoryCategoryMetricsRequest) GetContext() []*ResolveCategoryCategoryMetricsContext { + if x != nil { + return x.Context + } + return nil +} + +func (x *ResolveCategoryCategoryMetricsRequest) GetFieldArgs() *ResolveCategoryCategoryMetricsArgs { + if x != nil { + return x.FieldArgs + } + return nil +} + +type ResolveCategoryCategoryMetricsResult struct { + state protoimpl.MessageState `protogen:"open.v1"` + CategoryMetrics *CategoryMetrics `protobuf:"bytes,1,opt,name=category_metrics,json=categoryMetrics,proto3" json:"category_metrics,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ResolveCategoryCategoryMetricsResult) Reset() { + *x = ResolveCategoryCategoryMetricsResult{} + mi := &file_testdata_service_proto_msgTypes[136] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ResolveCategoryCategoryMetricsResult) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ResolveCategoryCategoryMetricsResult) ProtoMessage() {} + +func (x *ResolveCategoryCategoryMetricsResult) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[136] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ResolveCategoryCategoryMetricsResult.ProtoReflect.Descriptor instead. +func (*ResolveCategoryCategoryMetricsResult) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{136} +} + +func (x *ResolveCategoryCategoryMetricsResult) GetCategoryMetrics() *CategoryMetrics { + if x != nil { + return x.CategoryMetrics + } + return nil +} + +type ResolveCategoryCategoryMetricsResponse struct { + state protoimpl.MessageState `protogen:"open.v1"` + Result []*ResolveCategoryCategoryMetricsResult `protobuf:"bytes,1,rep,name=result,proto3" json:"result,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ResolveCategoryCategoryMetricsResponse) Reset() { + *x = ResolveCategoryCategoryMetricsResponse{} + mi := &file_testdata_service_proto_msgTypes[137] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ResolveCategoryCategoryMetricsResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ResolveCategoryCategoryMetricsResponse) ProtoMessage() {} + +func (x *ResolveCategoryCategoryMetricsResponse) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[137] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ResolveCategoryCategoryMetricsResponse.ProtoReflect.Descriptor instead. +func (*ResolveCategoryCategoryMetricsResponse) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{137} +} + +func (x *ResolveCategoryCategoryMetricsResponse) GetResult() []*ResolveCategoryCategoryMetricsResult { + if x != nil { + return x.Result + } + return nil +} + +type ResolveSubcategoryItemCountArgs struct { + state protoimpl.MessageState `protogen:"open.v1"` + Filters *SubcategoryItemFilter `protobuf:"bytes,1,opt,name=filters,proto3" json:"filters,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ResolveSubcategoryItemCountArgs) Reset() { + *x = ResolveSubcategoryItemCountArgs{} + mi := &file_testdata_service_proto_msgTypes[138] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ResolveSubcategoryItemCountArgs) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ResolveSubcategoryItemCountArgs) ProtoMessage() {} + +func (x *ResolveSubcategoryItemCountArgs) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[138] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ResolveSubcategoryItemCountArgs.ProtoReflect.Descriptor instead. +func (*ResolveSubcategoryItemCountArgs) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{138} +} + +func (x *ResolveSubcategoryItemCountArgs) GetFilters() *SubcategoryItemFilter { + if x != nil { + return x.Filters + } + return nil +} + +type ResolveSubcategoryItemCountContext struct { + state protoimpl.MessageState `protogen:"open.v1"` + Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ResolveSubcategoryItemCountContext) Reset() { + *x = ResolveSubcategoryItemCountContext{} + mi := &file_testdata_service_proto_msgTypes[139] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ResolveSubcategoryItemCountContext) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ResolveSubcategoryItemCountContext) ProtoMessage() {} + +func (x *ResolveSubcategoryItemCountContext) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[139] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ResolveSubcategoryItemCountContext.ProtoReflect.Descriptor instead. +func (*ResolveSubcategoryItemCountContext) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{139} +} + +func (x *ResolveSubcategoryItemCountContext) GetId() string { + if x != nil { + return x.Id + } + return "" +} + +type ResolveSubcategoryItemCountRequest struct { + state protoimpl.MessageState `protogen:"open.v1"` + // context provides the resolver context for the field itemCount of type Subcategory. + Context []*ResolveSubcategoryItemCountContext `protobuf:"bytes,1,rep,name=context,proto3" json:"context,omitempty"` + // field_args provides the arguments for the resolver field itemCount of type Subcategory. + FieldArgs *ResolveSubcategoryItemCountArgs `protobuf:"bytes,2,opt,name=field_args,json=fieldArgs,proto3" json:"field_args,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ResolveSubcategoryItemCountRequest) Reset() { + *x = ResolveSubcategoryItemCountRequest{} + mi := &file_testdata_service_proto_msgTypes[140] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ResolveSubcategoryItemCountRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ResolveSubcategoryItemCountRequest) ProtoMessage() {} + +func (x *ResolveSubcategoryItemCountRequest) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[140] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ResolveSubcategoryItemCountRequest.ProtoReflect.Descriptor instead. +func (*ResolveSubcategoryItemCountRequest) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{140} +} + +func (x *ResolveSubcategoryItemCountRequest) GetContext() []*ResolveSubcategoryItemCountContext { + if x != nil { + return x.Context + } + return nil +} + +func (x *ResolveSubcategoryItemCountRequest) GetFieldArgs() *ResolveSubcategoryItemCountArgs { + if x != nil { + return x.FieldArgs + } + return nil +} + +type ResolveSubcategoryItemCountResult struct { + state protoimpl.MessageState `protogen:"open.v1"` + ItemCount int32 `protobuf:"varint,1,opt,name=item_count,json=itemCount,proto3" json:"item_count,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ResolveSubcategoryItemCountResult) Reset() { + *x = ResolveSubcategoryItemCountResult{} + mi := &file_testdata_service_proto_msgTypes[141] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ResolveSubcategoryItemCountResult) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ResolveSubcategoryItemCountResult) ProtoMessage() {} + +func (x *ResolveSubcategoryItemCountResult) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[141] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ResolveSubcategoryItemCountResult.ProtoReflect.Descriptor instead. +func (*ResolveSubcategoryItemCountResult) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{141} +} + +func (x *ResolveSubcategoryItemCountResult) GetItemCount() int32 { + if x != nil { + return x.ItemCount + } + return 0 +} + +type ResolveSubcategoryItemCountResponse struct { + state protoimpl.MessageState `protogen:"open.v1"` + Result []*ResolveSubcategoryItemCountResult `protobuf:"bytes,1,rep,name=result,proto3" json:"result,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ResolveSubcategoryItemCountResponse) Reset() { + *x = ResolveSubcategoryItemCountResponse{} + mi := &file_testdata_service_proto_msgTypes[142] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ResolveSubcategoryItemCountResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ResolveSubcategoryItemCountResponse) ProtoMessage() {} + +func (x *ResolveSubcategoryItemCountResponse) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[142] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ResolveSubcategoryItemCountResponse.ProtoReflect.Descriptor instead. +func (*ResolveSubcategoryItemCountResponse) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{142} +} + +func (x *ResolveSubcategoryItemCountResponse) GetResult() []*ResolveSubcategoryItemCountResult { + if x != nil { + return x.Result + } + return nil +} + +type Product struct { + state protoimpl.MessageState `protogen:"open.v1"` + Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` + Name string `protobuf:"bytes,2,opt,name=name,proto3" json:"name,omitempty"` + Price float64 `protobuf:"fixed64,3,opt,name=price,proto3" json:"price,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *Product) Reset() { + *x = Product{} + mi := &file_testdata_service_proto_msgTypes[143] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *Product) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Product) ProtoMessage() {} + +func (x *Product) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[143] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use Product.ProtoReflect.Descriptor instead. +func (*Product) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{143} +} + +func (x *Product) GetId() string { + if x != nil { + return x.Id + } + return "" +} + +func (x *Product) GetName() string { + if x != nil { + return x.Name + } + return "" +} + +func (x *Product) GetPrice() float64 { + if x != nil { + return x.Price + } + return 0 +} + +type Storage struct { + state protoimpl.MessageState `protogen:"open.v1"` + Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` + Name string `protobuf:"bytes,2,opt,name=name,proto3" json:"name,omitempty"` + Location string `protobuf:"bytes,3,opt,name=location,proto3" json:"location,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *Storage) Reset() { + *x = Storage{} + mi := &file_testdata_service_proto_msgTypes[144] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *Storage) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Storage) ProtoMessage() {} + +func (x *Storage) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[144] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use Storage.ProtoReflect.Descriptor instead. +func (*Storage) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{144} +} + +func (x *Storage) GetId() string { + if x != nil { + return x.Id + } + return "" +} + +func (x *Storage) GetName() string { + if x != nil { + return x.Name + } + return "" +} + +func (x *Storage) GetLocation() string { + if x != nil { + return x.Location + } + return "" +} + +type Warehouse struct { + state protoimpl.MessageState `protogen:"open.v1"` + Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` + Name string `protobuf:"bytes,2,opt,name=name,proto3" json:"name,omitempty"` + Location string `protobuf:"bytes,3,opt,name=location,proto3" json:"location,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *Warehouse) Reset() { + *x = Warehouse{} + mi := &file_testdata_service_proto_msgTypes[145] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *Warehouse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Warehouse) ProtoMessage() {} + +func (x *Warehouse) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[145] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use Warehouse.ProtoReflect.Descriptor instead. +func (*Warehouse) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{145} +} + +func (x *Warehouse) GetId() string { + if x != nil { + return x.Id + } + return "" +} + +func (x *Warehouse) GetName() string { + if x != nil { + return x.Name + } + return "" +} + +func (x *Warehouse) GetLocation() string { + if x != nil { + return x.Location + } + return "" +} + +type User struct { + state protoimpl.MessageState `protogen:"open.v1"` + Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` + Name string `protobuf:"bytes,2,opt,name=name,proto3" json:"name,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *User) Reset() { + *x = User{} + mi := &file_testdata_service_proto_msgTypes[146] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *User) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*User) ProtoMessage() {} + +func (x *User) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[146] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use User.ProtoReflect.Descriptor instead. +func (*User) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{146} +} + +func (x *User) GetId() string { + if x != nil { + return x.Id + } + return "" +} + +func (x *User) GetName() string { + if x != nil { + return x.Name + } + return "" +} + +type NestedTypeA struct { + state protoimpl.MessageState `protogen:"open.v1"` + Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` + Name string `protobuf:"bytes,2,opt,name=name,proto3" json:"name,omitempty"` + B *NestedTypeB `protobuf:"bytes,3,opt,name=b,proto3" json:"b,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *NestedTypeA) Reset() { + *x = NestedTypeA{} + mi := &file_testdata_service_proto_msgTypes[147] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *NestedTypeA) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*NestedTypeA) ProtoMessage() {} + +func (x *NestedTypeA) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[147] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use NestedTypeA.ProtoReflect.Descriptor instead. +func (*NestedTypeA) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{147} +} + +func (x *NestedTypeA) GetId() string { + if x != nil { + return x.Id + } + return "" +} + +func (x *NestedTypeA) GetName() string { + if x != nil { + return x.Name + } + return "" +} + +func (x *NestedTypeA) GetB() *NestedTypeB { + if x != nil { + return x.B + } + return nil +} + +type RecursiveType struct { + state protoimpl.MessageState `protogen:"open.v1"` + Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` + Name string `protobuf:"bytes,2,opt,name=name,proto3" json:"name,omitempty"` + RecursiveType *RecursiveType `protobuf:"bytes,3,opt,name=recursive_type,json=recursiveType,proto3" json:"recursive_type,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *RecursiveType) Reset() { + *x = RecursiveType{} + mi := &file_testdata_service_proto_msgTypes[148] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *RecursiveType) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*RecursiveType) ProtoMessage() {} + +func (x *RecursiveType) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[148] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use RecursiveType.ProtoReflect.Descriptor instead. +func (*RecursiveType) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{148} +} + +func (x *RecursiveType) GetId() string { + if x != nil { + return x.Id + } + return "" +} + +func (x *RecursiveType) GetName() string { + if x != nil { + return x.Name + } + return "" +} + +func (x *RecursiveType) GetRecursiveType() *RecursiveType { + if x != nil { + return x.RecursiveType + } + return nil +} + +type TypeWithMultipleFilterFields struct { + state protoimpl.MessageState `protogen:"open.v1"` + Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` + Name string `protobuf:"bytes,2,opt,name=name,proto3" json:"name,omitempty"` + FilterField_1 string `protobuf:"bytes,3,opt,name=filter_field_1,json=filterField1,proto3" json:"filter_field_1,omitempty"` + FilterField_2 string `protobuf:"bytes,4,opt,name=filter_field_2,json=filterField2,proto3" json:"filter_field_2,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *TypeWithMultipleFilterFields) Reset() { + *x = TypeWithMultipleFilterFields{} + mi := &file_testdata_service_proto_msgTypes[149] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *TypeWithMultipleFilterFields) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*TypeWithMultipleFilterFields) ProtoMessage() {} + +func (x *TypeWithMultipleFilterFields) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[149] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use TypeWithMultipleFilterFields.ProtoReflect.Descriptor instead. +func (*TypeWithMultipleFilterFields) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{149} +} + +func (x *TypeWithMultipleFilterFields) GetId() string { + if x != nil { + return x.Id + } + return "" +} + +func (x *TypeWithMultipleFilterFields) GetName() string { + if x != nil { + return x.Name + } + return "" +} + +func (x *TypeWithMultipleFilterFields) GetFilterField_1() string { + if x != nil { + return x.FilterField_1 + } + return "" +} + +func (x *TypeWithMultipleFilterFields) GetFilterField_2() string { + if x != nil { + return x.FilterField_2 + } + return "" +} + +type FilterTypeInput struct { + state protoimpl.MessageState `protogen:"open.v1"` + FilterField_1 string `protobuf:"bytes,1,opt,name=filter_field_1,json=filterField1,proto3" json:"filter_field_1,omitempty"` + FilterField_2 string `protobuf:"bytes,2,opt,name=filter_field_2,json=filterField2,proto3" json:"filter_field_2,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *FilterTypeInput) Reset() { + *x = FilterTypeInput{} + mi := &file_testdata_service_proto_msgTypes[150] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *FilterTypeInput) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*FilterTypeInput) ProtoMessage() {} + +func (x *FilterTypeInput) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[150] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use FilterTypeInput.ProtoReflect.Descriptor instead. +func (*FilterTypeInput) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{150} +} + +func (x *FilterTypeInput) GetFilterField_1() string { + if x != nil { + return x.FilterField_1 + } + return "" +} + +func (x *FilterTypeInput) GetFilterField_2() string { + if x != nil { + return x.FilterField_2 + } + return "" +} + +type ComplexFilterTypeInput struct { + state protoimpl.MessageState `protogen:"open.v1"` + Filter *FilterType `protobuf:"bytes,1,opt,name=filter,proto3" json:"filter,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ComplexFilterTypeInput) Reset() { + *x = ComplexFilterTypeInput{} + mi := &file_testdata_service_proto_msgTypes[151] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ComplexFilterTypeInput) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ComplexFilterTypeInput) ProtoMessage() {} + +func (x *ComplexFilterTypeInput) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[151] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ComplexFilterTypeInput.ProtoReflect.Descriptor instead. +func (*ComplexFilterTypeInput) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{151} +} + +func (x *ComplexFilterTypeInput) GetFilter() *FilterType { + if x != nil { + return x.Filter + } + return nil +} + +type TypeWithComplexFilterInput struct { + state protoimpl.MessageState `protogen:"open.v1"` + Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` + Name string `protobuf:"bytes,2,opt,name=name,proto3" json:"name,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *TypeWithComplexFilterInput) Reset() { + *x = TypeWithComplexFilterInput{} + mi := &file_testdata_service_proto_msgTypes[152] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *TypeWithComplexFilterInput) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*TypeWithComplexFilterInput) ProtoMessage() {} + +func (x *TypeWithComplexFilterInput) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[152] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use TypeWithComplexFilterInput.ProtoReflect.Descriptor instead. +func (*TypeWithComplexFilterInput) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{152} +} + +func (x *TypeWithComplexFilterInput) GetId() string { + if x != nil { + return x.Id + } + return "" +} + +func (x *TypeWithComplexFilterInput) GetName() string { + if x != nil { + return x.Name + } + return "" +} + +type OrderInput struct { + state protoimpl.MessageState `protogen:"open.v1"` + OrderId string `protobuf:"bytes,1,opt,name=order_id,json=orderId,proto3" json:"order_id,omitempty"` + CustomerName string `protobuf:"bytes,2,opt,name=customer_name,json=customerName,proto3" json:"customer_name,omitempty"` + Lines []*OrderLineInput `protobuf:"bytes,3,rep,name=lines,proto3" json:"lines,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *OrderInput) Reset() { + *x = OrderInput{} + mi := &file_testdata_service_proto_msgTypes[153] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *OrderInput) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*OrderInput) ProtoMessage() {} + +func (x *OrderInput) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[153] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use OrderInput.ProtoReflect.Descriptor instead. +func (*OrderInput) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{153} +} + +func (x *OrderInput) GetOrderId() string { + if x != nil { + return x.OrderId + } + return "" +} + +func (x *OrderInput) GetCustomerName() string { + if x != nil { + return x.CustomerName + } + return "" +} + +func (x *OrderInput) GetLines() []*OrderLineInput { + if x != nil { + return x.Lines + } + return nil +} + +type Order struct { + state protoimpl.MessageState `protogen:"open.v1"` + OrderId string `protobuf:"bytes,1,opt,name=order_id,json=orderId,proto3" json:"order_id,omitempty"` + CustomerName string `protobuf:"bytes,2,opt,name=customer_name,json=customerName,proto3" json:"customer_name,omitempty"` + TotalItems int32 `protobuf:"varint,3,opt,name=total_items,json=totalItems,proto3" json:"total_items,omitempty"` + OrderLines *ListOfOrderLine `protobuf:"bytes,4,opt,name=order_lines,json=orderLines,proto3" json:"order_lines,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *Order) Reset() { + *x = Order{} + mi := &file_testdata_service_proto_msgTypes[154] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *Order) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Order) ProtoMessage() {} + +func (x *Order) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[154] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use Order.ProtoReflect.Descriptor instead. +func (*Order) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{154} +} + +func (x *Order) GetOrderId() string { + if x != nil { + return x.OrderId + } + return "" +} + +func (x *Order) GetCustomerName() string { + if x != nil { + return x.CustomerName + } + return "" +} + +func (x *Order) GetTotalItems() int32 { + if x != nil { + return x.TotalItems + } + return 0 +} + +func (x *Order) GetOrderLines() *ListOfOrderLine { + if x != nil { + return x.OrderLines + } + return nil +} + +type Category struct { + state protoimpl.MessageState `protogen:"open.v1"` + Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` + Name string `protobuf:"bytes,2,opt,name=name,proto3" json:"name,omitempty"` + Kind CategoryKind `protobuf:"varint,3,opt,name=kind,proto3,enum=productv1.CategoryKind" json:"kind,omitempty"` + Subcategories *ListOfSubcategory `protobuf:"bytes,5,opt,name=subcategories,proto3" json:"subcategories,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *Category) Reset() { + *x = Category{} + mi := &file_testdata_service_proto_msgTypes[155] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *Category) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Category) ProtoMessage() {} + +func (x *Category) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[155] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use Category.ProtoReflect.Descriptor instead. +func (*Category) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{155} +} + +func (x *Category) GetId() string { + if x != nil { + return x.Id + } + return "" +} + +func (x *Category) GetName() string { + if x != nil { + return x.Name + } + return "" +} + +func (x *Category) GetKind() CategoryKind { + if x != nil { + return x.Kind + } + return CategoryKind_CATEGORY_KIND_UNSPECIFIED +} + +func (x *Category) GetSubcategories() *ListOfSubcategory { + if x != nil { + return x.Subcategories + } + return nil +} + +type CategoryFilter struct { + state protoimpl.MessageState `protogen:"open.v1"` + Category CategoryKind `protobuf:"varint,1,opt,name=category,proto3,enum=productv1.CategoryKind" json:"category,omitempty"` + Pagination *Pagination `protobuf:"bytes,2,opt,name=pagination,proto3" json:"pagination,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *CategoryFilter) Reset() { + *x = CategoryFilter{} + mi := &file_testdata_service_proto_msgTypes[156] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *CategoryFilter) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*CategoryFilter) ProtoMessage() {} + +func (x *CategoryFilter) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[156] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use CategoryFilter.ProtoReflect.Descriptor instead. +func (*CategoryFilter) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{156} +} + +func (x *CategoryFilter) GetCategory() CategoryKind { + if x != nil { + return x.Category + } + return CategoryKind_CATEGORY_KIND_UNSPECIFIED +} + +func (x *CategoryFilter) GetPagination() *Pagination { + if x != nil { + return x.Pagination + } + return nil +} + +type Animal struct { + state protoimpl.MessageState `protogen:"open.v1"` + // Types that are valid to be assigned to Instance: + // + // *Animal_Cat + // *Animal_Dog + Instance isAnimal_Instance `protobuf_oneof:"instance"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *Animal) Reset() { + *x = Animal{} + mi := &file_testdata_service_proto_msgTypes[157] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *Animal) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Animal) ProtoMessage() {} + +func (x *Animal) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[157] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use Animal.ProtoReflect.Descriptor instead. +func (*Animal) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{157} +} + +func (x *Animal) GetInstance() isAnimal_Instance { + if x != nil { + return x.Instance + } + return nil +} + +func (x *Animal) GetCat() *Cat { + if x != nil { + if x, ok := x.Instance.(*Animal_Cat); ok { + return x.Cat + } + } + return nil +} + +func (x *Animal) GetDog() *Dog { + if x != nil { + if x, ok := x.Instance.(*Animal_Dog); ok { + return x.Dog + } + } + return nil +} + +type isAnimal_Instance interface { + isAnimal_Instance() +} + +type Animal_Cat struct { + Cat *Cat `protobuf:"bytes,1,opt,name=cat,proto3,oneof"` +} + +type Animal_Dog struct { + Dog *Dog `protobuf:"bytes,2,opt,name=dog,proto3,oneof"` +} + +func (*Animal_Cat) isAnimal_Instance() {} + +func (*Animal_Dog) isAnimal_Instance() {} + +type SearchInput struct { + state protoimpl.MessageState `protogen:"open.v1"` + Query string `protobuf:"bytes,1,opt,name=query,proto3" json:"query,omitempty"` + Limit *wrapperspb.Int32Value `protobuf:"bytes,2,opt,name=limit,proto3" json:"limit,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *SearchInput) Reset() { + *x = SearchInput{} + mi := &file_testdata_service_proto_msgTypes[158] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *SearchInput) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*SearchInput) ProtoMessage() {} + +func (x *SearchInput) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[158] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use SearchInput.ProtoReflect.Descriptor instead. +func (*SearchInput) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{158} +} + +func (x *SearchInput) GetQuery() string { + if x != nil { + return x.Query + } + return "" +} + +func (x *SearchInput) GetLimit() *wrapperspb.Int32Value { + if x != nil { + return x.Limit + } + return nil +} + +type SearchResult struct { + state protoimpl.MessageState `protogen:"open.v1"` + // Types that are valid to be assigned to Value: + // + // *SearchResult_Product + // *SearchResult_User + // *SearchResult_Category + Value isSearchResult_Value `protobuf_oneof:"value"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *SearchResult) Reset() { + *x = SearchResult{} + mi := &file_testdata_service_proto_msgTypes[159] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *SearchResult) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*SearchResult) ProtoMessage() {} + +func (x *SearchResult) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[159] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use SearchResult.ProtoReflect.Descriptor instead. +func (*SearchResult) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{159} +} + +func (x *SearchResult) GetValue() isSearchResult_Value { + if x != nil { + return x.Value + } + return nil +} + +func (x *SearchResult) GetProduct() *Product { + if x != nil { + if x, ok := x.Value.(*SearchResult_Product); ok { + return x.Product + } + } + return nil +} + +func (x *SearchResult) GetUser() *User { + if x != nil { + if x, ok := x.Value.(*SearchResult_User); ok { + return x.User + } + } + return nil +} + +func (x *SearchResult) GetCategory() *Category { + if x != nil { + if x, ok := x.Value.(*SearchResult_Category); ok { + return x.Category + } + } + return nil +} + +type isSearchResult_Value interface { + isSearchResult_Value() +} + +type SearchResult_Product struct { + Product *Product `protobuf:"bytes,1,opt,name=product,proto3,oneof"` +} + +type SearchResult_User struct { + User *User `protobuf:"bytes,2,opt,name=user,proto3,oneof"` +} + +type SearchResult_Category struct { + Category *Category `protobuf:"bytes,3,opt,name=category,proto3,oneof"` +} + +func (*SearchResult_Product) isSearchResult_Value() {} + +func (*SearchResult_User) isSearchResult_Value() {} + +func (*SearchResult_Category) isSearchResult_Value() {} + +type NullableFieldsType struct { + state protoimpl.MessageState `protogen:"open.v1"` + Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` + Name string `protobuf:"bytes,2,opt,name=name,proto3" json:"name,omitempty"` + OptionalString *wrapperspb.StringValue `protobuf:"bytes,3,opt,name=optional_string,json=optionalString,proto3" json:"optional_string,omitempty"` + OptionalInt *wrapperspb.Int32Value `protobuf:"bytes,4,opt,name=optional_int,json=optionalInt,proto3" json:"optional_int,omitempty"` + OptionalFloat *wrapperspb.DoubleValue `protobuf:"bytes,5,opt,name=optional_float,json=optionalFloat,proto3" json:"optional_float,omitempty"` + OptionalBoolean *wrapperspb.BoolValue `protobuf:"bytes,6,opt,name=optional_boolean,json=optionalBoolean,proto3" json:"optional_boolean,omitempty"` + RequiredString string `protobuf:"bytes,7,opt,name=required_string,json=requiredString,proto3" json:"required_string,omitempty"` + RequiredInt int32 `protobuf:"varint,8,opt,name=required_int,json=requiredInt,proto3" json:"required_int,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *NullableFieldsType) Reset() { + *x = NullableFieldsType{} + mi := &file_testdata_service_proto_msgTypes[160] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *NullableFieldsType) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*NullableFieldsType) ProtoMessage() {} + +func (x *NullableFieldsType) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[160] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use NullableFieldsType.ProtoReflect.Descriptor instead. +func (*NullableFieldsType) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{160} +} + +func (x *NullableFieldsType) GetId() string { + if x != nil { + return x.Id + } + return "" +} + +func (x *NullableFieldsType) GetName() string { + if x != nil { + return x.Name + } + return "" +} + +func (x *NullableFieldsType) GetOptionalString() *wrapperspb.StringValue { + if x != nil { + return x.OptionalString + } + return nil +} + +func (x *NullableFieldsType) GetOptionalInt() *wrapperspb.Int32Value { + if x != nil { + return x.OptionalInt + } + return nil +} + +func (x *NullableFieldsType) GetOptionalFloat() *wrapperspb.DoubleValue { + if x != nil { + return x.OptionalFloat + } + return nil +} + +func (x *NullableFieldsType) GetOptionalBoolean() *wrapperspb.BoolValue { + if x != nil { + return x.OptionalBoolean + } + return nil +} + +func (x *NullableFieldsType) GetRequiredString() string { + if x != nil { + return x.RequiredString + } + return "" +} + +func (x *NullableFieldsType) GetRequiredInt() int32 { + if x != nil { + return x.RequiredInt + } + return 0 +} + +type NullableFieldsFilter struct { + state protoimpl.MessageState `protogen:"open.v1"` + Name *wrapperspb.StringValue `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` + OptionalString *wrapperspb.StringValue `protobuf:"bytes,2,opt,name=optional_string,json=optionalString,proto3" json:"optional_string,omitempty"` + IncludeNulls *wrapperspb.BoolValue `protobuf:"bytes,3,opt,name=include_nulls,json=includeNulls,proto3" json:"include_nulls,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *NullableFieldsFilter) Reset() { + *x = NullableFieldsFilter{} + mi := &file_testdata_service_proto_msgTypes[161] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *NullableFieldsFilter) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*NullableFieldsFilter) ProtoMessage() {} + +func (x *NullableFieldsFilter) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[161] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use NullableFieldsFilter.ProtoReflect.Descriptor instead. +func (*NullableFieldsFilter) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{161} +} + +func (x *NullableFieldsFilter) GetName() *wrapperspb.StringValue { + if x != nil { + return x.Name + } + return nil +} + +func (x *NullableFieldsFilter) GetOptionalString() *wrapperspb.StringValue { + if x != nil { + return x.OptionalString + } + return nil +} + +func (x *NullableFieldsFilter) GetIncludeNulls() *wrapperspb.BoolValue { + if x != nil { + return x.IncludeNulls + } + return nil +} + +type BlogPost struct { + state protoimpl.MessageState `protogen:"open.v1"` + Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` + Title string `protobuf:"bytes,2,opt,name=title,proto3" json:"title,omitempty"` + Content string `protobuf:"bytes,3,opt,name=content,proto3" json:"content,omitempty"` + Tags []string `protobuf:"bytes,4,rep,name=tags,proto3" json:"tags,omitempty"` + OptionalTags *ListOfString `protobuf:"bytes,5,opt,name=optional_tags,json=optionalTags,proto3" json:"optional_tags,omitempty"` + Categories []string `protobuf:"bytes,6,rep,name=categories,proto3" json:"categories,omitempty"` + Keywords *ListOfString `protobuf:"bytes,7,opt,name=keywords,proto3" json:"keywords,omitempty"` + ViewCounts []int32 `protobuf:"varint,8,rep,packed,name=view_counts,json=viewCounts,proto3" json:"view_counts,omitempty"` + Ratings *ListOfFloat `protobuf:"bytes,9,opt,name=ratings,proto3" json:"ratings,omitempty"` + IsPublished *ListOfBoolean `protobuf:"bytes,10,opt,name=is_published,json=isPublished,proto3" json:"is_published,omitempty"` + TagGroups *ListOfListOfString `protobuf:"bytes,11,opt,name=tag_groups,json=tagGroups,proto3" json:"tag_groups,omitempty"` + RelatedTopics *ListOfListOfString `protobuf:"bytes,12,opt,name=related_topics,json=relatedTopics,proto3" json:"related_topics,omitempty"` + CommentThreads *ListOfListOfString `protobuf:"bytes,13,opt,name=comment_threads,json=commentThreads,proto3" json:"comment_threads,omitempty"` + Suggestions *ListOfListOfString `protobuf:"bytes,14,opt,name=suggestions,proto3" json:"suggestions,omitempty"` + RelatedCategories []*Category `protobuf:"bytes,15,rep,name=related_categories,json=relatedCategories,proto3" json:"related_categories,omitempty"` + Contributors []*User `protobuf:"bytes,16,rep,name=contributors,proto3" json:"contributors,omitempty"` + MentionedProducts *ListOfProduct `protobuf:"bytes,17,opt,name=mentioned_products,json=mentionedProducts,proto3" json:"mentioned_products,omitempty"` + MentionedUsers *ListOfUser `protobuf:"bytes,18,opt,name=mentioned_users,json=mentionedUsers,proto3" json:"mentioned_users,omitempty"` + CategoryGroups *ListOfListOfCategory `protobuf:"bytes,19,opt,name=category_groups,json=categoryGroups,proto3" json:"category_groups,omitempty"` + ContributorTeams *ListOfListOfUser `protobuf:"bytes,20,opt,name=contributor_teams,json=contributorTeams,proto3" json:"contributor_teams,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *BlogPost) Reset() { + *x = BlogPost{} + mi := &file_testdata_service_proto_msgTypes[162] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *BlogPost) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*BlogPost) ProtoMessage() {} + +func (x *BlogPost) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[162] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use BlogPost.ProtoReflect.Descriptor instead. +func (*BlogPost) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{162} +} + +func (x *BlogPost) GetId() string { + if x != nil { + return x.Id + } + return "" +} + +func (x *BlogPost) GetTitle() string { + if x != nil { + return x.Title + } + return "" +} + +func (x *BlogPost) GetContent() string { + if x != nil { + return x.Content + } + return "" +} + +func (x *BlogPost) GetTags() []string { + if x != nil { + return x.Tags + } + return nil +} + +func (x *BlogPost) GetOptionalTags() *ListOfString { + if x != nil { + return x.OptionalTags + } + return nil +} + +func (x *BlogPost) GetCategories() []string { + if x != nil { + return x.Categories + } + return nil +} + +func (x *BlogPost) GetKeywords() *ListOfString { + if x != nil { + return x.Keywords + } + return nil +} + +func (x *BlogPost) GetViewCounts() []int32 { + if x != nil { + return x.ViewCounts + } + return nil +} + +func (x *BlogPost) GetRatings() *ListOfFloat { + if x != nil { + return x.Ratings + } + return nil +} + +func (x *BlogPost) GetIsPublished() *ListOfBoolean { + if x != nil { + return x.IsPublished + } + return nil +} + +func (x *BlogPost) GetTagGroups() *ListOfListOfString { + if x != nil { + return x.TagGroups + } + return nil +} + +func (x *BlogPost) GetRelatedTopics() *ListOfListOfString { + if x != nil { + return x.RelatedTopics + } + return nil +} + +func (x *BlogPost) GetCommentThreads() *ListOfListOfString { + if x != nil { + return x.CommentThreads + } + return nil +} + +func (x *BlogPost) GetSuggestions() *ListOfListOfString { + if x != nil { + return x.Suggestions + } + return nil +} + +func (x *BlogPost) GetRelatedCategories() []*Category { + if x != nil { + return x.RelatedCategories + } + return nil +} + +func (x *BlogPost) GetContributors() []*User { + if x != nil { + return x.Contributors + } + return nil +} + +func (x *BlogPost) GetMentionedProducts() *ListOfProduct { + if x != nil { + return x.MentionedProducts + } + return nil +} + +func (x *BlogPost) GetMentionedUsers() *ListOfUser { + if x != nil { + return x.MentionedUsers + } + return nil +} + +func (x *BlogPost) GetCategoryGroups() *ListOfListOfCategory { + if x != nil { + return x.CategoryGroups + } + return nil +} + +func (x *BlogPost) GetContributorTeams() *ListOfListOfUser { + if x != nil { + return x.ContributorTeams + } + return nil +} + +type BlogPostFilter struct { + state protoimpl.MessageState `protogen:"open.v1"` + Title *wrapperspb.StringValue `protobuf:"bytes,1,opt,name=title,proto3" json:"title,omitempty"` + HasCategories *wrapperspb.BoolValue `protobuf:"bytes,2,opt,name=has_categories,json=hasCategories,proto3" json:"has_categories,omitempty"` + MinTags *wrapperspb.Int32Value `protobuf:"bytes,3,opt,name=min_tags,json=minTags,proto3" json:"min_tags,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *BlogPostFilter) Reset() { + *x = BlogPostFilter{} + mi := &file_testdata_service_proto_msgTypes[163] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *BlogPostFilter) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*BlogPostFilter) ProtoMessage() {} + +func (x *BlogPostFilter) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[163] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use BlogPostFilter.ProtoReflect.Descriptor instead. +func (*BlogPostFilter) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{163} +} + +func (x *BlogPostFilter) GetTitle() *wrapperspb.StringValue { + if x != nil { + return x.Title + } + return nil +} + +func (x *BlogPostFilter) GetHasCategories() *wrapperspb.BoolValue { + if x != nil { + return x.HasCategories + } + return nil +} + +func (x *BlogPostFilter) GetMinTags() *wrapperspb.Int32Value { + if x != nil { + return x.MinTags + } + return nil +} + +type Author struct { + state protoimpl.MessageState `protogen:"open.v1"` + Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` + Name string `protobuf:"bytes,2,opt,name=name,proto3" json:"name,omitempty"` + Email *wrapperspb.StringValue `protobuf:"bytes,3,opt,name=email,proto3" json:"email,omitempty"` + Skills []string `protobuf:"bytes,4,rep,name=skills,proto3" json:"skills,omitempty"` + Languages []string `protobuf:"bytes,5,rep,name=languages,proto3" json:"languages,omitempty"` + SocialLinks *ListOfString `protobuf:"bytes,6,opt,name=social_links,json=socialLinks,proto3" json:"social_links,omitempty"` + TeamsByProject *ListOfListOfString `protobuf:"bytes,7,opt,name=teams_by_project,json=teamsByProject,proto3" json:"teams_by_project,omitempty"` + Collaborations *ListOfListOfString `protobuf:"bytes,8,opt,name=collaborations,proto3" json:"collaborations,omitempty"` + WrittenPosts *ListOfBlogPost `protobuf:"bytes,9,opt,name=written_posts,json=writtenPosts,proto3" json:"written_posts,omitempty"` + FavoriteCategories []*Category `protobuf:"bytes,10,rep,name=favorite_categories,json=favoriteCategories,proto3" json:"favorite_categories,omitempty"` + RelatedAuthors *ListOfUser `protobuf:"bytes,11,opt,name=related_authors,json=relatedAuthors,proto3" json:"related_authors,omitempty"` + ProductReviews *ListOfProduct `protobuf:"bytes,12,opt,name=product_reviews,json=productReviews,proto3" json:"product_reviews,omitempty"` + AuthorGroups *ListOfListOfUser `protobuf:"bytes,13,opt,name=author_groups,json=authorGroups,proto3" json:"author_groups,omitempty"` + CategoryPreferences *ListOfListOfCategory `protobuf:"bytes,14,opt,name=category_preferences,json=categoryPreferences,proto3" json:"category_preferences,omitempty"` + ProjectTeams *ListOfListOfUser `protobuf:"bytes,15,opt,name=project_teams,json=projectTeams,proto3" json:"project_teams,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *Author) Reset() { + *x = Author{} + mi := &file_testdata_service_proto_msgTypes[164] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *Author) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Author) ProtoMessage() {} + +func (x *Author) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[164] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use Author.ProtoReflect.Descriptor instead. +func (*Author) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{164} +} + +func (x *Author) GetId() string { + if x != nil { + return x.Id + } + return "" +} + +func (x *Author) GetName() string { + if x != nil { + return x.Name + } + return "" +} + +func (x *Author) GetEmail() *wrapperspb.StringValue { + if x != nil { + return x.Email + } + return nil +} + +func (x *Author) GetSkills() []string { + if x != nil { + return x.Skills + } + return nil +} + +func (x *Author) GetLanguages() []string { + if x != nil { + return x.Languages + } + return nil +} + +func (x *Author) GetSocialLinks() *ListOfString { + if x != nil { + return x.SocialLinks + } + return nil +} + +func (x *Author) GetTeamsByProject() *ListOfListOfString { + if x != nil { + return x.TeamsByProject + } + return nil +} + +func (x *Author) GetCollaborations() *ListOfListOfString { + if x != nil { + return x.Collaborations + } + return nil +} + +func (x *Author) GetWrittenPosts() *ListOfBlogPost { + if x != nil { + return x.WrittenPosts + } + return nil +} + +func (x *Author) GetFavoriteCategories() []*Category { + if x != nil { + return x.FavoriteCategories + } + return nil +} + +func (x *Author) GetRelatedAuthors() *ListOfUser { + if x != nil { + return x.RelatedAuthors + } + return nil +} + +func (x *Author) GetProductReviews() *ListOfProduct { + if x != nil { + return x.ProductReviews + } + return nil +} + +func (x *Author) GetAuthorGroups() *ListOfListOfUser { + if x != nil { + return x.AuthorGroups + } + return nil +} + +func (x *Author) GetCategoryPreferences() *ListOfListOfCategory { + if x != nil { + return x.CategoryPreferences + } + return nil +} + +func (x *Author) GetProjectTeams() *ListOfListOfUser { + if x != nil { + return x.ProjectTeams + } + return nil +} + +type AuthorFilter struct { + state protoimpl.MessageState `protogen:"open.v1"` + Name *wrapperspb.StringValue `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` + HasTeams *wrapperspb.BoolValue `protobuf:"bytes,2,opt,name=has_teams,json=hasTeams,proto3" json:"has_teams,omitempty"` + SkillCount *wrapperspb.Int32Value `protobuf:"bytes,3,opt,name=skill_count,json=skillCount,proto3" json:"skill_count,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *AuthorFilter) Reset() { + *x = AuthorFilter{} + mi := &file_testdata_service_proto_msgTypes[165] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *AuthorFilter) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*AuthorFilter) ProtoMessage() {} + +func (x *AuthorFilter) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[165] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use AuthorFilter.ProtoReflect.Descriptor instead. +func (*AuthorFilter) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{165} +} + +func (x *AuthorFilter) GetName() *wrapperspb.StringValue { + if x != nil { + return x.Name + } + return nil +} + +func (x *AuthorFilter) GetHasTeams() *wrapperspb.BoolValue { + if x != nil { + return x.HasTeams + } + return nil +} + +func (x *AuthorFilter) GetSkillCount() *wrapperspb.Int32Value { + if x != nil { + return x.SkillCount + } + return nil +} + +type UserInput struct { + state protoimpl.MessageState `protogen:"open.v1"` + Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *UserInput) Reset() { + *x = UserInput{} + mi := &file_testdata_service_proto_msgTypes[166] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *UserInput) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*UserInput) ProtoMessage() {} + +func (x *UserInput) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[166] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use UserInput.ProtoReflect.Descriptor instead. +func (*UserInput) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{166} +} + +func (x *UserInput) GetName() string { + if x != nil { + return x.Name + } + return "" +} + +type ActionInput struct { + state protoimpl.MessageState `protogen:"open.v1"` + Type string `protobuf:"bytes,1,opt,name=type,proto3" json:"type,omitempty"` + Payload string `protobuf:"bytes,2,opt,name=payload,proto3" json:"payload,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ActionInput) Reset() { + *x = ActionInput{} + mi := &file_testdata_service_proto_msgTypes[167] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ActionInput) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ActionInput) ProtoMessage() {} + +func (x *ActionInput) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[167] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ActionInput.ProtoReflect.Descriptor instead. +func (*ActionInput) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{167} +} + +func (x *ActionInput) GetType() string { + if x != nil { + return x.Type + } + return "" +} + +func (x *ActionInput) GetPayload() string { + if x != nil { + return x.Payload + } + return "" +} + +type ActionResult struct { + state protoimpl.MessageState `protogen:"open.v1"` + // Types that are valid to be assigned to Value: + // + // *ActionResult_ActionSuccess + // *ActionResult_ActionError + Value isActionResult_Value `protobuf_oneof:"value"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ActionResult) Reset() { + *x = ActionResult{} + mi := &file_testdata_service_proto_msgTypes[168] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ActionResult) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ActionResult) ProtoMessage() {} + +func (x *ActionResult) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[168] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ActionResult.ProtoReflect.Descriptor instead. +func (*ActionResult) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{168} +} + +func (x *ActionResult) GetValue() isActionResult_Value { + if x != nil { + return x.Value + } + return nil +} + +func (x *ActionResult) GetActionSuccess() *ActionSuccess { + if x != nil { + if x, ok := x.Value.(*ActionResult_ActionSuccess); ok { + return x.ActionSuccess + } + } + return nil +} + +func (x *ActionResult) GetActionError() *ActionError { + if x != nil { + if x, ok := x.Value.(*ActionResult_ActionError); ok { + return x.ActionError + } + } + return nil +} + +type isActionResult_Value interface { + isActionResult_Value() +} + +type ActionResult_ActionSuccess struct { + ActionSuccess *ActionSuccess `protobuf:"bytes,1,opt,name=action_success,json=actionSuccess,proto3,oneof"` +} + +type ActionResult_ActionError struct { + ActionError *ActionError `protobuf:"bytes,2,opt,name=action_error,json=actionError,proto3,oneof"` +} + +func (*ActionResult_ActionSuccess) isActionResult_Value() {} + +func (*ActionResult_ActionError) isActionResult_Value() {} + +type NullableFieldsInput struct { + state protoimpl.MessageState `protogen:"open.v1"` + Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` + OptionalString *wrapperspb.StringValue `protobuf:"bytes,2,opt,name=optional_string,json=optionalString,proto3" json:"optional_string,omitempty"` + OptionalInt *wrapperspb.Int32Value `protobuf:"bytes,3,opt,name=optional_int,json=optionalInt,proto3" json:"optional_int,omitempty"` + OptionalFloat *wrapperspb.DoubleValue `protobuf:"bytes,4,opt,name=optional_float,json=optionalFloat,proto3" json:"optional_float,omitempty"` + OptionalBoolean *wrapperspb.BoolValue `protobuf:"bytes,5,opt,name=optional_boolean,json=optionalBoolean,proto3" json:"optional_boolean,omitempty"` + RequiredString string `protobuf:"bytes,6,opt,name=required_string,json=requiredString,proto3" json:"required_string,omitempty"` + RequiredInt int32 `protobuf:"varint,7,opt,name=required_int,json=requiredInt,proto3" json:"required_int,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *NullableFieldsInput) Reset() { + *x = NullableFieldsInput{} + mi := &file_testdata_service_proto_msgTypes[169] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *NullableFieldsInput) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*NullableFieldsInput) ProtoMessage() {} + +func (x *NullableFieldsInput) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[169] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use NullableFieldsInput.ProtoReflect.Descriptor instead. +func (*NullableFieldsInput) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{169} +} + +func (x *NullableFieldsInput) GetName() string { + if x != nil { + return x.Name + } + return "" +} + +func (x *NullableFieldsInput) GetOptionalString() *wrapperspb.StringValue { + if x != nil { + return x.OptionalString + } + return nil +} + +func (x *NullableFieldsInput) GetOptionalInt() *wrapperspb.Int32Value { + if x != nil { + return x.OptionalInt + } + return nil +} + +func (x *NullableFieldsInput) GetOptionalFloat() *wrapperspb.DoubleValue { + if x != nil { + return x.OptionalFloat + } + return nil +} + +func (x *NullableFieldsInput) GetOptionalBoolean() *wrapperspb.BoolValue { + if x != nil { + return x.OptionalBoolean + } + return nil +} + +func (x *NullableFieldsInput) GetRequiredString() string { + if x != nil { + return x.RequiredString + } + return "" +} + +func (x *NullableFieldsInput) GetRequiredInt() int32 { + if x != nil { + return x.RequiredInt + } + return 0 +} + +type BlogPostInput struct { + state protoimpl.MessageState `protogen:"open.v1"` + Title string `protobuf:"bytes,1,opt,name=title,proto3" json:"title,omitempty"` + Content string `protobuf:"bytes,2,opt,name=content,proto3" json:"content,omitempty"` + Tags []string `protobuf:"bytes,3,rep,name=tags,proto3" json:"tags,omitempty"` + OptionalTags *ListOfString `protobuf:"bytes,4,opt,name=optional_tags,json=optionalTags,proto3" json:"optional_tags,omitempty"` + Categories []string `protobuf:"bytes,5,rep,name=categories,proto3" json:"categories,omitempty"` + Keywords *ListOfString `protobuf:"bytes,6,opt,name=keywords,proto3" json:"keywords,omitempty"` + ViewCounts []int32 `protobuf:"varint,7,rep,packed,name=view_counts,json=viewCounts,proto3" json:"view_counts,omitempty"` + Ratings *ListOfFloat `protobuf:"bytes,8,opt,name=ratings,proto3" json:"ratings,omitempty"` + IsPublished *ListOfBoolean `protobuf:"bytes,9,opt,name=is_published,json=isPublished,proto3" json:"is_published,omitempty"` + TagGroups *ListOfListOfString `protobuf:"bytes,10,opt,name=tag_groups,json=tagGroups,proto3" json:"tag_groups,omitempty"` + RelatedTopics *ListOfListOfString `protobuf:"bytes,11,opt,name=related_topics,json=relatedTopics,proto3" json:"related_topics,omitempty"` + CommentThreads *ListOfListOfString `protobuf:"bytes,12,opt,name=comment_threads,json=commentThreads,proto3" json:"comment_threads,omitempty"` + Suggestions *ListOfListOfString `protobuf:"bytes,13,opt,name=suggestions,proto3" json:"suggestions,omitempty"` + RelatedCategories *ListOfCategoryInput `protobuf:"bytes,14,opt,name=related_categories,json=relatedCategories,proto3" json:"related_categories,omitempty"` + Contributors *ListOfUserInput `protobuf:"bytes,15,opt,name=contributors,proto3" json:"contributors,omitempty"` + CategoryGroups *ListOfListOfCategoryInput `protobuf:"bytes,16,opt,name=category_groups,json=categoryGroups,proto3" json:"category_groups,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *BlogPostInput) Reset() { + *x = BlogPostInput{} + mi := &file_testdata_service_proto_msgTypes[170] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *BlogPostInput) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*BlogPostInput) ProtoMessage() {} + +func (x *BlogPostInput) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[170] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use BlogPostInput.ProtoReflect.Descriptor instead. +func (*BlogPostInput) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{170} +} + +func (x *BlogPostInput) GetTitle() string { + if x != nil { + return x.Title + } + return "" +} + +func (x *BlogPostInput) GetContent() string { + if x != nil { + return x.Content + } + return "" +} + +func (x *BlogPostInput) GetTags() []string { + if x != nil { + return x.Tags + } + return nil +} + +func (x *BlogPostInput) GetOptionalTags() *ListOfString { + if x != nil { + return x.OptionalTags + } + return nil +} + +func (x *BlogPostInput) GetCategories() []string { + if x != nil { + return x.Categories + } + return nil +} + +func (x *BlogPostInput) GetKeywords() *ListOfString { + if x != nil { + return x.Keywords + } + return nil +} + +func (x *BlogPostInput) GetViewCounts() []int32 { + if x != nil { + return x.ViewCounts + } + return nil +} + +func (x *BlogPostInput) GetRatings() *ListOfFloat { + if x != nil { + return x.Ratings + } + return nil +} + +func (x *BlogPostInput) GetIsPublished() *ListOfBoolean { + if x != nil { + return x.IsPublished + } + return nil +} + +func (x *BlogPostInput) GetTagGroups() *ListOfListOfString { + if x != nil { + return x.TagGroups + } + return nil +} + +func (x *BlogPostInput) GetRelatedTopics() *ListOfListOfString { + if x != nil { + return x.RelatedTopics + } + return nil +} + +func (x *BlogPostInput) GetCommentThreads() *ListOfListOfString { + if x != nil { + return x.CommentThreads + } + return nil +} + +func (x *BlogPostInput) GetSuggestions() *ListOfListOfString { + if x != nil { + return x.Suggestions + } + return nil +} + +func (x *BlogPostInput) GetRelatedCategories() *ListOfCategoryInput { + if x != nil { + return x.RelatedCategories + } + return nil +} + +func (x *BlogPostInput) GetContributors() *ListOfUserInput { + if x != nil { + return x.Contributors + } + return nil +} + +func (x *BlogPostInput) GetCategoryGroups() *ListOfListOfCategoryInput { + if x != nil { + return x.CategoryGroups + } + return nil +} + +type AuthorInput struct { + state protoimpl.MessageState `protogen:"open.v1"` + Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` + Email *wrapperspb.StringValue `protobuf:"bytes,2,opt,name=email,proto3" json:"email,omitempty"` + Skills []string `protobuf:"bytes,3,rep,name=skills,proto3" json:"skills,omitempty"` + Languages []string `protobuf:"bytes,4,rep,name=languages,proto3" json:"languages,omitempty"` + SocialLinks *ListOfString `protobuf:"bytes,5,opt,name=social_links,json=socialLinks,proto3" json:"social_links,omitempty"` + TeamsByProject *ListOfListOfString `protobuf:"bytes,6,opt,name=teams_by_project,json=teamsByProject,proto3" json:"teams_by_project,omitempty"` + Collaborations *ListOfListOfString `protobuf:"bytes,7,opt,name=collaborations,proto3" json:"collaborations,omitempty"` + FavoriteCategories []*CategoryInput `protobuf:"bytes,8,rep,name=favorite_categories,json=favoriteCategories,proto3" json:"favorite_categories,omitempty"` + AuthorGroups *ListOfListOfUserInput `protobuf:"bytes,9,opt,name=author_groups,json=authorGroups,proto3" json:"author_groups,omitempty"` + ProjectTeams *ListOfListOfUserInput `protobuf:"bytes,10,opt,name=project_teams,json=projectTeams,proto3" json:"project_teams,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *AuthorInput) Reset() { + *x = AuthorInput{} + mi := &file_testdata_service_proto_msgTypes[171] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *AuthorInput) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*AuthorInput) ProtoMessage() {} + +func (x *AuthorInput) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[171] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use AuthorInput.ProtoReflect.Descriptor instead. +func (*AuthorInput) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{171} +} + +func (x *AuthorInput) GetName() string { + if x != nil { + return x.Name + } + return "" +} + +func (x *AuthorInput) GetEmail() *wrapperspb.StringValue { + if x != nil { + return x.Email + } + return nil +} + +func (x *AuthorInput) GetSkills() []string { + if x != nil { + return x.Skills + } + return nil +} + +func (x *AuthorInput) GetLanguages() []string { + if x != nil { + return x.Languages + } + return nil +} + +func (x *AuthorInput) GetSocialLinks() *ListOfString { + if x != nil { + return x.SocialLinks + } + return nil +} + +func (x *AuthorInput) GetTeamsByProject() *ListOfListOfString { + if x != nil { + return x.TeamsByProject + } + return nil +} + +func (x *AuthorInput) GetCollaborations() *ListOfListOfString { + if x != nil { + return x.Collaborations + } + return nil +} + +func (x *AuthorInput) GetFavoriteCategories() []*CategoryInput { + if x != nil { + return x.FavoriteCategories + } + return nil +} + +func (x *AuthorInput) GetAuthorGroups() *ListOfListOfUserInput { + if x != nil { + return x.AuthorGroups + } + return nil +} + +func (x *AuthorInput) GetProjectTeams() *ListOfListOfUserInput { + if x != nil { + return x.ProjectTeams + } + return nil +} + +type NestedTypeB struct { + state protoimpl.MessageState `protogen:"open.v1"` + Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` + Name string `protobuf:"bytes,2,opt,name=name,proto3" json:"name,omitempty"` + C *NestedTypeC `protobuf:"bytes,3,opt,name=c,proto3" json:"c,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *NestedTypeB) Reset() { + *x = NestedTypeB{} + mi := &file_testdata_service_proto_msgTypes[172] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *NestedTypeB) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*NestedTypeB) ProtoMessage() {} + +func (x *NestedTypeB) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[172] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use NestedTypeB.ProtoReflect.Descriptor instead. +func (*NestedTypeB) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{172} +} + +func (x *NestedTypeB) GetId() string { + if x != nil { + return x.Id + } + return "" +} + +func (x *NestedTypeB) GetName() string { + if x != nil { + return x.Name + } + return "" +} + +func (x *NestedTypeB) GetC() *NestedTypeC { + if x != nil { + return x.C + } + return nil +} + +type NestedTypeC struct { + state protoimpl.MessageState `protogen:"open.v1"` + Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` + Name string `protobuf:"bytes,2,opt,name=name,proto3" json:"name,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *NestedTypeC) Reset() { + *x = NestedTypeC{} + mi := &file_testdata_service_proto_msgTypes[173] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *NestedTypeC) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*NestedTypeC) ProtoMessage() {} + +func (x *NestedTypeC) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[173] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use NestedTypeC.ProtoReflect.Descriptor instead. +func (*NestedTypeC) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{173} +} + +func (x *NestedTypeC) GetId() string { + if x != nil { + return x.Id + } + return "" +} + +func (x *NestedTypeC) GetName() string { + if x != nil { + return x.Name + } + return "" +} + +type FilterType struct { + state protoimpl.MessageState `protogen:"open.v1"` + Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` + FilterField_1 string `protobuf:"bytes,2,opt,name=filter_field_1,json=filterField1,proto3" json:"filter_field_1,omitempty"` + FilterField_2 string `protobuf:"bytes,3,opt,name=filter_field_2,json=filterField2,proto3" json:"filter_field_2,omitempty"` + Pagination *Pagination `protobuf:"bytes,4,opt,name=pagination,proto3" json:"pagination,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *FilterType) Reset() { + *x = FilterType{} + mi := &file_testdata_service_proto_msgTypes[174] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *FilterType) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*FilterType) ProtoMessage() {} + +func (x *FilterType) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[174] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use FilterType.ProtoReflect.Descriptor instead. +func (*FilterType) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{174} +} + +func (x *FilterType) GetName() string { + if x != nil { + return x.Name + } + return "" +} + +func (x *FilterType) GetFilterField_1() string { + if x != nil { + return x.FilterField_1 + } + return "" +} + +func (x *FilterType) GetFilterField_2() string { + if x != nil { + return x.FilterField_2 + } + return "" +} + +func (x *FilterType) GetPagination() *Pagination { + if x != nil { + return x.Pagination + } + return nil +} + +type Pagination struct { + state protoimpl.MessageState `protogen:"open.v1"` + Page int32 `protobuf:"varint,1,opt,name=page,proto3" json:"page,omitempty"` + PerPage int32 `protobuf:"varint,2,opt,name=per_page,json=perPage,proto3" json:"per_page,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *Pagination) Reset() { + *x = Pagination{} + mi := &file_testdata_service_proto_msgTypes[175] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *Pagination) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Pagination) ProtoMessage() {} + +func (x *Pagination) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[175] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use Pagination.ProtoReflect.Descriptor instead. +func (*Pagination) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{175} +} + +func (x *Pagination) GetPage() int32 { + if x != nil { + return x.Page + } + return 0 +} + +func (x *Pagination) GetPerPage() int32 { + if x != nil { + return x.PerPage + } + return 0 +} + +type OrderLineInput struct { + state protoimpl.MessageState `protogen:"open.v1"` + ProductId string `protobuf:"bytes,1,opt,name=product_id,json=productId,proto3" json:"product_id,omitempty"` + Quantity int32 `protobuf:"varint,2,opt,name=quantity,proto3" json:"quantity,omitempty"` + Modifiers *ListOfString `protobuf:"bytes,3,opt,name=modifiers,proto3" json:"modifiers,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *OrderLineInput) Reset() { + *x = OrderLineInput{} + mi := &file_testdata_service_proto_msgTypes[176] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *OrderLineInput) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*OrderLineInput) ProtoMessage() {} + +func (x *OrderLineInput) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[176] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use OrderLineInput.ProtoReflect.Descriptor instead. +func (*OrderLineInput) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{176} +} + +func (x *OrderLineInput) GetProductId() string { + if x != nil { + return x.ProductId + } + return "" +} + +func (x *OrderLineInput) GetQuantity() int32 { + if x != nil { + return x.Quantity + } + return 0 +} + +func (x *OrderLineInput) GetModifiers() *ListOfString { + if x != nil { + return x.Modifiers + } + return nil +} + +type OrderLine struct { + state protoimpl.MessageState `protogen:"open.v1"` + ProductId string `protobuf:"bytes,1,opt,name=product_id,json=productId,proto3" json:"product_id,omitempty"` + Quantity int32 `protobuf:"varint,2,opt,name=quantity,proto3" json:"quantity,omitempty"` + Modifiers *ListOfString `protobuf:"bytes,3,opt,name=modifiers,proto3" json:"modifiers,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *OrderLine) Reset() { + *x = OrderLine{} + mi := &file_testdata_service_proto_msgTypes[177] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *OrderLine) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*OrderLine) ProtoMessage() {} + +func (x *OrderLine) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[177] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use OrderLine.ProtoReflect.Descriptor instead. +func (*OrderLine) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{177} +} + +func (x *OrderLine) GetProductId() string { + if x != nil { + return x.ProductId + } + return "" +} + +func (x *OrderLine) GetQuantity() int32 { + if x != nil { + return x.Quantity + } + return 0 +} + +func (x *OrderLine) GetModifiers() *ListOfString { + if x != nil { + return x.Modifiers + } + return nil +} + +type Subcategory struct { + state protoimpl.MessageState `protogen:"open.v1"` + Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` + Name string `protobuf:"bytes,2,opt,name=name,proto3" json:"name,omitempty"` + Description *wrapperspb.StringValue `protobuf:"bytes,3,opt,name=description,proto3" json:"description,omitempty"` + IsActive bool `protobuf:"varint,4,opt,name=is_active,json=isActive,proto3" json:"is_active,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *Subcategory) Reset() { + *x = Subcategory{} + mi := &file_testdata_service_proto_msgTypes[178] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *Subcategory) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Subcategory) ProtoMessage() {} + +func (x *Subcategory) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[178] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use Subcategory.ProtoReflect.Descriptor instead. +func (*Subcategory) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{178} +} + +func (x *Subcategory) GetId() string { + if x != nil { + return x.Id + } + return "" +} + +func (x *Subcategory) GetName() string { + if x != nil { + return x.Name + } + return "" +} + +func (x *Subcategory) GetDescription() *wrapperspb.StringValue { + if x != nil { + return x.Description + } + return nil +} + +func (x *Subcategory) GetIsActive() bool { + if x != nil { + return x.IsActive + } + return false +} + +type CategoryMetrics struct { + state protoimpl.MessageState `protogen:"open.v1"` + Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` + MetricType string `protobuf:"bytes,2,opt,name=metric_type,json=metricType,proto3" json:"metric_type,omitempty"` + Value float64 `protobuf:"fixed64,3,opt,name=value,proto3" json:"value,omitempty"` + Timestamp string `protobuf:"bytes,4,opt,name=timestamp,proto3" json:"timestamp,omitempty"` + CategoryId string `protobuf:"bytes,5,opt,name=category_id,json=categoryId,proto3" json:"category_id,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *CategoryMetrics) Reset() { + *x = CategoryMetrics{} + mi := &file_testdata_service_proto_msgTypes[179] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *CategoryMetrics) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*CategoryMetrics) ProtoMessage() {} + +func (x *CategoryMetrics) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[179] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use CategoryMetrics.ProtoReflect.Descriptor instead. +func (*CategoryMetrics) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{179} +} + +func (x *CategoryMetrics) GetId() string { + if x != nil { + return x.Id + } + return "" +} + +func (x *CategoryMetrics) GetMetricType() string { + if x != nil { + return x.MetricType + } + return "" +} + +func (x *CategoryMetrics) GetValue() float64 { + if x != nil { + return x.Value + } + return 0 +} + +func (x *CategoryMetrics) GetTimestamp() string { + if x != nil { + return x.Timestamp + } + return "" +} + +func (x *CategoryMetrics) GetCategoryId() string { + if x != nil { + return x.CategoryId + } + return "" +} + +type Cat struct { + state protoimpl.MessageState `protogen:"open.v1"` + Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` + Name string `protobuf:"bytes,2,opt,name=name,proto3" json:"name,omitempty"` + Kind string `protobuf:"bytes,3,opt,name=kind,proto3" json:"kind,omitempty"` + MeowVolume int32 `protobuf:"varint,4,opt,name=meow_volume,json=meowVolume,proto3" json:"meow_volume,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *Cat) Reset() { + *x = Cat{} + mi := &file_testdata_service_proto_msgTypes[180] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *Cat) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Cat) ProtoMessage() {} + +func (x *Cat) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[180] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use Cat.ProtoReflect.Descriptor instead. +func (*Cat) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{180} +} + +func (x *Cat) GetId() string { + if x != nil { + return x.Id + } + return "" +} + +func (x *Cat) GetName() string { + if x != nil { + return x.Name + } + return "" +} + +func (x *Cat) GetKind() string { + if x != nil { + return x.Kind + } + return "" +} + +func (x *Cat) GetMeowVolume() int32 { + if x != nil { + return x.MeowVolume + } + return 0 +} + +type Dog struct { + state protoimpl.MessageState `protogen:"open.v1"` + Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` + Name string `protobuf:"bytes,2,opt,name=name,proto3" json:"name,omitempty"` + Kind string `protobuf:"bytes,3,opt,name=kind,proto3" json:"kind,omitempty"` + BarkVolume int32 `protobuf:"varint,4,opt,name=bark_volume,json=barkVolume,proto3" json:"bark_volume,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *Dog) Reset() { + *x = Dog{} + mi := &file_testdata_service_proto_msgTypes[181] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *Dog) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Dog) ProtoMessage() {} + +func (x *Dog) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[181] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use Dog.ProtoReflect.Descriptor instead. +func (*Dog) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{181} +} + +func (x *Dog) GetId() string { + if x != nil { + return x.Id + } + return "" +} + +func (x *Dog) GetName() string { + if x != nil { + return x.Name + } + return "" +} + +func (x *Dog) GetKind() string { + if x != nil { + return x.Kind + } + return "" +} + +func (x *Dog) GetBarkVolume() int32 { + if x != nil { + return x.BarkVolume + } + return 0 +} + +type ActionSuccess struct { + state protoimpl.MessageState `protogen:"open.v1"` + Message string `protobuf:"bytes,1,opt,name=message,proto3" json:"message,omitempty"` + Timestamp string `protobuf:"bytes,2,opt,name=timestamp,proto3" json:"timestamp,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ActionSuccess) Reset() { + *x = ActionSuccess{} + mi := &file_testdata_service_proto_msgTypes[182] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ActionSuccess) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ActionSuccess) ProtoMessage() {} + +func (x *ActionSuccess) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[182] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ActionSuccess.ProtoReflect.Descriptor instead. +func (*ActionSuccess) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{182} +} + +func (x *ActionSuccess) GetMessage() string { + if x != nil { + return x.Message + } + return "" +} + +func (x *ActionSuccess) GetTimestamp() string { + if x != nil { + return x.Timestamp + } + return "" +} + +type ActionError struct { + state protoimpl.MessageState `protogen:"open.v1"` + Message string `protobuf:"bytes,1,opt,name=message,proto3" json:"message,omitempty"` + Code string `protobuf:"bytes,2,opt,name=code,proto3" json:"code,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ActionError) Reset() { + *x = ActionError{} + mi := &file_testdata_service_proto_msgTypes[183] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ActionError) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ActionError) ProtoMessage() {} + +func (x *ActionError) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[183] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ActionError.ProtoReflect.Descriptor instead. +func (*ActionError) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{183} +} + +func (x *ActionError) GetMessage() string { + if x != nil { + return x.Message + } + return "" +} + +func (x *ActionError) GetCode() string { + if x != nil { + return x.Code + } + return "" +} + +type CategoryInput struct { + state protoimpl.MessageState `protogen:"open.v1"` + Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` + Kind CategoryKind `protobuf:"varint,2,opt,name=kind,proto3,enum=productv1.CategoryKind" json:"kind,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *CategoryInput) Reset() { + *x = CategoryInput{} + mi := &file_testdata_service_proto_msgTypes[184] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *CategoryInput) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*CategoryInput) ProtoMessage() {} + +func (x *CategoryInput) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[184] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use CategoryInput.ProtoReflect.Descriptor instead. +func (*CategoryInput) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{184} +} + +func (x *CategoryInput) GetName() string { + if x != nil { + return x.Name + } + return "" +} + +func (x *CategoryInput) GetKind() CategoryKind { + if x != nil { + return x.Kind + } + return CategoryKind_CATEGORY_KIND_UNSPECIFIED +} + +type ProductCountFilter struct { + state protoimpl.MessageState `protogen:"open.v1"` + MinPrice *wrapperspb.DoubleValue `protobuf:"bytes,1,opt,name=min_price,json=minPrice,proto3" json:"min_price,omitempty"` + MaxPrice *wrapperspb.DoubleValue `protobuf:"bytes,2,opt,name=max_price,json=maxPrice,proto3" json:"max_price,omitempty"` + InStock *wrapperspb.BoolValue `protobuf:"bytes,3,opt,name=in_stock,json=inStock,proto3" json:"in_stock,omitempty"` + SearchTerm *wrapperspb.StringValue `protobuf:"bytes,4,opt,name=search_term,json=searchTerm,proto3" json:"search_term,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ProductCountFilter) Reset() { + *x = ProductCountFilter{} + mi := &file_testdata_service_proto_msgTypes[185] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ProductCountFilter) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ProductCountFilter) ProtoMessage() {} + +func (x *ProductCountFilter) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[185] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ProductCountFilter.ProtoReflect.Descriptor instead. +func (*ProductCountFilter) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{185} +} + +func (x *ProductCountFilter) GetMinPrice() *wrapperspb.DoubleValue { + if x != nil { + return x.MinPrice + } + return nil +} + +func (x *ProductCountFilter) GetMaxPrice() *wrapperspb.DoubleValue { + if x != nil { + return x.MaxPrice + } + return nil +} + +func (x *ProductCountFilter) GetInStock() *wrapperspb.BoolValue { + if x != nil { + return x.InStock + } + return nil +} + +func (x *ProductCountFilter) GetSearchTerm() *wrapperspb.StringValue { + if x != nil { + return x.SearchTerm + } + return nil +} + +type SubcategoryItemFilter struct { + state protoimpl.MessageState `protogen:"open.v1"` + MinPrice *wrapperspb.DoubleValue `protobuf:"bytes,1,opt,name=min_price,json=minPrice,proto3" json:"min_price,omitempty"` + MaxPrice *wrapperspb.DoubleValue `protobuf:"bytes,2,opt,name=max_price,json=maxPrice,proto3" json:"max_price,omitempty"` + InStock *wrapperspb.BoolValue `protobuf:"bytes,3,opt,name=in_stock,json=inStock,proto3" json:"in_stock,omitempty"` + IsActive *wrapperspb.BoolValue `protobuf:"bytes,4,opt,name=is_active,json=isActive,proto3" json:"is_active,omitempty"` + SearchTerm *wrapperspb.StringValue `protobuf:"bytes,5,opt,name=search_term,json=searchTerm,proto3" json:"search_term,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *SubcategoryItemFilter) Reset() { + *x = SubcategoryItemFilter{} + mi := &file_testdata_service_proto_msgTypes[186] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *SubcategoryItemFilter) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*SubcategoryItemFilter) ProtoMessage() {} + +func (x *SubcategoryItemFilter) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[186] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use SubcategoryItemFilter.ProtoReflect.Descriptor instead. +func (*SubcategoryItemFilter) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{186} +} + +func (x *SubcategoryItemFilter) GetMinPrice() *wrapperspb.DoubleValue { + if x != nil { + return x.MinPrice + } + return nil +} + +func (x *SubcategoryItemFilter) GetMaxPrice() *wrapperspb.DoubleValue { + if x != nil { + return x.MaxPrice + } + return nil +} + +func (x *SubcategoryItemFilter) GetInStock() *wrapperspb.BoolValue { + if x != nil { + return x.InStock + } + return nil +} + +func (x *SubcategoryItemFilter) GetIsActive() *wrapperspb.BoolValue { + if x != nil { + return x.IsActive + } + return nil +} + +func (x *SubcategoryItemFilter) GetSearchTerm() *wrapperspb.StringValue { + if x != nil { + return x.SearchTerm + } + return nil +} + +type ShippingEstimateInput struct { + state protoimpl.MessageState `protogen:"open.v1"` + Destination ShippingDestination `protobuf:"varint,1,opt,name=destination,proto3,enum=productv1.ShippingDestination" json:"destination,omitempty"` + Weight float64 `protobuf:"fixed64,2,opt,name=weight,proto3" json:"weight,omitempty"` + Expedited *wrapperspb.BoolValue `protobuf:"bytes,3,opt,name=expedited,proto3" json:"expedited,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ShippingEstimateInput) Reset() { + *x = ShippingEstimateInput{} + mi := &file_testdata_service_proto_msgTypes[187] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ShippingEstimateInput) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ShippingEstimateInput) ProtoMessage() {} + +func (x *ShippingEstimateInput) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[187] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ShippingEstimateInput.ProtoReflect.Descriptor instead. +func (*ShippingEstimateInput) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{187} +} + +func (x *ShippingEstimateInput) GetDestination() ShippingDestination { + if x != nil { + return x.Destination + } + return ShippingDestination_SHIPPING_DESTINATION_UNSPECIFIED +} + +func (x *ShippingEstimateInput) GetWeight() float64 { + if x != nil { + return x.Weight + } + return 0 +} + +func (x *ShippingEstimateInput) GetExpedited() *wrapperspb.BoolValue { + if x != nil { + return x.Expedited + } + return nil +} + +type ListOfAuthorFilter_List struct { + state protoimpl.MessageState `protogen:"open.v1"` + Items []*AuthorFilter `protobuf:"bytes,1,rep,name=items,proto3" json:"items,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ListOfAuthorFilter_List) Reset() { + *x = ListOfAuthorFilter_List{} + mi := &file_testdata_service_proto_msgTypes[188] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ListOfAuthorFilter_List) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ListOfAuthorFilter_List) ProtoMessage() {} + +func (x *ListOfAuthorFilter_List) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[188] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ListOfAuthorFilter_List.ProtoReflect.Descriptor instead. +func (*ListOfAuthorFilter_List) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{0, 0} +} + +func (x *ListOfAuthorFilter_List) GetItems() []*AuthorFilter { + if x != nil { + return x.Items + } + return nil +} + +type ListOfAuthorInput_List struct { + state protoimpl.MessageState `protogen:"open.v1"` + Items []*AuthorInput `protobuf:"bytes,1,rep,name=items,proto3" json:"items,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ListOfAuthorInput_List) Reset() { + *x = ListOfAuthorInput_List{} + mi := &file_testdata_service_proto_msgTypes[189] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ListOfAuthorInput_List) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ListOfAuthorInput_List) ProtoMessage() {} + +func (x *ListOfAuthorInput_List) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[189] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ListOfAuthorInput_List.ProtoReflect.Descriptor instead. +func (*ListOfAuthorInput_List) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{1, 0} +} + +func (x *ListOfAuthorInput_List) GetItems() []*AuthorInput { + if x != nil { + return x.Items + } + return nil +} + +type ListOfBlogPost_List struct { + state protoimpl.MessageState `protogen:"open.v1"` + Items []*BlogPost `protobuf:"bytes,1,rep,name=items,proto3" json:"items,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ListOfBlogPost_List) Reset() { + *x = ListOfBlogPost_List{} + mi := &file_testdata_service_proto_msgTypes[190] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ListOfBlogPost_List) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ListOfBlogPost_List) ProtoMessage() {} + +func (x *ListOfBlogPost_List) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[190] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ListOfBlogPost_List.ProtoReflect.Descriptor instead. +func (*ListOfBlogPost_List) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{2, 0} +} + +func (x *ListOfBlogPost_List) GetItems() []*BlogPost { + if x != nil { + return x.Items + } + return nil +} + +type ListOfBlogPostFilter_List struct { + state protoimpl.MessageState `protogen:"open.v1"` + Items []*BlogPostFilter `protobuf:"bytes,1,rep,name=items,proto3" json:"items,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ListOfBlogPostFilter_List) Reset() { + *x = ListOfBlogPostFilter_List{} + mi := &file_testdata_service_proto_msgTypes[191] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ListOfBlogPostFilter_List) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ListOfBlogPostFilter_List) ProtoMessage() {} + +func (x *ListOfBlogPostFilter_List) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[191] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ListOfBlogPostFilter_List.ProtoReflect.Descriptor instead. +func (*ListOfBlogPostFilter_List) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{3, 0} +} + +func (x *ListOfBlogPostFilter_List) GetItems() []*BlogPostFilter { + if x != nil { + return x.Items + } + return nil +} + +type ListOfBlogPostInput_List struct { + state protoimpl.MessageState `protogen:"open.v1"` + Items []*BlogPostInput `protobuf:"bytes,1,rep,name=items,proto3" json:"items,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ListOfBlogPostInput_List) Reset() { + *x = ListOfBlogPostInput_List{} + mi := &file_testdata_service_proto_msgTypes[192] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ListOfBlogPostInput_List) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ListOfBlogPostInput_List) ProtoMessage() {} + +func (x *ListOfBlogPostInput_List) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[192] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ListOfBlogPostInput_List.ProtoReflect.Descriptor instead. +func (*ListOfBlogPostInput_List) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{4, 0} +} + +func (x *ListOfBlogPostInput_List) GetItems() []*BlogPostInput { + if x != nil { + return x.Items + } + return nil +} + +type ListOfBoolean_List struct { + state protoimpl.MessageState `protogen:"open.v1"` + Items []bool `protobuf:"varint,1,rep,packed,name=items,proto3" json:"items,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ListOfBoolean_List) Reset() { + *x = ListOfBoolean_List{} + mi := &file_testdata_service_proto_msgTypes[193] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ListOfBoolean_List) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ListOfBoolean_List) ProtoMessage() {} + +func (x *ListOfBoolean_List) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[193] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ListOfBoolean_List.ProtoReflect.Descriptor instead. +func (*ListOfBoolean_List) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{5, 0} +} + +func (x *ListOfBoolean_List) GetItems() []bool { + if x != nil { + return x.Items + } + return nil +} + +type ListOfCategory_List struct { + state protoimpl.MessageState `protogen:"open.v1"` + Items []*Category `protobuf:"bytes,1,rep,name=items,proto3" json:"items,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ListOfCategory_List) Reset() { + *x = ListOfCategory_List{} + mi := &file_testdata_service_proto_msgTypes[194] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ListOfCategory_List) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ListOfCategory_List) ProtoMessage() {} + +func (x *ListOfCategory_List) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[194] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ListOfCategory_List.ProtoReflect.Descriptor instead. +func (*ListOfCategory_List) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{6, 0} +} + +func (x *ListOfCategory_List) GetItems() []*Category { + if x != nil { + return x.Items + } + return nil +} + +type ListOfCategoryInput_List struct { + state protoimpl.MessageState `protogen:"open.v1"` + Items []*CategoryInput `protobuf:"bytes,1,rep,name=items,proto3" json:"items,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ListOfCategoryInput_List) Reset() { + *x = ListOfCategoryInput_List{} + mi := &file_testdata_service_proto_msgTypes[195] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ListOfCategoryInput_List) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ListOfCategoryInput_List) ProtoMessage() {} + +func (x *ListOfCategoryInput_List) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[195] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ListOfCategoryInput_List.ProtoReflect.Descriptor instead. +func (*ListOfCategoryInput_List) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{7, 0} +} + +func (x *ListOfCategoryInput_List) GetItems() []*CategoryInput { + if x != nil { + return x.Items + } + return nil +} + +type ListOfFloat_List struct { + state protoimpl.MessageState `protogen:"open.v1"` + Items []float64 `protobuf:"fixed64,1,rep,packed,name=items,proto3" json:"items,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ListOfFloat_List) Reset() { + *x = ListOfFloat_List{} + mi := &file_testdata_service_proto_msgTypes[196] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ListOfFloat_List) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ListOfFloat_List) ProtoMessage() {} + +func (x *ListOfFloat_List) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[196] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ListOfFloat_List.ProtoReflect.Descriptor instead. +func (*ListOfFloat_List) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{8, 0} +} + +func (x *ListOfFloat_List) GetItems() []float64 { + if x != nil { + return x.Items + } + return nil +} + +type ListOfListOfCategory_List struct { + state protoimpl.MessageState `protogen:"open.v1"` + Items []*ListOfCategory `protobuf:"bytes,1,rep,name=items,proto3" json:"items,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ListOfListOfCategory_List) Reset() { + *x = ListOfListOfCategory_List{} + mi := &file_testdata_service_proto_msgTypes[197] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ListOfListOfCategory_List) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ListOfListOfCategory_List) ProtoMessage() {} + +func (x *ListOfListOfCategory_List) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[197] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ListOfListOfCategory_List.ProtoReflect.Descriptor instead. +func (*ListOfListOfCategory_List) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{9, 0} +} + +func (x *ListOfListOfCategory_List) GetItems() []*ListOfCategory { + if x != nil { + return x.Items + } + return nil +} + +type ListOfListOfCategoryInput_List struct { + state protoimpl.MessageState `protogen:"open.v1"` + Items []*ListOfCategoryInput `protobuf:"bytes,1,rep,name=items,proto3" json:"items,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ListOfListOfCategoryInput_List) Reset() { + *x = ListOfListOfCategoryInput_List{} + mi := &file_testdata_service_proto_msgTypes[198] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ListOfListOfCategoryInput_List) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ListOfListOfCategoryInput_List) ProtoMessage() {} + +func (x *ListOfListOfCategoryInput_List) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[198] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ListOfListOfCategoryInput_List.ProtoReflect.Descriptor instead. +func (*ListOfListOfCategoryInput_List) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{10, 0} +} + +func (x *ListOfListOfCategoryInput_List) GetItems() []*ListOfCategoryInput { + if x != nil { + return x.Items + } + return nil +} + +type ListOfListOfString_List struct { + state protoimpl.MessageState `protogen:"open.v1"` + Items []*ListOfString `protobuf:"bytes,1,rep,name=items,proto3" json:"items,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ListOfListOfString_List) Reset() { + *x = ListOfListOfString_List{} + mi := &file_testdata_service_proto_msgTypes[199] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ListOfListOfString_List) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ListOfListOfString_List) ProtoMessage() {} + +func (x *ListOfListOfString_List) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[199] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ListOfListOfString_List.ProtoReflect.Descriptor instead. +func (*ListOfListOfString_List) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{11, 0} +} + +func (x *ListOfListOfString_List) GetItems() []*ListOfString { + if x != nil { + return x.Items + } + return nil +} + +type ListOfListOfUser_List struct { + state protoimpl.MessageState `protogen:"open.v1"` + Items []*ListOfUser `protobuf:"bytes,1,rep,name=items,proto3" json:"items,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ListOfListOfUser_List) Reset() { + *x = ListOfListOfUser_List{} + mi := &file_testdata_service_proto_msgTypes[200] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ListOfListOfUser_List) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ListOfListOfUser_List) ProtoMessage() {} + +func (x *ListOfListOfUser_List) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[200] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ListOfListOfUser_List.ProtoReflect.Descriptor instead. +func (*ListOfListOfUser_List) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{12, 0} +} + +func (x *ListOfListOfUser_List) GetItems() []*ListOfUser { + if x != nil { + return x.Items + } + return nil +} + +type ListOfListOfUserInput_List struct { + state protoimpl.MessageState `protogen:"open.v1"` + Items []*ListOfUserInput `protobuf:"bytes,1,rep,name=items,proto3" json:"items,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ListOfListOfUserInput_List) Reset() { + *x = ListOfListOfUserInput_List{} + mi := &file_testdata_service_proto_msgTypes[201] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ListOfListOfUserInput_List) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ListOfListOfUserInput_List) ProtoMessage() {} + +func (x *ListOfListOfUserInput_List) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[201] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ListOfListOfUserInput_List.ProtoReflect.Descriptor instead. +func (*ListOfListOfUserInput_List) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{13, 0} +} + +func (x *ListOfListOfUserInput_List) GetItems() []*ListOfUserInput { + if x != nil { + return x.Items + } + return nil +} + +type ListOfOrderLine_List struct { + state protoimpl.MessageState `protogen:"open.v1"` + Items []*OrderLine `protobuf:"bytes,1,rep,name=items,proto3" json:"items,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ListOfOrderLine_List) Reset() { + *x = ListOfOrderLine_List{} + mi := &file_testdata_service_proto_msgTypes[202] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ListOfOrderLine_List) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ListOfOrderLine_List) ProtoMessage() {} + +func (x *ListOfOrderLine_List) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[202] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ListOfOrderLine_List.ProtoReflect.Descriptor instead. +func (*ListOfOrderLine_List) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{14, 0} +} + +func (x *ListOfOrderLine_List) GetItems() []*OrderLine { + if x != nil { + return x.Items + } + return nil +} + +type ListOfProduct_List struct { + state protoimpl.MessageState `protogen:"open.v1"` + Items []*Product `protobuf:"bytes,1,rep,name=items,proto3" json:"items,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ListOfProduct_List) Reset() { + *x = ListOfProduct_List{} + mi := &file_testdata_service_proto_msgTypes[203] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ListOfProduct_List) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ListOfProduct_List) ProtoMessage() {} + +func (x *ListOfProduct_List) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[203] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ListOfProduct_List.ProtoReflect.Descriptor instead. +func (*ListOfProduct_List) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{15, 0} +} + +func (x *ListOfProduct_List) GetItems() []*Product { + if x != nil { + return x.Items + } + return nil +} + +type ListOfString_List struct { + state protoimpl.MessageState `protogen:"open.v1"` + Items []string `protobuf:"bytes,1,rep,name=items,proto3" json:"items,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ListOfString_List) Reset() { + *x = ListOfString_List{} + mi := &file_testdata_service_proto_msgTypes[204] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ListOfString_List) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ListOfString_List) ProtoMessage() {} + +func (x *ListOfString_List) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[204] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ListOfString_List.ProtoReflect.Descriptor instead. +func (*ListOfString_List) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{16, 0} +} + +func (x *ListOfString_List) GetItems() []string { + if x != nil { + return x.Items + } + return nil +} + +type ListOfSubcategory_List struct { + state protoimpl.MessageState `protogen:"open.v1"` + Items []*Subcategory `protobuf:"bytes,1,rep,name=items,proto3" json:"items,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ListOfSubcategory_List) Reset() { + *x = ListOfSubcategory_List{} + mi := &file_testdata_service_proto_msgTypes[205] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ListOfSubcategory_List) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ListOfSubcategory_List) ProtoMessage() {} + +func (x *ListOfSubcategory_List) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[205] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ListOfSubcategory_List.ProtoReflect.Descriptor instead. +func (*ListOfSubcategory_List) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{17, 0} +} + +func (x *ListOfSubcategory_List) GetItems() []*Subcategory { + if x != nil { + return x.Items + } + return nil +} + +type ListOfUser_List struct { + state protoimpl.MessageState `protogen:"open.v1"` + Items []*User `protobuf:"bytes,1,rep,name=items,proto3" json:"items,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ListOfUser_List) Reset() { + *x = ListOfUser_List{} + mi := &file_testdata_service_proto_msgTypes[206] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ListOfUser_List) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ListOfUser_List) ProtoMessage() {} + +func (x *ListOfUser_List) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[206] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ListOfUser_List.ProtoReflect.Descriptor instead. +func (*ListOfUser_List) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{18, 0} +} + +func (x *ListOfUser_List) GetItems() []*User { + if x != nil { + return x.Items + } + return nil +} + +type ListOfUserInput_List struct { + state protoimpl.MessageState `protogen:"open.v1"` + Items []*UserInput `protobuf:"bytes,1,rep,name=items,proto3" json:"items,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ListOfUserInput_List) Reset() { + *x = ListOfUserInput_List{} + mi := &file_testdata_service_proto_msgTypes[207] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ListOfUserInput_List) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ListOfUserInput_List) ProtoMessage() {} + +func (x *ListOfUserInput_List) ProtoReflect() protoreflect.Message { + mi := &file_testdata_service_proto_msgTypes[207] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ListOfUserInput_List.ProtoReflect.Descriptor instead. +func (*ListOfUserInput_List) Descriptor() ([]byte, []int) { + return file_testdata_service_proto_rawDescGZIP(), []int{19, 0} +} + +func (x *ListOfUserInput_List) GetItems() []*UserInput { + if x != nil { + return x.Items + } + return nil +} + +var File_testdata_service_proto protoreflect.FileDescriptor + +const file_testdata_service_proto_rawDesc = "" + + "\n" + + "\x16testdata/service.proto\x12\tproductv1\x1a\x1egoogle/protobuf/wrappers.proto\"\x83\x01\n" + + "\x12ListOfAuthorFilter\x126\n" + + "\x04list\x18\x01 \x01(\v2\".productv1.ListOfAuthorFilter.ListR\x04list\x1a5\n" + + "\x04List\x12-\n" + + "\x05items\x18\x01 \x03(\v2\x17.productv1.AuthorFilterR\x05items\"\x80\x01\n" + + "\x11ListOfAuthorInput\x125\n" + + "\x04list\x18\x01 \x01(\v2!.productv1.ListOfAuthorInput.ListR\x04list\x1a4\n" + + "\x04List\x12,\n" + + "\x05items\x18\x01 \x03(\v2\x16.productv1.AuthorInputR\x05items\"w\n" + + "\x0eListOfBlogPost\x122\n" + + "\x04list\x18\x01 \x01(\v2\x1e.productv1.ListOfBlogPost.ListR\x04list\x1a1\n" + + "\x04List\x12)\n" + + "\x05items\x18\x01 \x03(\v2\x13.productv1.BlogPostR\x05items\"\x89\x01\n" + + "\x14ListOfBlogPostFilter\x128\n" + + "\x04list\x18\x01 \x01(\v2$.productv1.ListOfBlogPostFilter.ListR\x04list\x1a7\n" + + "\x04List\x12/\n" + + "\x05items\x18\x01 \x03(\v2\x19.productv1.BlogPostFilterR\x05items\"\x86\x01\n" + + "\x13ListOfBlogPostInput\x127\n" + + "\x04list\x18\x01 \x01(\v2#.productv1.ListOfBlogPostInput.ListR\x04list\x1a6\n" + + "\x04List\x12.\n" + + "\x05items\x18\x01 \x03(\v2\x18.productv1.BlogPostInputR\x05items\"`\n" + + "\rListOfBoolean\x121\n" + + "\x04list\x18\x01 \x01(\v2\x1d.productv1.ListOfBoolean.ListR\x04list\x1a\x1c\n" + + "\x04List\x12\x14\n" + + "\x05items\x18\x01 \x03(\bR\x05items\"w\n" + + "\x0eListOfCategory\x122\n" + + "\x04list\x18\x01 \x01(\v2\x1e.productv1.ListOfCategory.ListR\x04list\x1a1\n" + + "\x04List\x12)\n" + + "\x05items\x18\x01 \x03(\v2\x13.productv1.CategoryR\x05items\"\x86\x01\n" + + "\x13ListOfCategoryInput\x127\n" + + "\x04list\x18\x01 \x01(\v2#.productv1.ListOfCategoryInput.ListR\x04list\x1a6\n" + + "\x04List\x12.\n" + + "\x05items\x18\x01 \x03(\v2\x18.productv1.CategoryInputR\x05items\"\\\n" + + "\vListOfFloat\x12/\n" + + "\x04list\x18\x01 \x01(\v2\x1b.productv1.ListOfFloat.ListR\x04list\x1a\x1c\n" + + "\x04List\x12\x14\n" + + "\x05items\x18\x01 \x03(\x01R\x05items\"\x89\x01\n" + + "\x14ListOfListOfCategory\x128\n" + + "\x04list\x18\x01 \x01(\v2$.productv1.ListOfListOfCategory.ListR\x04list\x1a7\n" + + "\x04List\x12/\n" + + "\x05items\x18\x01 \x03(\v2\x19.productv1.ListOfCategoryR\x05items\"\x98\x01\n" + + "\x19ListOfListOfCategoryInput\x12=\n" + + "\x04list\x18\x01 \x01(\v2).productv1.ListOfListOfCategoryInput.ListR\x04list\x1a<\n" + + "\x04List\x124\n" + + "\x05items\x18\x01 \x03(\v2\x1e.productv1.ListOfCategoryInputR\x05items\"\x83\x01\n" + + "\x12ListOfListOfString\x126\n" + + "\x04list\x18\x01 \x01(\v2\".productv1.ListOfListOfString.ListR\x04list\x1a5\n" + + "\x04List\x12-\n" + + "\x05items\x18\x01 \x03(\v2\x17.productv1.ListOfStringR\x05items\"}\n" + + "\x10ListOfListOfUser\x124\n" + + "\x04list\x18\x01 \x01(\v2 .productv1.ListOfListOfUser.ListR\x04list\x1a3\n" + + "\x04List\x12+\n" + + "\x05items\x18\x01 \x03(\v2\x15.productv1.ListOfUserR\x05items\"\x8c\x01\n" + + "\x15ListOfListOfUserInput\x129\n" + + "\x04list\x18\x01 \x01(\v2%.productv1.ListOfListOfUserInput.ListR\x04list\x1a8\n" + + "\x04List\x120\n" + + "\x05items\x18\x01 \x03(\v2\x1a.productv1.ListOfUserInputR\x05items\"z\n" + + "\x0fListOfOrderLine\x123\n" + + "\x04list\x18\x01 \x01(\v2\x1f.productv1.ListOfOrderLine.ListR\x04list\x1a2\n" + + "\x04List\x12*\n" + + "\x05items\x18\x01 \x03(\v2\x14.productv1.OrderLineR\x05items\"t\n" + + "\rListOfProduct\x121\n" + + "\x04list\x18\x01 \x01(\v2\x1d.productv1.ListOfProduct.ListR\x04list\x1a0\n" + + "\x04List\x12(\n" + + "\x05items\x18\x01 \x03(\v2\x12.productv1.ProductR\x05items\"^\n" + + "\fListOfString\x120\n" + + "\x04list\x18\x01 \x01(\v2\x1c.productv1.ListOfString.ListR\x04list\x1a\x1c\n" + + "\x04List\x12\x14\n" + + "\x05items\x18\x01 \x03(\tR\x05items\"\x80\x01\n" + + "\x11ListOfSubcategory\x125\n" + + "\x04list\x18\x01 \x01(\v2!.productv1.ListOfSubcategory.ListR\x04list\x1a4\n" + + "\x04List\x12,\n" + + "\x05items\x18\x01 \x03(\v2\x16.productv1.SubcategoryR\x05items\"k\n" + + "\n" + + "ListOfUser\x12.\n" + + "\x04list\x18\x01 \x01(\v2\x1a.productv1.ListOfUser.ListR\x04list\x1a-\n" + + "\x04List\x12%\n" + + "\x05items\x18\x01 \x03(\v2\x0f.productv1.UserR\x05items\"z\n" + + "\x0fListOfUserInput\x123\n" + + "\x04list\x18\x01 \x01(\v2\x1f.productv1.ListOfUserInput.ListR\x04list\x1a2\n" + + "\x04List\x12*\n" + + "\x05items\x18\x01 \x03(\v2\x14.productv1.UserInputR\x05items\"-\n" + + "\x1bLookupProductByIdRequestKey\x12\x0e\n" + + "\x02id\x18\x01 \x01(\tR\x02id\"V\n" + + "\x18LookupProductByIdRequest\x12:\n" + + "\x04keys\x18\x01 \x03(\v2&.productv1.LookupProductByIdRequestKeyR\x04keys\"G\n" + + "\x19LookupProductByIdResponse\x12*\n" + + "\x06result\x18\x01 \x03(\v2\x12.productv1.ProductR\x06result\"-\n" + + "\x1bLookupStorageByIdRequestKey\x12\x0e\n" + + "\x02id\x18\x01 \x01(\tR\x02id\"V\n" + + "\x18LookupStorageByIdRequest\x12:\n" + + "\x04keys\x18\x01 \x03(\v2&.productv1.LookupStorageByIdRequestKeyR\x04keys\"G\n" + + "\x19LookupStorageByIdResponse\x12*\n" + + "\x06result\x18\x01 \x03(\v2\x12.productv1.StorageR\x06result\"/\n" + + "\x1dLookupWarehouseByIdRequestKey\x12\x0e\n" + + "\x02id\x18\x01 \x01(\tR\x02id\"Z\n" + + "\x1aLookupWarehouseByIdRequest\x12<\n" + + "\x04keys\x18\x01 \x03(\v2(.productv1.LookupWarehouseByIdRequestKeyR\x04keys\"K\n" + + "\x1bLookupWarehouseByIdResponse\x12,\n" + + "\x06result\x18\x01 \x03(\v2\x14.productv1.WarehouseR\x06result\"\x13\n" + + "\x11QueryUsersRequest\";\n" + + "\x12QueryUsersResponse\x12%\n" + + "\x05users\x18\x01 \x03(\v2\x0f.productv1.UserR\x05users\"\"\n" + + "\x10QueryUserRequest\x12\x0e\n" + + "\x02id\x18\x01 \x01(\tR\x02id\"8\n" + + "\x11QueryUserResponse\x12#\n" + + "\x04user\x18\x01 \x01(\v2\x0f.productv1.UserR\x04user\"\x18\n" + + "\x16QueryNestedTypeRequest\"R\n" + + "\x17QueryNestedTypeResponse\x127\n" + + "\vnested_type\x18\x01 \x03(\v2\x16.productv1.NestedTypeAR\n" + + "nestedType\"\x1b\n" + + "\x19QueryRecursiveTypeRequest\"]\n" + + "\x1aQueryRecursiveTypeResponse\x12?\n" + + "\x0erecursive_type\x18\x01 \x01(\v2\x18.productv1.RecursiveTypeR\rrecursiveType\"q\n" + + "#QueryTypeFilterWithArgumentsRequest\x12$\n" + + "\x0efilter_field_1\x18\x01 \x01(\tR\ffilterField1\x12$\n" + + "\x0efilter_field_2\x18\x02 \x01(\tR\ffilterField2\"\x8c\x01\n" + + "$QueryTypeFilterWithArgumentsResponse\x12d\n" + + "\x1atype_filter_with_arguments\x18\x01 \x03(\v2'.productv1.TypeWithMultipleFilterFieldsR\x17typeFilterWithArguments\"^\n" + + "(QueryTypeWithMultipleFilterFieldsRequest\x122\n" + + "\x06filter\x18\x01 \x01(\v2\x1a.productv1.FilterTypeInputR\x06filter\"\x9c\x01\n" + + ")QueryTypeWithMultipleFilterFieldsResponse\x12o\n" + + " type_with_multiple_filter_fields\x18\x01 \x03(\v2'.productv1.TypeWithMultipleFilterFieldsR\x1ctypeWithMultipleFilterFields\"Z\n" + + "\x1dQueryComplexFilterTypeRequest\x129\n" + + "\x06filter\x18\x01 \x01(\v2!.productv1.ComplexFilterTypeInputR\x06filter\"w\n" + + "\x1eQueryComplexFilterTypeResponse\x12U\n" + + "\x13complex_filter_type\x18\x01 \x03(\v2%.productv1.TypeWithComplexFilterInputR\x11complexFilterType\"L\n" + + "\x1bQueryCalculateTotalsRequest\x12-\n" + + "\x06orders\x18\x01 \x03(\v2\x15.productv1.OrderInputR\x06orders\"[\n" + + "\x1cQueryCalculateTotalsResponse\x12;\n" + + "\x10calculate_totals\x18\x01 \x03(\v2\x10.productv1.OrderR\x0fcalculateTotals\"\x18\n" + + "\x16QueryCategoriesRequest\"N\n" + + "\x17QueryCategoriesResponse\x123\n" + + "\n" + + "categories\x18\x01 \x03(\v2\x13.productv1.CategoryR\n" + + "categories\"K\n" + + "\x1cQueryCategoriesByKindRequest\x12+\n" + + "\x04kind\x18\x01 \x01(\x0e2\x17.productv1.CategoryKindR\x04kind\"b\n" + + "\x1dQueryCategoriesByKindResponse\x12A\n" + + "\x12categories_by_kind\x18\x01 \x03(\v2\x13.productv1.CategoryR\x10categoriesByKind\"N\n" + + "\x1dQueryCategoriesByKindsRequest\x12-\n" + + "\x05kinds\x18\x01 \x03(\x0e2\x17.productv1.CategoryKindR\x05kinds\"e\n" + + "\x1eQueryCategoriesByKindsResponse\x12C\n" + + "\x13categories_by_kinds\x18\x01 \x03(\v2\x13.productv1.CategoryR\x11categoriesByKinds\"Q\n" + + "\x1cQueryFilterCategoriesRequest\x121\n" + + "\x06filter\x18\x01 \x01(\v2\x19.productv1.CategoryFilterR\x06filter\"a\n" + + "\x1dQueryFilterCategoriesResponse\x12@\n" + + "\x11filter_categories\x18\x01 \x03(\v2\x13.productv1.CategoryR\x10filterCategories\"\x17\n" + + "\x15QueryRandomPetRequest\"J\n" + + "\x16QueryRandomPetResponse\x120\n" + + "\n" + + "random_pet\x18\x01 \x01(\v2\x11.productv1.AnimalR\trandomPet\"\x15\n" + + "\x13QueryAllPetsRequest\"D\n" + + "\x14QueryAllPetsResponse\x12,\n" + + "\ball_pets\x18\x01 \x03(\v2\x11.productv1.AnimalR\aallPets\"B\n" + + "\x12QuerySearchRequest\x12,\n" + + "\x05input\x18\x01 \x01(\v2\x16.productv1.SearchInputR\x05input\"F\n" + + "\x13QuerySearchResponse\x12/\n" + + "\x06search\x18\x01 \x03(\v2\x17.productv1.SearchResultR\x06search\" \n" + + "\x1eQueryRandomSearchResultRequest\"l\n" + + "\x1fQueryRandomSearchResultResponse\x12I\n" + + "\x14random_search_result\x18\x01 \x01(\v2\x17.productv1.SearchResultR\x12randomSearchResult\" \n" + + "\x1eQueryNullableFieldsTypeRequest\"r\n" + + "\x1fQueryNullableFieldsTypeResponse\x12O\n" + + "\x14nullable_fields_type\x18\x01 \x01(\v2\x1d.productv1.NullableFieldsTypeR\x12nullableFieldsType\"4\n" + + "\"QueryNullableFieldsTypeByIdRequest\x12\x0e\n" + + "\x02id\x18\x01 \x01(\tR\x02id\"\x80\x01\n" + + "#QueryNullableFieldsTypeByIdResponse\x12Y\n" + + "\x1anullable_fields_type_by_id\x18\x01 \x01(\v2\x1d.productv1.NullableFieldsTypeR\x16nullableFieldsTypeById\"c\n" + + "(QueryNullableFieldsTypeWithFilterRequest\x127\n" + + "\x06filter\x18\x01 \x01(\v2\x1f.productv1.NullableFieldsFilterR\x06filter\"\x92\x01\n" + + ")QueryNullableFieldsTypeWithFilterResponse\x12e\n" + + " nullable_fields_type_with_filter\x18\x01 \x03(\v2\x1d.productv1.NullableFieldsTypeR\x1cnullableFieldsTypeWithFilter\"$\n" + + "\"QueryAllNullableFieldsTypesRequest\"\x7f\n" + + "#QueryAllNullableFieldsTypesResponse\x12X\n" + + "\x19all_nullable_fields_types\x18\x01 \x03(\v2\x1d.productv1.NullableFieldsTypeR\x16allNullableFieldsTypes\"\x16\n" + + "\x14QueryBlogPostRequest\"I\n" + + "\x15QueryBlogPostResponse\x120\n" + + "\tblog_post\x18\x01 \x01(\v2\x13.productv1.BlogPostR\bblogPost\"*\n" + + "\x18QueryBlogPostByIdRequest\x12\x0e\n" + + "\x02id\x18\x01 \x01(\tR\x02id\"W\n" + + "\x19QueryBlogPostByIdResponse\x12:\n" + + "\x0fblog_post_by_id\x18\x01 \x01(\v2\x13.productv1.BlogPostR\fblogPostById\"T\n" + + "\x1fQueryBlogPostsWithFilterRequest\x121\n" + + "\x06filter\x18\x01 \x01(\v2\x19.productv1.BlogPostFilterR\x06filter\"l\n" + + " QueryBlogPostsWithFilterResponse\x12H\n" + + "\x16blog_posts_with_filter\x18\x01 \x03(\v2\x13.productv1.BlogPostR\x13blogPostsWithFilter\"\x1a\n" + + "\x18QueryAllBlogPostsRequest\"V\n" + + "\x19QueryAllBlogPostsResponse\x129\n" + + "\x0eall_blog_posts\x18\x01 \x03(\v2\x13.productv1.BlogPostR\fallBlogPosts\"\x14\n" + + "\x12QueryAuthorRequest\"@\n" + + "\x13QueryAuthorResponse\x12)\n" + + "\x06author\x18\x01 \x01(\v2\x11.productv1.AuthorR\x06author\"(\n" + + "\x16QueryAuthorByIdRequest\x12\x0e\n" + + "\x02id\x18\x01 \x01(\tR\x02id\"N\n" + + "\x17QueryAuthorByIdResponse\x123\n" + + "\fauthor_by_id\x18\x01 \x01(\v2\x11.productv1.AuthorR\n" + + "authorById\"P\n" + + "\x1dQueryAuthorsWithFilterRequest\x12/\n" + + "\x06filter\x18\x01 \x01(\v2\x17.productv1.AuthorFilterR\x06filter\"c\n" + + "\x1eQueryAuthorsWithFilterResponse\x12A\n" + + "\x13authors_with_filter\x18\x01 \x03(\v2\x11.productv1.AuthorR\x11authorsWithFilter\"\x18\n" + + "\x16QueryAllAuthorsRequest\"M\n" + + "\x17QueryAllAuthorsResponse\x122\n" + + "\vall_authors\x18\x01 \x03(\v2\x11.productv1.AuthorR\n" + + "allAuthors\"X\n" + + "\x1dQueryBulkSearchAuthorsRequest\x127\n" + + "\afilters\x18\x01 \x01(\v2\x1d.productv1.ListOfAuthorFilterR\afilters\"c\n" + + "\x1eQueryBulkSearchAuthorsResponse\x12A\n" + + "\x13bulk_search_authors\x18\x01 \x03(\v2\x11.productv1.AuthorR\x11bulkSearchAuthors\"\\\n" + + "\x1fQueryBulkSearchBlogPostsRequest\x129\n" + + "\afilters\x18\x01 \x01(\v2\x1f.productv1.ListOfBlogPostFilterR\afilters\"l\n" + + " QueryBulkSearchBlogPostsResponse\x12H\n" + + "\x16bulk_search_blog_posts\x18\x01 \x03(\v2\x13.productv1.BlogPostR\x13bulkSearchBlogPosts\"G\n" + + "\x19MutationCreateUserRequest\x12*\n" + + "\x05input\x18\x01 \x01(\v2\x14.productv1.UserInputR\x05input\"N\n" + + "\x1aMutationCreateUserResponse\x120\n" + + "\vcreate_user\x18\x01 \x01(\v2\x0f.productv1.UserR\n" + + "createUser\"L\n" + + "\x1cMutationPerformActionRequest\x12,\n" + + "\x05input\x18\x01 \x01(\v2\x16.productv1.ActionInputR\x05input\"_\n" + + "\x1dMutationPerformActionResponse\x12>\n" + + "\x0eperform_action\x18\x01 \x01(\v2\x17.productv1.ActionResultR\rperformAction\"_\n" + + "'MutationCreateNullableFieldsTypeRequest\x124\n" + + "\x05input\x18\x01 \x01(\v2\x1e.productv1.NullableFieldsInputR\x05input\"\x88\x01\n" + + "(MutationCreateNullableFieldsTypeResponse\x12\\\n" + + "\x1bcreate_nullable_fields_type\x18\x01 \x01(\v2\x1d.productv1.NullableFieldsTypeR\x18createNullableFieldsType\"o\n" + + "'MutationUpdateNullableFieldsTypeRequest\x12\x0e\n" + + "\x02id\x18\x01 \x01(\tR\x02id\x124\n" + + "\x05input\x18\x02 \x01(\v2\x1e.productv1.NullableFieldsInputR\x05input\"\x88\x01\n" + + "(MutationUpdateNullableFieldsTypeResponse\x12\\\n" + + "\x1bupdate_nullable_fields_type\x18\x01 \x01(\v2\x1d.productv1.NullableFieldsTypeR\x18updateNullableFieldsType\"O\n" + + "\x1dMutationCreateBlogPostRequest\x12.\n" + + "\x05input\x18\x01 \x01(\v2\x18.productv1.BlogPostInputR\x05input\"_\n" + + "\x1eMutationCreateBlogPostResponse\x12=\n" + + "\x10create_blog_post\x18\x01 \x01(\v2\x13.productv1.BlogPostR\x0ecreateBlogPost\"_\n" + + "\x1dMutationUpdateBlogPostRequest\x12\x0e\n" + + "\x02id\x18\x01 \x01(\tR\x02id\x12.\n" + + "\x05input\x18\x02 \x01(\v2\x18.productv1.BlogPostInputR\x05input\"_\n" + + "\x1eMutationUpdateBlogPostResponse\x12=\n" + + "\x10update_blog_post\x18\x01 \x01(\v2\x13.productv1.BlogPostR\x0eupdateBlogPost\"K\n" + + "\x1bMutationCreateAuthorRequest\x12,\n" + + "\x05input\x18\x01 \x01(\v2\x16.productv1.AuthorInputR\x05input\"V\n" + + "\x1cMutationCreateAuthorResponse\x126\n" + + "\rcreate_author\x18\x01 \x01(\v2\x11.productv1.AuthorR\fcreateAuthor\"[\n" + + "\x1bMutationUpdateAuthorRequest\x12\x0e\n" + + "\x02id\x18\x01 \x01(\tR\x02id\x12,\n" + + "\x05input\x18\x02 \x01(\v2\x16.productv1.AuthorInputR\x05input\"V\n" + + "\x1cMutationUpdateAuthorResponse\x126\n" + + "\rupdate_author\x18\x01 \x01(\v2\x11.productv1.AuthorR\fupdateAuthor\"Z\n" + + " MutationBulkCreateAuthorsRequest\x126\n" + + "\aauthors\x18\x01 \x01(\v2\x1c.productv1.ListOfAuthorInputR\aauthors\"f\n" + + "!MutationBulkCreateAuthorsResponse\x12A\n" + + "\x13bulk_create_authors\x18\x01 \x03(\v2\x11.productv1.AuthorR\x11bulkCreateAuthors\"Z\n" + + " MutationBulkUpdateAuthorsRequest\x126\n" + + "\aauthors\x18\x01 \x01(\v2\x1c.productv1.ListOfAuthorInputR\aauthors\"f\n" + + "!MutationBulkUpdateAuthorsResponse\x12A\n" + + "\x13bulk_update_authors\x18\x01 \x03(\v2\x11.productv1.AuthorR\x11bulkUpdateAuthors\"c\n" + + "\"MutationBulkCreateBlogPostsRequest\x12=\n" + + "\n" + + "blog_posts\x18\x01 \x01(\v2\x1e.productv1.ListOfBlogPostInputR\tblogPosts\"o\n" + + "#MutationBulkCreateBlogPostsResponse\x12H\n" + + "\x16bulk_create_blog_posts\x18\x01 \x03(\v2\x13.productv1.BlogPostR\x13bulkCreateBlogPosts\"c\n" + + "\"MutationBulkUpdateBlogPostsRequest\x12=\n" + + "\n" + + "blog_posts\x18\x01 \x01(\v2\x1e.productv1.ListOfBlogPostInputR\tblogPosts\"o\n" + + "#MutationBulkUpdateBlogPostsResponse\x12H\n" + + "\x16bulk_update_blog_posts\x18\x01 \x03(\v2\x13.productv1.BlogPostR\x13bulkUpdateBlogPosts\"\\\n" + + "\"ResolveProductShippingEstimateArgs\x126\n" + + "\x05input\x18\x01 \x01(\v2 .productv1.ShippingEstimateInputR\x05input\"M\n" + + "%ResolveProductShippingEstimateContext\x12\x0e\n" + + "\x02id\x18\x01 \x01(\tR\x02id\x12\x14\n" + + "\x05price\x18\x02 \x01(\x01R\x05price\"\xc1\x01\n" + + "%ResolveProductShippingEstimateRequest\x12J\n" + + "\acontext\x18\x01 \x03(\v20.productv1.ResolveProductShippingEstimateContextR\acontext\x12L\n" + + "\n" + + "field_args\x18\x02 \x01(\v2-.productv1.ResolveProductShippingEstimateArgsR\tfieldArgs\"S\n" + + "$ResolveProductShippingEstimateResult\x12+\n" + + "\x11shipping_estimate\x18\x01 \x01(\x01R\x10shippingEstimate\"q\n" + + "&ResolveProductShippingEstimateResponse\x12G\n" + + "\x06result\x18\x01 \x03(\v2/.productv1.ResolveProductShippingEstimateResultR\x06result\"D\n" + + "%ResolveProductRecommendedCategoryArgs\x12\x1b\n" + + "\tmax_price\x18\x01 \x01(\x05R\bmaxPrice\"d\n" + + "(ResolveProductRecommendedCategoryContext\x12\x0e\n" + + "\x02id\x18\x01 \x01(\tR\x02id\x12\x12\n" + + "\x04name\x18\x02 \x01(\tR\x04name\x12\x14\n" + + "\x05price\x18\x03 \x01(\x01R\x05price\"\xca\x01\n" + + "(ResolveProductRecommendedCategoryRequest\x12M\n" + + "\acontext\x18\x01 \x03(\v23.productv1.ResolveProductRecommendedCategoryContextR\acontext\x12O\n" + + "\n" + + "field_args\x18\x02 \x01(\v20.productv1.ResolveProductRecommendedCategoryArgsR\tfieldArgs\"q\n" + + "'ResolveProductRecommendedCategoryResult\x12F\n" + + "\x14recommended_category\x18\x01 \x01(\v2\x13.productv1.CategoryR\x13recommendedCategory\"w\n" + + ")ResolveProductRecommendedCategoryResponse\x12J\n" + + "\x06result\x18\x01 \x03(\v22.productv1.ResolveProductRecommendedCategoryResultR\x06result\"Z\n" + + "\x1fResolveCategoryProductCountArgs\x127\n" + + "\afilters\x18\x01 \x01(\v2\x1d.productv1.ProductCountFilterR\afilters\"H\n" + + "\"ResolveCategoryProductCountContext\x12\x0e\n" + + "\x02id\x18\x01 \x01(\tR\x02id\x12\x12\n" + + "\x04name\x18\x02 \x01(\tR\x04name\"\xb8\x01\n" + + "\"ResolveCategoryProductCountRequest\x12G\n" + + "\acontext\x18\x01 \x03(\v2-.productv1.ResolveCategoryProductCountContextR\acontext\x12I\n" + + "\n" + + "field_args\x18\x02 \x01(\v2*.productv1.ResolveCategoryProductCountArgsR\tfieldArgs\"H\n" + + "!ResolveCategoryProductCountResult\x12#\n" + + "\rproduct_count\x18\x01 \x01(\x05R\fproductCount\"k\n" + + "#ResolveCategoryProductCountResponse\x12D\n" + + "\x06result\x18\x01 \x03(\v2,.productv1.ResolveCategoryProductCountResultR\x06result\"_\n" + + "\"ResolveCategoryPopularityScoreArgs\x129\n" + + "\tthreshold\x18\x01 \x01(\v2\x1b.google.protobuf.Int32ValueR\tthreshold\"7\n" + + "%ResolveCategoryPopularityScoreContext\x12\x0e\n" + + "\x02id\x18\x01 \x01(\tR\x02id\"\xc1\x01\n" + + "%ResolveCategoryPopularityScoreRequest\x12J\n" + + "\acontext\x18\x01 \x03(\v20.productv1.ResolveCategoryPopularityScoreContextR\acontext\x12L\n" + + "\n" + + "field_args\x18\x02 \x01(\v2-.productv1.ResolveCategoryPopularityScoreArgsR\tfieldArgs\"n\n" + + "$ResolveCategoryPopularityScoreResult\x12F\n" + + "\x10popularity_score\x18\x01 \x01(\v2\x1b.google.protobuf.Int32ValueR\x0fpopularityScore\"q\n" + + "&ResolveCategoryPopularityScoreResponse\x12G\n" + + "\x06result\x18\x01 \x03(\v2/.productv1.ResolveCategoryPopularityScoreResultR\x06result\"E\n" + + "\"ResolveCategoryCategoryMetricsArgs\x12\x1f\n" + + "\vmetric_type\x18\x01 \x01(\tR\n" + + "metricType\"K\n" + + "%ResolveCategoryCategoryMetricsContext\x12\x0e\n" + + "\x02id\x18\x01 \x01(\tR\x02id\x12\x12\n" + + "\x04name\x18\x02 \x01(\tR\x04name\"\xc1\x01\n" + + "%ResolveCategoryCategoryMetricsRequest\x12J\n" + + "\acontext\x18\x01 \x03(\v20.productv1.ResolveCategoryCategoryMetricsContextR\acontext\x12L\n" + + "\n" + + "field_args\x18\x02 \x01(\v2-.productv1.ResolveCategoryCategoryMetricsArgsR\tfieldArgs\"m\n" + + "$ResolveCategoryCategoryMetricsResult\x12E\n" + + "\x10category_metrics\x18\x01 \x01(\v2\x1a.productv1.CategoryMetricsR\x0fcategoryMetrics\"q\n" + + "&ResolveCategoryCategoryMetricsResponse\x12G\n" + + "\x06result\x18\x01 \x03(\v2/.productv1.ResolveCategoryCategoryMetricsResultR\x06result\"]\n" + + "\x1fResolveSubcategoryItemCountArgs\x12:\n" + + "\afilters\x18\x01 \x01(\v2 .productv1.SubcategoryItemFilterR\afilters\"4\n" + + "\"ResolveSubcategoryItemCountContext\x12\x0e\n" + + "\x02id\x18\x01 \x01(\tR\x02id\"\xb8\x01\n" + + "\"ResolveSubcategoryItemCountRequest\x12G\n" + + "\acontext\x18\x01 \x03(\v2-.productv1.ResolveSubcategoryItemCountContextR\acontext\x12I\n" + + "\n" + + "field_args\x18\x02 \x01(\v2*.productv1.ResolveSubcategoryItemCountArgsR\tfieldArgs\"B\n" + + "!ResolveSubcategoryItemCountResult\x12\x1d\n" + + "\n" + + "item_count\x18\x01 \x01(\x05R\titemCount\"k\n" + + "#ResolveSubcategoryItemCountResponse\x12D\n" + + "\x06result\x18\x01 \x03(\v2,.productv1.ResolveSubcategoryItemCountResultR\x06result\"C\n" + + "\aProduct\x12\x0e\n" + + "\x02id\x18\x01 \x01(\tR\x02id\x12\x12\n" + + "\x04name\x18\x02 \x01(\tR\x04name\x12\x14\n" + + "\x05price\x18\x03 \x01(\x01R\x05price\"I\n" + + "\aStorage\x12\x0e\n" + + "\x02id\x18\x01 \x01(\tR\x02id\x12\x12\n" + + "\x04name\x18\x02 \x01(\tR\x04name\x12\x1a\n" + + "\blocation\x18\x03 \x01(\tR\blocation\"K\n" + + "\tWarehouse\x12\x0e\n" + + "\x02id\x18\x01 \x01(\tR\x02id\x12\x12\n" + + "\x04name\x18\x02 \x01(\tR\x04name\x12\x1a\n" + + "\blocation\x18\x03 \x01(\tR\blocation\"*\n" + + "\x04User\x12\x0e\n" + + "\x02id\x18\x01 \x01(\tR\x02id\x12\x12\n" + + "\x04name\x18\x02 \x01(\tR\x04name\"W\n" + + "\vNestedTypeA\x12\x0e\n" + + "\x02id\x18\x01 \x01(\tR\x02id\x12\x12\n" + + "\x04name\x18\x02 \x01(\tR\x04name\x12$\n" + + "\x01b\x18\x03 \x01(\v2\x16.productv1.NestedTypeBR\x01b\"t\n" + + "\rRecursiveType\x12\x0e\n" + + "\x02id\x18\x01 \x01(\tR\x02id\x12\x12\n" + + "\x04name\x18\x02 \x01(\tR\x04name\x12?\n" + + "\x0erecursive_type\x18\x03 \x01(\v2\x18.productv1.RecursiveTypeR\rrecursiveType\"\x8e\x01\n" + + "\x1cTypeWithMultipleFilterFields\x12\x0e\n" + + "\x02id\x18\x01 \x01(\tR\x02id\x12\x12\n" + + "\x04name\x18\x02 \x01(\tR\x04name\x12$\n" + + "\x0efilter_field_1\x18\x03 \x01(\tR\ffilterField1\x12$\n" + + "\x0efilter_field_2\x18\x04 \x01(\tR\ffilterField2\"]\n" + + "\x0fFilterTypeInput\x12$\n" + + "\x0efilter_field_1\x18\x01 \x01(\tR\ffilterField1\x12$\n" + + "\x0efilter_field_2\x18\x02 \x01(\tR\ffilterField2\"G\n" + + "\x16ComplexFilterTypeInput\x12-\n" + + "\x06filter\x18\x01 \x01(\v2\x15.productv1.FilterTypeR\x06filter\"@\n" + + "\x1aTypeWithComplexFilterInput\x12\x0e\n" + + "\x02id\x18\x01 \x01(\tR\x02id\x12\x12\n" + + "\x04name\x18\x02 \x01(\tR\x04name\"}\n" + + "\n" + + "OrderInput\x12\x19\n" + + "\border_id\x18\x01 \x01(\tR\aorderId\x12#\n" + + "\rcustomer_name\x18\x02 \x01(\tR\fcustomerName\x12/\n" + + "\x05lines\x18\x03 \x03(\v2\x19.productv1.OrderLineInputR\x05lines\"\xa5\x01\n" + + "\x05Order\x12\x19\n" + + "\border_id\x18\x01 \x01(\tR\aorderId\x12#\n" + + "\rcustomer_name\x18\x02 \x01(\tR\fcustomerName\x12\x1f\n" + + "\vtotal_items\x18\x03 \x01(\x05R\n" + + "totalItems\x12;\n" + + "\vorder_lines\x18\x04 \x01(\v2\x1a.productv1.ListOfOrderLineR\n" + + "orderLines\"\x9f\x01\n" + + "\bCategory\x12\x0e\n" + + "\x02id\x18\x01 \x01(\tR\x02id\x12\x12\n" + + "\x04name\x18\x02 \x01(\tR\x04name\x12+\n" + + "\x04kind\x18\x03 \x01(\x0e2\x17.productv1.CategoryKindR\x04kind\x12B\n" + + "\rsubcategories\x18\x05 \x01(\v2\x1c.productv1.ListOfSubcategoryR\rsubcategories\"|\n" + + "\x0eCategoryFilter\x123\n" + + "\bcategory\x18\x01 \x01(\x0e2\x17.productv1.CategoryKindR\bcategory\x125\n" + + "\n" + + "pagination\x18\x02 \x01(\v2\x15.productv1.PaginationR\n" + + "pagination\"\\\n" + + "\x06Animal\x12\"\n" + + "\x03cat\x18\x01 \x01(\v2\x0e.productv1.CatH\x00R\x03cat\x12\"\n" + + "\x03dog\x18\x02 \x01(\v2\x0e.productv1.DogH\x00R\x03dogB\n" + + "\n" + + "\binstance\"V\n" + + "\vSearchInput\x12\x14\n" + + "\x05query\x18\x01 \x01(\tR\x05query\x121\n" + + "\x05limit\x18\x02 \x01(\v2\x1b.google.protobuf.Int32ValueR\x05limit\"\xa1\x01\n" + + "\fSearchResult\x12.\n" + + "\aproduct\x18\x01 \x01(\v2\x12.productv1.ProductH\x00R\aproduct\x12%\n" + + "\x04user\x18\x02 \x01(\v2\x0f.productv1.UserH\x00R\x04user\x121\n" + + "\bcategory\x18\x03 \x01(\v2\x13.productv1.CategoryH\x00R\bcategoryB\a\n" + + "\x05value\"\x97\x03\n" + + "\x12NullableFieldsType\x12\x0e\n" + + "\x02id\x18\x01 \x01(\tR\x02id\x12\x12\n" + + "\x04name\x18\x02 \x01(\tR\x04name\x12E\n" + + "\x0foptional_string\x18\x03 \x01(\v2\x1c.google.protobuf.StringValueR\x0eoptionalString\x12>\n" + + "\foptional_int\x18\x04 \x01(\v2\x1b.google.protobuf.Int32ValueR\voptionalInt\x12C\n" + + "\x0eoptional_float\x18\x05 \x01(\v2\x1c.google.protobuf.DoubleValueR\roptionalFloat\x12E\n" + + "\x10optional_boolean\x18\x06 \x01(\v2\x1a.google.protobuf.BoolValueR\x0foptionalBoolean\x12'\n" + + "\x0frequired_string\x18\a \x01(\tR\x0erequiredString\x12!\n" + + "\frequired_int\x18\b \x01(\x05R\vrequiredInt\"\xd0\x01\n" + + "\x14NullableFieldsFilter\x120\n" + + "\x04name\x18\x01 \x01(\v2\x1c.google.protobuf.StringValueR\x04name\x12E\n" + + "\x0foptional_string\x18\x02 \x01(\v2\x1c.google.protobuf.StringValueR\x0eoptionalString\x12?\n" + + "\rinclude_nulls\x18\x03 \x01(\v2\x1a.google.protobuf.BoolValueR\fincludeNulls\"\xa4\b\n" + + "\bBlogPost\x12\x0e\n" + + "\x02id\x18\x01 \x01(\tR\x02id\x12\x14\n" + + "\x05title\x18\x02 \x01(\tR\x05title\x12\x18\n" + + "\acontent\x18\x03 \x01(\tR\acontent\x12\x12\n" + + "\x04tags\x18\x04 \x03(\tR\x04tags\x12<\n" + + "\roptional_tags\x18\x05 \x01(\v2\x17.productv1.ListOfStringR\foptionalTags\x12\x1e\n" + + "\n" + + "categories\x18\x06 \x03(\tR\n" + + "categories\x123\n" + + "\bkeywords\x18\a \x01(\v2\x17.productv1.ListOfStringR\bkeywords\x12\x1f\n" + + "\vview_counts\x18\b \x03(\x05R\n" + + "viewCounts\x120\n" + + "\aratings\x18\t \x01(\v2\x16.productv1.ListOfFloatR\aratings\x12;\n" + + "\fis_published\x18\n" + + " \x01(\v2\x18.productv1.ListOfBooleanR\visPublished\x12<\n" + + "\n" + + "tag_groups\x18\v \x01(\v2\x1d.productv1.ListOfListOfStringR\ttagGroups\x12D\n" + + "\x0erelated_topics\x18\f \x01(\v2\x1d.productv1.ListOfListOfStringR\rrelatedTopics\x12F\n" + + "\x0fcomment_threads\x18\r \x01(\v2\x1d.productv1.ListOfListOfStringR\x0ecommentThreads\x12?\n" + + "\vsuggestions\x18\x0e \x01(\v2\x1d.productv1.ListOfListOfStringR\vsuggestions\x12B\n" + + "\x12related_categories\x18\x0f \x03(\v2\x13.productv1.CategoryR\x11relatedCategories\x123\n" + + "\fcontributors\x18\x10 \x03(\v2\x0f.productv1.UserR\fcontributors\x12G\n" + + "\x12mentioned_products\x18\x11 \x01(\v2\x18.productv1.ListOfProductR\x11mentionedProducts\x12>\n" + + "\x0fmentioned_users\x18\x12 \x01(\v2\x15.productv1.ListOfUserR\x0ementionedUsers\x12H\n" + + "\x0fcategory_groups\x18\x13 \x01(\v2\x1f.productv1.ListOfListOfCategoryR\x0ecategoryGroups\x12H\n" + + "\x11contributor_teams\x18\x14 \x01(\v2\x1b.productv1.ListOfListOfUserR\x10contributorTeams\"\xbf\x01\n" + + "\x0eBlogPostFilter\x122\n" + + "\x05title\x18\x01 \x01(\v2\x1c.google.protobuf.StringValueR\x05title\x12A\n" + + "\x0ehas_categories\x18\x02 \x01(\v2\x1a.google.protobuf.BoolValueR\rhasCategories\x126\n" + + "\bmin_tags\x18\x03 \x01(\v2\x1b.google.protobuf.Int32ValueR\aminTags\"\xc3\x06\n" + + "\x06Author\x12\x0e\n" + + "\x02id\x18\x01 \x01(\tR\x02id\x12\x12\n" + + "\x04name\x18\x02 \x01(\tR\x04name\x122\n" + + "\x05email\x18\x03 \x01(\v2\x1c.google.protobuf.StringValueR\x05email\x12\x16\n" + + "\x06skills\x18\x04 \x03(\tR\x06skills\x12\x1c\n" + + "\tlanguages\x18\x05 \x03(\tR\tlanguages\x12:\n" + + "\fsocial_links\x18\x06 \x01(\v2\x17.productv1.ListOfStringR\vsocialLinks\x12G\n" + + "\x10teams_by_project\x18\a \x01(\v2\x1d.productv1.ListOfListOfStringR\x0eteamsByProject\x12E\n" + + "\x0ecollaborations\x18\b \x01(\v2\x1d.productv1.ListOfListOfStringR\x0ecollaborations\x12>\n" + + "\rwritten_posts\x18\t \x01(\v2\x19.productv1.ListOfBlogPostR\fwrittenPosts\x12D\n" + + "\x13favorite_categories\x18\n" + + " \x03(\v2\x13.productv1.CategoryR\x12favoriteCategories\x12>\n" + + "\x0frelated_authors\x18\v \x01(\v2\x15.productv1.ListOfUserR\x0erelatedAuthors\x12A\n" + + "\x0fproduct_reviews\x18\f \x01(\v2\x18.productv1.ListOfProductR\x0eproductReviews\x12@\n" + + "\rauthor_groups\x18\r \x01(\v2\x1b.productv1.ListOfListOfUserR\fauthorGroups\x12R\n" + + "\x14category_preferences\x18\x0e \x01(\v2\x1f.productv1.ListOfListOfCategoryR\x13categoryPreferences\x12@\n" + + "\rproject_teams\x18\x0f \x01(\v2\x1b.productv1.ListOfListOfUserR\fprojectTeams\"\xb7\x01\n" + + "\fAuthorFilter\x120\n" + + "\x04name\x18\x01 \x01(\v2\x1c.google.protobuf.StringValueR\x04name\x127\n" + + "\thas_teams\x18\x02 \x01(\v2\x1a.google.protobuf.BoolValueR\bhasTeams\x12<\n" + + "\vskill_count\x18\x03 \x01(\v2\x1b.google.protobuf.Int32ValueR\n" + + "skillCount\"\x1f\n" + + "\tUserInput\x12\x12\n" + + "\x04name\x18\x01 \x01(\tR\x04name\";\n" + + "\vActionInput\x12\x12\n" + + "\x04type\x18\x01 \x01(\tR\x04type\x12\x18\n" + + "\apayload\x18\x02 \x01(\tR\apayload\"\x97\x01\n" + + "\fActionResult\x12A\n" + + "\x0eaction_success\x18\x01 \x01(\v2\x18.productv1.ActionSuccessH\x00R\ractionSuccess\x12;\n" + + "\faction_error\x18\x02 \x01(\v2\x16.productv1.ActionErrorH\x00R\vactionErrorB\a\n" + + "\x05value\"\x88\x03\n" + + "\x13NullableFieldsInput\x12\x12\n" + + "\x04name\x18\x01 \x01(\tR\x04name\x12E\n" + + "\x0foptional_string\x18\x02 \x01(\v2\x1c.google.protobuf.StringValueR\x0eoptionalString\x12>\n" + + "\foptional_int\x18\x03 \x01(\v2\x1b.google.protobuf.Int32ValueR\voptionalInt\x12C\n" + + "\x0eoptional_float\x18\x04 \x01(\v2\x1c.google.protobuf.DoubleValueR\roptionalFloat\x12E\n" + + "\x10optional_boolean\x18\x05 \x01(\v2\x1a.google.protobuf.BoolValueR\x0foptionalBoolean\x12'\n" + + "\x0frequired_string\x18\x06 \x01(\tR\x0erequiredString\x12!\n" + + "\frequired_int\x18\a \x01(\x05R\vrequiredInt\"\xe1\x06\n" + + "\rBlogPostInput\x12\x14\n" + + "\x05title\x18\x01 \x01(\tR\x05title\x12\x18\n" + + "\acontent\x18\x02 \x01(\tR\acontent\x12\x12\n" + + "\x04tags\x18\x03 \x03(\tR\x04tags\x12<\n" + + "\roptional_tags\x18\x04 \x01(\v2\x17.productv1.ListOfStringR\foptionalTags\x12\x1e\n" + + "\n" + + "categories\x18\x05 \x03(\tR\n" + + "categories\x123\n" + + "\bkeywords\x18\x06 \x01(\v2\x17.productv1.ListOfStringR\bkeywords\x12\x1f\n" + + "\vview_counts\x18\a \x03(\x05R\n" + + "viewCounts\x120\n" + + "\aratings\x18\b \x01(\v2\x16.productv1.ListOfFloatR\aratings\x12;\n" + + "\fis_published\x18\t \x01(\v2\x18.productv1.ListOfBooleanR\visPublished\x12<\n" + + "\n" + + "tag_groups\x18\n" + + " \x01(\v2\x1d.productv1.ListOfListOfStringR\ttagGroups\x12D\n" + + "\x0erelated_topics\x18\v \x01(\v2\x1d.productv1.ListOfListOfStringR\rrelatedTopics\x12F\n" + + "\x0fcomment_threads\x18\f \x01(\v2\x1d.productv1.ListOfListOfStringR\x0ecommentThreads\x12?\n" + + "\vsuggestions\x18\r \x01(\v2\x1d.productv1.ListOfListOfStringR\vsuggestions\x12M\n" + + "\x12related_categories\x18\x0e \x01(\v2\x1e.productv1.ListOfCategoryInputR\x11relatedCategories\x12>\n" + + "\fcontributors\x18\x0f \x01(\v2\x1a.productv1.ListOfUserInputR\fcontributors\x12M\n" + + "\x0fcategory_groups\x18\x10 \x01(\v2$.productv1.ListOfListOfCategoryInputR\x0ecategoryGroups\"\xb0\x04\n" + + "\vAuthorInput\x12\x12\n" + + "\x04name\x18\x01 \x01(\tR\x04name\x122\n" + + "\x05email\x18\x02 \x01(\v2\x1c.google.protobuf.StringValueR\x05email\x12\x16\n" + + "\x06skills\x18\x03 \x03(\tR\x06skills\x12\x1c\n" + + "\tlanguages\x18\x04 \x03(\tR\tlanguages\x12:\n" + + "\fsocial_links\x18\x05 \x01(\v2\x17.productv1.ListOfStringR\vsocialLinks\x12G\n" + + "\x10teams_by_project\x18\x06 \x01(\v2\x1d.productv1.ListOfListOfStringR\x0eteamsByProject\x12E\n" + + "\x0ecollaborations\x18\a \x01(\v2\x1d.productv1.ListOfListOfStringR\x0ecollaborations\x12I\n" + + "\x13favorite_categories\x18\b \x03(\v2\x18.productv1.CategoryInputR\x12favoriteCategories\x12E\n" + + "\rauthor_groups\x18\t \x01(\v2 .productv1.ListOfListOfUserInputR\fauthorGroups\x12E\n" + + "\rproject_teams\x18\n" + + " \x01(\v2 .productv1.ListOfListOfUserInputR\fprojectTeams\"W\n" + + "\vNestedTypeB\x12\x0e\n" + + "\x02id\x18\x01 \x01(\tR\x02id\x12\x12\n" + + "\x04name\x18\x02 \x01(\tR\x04name\x12$\n" + + "\x01c\x18\x03 \x01(\v2\x16.productv1.NestedTypeCR\x01c\"1\n" + + "\vNestedTypeC\x12\x0e\n" + + "\x02id\x18\x01 \x01(\tR\x02id\x12\x12\n" + + "\x04name\x18\x02 \x01(\tR\x04name\"\xa3\x01\n" + + "\n" + + "FilterType\x12\x12\n" + + "\x04name\x18\x01 \x01(\tR\x04name\x12$\n" + + "\x0efilter_field_1\x18\x02 \x01(\tR\ffilterField1\x12$\n" + + "\x0efilter_field_2\x18\x03 \x01(\tR\ffilterField2\x125\n" + + "\n" + + "pagination\x18\x04 \x01(\v2\x15.productv1.PaginationR\n" + + "pagination\";\n" + + "\n" + + "Pagination\x12\x12\n" + + "\x04page\x18\x01 \x01(\x05R\x04page\x12\x19\n" + + "\bper_page\x18\x02 \x01(\x05R\aperPage\"\x82\x01\n" + + "\x0eOrderLineInput\x12\x1d\n" + + "\n" + + "product_id\x18\x01 \x01(\tR\tproductId\x12\x1a\n" + + "\bquantity\x18\x02 \x01(\x05R\bquantity\x125\n" + + "\tmodifiers\x18\x03 \x01(\v2\x17.productv1.ListOfStringR\tmodifiers\"}\n" + + "\tOrderLine\x12\x1d\n" + + "\n" + + "product_id\x18\x01 \x01(\tR\tproductId\x12\x1a\n" + + "\bquantity\x18\x02 \x01(\x05R\bquantity\x125\n" + + "\tmodifiers\x18\x03 \x01(\v2\x17.productv1.ListOfStringR\tmodifiers\"\x8e\x01\n" + + "\vSubcategory\x12\x0e\n" + + "\x02id\x18\x01 \x01(\tR\x02id\x12\x12\n" + + "\x04name\x18\x02 \x01(\tR\x04name\x12>\n" + + "\vdescription\x18\x03 \x01(\v2\x1c.google.protobuf.StringValueR\vdescription\x12\x1b\n" + + "\tis_active\x18\x04 \x01(\bR\bisActive\"\x97\x01\n" + + "\x0fCategoryMetrics\x12\x0e\n" + + "\x02id\x18\x01 \x01(\tR\x02id\x12\x1f\n" + + "\vmetric_type\x18\x02 \x01(\tR\n" + + "metricType\x12\x14\n" + + "\x05value\x18\x03 \x01(\x01R\x05value\x12\x1c\n" + + "\ttimestamp\x18\x04 \x01(\tR\ttimestamp\x12\x1f\n" + + "\vcategory_id\x18\x05 \x01(\tR\n" + + "categoryId\"^\n" + + "\x03Cat\x12\x0e\n" + + "\x02id\x18\x01 \x01(\tR\x02id\x12\x12\n" + + "\x04name\x18\x02 \x01(\tR\x04name\x12\x12\n" + + "\x04kind\x18\x03 \x01(\tR\x04kind\x12\x1f\n" + + "\vmeow_volume\x18\x04 \x01(\x05R\n" + + "meowVolume\"^\n" + + "\x03Dog\x12\x0e\n" + + "\x02id\x18\x01 \x01(\tR\x02id\x12\x12\n" + + "\x04name\x18\x02 \x01(\tR\x04name\x12\x12\n" + + "\x04kind\x18\x03 \x01(\tR\x04kind\x12\x1f\n" + + "\vbark_volume\x18\x04 \x01(\x05R\n" + + "barkVolume\"G\n" + + "\rActionSuccess\x12\x18\n" + + "\amessage\x18\x01 \x01(\tR\amessage\x12\x1c\n" + + "\ttimestamp\x18\x02 \x01(\tR\ttimestamp\";\n" + + "\vActionError\x12\x18\n" + + "\amessage\x18\x01 \x01(\tR\amessage\x12\x12\n" + + "\x04code\x18\x02 \x01(\tR\x04code\"P\n" + + "\rCategoryInput\x12\x12\n" + + "\x04name\x18\x01 \x01(\tR\x04name\x12+\n" + + "\x04kind\x18\x02 \x01(\x0e2\x17.productv1.CategoryKindR\x04kind\"\x80\x02\n" + + "\x12ProductCountFilter\x129\n" + + "\tmin_price\x18\x01 \x01(\v2\x1c.google.protobuf.DoubleValueR\bminPrice\x129\n" + + "\tmax_price\x18\x02 \x01(\v2\x1c.google.protobuf.DoubleValueR\bmaxPrice\x125\n" + + "\bin_stock\x18\x03 \x01(\v2\x1a.google.protobuf.BoolValueR\ainStock\x12=\n" + + "\vsearch_term\x18\x04 \x01(\v2\x1c.google.protobuf.StringValueR\n" + + "searchTerm\"\xbc\x02\n" + + "\x15SubcategoryItemFilter\x129\n" + + "\tmin_price\x18\x01 \x01(\v2\x1c.google.protobuf.DoubleValueR\bminPrice\x129\n" + + "\tmax_price\x18\x02 \x01(\v2\x1c.google.protobuf.DoubleValueR\bmaxPrice\x125\n" + + "\bin_stock\x18\x03 \x01(\v2\x1a.google.protobuf.BoolValueR\ainStock\x127\n" + + "\tis_active\x18\x04 \x01(\v2\x1a.google.protobuf.BoolValueR\bisActive\x12=\n" + + "\vsearch_term\x18\x05 \x01(\v2\x1c.google.protobuf.StringValueR\n" + + "searchTerm\"\xab\x01\n" + + "\x15ShippingEstimateInput\x12@\n" + + "\vdestination\x18\x01 \x01(\x0e2\x1e.productv1.ShippingDestinationR\vdestination\x12\x16\n" + + "\x06weight\x18\x02 \x01(\x01R\x06weight\x128\n" + + "\texpedited\x18\x03 \x01(\v2\x1a.google.protobuf.BoolValueR\texpedited*\x9a\x01\n" + + "\fCategoryKind\x12\x1d\n" + + "\x19CATEGORY_KIND_UNSPECIFIED\x10\x00\x12\x16\n" + + "\x12CATEGORY_KIND_BOOK\x10\x01\x12\x1d\n" + + "\x19CATEGORY_KIND_ELECTRONICS\x10\x02\x12\x1b\n" + + "\x17CATEGORY_KIND_FURNITURE\x10\x03\x12\x17\n" + + "\x13CATEGORY_KIND_OTHER\x10\x04*\xa8\x01\n" + + "\x13ShippingDestination\x12$\n" + + " SHIPPING_DESTINATION_UNSPECIFIED\x10\x00\x12!\n" + + "\x1dSHIPPING_DESTINATION_DOMESTIC\x10\x01\x12 \n" + + "\x1cSHIPPING_DESTINATION_EXPRESS\x10\x02\x12&\n" + + "\"SHIPPING_DESTINATION_INTERNATIONAL\x10\x032\xe9,\n" + + "\x0eProductService\x12`\n" + + "\x11LookupProductById\x12#.productv1.LookupProductByIdRequest\x1a$.productv1.LookupProductByIdResponse\"\x00\x12`\n" + + "\x11LookupStorageById\x12#.productv1.LookupStorageByIdRequest\x1a$.productv1.LookupStorageByIdResponse\"\x00\x12f\n" + + "\x13LookupWarehouseById\x12%.productv1.LookupWarehouseByIdRequest\x1a&.productv1.LookupWarehouseByIdResponse\"\x00\x12x\n" + + "\x19MutationBulkCreateAuthors\x12+.productv1.MutationBulkCreateAuthorsRequest\x1a,.productv1.MutationBulkCreateAuthorsResponse\"\x00\x12~\n" + + "\x1bMutationBulkCreateBlogPosts\x12-.productv1.MutationBulkCreateBlogPostsRequest\x1a..productv1.MutationBulkCreateBlogPostsResponse\"\x00\x12x\n" + + "\x19MutationBulkUpdateAuthors\x12+.productv1.MutationBulkUpdateAuthorsRequest\x1a,.productv1.MutationBulkUpdateAuthorsResponse\"\x00\x12~\n" + + "\x1bMutationBulkUpdateBlogPosts\x12-.productv1.MutationBulkUpdateBlogPostsRequest\x1a..productv1.MutationBulkUpdateBlogPostsResponse\"\x00\x12i\n" + + "\x14MutationCreateAuthor\x12&.productv1.MutationCreateAuthorRequest\x1a'.productv1.MutationCreateAuthorResponse\"\x00\x12o\n" + + "\x16MutationCreateBlogPost\x12(.productv1.MutationCreateBlogPostRequest\x1a).productv1.MutationCreateBlogPostResponse\"\x00\x12\x8d\x01\n" + + " MutationCreateNullableFieldsType\x122.productv1.MutationCreateNullableFieldsTypeRequest\x1a3.productv1.MutationCreateNullableFieldsTypeResponse\"\x00\x12c\n" + + "\x12MutationCreateUser\x12$.productv1.MutationCreateUserRequest\x1a%.productv1.MutationCreateUserResponse\"\x00\x12l\n" + + "\x15MutationPerformAction\x12'.productv1.MutationPerformActionRequest\x1a(.productv1.MutationPerformActionResponse\"\x00\x12i\n" + + "\x14MutationUpdateAuthor\x12&.productv1.MutationUpdateAuthorRequest\x1a'.productv1.MutationUpdateAuthorResponse\"\x00\x12o\n" + + "\x16MutationUpdateBlogPost\x12(.productv1.MutationUpdateBlogPostRequest\x1a).productv1.MutationUpdateBlogPostResponse\"\x00\x12\x8d\x01\n" + + " MutationUpdateNullableFieldsType\x122.productv1.MutationUpdateNullableFieldsTypeRequest\x1a3.productv1.MutationUpdateNullableFieldsTypeResponse\"\x00\x12Z\n" + + "\x0fQueryAllAuthors\x12!.productv1.QueryAllAuthorsRequest\x1a\".productv1.QueryAllAuthorsResponse\"\x00\x12`\n" + + "\x11QueryAllBlogPosts\x12#.productv1.QueryAllBlogPostsRequest\x1a$.productv1.QueryAllBlogPostsResponse\"\x00\x12~\n" + + "\x1bQueryAllNullableFieldsTypes\x12-.productv1.QueryAllNullableFieldsTypesRequest\x1a..productv1.QueryAllNullableFieldsTypesResponse\"\x00\x12Q\n" + + "\fQueryAllPets\x12\x1e.productv1.QueryAllPetsRequest\x1a\x1f.productv1.QueryAllPetsResponse\"\x00\x12N\n" + + "\vQueryAuthor\x12\x1d.productv1.QueryAuthorRequest\x1a\x1e.productv1.QueryAuthorResponse\"\x00\x12Z\n" + + "\x0fQueryAuthorById\x12!.productv1.QueryAuthorByIdRequest\x1a\".productv1.QueryAuthorByIdResponse\"\x00\x12o\n" + + "\x16QueryAuthorsWithFilter\x12(.productv1.QueryAuthorsWithFilterRequest\x1a).productv1.QueryAuthorsWithFilterResponse\"\x00\x12T\n" + + "\rQueryBlogPost\x12\x1f.productv1.QueryBlogPostRequest\x1a .productv1.QueryBlogPostResponse\"\x00\x12`\n" + + "\x11QueryBlogPostById\x12#.productv1.QueryBlogPostByIdRequest\x1a$.productv1.QueryBlogPostByIdResponse\"\x00\x12u\n" + + "\x18QueryBlogPostsWithFilter\x12*.productv1.QueryBlogPostsWithFilterRequest\x1a+.productv1.QueryBlogPostsWithFilterResponse\"\x00\x12o\n" + + "\x16QueryBulkSearchAuthors\x12(.productv1.QueryBulkSearchAuthorsRequest\x1a).productv1.QueryBulkSearchAuthorsResponse\"\x00\x12u\n" + + "\x18QueryBulkSearchBlogPosts\x12*.productv1.QueryBulkSearchBlogPostsRequest\x1a+.productv1.QueryBulkSearchBlogPostsResponse\"\x00\x12i\n" + + "\x14QueryCalculateTotals\x12&.productv1.QueryCalculateTotalsRequest\x1a'.productv1.QueryCalculateTotalsResponse\"\x00\x12Z\n" + + "\x0fQueryCategories\x12!.productv1.QueryCategoriesRequest\x1a\".productv1.QueryCategoriesResponse\"\x00\x12l\n" + + "\x15QueryCategoriesByKind\x12'.productv1.QueryCategoriesByKindRequest\x1a(.productv1.QueryCategoriesByKindResponse\"\x00\x12o\n" + + "\x16QueryCategoriesByKinds\x12(.productv1.QueryCategoriesByKindsRequest\x1a).productv1.QueryCategoriesByKindsResponse\"\x00\x12o\n" + + "\x16QueryComplexFilterType\x12(.productv1.QueryComplexFilterTypeRequest\x1a).productv1.QueryComplexFilterTypeResponse\"\x00\x12l\n" + + "\x15QueryFilterCategories\x12'.productv1.QueryFilterCategoriesRequest\x1a(.productv1.QueryFilterCategoriesResponse\"\x00\x12Z\n" + + "\x0fQueryNestedType\x12!.productv1.QueryNestedTypeRequest\x1a\".productv1.QueryNestedTypeResponse\"\x00\x12r\n" + + "\x17QueryNullableFieldsType\x12).productv1.QueryNullableFieldsTypeRequest\x1a*.productv1.QueryNullableFieldsTypeResponse\"\x00\x12~\n" + + "\x1bQueryNullableFieldsTypeById\x12-.productv1.QueryNullableFieldsTypeByIdRequest\x1a..productv1.QueryNullableFieldsTypeByIdResponse\"\x00\x12\x90\x01\n" + + "!QueryNullableFieldsTypeWithFilter\x123.productv1.QueryNullableFieldsTypeWithFilterRequest\x1a4.productv1.QueryNullableFieldsTypeWithFilterResponse\"\x00\x12W\n" + + "\x0eQueryRandomPet\x12 .productv1.QueryRandomPetRequest\x1a!.productv1.QueryRandomPetResponse\"\x00\x12r\n" + + "\x17QueryRandomSearchResult\x12).productv1.QueryRandomSearchResultRequest\x1a*.productv1.QueryRandomSearchResultResponse\"\x00\x12c\n" + + "\x12QueryRecursiveType\x12$.productv1.QueryRecursiveTypeRequest\x1a%.productv1.QueryRecursiveTypeResponse\"\x00\x12N\n" + + "\vQuerySearch\x12\x1d.productv1.QuerySearchRequest\x1a\x1e.productv1.QuerySearchResponse\"\x00\x12\x81\x01\n" + + "\x1cQueryTypeFilterWithArguments\x12..productv1.QueryTypeFilterWithArgumentsRequest\x1a/.productv1.QueryTypeFilterWithArgumentsResponse\"\x00\x12\x90\x01\n" + + "!QueryTypeWithMultipleFilterFields\x123.productv1.QueryTypeWithMultipleFilterFieldsRequest\x1a4.productv1.QueryTypeWithMultipleFilterFieldsResponse\"\x00\x12H\n" + + "\tQueryUser\x12\x1b.productv1.QueryUserRequest\x1a\x1c.productv1.QueryUserResponse\"\x00\x12K\n" + + "\n" + + "QueryUsers\x12\x1c.productv1.QueryUsersRequest\x1a\x1d.productv1.QueryUsersResponse\"\x00\x12\x87\x01\n" + + "\x1eResolveCategoryCategoryMetrics\x120.productv1.ResolveCategoryCategoryMetricsRequest\x1a1.productv1.ResolveCategoryCategoryMetricsResponse\"\x00\x12\x87\x01\n" + + "\x1eResolveCategoryPopularityScore\x120.productv1.ResolveCategoryPopularityScoreRequest\x1a1.productv1.ResolveCategoryPopularityScoreResponse\"\x00\x12~\n" + + "\x1bResolveCategoryProductCount\x12-.productv1.ResolveCategoryProductCountRequest\x1a..productv1.ResolveCategoryProductCountResponse\"\x00\x12\x90\x01\n" + + "!ResolveProductRecommendedCategory\x123.productv1.ResolveProductRecommendedCategoryRequest\x1a4.productv1.ResolveProductRecommendedCategoryResponse\"\x00\x12\x87\x01\n" + + "\x1eResolveProductShippingEstimate\x120.productv1.ResolveProductShippingEstimateRequest\x1a1.productv1.ResolveProductShippingEstimateResponse\"\x00\x12~\n" + + "\x1bResolveSubcategoryItemCount\x12-.productv1.ResolveSubcategoryItemCountRequest\x1a..productv1.ResolveSubcategoryItemCountResponse\"\x00B%Z#cosmo/pkg/proto/productv1;productv1b\x06proto3" + +var ( + file_testdata_service_proto_rawDescOnce sync.Once + file_testdata_service_proto_rawDescData []byte +) + +func file_testdata_service_proto_rawDescGZIP() []byte { + file_testdata_service_proto_rawDescOnce.Do(func() { + file_testdata_service_proto_rawDescData = protoimpl.X.CompressGZIP(unsafe.Slice(unsafe.StringData(file_testdata_service_proto_rawDesc), len(file_testdata_service_proto_rawDesc))) + }) + return file_testdata_service_proto_rawDescData +} + +var file_testdata_service_proto_enumTypes = make([]protoimpl.EnumInfo, 2) +var file_testdata_service_proto_msgTypes = make([]protoimpl.MessageInfo, 208) +var file_testdata_service_proto_goTypes = []any{ + (CategoryKind)(0), // 0: productv1.CategoryKind + (ShippingDestination)(0), // 1: productv1.ShippingDestination + (*ListOfAuthorFilter)(nil), // 2: productv1.ListOfAuthorFilter + (*ListOfAuthorInput)(nil), // 3: productv1.ListOfAuthorInput + (*ListOfBlogPost)(nil), // 4: productv1.ListOfBlogPost + (*ListOfBlogPostFilter)(nil), // 5: productv1.ListOfBlogPostFilter + (*ListOfBlogPostInput)(nil), // 6: productv1.ListOfBlogPostInput + (*ListOfBoolean)(nil), // 7: productv1.ListOfBoolean + (*ListOfCategory)(nil), // 8: productv1.ListOfCategory + (*ListOfCategoryInput)(nil), // 9: productv1.ListOfCategoryInput + (*ListOfFloat)(nil), // 10: productv1.ListOfFloat + (*ListOfListOfCategory)(nil), // 11: productv1.ListOfListOfCategory + (*ListOfListOfCategoryInput)(nil), // 12: productv1.ListOfListOfCategoryInput + (*ListOfListOfString)(nil), // 13: productv1.ListOfListOfString + (*ListOfListOfUser)(nil), // 14: productv1.ListOfListOfUser + (*ListOfListOfUserInput)(nil), // 15: productv1.ListOfListOfUserInput + (*ListOfOrderLine)(nil), // 16: productv1.ListOfOrderLine + (*ListOfProduct)(nil), // 17: productv1.ListOfProduct + (*ListOfString)(nil), // 18: productv1.ListOfString + (*ListOfSubcategory)(nil), // 19: productv1.ListOfSubcategory + (*ListOfUser)(nil), // 20: productv1.ListOfUser + (*ListOfUserInput)(nil), // 21: productv1.ListOfUserInput + (*LookupProductByIdRequestKey)(nil), // 22: productv1.LookupProductByIdRequestKey + (*LookupProductByIdRequest)(nil), // 23: productv1.LookupProductByIdRequest + (*LookupProductByIdResponse)(nil), // 24: productv1.LookupProductByIdResponse + (*LookupStorageByIdRequestKey)(nil), // 25: productv1.LookupStorageByIdRequestKey + (*LookupStorageByIdRequest)(nil), // 26: productv1.LookupStorageByIdRequest + (*LookupStorageByIdResponse)(nil), // 27: productv1.LookupStorageByIdResponse + (*LookupWarehouseByIdRequestKey)(nil), // 28: productv1.LookupWarehouseByIdRequestKey + (*LookupWarehouseByIdRequest)(nil), // 29: productv1.LookupWarehouseByIdRequest + (*LookupWarehouseByIdResponse)(nil), // 30: productv1.LookupWarehouseByIdResponse + (*QueryUsersRequest)(nil), // 31: productv1.QueryUsersRequest + (*QueryUsersResponse)(nil), // 32: productv1.QueryUsersResponse + (*QueryUserRequest)(nil), // 33: productv1.QueryUserRequest + (*QueryUserResponse)(nil), // 34: productv1.QueryUserResponse + (*QueryNestedTypeRequest)(nil), // 35: productv1.QueryNestedTypeRequest + (*QueryNestedTypeResponse)(nil), // 36: productv1.QueryNestedTypeResponse + (*QueryRecursiveTypeRequest)(nil), // 37: productv1.QueryRecursiveTypeRequest + (*QueryRecursiveTypeResponse)(nil), // 38: productv1.QueryRecursiveTypeResponse + (*QueryTypeFilterWithArgumentsRequest)(nil), // 39: productv1.QueryTypeFilterWithArgumentsRequest + (*QueryTypeFilterWithArgumentsResponse)(nil), // 40: productv1.QueryTypeFilterWithArgumentsResponse + (*QueryTypeWithMultipleFilterFieldsRequest)(nil), // 41: productv1.QueryTypeWithMultipleFilterFieldsRequest + (*QueryTypeWithMultipleFilterFieldsResponse)(nil), // 42: productv1.QueryTypeWithMultipleFilterFieldsResponse + (*QueryComplexFilterTypeRequest)(nil), // 43: productv1.QueryComplexFilterTypeRequest + (*QueryComplexFilterTypeResponse)(nil), // 44: productv1.QueryComplexFilterTypeResponse + (*QueryCalculateTotalsRequest)(nil), // 45: productv1.QueryCalculateTotalsRequest + (*QueryCalculateTotalsResponse)(nil), // 46: productv1.QueryCalculateTotalsResponse + (*QueryCategoriesRequest)(nil), // 47: productv1.QueryCategoriesRequest + (*QueryCategoriesResponse)(nil), // 48: productv1.QueryCategoriesResponse + (*QueryCategoriesByKindRequest)(nil), // 49: productv1.QueryCategoriesByKindRequest + (*QueryCategoriesByKindResponse)(nil), // 50: productv1.QueryCategoriesByKindResponse + (*QueryCategoriesByKindsRequest)(nil), // 51: productv1.QueryCategoriesByKindsRequest + (*QueryCategoriesByKindsResponse)(nil), // 52: productv1.QueryCategoriesByKindsResponse + (*QueryFilterCategoriesRequest)(nil), // 53: productv1.QueryFilterCategoriesRequest + (*QueryFilterCategoriesResponse)(nil), // 54: productv1.QueryFilterCategoriesResponse + (*QueryRandomPetRequest)(nil), // 55: productv1.QueryRandomPetRequest + (*QueryRandomPetResponse)(nil), // 56: productv1.QueryRandomPetResponse + (*QueryAllPetsRequest)(nil), // 57: productv1.QueryAllPetsRequest + (*QueryAllPetsResponse)(nil), // 58: productv1.QueryAllPetsResponse + (*QuerySearchRequest)(nil), // 59: productv1.QuerySearchRequest + (*QuerySearchResponse)(nil), // 60: productv1.QuerySearchResponse + (*QueryRandomSearchResultRequest)(nil), // 61: productv1.QueryRandomSearchResultRequest + (*QueryRandomSearchResultResponse)(nil), // 62: productv1.QueryRandomSearchResultResponse + (*QueryNullableFieldsTypeRequest)(nil), // 63: productv1.QueryNullableFieldsTypeRequest + (*QueryNullableFieldsTypeResponse)(nil), // 64: productv1.QueryNullableFieldsTypeResponse + (*QueryNullableFieldsTypeByIdRequest)(nil), // 65: productv1.QueryNullableFieldsTypeByIdRequest + (*QueryNullableFieldsTypeByIdResponse)(nil), // 66: productv1.QueryNullableFieldsTypeByIdResponse + (*QueryNullableFieldsTypeWithFilterRequest)(nil), // 67: productv1.QueryNullableFieldsTypeWithFilterRequest + (*QueryNullableFieldsTypeWithFilterResponse)(nil), // 68: productv1.QueryNullableFieldsTypeWithFilterResponse + (*QueryAllNullableFieldsTypesRequest)(nil), // 69: productv1.QueryAllNullableFieldsTypesRequest + (*QueryAllNullableFieldsTypesResponse)(nil), // 70: productv1.QueryAllNullableFieldsTypesResponse + (*QueryBlogPostRequest)(nil), // 71: productv1.QueryBlogPostRequest + (*QueryBlogPostResponse)(nil), // 72: productv1.QueryBlogPostResponse + (*QueryBlogPostByIdRequest)(nil), // 73: productv1.QueryBlogPostByIdRequest + (*QueryBlogPostByIdResponse)(nil), // 74: productv1.QueryBlogPostByIdResponse + (*QueryBlogPostsWithFilterRequest)(nil), // 75: productv1.QueryBlogPostsWithFilterRequest + (*QueryBlogPostsWithFilterResponse)(nil), // 76: productv1.QueryBlogPostsWithFilterResponse + (*QueryAllBlogPostsRequest)(nil), // 77: productv1.QueryAllBlogPostsRequest + (*QueryAllBlogPostsResponse)(nil), // 78: productv1.QueryAllBlogPostsResponse + (*QueryAuthorRequest)(nil), // 79: productv1.QueryAuthorRequest + (*QueryAuthorResponse)(nil), // 80: productv1.QueryAuthorResponse + (*QueryAuthorByIdRequest)(nil), // 81: productv1.QueryAuthorByIdRequest + (*QueryAuthorByIdResponse)(nil), // 82: productv1.QueryAuthorByIdResponse + (*QueryAuthorsWithFilterRequest)(nil), // 83: productv1.QueryAuthorsWithFilterRequest + (*QueryAuthorsWithFilterResponse)(nil), // 84: productv1.QueryAuthorsWithFilterResponse + (*QueryAllAuthorsRequest)(nil), // 85: productv1.QueryAllAuthorsRequest + (*QueryAllAuthorsResponse)(nil), // 86: productv1.QueryAllAuthorsResponse + (*QueryBulkSearchAuthorsRequest)(nil), // 87: productv1.QueryBulkSearchAuthorsRequest + (*QueryBulkSearchAuthorsResponse)(nil), // 88: productv1.QueryBulkSearchAuthorsResponse + (*QueryBulkSearchBlogPostsRequest)(nil), // 89: productv1.QueryBulkSearchBlogPostsRequest + (*QueryBulkSearchBlogPostsResponse)(nil), // 90: productv1.QueryBulkSearchBlogPostsResponse + (*MutationCreateUserRequest)(nil), // 91: productv1.MutationCreateUserRequest + (*MutationCreateUserResponse)(nil), // 92: productv1.MutationCreateUserResponse + (*MutationPerformActionRequest)(nil), // 93: productv1.MutationPerformActionRequest + (*MutationPerformActionResponse)(nil), // 94: productv1.MutationPerformActionResponse + (*MutationCreateNullableFieldsTypeRequest)(nil), // 95: productv1.MutationCreateNullableFieldsTypeRequest + (*MutationCreateNullableFieldsTypeResponse)(nil), // 96: productv1.MutationCreateNullableFieldsTypeResponse + (*MutationUpdateNullableFieldsTypeRequest)(nil), // 97: productv1.MutationUpdateNullableFieldsTypeRequest + (*MutationUpdateNullableFieldsTypeResponse)(nil), // 98: productv1.MutationUpdateNullableFieldsTypeResponse + (*MutationCreateBlogPostRequest)(nil), // 99: productv1.MutationCreateBlogPostRequest + (*MutationCreateBlogPostResponse)(nil), // 100: productv1.MutationCreateBlogPostResponse + (*MutationUpdateBlogPostRequest)(nil), // 101: productv1.MutationUpdateBlogPostRequest + (*MutationUpdateBlogPostResponse)(nil), // 102: productv1.MutationUpdateBlogPostResponse + (*MutationCreateAuthorRequest)(nil), // 103: productv1.MutationCreateAuthorRequest + (*MutationCreateAuthorResponse)(nil), // 104: productv1.MutationCreateAuthorResponse + (*MutationUpdateAuthorRequest)(nil), // 105: productv1.MutationUpdateAuthorRequest + (*MutationUpdateAuthorResponse)(nil), // 106: productv1.MutationUpdateAuthorResponse + (*MutationBulkCreateAuthorsRequest)(nil), // 107: productv1.MutationBulkCreateAuthorsRequest + (*MutationBulkCreateAuthorsResponse)(nil), // 108: productv1.MutationBulkCreateAuthorsResponse + (*MutationBulkUpdateAuthorsRequest)(nil), // 109: productv1.MutationBulkUpdateAuthorsRequest + (*MutationBulkUpdateAuthorsResponse)(nil), // 110: productv1.MutationBulkUpdateAuthorsResponse + (*MutationBulkCreateBlogPostsRequest)(nil), // 111: productv1.MutationBulkCreateBlogPostsRequest + (*MutationBulkCreateBlogPostsResponse)(nil), // 112: productv1.MutationBulkCreateBlogPostsResponse + (*MutationBulkUpdateBlogPostsRequest)(nil), // 113: productv1.MutationBulkUpdateBlogPostsRequest + (*MutationBulkUpdateBlogPostsResponse)(nil), // 114: productv1.MutationBulkUpdateBlogPostsResponse + (*ResolveProductShippingEstimateArgs)(nil), // 115: productv1.ResolveProductShippingEstimateArgs + (*ResolveProductShippingEstimateContext)(nil), // 116: productv1.ResolveProductShippingEstimateContext + (*ResolveProductShippingEstimateRequest)(nil), // 117: productv1.ResolveProductShippingEstimateRequest + (*ResolveProductShippingEstimateResult)(nil), // 118: productv1.ResolveProductShippingEstimateResult + (*ResolveProductShippingEstimateResponse)(nil), // 119: productv1.ResolveProductShippingEstimateResponse + (*ResolveProductRecommendedCategoryArgs)(nil), // 120: productv1.ResolveProductRecommendedCategoryArgs + (*ResolveProductRecommendedCategoryContext)(nil), // 121: productv1.ResolveProductRecommendedCategoryContext + (*ResolveProductRecommendedCategoryRequest)(nil), // 122: productv1.ResolveProductRecommendedCategoryRequest + (*ResolveProductRecommendedCategoryResult)(nil), // 123: productv1.ResolveProductRecommendedCategoryResult + (*ResolveProductRecommendedCategoryResponse)(nil), // 124: productv1.ResolveProductRecommendedCategoryResponse + (*ResolveCategoryProductCountArgs)(nil), // 125: productv1.ResolveCategoryProductCountArgs + (*ResolveCategoryProductCountContext)(nil), // 126: productv1.ResolveCategoryProductCountContext + (*ResolveCategoryProductCountRequest)(nil), // 127: productv1.ResolveCategoryProductCountRequest + (*ResolveCategoryProductCountResult)(nil), // 128: productv1.ResolveCategoryProductCountResult + (*ResolveCategoryProductCountResponse)(nil), // 129: productv1.ResolveCategoryProductCountResponse + (*ResolveCategoryPopularityScoreArgs)(nil), // 130: productv1.ResolveCategoryPopularityScoreArgs + (*ResolveCategoryPopularityScoreContext)(nil), // 131: productv1.ResolveCategoryPopularityScoreContext + (*ResolveCategoryPopularityScoreRequest)(nil), // 132: productv1.ResolveCategoryPopularityScoreRequest + (*ResolveCategoryPopularityScoreResult)(nil), // 133: productv1.ResolveCategoryPopularityScoreResult + (*ResolveCategoryPopularityScoreResponse)(nil), // 134: productv1.ResolveCategoryPopularityScoreResponse + (*ResolveCategoryCategoryMetricsArgs)(nil), // 135: productv1.ResolveCategoryCategoryMetricsArgs + (*ResolveCategoryCategoryMetricsContext)(nil), // 136: productv1.ResolveCategoryCategoryMetricsContext + (*ResolveCategoryCategoryMetricsRequest)(nil), // 137: productv1.ResolveCategoryCategoryMetricsRequest + (*ResolveCategoryCategoryMetricsResult)(nil), // 138: productv1.ResolveCategoryCategoryMetricsResult + (*ResolveCategoryCategoryMetricsResponse)(nil), // 139: productv1.ResolveCategoryCategoryMetricsResponse + (*ResolveSubcategoryItemCountArgs)(nil), // 140: productv1.ResolveSubcategoryItemCountArgs + (*ResolveSubcategoryItemCountContext)(nil), // 141: productv1.ResolveSubcategoryItemCountContext + (*ResolveSubcategoryItemCountRequest)(nil), // 142: productv1.ResolveSubcategoryItemCountRequest + (*ResolveSubcategoryItemCountResult)(nil), // 143: productv1.ResolveSubcategoryItemCountResult + (*ResolveSubcategoryItemCountResponse)(nil), // 144: productv1.ResolveSubcategoryItemCountResponse + (*Product)(nil), // 145: productv1.Product + (*Storage)(nil), // 146: productv1.Storage + (*Warehouse)(nil), // 147: productv1.Warehouse + (*User)(nil), // 148: productv1.User + (*NestedTypeA)(nil), // 149: productv1.NestedTypeA + (*RecursiveType)(nil), // 150: productv1.RecursiveType + (*TypeWithMultipleFilterFields)(nil), // 151: productv1.TypeWithMultipleFilterFields + (*FilterTypeInput)(nil), // 152: productv1.FilterTypeInput + (*ComplexFilterTypeInput)(nil), // 153: productv1.ComplexFilterTypeInput + (*TypeWithComplexFilterInput)(nil), // 154: productv1.TypeWithComplexFilterInput + (*OrderInput)(nil), // 155: productv1.OrderInput + (*Order)(nil), // 156: productv1.Order + (*Category)(nil), // 157: productv1.Category + (*CategoryFilter)(nil), // 158: productv1.CategoryFilter + (*Animal)(nil), // 159: productv1.Animal + (*SearchInput)(nil), // 160: productv1.SearchInput + (*SearchResult)(nil), // 161: productv1.SearchResult + (*NullableFieldsType)(nil), // 162: productv1.NullableFieldsType + (*NullableFieldsFilter)(nil), // 163: productv1.NullableFieldsFilter + (*BlogPost)(nil), // 164: productv1.BlogPost + (*BlogPostFilter)(nil), // 165: productv1.BlogPostFilter + (*Author)(nil), // 166: productv1.Author + (*AuthorFilter)(nil), // 167: productv1.AuthorFilter + (*UserInput)(nil), // 168: productv1.UserInput + (*ActionInput)(nil), // 169: productv1.ActionInput + (*ActionResult)(nil), // 170: productv1.ActionResult + (*NullableFieldsInput)(nil), // 171: productv1.NullableFieldsInput + (*BlogPostInput)(nil), // 172: productv1.BlogPostInput + (*AuthorInput)(nil), // 173: productv1.AuthorInput + (*NestedTypeB)(nil), // 174: productv1.NestedTypeB + (*NestedTypeC)(nil), // 175: productv1.NestedTypeC + (*FilterType)(nil), // 176: productv1.FilterType + (*Pagination)(nil), // 177: productv1.Pagination + (*OrderLineInput)(nil), // 178: productv1.OrderLineInput + (*OrderLine)(nil), // 179: productv1.OrderLine + (*Subcategory)(nil), // 180: productv1.Subcategory + (*CategoryMetrics)(nil), // 181: productv1.CategoryMetrics + (*Cat)(nil), // 182: productv1.Cat + (*Dog)(nil), // 183: productv1.Dog + (*ActionSuccess)(nil), // 184: productv1.ActionSuccess + (*ActionError)(nil), // 185: productv1.ActionError + (*CategoryInput)(nil), // 186: productv1.CategoryInput + (*ProductCountFilter)(nil), // 187: productv1.ProductCountFilter + (*SubcategoryItemFilter)(nil), // 188: productv1.SubcategoryItemFilter + (*ShippingEstimateInput)(nil), // 189: productv1.ShippingEstimateInput + (*ListOfAuthorFilter_List)(nil), // 190: productv1.ListOfAuthorFilter.List + (*ListOfAuthorInput_List)(nil), // 191: productv1.ListOfAuthorInput.List + (*ListOfBlogPost_List)(nil), // 192: productv1.ListOfBlogPost.List + (*ListOfBlogPostFilter_List)(nil), // 193: productv1.ListOfBlogPostFilter.List + (*ListOfBlogPostInput_List)(nil), // 194: productv1.ListOfBlogPostInput.List + (*ListOfBoolean_List)(nil), // 195: productv1.ListOfBoolean.List + (*ListOfCategory_List)(nil), // 196: productv1.ListOfCategory.List + (*ListOfCategoryInput_List)(nil), // 197: productv1.ListOfCategoryInput.List + (*ListOfFloat_List)(nil), // 198: productv1.ListOfFloat.List + (*ListOfListOfCategory_List)(nil), // 199: productv1.ListOfListOfCategory.List + (*ListOfListOfCategoryInput_List)(nil), // 200: productv1.ListOfListOfCategoryInput.List + (*ListOfListOfString_List)(nil), // 201: productv1.ListOfListOfString.List + (*ListOfListOfUser_List)(nil), // 202: productv1.ListOfListOfUser.List + (*ListOfListOfUserInput_List)(nil), // 203: productv1.ListOfListOfUserInput.List + (*ListOfOrderLine_List)(nil), // 204: productv1.ListOfOrderLine.List + (*ListOfProduct_List)(nil), // 205: productv1.ListOfProduct.List + (*ListOfString_List)(nil), // 206: productv1.ListOfString.List + (*ListOfSubcategory_List)(nil), // 207: productv1.ListOfSubcategory.List + (*ListOfUser_List)(nil), // 208: productv1.ListOfUser.List + (*ListOfUserInput_List)(nil), // 209: productv1.ListOfUserInput.List + (*wrapperspb.Int32Value)(nil), // 210: google.protobuf.Int32Value + (*wrapperspb.StringValue)(nil), // 211: google.protobuf.StringValue + (*wrapperspb.DoubleValue)(nil), // 212: google.protobuf.DoubleValue + (*wrapperspb.BoolValue)(nil), // 213: google.protobuf.BoolValue +} +var file_testdata_service_proto_depIdxs = []int32{ + 190, // 0: productv1.ListOfAuthorFilter.list:type_name -> productv1.ListOfAuthorFilter.List + 191, // 1: productv1.ListOfAuthorInput.list:type_name -> productv1.ListOfAuthorInput.List + 192, // 2: productv1.ListOfBlogPost.list:type_name -> productv1.ListOfBlogPost.List + 193, // 3: productv1.ListOfBlogPostFilter.list:type_name -> productv1.ListOfBlogPostFilter.List + 194, // 4: productv1.ListOfBlogPostInput.list:type_name -> productv1.ListOfBlogPostInput.List + 195, // 5: productv1.ListOfBoolean.list:type_name -> productv1.ListOfBoolean.List + 196, // 6: productv1.ListOfCategory.list:type_name -> productv1.ListOfCategory.List + 197, // 7: productv1.ListOfCategoryInput.list:type_name -> productv1.ListOfCategoryInput.List + 198, // 8: productv1.ListOfFloat.list:type_name -> productv1.ListOfFloat.List + 199, // 9: productv1.ListOfListOfCategory.list:type_name -> productv1.ListOfListOfCategory.List + 200, // 10: productv1.ListOfListOfCategoryInput.list:type_name -> productv1.ListOfListOfCategoryInput.List + 201, // 11: productv1.ListOfListOfString.list:type_name -> productv1.ListOfListOfString.List + 202, // 12: productv1.ListOfListOfUser.list:type_name -> productv1.ListOfListOfUser.List + 203, // 13: productv1.ListOfListOfUserInput.list:type_name -> productv1.ListOfListOfUserInput.List + 204, // 14: productv1.ListOfOrderLine.list:type_name -> productv1.ListOfOrderLine.List + 205, // 15: productv1.ListOfProduct.list:type_name -> productv1.ListOfProduct.List + 206, // 16: productv1.ListOfString.list:type_name -> productv1.ListOfString.List + 207, // 17: productv1.ListOfSubcategory.list:type_name -> productv1.ListOfSubcategory.List + 208, // 18: productv1.ListOfUser.list:type_name -> productv1.ListOfUser.List + 209, // 19: productv1.ListOfUserInput.list:type_name -> productv1.ListOfUserInput.List + 22, // 20: productv1.LookupProductByIdRequest.keys:type_name -> productv1.LookupProductByIdRequestKey + 145, // 21: productv1.LookupProductByIdResponse.result:type_name -> productv1.Product + 25, // 22: productv1.LookupStorageByIdRequest.keys:type_name -> productv1.LookupStorageByIdRequestKey + 146, // 23: productv1.LookupStorageByIdResponse.result:type_name -> productv1.Storage + 28, // 24: productv1.LookupWarehouseByIdRequest.keys:type_name -> productv1.LookupWarehouseByIdRequestKey + 147, // 25: productv1.LookupWarehouseByIdResponse.result:type_name -> productv1.Warehouse + 148, // 26: productv1.QueryUsersResponse.users:type_name -> productv1.User + 148, // 27: productv1.QueryUserResponse.user:type_name -> productv1.User + 149, // 28: productv1.QueryNestedTypeResponse.nested_type:type_name -> productv1.NestedTypeA + 150, // 29: productv1.QueryRecursiveTypeResponse.recursive_type:type_name -> productv1.RecursiveType + 151, // 30: productv1.QueryTypeFilterWithArgumentsResponse.type_filter_with_arguments:type_name -> productv1.TypeWithMultipleFilterFields + 152, // 31: productv1.QueryTypeWithMultipleFilterFieldsRequest.filter:type_name -> productv1.FilterTypeInput + 151, // 32: productv1.QueryTypeWithMultipleFilterFieldsResponse.type_with_multiple_filter_fields:type_name -> productv1.TypeWithMultipleFilterFields + 153, // 33: productv1.QueryComplexFilterTypeRequest.filter:type_name -> productv1.ComplexFilterTypeInput + 154, // 34: productv1.QueryComplexFilterTypeResponse.complex_filter_type:type_name -> productv1.TypeWithComplexFilterInput + 155, // 35: productv1.QueryCalculateTotalsRequest.orders:type_name -> productv1.OrderInput + 156, // 36: productv1.QueryCalculateTotalsResponse.calculate_totals:type_name -> productv1.Order + 157, // 37: productv1.QueryCategoriesResponse.categories:type_name -> productv1.Category + 0, // 38: productv1.QueryCategoriesByKindRequest.kind:type_name -> productv1.CategoryKind + 157, // 39: productv1.QueryCategoriesByKindResponse.categories_by_kind:type_name -> productv1.Category + 0, // 40: productv1.QueryCategoriesByKindsRequest.kinds:type_name -> productv1.CategoryKind + 157, // 41: productv1.QueryCategoriesByKindsResponse.categories_by_kinds:type_name -> productv1.Category + 158, // 42: productv1.QueryFilterCategoriesRequest.filter:type_name -> productv1.CategoryFilter + 157, // 43: productv1.QueryFilterCategoriesResponse.filter_categories:type_name -> productv1.Category + 159, // 44: productv1.QueryRandomPetResponse.random_pet:type_name -> productv1.Animal + 159, // 45: productv1.QueryAllPetsResponse.all_pets:type_name -> productv1.Animal + 160, // 46: productv1.QuerySearchRequest.input:type_name -> productv1.SearchInput + 161, // 47: productv1.QuerySearchResponse.search:type_name -> productv1.SearchResult + 161, // 48: productv1.QueryRandomSearchResultResponse.random_search_result:type_name -> productv1.SearchResult + 162, // 49: productv1.QueryNullableFieldsTypeResponse.nullable_fields_type:type_name -> productv1.NullableFieldsType + 162, // 50: productv1.QueryNullableFieldsTypeByIdResponse.nullable_fields_type_by_id:type_name -> productv1.NullableFieldsType + 163, // 51: productv1.QueryNullableFieldsTypeWithFilterRequest.filter:type_name -> productv1.NullableFieldsFilter + 162, // 52: productv1.QueryNullableFieldsTypeWithFilterResponse.nullable_fields_type_with_filter:type_name -> productv1.NullableFieldsType + 162, // 53: productv1.QueryAllNullableFieldsTypesResponse.all_nullable_fields_types:type_name -> productv1.NullableFieldsType + 164, // 54: productv1.QueryBlogPostResponse.blog_post:type_name -> productv1.BlogPost + 164, // 55: productv1.QueryBlogPostByIdResponse.blog_post_by_id:type_name -> productv1.BlogPost + 165, // 56: productv1.QueryBlogPostsWithFilterRequest.filter:type_name -> productv1.BlogPostFilter + 164, // 57: productv1.QueryBlogPostsWithFilterResponse.blog_posts_with_filter:type_name -> productv1.BlogPost + 164, // 58: productv1.QueryAllBlogPostsResponse.all_blog_posts:type_name -> productv1.BlogPost + 166, // 59: productv1.QueryAuthorResponse.author:type_name -> productv1.Author + 166, // 60: productv1.QueryAuthorByIdResponse.author_by_id:type_name -> productv1.Author + 167, // 61: productv1.QueryAuthorsWithFilterRequest.filter:type_name -> productv1.AuthorFilter + 166, // 62: productv1.QueryAuthorsWithFilterResponse.authors_with_filter:type_name -> productv1.Author + 166, // 63: productv1.QueryAllAuthorsResponse.all_authors:type_name -> productv1.Author + 2, // 64: productv1.QueryBulkSearchAuthorsRequest.filters:type_name -> productv1.ListOfAuthorFilter + 166, // 65: productv1.QueryBulkSearchAuthorsResponse.bulk_search_authors:type_name -> productv1.Author + 5, // 66: productv1.QueryBulkSearchBlogPostsRequest.filters:type_name -> productv1.ListOfBlogPostFilter + 164, // 67: productv1.QueryBulkSearchBlogPostsResponse.bulk_search_blog_posts:type_name -> productv1.BlogPost + 168, // 68: productv1.MutationCreateUserRequest.input:type_name -> productv1.UserInput + 148, // 69: productv1.MutationCreateUserResponse.create_user:type_name -> productv1.User + 169, // 70: productv1.MutationPerformActionRequest.input:type_name -> productv1.ActionInput + 170, // 71: productv1.MutationPerformActionResponse.perform_action:type_name -> productv1.ActionResult + 171, // 72: productv1.MutationCreateNullableFieldsTypeRequest.input:type_name -> productv1.NullableFieldsInput + 162, // 73: productv1.MutationCreateNullableFieldsTypeResponse.create_nullable_fields_type:type_name -> productv1.NullableFieldsType + 171, // 74: productv1.MutationUpdateNullableFieldsTypeRequest.input:type_name -> productv1.NullableFieldsInput + 162, // 75: productv1.MutationUpdateNullableFieldsTypeResponse.update_nullable_fields_type:type_name -> productv1.NullableFieldsType + 172, // 76: productv1.MutationCreateBlogPostRequest.input:type_name -> productv1.BlogPostInput + 164, // 77: productv1.MutationCreateBlogPostResponse.create_blog_post:type_name -> productv1.BlogPost + 172, // 78: productv1.MutationUpdateBlogPostRequest.input:type_name -> productv1.BlogPostInput + 164, // 79: productv1.MutationUpdateBlogPostResponse.update_blog_post:type_name -> productv1.BlogPost + 173, // 80: productv1.MutationCreateAuthorRequest.input:type_name -> productv1.AuthorInput + 166, // 81: productv1.MutationCreateAuthorResponse.create_author:type_name -> productv1.Author + 173, // 82: productv1.MutationUpdateAuthorRequest.input:type_name -> productv1.AuthorInput + 166, // 83: productv1.MutationUpdateAuthorResponse.update_author:type_name -> productv1.Author + 3, // 84: productv1.MutationBulkCreateAuthorsRequest.authors:type_name -> productv1.ListOfAuthorInput + 166, // 85: productv1.MutationBulkCreateAuthorsResponse.bulk_create_authors:type_name -> productv1.Author + 3, // 86: productv1.MutationBulkUpdateAuthorsRequest.authors:type_name -> productv1.ListOfAuthorInput + 166, // 87: productv1.MutationBulkUpdateAuthorsResponse.bulk_update_authors:type_name -> productv1.Author + 6, // 88: productv1.MutationBulkCreateBlogPostsRequest.blog_posts:type_name -> productv1.ListOfBlogPostInput + 164, // 89: productv1.MutationBulkCreateBlogPostsResponse.bulk_create_blog_posts:type_name -> productv1.BlogPost + 6, // 90: productv1.MutationBulkUpdateBlogPostsRequest.blog_posts:type_name -> productv1.ListOfBlogPostInput + 164, // 91: productv1.MutationBulkUpdateBlogPostsResponse.bulk_update_blog_posts:type_name -> productv1.BlogPost + 189, // 92: productv1.ResolveProductShippingEstimateArgs.input:type_name -> productv1.ShippingEstimateInput + 116, // 93: productv1.ResolveProductShippingEstimateRequest.context:type_name -> productv1.ResolveProductShippingEstimateContext + 115, // 94: productv1.ResolveProductShippingEstimateRequest.field_args:type_name -> productv1.ResolveProductShippingEstimateArgs + 118, // 95: productv1.ResolveProductShippingEstimateResponse.result:type_name -> productv1.ResolveProductShippingEstimateResult + 121, // 96: productv1.ResolveProductRecommendedCategoryRequest.context:type_name -> productv1.ResolveProductRecommendedCategoryContext + 120, // 97: productv1.ResolveProductRecommendedCategoryRequest.field_args:type_name -> productv1.ResolveProductRecommendedCategoryArgs + 157, // 98: productv1.ResolveProductRecommendedCategoryResult.recommended_category:type_name -> productv1.Category + 123, // 99: productv1.ResolveProductRecommendedCategoryResponse.result:type_name -> productv1.ResolveProductRecommendedCategoryResult + 187, // 100: productv1.ResolveCategoryProductCountArgs.filters:type_name -> productv1.ProductCountFilter + 126, // 101: productv1.ResolveCategoryProductCountRequest.context:type_name -> productv1.ResolveCategoryProductCountContext + 125, // 102: productv1.ResolveCategoryProductCountRequest.field_args:type_name -> productv1.ResolveCategoryProductCountArgs + 128, // 103: productv1.ResolveCategoryProductCountResponse.result:type_name -> productv1.ResolveCategoryProductCountResult + 210, // 104: productv1.ResolveCategoryPopularityScoreArgs.threshold:type_name -> google.protobuf.Int32Value + 131, // 105: productv1.ResolveCategoryPopularityScoreRequest.context:type_name -> productv1.ResolveCategoryPopularityScoreContext + 130, // 106: productv1.ResolveCategoryPopularityScoreRequest.field_args:type_name -> productv1.ResolveCategoryPopularityScoreArgs + 210, // 107: productv1.ResolveCategoryPopularityScoreResult.popularity_score:type_name -> google.protobuf.Int32Value + 133, // 108: productv1.ResolveCategoryPopularityScoreResponse.result:type_name -> productv1.ResolveCategoryPopularityScoreResult + 136, // 109: productv1.ResolveCategoryCategoryMetricsRequest.context:type_name -> productv1.ResolveCategoryCategoryMetricsContext + 135, // 110: productv1.ResolveCategoryCategoryMetricsRequest.field_args:type_name -> productv1.ResolveCategoryCategoryMetricsArgs + 181, // 111: productv1.ResolveCategoryCategoryMetricsResult.category_metrics:type_name -> productv1.CategoryMetrics + 138, // 112: productv1.ResolveCategoryCategoryMetricsResponse.result:type_name -> productv1.ResolveCategoryCategoryMetricsResult + 188, // 113: productv1.ResolveSubcategoryItemCountArgs.filters:type_name -> productv1.SubcategoryItemFilter + 141, // 114: productv1.ResolveSubcategoryItemCountRequest.context:type_name -> productv1.ResolveSubcategoryItemCountContext + 140, // 115: productv1.ResolveSubcategoryItemCountRequest.field_args:type_name -> productv1.ResolveSubcategoryItemCountArgs + 143, // 116: productv1.ResolveSubcategoryItemCountResponse.result:type_name -> productv1.ResolveSubcategoryItemCountResult + 174, // 117: productv1.NestedTypeA.b:type_name -> productv1.NestedTypeB + 150, // 118: productv1.RecursiveType.recursive_type:type_name -> productv1.RecursiveType + 176, // 119: productv1.ComplexFilterTypeInput.filter:type_name -> productv1.FilterType + 178, // 120: productv1.OrderInput.lines:type_name -> productv1.OrderLineInput + 16, // 121: productv1.Order.order_lines:type_name -> productv1.ListOfOrderLine + 0, // 122: productv1.Category.kind:type_name -> productv1.CategoryKind + 19, // 123: productv1.Category.subcategories:type_name -> productv1.ListOfSubcategory + 0, // 124: productv1.CategoryFilter.category:type_name -> productv1.CategoryKind + 177, // 125: productv1.CategoryFilter.pagination:type_name -> productv1.Pagination + 182, // 126: productv1.Animal.cat:type_name -> productv1.Cat + 183, // 127: productv1.Animal.dog:type_name -> productv1.Dog + 210, // 128: productv1.SearchInput.limit:type_name -> google.protobuf.Int32Value + 145, // 129: productv1.SearchResult.product:type_name -> productv1.Product + 148, // 130: productv1.SearchResult.user:type_name -> productv1.User + 157, // 131: productv1.SearchResult.category:type_name -> productv1.Category + 211, // 132: productv1.NullableFieldsType.optional_string:type_name -> google.protobuf.StringValue + 210, // 133: productv1.NullableFieldsType.optional_int:type_name -> google.protobuf.Int32Value + 212, // 134: productv1.NullableFieldsType.optional_float:type_name -> google.protobuf.DoubleValue + 213, // 135: productv1.NullableFieldsType.optional_boolean:type_name -> google.protobuf.BoolValue + 211, // 136: productv1.NullableFieldsFilter.name:type_name -> google.protobuf.StringValue + 211, // 137: productv1.NullableFieldsFilter.optional_string:type_name -> google.protobuf.StringValue + 213, // 138: productv1.NullableFieldsFilter.include_nulls:type_name -> google.protobuf.BoolValue + 18, // 139: productv1.BlogPost.optional_tags:type_name -> productv1.ListOfString + 18, // 140: productv1.BlogPost.keywords:type_name -> productv1.ListOfString + 10, // 141: productv1.BlogPost.ratings:type_name -> productv1.ListOfFloat + 7, // 142: productv1.BlogPost.is_published:type_name -> productv1.ListOfBoolean + 13, // 143: productv1.BlogPost.tag_groups:type_name -> productv1.ListOfListOfString + 13, // 144: productv1.BlogPost.related_topics:type_name -> productv1.ListOfListOfString + 13, // 145: productv1.BlogPost.comment_threads:type_name -> productv1.ListOfListOfString + 13, // 146: productv1.BlogPost.suggestions:type_name -> productv1.ListOfListOfString + 157, // 147: productv1.BlogPost.related_categories:type_name -> productv1.Category + 148, // 148: productv1.BlogPost.contributors:type_name -> productv1.User + 17, // 149: productv1.BlogPost.mentioned_products:type_name -> productv1.ListOfProduct + 20, // 150: productv1.BlogPost.mentioned_users:type_name -> productv1.ListOfUser + 11, // 151: productv1.BlogPost.category_groups:type_name -> productv1.ListOfListOfCategory + 14, // 152: productv1.BlogPost.contributor_teams:type_name -> productv1.ListOfListOfUser + 211, // 153: productv1.BlogPostFilter.title:type_name -> google.protobuf.StringValue + 213, // 154: productv1.BlogPostFilter.has_categories:type_name -> google.protobuf.BoolValue + 210, // 155: productv1.BlogPostFilter.min_tags:type_name -> google.protobuf.Int32Value + 211, // 156: productv1.Author.email:type_name -> google.protobuf.StringValue + 18, // 157: productv1.Author.social_links:type_name -> productv1.ListOfString + 13, // 158: productv1.Author.teams_by_project:type_name -> productv1.ListOfListOfString + 13, // 159: productv1.Author.collaborations:type_name -> productv1.ListOfListOfString + 4, // 160: productv1.Author.written_posts:type_name -> productv1.ListOfBlogPost + 157, // 161: productv1.Author.favorite_categories:type_name -> productv1.Category + 20, // 162: productv1.Author.related_authors:type_name -> productv1.ListOfUser + 17, // 163: productv1.Author.product_reviews:type_name -> productv1.ListOfProduct + 14, // 164: productv1.Author.author_groups:type_name -> productv1.ListOfListOfUser + 11, // 165: productv1.Author.category_preferences:type_name -> productv1.ListOfListOfCategory + 14, // 166: productv1.Author.project_teams:type_name -> productv1.ListOfListOfUser + 211, // 167: productv1.AuthorFilter.name:type_name -> google.protobuf.StringValue + 213, // 168: productv1.AuthorFilter.has_teams:type_name -> google.protobuf.BoolValue + 210, // 169: productv1.AuthorFilter.skill_count:type_name -> google.protobuf.Int32Value + 184, // 170: productv1.ActionResult.action_success:type_name -> productv1.ActionSuccess + 185, // 171: productv1.ActionResult.action_error:type_name -> productv1.ActionError + 211, // 172: productv1.NullableFieldsInput.optional_string:type_name -> google.protobuf.StringValue + 210, // 173: productv1.NullableFieldsInput.optional_int:type_name -> google.protobuf.Int32Value + 212, // 174: productv1.NullableFieldsInput.optional_float:type_name -> google.protobuf.DoubleValue + 213, // 175: productv1.NullableFieldsInput.optional_boolean:type_name -> google.protobuf.BoolValue + 18, // 176: productv1.BlogPostInput.optional_tags:type_name -> productv1.ListOfString + 18, // 177: productv1.BlogPostInput.keywords:type_name -> productv1.ListOfString + 10, // 178: productv1.BlogPostInput.ratings:type_name -> productv1.ListOfFloat + 7, // 179: productv1.BlogPostInput.is_published:type_name -> productv1.ListOfBoolean + 13, // 180: productv1.BlogPostInput.tag_groups:type_name -> productv1.ListOfListOfString + 13, // 181: productv1.BlogPostInput.related_topics:type_name -> productv1.ListOfListOfString + 13, // 182: productv1.BlogPostInput.comment_threads:type_name -> productv1.ListOfListOfString + 13, // 183: productv1.BlogPostInput.suggestions:type_name -> productv1.ListOfListOfString + 9, // 184: productv1.BlogPostInput.related_categories:type_name -> productv1.ListOfCategoryInput + 21, // 185: productv1.BlogPostInput.contributors:type_name -> productv1.ListOfUserInput + 12, // 186: productv1.BlogPostInput.category_groups:type_name -> productv1.ListOfListOfCategoryInput + 211, // 187: productv1.AuthorInput.email:type_name -> google.protobuf.StringValue + 18, // 188: productv1.AuthorInput.social_links:type_name -> productv1.ListOfString + 13, // 189: productv1.AuthorInput.teams_by_project:type_name -> productv1.ListOfListOfString + 13, // 190: productv1.AuthorInput.collaborations:type_name -> productv1.ListOfListOfString + 186, // 191: productv1.AuthorInput.favorite_categories:type_name -> productv1.CategoryInput + 15, // 192: productv1.AuthorInput.author_groups:type_name -> productv1.ListOfListOfUserInput + 15, // 193: productv1.AuthorInput.project_teams:type_name -> productv1.ListOfListOfUserInput + 175, // 194: productv1.NestedTypeB.c:type_name -> productv1.NestedTypeC + 177, // 195: productv1.FilterType.pagination:type_name -> productv1.Pagination + 18, // 196: productv1.OrderLineInput.modifiers:type_name -> productv1.ListOfString + 18, // 197: productv1.OrderLine.modifiers:type_name -> productv1.ListOfString + 211, // 198: productv1.Subcategory.description:type_name -> google.protobuf.StringValue + 0, // 199: productv1.CategoryInput.kind:type_name -> productv1.CategoryKind + 212, // 200: productv1.ProductCountFilter.min_price:type_name -> google.protobuf.DoubleValue + 212, // 201: productv1.ProductCountFilter.max_price:type_name -> google.protobuf.DoubleValue + 213, // 202: productv1.ProductCountFilter.in_stock:type_name -> google.protobuf.BoolValue + 211, // 203: productv1.ProductCountFilter.search_term:type_name -> google.protobuf.StringValue + 212, // 204: productv1.SubcategoryItemFilter.min_price:type_name -> google.protobuf.DoubleValue + 212, // 205: productv1.SubcategoryItemFilter.max_price:type_name -> google.protobuf.DoubleValue + 213, // 206: productv1.SubcategoryItemFilter.in_stock:type_name -> google.protobuf.BoolValue + 213, // 207: productv1.SubcategoryItemFilter.is_active:type_name -> google.protobuf.BoolValue + 211, // 208: productv1.SubcategoryItemFilter.search_term:type_name -> google.protobuf.StringValue + 1, // 209: productv1.ShippingEstimateInput.destination:type_name -> productv1.ShippingDestination + 213, // 210: productv1.ShippingEstimateInput.expedited:type_name -> google.protobuf.BoolValue + 167, // 211: productv1.ListOfAuthorFilter.List.items:type_name -> productv1.AuthorFilter + 173, // 212: productv1.ListOfAuthorInput.List.items:type_name -> productv1.AuthorInput + 164, // 213: productv1.ListOfBlogPost.List.items:type_name -> productv1.BlogPost + 165, // 214: productv1.ListOfBlogPostFilter.List.items:type_name -> productv1.BlogPostFilter + 172, // 215: productv1.ListOfBlogPostInput.List.items:type_name -> productv1.BlogPostInput + 157, // 216: productv1.ListOfCategory.List.items:type_name -> productv1.Category + 186, // 217: productv1.ListOfCategoryInput.List.items:type_name -> productv1.CategoryInput + 8, // 218: productv1.ListOfListOfCategory.List.items:type_name -> productv1.ListOfCategory + 9, // 219: productv1.ListOfListOfCategoryInput.List.items:type_name -> productv1.ListOfCategoryInput + 18, // 220: productv1.ListOfListOfString.List.items:type_name -> productv1.ListOfString + 20, // 221: productv1.ListOfListOfUser.List.items:type_name -> productv1.ListOfUser + 21, // 222: productv1.ListOfListOfUserInput.List.items:type_name -> productv1.ListOfUserInput + 179, // 223: productv1.ListOfOrderLine.List.items:type_name -> productv1.OrderLine + 145, // 224: productv1.ListOfProduct.List.items:type_name -> productv1.Product + 180, // 225: productv1.ListOfSubcategory.List.items:type_name -> productv1.Subcategory + 148, // 226: productv1.ListOfUser.List.items:type_name -> productv1.User + 168, // 227: productv1.ListOfUserInput.List.items:type_name -> productv1.UserInput + 23, // 228: productv1.ProductService.LookupProductById:input_type -> productv1.LookupProductByIdRequest + 26, // 229: productv1.ProductService.LookupStorageById:input_type -> productv1.LookupStorageByIdRequest + 29, // 230: productv1.ProductService.LookupWarehouseById:input_type -> productv1.LookupWarehouseByIdRequest + 107, // 231: productv1.ProductService.MutationBulkCreateAuthors:input_type -> productv1.MutationBulkCreateAuthorsRequest + 111, // 232: productv1.ProductService.MutationBulkCreateBlogPosts:input_type -> productv1.MutationBulkCreateBlogPostsRequest + 109, // 233: productv1.ProductService.MutationBulkUpdateAuthors:input_type -> productv1.MutationBulkUpdateAuthorsRequest + 113, // 234: productv1.ProductService.MutationBulkUpdateBlogPosts:input_type -> productv1.MutationBulkUpdateBlogPostsRequest + 103, // 235: productv1.ProductService.MutationCreateAuthor:input_type -> productv1.MutationCreateAuthorRequest + 99, // 236: productv1.ProductService.MutationCreateBlogPost:input_type -> productv1.MutationCreateBlogPostRequest + 95, // 237: productv1.ProductService.MutationCreateNullableFieldsType:input_type -> productv1.MutationCreateNullableFieldsTypeRequest + 91, // 238: productv1.ProductService.MutationCreateUser:input_type -> productv1.MutationCreateUserRequest + 93, // 239: productv1.ProductService.MutationPerformAction:input_type -> productv1.MutationPerformActionRequest + 105, // 240: productv1.ProductService.MutationUpdateAuthor:input_type -> productv1.MutationUpdateAuthorRequest + 101, // 241: productv1.ProductService.MutationUpdateBlogPost:input_type -> productv1.MutationUpdateBlogPostRequest + 97, // 242: productv1.ProductService.MutationUpdateNullableFieldsType:input_type -> productv1.MutationUpdateNullableFieldsTypeRequest + 85, // 243: productv1.ProductService.QueryAllAuthors:input_type -> productv1.QueryAllAuthorsRequest + 77, // 244: productv1.ProductService.QueryAllBlogPosts:input_type -> productv1.QueryAllBlogPostsRequest + 69, // 245: productv1.ProductService.QueryAllNullableFieldsTypes:input_type -> productv1.QueryAllNullableFieldsTypesRequest + 57, // 246: productv1.ProductService.QueryAllPets:input_type -> productv1.QueryAllPetsRequest + 79, // 247: productv1.ProductService.QueryAuthor:input_type -> productv1.QueryAuthorRequest + 81, // 248: productv1.ProductService.QueryAuthorById:input_type -> productv1.QueryAuthorByIdRequest + 83, // 249: productv1.ProductService.QueryAuthorsWithFilter:input_type -> productv1.QueryAuthorsWithFilterRequest + 71, // 250: productv1.ProductService.QueryBlogPost:input_type -> productv1.QueryBlogPostRequest + 73, // 251: productv1.ProductService.QueryBlogPostById:input_type -> productv1.QueryBlogPostByIdRequest + 75, // 252: productv1.ProductService.QueryBlogPostsWithFilter:input_type -> productv1.QueryBlogPostsWithFilterRequest + 87, // 253: productv1.ProductService.QueryBulkSearchAuthors:input_type -> productv1.QueryBulkSearchAuthorsRequest + 89, // 254: productv1.ProductService.QueryBulkSearchBlogPosts:input_type -> productv1.QueryBulkSearchBlogPostsRequest + 45, // 255: productv1.ProductService.QueryCalculateTotals:input_type -> productv1.QueryCalculateTotalsRequest + 47, // 256: productv1.ProductService.QueryCategories:input_type -> productv1.QueryCategoriesRequest + 49, // 257: productv1.ProductService.QueryCategoriesByKind:input_type -> productv1.QueryCategoriesByKindRequest + 51, // 258: productv1.ProductService.QueryCategoriesByKinds:input_type -> productv1.QueryCategoriesByKindsRequest + 43, // 259: productv1.ProductService.QueryComplexFilterType:input_type -> productv1.QueryComplexFilterTypeRequest + 53, // 260: productv1.ProductService.QueryFilterCategories:input_type -> productv1.QueryFilterCategoriesRequest + 35, // 261: productv1.ProductService.QueryNestedType:input_type -> productv1.QueryNestedTypeRequest + 63, // 262: productv1.ProductService.QueryNullableFieldsType:input_type -> productv1.QueryNullableFieldsTypeRequest + 65, // 263: productv1.ProductService.QueryNullableFieldsTypeById:input_type -> productv1.QueryNullableFieldsTypeByIdRequest + 67, // 264: productv1.ProductService.QueryNullableFieldsTypeWithFilter:input_type -> productv1.QueryNullableFieldsTypeWithFilterRequest + 55, // 265: productv1.ProductService.QueryRandomPet:input_type -> productv1.QueryRandomPetRequest + 61, // 266: productv1.ProductService.QueryRandomSearchResult:input_type -> productv1.QueryRandomSearchResultRequest + 37, // 267: productv1.ProductService.QueryRecursiveType:input_type -> productv1.QueryRecursiveTypeRequest + 59, // 268: productv1.ProductService.QuerySearch:input_type -> productv1.QuerySearchRequest + 39, // 269: productv1.ProductService.QueryTypeFilterWithArguments:input_type -> productv1.QueryTypeFilterWithArgumentsRequest + 41, // 270: productv1.ProductService.QueryTypeWithMultipleFilterFields:input_type -> productv1.QueryTypeWithMultipleFilterFieldsRequest + 33, // 271: productv1.ProductService.QueryUser:input_type -> productv1.QueryUserRequest + 31, // 272: productv1.ProductService.QueryUsers:input_type -> productv1.QueryUsersRequest + 137, // 273: productv1.ProductService.ResolveCategoryCategoryMetrics:input_type -> productv1.ResolveCategoryCategoryMetricsRequest + 132, // 274: productv1.ProductService.ResolveCategoryPopularityScore:input_type -> productv1.ResolveCategoryPopularityScoreRequest + 127, // 275: productv1.ProductService.ResolveCategoryProductCount:input_type -> productv1.ResolveCategoryProductCountRequest + 122, // 276: productv1.ProductService.ResolveProductRecommendedCategory:input_type -> productv1.ResolveProductRecommendedCategoryRequest + 117, // 277: productv1.ProductService.ResolveProductShippingEstimate:input_type -> productv1.ResolveProductShippingEstimateRequest + 142, // 278: productv1.ProductService.ResolveSubcategoryItemCount:input_type -> productv1.ResolveSubcategoryItemCountRequest + 24, // 279: productv1.ProductService.LookupProductById:output_type -> productv1.LookupProductByIdResponse + 27, // 280: productv1.ProductService.LookupStorageById:output_type -> productv1.LookupStorageByIdResponse + 30, // 281: productv1.ProductService.LookupWarehouseById:output_type -> productv1.LookupWarehouseByIdResponse + 108, // 282: productv1.ProductService.MutationBulkCreateAuthors:output_type -> productv1.MutationBulkCreateAuthorsResponse + 112, // 283: productv1.ProductService.MutationBulkCreateBlogPosts:output_type -> productv1.MutationBulkCreateBlogPostsResponse + 110, // 284: productv1.ProductService.MutationBulkUpdateAuthors:output_type -> productv1.MutationBulkUpdateAuthorsResponse + 114, // 285: productv1.ProductService.MutationBulkUpdateBlogPosts:output_type -> productv1.MutationBulkUpdateBlogPostsResponse + 104, // 286: productv1.ProductService.MutationCreateAuthor:output_type -> productv1.MutationCreateAuthorResponse + 100, // 287: productv1.ProductService.MutationCreateBlogPost:output_type -> productv1.MutationCreateBlogPostResponse + 96, // 288: productv1.ProductService.MutationCreateNullableFieldsType:output_type -> productv1.MutationCreateNullableFieldsTypeResponse + 92, // 289: productv1.ProductService.MutationCreateUser:output_type -> productv1.MutationCreateUserResponse + 94, // 290: productv1.ProductService.MutationPerformAction:output_type -> productv1.MutationPerformActionResponse + 106, // 291: productv1.ProductService.MutationUpdateAuthor:output_type -> productv1.MutationUpdateAuthorResponse + 102, // 292: productv1.ProductService.MutationUpdateBlogPost:output_type -> productv1.MutationUpdateBlogPostResponse + 98, // 293: productv1.ProductService.MutationUpdateNullableFieldsType:output_type -> productv1.MutationUpdateNullableFieldsTypeResponse + 86, // 294: productv1.ProductService.QueryAllAuthors:output_type -> productv1.QueryAllAuthorsResponse + 78, // 295: productv1.ProductService.QueryAllBlogPosts:output_type -> productv1.QueryAllBlogPostsResponse + 70, // 296: productv1.ProductService.QueryAllNullableFieldsTypes:output_type -> productv1.QueryAllNullableFieldsTypesResponse + 58, // 297: productv1.ProductService.QueryAllPets:output_type -> productv1.QueryAllPetsResponse + 80, // 298: productv1.ProductService.QueryAuthor:output_type -> productv1.QueryAuthorResponse + 82, // 299: productv1.ProductService.QueryAuthorById:output_type -> productv1.QueryAuthorByIdResponse + 84, // 300: productv1.ProductService.QueryAuthorsWithFilter:output_type -> productv1.QueryAuthorsWithFilterResponse + 72, // 301: productv1.ProductService.QueryBlogPost:output_type -> productv1.QueryBlogPostResponse + 74, // 302: productv1.ProductService.QueryBlogPostById:output_type -> productv1.QueryBlogPostByIdResponse + 76, // 303: productv1.ProductService.QueryBlogPostsWithFilter:output_type -> productv1.QueryBlogPostsWithFilterResponse + 88, // 304: productv1.ProductService.QueryBulkSearchAuthors:output_type -> productv1.QueryBulkSearchAuthorsResponse + 90, // 305: productv1.ProductService.QueryBulkSearchBlogPosts:output_type -> productv1.QueryBulkSearchBlogPostsResponse + 46, // 306: productv1.ProductService.QueryCalculateTotals:output_type -> productv1.QueryCalculateTotalsResponse + 48, // 307: productv1.ProductService.QueryCategories:output_type -> productv1.QueryCategoriesResponse + 50, // 308: productv1.ProductService.QueryCategoriesByKind:output_type -> productv1.QueryCategoriesByKindResponse + 52, // 309: productv1.ProductService.QueryCategoriesByKinds:output_type -> productv1.QueryCategoriesByKindsResponse + 44, // 310: productv1.ProductService.QueryComplexFilterType:output_type -> productv1.QueryComplexFilterTypeResponse + 54, // 311: productv1.ProductService.QueryFilterCategories:output_type -> productv1.QueryFilterCategoriesResponse + 36, // 312: productv1.ProductService.QueryNestedType:output_type -> productv1.QueryNestedTypeResponse + 64, // 313: productv1.ProductService.QueryNullableFieldsType:output_type -> productv1.QueryNullableFieldsTypeResponse + 66, // 314: productv1.ProductService.QueryNullableFieldsTypeById:output_type -> productv1.QueryNullableFieldsTypeByIdResponse + 68, // 315: productv1.ProductService.QueryNullableFieldsTypeWithFilter:output_type -> productv1.QueryNullableFieldsTypeWithFilterResponse + 56, // 316: productv1.ProductService.QueryRandomPet:output_type -> productv1.QueryRandomPetResponse + 62, // 317: productv1.ProductService.QueryRandomSearchResult:output_type -> productv1.QueryRandomSearchResultResponse + 38, // 318: productv1.ProductService.QueryRecursiveType:output_type -> productv1.QueryRecursiveTypeResponse + 60, // 319: productv1.ProductService.QuerySearch:output_type -> productv1.QuerySearchResponse + 40, // 320: productv1.ProductService.QueryTypeFilterWithArguments:output_type -> productv1.QueryTypeFilterWithArgumentsResponse + 42, // 321: productv1.ProductService.QueryTypeWithMultipleFilterFields:output_type -> productv1.QueryTypeWithMultipleFilterFieldsResponse + 34, // 322: productv1.ProductService.QueryUser:output_type -> productv1.QueryUserResponse + 32, // 323: productv1.ProductService.QueryUsers:output_type -> productv1.QueryUsersResponse + 139, // 324: productv1.ProductService.ResolveCategoryCategoryMetrics:output_type -> productv1.ResolveCategoryCategoryMetricsResponse + 134, // 325: productv1.ProductService.ResolveCategoryPopularityScore:output_type -> productv1.ResolveCategoryPopularityScoreResponse + 129, // 326: productv1.ProductService.ResolveCategoryProductCount:output_type -> productv1.ResolveCategoryProductCountResponse + 124, // 327: productv1.ProductService.ResolveProductRecommendedCategory:output_type -> productv1.ResolveProductRecommendedCategoryResponse + 119, // 328: productv1.ProductService.ResolveProductShippingEstimate:output_type -> productv1.ResolveProductShippingEstimateResponse + 144, // 329: productv1.ProductService.ResolveSubcategoryItemCount:output_type -> productv1.ResolveSubcategoryItemCountResponse + 279, // [279:330] is the sub-list for method output_type + 228, // [228:279] is the sub-list for method input_type + 228, // [228:228] is the sub-list for extension type_name + 228, // [228:228] is the sub-list for extension extendee + 0, // [0:228] is the sub-list for field type_name +} + +func init() { file_testdata_service_proto_init() } +func file_testdata_service_proto_init() { + if File_testdata_service_proto != nil { + return + } + file_testdata_service_proto_msgTypes[157].OneofWrappers = []any{ + (*Animal_Cat)(nil), + (*Animal_Dog)(nil), + } + file_testdata_service_proto_msgTypes[159].OneofWrappers = []any{ + (*SearchResult_Product)(nil), + (*SearchResult_User)(nil), + (*SearchResult_Category)(nil), + } + file_testdata_service_proto_msgTypes[168].OneofWrappers = []any{ + (*ActionResult_ActionSuccess)(nil), + (*ActionResult_ActionError)(nil), + } + type x struct{} + out := protoimpl.TypeBuilder{ + File: protoimpl.DescBuilder{ + GoPackagePath: reflect.TypeOf(x{}).PkgPath(), + RawDescriptor: unsafe.Slice(unsafe.StringData(file_testdata_service_proto_rawDesc), len(file_testdata_service_proto_rawDesc)), + NumEnums: 2, + NumMessages: 208, + NumExtensions: 0, + NumServices: 1, + }, + GoTypes: file_testdata_service_proto_goTypes, + DependencyIndexes: file_testdata_service_proto_depIdxs, + EnumInfos: file_testdata_service_proto_enumTypes, + MessageInfos: file_testdata_service_proto_msgTypes, + }.Build() + File_testdata_service_proto = out.File + file_testdata_service_proto_goTypes = nil + file_testdata_service_proto_depIdxs = nil +} diff --git a/v2/pkg/grpctest/productv1/testdata/service_grpc.pb.go b/v2/pkg/grpctest/productv1/testdata/service_grpc.pb.go new file mode 100644 index 000000000..d7980a7b0 --- /dev/null +++ b/v2/pkg/grpctest/productv1/testdata/service_grpc.pb.go @@ -0,0 +1,2031 @@ +// Code generated by protoc-gen-go-grpc. DO NOT EDIT. +// versions: +// - protoc-gen-go-grpc v1.5.1 +// - protoc v6.32.0 +// source: testdata/service.proto + +package productv1 + +import ( + context "context" + grpc "google.golang.org/grpc" + codes "google.golang.org/grpc/codes" + status "google.golang.org/grpc/status" +) + +// This is a compile-time assertion to ensure that this generated file +// is compatible with the grpc package it is being compiled against. +// Requires gRPC-Go v1.64.0 or later. +const _ = grpc.SupportPackageIsVersion9 + +const ( + ProductService_LookupProductById_FullMethodName = "/productv1.ProductService/LookupProductById" + ProductService_LookupStorageById_FullMethodName = "/productv1.ProductService/LookupStorageById" + ProductService_LookupWarehouseById_FullMethodName = "/productv1.ProductService/LookupWarehouseById" + ProductService_MutationBulkCreateAuthors_FullMethodName = "/productv1.ProductService/MutationBulkCreateAuthors" + ProductService_MutationBulkCreateBlogPosts_FullMethodName = "/productv1.ProductService/MutationBulkCreateBlogPosts" + ProductService_MutationBulkUpdateAuthors_FullMethodName = "/productv1.ProductService/MutationBulkUpdateAuthors" + ProductService_MutationBulkUpdateBlogPosts_FullMethodName = "/productv1.ProductService/MutationBulkUpdateBlogPosts" + ProductService_MutationCreateAuthor_FullMethodName = "/productv1.ProductService/MutationCreateAuthor" + ProductService_MutationCreateBlogPost_FullMethodName = "/productv1.ProductService/MutationCreateBlogPost" + ProductService_MutationCreateNullableFieldsType_FullMethodName = "/productv1.ProductService/MutationCreateNullableFieldsType" + ProductService_MutationCreateUser_FullMethodName = "/productv1.ProductService/MutationCreateUser" + ProductService_MutationPerformAction_FullMethodName = "/productv1.ProductService/MutationPerformAction" + ProductService_MutationUpdateAuthor_FullMethodName = "/productv1.ProductService/MutationUpdateAuthor" + ProductService_MutationUpdateBlogPost_FullMethodName = "/productv1.ProductService/MutationUpdateBlogPost" + ProductService_MutationUpdateNullableFieldsType_FullMethodName = "/productv1.ProductService/MutationUpdateNullableFieldsType" + ProductService_QueryAllAuthors_FullMethodName = "/productv1.ProductService/QueryAllAuthors" + ProductService_QueryAllBlogPosts_FullMethodName = "/productv1.ProductService/QueryAllBlogPosts" + ProductService_QueryAllNullableFieldsTypes_FullMethodName = "/productv1.ProductService/QueryAllNullableFieldsTypes" + ProductService_QueryAllPets_FullMethodName = "/productv1.ProductService/QueryAllPets" + ProductService_QueryAuthor_FullMethodName = "/productv1.ProductService/QueryAuthor" + ProductService_QueryAuthorById_FullMethodName = "/productv1.ProductService/QueryAuthorById" + ProductService_QueryAuthorsWithFilter_FullMethodName = "/productv1.ProductService/QueryAuthorsWithFilter" + ProductService_QueryBlogPost_FullMethodName = "/productv1.ProductService/QueryBlogPost" + ProductService_QueryBlogPostById_FullMethodName = "/productv1.ProductService/QueryBlogPostById" + ProductService_QueryBlogPostsWithFilter_FullMethodName = "/productv1.ProductService/QueryBlogPostsWithFilter" + ProductService_QueryBulkSearchAuthors_FullMethodName = "/productv1.ProductService/QueryBulkSearchAuthors" + ProductService_QueryBulkSearchBlogPosts_FullMethodName = "/productv1.ProductService/QueryBulkSearchBlogPosts" + ProductService_QueryCalculateTotals_FullMethodName = "/productv1.ProductService/QueryCalculateTotals" + ProductService_QueryCategories_FullMethodName = "/productv1.ProductService/QueryCategories" + ProductService_QueryCategoriesByKind_FullMethodName = "/productv1.ProductService/QueryCategoriesByKind" + ProductService_QueryCategoriesByKinds_FullMethodName = "/productv1.ProductService/QueryCategoriesByKinds" + ProductService_QueryComplexFilterType_FullMethodName = "/productv1.ProductService/QueryComplexFilterType" + ProductService_QueryFilterCategories_FullMethodName = "/productv1.ProductService/QueryFilterCategories" + ProductService_QueryNestedType_FullMethodName = "/productv1.ProductService/QueryNestedType" + ProductService_QueryNullableFieldsType_FullMethodName = "/productv1.ProductService/QueryNullableFieldsType" + ProductService_QueryNullableFieldsTypeById_FullMethodName = "/productv1.ProductService/QueryNullableFieldsTypeById" + ProductService_QueryNullableFieldsTypeWithFilter_FullMethodName = "/productv1.ProductService/QueryNullableFieldsTypeWithFilter" + ProductService_QueryRandomPet_FullMethodName = "/productv1.ProductService/QueryRandomPet" + ProductService_QueryRandomSearchResult_FullMethodName = "/productv1.ProductService/QueryRandomSearchResult" + ProductService_QueryRecursiveType_FullMethodName = "/productv1.ProductService/QueryRecursiveType" + ProductService_QuerySearch_FullMethodName = "/productv1.ProductService/QuerySearch" + ProductService_QueryTypeFilterWithArguments_FullMethodName = "/productv1.ProductService/QueryTypeFilterWithArguments" + ProductService_QueryTypeWithMultipleFilterFields_FullMethodName = "/productv1.ProductService/QueryTypeWithMultipleFilterFields" + ProductService_QueryUser_FullMethodName = "/productv1.ProductService/QueryUser" + ProductService_QueryUsers_FullMethodName = "/productv1.ProductService/QueryUsers" + ProductService_ResolveCategoryCategoryMetrics_FullMethodName = "/productv1.ProductService/ResolveCategoryCategoryMetrics" + ProductService_ResolveCategoryPopularityScore_FullMethodName = "/productv1.ProductService/ResolveCategoryPopularityScore" + ProductService_ResolveCategoryProductCount_FullMethodName = "/productv1.ProductService/ResolveCategoryProductCount" + ProductService_ResolveProductRecommendedCategory_FullMethodName = "/productv1.ProductService/ResolveProductRecommendedCategory" + ProductService_ResolveProductShippingEstimate_FullMethodName = "/productv1.ProductService/ResolveProductShippingEstimate" + ProductService_ResolveSubcategoryItemCount_FullMethodName = "/productv1.ProductService/ResolveSubcategoryItemCount" +) + +// ProductServiceClient is the client API for ProductService service. +// +// For semantics around ctx use and closing/ending streaming RPCs, please refer to https://pkg.go.dev/google.golang.org/grpc/?tab=doc#ClientConn.NewStream. +// +// Service definition for ProductService +type ProductServiceClient interface { + // Lookup Product entity by id + LookupProductById(ctx context.Context, in *LookupProductByIdRequest, opts ...grpc.CallOption) (*LookupProductByIdResponse, error) + // Lookup Storage entity by id + LookupStorageById(ctx context.Context, in *LookupStorageByIdRequest, opts ...grpc.CallOption) (*LookupStorageByIdResponse, error) + // Lookup Warehouse entity by id + LookupWarehouseById(ctx context.Context, in *LookupWarehouseByIdRequest, opts ...grpc.CallOption) (*LookupWarehouseByIdResponse, error) + MutationBulkCreateAuthors(ctx context.Context, in *MutationBulkCreateAuthorsRequest, opts ...grpc.CallOption) (*MutationBulkCreateAuthorsResponse, error) + MutationBulkCreateBlogPosts(ctx context.Context, in *MutationBulkCreateBlogPostsRequest, opts ...grpc.CallOption) (*MutationBulkCreateBlogPostsResponse, error) + MutationBulkUpdateAuthors(ctx context.Context, in *MutationBulkUpdateAuthorsRequest, opts ...grpc.CallOption) (*MutationBulkUpdateAuthorsResponse, error) + MutationBulkUpdateBlogPosts(ctx context.Context, in *MutationBulkUpdateBlogPostsRequest, opts ...grpc.CallOption) (*MutationBulkUpdateBlogPostsResponse, error) + MutationCreateAuthor(ctx context.Context, in *MutationCreateAuthorRequest, opts ...grpc.CallOption) (*MutationCreateAuthorResponse, error) + MutationCreateBlogPost(ctx context.Context, in *MutationCreateBlogPostRequest, opts ...grpc.CallOption) (*MutationCreateBlogPostResponse, error) + MutationCreateNullableFieldsType(ctx context.Context, in *MutationCreateNullableFieldsTypeRequest, opts ...grpc.CallOption) (*MutationCreateNullableFieldsTypeResponse, error) + MutationCreateUser(ctx context.Context, in *MutationCreateUserRequest, opts ...grpc.CallOption) (*MutationCreateUserResponse, error) + MutationPerformAction(ctx context.Context, in *MutationPerformActionRequest, opts ...grpc.CallOption) (*MutationPerformActionResponse, error) + MutationUpdateAuthor(ctx context.Context, in *MutationUpdateAuthorRequest, opts ...grpc.CallOption) (*MutationUpdateAuthorResponse, error) + MutationUpdateBlogPost(ctx context.Context, in *MutationUpdateBlogPostRequest, opts ...grpc.CallOption) (*MutationUpdateBlogPostResponse, error) + MutationUpdateNullableFieldsType(ctx context.Context, in *MutationUpdateNullableFieldsTypeRequest, opts ...grpc.CallOption) (*MutationUpdateNullableFieldsTypeResponse, error) + QueryAllAuthors(ctx context.Context, in *QueryAllAuthorsRequest, opts ...grpc.CallOption) (*QueryAllAuthorsResponse, error) + QueryAllBlogPosts(ctx context.Context, in *QueryAllBlogPostsRequest, opts ...grpc.CallOption) (*QueryAllBlogPostsResponse, error) + QueryAllNullableFieldsTypes(ctx context.Context, in *QueryAllNullableFieldsTypesRequest, opts ...grpc.CallOption) (*QueryAllNullableFieldsTypesResponse, error) + QueryAllPets(ctx context.Context, in *QueryAllPetsRequest, opts ...grpc.CallOption) (*QueryAllPetsResponse, error) + QueryAuthor(ctx context.Context, in *QueryAuthorRequest, opts ...grpc.CallOption) (*QueryAuthorResponse, error) + QueryAuthorById(ctx context.Context, in *QueryAuthorByIdRequest, opts ...grpc.CallOption) (*QueryAuthorByIdResponse, error) + QueryAuthorsWithFilter(ctx context.Context, in *QueryAuthorsWithFilterRequest, opts ...grpc.CallOption) (*QueryAuthorsWithFilterResponse, error) + QueryBlogPost(ctx context.Context, in *QueryBlogPostRequest, opts ...grpc.CallOption) (*QueryBlogPostResponse, error) + QueryBlogPostById(ctx context.Context, in *QueryBlogPostByIdRequest, opts ...grpc.CallOption) (*QueryBlogPostByIdResponse, error) + QueryBlogPostsWithFilter(ctx context.Context, in *QueryBlogPostsWithFilterRequest, opts ...grpc.CallOption) (*QueryBlogPostsWithFilterResponse, error) + QueryBulkSearchAuthors(ctx context.Context, in *QueryBulkSearchAuthorsRequest, opts ...grpc.CallOption) (*QueryBulkSearchAuthorsResponse, error) + QueryBulkSearchBlogPosts(ctx context.Context, in *QueryBulkSearchBlogPostsRequest, opts ...grpc.CallOption) (*QueryBulkSearchBlogPostsResponse, error) + QueryCalculateTotals(ctx context.Context, in *QueryCalculateTotalsRequest, opts ...grpc.CallOption) (*QueryCalculateTotalsResponse, error) + QueryCategories(ctx context.Context, in *QueryCategoriesRequest, opts ...grpc.CallOption) (*QueryCategoriesResponse, error) + QueryCategoriesByKind(ctx context.Context, in *QueryCategoriesByKindRequest, opts ...grpc.CallOption) (*QueryCategoriesByKindResponse, error) + QueryCategoriesByKinds(ctx context.Context, in *QueryCategoriesByKindsRequest, opts ...grpc.CallOption) (*QueryCategoriesByKindsResponse, error) + QueryComplexFilterType(ctx context.Context, in *QueryComplexFilterTypeRequest, opts ...grpc.CallOption) (*QueryComplexFilterTypeResponse, error) + QueryFilterCategories(ctx context.Context, in *QueryFilterCategoriesRequest, opts ...grpc.CallOption) (*QueryFilterCategoriesResponse, error) + QueryNestedType(ctx context.Context, in *QueryNestedTypeRequest, opts ...grpc.CallOption) (*QueryNestedTypeResponse, error) + QueryNullableFieldsType(ctx context.Context, in *QueryNullableFieldsTypeRequest, opts ...grpc.CallOption) (*QueryNullableFieldsTypeResponse, error) + QueryNullableFieldsTypeById(ctx context.Context, in *QueryNullableFieldsTypeByIdRequest, opts ...grpc.CallOption) (*QueryNullableFieldsTypeByIdResponse, error) + QueryNullableFieldsTypeWithFilter(ctx context.Context, in *QueryNullableFieldsTypeWithFilterRequest, opts ...grpc.CallOption) (*QueryNullableFieldsTypeWithFilterResponse, error) + QueryRandomPet(ctx context.Context, in *QueryRandomPetRequest, opts ...grpc.CallOption) (*QueryRandomPetResponse, error) + QueryRandomSearchResult(ctx context.Context, in *QueryRandomSearchResultRequest, opts ...grpc.CallOption) (*QueryRandomSearchResultResponse, error) + QueryRecursiveType(ctx context.Context, in *QueryRecursiveTypeRequest, opts ...grpc.CallOption) (*QueryRecursiveTypeResponse, error) + QuerySearch(ctx context.Context, in *QuerySearchRequest, opts ...grpc.CallOption) (*QuerySearchResponse, error) + QueryTypeFilterWithArguments(ctx context.Context, in *QueryTypeFilterWithArgumentsRequest, opts ...grpc.CallOption) (*QueryTypeFilterWithArgumentsResponse, error) + QueryTypeWithMultipleFilterFields(ctx context.Context, in *QueryTypeWithMultipleFilterFieldsRequest, opts ...grpc.CallOption) (*QueryTypeWithMultipleFilterFieldsResponse, error) + QueryUser(ctx context.Context, in *QueryUserRequest, opts ...grpc.CallOption) (*QueryUserResponse, error) + QueryUsers(ctx context.Context, in *QueryUsersRequest, opts ...grpc.CallOption) (*QueryUsersResponse, error) + ResolveCategoryCategoryMetrics(ctx context.Context, in *ResolveCategoryCategoryMetricsRequest, opts ...grpc.CallOption) (*ResolveCategoryCategoryMetricsResponse, error) + ResolveCategoryPopularityScore(ctx context.Context, in *ResolveCategoryPopularityScoreRequest, opts ...grpc.CallOption) (*ResolveCategoryPopularityScoreResponse, error) + ResolveCategoryProductCount(ctx context.Context, in *ResolveCategoryProductCountRequest, opts ...grpc.CallOption) (*ResolveCategoryProductCountResponse, error) + ResolveProductRecommendedCategory(ctx context.Context, in *ResolveProductRecommendedCategoryRequest, opts ...grpc.CallOption) (*ResolveProductRecommendedCategoryResponse, error) + ResolveProductShippingEstimate(ctx context.Context, in *ResolveProductShippingEstimateRequest, opts ...grpc.CallOption) (*ResolveProductShippingEstimateResponse, error) + ResolveSubcategoryItemCount(ctx context.Context, in *ResolveSubcategoryItemCountRequest, opts ...grpc.CallOption) (*ResolveSubcategoryItemCountResponse, error) +} + +type productServiceClient struct { + cc grpc.ClientConnInterface +} + +func NewProductServiceClient(cc grpc.ClientConnInterface) ProductServiceClient { + return &productServiceClient{cc} +} + +func (c *productServiceClient) LookupProductById(ctx context.Context, in *LookupProductByIdRequest, opts ...grpc.CallOption) (*LookupProductByIdResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(LookupProductByIdResponse) + err := c.cc.Invoke(ctx, ProductService_LookupProductById_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *productServiceClient) LookupStorageById(ctx context.Context, in *LookupStorageByIdRequest, opts ...grpc.CallOption) (*LookupStorageByIdResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(LookupStorageByIdResponse) + err := c.cc.Invoke(ctx, ProductService_LookupStorageById_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *productServiceClient) LookupWarehouseById(ctx context.Context, in *LookupWarehouseByIdRequest, opts ...grpc.CallOption) (*LookupWarehouseByIdResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(LookupWarehouseByIdResponse) + err := c.cc.Invoke(ctx, ProductService_LookupWarehouseById_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *productServiceClient) MutationBulkCreateAuthors(ctx context.Context, in *MutationBulkCreateAuthorsRequest, opts ...grpc.CallOption) (*MutationBulkCreateAuthorsResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(MutationBulkCreateAuthorsResponse) + err := c.cc.Invoke(ctx, ProductService_MutationBulkCreateAuthors_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *productServiceClient) MutationBulkCreateBlogPosts(ctx context.Context, in *MutationBulkCreateBlogPostsRequest, opts ...grpc.CallOption) (*MutationBulkCreateBlogPostsResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(MutationBulkCreateBlogPostsResponse) + err := c.cc.Invoke(ctx, ProductService_MutationBulkCreateBlogPosts_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *productServiceClient) MutationBulkUpdateAuthors(ctx context.Context, in *MutationBulkUpdateAuthorsRequest, opts ...grpc.CallOption) (*MutationBulkUpdateAuthorsResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(MutationBulkUpdateAuthorsResponse) + err := c.cc.Invoke(ctx, ProductService_MutationBulkUpdateAuthors_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *productServiceClient) MutationBulkUpdateBlogPosts(ctx context.Context, in *MutationBulkUpdateBlogPostsRequest, opts ...grpc.CallOption) (*MutationBulkUpdateBlogPostsResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(MutationBulkUpdateBlogPostsResponse) + err := c.cc.Invoke(ctx, ProductService_MutationBulkUpdateBlogPosts_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *productServiceClient) MutationCreateAuthor(ctx context.Context, in *MutationCreateAuthorRequest, opts ...grpc.CallOption) (*MutationCreateAuthorResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(MutationCreateAuthorResponse) + err := c.cc.Invoke(ctx, ProductService_MutationCreateAuthor_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *productServiceClient) MutationCreateBlogPost(ctx context.Context, in *MutationCreateBlogPostRequest, opts ...grpc.CallOption) (*MutationCreateBlogPostResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(MutationCreateBlogPostResponse) + err := c.cc.Invoke(ctx, ProductService_MutationCreateBlogPost_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *productServiceClient) MutationCreateNullableFieldsType(ctx context.Context, in *MutationCreateNullableFieldsTypeRequest, opts ...grpc.CallOption) (*MutationCreateNullableFieldsTypeResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(MutationCreateNullableFieldsTypeResponse) + err := c.cc.Invoke(ctx, ProductService_MutationCreateNullableFieldsType_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *productServiceClient) MutationCreateUser(ctx context.Context, in *MutationCreateUserRequest, opts ...grpc.CallOption) (*MutationCreateUserResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(MutationCreateUserResponse) + err := c.cc.Invoke(ctx, ProductService_MutationCreateUser_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *productServiceClient) MutationPerformAction(ctx context.Context, in *MutationPerformActionRequest, opts ...grpc.CallOption) (*MutationPerformActionResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(MutationPerformActionResponse) + err := c.cc.Invoke(ctx, ProductService_MutationPerformAction_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *productServiceClient) MutationUpdateAuthor(ctx context.Context, in *MutationUpdateAuthorRequest, opts ...grpc.CallOption) (*MutationUpdateAuthorResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(MutationUpdateAuthorResponse) + err := c.cc.Invoke(ctx, ProductService_MutationUpdateAuthor_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *productServiceClient) MutationUpdateBlogPost(ctx context.Context, in *MutationUpdateBlogPostRequest, opts ...grpc.CallOption) (*MutationUpdateBlogPostResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(MutationUpdateBlogPostResponse) + err := c.cc.Invoke(ctx, ProductService_MutationUpdateBlogPost_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *productServiceClient) MutationUpdateNullableFieldsType(ctx context.Context, in *MutationUpdateNullableFieldsTypeRequest, opts ...grpc.CallOption) (*MutationUpdateNullableFieldsTypeResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(MutationUpdateNullableFieldsTypeResponse) + err := c.cc.Invoke(ctx, ProductService_MutationUpdateNullableFieldsType_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *productServiceClient) QueryAllAuthors(ctx context.Context, in *QueryAllAuthorsRequest, opts ...grpc.CallOption) (*QueryAllAuthorsResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(QueryAllAuthorsResponse) + err := c.cc.Invoke(ctx, ProductService_QueryAllAuthors_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *productServiceClient) QueryAllBlogPosts(ctx context.Context, in *QueryAllBlogPostsRequest, opts ...grpc.CallOption) (*QueryAllBlogPostsResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(QueryAllBlogPostsResponse) + err := c.cc.Invoke(ctx, ProductService_QueryAllBlogPosts_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *productServiceClient) QueryAllNullableFieldsTypes(ctx context.Context, in *QueryAllNullableFieldsTypesRequest, opts ...grpc.CallOption) (*QueryAllNullableFieldsTypesResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(QueryAllNullableFieldsTypesResponse) + err := c.cc.Invoke(ctx, ProductService_QueryAllNullableFieldsTypes_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *productServiceClient) QueryAllPets(ctx context.Context, in *QueryAllPetsRequest, opts ...grpc.CallOption) (*QueryAllPetsResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(QueryAllPetsResponse) + err := c.cc.Invoke(ctx, ProductService_QueryAllPets_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *productServiceClient) QueryAuthor(ctx context.Context, in *QueryAuthorRequest, opts ...grpc.CallOption) (*QueryAuthorResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(QueryAuthorResponse) + err := c.cc.Invoke(ctx, ProductService_QueryAuthor_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *productServiceClient) QueryAuthorById(ctx context.Context, in *QueryAuthorByIdRequest, opts ...grpc.CallOption) (*QueryAuthorByIdResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(QueryAuthorByIdResponse) + err := c.cc.Invoke(ctx, ProductService_QueryAuthorById_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *productServiceClient) QueryAuthorsWithFilter(ctx context.Context, in *QueryAuthorsWithFilterRequest, opts ...grpc.CallOption) (*QueryAuthorsWithFilterResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(QueryAuthorsWithFilterResponse) + err := c.cc.Invoke(ctx, ProductService_QueryAuthorsWithFilter_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *productServiceClient) QueryBlogPost(ctx context.Context, in *QueryBlogPostRequest, opts ...grpc.CallOption) (*QueryBlogPostResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(QueryBlogPostResponse) + err := c.cc.Invoke(ctx, ProductService_QueryBlogPost_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *productServiceClient) QueryBlogPostById(ctx context.Context, in *QueryBlogPostByIdRequest, opts ...grpc.CallOption) (*QueryBlogPostByIdResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(QueryBlogPostByIdResponse) + err := c.cc.Invoke(ctx, ProductService_QueryBlogPostById_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *productServiceClient) QueryBlogPostsWithFilter(ctx context.Context, in *QueryBlogPostsWithFilterRequest, opts ...grpc.CallOption) (*QueryBlogPostsWithFilterResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(QueryBlogPostsWithFilterResponse) + err := c.cc.Invoke(ctx, ProductService_QueryBlogPostsWithFilter_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *productServiceClient) QueryBulkSearchAuthors(ctx context.Context, in *QueryBulkSearchAuthorsRequest, opts ...grpc.CallOption) (*QueryBulkSearchAuthorsResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(QueryBulkSearchAuthorsResponse) + err := c.cc.Invoke(ctx, ProductService_QueryBulkSearchAuthors_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *productServiceClient) QueryBulkSearchBlogPosts(ctx context.Context, in *QueryBulkSearchBlogPostsRequest, opts ...grpc.CallOption) (*QueryBulkSearchBlogPostsResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(QueryBulkSearchBlogPostsResponse) + err := c.cc.Invoke(ctx, ProductService_QueryBulkSearchBlogPosts_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *productServiceClient) QueryCalculateTotals(ctx context.Context, in *QueryCalculateTotalsRequest, opts ...grpc.CallOption) (*QueryCalculateTotalsResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(QueryCalculateTotalsResponse) + err := c.cc.Invoke(ctx, ProductService_QueryCalculateTotals_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *productServiceClient) QueryCategories(ctx context.Context, in *QueryCategoriesRequest, opts ...grpc.CallOption) (*QueryCategoriesResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(QueryCategoriesResponse) + err := c.cc.Invoke(ctx, ProductService_QueryCategories_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *productServiceClient) QueryCategoriesByKind(ctx context.Context, in *QueryCategoriesByKindRequest, opts ...grpc.CallOption) (*QueryCategoriesByKindResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(QueryCategoriesByKindResponse) + err := c.cc.Invoke(ctx, ProductService_QueryCategoriesByKind_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *productServiceClient) QueryCategoriesByKinds(ctx context.Context, in *QueryCategoriesByKindsRequest, opts ...grpc.CallOption) (*QueryCategoriesByKindsResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(QueryCategoriesByKindsResponse) + err := c.cc.Invoke(ctx, ProductService_QueryCategoriesByKinds_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *productServiceClient) QueryComplexFilterType(ctx context.Context, in *QueryComplexFilterTypeRequest, opts ...grpc.CallOption) (*QueryComplexFilterTypeResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(QueryComplexFilterTypeResponse) + err := c.cc.Invoke(ctx, ProductService_QueryComplexFilterType_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *productServiceClient) QueryFilterCategories(ctx context.Context, in *QueryFilterCategoriesRequest, opts ...grpc.CallOption) (*QueryFilterCategoriesResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(QueryFilterCategoriesResponse) + err := c.cc.Invoke(ctx, ProductService_QueryFilterCategories_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *productServiceClient) QueryNestedType(ctx context.Context, in *QueryNestedTypeRequest, opts ...grpc.CallOption) (*QueryNestedTypeResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(QueryNestedTypeResponse) + err := c.cc.Invoke(ctx, ProductService_QueryNestedType_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *productServiceClient) QueryNullableFieldsType(ctx context.Context, in *QueryNullableFieldsTypeRequest, opts ...grpc.CallOption) (*QueryNullableFieldsTypeResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(QueryNullableFieldsTypeResponse) + err := c.cc.Invoke(ctx, ProductService_QueryNullableFieldsType_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *productServiceClient) QueryNullableFieldsTypeById(ctx context.Context, in *QueryNullableFieldsTypeByIdRequest, opts ...grpc.CallOption) (*QueryNullableFieldsTypeByIdResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(QueryNullableFieldsTypeByIdResponse) + err := c.cc.Invoke(ctx, ProductService_QueryNullableFieldsTypeById_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *productServiceClient) QueryNullableFieldsTypeWithFilter(ctx context.Context, in *QueryNullableFieldsTypeWithFilterRequest, opts ...grpc.CallOption) (*QueryNullableFieldsTypeWithFilterResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(QueryNullableFieldsTypeWithFilterResponse) + err := c.cc.Invoke(ctx, ProductService_QueryNullableFieldsTypeWithFilter_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *productServiceClient) QueryRandomPet(ctx context.Context, in *QueryRandomPetRequest, opts ...grpc.CallOption) (*QueryRandomPetResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(QueryRandomPetResponse) + err := c.cc.Invoke(ctx, ProductService_QueryRandomPet_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *productServiceClient) QueryRandomSearchResult(ctx context.Context, in *QueryRandomSearchResultRequest, opts ...grpc.CallOption) (*QueryRandomSearchResultResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(QueryRandomSearchResultResponse) + err := c.cc.Invoke(ctx, ProductService_QueryRandomSearchResult_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *productServiceClient) QueryRecursiveType(ctx context.Context, in *QueryRecursiveTypeRequest, opts ...grpc.CallOption) (*QueryRecursiveTypeResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(QueryRecursiveTypeResponse) + err := c.cc.Invoke(ctx, ProductService_QueryRecursiveType_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *productServiceClient) QuerySearch(ctx context.Context, in *QuerySearchRequest, opts ...grpc.CallOption) (*QuerySearchResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(QuerySearchResponse) + err := c.cc.Invoke(ctx, ProductService_QuerySearch_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *productServiceClient) QueryTypeFilterWithArguments(ctx context.Context, in *QueryTypeFilterWithArgumentsRequest, opts ...grpc.CallOption) (*QueryTypeFilterWithArgumentsResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(QueryTypeFilterWithArgumentsResponse) + err := c.cc.Invoke(ctx, ProductService_QueryTypeFilterWithArguments_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *productServiceClient) QueryTypeWithMultipleFilterFields(ctx context.Context, in *QueryTypeWithMultipleFilterFieldsRequest, opts ...grpc.CallOption) (*QueryTypeWithMultipleFilterFieldsResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(QueryTypeWithMultipleFilterFieldsResponse) + err := c.cc.Invoke(ctx, ProductService_QueryTypeWithMultipleFilterFields_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *productServiceClient) QueryUser(ctx context.Context, in *QueryUserRequest, opts ...grpc.CallOption) (*QueryUserResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(QueryUserResponse) + err := c.cc.Invoke(ctx, ProductService_QueryUser_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *productServiceClient) QueryUsers(ctx context.Context, in *QueryUsersRequest, opts ...grpc.CallOption) (*QueryUsersResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(QueryUsersResponse) + err := c.cc.Invoke(ctx, ProductService_QueryUsers_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *productServiceClient) ResolveCategoryCategoryMetrics(ctx context.Context, in *ResolveCategoryCategoryMetricsRequest, opts ...grpc.CallOption) (*ResolveCategoryCategoryMetricsResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(ResolveCategoryCategoryMetricsResponse) + err := c.cc.Invoke(ctx, ProductService_ResolveCategoryCategoryMetrics_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *productServiceClient) ResolveCategoryPopularityScore(ctx context.Context, in *ResolveCategoryPopularityScoreRequest, opts ...grpc.CallOption) (*ResolveCategoryPopularityScoreResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(ResolveCategoryPopularityScoreResponse) + err := c.cc.Invoke(ctx, ProductService_ResolveCategoryPopularityScore_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *productServiceClient) ResolveCategoryProductCount(ctx context.Context, in *ResolveCategoryProductCountRequest, opts ...grpc.CallOption) (*ResolveCategoryProductCountResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(ResolveCategoryProductCountResponse) + err := c.cc.Invoke(ctx, ProductService_ResolveCategoryProductCount_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *productServiceClient) ResolveProductRecommendedCategory(ctx context.Context, in *ResolveProductRecommendedCategoryRequest, opts ...grpc.CallOption) (*ResolveProductRecommendedCategoryResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(ResolveProductRecommendedCategoryResponse) + err := c.cc.Invoke(ctx, ProductService_ResolveProductRecommendedCategory_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *productServiceClient) ResolveProductShippingEstimate(ctx context.Context, in *ResolveProductShippingEstimateRequest, opts ...grpc.CallOption) (*ResolveProductShippingEstimateResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(ResolveProductShippingEstimateResponse) + err := c.cc.Invoke(ctx, ProductService_ResolveProductShippingEstimate_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *productServiceClient) ResolveSubcategoryItemCount(ctx context.Context, in *ResolveSubcategoryItemCountRequest, opts ...grpc.CallOption) (*ResolveSubcategoryItemCountResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(ResolveSubcategoryItemCountResponse) + err := c.cc.Invoke(ctx, ProductService_ResolveSubcategoryItemCount_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +// ProductServiceServer is the server API for ProductService service. +// All implementations must embed UnimplementedProductServiceServer +// for forward compatibility. +// +// Service definition for ProductService +type ProductServiceServer interface { + // Lookup Product entity by id + LookupProductById(context.Context, *LookupProductByIdRequest) (*LookupProductByIdResponse, error) + // Lookup Storage entity by id + LookupStorageById(context.Context, *LookupStorageByIdRequest) (*LookupStorageByIdResponse, error) + // Lookup Warehouse entity by id + LookupWarehouseById(context.Context, *LookupWarehouseByIdRequest) (*LookupWarehouseByIdResponse, error) + MutationBulkCreateAuthors(context.Context, *MutationBulkCreateAuthorsRequest) (*MutationBulkCreateAuthorsResponse, error) + MutationBulkCreateBlogPosts(context.Context, *MutationBulkCreateBlogPostsRequest) (*MutationBulkCreateBlogPostsResponse, error) + MutationBulkUpdateAuthors(context.Context, *MutationBulkUpdateAuthorsRequest) (*MutationBulkUpdateAuthorsResponse, error) + MutationBulkUpdateBlogPosts(context.Context, *MutationBulkUpdateBlogPostsRequest) (*MutationBulkUpdateBlogPostsResponse, error) + MutationCreateAuthor(context.Context, *MutationCreateAuthorRequest) (*MutationCreateAuthorResponse, error) + MutationCreateBlogPost(context.Context, *MutationCreateBlogPostRequest) (*MutationCreateBlogPostResponse, error) + MutationCreateNullableFieldsType(context.Context, *MutationCreateNullableFieldsTypeRequest) (*MutationCreateNullableFieldsTypeResponse, error) + MutationCreateUser(context.Context, *MutationCreateUserRequest) (*MutationCreateUserResponse, error) + MutationPerformAction(context.Context, *MutationPerformActionRequest) (*MutationPerformActionResponse, error) + MutationUpdateAuthor(context.Context, *MutationUpdateAuthorRequest) (*MutationUpdateAuthorResponse, error) + MutationUpdateBlogPost(context.Context, *MutationUpdateBlogPostRequest) (*MutationUpdateBlogPostResponse, error) + MutationUpdateNullableFieldsType(context.Context, *MutationUpdateNullableFieldsTypeRequest) (*MutationUpdateNullableFieldsTypeResponse, error) + QueryAllAuthors(context.Context, *QueryAllAuthorsRequest) (*QueryAllAuthorsResponse, error) + QueryAllBlogPosts(context.Context, *QueryAllBlogPostsRequest) (*QueryAllBlogPostsResponse, error) + QueryAllNullableFieldsTypes(context.Context, *QueryAllNullableFieldsTypesRequest) (*QueryAllNullableFieldsTypesResponse, error) + QueryAllPets(context.Context, *QueryAllPetsRequest) (*QueryAllPetsResponse, error) + QueryAuthor(context.Context, *QueryAuthorRequest) (*QueryAuthorResponse, error) + QueryAuthorById(context.Context, *QueryAuthorByIdRequest) (*QueryAuthorByIdResponse, error) + QueryAuthorsWithFilter(context.Context, *QueryAuthorsWithFilterRequest) (*QueryAuthorsWithFilterResponse, error) + QueryBlogPost(context.Context, *QueryBlogPostRequest) (*QueryBlogPostResponse, error) + QueryBlogPostById(context.Context, *QueryBlogPostByIdRequest) (*QueryBlogPostByIdResponse, error) + QueryBlogPostsWithFilter(context.Context, *QueryBlogPostsWithFilterRequest) (*QueryBlogPostsWithFilterResponse, error) + QueryBulkSearchAuthors(context.Context, *QueryBulkSearchAuthorsRequest) (*QueryBulkSearchAuthorsResponse, error) + QueryBulkSearchBlogPosts(context.Context, *QueryBulkSearchBlogPostsRequest) (*QueryBulkSearchBlogPostsResponse, error) + QueryCalculateTotals(context.Context, *QueryCalculateTotalsRequest) (*QueryCalculateTotalsResponse, error) + QueryCategories(context.Context, *QueryCategoriesRequest) (*QueryCategoriesResponse, error) + QueryCategoriesByKind(context.Context, *QueryCategoriesByKindRequest) (*QueryCategoriesByKindResponse, error) + QueryCategoriesByKinds(context.Context, *QueryCategoriesByKindsRequest) (*QueryCategoriesByKindsResponse, error) + QueryComplexFilterType(context.Context, *QueryComplexFilterTypeRequest) (*QueryComplexFilterTypeResponse, error) + QueryFilterCategories(context.Context, *QueryFilterCategoriesRequest) (*QueryFilterCategoriesResponse, error) + QueryNestedType(context.Context, *QueryNestedTypeRequest) (*QueryNestedTypeResponse, error) + QueryNullableFieldsType(context.Context, *QueryNullableFieldsTypeRequest) (*QueryNullableFieldsTypeResponse, error) + QueryNullableFieldsTypeById(context.Context, *QueryNullableFieldsTypeByIdRequest) (*QueryNullableFieldsTypeByIdResponse, error) + QueryNullableFieldsTypeWithFilter(context.Context, *QueryNullableFieldsTypeWithFilterRequest) (*QueryNullableFieldsTypeWithFilterResponse, error) + QueryRandomPet(context.Context, *QueryRandomPetRequest) (*QueryRandomPetResponse, error) + QueryRandomSearchResult(context.Context, *QueryRandomSearchResultRequest) (*QueryRandomSearchResultResponse, error) + QueryRecursiveType(context.Context, *QueryRecursiveTypeRequest) (*QueryRecursiveTypeResponse, error) + QuerySearch(context.Context, *QuerySearchRequest) (*QuerySearchResponse, error) + QueryTypeFilterWithArguments(context.Context, *QueryTypeFilterWithArgumentsRequest) (*QueryTypeFilterWithArgumentsResponse, error) + QueryTypeWithMultipleFilterFields(context.Context, *QueryTypeWithMultipleFilterFieldsRequest) (*QueryTypeWithMultipleFilterFieldsResponse, error) + QueryUser(context.Context, *QueryUserRequest) (*QueryUserResponse, error) + QueryUsers(context.Context, *QueryUsersRequest) (*QueryUsersResponse, error) + ResolveCategoryCategoryMetrics(context.Context, *ResolveCategoryCategoryMetricsRequest) (*ResolveCategoryCategoryMetricsResponse, error) + ResolveCategoryPopularityScore(context.Context, *ResolveCategoryPopularityScoreRequest) (*ResolveCategoryPopularityScoreResponse, error) + ResolveCategoryProductCount(context.Context, *ResolveCategoryProductCountRequest) (*ResolveCategoryProductCountResponse, error) + ResolveProductRecommendedCategory(context.Context, *ResolveProductRecommendedCategoryRequest) (*ResolveProductRecommendedCategoryResponse, error) + ResolveProductShippingEstimate(context.Context, *ResolveProductShippingEstimateRequest) (*ResolveProductShippingEstimateResponse, error) + ResolveSubcategoryItemCount(context.Context, *ResolveSubcategoryItemCountRequest) (*ResolveSubcategoryItemCountResponse, error) + mustEmbedUnimplementedProductServiceServer() +} + +// UnimplementedProductServiceServer must be embedded to have +// forward compatible implementations. +// +// NOTE: this should be embedded by value instead of pointer to avoid a nil +// pointer dereference when methods are called. +type UnimplementedProductServiceServer struct{} + +func (UnimplementedProductServiceServer) LookupProductById(context.Context, *LookupProductByIdRequest) (*LookupProductByIdResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method LookupProductById not implemented") +} +func (UnimplementedProductServiceServer) LookupStorageById(context.Context, *LookupStorageByIdRequest) (*LookupStorageByIdResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method LookupStorageById not implemented") +} +func (UnimplementedProductServiceServer) LookupWarehouseById(context.Context, *LookupWarehouseByIdRequest) (*LookupWarehouseByIdResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method LookupWarehouseById not implemented") +} +func (UnimplementedProductServiceServer) MutationBulkCreateAuthors(context.Context, *MutationBulkCreateAuthorsRequest) (*MutationBulkCreateAuthorsResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method MutationBulkCreateAuthors not implemented") +} +func (UnimplementedProductServiceServer) MutationBulkCreateBlogPosts(context.Context, *MutationBulkCreateBlogPostsRequest) (*MutationBulkCreateBlogPostsResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method MutationBulkCreateBlogPosts not implemented") +} +func (UnimplementedProductServiceServer) MutationBulkUpdateAuthors(context.Context, *MutationBulkUpdateAuthorsRequest) (*MutationBulkUpdateAuthorsResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method MutationBulkUpdateAuthors not implemented") +} +func (UnimplementedProductServiceServer) MutationBulkUpdateBlogPosts(context.Context, *MutationBulkUpdateBlogPostsRequest) (*MutationBulkUpdateBlogPostsResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method MutationBulkUpdateBlogPosts not implemented") +} +func (UnimplementedProductServiceServer) MutationCreateAuthor(context.Context, *MutationCreateAuthorRequest) (*MutationCreateAuthorResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method MutationCreateAuthor not implemented") +} +func (UnimplementedProductServiceServer) MutationCreateBlogPost(context.Context, *MutationCreateBlogPostRequest) (*MutationCreateBlogPostResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method MutationCreateBlogPost not implemented") +} +func (UnimplementedProductServiceServer) MutationCreateNullableFieldsType(context.Context, *MutationCreateNullableFieldsTypeRequest) (*MutationCreateNullableFieldsTypeResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method MutationCreateNullableFieldsType not implemented") +} +func (UnimplementedProductServiceServer) MutationCreateUser(context.Context, *MutationCreateUserRequest) (*MutationCreateUserResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method MutationCreateUser not implemented") +} +func (UnimplementedProductServiceServer) MutationPerformAction(context.Context, *MutationPerformActionRequest) (*MutationPerformActionResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method MutationPerformAction not implemented") +} +func (UnimplementedProductServiceServer) MutationUpdateAuthor(context.Context, *MutationUpdateAuthorRequest) (*MutationUpdateAuthorResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method MutationUpdateAuthor not implemented") +} +func (UnimplementedProductServiceServer) MutationUpdateBlogPost(context.Context, *MutationUpdateBlogPostRequest) (*MutationUpdateBlogPostResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method MutationUpdateBlogPost not implemented") +} +func (UnimplementedProductServiceServer) MutationUpdateNullableFieldsType(context.Context, *MutationUpdateNullableFieldsTypeRequest) (*MutationUpdateNullableFieldsTypeResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method MutationUpdateNullableFieldsType not implemented") +} +func (UnimplementedProductServiceServer) QueryAllAuthors(context.Context, *QueryAllAuthorsRequest) (*QueryAllAuthorsResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method QueryAllAuthors not implemented") +} +func (UnimplementedProductServiceServer) QueryAllBlogPosts(context.Context, *QueryAllBlogPostsRequest) (*QueryAllBlogPostsResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method QueryAllBlogPosts not implemented") +} +func (UnimplementedProductServiceServer) QueryAllNullableFieldsTypes(context.Context, *QueryAllNullableFieldsTypesRequest) (*QueryAllNullableFieldsTypesResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method QueryAllNullableFieldsTypes not implemented") +} +func (UnimplementedProductServiceServer) QueryAllPets(context.Context, *QueryAllPetsRequest) (*QueryAllPetsResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method QueryAllPets not implemented") +} +func (UnimplementedProductServiceServer) QueryAuthor(context.Context, *QueryAuthorRequest) (*QueryAuthorResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method QueryAuthor not implemented") +} +func (UnimplementedProductServiceServer) QueryAuthorById(context.Context, *QueryAuthorByIdRequest) (*QueryAuthorByIdResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method QueryAuthorById not implemented") +} +func (UnimplementedProductServiceServer) QueryAuthorsWithFilter(context.Context, *QueryAuthorsWithFilterRequest) (*QueryAuthorsWithFilterResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method QueryAuthorsWithFilter not implemented") +} +func (UnimplementedProductServiceServer) QueryBlogPost(context.Context, *QueryBlogPostRequest) (*QueryBlogPostResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method QueryBlogPost not implemented") +} +func (UnimplementedProductServiceServer) QueryBlogPostById(context.Context, *QueryBlogPostByIdRequest) (*QueryBlogPostByIdResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method QueryBlogPostById not implemented") +} +func (UnimplementedProductServiceServer) QueryBlogPostsWithFilter(context.Context, *QueryBlogPostsWithFilterRequest) (*QueryBlogPostsWithFilterResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method QueryBlogPostsWithFilter not implemented") +} +func (UnimplementedProductServiceServer) QueryBulkSearchAuthors(context.Context, *QueryBulkSearchAuthorsRequest) (*QueryBulkSearchAuthorsResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method QueryBulkSearchAuthors not implemented") +} +func (UnimplementedProductServiceServer) QueryBulkSearchBlogPosts(context.Context, *QueryBulkSearchBlogPostsRequest) (*QueryBulkSearchBlogPostsResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method QueryBulkSearchBlogPosts not implemented") +} +func (UnimplementedProductServiceServer) QueryCalculateTotals(context.Context, *QueryCalculateTotalsRequest) (*QueryCalculateTotalsResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method QueryCalculateTotals not implemented") +} +func (UnimplementedProductServiceServer) QueryCategories(context.Context, *QueryCategoriesRequest) (*QueryCategoriesResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method QueryCategories not implemented") +} +func (UnimplementedProductServiceServer) QueryCategoriesByKind(context.Context, *QueryCategoriesByKindRequest) (*QueryCategoriesByKindResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method QueryCategoriesByKind not implemented") +} +func (UnimplementedProductServiceServer) QueryCategoriesByKinds(context.Context, *QueryCategoriesByKindsRequest) (*QueryCategoriesByKindsResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method QueryCategoriesByKinds not implemented") +} +func (UnimplementedProductServiceServer) QueryComplexFilterType(context.Context, *QueryComplexFilterTypeRequest) (*QueryComplexFilterTypeResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method QueryComplexFilterType not implemented") +} +func (UnimplementedProductServiceServer) QueryFilterCategories(context.Context, *QueryFilterCategoriesRequest) (*QueryFilterCategoriesResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method QueryFilterCategories not implemented") +} +func (UnimplementedProductServiceServer) QueryNestedType(context.Context, *QueryNestedTypeRequest) (*QueryNestedTypeResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method QueryNestedType not implemented") +} +func (UnimplementedProductServiceServer) QueryNullableFieldsType(context.Context, *QueryNullableFieldsTypeRequest) (*QueryNullableFieldsTypeResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method QueryNullableFieldsType not implemented") +} +func (UnimplementedProductServiceServer) QueryNullableFieldsTypeById(context.Context, *QueryNullableFieldsTypeByIdRequest) (*QueryNullableFieldsTypeByIdResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method QueryNullableFieldsTypeById not implemented") +} +func (UnimplementedProductServiceServer) QueryNullableFieldsTypeWithFilter(context.Context, *QueryNullableFieldsTypeWithFilterRequest) (*QueryNullableFieldsTypeWithFilterResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method QueryNullableFieldsTypeWithFilter not implemented") +} +func (UnimplementedProductServiceServer) QueryRandomPet(context.Context, *QueryRandomPetRequest) (*QueryRandomPetResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method QueryRandomPet not implemented") +} +func (UnimplementedProductServiceServer) QueryRandomSearchResult(context.Context, *QueryRandomSearchResultRequest) (*QueryRandomSearchResultResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method QueryRandomSearchResult not implemented") +} +func (UnimplementedProductServiceServer) QueryRecursiveType(context.Context, *QueryRecursiveTypeRequest) (*QueryRecursiveTypeResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method QueryRecursiveType not implemented") +} +func (UnimplementedProductServiceServer) QuerySearch(context.Context, *QuerySearchRequest) (*QuerySearchResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method QuerySearch not implemented") +} +func (UnimplementedProductServiceServer) QueryTypeFilterWithArguments(context.Context, *QueryTypeFilterWithArgumentsRequest) (*QueryTypeFilterWithArgumentsResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method QueryTypeFilterWithArguments not implemented") +} +func (UnimplementedProductServiceServer) QueryTypeWithMultipleFilterFields(context.Context, *QueryTypeWithMultipleFilterFieldsRequest) (*QueryTypeWithMultipleFilterFieldsResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method QueryTypeWithMultipleFilterFields not implemented") +} +func (UnimplementedProductServiceServer) QueryUser(context.Context, *QueryUserRequest) (*QueryUserResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method QueryUser not implemented") +} +func (UnimplementedProductServiceServer) QueryUsers(context.Context, *QueryUsersRequest) (*QueryUsersResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method QueryUsers not implemented") +} +func (UnimplementedProductServiceServer) ResolveCategoryCategoryMetrics(context.Context, *ResolveCategoryCategoryMetricsRequest) (*ResolveCategoryCategoryMetricsResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method ResolveCategoryCategoryMetrics not implemented") +} +func (UnimplementedProductServiceServer) ResolveCategoryPopularityScore(context.Context, *ResolveCategoryPopularityScoreRequest) (*ResolveCategoryPopularityScoreResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method ResolveCategoryPopularityScore not implemented") +} +func (UnimplementedProductServiceServer) ResolveCategoryProductCount(context.Context, *ResolveCategoryProductCountRequest) (*ResolveCategoryProductCountResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method ResolveCategoryProductCount not implemented") +} +func (UnimplementedProductServiceServer) ResolveProductRecommendedCategory(context.Context, *ResolveProductRecommendedCategoryRequest) (*ResolveProductRecommendedCategoryResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method ResolveProductRecommendedCategory not implemented") +} +func (UnimplementedProductServiceServer) ResolveProductShippingEstimate(context.Context, *ResolveProductShippingEstimateRequest) (*ResolveProductShippingEstimateResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method ResolveProductShippingEstimate not implemented") +} +func (UnimplementedProductServiceServer) ResolveSubcategoryItemCount(context.Context, *ResolveSubcategoryItemCountRequest) (*ResolveSubcategoryItemCountResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method ResolveSubcategoryItemCount not implemented") +} +func (UnimplementedProductServiceServer) mustEmbedUnimplementedProductServiceServer() {} +func (UnimplementedProductServiceServer) testEmbeddedByValue() {} + +// UnsafeProductServiceServer may be embedded to opt out of forward compatibility for this service. +// Use of this interface is not recommended, as added methods to ProductServiceServer will +// result in compilation errors. +type UnsafeProductServiceServer interface { + mustEmbedUnimplementedProductServiceServer() +} + +func RegisterProductServiceServer(s grpc.ServiceRegistrar, srv ProductServiceServer) { + // If the following call pancis, it indicates UnimplementedProductServiceServer was + // embedded by pointer and is nil. This will cause panics if an + // unimplemented method is ever invoked, so we test this at initialization + // time to prevent it from happening at runtime later due to I/O. + if t, ok := srv.(interface{ testEmbeddedByValue() }); ok { + t.testEmbeddedByValue() + } + s.RegisterService(&ProductService_ServiceDesc, srv) +} + +func _ProductService_LookupProductById_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(LookupProductByIdRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(ProductServiceServer).LookupProductById(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: ProductService_LookupProductById_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(ProductServiceServer).LookupProductById(ctx, req.(*LookupProductByIdRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _ProductService_LookupStorageById_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(LookupStorageByIdRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(ProductServiceServer).LookupStorageById(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: ProductService_LookupStorageById_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(ProductServiceServer).LookupStorageById(ctx, req.(*LookupStorageByIdRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _ProductService_LookupWarehouseById_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(LookupWarehouseByIdRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(ProductServiceServer).LookupWarehouseById(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: ProductService_LookupWarehouseById_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(ProductServiceServer).LookupWarehouseById(ctx, req.(*LookupWarehouseByIdRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _ProductService_MutationBulkCreateAuthors_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(MutationBulkCreateAuthorsRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(ProductServiceServer).MutationBulkCreateAuthors(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: ProductService_MutationBulkCreateAuthors_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(ProductServiceServer).MutationBulkCreateAuthors(ctx, req.(*MutationBulkCreateAuthorsRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _ProductService_MutationBulkCreateBlogPosts_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(MutationBulkCreateBlogPostsRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(ProductServiceServer).MutationBulkCreateBlogPosts(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: ProductService_MutationBulkCreateBlogPosts_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(ProductServiceServer).MutationBulkCreateBlogPosts(ctx, req.(*MutationBulkCreateBlogPostsRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _ProductService_MutationBulkUpdateAuthors_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(MutationBulkUpdateAuthorsRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(ProductServiceServer).MutationBulkUpdateAuthors(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: ProductService_MutationBulkUpdateAuthors_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(ProductServiceServer).MutationBulkUpdateAuthors(ctx, req.(*MutationBulkUpdateAuthorsRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _ProductService_MutationBulkUpdateBlogPosts_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(MutationBulkUpdateBlogPostsRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(ProductServiceServer).MutationBulkUpdateBlogPosts(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: ProductService_MutationBulkUpdateBlogPosts_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(ProductServiceServer).MutationBulkUpdateBlogPosts(ctx, req.(*MutationBulkUpdateBlogPostsRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _ProductService_MutationCreateAuthor_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(MutationCreateAuthorRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(ProductServiceServer).MutationCreateAuthor(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: ProductService_MutationCreateAuthor_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(ProductServiceServer).MutationCreateAuthor(ctx, req.(*MutationCreateAuthorRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _ProductService_MutationCreateBlogPost_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(MutationCreateBlogPostRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(ProductServiceServer).MutationCreateBlogPost(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: ProductService_MutationCreateBlogPost_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(ProductServiceServer).MutationCreateBlogPost(ctx, req.(*MutationCreateBlogPostRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _ProductService_MutationCreateNullableFieldsType_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(MutationCreateNullableFieldsTypeRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(ProductServiceServer).MutationCreateNullableFieldsType(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: ProductService_MutationCreateNullableFieldsType_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(ProductServiceServer).MutationCreateNullableFieldsType(ctx, req.(*MutationCreateNullableFieldsTypeRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _ProductService_MutationCreateUser_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(MutationCreateUserRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(ProductServiceServer).MutationCreateUser(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: ProductService_MutationCreateUser_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(ProductServiceServer).MutationCreateUser(ctx, req.(*MutationCreateUserRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _ProductService_MutationPerformAction_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(MutationPerformActionRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(ProductServiceServer).MutationPerformAction(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: ProductService_MutationPerformAction_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(ProductServiceServer).MutationPerformAction(ctx, req.(*MutationPerformActionRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _ProductService_MutationUpdateAuthor_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(MutationUpdateAuthorRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(ProductServiceServer).MutationUpdateAuthor(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: ProductService_MutationUpdateAuthor_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(ProductServiceServer).MutationUpdateAuthor(ctx, req.(*MutationUpdateAuthorRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _ProductService_MutationUpdateBlogPost_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(MutationUpdateBlogPostRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(ProductServiceServer).MutationUpdateBlogPost(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: ProductService_MutationUpdateBlogPost_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(ProductServiceServer).MutationUpdateBlogPost(ctx, req.(*MutationUpdateBlogPostRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _ProductService_MutationUpdateNullableFieldsType_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(MutationUpdateNullableFieldsTypeRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(ProductServiceServer).MutationUpdateNullableFieldsType(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: ProductService_MutationUpdateNullableFieldsType_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(ProductServiceServer).MutationUpdateNullableFieldsType(ctx, req.(*MutationUpdateNullableFieldsTypeRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _ProductService_QueryAllAuthors_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(QueryAllAuthorsRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(ProductServiceServer).QueryAllAuthors(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: ProductService_QueryAllAuthors_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(ProductServiceServer).QueryAllAuthors(ctx, req.(*QueryAllAuthorsRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _ProductService_QueryAllBlogPosts_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(QueryAllBlogPostsRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(ProductServiceServer).QueryAllBlogPosts(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: ProductService_QueryAllBlogPosts_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(ProductServiceServer).QueryAllBlogPosts(ctx, req.(*QueryAllBlogPostsRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _ProductService_QueryAllNullableFieldsTypes_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(QueryAllNullableFieldsTypesRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(ProductServiceServer).QueryAllNullableFieldsTypes(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: ProductService_QueryAllNullableFieldsTypes_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(ProductServiceServer).QueryAllNullableFieldsTypes(ctx, req.(*QueryAllNullableFieldsTypesRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _ProductService_QueryAllPets_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(QueryAllPetsRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(ProductServiceServer).QueryAllPets(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: ProductService_QueryAllPets_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(ProductServiceServer).QueryAllPets(ctx, req.(*QueryAllPetsRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _ProductService_QueryAuthor_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(QueryAuthorRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(ProductServiceServer).QueryAuthor(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: ProductService_QueryAuthor_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(ProductServiceServer).QueryAuthor(ctx, req.(*QueryAuthorRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _ProductService_QueryAuthorById_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(QueryAuthorByIdRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(ProductServiceServer).QueryAuthorById(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: ProductService_QueryAuthorById_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(ProductServiceServer).QueryAuthorById(ctx, req.(*QueryAuthorByIdRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _ProductService_QueryAuthorsWithFilter_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(QueryAuthorsWithFilterRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(ProductServiceServer).QueryAuthorsWithFilter(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: ProductService_QueryAuthorsWithFilter_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(ProductServiceServer).QueryAuthorsWithFilter(ctx, req.(*QueryAuthorsWithFilterRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _ProductService_QueryBlogPost_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(QueryBlogPostRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(ProductServiceServer).QueryBlogPost(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: ProductService_QueryBlogPost_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(ProductServiceServer).QueryBlogPost(ctx, req.(*QueryBlogPostRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _ProductService_QueryBlogPostById_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(QueryBlogPostByIdRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(ProductServiceServer).QueryBlogPostById(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: ProductService_QueryBlogPostById_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(ProductServiceServer).QueryBlogPostById(ctx, req.(*QueryBlogPostByIdRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _ProductService_QueryBlogPostsWithFilter_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(QueryBlogPostsWithFilterRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(ProductServiceServer).QueryBlogPostsWithFilter(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: ProductService_QueryBlogPostsWithFilter_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(ProductServiceServer).QueryBlogPostsWithFilter(ctx, req.(*QueryBlogPostsWithFilterRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _ProductService_QueryBulkSearchAuthors_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(QueryBulkSearchAuthorsRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(ProductServiceServer).QueryBulkSearchAuthors(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: ProductService_QueryBulkSearchAuthors_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(ProductServiceServer).QueryBulkSearchAuthors(ctx, req.(*QueryBulkSearchAuthorsRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _ProductService_QueryBulkSearchBlogPosts_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(QueryBulkSearchBlogPostsRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(ProductServiceServer).QueryBulkSearchBlogPosts(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: ProductService_QueryBulkSearchBlogPosts_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(ProductServiceServer).QueryBulkSearchBlogPosts(ctx, req.(*QueryBulkSearchBlogPostsRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _ProductService_QueryCalculateTotals_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(QueryCalculateTotalsRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(ProductServiceServer).QueryCalculateTotals(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: ProductService_QueryCalculateTotals_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(ProductServiceServer).QueryCalculateTotals(ctx, req.(*QueryCalculateTotalsRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _ProductService_QueryCategories_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(QueryCategoriesRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(ProductServiceServer).QueryCategories(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: ProductService_QueryCategories_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(ProductServiceServer).QueryCategories(ctx, req.(*QueryCategoriesRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _ProductService_QueryCategoriesByKind_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(QueryCategoriesByKindRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(ProductServiceServer).QueryCategoriesByKind(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: ProductService_QueryCategoriesByKind_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(ProductServiceServer).QueryCategoriesByKind(ctx, req.(*QueryCategoriesByKindRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _ProductService_QueryCategoriesByKinds_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(QueryCategoriesByKindsRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(ProductServiceServer).QueryCategoriesByKinds(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: ProductService_QueryCategoriesByKinds_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(ProductServiceServer).QueryCategoriesByKinds(ctx, req.(*QueryCategoriesByKindsRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _ProductService_QueryComplexFilterType_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(QueryComplexFilterTypeRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(ProductServiceServer).QueryComplexFilterType(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: ProductService_QueryComplexFilterType_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(ProductServiceServer).QueryComplexFilterType(ctx, req.(*QueryComplexFilterTypeRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _ProductService_QueryFilterCategories_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(QueryFilterCategoriesRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(ProductServiceServer).QueryFilterCategories(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: ProductService_QueryFilterCategories_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(ProductServiceServer).QueryFilterCategories(ctx, req.(*QueryFilterCategoriesRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _ProductService_QueryNestedType_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(QueryNestedTypeRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(ProductServiceServer).QueryNestedType(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: ProductService_QueryNestedType_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(ProductServiceServer).QueryNestedType(ctx, req.(*QueryNestedTypeRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _ProductService_QueryNullableFieldsType_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(QueryNullableFieldsTypeRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(ProductServiceServer).QueryNullableFieldsType(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: ProductService_QueryNullableFieldsType_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(ProductServiceServer).QueryNullableFieldsType(ctx, req.(*QueryNullableFieldsTypeRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _ProductService_QueryNullableFieldsTypeById_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(QueryNullableFieldsTypeByIdRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(ProductServiceServer).QueryNullableFieldsTypeById(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: ProductService_QueryNullableFieldsTypeById_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(ProductServiceServer).QueryNullableFieldsTypeById(ctx, req.(*QueryNullableFieldsTypeByIdRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _ProductService_QueryNullableFieldsTypeWithFilter_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(QueryNullableFieldsTypeWithFilterRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(ProductServiceServer).QueryNullableFieldsTypeWithFilter(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: ProductService_QueryNullableFieldsTypeWithFilter_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(ProductServiceServer).QueryNullableFieldsTypeWithFilter(ctx, req.(*QueryNullableFieldsTypeWithFilterRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _ProductService_QueryRandomPet_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(QueryRandomPetRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(ProductServiceServer).QueryRandomPet(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: ProductService_QueryRandomPet_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(ProductServiceServer).QueryRandomPet(ctx, req.(*QueryRandomPetRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _ProductService_QueryRandomSearchResult_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(QueryRandomSearchResultRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(ProductServiceServer).QueryRandomSearchResult(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: ProductService_QueryRandomSearchResult_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(ProductServiceServer).QueryRandomSearchResult(ctx, req.(*QueryRandomSearchResultRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _ProductService_QueryRecursiveType_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(QueryRecursiveTypeRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(ProductServiceServer).QueryRecursiveType(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: ProductService_QueryRecursiveType_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(ProductServiceServer).QueryRecursiveType(ctx, req.(*QueryRecursiveTypeRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _ProductService_QuerySearch_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(QuerySearchRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(ProductServiceServer).QuerySearch(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: ProductService_QuerySearch_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(ProductServiceServer).QuerySearch(ctx, req.(*QuerySearchRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _ProductService_QueryTypeFilterWithArguments_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(QueryTypeFilterWithArgumentsRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(ProductServiceServer).QueryTypeFilterWithArguments(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: ProductService_QueryTypeFilterWithArguments_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(ProductServiceServer).QueryTypeFilterWithArguments(ctx, req.(*QueryTypeFilterWithArgumentsRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _ProductService_QueryTypeWithMultipleFilterFields_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(QueryTypeWithMultipleFilterFieldsRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(ProductServiceServer).QueryTypeWithMultipleFilterFields(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: ProductService_QueryTypeWithMultipleFilterFields_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(ProductServiceServer).QueryTypeWithMultipleFilterFields(ctx, req.(*QueryTypeWithMultipleFilterFieldsRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _ProductService_QueryUser_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(QueryUserRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(ProductServiceServer).QueryUser(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: ProductService_QueryUser_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(ProductServiceServer).QueryUser(ctx, req.(*QueryUserRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _ProductService_QueryUsers_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(QueryUsersRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(ProductServiceServer).QueryUsers(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: ProductService_QueryUsers_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(ProductServiceServer).QueryUsers(ctx, req.(*QueryUsersRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _ProductService_ResolveCategoryCategoryMetrics_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(ResolveCategoryCategoryMetricsRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(ProductServiceServer).ResolveCategoryCategoryMetrics(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: ProductService_ResolveCategoryCategoryMetrics_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(ProductServiceServer).ResolveCategoryCategoryMetrics(ctx, req.(*ResolveCategoryCategoryMetricsRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _ProductService_ResolveCategoryPopularityScore_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(ResolveCategoryPopularityScoreRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(ProductServiceServer).ResolveCategoryPopularityScore(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: ProductService_ResolveCategoryPopularityScore_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(ProductServiceServer).ResolveCategoryPopularityScore(ctx, req.(*ResolveCategoryPopularityScoreRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _ProductService_ResolveCategoryProductCount_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(ResolveCategoryProductCountRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(ProductServiceServer).ResolveCategoryProductCount(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: ProductService_ResolveCategoryProductCount_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(ProductServiceServer).ResolveCategoryProductCount(ctx, req.(*ResolveCategoryProductCountRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _ProductService_ResolveProductRecommendedCategory_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(ResolveProductRecommendedCategoryRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(ProductServiceServer).ResolveProductRecommendedCategory(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: ProductService_ResolveProductRecommendedCategory_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(ProductServiceServer).ResolveProductRecommendedCategory(ctx, req.(*ResolveProductRecommendedCategoryRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _ProductService_ResolveProductShippingEstimate_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(ResolveProductShippingEstimateRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(ProductServiceServer).ResolveProductShippingEstimate(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: ProductService_ResolveProductShippingEstimate_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(ProductServiceServer).ResolveProductShippingEstimate(ctx, req.(*ResolveProductShippingEstimateRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _ProductService_ResolveSubcategoryItemCount_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(ResolveSubcategoryItemCountRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(ProductServiceServer).ResolveSubcategoryItemCount(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: ProductService_ResolveSubcategoryItemCount_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(ProductServiceServer).ResolveSubcategoryItemCount(ctx, req.(*ResolveSubcategoryItemCountRequest)) + } + return interceptor(ctx, in, info, handler) +} + +// ProductService_ServiceDesc is the grpc.ServiceDesc for ProductService service. +// It's only intended for direct use with grpc.RegisterService, +// and not to be introspected or modified (even as a copy) +var ProductService_ServiceDesc = grpc.ServiceDesc{ + ServiceName: "productv1.ProductService", + HandlerType: (*ProductServiceServer)(nil), + Methods: []grpc.MethodDesc{ + { + MethodName: "LookupProductById", + Handler: _ProductService_LookupProductById_Handler, + }, + { + MethodName: "LookupStorageById", + Handler: _ProductService_LookupStorageById_Handler, + }, + { + MethodName: "LookupWarehouseById", + Handler: _ProductService_LookupWarehouseById_Handler, + }, + { + MethodName: "MutationBulkCreateAuthors", + Handler: _ProductService_MutationBulkCreateAuthors_Handler, + }, + { + MethodName: "MutationBulkCreateBlogPosts", + Handler: _ProductService_MutationBulkCreateBlogPosts_Handler, + }, + { + MethodName: "MutationBulkUpdateAuthors", + Handler: _ProductService_MutationBulkUpdateAuthors_Handler, + }, + { + MethodName: "MutationBulkUpdateBlogPosts", + Handler: _ProductService_MutationBulkUpdateBlogPosts_Handler, + }, + { + MethodName: "MutationCreateAuthor", + Handler: _ProductService_MutationCreateAuthor_Handler, + }, + { + MethodName: "MutationCreateBlogPost", + Handler: _ProductService_MutationCreateBlogPost_Handler, + }, + { + MethodName: "MutationCreateNullableFieldsType", + Handler: _ProductService_MutationCreateNullableFieldsType_Handler, + }, + { + MethodName: "MutationCreateUser", + Handler: _ProductService_MutationCreateUser_Handler, + }, + { + MethodName: "MutationPerformAction", + Handler: _ProductService_MutationPerformAction_Handler, + }, + { + MethodName: "MutationUpdateAuthor", + Handler: _ProductService_MutationUpdateAuthor_Handler, + }, + { + MethodName: "MutationUpdateBlogPost", + Handler: _ProductService_MutationUpdateBlogPost_Handler, + }, + { + MethodName: "MutationUpdateNullableFieldsType", + Handler: _ProductService_MutationUpdateNullableFieldsType_Handler, + }, + { + MethodName: "QueryAllAuthors", + Handler: _ProductService_QueryAllAuthors_Handler, + }, + { + MethodName: "QueryAllBlogPosts", + Handler: _ProductService_QueryAllBlogPosts_Handler, + }, + { + MethodName: "QueryAllNullableFieldsTypes", + Handler: _ProductService_QueryAllNullableFieldsTypes_Handler, + }, + { + MethodName: "QueryAllPets", + Handler: _ProductService_QueryAllPets_Handler, + }, + { + MethodName: "QueryAuthor", + Handler: _ProductService_QueryAuthor_Handler, + }, + { + MethodName: "QueryAuthorById", + Handler: _ProductService_QueryAuthorById_Handler, + }, + { + MethodName: "QueryAuthorsWithFilter", + Handler: _ProductService_QueryAuthorsWithFilter_Handler, + }, + { + MethodName: "QueryBlogPost", + Handler: _ProductService_QueryBlogPost_Handler, + }, + { + MethodName: "QueryBlogPostById", + Handler: _ProductService_QueryBlogPostById_Handler, + }, + { + MethodName: "QueryBlogPostsWithFilter", + Handler: _ProductService_QueryBlogPostsWithFilter_Handler, + }, + { + MethodName: "QueryBulkSearchAuthors", + Handler: _ProductService_QueryBulkSearchAuthors_Handler, + }, + { + MethodName: "QueryBulkSearchBlogPosts", + Handler: _ProductService_QueryBulkSearchBlogPosts_Handler, + }, + { + MethodName: "QueryCalculateTotals", + Handler: _ProductService_QueryCalculateTotals_Handler, + }, + { + MethodName: "QueryCategories", + Handler: _ProductService_QueryCategories_Handler, + }, + { + MethodName: "QueryCategoriesByKind", + Handler: _ProductService_QueryCategoriesByKind_Handler, + }, + { + MethodName: "QueryCategoriesByKinds", + Handler: _ProductService_QueryCategoriesByKinds_Handler, + }, + { + MethodName: "QueryComplexFilterType", + Handler: _ProductService_QueryComplexFilterType_Handler, + }, + { + MethodName: "QueryFilterCategories", + Handler: _ProductService_QueryFilterCategories_Handler, + }, + { + MethodName: "QueryNestedType", + Handler: _ProductService_QueryNestedType_Handler, + }, + { + MethodName: "QueryNullableFieldsType", + Handler: _ProductService_QueryNullableFieldsType_Handler, + }, + { + MethodName: "QueryNullableFieldsTypeById", + Handler: _ProductService_QueryNullableFieldsTypeById_Handler, + }, + { + MethodName: "QueryNullableFieldsTypeWithFilter", + Handler: _ProductService_QueryNullableFieldsTypeWithFilter_Handler, + }, + { + MethodName: "QueryRandomPet", + Handler: _ProductService_QueryRandomPet_Handler, + }, + { + MethodName: "QueryRandomSearchResult", + Handler: _ProductService_QueryRandomSearchResult_Handler, + }, + { + MethodName: "QueryRecursiveType", + Handler: _ProductService_QueryRecursiveType_Handler, + }, + { + MethodName: "QuerySearch", + Handler: _ProductService_QuerySearch_Handler, + }, + { + MethodName: "QueryTypeFilterWithArguments", + Handler: _ProductService_QueryTypeFilterWithArguments_Handler, + }, + { + MethodName: "QueryTypeWithMultipleFilterFields", + Handler: _ProductService_QueryTypeWithMultipleFilterFields_Handler, + }, + { + MethodName: "QueryUser", + Handler: _ProductService_QueryUser_Handler, + }, + { + MethodName: "QueryUsers", + Handler: _ProductService_QueryUsers_Handler, + }, + { + MethodName: "ResolveCategoryCategoryMetrics", + Handler: _ProductService_ResolveCategoryCategoryMetrics_Handler, + }, + { + MethodName: "ResolveCategoryPopularityScore", + Handler: _ProductService_ResolveCategoryPopularityScore_Handler, + }, + { + MethodName: "ResolveCategoryProductCount", + Handler: _ProductService_ResolveCategoryProductCount_Handler, + }, + { + MethodName: "ResolveProductRecommendedCategory", + Handler: _ProductService_ResolveProductRecommendedCategory_Handler, + }, + { + MethodName: "ResolveProductShippingEstimate", + Handler: _ProductService_ResolveProductShippingEstimate_Handler, + }, + { + MethodName: "ResolveSubcategoryItemCount", + Handler: _ProductService_ResolveSubcategoryItemCount_Handler, + }, + }, + Streams: []grpc.StreamDesc{}, + Metadata: "testdata/service.proto", +} diff --git a/v2/pkg/grpctest/testdata/products.graphqls b/v2/pkg/grpctest/testdata/products.graphqls index 43547d36a..8594ee329 100644 --- a/v2/pkg/grpctest/testdata/products.graphqls +++ b/v2/pkg/grpctest/testdata/products.graphqls @@ -5,7 +5,8 @@ type Product @key(fields: "id") { id: ID! name: String! price: Float! - # shippingEstimate(input: ShippingEstimateInput!): Float! + shippingEstimate(input: ShippingEstimateInput!): Float! @configureResolver(context: "id price") + recommendedCategory(maxPrice: Int!): Category @configureResolver(context: "id name price") } type Storage @key(fields: "id") { @@ -352,6 +353,18 @@ input SubcategoryItemFilter { searchTerm: String } +enum ShippingDestination { + DOMESTIC + EXPRESS + INTERNATIONAL +} + +input ShippingEstimateInput { + destination: ShippingDestination! + weight: Float! + expedited: Boolean +} + type Query { _entities(representations: [_Any!]!): [_Entity!]! users: [User!]! From 33e58cbbf66f92d89bcf9af9c87b4a69754481e7 Mon Sep 17 00:00:00 2001 From: Ludwig Bedacht Date: Thu, 16 Oct 2025 10:29:02 +0200 Subject: [PATCH 10/72] chore: improvements --- .../execution_plan_federation_test.go | 624 +++++++++++++----- .../grpc_datasource/execution_plan_visitor.go | 6 +- .../execution_plan_visitor_federation.go | 7 +- 3 files changed, 459 insertions(+), 178 deletions(-) diff --git a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_federation_test.go b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_federation_test.go index 7e636d0ab..a0fd7ce2d 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_federation_test.go +++ b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_federation_test.go @@ -268,179 +268,6 @@ func TestEntityLookup(t *testing.T) { }, }, }, - { - name: "Should create an execution plan for an entity lookup with a field resolver", - query: `query EntityLookup($representations: [_Any!]!, $input: ShippingEstimateInput!) { _entities(representations: $representations) { ... on Product { __typename id name price shippingEstimate(input: $input) } } }`, - mapping: testMapping(), - federationConfigs: plan.FederationFieldConfigurations{ - { - TypeName: "Product", - SelectionSet: "id", - }, - }, - expectedPlan: &RPCExecutionPlan{ - Calls: []RPCCall{ - { - ServiceName: "Products", - MethodName: "LookupProductById", - Kind: CallKindEntity, - Request: RPCMessage{ - Name: "LookupProductByIdRequest", - Fields: []RPCField{ - { - Name: "keys", - TypeName: string(DataTypeMessage), - Repeated: true, - JSONPath: "representations", - Message: &RPCMessage{ - Name: "LookupProductByIdKey", - MemberTypes: []string{"Product"}, - Fields: []RPCField{ - { - Name: "id", - TypeName: string(DataTypeString), - JSONPath: "id", - }, - }, - }, - }, - }, - }, - Response: RPCMessage{ - Name: "LookupProductByIdResponse", - Fields: []RPCField{ - { - Name: "result", - TypeName: string(DataTypeMessage), - Repeated: true, - JSONPath: "_entities", - Message: &RPCMessage{ - Name: "Product", - Fields: []RPCField{ - { - Name: "__typename", - TypeName: string(DataTypeString), - JSONPath: "__typename", - StaticValue: "Product", - }, - { - Name: "id", - TypeName: string(DataTypeString), - JSONPath: "id", - }, - { - Name: "name", - TypeName: string(DataTypeString), - JSONPath: "name", - }, - - { - Name: "price", - TypeName: string(DataTypeDouble), - JSONPath: "price", - }, - }, - }, - }, - }, - }, - }, - { - ServiceName: "Products", - MethodName: "ResolveProductShippingEstimate", - Kind: CallKindResolve, - DependentCalls: []int{0}, - ResponsePath: buildPath("_entities.shippingEstimate"), - Request: RPCMessage{ - Name: "ResolveProductShippingEstimateRequest", - Fields: []RPCField{ - { - Name: "context", - TypeName: string(DataTypeMessage), - JSONPath: "", - Repeated: true, - Message: &RPCMessage{ - Name: "ResolveProductShippingEstimateContext", - Fields: []RPCField{ - { - Name: "id", - TypeName: string(DataTypeString), - JSONPath: "id", - ResolvePath: buildPath("result.id"), - }, - { - Name: "price", - TypeName: string(DataTypeDouble), - JSONPath: "price", - ResolvePath: buildPath("result.price"), - }, - }, - }, - }, - { - Name: "field_args", - TypeName: string(DataTypeMessage), - JSONPath: "", - Message: &RPCMessage{ - Name: "ResolveProductShippingEstimateArgs", - Fields: []RPCField{ - { - Name: "input", - TypeName: string(DataTypeMessage), - JSONPath: "input", - Message: &RPCMessage{ - Name: "ShippingEstimateInput", - Fields: []RPCField{ - { - Name: "destination", - TypeName: string(DataTypeEnum), - JSONPath: "destination", - EnumName: "ShippingDestination", - }, - { - Name: "weight", - TypeName: string(DataTypeDouble), - JSONPath: "weight", - }, - { - Name: "expedited", - TypeName: string(DataTypeBool), - JSONPath: "expedited", - Optional: true, - }, - }, - }, - }, - }, - }, - }, - }, - }, - Response: RPCMessage{ - Name: "ResolveProductShippingEstimateResponse", - Fields: []RPCField{ - { - Name: "result", - TypeName: string(DataTypeMessage), - JSONPath: "result", - Repeated: true, - Message: &RPCMessage{ - Name: "ResolveProductShippingEstimateResult", - Fields: []RPCField{ - { - Name: "shipping_estimate", - TypeName: string(DataTypeDouble), - JSONPath: "shippingEstimate", - }, - }, - }, - }, - }, - }, - }, - }, - }, - }, } for _, tt := range tests { @@ -1038,6 +865,457 @@ func TestEntityKeys(t *testing.T) { } } +func TestEntityLookupWithFieldResolvers(t *testing.T) { + t.Parallel() + tests := []struct { + name string + query string + expectedPlan *RPCExecutionPlan + mapping *GRPCMapping + federationConfigs plan.FederationFieldConfigurations + }{ + + { + name: "Should create an execution plan for an entity lookup with a field resolver", + query: `query EntityLookup($representations: [_Any!]!, $input: ShippingEstimateInput!) { _entities(representations: $representations) { ... on Product { __typename id name price shippingEstimate(input: $input) } } }`, + mapping: testMapping(), + federationConfigs: plan.FederationFieldConfigurations{ + { + TypeName: "Product", + SelectionSet: "id", + }, + }, + expectedPlan: &RPCExecutionPlan{ + Calls: []RPCCall{ + { + ServiceName: "Products", + MethodName: "LookupProductById", + Kind: CallKindEntity, + Request: RPCMessage{ + Name: "LookupProductByIdRequest", + Fields: []RPCField{ + { + Name: "keys", + TypeName: string(DataTypeMessage), + Repeated: true, + JSONPath: "representations", + Message: &RPCMessage{ + Name: "LookupProductByIdKey", + MemberTypes: []string{"Product"}, + Fields: []RPCField{ + { + Name: "id", + TypeName: string(DataTypeString), + JSONPath: "id", + }, + }, + }, + }, + }, + }, + Response: RPCMessage{ + Name: "LookupProductByIdResponse", + Fields: []RPCField{ + { + Name: "result", + TypeName: string(DataTypeMessage), + Repeated: true, + JSONPath: "_entities", + Message: &RPCMessage{ + Name: "Product", + Fields: []RPCField{ + { + Name: "__typename", + TypeName: string(DataTypeString), + JSONPath: "__typename", + StaticValue: "Product", + }, + { + Name: "id", + TypeName: string(DataTypeString), + JSONPath: "id", + }, + { + Name: "name", + TypeName: string(DataTypeString), + JSONPath: "name", + }, + + { + Name: "price", + TypeName: string(DataTypeDouble), + JSONPath: "price", + }, + }, + }, + }, + }, + }, + }, + { + ServiceName: "Products", + MethodName: "ResolveProductShippingEstimate", + Kind: CallKindResolve, + DependentCalls: []int{0}, + ResponsePath: buildPath("_entities.shippingEstimate"), + Request: RPCMessage{ + Name: "ResolveProductShippingEstimateRequest", + Fields: []RPCField{ + { + Name: "context", + TypeName: string(DataTypeMessage), + JSONPath: "", + Repeated: true, + Message: &RPCMessage{ + Name: "ResolveProductShippingEstimateContext", + Fields: []RPCField{ + { + Name: "id", + TypeName: string(DataTypeString), + JSONPath: "id", + ResolvePath: buildPath("result.id"), + }, + { + Name: "price", + TypeName: string(DataTypeDouble), + JSONPath: "price", + ResolvePath: buildPath("result.price"), + }, + }, + }, + }, + { + Name: "field_args", + TypeName: string(DataTypeMessage), + JSONPath: "", + Message: &RPCMessage{ + Name: "ResolveProductShippingEstimateArgs", + Fields: []RPCField{ + { + Name: "input", + TypeName: string(DataTypeMessage), + JSONPath: "input", + Message: &RPCMessage{ + Name: "ShippingEstimateInput", + Fields: []RPCField{ + { + Name: "destination", + TypeName: string(DataTypeEnum), + JSONPath: "destination", + EnumName: "ShippingDestination", + }, + { + Name: "weight", + TypeName: string(DataTypeDouble), + JSONPath: "weight", + }, + { + Name: "expedited", + TypeName: string(DataTypeBool), + JSONPath: "expedited", + Optional: true, + }, + }, + }, + }, + }, + }, + }, + }, + }, + Response: RPCMessage{ + Name: "ResolveProductShippingEstimateResponse", + Fields: []RPCField{ + { + Name: "result", + TypeName: string(DataTypeMessage), + JSONPath: "result", + Repeated: true, + Message: &RPCMessage{ + Name: "ResolveProductShippingEstimateResult", + Fields: []RPCField{ + { + Name: "shipping_estimate", + TypeName: string(DataTypeDouble), + JSONPath: "shippingEstimate", + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + { + name: "Should create an execution plan for multiple entity lookups with field resolvers", + query: `query MultiEntityLookup($representations: [_Any!]!, $input: ShippingEstimateInput!) { _entities(representations: $representations) { ... on Storage { __typename id name location } ... on Product { __typename id name price shippingEstimate(input: $input) } } }`, + mapping: testMapping(), + federationConfigs: plan.FederationFieldConfigurations{ + { + TypeName: "Storage", + SelectionSet: "id", + }, + { + TypeName: "Product", + SelectionSet: "id", + }, + }, + expectedPlan: &RPCExecutionPlan{ + Calls: []RPCCall{ + { + ServiceName: "Products", + MethodName: "LookupStorageById", + Kind: CallKindEntity, + Request: RPCMessage{ + Name: "LookupStorageByIdRequest", + Fields: []RPCField{ + { + Name: "keys", + TypeName: string(DataTypeMessage), + Repeated: true, + JSONPath: "representations", + Message: &RPCMessage{ + Name: "LookupStorageByIdKey", + MemberTypes: []string{"Storage"}, + Fields: []RPCField{ + { + Name: "id", + TypeName: string(DataTypeString), + JSONPath: "id", + }, + }, + }, + }, + }, + }, + Response: RPCMessage{ + Name: "LookupStorageByIdResponse", + Fields: []RPCField{ + { + Name: "result", + TypeName: string(DataTypeMessage), + Repeated: true, + JSONPath: "_entities", + Message: &RPCMessage{ + Name: "Storage", + Fields: []RPCField{ + { + Name: "__typename", + TypeName: string(DataTypeString), + JSONPath: "__typename", + StaticValue: "Storage", + }, + { + Name: "id", + TypeName: string(DataTypeString), + JSONPath: "id", + }, + { + Name: "name", + TypeName: string(DataTypeString), + JSONPath: "name", + }, + { + Name: "location", + TypeName: string(DataTypeString), + JSONPath: "location", + }, + }, + }, + }, + }, + }, + }, + { + ServiceName: "Products", + MethodName: "LookupProductById", + Kind: CallKindEntity, + Request: RPCMessage{ + Name: "LookupProductByIdRequest", + Fields: []RPCField{ + { + Name: "keys", + TypeName: string(DataTypeMessage), + Repeated: true, + JSONPath: "representations", + Message: &RPCMessage{ + Name: "LookupProductByIdKey", + MemberTypes: []string{"Product"}, + Fields: []RPCField{ + { + Name: "id", + TypeName: string(DataTypeString), + JSONPath: "id", + }, + }, + }, + }, + }, + }, + Response: RPCMessage{ + Name: "LookupProductByIdResponse", + Fields: []RPCField{ + { + Name: "result", + TypeName: string(DataTypeMessage), + Repeated: true, + JSONPath: "_entities", + Message: &RPCMessage{ + Name: "Product", + Fields: []RPCField{ + { + Name: "__typename", + TypeName: string(DataTypeString), + JSONPath: "__typename", + StaticValue: "Product", + }, + { + Name: "id", + TypeName: string(DataTypeString), + JSONPath: "id", + }, + { + Name: "name", + TypeName: string(DataTypeString), + JSONPath: "name", + }, + { + Name: "price", + TypeName: string(DataTypeDouble), + JSONPath: "price", + }, + }, + }, + }, + }, + }, + }, + { + ServiceName: "Products", + MethodName: "ResolveProductShippingEstimate", + Kind: CallKindResolve, + DependentCalls: []int{1}, + ResponsePath: buildPath("_entities.shippingEstimate"), + Request: RPCMessage{ + Name: "ResolveProductShippingEstimateRequest", + Fields: []RPCField{ + { + Name: "context", + TypeName: string(DataTypeMessage), + JSONPath: "", + Repeated: true, + Message: &RPCMessage{ + Name: "ResolveProductShippingEstimateContext", + Fields: []RPCField{ + { + Name: "id", + TypeName: string(DataTypeString), + JSONPath: "id", + ResolvePath: buildPath("result.id"), + }, + { + Name: "price", + TypeName: string(DataTypeDouble), + JSONPath: "price", + ResolvePath: buildPath("result.price"), + }, + }, + }, + }, + { + Name: "field_args", + TypeName: string(DataTypeMessage), + JSONPath: "", + Message: &RPCMessage{ + Name: "ResolveProductShippingEstimateArgs", + Fields: []RPCField{ + { + Name: "input", + TypeName: string(DataTypeMessage), + JSONPath: "input", + Message: &RPCMessage{ + Name: "ShippingEstimateInput", + Fields: []RPCField{ + { + Name: "destination", + TypeName: string(DataTypeEnum), + JSONPath: "destination", + EnumName: "ShippingDestination", + }, + { + Name: "weight", + TypeName: string(DataTypeDouble), + JSONPath: "weight", + }, + { + Name: "expedited", + TypeName: string(DataTypeBool), + JSONPath: "expedited", + Optional: true, + }, + }, + }, + }, + }, + }, + }, + }, + }, + Response: RPCMessage{ + Name: "ResolveProductShippingEstimateResponse", + Fields: []RPCField{ + { + Name: "result", + TypeName: string(DataTypeMessage), + JSONPath: "result", + Repeated: true, + Message: &RPCMessage{ + Name: "ResolveProductShippingEstimateResult", + Fields: []RPCField{ + { + Name: "shipping_estimate", + TypeName: string(DataTypeDouble), + JSONPath: "shippingEstimate", + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + } + + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + t.Parallel() + // Parse the GraphQL schema + schemaDoc := grpctest.MustGraphQLSchema(t) + + // Parse the GraphQL query + queryDoc, report := astparser.ParseGraphqlDocumentString(tt.query) + if report.HasErrors() { + t.Fatalf("failed to parse query: %s", report.Error()) + } + + planner := NewPlanner("Products", tt.mapping, tt.federationConfigs) + plan, err := planner.PlanOperation(&queryDoc, &schemaDoc) + if err != nil { + t.Fatalf("failed to plan operation: %s", err) + } + + diff := cmp.Diff(tt.expectedPlan, plan) + if diff != "" { + t.Fatalf("execution plan mismatch: %s", diff) + } + }) + } +} + func runFederationTest(t *testing.T, tt struct { name string query string diff --git a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor.go b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor.go index 6f4ca132f..3bfe086c4 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor.go +++ b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor.go @@ -301,6 +301,8 @@ func (r *rpcPlanVisitor) handleRootField(isRootField bool, ref int) error { ServiceName: r.planCtx.resolveServiceName(r.subgraphName), } + r.relatedCallID = r.currentCallID + r.planInfo.currentRequestMessage = &r.currentCall.Request r.planInfo.currentResponseMessage = &r.currentCall.Response @@ -350,7 +352,6 @@ func (r *rpcPlanVisitor) EnterField(ref int) { // TODO: this needs to be available for both visitors and added to the plancontext if fieldArgs := r.operation.FieldArguments(ref); !inRootField && len(fieldArgs) > 0 { // We don't want to add fields from the selection set to the actual call - r.relatedCallID++ // TODO: handle this for multiple queries resolvedField := resolvedField{ callerRef: r.relatedCallID, parentTypeRef: r.walker.EnclosingTypeDefinition.Ref, @@ -367,6 +368,9 @@ func (r *rpcPlanVisitor) EnterField(ref int) { r.resolvedFields = append(r.resolvedFields, resolvedField) r.resolvedFieldIndex = len(r.resolvedFields) - 1 r.fieldPath = r.fieldPath.WithFieldNameItem(r.operation.FieldNameBytes(ref)) + + // In case of nested fields with arguments, we need to increment the related call ID. + r.relatedCallID++ return } diff --git a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor_federation.go b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor_federation.go index 8ebc790e6..58243b840 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor_federation.go +++ b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor_federation.go @@ -47,10 +47,9 @@ type rpcPlanVisitorFederation struct { entityInfo entityInfo federationConfigData []federationConfigData - plan *RPCExecutionPlan - subgraphName string - currentCall *RPCCall - currentCallIndex int + plan *RPCExecutionPlan + subgraphName string + currentCall *RPCCall relatedCallID int resolvedFieldIndex int From fb4c82009b44fccc9ca30ebe3e877de2fc3cd2e2 Mon Sep 17 00:00:00 2001 From: Ludwig Bedacht Date: Thu, 16 Oct 2025 10:31:11 +0200 Subject: [PATCH 11/72] chore: lint --- v2/pkg/grpctest/cmd/mapping_helper/main.go | 1 + v2/pkg/grpctest/mockservice.go | 2 +- 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/v2/pkg/grpctest/cmd/mapping_helper/main.go b/v2/pkg/grpctest/cmd/mapping_helper/main.go index 9280120f4..5d654db33 100644 --- a/v2/pkg/grpctest/cmd/mapping_helper/main.go +++ b/v2/pkg/grpctest/cmd/mapping_helper/main.go @@ -11,6 +11,7 @@ import ( type TemplateData struct { JSONMapping + ResolveRPCs map[string][]ResolveRPC } diff --git a/v2/pkg/grpctest/mockservice.go b/v2/pkg/grpctest/mockservice.go index 5e504a2f3..3a50d1c1f 100644 --- a/v2/pkg/grpctest/mockservice.go +++ b/v2/pkg/grpctest/mockservice.go @@ -37,7 +37,7 @@ func (s *MockService) ResolveProductRecommendedCategory(_ context.Context, req * }) } else { // Create a recommended category based on product context - categoryKind := productv1.CategoryKind_CATEGORY_KIND_OTHER + var categoryKind productv1.CategoryKind if ctx.GetPrice() < 50 { categoryKind = productv1.CategoryKind_CATEGORY_KIND_BOOK } else if ctx.GetPrice() < 200 { From 46cde413b09446c756917a856a173149bb893108 Mon Sep 17 00:00:00 2001 From: Ludwig Bedacht Date: Thu, 16 Oct 2025 12:24:25 +0200 Subject: [PATCH 12/72] chore: improve compiler --- .../datasource/grpc_datasource/compiler.go | 20 ++++++++++++++----- 1 file changed, 15 insertions(+), 5 deletions(-) diff --git a/v2/pkg/engine/datasource/grpc_datasource/compiler.go b/v2/pkg/engine/datasource/grpc_datasource/compiler.go index 6a5cdda75..9875ce74d 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/compiler.go +++ b/v2/pkg/engine/datasource/grpc_datasource/compiler.go @@ -182,9 +182,9 @@ type Message struct { // FieldByName returns a field by its name. // Returns nil if no field with the given name exists. func (m *Message) FieldByName(name string) *Field { - for _, field := range m.Fields { + for index, field := range m.Fields { if field.Name == name { - return &field + return &m.Fields[index] } } @@ -502,6 +502,10 @@ func (p *RPCCompiler) CompileNode(graph *DependencyGraph, fetch FetchItem, input return ServiceCall{}, err } + if len(context) == 0 { + return ServiceCall{}, fmt.Errorf("context is required for resolve calls") + } + request = p.buildProtoMessageWithContext(inputMessage, &call.Request, inputData, context) } @@ -605,7 +609,7 @@ func (p *RPCCompiler) buildProtoMessageWithContext(inputMessage Message, rpcMess rootMessage := dynamicpb.NewMessage(inputMessage.Desc) if len(inputMessage.Fields) != 2 { - p.report.AddInternalError(fmt.Errorf("message %s must have exactly one key field", inputMessage.Name)) + p.report.AddInternalError(fmt.Errorf("message %s must have exactly two fields: context and field_args", inputMessage.Name)) return nil } @@ -640,14 +644,20 @@ func (p *RPCCompiler) buildProtoMessageWithContext(inputMessage Message, rpcMess contextList.Append(val) } - argsMessage := p.doc.Messages[inputMessage.FieldByName("field_args").MessageRef] + argsSchemaField := inputMessage.FieldByName("field_args") + if argsSchemaField == nil { + p.report.AddInternalError(fmt.Errorf("field_args field not found in message %s", inputMessage.Name)) + return nil + } + + argsMessage := p.doc.Messages[argsSchemaField.MessageRef] argsRPCField := rpcMessage.Fields.ByName("field_args") if argsRPCField == nil { p.report.AddInternalError(fmt.Errorf("field_args field not found in message %s", rpcMessage.Name)) return nil } - args := p.buildProtoMessage(argsMessage, rpcMessage.Fields[1].Message, data) + args := p.buildProtoMessage(argsMessage, argsRPCField.Message, data) // // Set the key list p.setMessageValue(rootMessage, contextSchemaField.Name, protoref.ValueOfList(contextList)) From 17a06a1dacc05356784a0db276da22a6cb803888 Mon Sep 17 00:00:00 2001 From: Ludwig Bedacht Date: Thu, 16 Oct 2025 12:37:45 +0200 Subject: [PATCH 13/72] chore: fix nested list traversal --- .../engine/datasource/grpc_datasource/compiler.go | 15 ++++++++------- 1 file changed, 8 insertions(+), 7 deletions(-) diff --git a/v2/pkg/engine/datasource/grpc_datasource/compiler.go b/v2/pkg/engine/datasource/grpc_datasource/compiler.go index 9875ce74d..78be8d1f3 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/compiler.go +++ b/v2/pkg/engine/datasource/grpc_datasource/compiler.go @@ -832,22 +832,23 @@ func (p *RPCCompiler) resolveUnderlyingListItems(value protoref.Value, nestingLe return nil } - if itemsValue.List().Len() == 0 { + itemsList := itemsValue.List() + if itemsList.Len() == 0 { return nil } if nestingLevel > 1 { - items := make([]protoref.Value, 0, listMsg.List().Len()) - for i := 0; i < listMsg.List().Len(); i++ { - items = append(items, p.resolveUnderlyingListItems(listMsg.List().Get(i), nestingLevel-1)...) + items := make([]protoref.Value, 0, itemsList.Len()) + for i := 0; i < itemsList.Len(); i++ { + items = append(items, p.resolveUnderlyingListItems(itemsList.Get(i), nestingLevel-1)...) } return items } - result := make([]protoref.Value, itemsValue.List().Len()) - for i := 0; i < itemsValue.List().Len(); i++ { - result[i] = itemsValue.List().Get(i) + result := make([]protoref.Value, itemsList.Len()) + for i := 0; i < itemsList.Len(); i++ { + result[i] = itemsList.Get(i) } return result From c7d5e7d8a37daba9965f89a6e144a965d75b0804 Mon Sep 17 00:00:00 2001 From: Ludwig Bedacht Date: Thu, 16 Oct 2025 14:58:47 +0200 Subject: [PATCH 14/72] chore: add length check on type definitions --- .../datasource/grpc_datasource/execution_plan_visitor.go | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor.go b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor.go index 3bfe086c4..f39d20faa 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor.go +++ b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor.go @@ -157,6 +157,11 @@ func (r *rpcPlanVisitor) EnterArgument(ref int) { return } + if len(r.walker.TypeDefinitions) < 2 { + r.walker.StopWithInternalErr(fmt.Errorf("internal: unexpected type stack depth for argument on %s", r.operation.FieldNameString(ancestor.Ref))) + return + } + // As we check that we are inside of a field we can safely access the second to last type definition. parentTypeNode := r.walker.TypeDefinitions[len(r.walker.TypeDefinitions)-2] fieldDefinitionRef, exists := r.definition.NodeFieldDefinitionByName(parentTypeNode, r.operation.FieldNameBytes(ancestor.Ref)) From 3fa4321347cb35922f1c06e184729d46777a159d Mon Sep 17 00:00:00 2001 From: Ludwig Bedacht Date: Thu, 16 Oct 2025 16:17:35 +0200 Subject: [PATCH 15/72] chore: fix makefile --- v2/pkg/grpctest/Makefile | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/v2/pkg/grpctest/Makefile b/v2/pkg/grpctest/Makefile index 4fda449fd..b4ed22c2b 100644 --- a/v2/pkg/grpctest/Makefile +++ b/v2/pkg/grpctest/Makefile @@ -10,8 +10,6 @@ generate-proto: install-protoc protoc --go_out=productv1 --go_opt=paths=source_relative \ --go-grpc_out=productv1 --go-grpc_opt=paths=source_relative \ product.proto - mv $(mkfile_dir)/testdata/service.proto $(mkfile_dir)/product.proto - rm -f $(mkfile_dir)/testdata/service.proto.lock.json .PHONY: build-plugin build-plugin: @@ -20,6 +18,8 @@ build-plugin: .PHONY: regenerate-proto regenerate-proto: pnpx wgc@latest grpc-service generate -i testdata/products.graphqls -o testdata/ -p productv1 -g "cosmo/pkg/proto/productv1;productv1" Product + mv $(mkfile_dir)/testdata/service.proto $(mkfile_dir)/product.proto + rm -f $(mkfile_dir)/testdata/service.proto.lock.json .PHONY: generate-mapping-code generate-mapping-code: regenerate-proto @@ -31,6 +31,8 @@ generate-mapping-code: regenerate-proto .PHONY: regenerate-proto-local regenerate-proto-local: pnpx tsx --env-file ../../../../cosmo/cli/.env ../../../../cosmo/cli/src/index.ts grpc-service generate -i testdata/products.graphqls -o testdata/ -p productv1 -g "cosmo/pkg/proto/productv1;productv1" Product + mv $(mkfile_dir)/testdata/service.proto $(mkfile_dir)/product.proto + rm -f $(mkfile_dir)/testdata/service.proto.lock.json .PHONY: generate-mapping-code-local generate-mapping-code-local: regenerate-proto-local From 5a04f78fbd6ceb7ab9e731a3abd0b01e26725455 Mon Sep 17 00:00:00 2001 From: Ludwig Bedacht Date: Tue, 21 Oct 2025 10:43:03 +0200 Subject: [PATCH 16/72] chore: improve retrieving field selections from a selection set --- v2/pkg/ast/ast_selection.go | 7 +++++++ v2/pkg/engine/datasource/grpc_datasource/compiler.go | 2 +- .../datasource/grpc_datasource/execution_plan_visitor.go | 6 +----- 3 files changed, 9 insertions(+), 6 deletions(-) diff --git a/v2/pkg/ast/ast_selection.go b/v2/pkg/ast/ast_selection.go index d67ef9f90..fecd4229d 100644 --- a/v2/pkg/ast/ast_selection.go +++ b/v2/pkg/ast/ast_selection.go @@ -3,6 +3,7 @@ package ast import ( "bytes" "fmt" + "strings" "github.com/wundergraph/graphql-go-tools/v2/pkg/internal/unsafebytes" "github.com/wundergraph/graphql-go-tools/v2/pkg/lexer/position" @@ -241,3 +242,9 @@ func (d *Document) SelectionSetFieldNames(set int) (fieldNames []string) { } return } + +// SelectionSetFieldSetString returns a string of the field names in the selection set separated by a space +// Example: "{ name status }" -> "name status" +func (d *Document) SelectionSetFieldSetString(set int) string { + return strings.Join(d.SelectionSetFieldNames(set), " ") +} diff --git a/v2/pkg/engine/datasource/grpc_datasource/compiler.go b/v2/pkg/engine/datasource/grpc_datasource/compiler.go index 78be8d1f3..4cf52b0f0 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/compiler.go +++ b/v2/pkg/engine/datasource/grpc_datasource/compiler.go @@ -632,7 +632,7 @@ func (p *RPCCompiler) buildProtoMessageWithContext(inputMessage Message, rpcMess } contextList := p.newEmptyListMessageByName(rootMessage, contextSchemaField.Name) - contextData := p.resolveContextData(context[0], contextRPCField) + contextData := p.resolveContextData(context[0], contextRPCField) // TODO handle multiple contexts (resolver requires another resolver) for _, data := range contextData { val := contextList.NewElement() diff --git a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor.go b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor.go index f39d20faa..e4b616d7d 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor.go +++ b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor.go @@ -11,7 +11,6 @@ import ( "github.com/wundergraph/graphql-go-tools/v2/pkg/ast" "github.com/wundergraph/graphql-go-tools/v2/pkg/astvisitor" "github.com/wundergraph/graphql-go-tools/v2/pkg/engine/plan" - "github.com/wundergraph/graphql-go-tools/v2/pkg/internal/unsafebytes" "github.com/wundergraph/graphql-go-tools/v2/pkg/operationreport" ) @@ -201,10 +200,7 @@ func (r *rpcPlanVisitor) EnterSelectionSet(ref int) { // If we are inside of a resolved field that selects multiple fields, we get all the fields from the input and pass them to the required fields visitor. if r.resolvedFieldIndex != ast.InvalidRef { - lbrace := r.operation.SelectionSets[ref].LBrace.CharEnd - rbrace := r.operation.SelectionSets[ref].RBrace.CharStart - 1 - - r.resolvedFields[r.resolvedFieldIndex].requiredFields = unsafebytes.BytesToString(r.operation.Input.RawBytes[lbrace:rbrace]) + r.resolvedFields[r.resolvedFieldIndex].requiredFields = r.operation.SelectionSetFieldSetString(ref) r.walker.SkipNode() return } From 9e7404278a8123ea749919d7db9d99546d6a513e Mon Sep 17 00:00:00 2001 From: Ludwig Bedacht Date: Tue, 21 Oct 2025 15:31:10 +0200 Subject: [PATCH 17/72] chore: add namespace to directive --- .../grpc_datasource/execution_plan.go | 2 +- v2/pkg/grpctest/testdata/products.graphqls | 20 +++++++++---------- 2 files changed, 10 insertions(+), 12 deletions(-) diff --git a/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go b/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go index ae8e8714a..9e6e1583e 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go +++ b/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go @@ -16,7 +16,7 @@ const ( knownTypeOptionalFieldValueName = "value" // resolverContextDirectiveName is the name of the directive that is used to configure the resolver context. - resolverContextDirectiveName = "configureResolver" + resolverContextDirectiveName = "connect__configureResolver" ) // OneOfType represents the type of a oneof field in a protobuf message. diff --git a/v2/pkg/grpctest/testdata/products.graphqls b/v2/pkg/grpctest/testdata/products.graphqls index 8594ee329..3be169054 100644 --- a/v2/pkg/grpctest/testdata/products.graphqls +++ b/v2/pkg/grpctest/testdata/products.graphqls @@ -1,12 +1,10 @@ # Directive for gRPC field resolvers -directive @configureResolver(context: openfed__FieldSet!) on FIELD_DEFINITION - type Product @key(fields: "id") { id: ID! name: String! price: Float! - shippingEstimate(input: ShippingEstimateInput!): Float! @configureResolver(context: "id price") - recommendedCategory(maxPrice: Int!): Category @configureResolver(context: "id name price") + shippingEstimate(input: ShippingEstimateInput!): Float! @connect__configureResolver(context: "id price") + recommendedCategory(maxPrice: Int!): Category @connect__configureResolver(context: "id name price") } type Storage @key(fields: "id") { @@ -21,7 +19,6 @@ type Warehouse @key(fields: "id") { location: String! } - type User { id: ID! name: String! @@ -117,10 +114,10 @@ type Category { id: ID! name: String! kind: CategoryKind! - productCount(filters: ProductCountFilter): Int! @configureResolver(context: "id name") + productCount(filters: ProductCountFilter): Int! @connect__configureResolver(context: "id name") subcategories: [Subcategory!] - popularityScore(threshold: Int): Int @configureResolver(context: "id") - categoryMetrics(metricType: String!): CategoryMetrics @configureResolver(context: "id name") + popularityScore(threshold: Int): Int @connect__configureResolver(context: "id") + categoryMetrics(metricType: String!): CategoryMetrics @connect__configureResolver(context: "id name") } type Subcategory { @@ -128,7 +125,7 @@ type Subcategory { name: String! description: String isActive: Boolean! - itemCount(filters: SubcategoryItemFilter): Int! @configureResolver(context: "id") + itemCount(filters: SubcategoryItemFilter): Int! @connect__configureResolver(context: "id") } type CategoryMetrics { @@ -449,8 +446,9 @@ type Mutation { bulkUpdateBlogPosts(blogPosts: [BlogPostInput!]): [BlogPost!]! } - - union _Entity = Product | Storage | Warehouse scalar _Any scalar openfed__FieldSet + +scalar connect__FieldSet +directive @connect__configureResolver(context: connect__FieldSet!) on FIELD_DEFINITION From dc5b630eb72d483e305c3380fd7a38a19323db07 Mon Sep 17 00:00:00 2001 From: Ludwig Bedacht Date: Wed, 22 Oct 2025 14:00:53 +0200 Subject: [PATCH 18/72] chore: update directive name --- .../datasource/grpc_datasource/execution_plan.go | 2 +- v2/pkg/grpctest/testdata/products.graphqls | 14 +++++++------- 2 files changed, 8 insertions(+), 8 deletions(-) diff --git a/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go b/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go index 9e6e1583e..ed50ed474 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go +++ b/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go @@ -16,7 +16,7 @@ const ( knownTypeOptionalFieldValueName = "value" // resolverContextDirectiveName is the name of the directive that is used to configure the resolver context. - resolverContextDirectiveName = "connect__configureResolver" + resolverContextDirectiveName = "connect__fieldResolver" ) // OneOfType represents the type of a oneof field in a protobuf message. diff --git a/v2/pkg/grpctest/testdata/products.graphqls b/v2/pkg/grpctest/testdata/products.graphqls index 3be169054..d2b6007d5 100644 --- a/v2/pkg/grpctest/testdata/products.graphqls +++ b/v2/pkg/grpctest/testdata/products.graphqls @@ -3,8 +3,8 @@ type Product @key(fields: "id") { id: ID! name: String! price: Float! - shippingEstimate(input: ShippingEstimateInput!): Float! @connect__configureResolver(context: "id price") - recommendedCategory(maxPrice: Int!): Category @connect__configureResolver(context: "id name price") + shippingEstimate(input: ShippingEstimateInput!): Float! @connect__fieldResolver(context: "id price") + recommendedCategory(maxPrice: Int!): Category @connect__fieldResolver(context: "id name price") } type Storage @key(fields: "id") { @@ -114,10 +114,10 @@ type Category { id: ID! name: String! kind: CategoryKind! - productCount(filters: ProductCountFilter): Int! @connect__configureResolver(context: "id name") + productCount(filters: ProductCountFilter): Int! @connect__fieldResolver(context: "id name") subcategories: [Subcategory!] - popularityScore(threshold: Int): Int @connect__configureResolver(context: "id") - categoryMetrics(metricType: String!): CategoryMetrics @connect__configureResolver(context: "id name") + popularityScore(threshold: Int): Int @connect__fieldResolver(context: "id") + categoryMetrics(metricType: String!): CategoryMetrics @connect__fieldResolver(context: "id name") } type Subcategory { @@ -125,7 +125,7 @@ type Subcategory { name: String! description: String isActive: Boolean! - itemCount(filters: SubcategoryItemFilter): Int! @connect__configureResolver(context: "id") + itemCount(filters: SubcategoryItemFilter): Int! @connect__fieldResolver(context: "id") } type CategoryMetrics { @@ -451,4 +451,4 @@ scalar _Any scalar openfed__FieldSet scalar connect__FieldSet -directive @connect__configureResolver(context: connect__FieldSet!) on FIELD_DEFINITION +directive @connect__fieldResolver(context: connect__FieldSet!) on FIELD_DEFINITION From af8ea7c988784dff4d21118990bf8062799a2005 Mon Sep 17 00:00:00 2001 From: Ludwig Bedacht Date: Wed, 22 Oct 2025 14:27:04 +0200 Subject: [PATCH 19/72] chore: check for exact length --- v2/pkg/engine/datasource/grpc_datasource/json_builder.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/v2/pkg/engine/datasource/grpc_datasource/json_builder.go b/v2/pkg/engine/datasource/grpc_datasource/json_builder.go index e6133e320..dc286c22b 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/json_builder.go +++ b/v2/pkg/engine/datasource/grpc_datasource/json_builder.go @@ -246,8 +246,8 @@ func (j *jsonBuilder) mergeWithPath(base *astjson.Value, resolved *astjson.Value responseValues = append(responseValues, values...) } - if len(resolvedValues) < len(responseValues) { - return fmt.Errorf("length of values doesn't match the length of the result array, expected %d, got %d", len(responseValues), len(resolvedValues)) + if len(resolvedValues) != len(responseValues) { + return fmt.Errorf("length of values doesn't match the length of the result array, expected %d, got %d", len(resolvedValues), len(responseValues)) } for i := range responseValues { From cf72a8d8917e987e1887e38671a47fd9a70508d4 Mon Sep 17 00:00:00 2001 From: Ludwig Bedacht Date: Wed, 22 Oct 2025 14:31:11 +0200 Subject: [PATCH 20/72] chore: do not swallow error from marshalling --- v2/pkg/engine/datasource/grpc_datasource/grpc_datasource.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/v2/pkg/engine/datasource/grpc_datasource/grpc_datasource.go b/v2/pkg/engine/datasource/grpc_datasource/grpc_datasource.go index 02a0a1246..b63060f22 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/grpc_datasource.go +++ b/v2/pkg/engine/datasource/grpc_datasource/grpc_datasource.go @@ -125,7 +125,7 @@ func (d *DataSource) Load(ctx context.Context, input []byte, out *bytes.Buffer) response, err := builder.marshalResponseJSON(&a, &serviceCall.RPC.Response, serviceCall.Output) if err != nil { - return nil + return err } // In case of a federated response, we need to ensure that the response is valid. From 329ef61c75c522387b644eede75818af6e4391e4 Mon Sep 17 00:00:00 2001 From: Ludwig Bedacht Date: Wed, 22 Oct 2025 14:47:38 +0200 Subject: [PATCH 21/72] chore: add comments --- .../datasource/grpc_datasource/fetch.go | 33 ++++++++++++++++++- 1 file changed, 32 insertions(+), 1 deletion(-) diff --git a/v2/pkg/engine/datasource/grpc_datasource/fetch.go b/v2/pkg/engine/datasource/grpc_datasource/fetch.go index 74f0c012e..7c0751735 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/fetch.go +++ b/v2/pkg/engine/datasource/grpc_datasource/fetch.go @@ -5,6 +5,8 @@ import ( "sync" ) +// FetchItem is a single fetch item in the execution plan. +// It contains the call information, the service call, and a list of references to the dependent fetches. type FetchItem struct { ID int Plan *RPCCall @@ -12,12 +14,21 @@ type FetchItem struct { DependentFetches []int } +// DependencyGraph is a graph of the calls in the execution plan. +// It is used to determine the order in which to execute the calls. type DependencyGraph struct { mu sync.Mutex fetches []FetchItem // nodes is a list of lists of dependent calls. // Each node index corresponds to a call index in the execution plan // and the list contains the corresponding dependent calls indices. + // Visual representation of the nodes: + // 0 -> [] // no dependencies + // 1 -> [0] // depends on 0 + // 2 -> [0] // depends on 0 + // 3 -> [0] // depends on 0 + // 4 -> [0, 2] // depends on 0 and 2 + // 5 -> [0, 2, 3] // depends on 0, 2 and 3 nodes [][]int } @@ -27,6 +38,8 @@ func NewDependencyGraph(executionPlan *RPCExecutionPlan) *DependencyGraph { fetches: make([]FetchItem, len(executionPlan.Calls)), } + // Initialize the graph with the calls in the execution plan. + // We create a FetchItem for each call and store the dependent call references. for index, call := range executionPlan.Calls { graph.nodes[index] = call.DependentCalls graph.fetches[index] = FetchItem{ @@ -40,6 +53,11 @@ func NewDependencyGraph(executionPlan *RPCExecutionPlan) *DependencyGraph { return graph } +// TopologicalSortResolve sorts the calls in the execution plan in a topological order. +// In order to perform calls in the correct order, we need to determine the dependencies between the calls. +// We are using a depth-first search to determine the dependencies between the calls by +// building an index map of the call hierarchy. Each index in the index map corresponds to a call index in the execution plan. +// The map contains the level of the call in the hierarchy. The root call has a level of 0. func (g *DependencyGraph) TopologicalSortResolve(resolver func(nodes []FetchItem) error) error { // We are using a slice to store the batch index for each noded ordered. callHierarchyRefs := initializeSlice(len(g.nodes), -1) @@ -51,19 +69,24 @@ func (g *DependencyGraph) TopologicalSortResolve(resolver func(nodes []FetchItem return fmt.Errorf("cycle detected") } + // We are marking the call as visited to avoid cycles. cycleChecks[index] = true if len(g.nodes[index]) == 0 { + // If the call has no dependencies, we are setting the level to 0 and return early. callHierarchyRefs[index] = 0 return nil } currentLevel := 0 + // We are iterating over the dependent calls of the current call. for _, depCallIndex := range g.nodes[index] { if depCallIndex < 0 || depCallIndex >= len(g.nodes) { return fmt.Errorf("unable to find dependent call %d in execution plan", depCallIndex) } + // If the dependent call has already been visited, we are checking if the level of the dependent call is greater than the current level. + // If it is, we are updating the current level to the level of the dependent call. if depLevel := callHierarchyRefs[depCallIndex]; depLevel >= 0 { if depLevel > currentLevel { currentLevel = depLevel @@ -71,15 +94,18 @@ func (g *DependencyGraph) TopologicalSortResolve(resolver func(nodes []FetchItem continue } + // If the dependent call has not been visited, we are visiting it. if err := visit(depCallIndex); err != nil { return err } + // If the level of the dependent call is greater than the current level, we are updating the current level to the level of the dependent call. if l := callHierarchyRefs[depCallIndex]; l > currentLevel { currentLevel = l } } + // After receiving the maximum level of the dependent calls, we increment the level by 1. callHierarchyRefs[index] = currentLevel + 1 return nil } @@ -92,12 +118,13 @@ func (g *DependencyGraph) TopologicalSortResolve(resolver func(nodes []FetchItem clear(cycleChecks) } + // After setting up the call hierarchy, we are grouping the calls by their level. chunks := make(map[int][]FetchItem) - for callIndex, chunkIndex := range callHierarchyRefs { chunks[chunkIndex] = append(chunks[chunkIndex], g.fetches[callIndex]) } + // We are iterating over the chunks and resolving the calls in the correct order. for i := 0; i < len(chunks); i++ { if err := resolver(chunks[i]); err != nil { return err @@ -107,6 +134,7 @@ func (g *DependencyGraph) TopologicalSortResolve(resolver func(nodes []FetchItem return nil } +// Fetch returns the fetch item for a given index. func (g *DependencyGraph) Fetch(index int) (FetchItem, error) { if index < 0 || index >= len(g.fetches) { return FetchItem{}, fmt.Errorf("unable to find fetch %d in execution plan", index) @@ -115,6 +143,7 @@ func (g *DependencyGraph) Fetch(index int) (FetchItem, error) { return g.fetches[index], nil } +// FetchDependencies returns the dependencies for a given fetch item. func (g *DependencyGraph) FetchDependencies(fetch *FetchItem) ([]FetchItem, error) { dependencies := make([]FetchItem, 0, len(fetch.DependentFetches)) @@ -129,12 +158,14 @@ func (g *DependencyGraph) FetchDependencies(fetch *FetchItem) ([]FetchItem, erro return dependencies, nil } +// SetFetchData sets the service call for a given index. func (g *DependencyGraph) SetFetchData(index int, serviceCall *ServiceCall) { g.mu.Lock() g.fetches[index].ServiceCall = serviceCall g.mu.Unlock() } +// initializeSlice initializes a slice with a given length and a given value. func initializeSlice[T any](len int, zero T) []T { s := make([]T, len) for i := range s { From 5021522d28aaf46cada0c968fff7b7dc0b3c20c4 Mon Sep 17 00:00:00 2001 From: Ludwig Bedacht Date: Wed, 22 Oct 2025 17:37:59 +0200 Subject: [PATCH 22/72] chore: improvements --- .../datasource/grpc_datasource/compiler.go | 27 ++++++++++------- .../grpc_datasource/execution_plan.go | 20 ++++++------- .../grpc_datasource/execution_plan_test.go | 29 ++++++++++++++----- .../datasource/grpc_datasource/fetch.go | 4 --- 4 files changed, 48 insertions(+), 32 deletions(-) diff --git a/v2/pkg/engine/datasource/grpc_datasource/compiler.go b/v2/pkg/engine/datasource/grpc_datasource/compiler.go index 4cf52b0f0..34dbef626 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/compiler.go +++ b/v2/pkg/engine/datasource/grpc_datasource/compiler.go @@ -7,13 +7,13 @@ import ( "strings" "github.com/bufbuild/protocompile" - "github.com/cespare/xxhash/v2" "github.com/tidwall/gjson" protoref "google.golang.org/protobuf/reflect/protoreflect" "google.golang.org/protobuf/types/dynamicpb" "github.com/wundergraph/graphql-go-tools/v2/pkg/ast" "github.com/wundergraph/graphql-go-tools/v2/pkg/operationreport" + "github.com/wundergraph/graphql-go-tools/v2/pkg/pool" ) const ( @@ -123,8 +123,11 @@ type Document struct { // newNode creates a new node in the document. func (d *Document) newNode(ref int, name string, kind NodeKind) { - h := xxhash.Sum64String(name) - d.nodes[h] = node{ + digest := pool.Hash64.Get() + defer pool.Hash64.Put(digest) + _, _ = digest.WriteString(name) + + d.nodes[digest.Sum64()] = node{ ref: ref, kind: kind, } @@ -133,8 +136,11 @@ func (d *Document) newNode(ref int, name string, kind NodeKind) { // nodeByName returns a node by its name. // Returns false if the node does not exist. func (d *Document) nodeByName(name string) (node, bool) { - h := xxhash.Sum64String(name) - node, exists := d.nodes[h] + digest := pool.Hash64.Get() + defer pool.Hash64.Put(digest) + _, _ = digest.WriteString(name) + + node, exists := d.nodes[digest.Sum64()] return node, exists } @@ -833,21 +839,22 @@ func (p *RPCCompiler) resolveUnderlyingListItems(value protoref.Value, nestingLe } itemsList := itemsValue.List() - if itemsList.Len() == 0 { + itemsListLen := itemsList.Len() + if itemsListLen == 0 { return nil } if nestingLevel > 1 { - items := make([]protoref.Value, 0, itemsList.Len()) - for i := 0; i < itemsList.Len(); i++ { + items := make([]protoref.Value, 0, itemsListLen) + for i := 0; i < itemsListLen; i++ { items = append(items, p.resolveUnderlyingListItems(itemsList.Get(i), nestingLevel-1)...) } return items } - result := make([]protoref.Value, itemsList.Len()) - for i := 0; i < itemsList.Len(); i++ { + result := make([]protoref.Value, itemsListLen) + for i := 0; i < itemsListLen; i++ { result[i] = itemsList.Get(i) } diff --git a/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go b/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go index ed50ed474..ed3124b2b 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go +++ b/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go @@ -923,8 +923,8 @@ func (r *rpcPlanningContext) createResolverRPCCalls(subgraphName string, resolve return nil, err } - contextMessage.Fields = make(RPCFields, 0, len(resolvedField.contextFields)) - for _, contextField := range resolvedField.contextFields { + contextMessage.Fields = make(RPCFields, len(resolvedField.contextFields)) + for i := range resolvedField.contextFields { typeDefNode, found := r.definition.NodeByNameStr(r.definition.ResolveTypeNameString(resolvedField.parentTypeRef)) if !found { return nil, fmt.Errorf("type definition node not found for type: %s", r.definition.ResolveTypeNameString(resolvedField.parentTypeRef)) @@ -932,29 +932,29 @@ func (r *rpcPlanningContext) createResolverRPCCalls(subgraphName string, resolve field, err := r.buildField( typeDefNode, - contextField.fieldRef, - r.definition.FieldDefinitionNameString(contextField.fieldRef), + resolvedField.contextFields[i].fieldRef, + r.definition.FieldDefinitionNameString(resolvedField.contextFields[i].fieldRef), "", ) - field.ResolvePath = contextField.resolvePath + field.ResolvePath = resolvedField.contextFields[i].resolvePath if err != nil { return nil, err } - contextMessage.Fields = append(contextMessage.Fields, field) + contextMessage.Fields[i] = field } - fieldArgsMessage.Fields = make(RPCFields, 0, len(resolvedField.fieldArguments)) - for _, fieldArgument := range resolvedField.fieldArguments { - field, err := r.createRPCFieldFromFieldArgument(fieldArgument) + fieldArgsMessage.Fields = make(RPCFields, len(resolvedField.fieldArguments)) + for i := range resolvedField.fieldArguments { + field, err := r.createRPCFieldFromFieldArgument(resolvedField.fieldArguments[i]) if err != nil { return nil, err } - fieldArgsMessage.Fields = append(fieldArgsMessage.Fields, field) + fieldArgsMessage.Fields[i] = field } calls = append(calls, call) diff --git a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_test.go b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_test.go index 385e5d445..d02860805 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_test.go +++ b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_test.go @@ -50,16 +50,29 @@ func runTest(t *testing.T, testCase testCase) { } } +// buildPath builds a path from a string which is a dot-separated list of field names. func buildPath(path string) ast.Path { - pathElements := strings.Split(path, ".") - pathItems := make([]ast.PathItem, 0, len(pathElements)) - for _, element := range pathElements { - pathItems = append(pathItems, ast.PathItem{ - Kind: ast.FieldName, - FieldName: []byte(element), - }) + b := make([]byte, len(path)) + copy(b, path) + n := 1 + for i := 0; i < len(b); i++ { + if b[i] == '.' { + n++ + } + } + items := make([]ast.PathItem, n) + start, seg := 0, 0 + for i := 0; i <= len(b); i++ { + if i == len(b) || b[i] == '.' { + items[seg] = ast.PathItem{ + Kind: ast.FieldName, + FieldName: b[start:i], + } + seg++ + start = i + 1 + } } - return pathItems + return items } func TestQueryExecutionPlans(t *testing.T) { diff --git a/v2/pkg/engine/datasource/grpc_datasource/fetch.go b/v2/pkg/engine/datasource/grpc_datasource/fetch.go index 7c0751735..b241af2b2 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/fetch.go +++ b/v2/pkg/engine/datasource/grpc_datasource/fetch.go @@ -2,7 +2,6 @@ package grpcdatasource import ( "fmt" - "sync" ) // FetchItem is a single fetch item in the execution plan. @@ -17,7 +16,6 @@ type FetchItem struct { // DependencyGraph is a graph of the calls in the execution plan. // It is used to determine the order in which to execute the calls. type DependencyGraph struct { - mu sync.Mutex fetches []FetchItem // nodes is a list of lists of dependent calls. // Each node index corresponds to a call index in the execution plan @@ -160,9 +158,7 @@ func (g *DependencyGraph) FetchDependencies(fetch *FetchItem) ([]FetchItem, erro // SetFetchData sets the service call for a given index. func (g *DependencyGraph) SetFetchData(index int, serviceCall *ServiceCall) { - g.mu.Lock() g.fetches[index].ServiceCall = serviceCall - g.mu.Unlock() } // initializeSlice initializes a slice with a given length and a given value. From 39bba4bff47a41bfa2eb422b45aaa72313c211a6 Mon Sep 17 00:00:00 2001 From: Ludwig Bedacht Date: Thu, 23 Oct 2025 10:35:44 +0200 Subject: [PATCH 23/72] chore: remove unnecessary muxes --- go.work.sum | 2 ++ .../datasource/grpc_datasource/grpc_datasource.go | 4 ---- .../grpc_datasource/grpc_datasource_test.go | 11 +++-------- .../engine/datasource/grpc_datasource/json_builder.go | 6 +++++- 4 files changed, 10 insertions(+), 13 deletions(-) diff --git a/go.work.sum b/go.work.sum index f13214003..d66f874bf 100644 --- a/go.work.sum +++ b/go.work.sum @@ -305,6 +305,7 @@ golang.org/x/mod v0.12.0/go.mod h1:iBbtSCu2XBx23ZKBPSOrRkjjQPZFPuis4dIYUhu/chs= golang.org/x/mod v0.15.0/go.mod h1:hTbmBsO62+eylJbnUtE2MGJUyE7QWk4xUqPFrRgJ+7c= golang.org/x/mod v0.17.0/go.mod h1:hTbmBsO62+eylJbnUtE2MGJUyE7QWk4xUqPFrRgJ+7c= golang.org/x/mod v0.25.0/go.mod h1:IXM97Txy2VM4PJ3gI61r1YEk/gAj6zAHN3AdZt6S9Ww= +golang.org/x/mod v0.28.0/go.mod h1:yfB/L0NOf/kmEbXjzCPOx1iK1fRutOydrCMsqRhEBxI= golang.org/x/net v0.6.0/go.mod h1:2Tu9+aMcznHK/AK1HMvgo6xiTLG5rD5rZLDS+rp2Bjs= golang.org/x/net v0.10.0/go.mod h1:0qNGK6F8kojg2nk9dLZ2mShWaEBan6FAoqfSigmmuDg= golang.org/x/net v0.15.0/go.mod h1:idbUs1IY1+zTqbi8yxTbhexhEEk5ur9LInksu6HrEpk= @@ -357,6 +358,7 @@ golang.org/x/tools v0.6.0/go.mod h1:Xwgl3UAJ/d3gWutnCtw505GrjyAbvKui8lOU390QaIU= golang.org/x/tools v0.13.0/go.mod h1:HvlwmtVNQAhOuCjW7xxvovg8wbNq7LwfXh/k7wXUl58= golang.org/x/tools v0.21.1-0.20240508182429-e35e4ccd0d2d/go.mod h1:aiJjzUbINMkxbQROHiO6hDPo2LHcIPhhQsa9DLh0yGk= golang.org/x/tools v0.34.0/go.mod h1:pAP9OwEaY1CAW3HOmg3hLZC5Z0CCmzjAF2UQMSqNARg= +golang.org/x/tools v0.37.0/go.mod h1:MBN5QPQtLMHVdvsbtarmTNukZDdgwdwlO5qGacAzF0w= golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20200804184101-5ec99f83aff1 h1:go1bK/D/BFZV2I8cIQd1NKEZ+0owSTG1fDTci4IqFcE= gonum.org/v1/plot v0.10.1 h1:dnifSs43YJuNMDzB7v8wV64O4ABBHReuAVAoBxqBqS4= diff --git a/v2/pkg/engine/datasource/grpc_datasource/grpc_datasource.go b/v2/pkg/engine/datasource/grpc_datasource/grpc_datasource.go index b63060f22..afee61fce 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/grpc_datasource.go +++ b/v2/pkg/engine/datasource/grpc_datasource/grpc_datasource.go @@ -10,7 +10,6 @@ import ( "bytes" "context" "fmt" - "sync" "github.com/tidwall/gjson" "golang.org/x/sync/errgroup" @@ -109,7 +108,6 @@ func (d *DataSource) Load(ctx context.Context, input []byte, out *bytes.Buffer) results := make([]resultData, len(serviceCalls)) errGrp, errGrpCtx := errgroup.WithContext(ctx) - mu := sync.Mutex{} // make gRPC calls for index, serviceCall := range serviceCalls { @@ -137,13 +135,11 @@ func (d *DataSource) Load(ctx context.Context, input []byte, out *bytes.Buffer) } } - mu.Lock() results[index] = resultData{ kind: serviceCall.RPC.Kind, response: response, responsePath: serviceCall.RPC.ResponsePath, } - mu.Unlock() return nil }) diff --git a/v2/pkg/engine/datasource/grpc_datasource/grpc_datasource_test.go b/v2/pkg/engine/datasource/grpc_datasource/grpc_datasource_test.go index a69bd5f52..44bd9dc51 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/grpc_datasource_test.go +++ b/v2/pkg/engine/datasource/grpc_datasource/grpc_datasource_test.go @@ -84,18 +84,18 @@ func Benchmark_DataSource_Load_WithFieldArguments(b *testing.B) { b.ReportAllocs() b.ResetTimer() + mapping := testMapping() for b.Loop() { ds, err := NewDataSource(conn, DataSourceConfig{ Operation: &queryDoc, Definition: &schemaDoc, SubgraphName: "Products", Compiler: compiler, - Mapping: testMapping(), + Mapping: mapping, }) require.NoError(b, err) - output := new(bytes.Buffer) - err = ds.Load(context.Background(), []byte(`{"query":"`+query+`","body":`+variables+`}`), output) + err = ds.Load(context.Background(), []byte(`{"query":"`+query+`","body":`+variables+`}`), new(bytes.Buffer)) require.NoError(b, err) } } @@ -3932,11 +3932,6 @@ func Test_Datasource_Load_WithFieldResolvers(t *testing.T) { err = ds.Load(context.Background(), []byte(input), output) require.NoError(t, err) - pretty := new(bytes.Buffer) - err = json.Indent(pretty, output.Bytes(), "", " ") - require.NoError(t, err) - fmt.Println(pretty.String()) - // Parse the response var resp graphqlResponse diff --git a/v2/pkg/engine/datasource/grpc_datasource/json_builder.go b/v2/pkg/engine/datasource/grpc_datasource/json_builder.go index dc286c22b..0dc12bc8f 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/json_builder.go +++ b/v2/pkg/engine/datasource/grpc_datasource/json_builder.go @@ -221,6 +221,10 @@ func (j *jsonBuilder) mergeEntities(left *astjson.Value, right *astjson.Value) ( } func (j *jsonBuilder) mergeWithPath(base *astjson.Value, resolved *astjson.Value, path ast.Path) error { + if len(path) == 0 { + return errors.New("path is empty") + } + resolvedValues := resolved.GetArray(resolveResponsePath) searchPath := path[:len(path)-1] @@ -509,7 +513,7 @@ func (j *jsonBuilder) traverseList(level int, arena *astjson.Arena, current *ast return arena.NewNull(), nil } - return arena.NewArray(), fmt.Errorf("cannot add null item to response for non nullable list") + return arena.NewArray(), errors.New("cannot add null item to response for non nullable list") } // The actual list is always at field number 1 in the wrapper From 9bae24935e164c1f9ffe2bee7ac8ed436e043cf9 Mon Sep 17 00:00:00 2001 From: Ludwig Bedacht Date: Thu, 23 Oct 2025 11:28:45 +0200 Subject: [PATCH 24/72] chore: use proper type instead of copying --- .../datasource/grpc_datasource/compiler.go | 2 +- .../grpc_datasource/compiler_test.go | 124 +- .../grpc_datasource/execution_plan.go | 85 +- .../execution_plan_composite_test.go | 460 ++--- .../execution_plan_federation_test.go | 684 ++++---- .../execution_plan_field_resolvers_test.go | 628 +++---- .../execution_plan_list_test.go | 706 ++++---- .../execution_plan_nullable_test.go | 504 +++--- .../grpc_datasource/execution_plan_test.go | 1528 ++++++++--------- .../execution_plan_visitor_federation.go | 18 +- .../grpc_datasource/grpc_datasource_test.go | 37 +- 11 files changed, 2390 insertions(+), 2386 deletions(-) diff --git a/v2/pkg/engine/datasource/grpc_datasource/compiler.go b/v2/pkg/engine/datasource/grpc_datasource/compiler.go index 34dbef626..3beb3090a 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/compiler.go +++ b/v2/pkg/engine/datasource/grpc_datasource/compiler.go @@ -1050,7 +1050,7 @@ func (p *RPCCompiler) traverseList(rootMsg protoref.Message, level int, field Fi itemsField := itemsFieldMsg.Mutable(itemsFieldDesc).List() if level >= rpcField.ListMetadata.NestingLevel { - switch DataType(rpcField.TypeName) { + switch DataType(rpcField.ProtoTypeName) { case DataTypeMessage: itemsFieldMsg, ok := p.doc.MessageByName(rpcField.Message.Name) if !ok { diff --git a/v2/pkg/engine/datasource/grpc_datasource/compiler_test.go b/v2/pkg/engine/datasource/grpc_datasource/compiler_test.go index 231c75728..4815ed1dd 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/compiler_test.go +++ b/v2/pkg/engine/datasource/grpc_datasource/compiler_test.go @@ -209,23 +209,23 @@ func TestBuildProtoMessage(t *testing.T) { Name: "LookupProductByIdRequest", Fields: []RPCField{ { - Name: "inputs", - TypeName: string(DataTypeMessage), - Repeated: true, - JSONPath: "representations", // Path to extract data from GraphQL variables + Name: "inputs", + ProtoTypeName: DataTypeMessage, + Repeated: true, + JSONPath: "representations", // Path to extract data from GraphQL variables Message: &RPCMessage{ Name: "LookupProductByIdInput", Fields: []RPCField{ { - Name: "key", - TypeName: string(DataTypeMessage), + Name: "key", + ProtoTypeName: DataTypeMessage, Message: &RPCMessage{ Name: "ProductByIdKey", Fields: []RPCField{ { - Name: "id", - TypeName: string(DataTypeString), - JSONPath: "id", // Extract 'id' from each representation + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", // Extract 'id' from each representation }, }, }, @@ -240,33 +240,33 @@ func TestBuildProtoMessage(t *testing.T) { Name: "LookupProductByIdResponse", Fields: []RPCField{ { - Name: "results", - TypeName: string(DataTypeMessage), - Repeated: true, - JSONPath: "results", + Name: "results", + ProtoTypeName: DataTypeMessage, + Repeated: true, + JSONPath: "results", Message: &RPCMessage{ Name: "LookupProductByIdResult", Fields: []RPCField{ { - Name: "product", - TypeName: string(DataTypeMessage), + Name: "product", + ProtoTypeName: DataTypeMessage, Message: &RPCMessage{ Name: "Product", Fields: []RPCField{ { - Name: "id", - TypeName: string(DataTypeString), - JSONPath: "id", + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", }, { - Name: "name", - TypeName: string(DataTypeString), - JSONPath: "name", + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", }, { - Name: "price", - TypeName: string(DataTypeDouble), - JSONPath: "price", + Name: "price", + ProtoTypeName: DataTypeDouble, + JSONPath: "price", }, }, }, @@ -307,47 +307,47 @@ func TestCompileNestedLists(t *testing.T) { Name: "QueryCalculateTotalsRequest", Fields: []RPCField{ { - Name: "orders", - TypeName: string(DataTypeMessage), - JSONPath: "orders", - Repeated: true, + Name: "orders", + ProtoTypeName: DataTypeMessage, + JSONPath: "orders", + Repeated: true, Message: &RPCMessage{ Name: "OrderInput", Fields: []RPCField{ { - Name: "order_id", - TypeName: string(DataTypeString), - JSONPath: "orderId", + Name: "order_id", + ProtoTypeName: DataTypeString, + JSONPath: "orderId", }, { - Name: "customer_name", - TypeName: string(DataTypeString), - JSONPath: "customerName", + Name: "customer_name", + ProtoTypeName: DataTypeString, + JSONPath: "customerName", }, { - Name: "lines", - TypeName: string(DataTypeMessage), - JSONPath: "lines", - Repeated: true, + Name: "lines", + ProtoTypeName: DataTypeMessage, + JSONPath: "lines", + Repeated: true, Message: &RPCMessage{ Name: "OrderLineInput", Fields: []RPCField{ { - Name: "product_id", - TypeName: string(DataTypeString), - JSONPath: "productId", + Name: "product_id", + ProtoTypeName: DataTypeString, + JSONPath: "productId", }, { - Name: "quantity", - TypeName: string(DataTypeInt32), - JSONPath: "quantity", + Name: "quantity", + ProtoTypeName: DataTypeInt32, + JSONPath: "quantity", }, { - Name: "modifiers", - TypeName: string(DataTypeString), - JSONPath: "modifiers", - Optional: true, - IsListType: true, + Name: "modifiers", + ProtoTypeName: DataTypeString, + JSONPath: "modifiers", + Optional: true, + IsListType: true, ListMetadata: &ListMetadata{ NestingLevel: 1, LevelInfo: []LevelInfo{ @@ -369,27 +369,27 @@ func TestCompileNestedLists(t *testing.T) { Name: "QueryCalculateTotalsResponse", Fields: []RPCField{ { - Name: "calculate_totals", - TypeName: string(DataTypeMessage), - JSONPath: "calculateTotals", - Repeated: true, + Name: "calculate_totals", + ProtoTypeName: DataTypeMessage, + JSONPath: "calculateTotals", + Repeated: true, Message: &RPCMessage{ Name: "Order", Fields: []RPCField{ { - Name: "order_id", - TypeName: string(DataTypeString), - JSONPath: "orderId", + Name: "order_id", + ProtoTypeName: DataTypeString, + JSONPath: "orderId", }, { - Name: "customer_name", - TypeName: string(DataTypeString), - JSONPath: "customerName", + Name: "customer_name", + ProtoTypeName: DataTypeString, + JSONPath: "customerName", }, { - Name: "total_items", - TypeName: string(DataTypeInt32), - JSONPath: "totalItems", + Name: "total_items", + ProtoTypeName: DataTypeInt32, + JSONPath: "totalItems", }, }, }, diff --git a/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go b/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go index ed3124b2b..2d6cb2431 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go +++ b/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go @@ -17,6 +17,9 @@ const ( // resolverContextDirectiveName is the name of the directive that is used to configure the resolver context. resolverContextDirectiveName = "connect__fieldResolver" + + // typenameFieldName is the name of the field that is used to store the typename of the object. + typenameFieldName = "__typename" ) // OneOfType represents the type of a oneof field in a protobuf message. @@ -127,15 +130,15 @@ func (r *RPCMessage) SelectValidTypes(typeName string) []string { } func (r *RPCMessage) AppendTypeNameField(typeName string) { - if r.Fields != nil && r.Fields.Exists("__typename", "") { + if r.Fields != nil && r.Fields.Exists(typenameFieldName, "") { return } r.Fields = append(r.Fields, RPCField{ - Name: "__typename", - TypeName: DataTypeString.String(), - StaticValue: typeName, - JSONPath: "__typename", + Name: typenameFieldName, + ProtoTypeName: DataTypeString, + StaticValue: typeName, + JSONPath: typenameFieldName, }) } @@ -187,8 +190,8 @@ type RPCField struct { Repeated bool // Name is the name of the field as defined in the protobuf message Name string - // TypeName is the name of the type of the field in the protobuf definition - TypeName string + // ProtoTypeName is the name of the type of the field in the protobuf definition + ProtoTypeName DataType // JSONPath either holds the path to the variable definition for the request message, // or defines the name of the response field in the message. JSONPath string @@ -234,11 +237,11 @@ func (r *RPCField) ToOptionalTypeMessage(protoName string) *RPCMessage { Name: protoName, Fields: RPCFields{ RPCField{ - Name: knownTypeOptionalFieldValueName, - JSONPath: r.JSONPath, - TypeName: r.TypeName, - Repeated: r.Repeated, - EnumName: r.EnumName, + Name: knownTypeOptionalFieldValueName, + JSONPath: r.JSONPath, + ProtoTypeName: r.ProtoTypeName, + Repeated: r.Repeated, + EnumName: r.EnumName, }, }, } @@ -255,7 +258,7 @@ func (r *RPCField) AliasOrPath() string { // IsOptionalScalar checks if the field is an optional scalar value. func (r *RPCField) IsOptionalScalar() bool { - return r.Optional && r.TypeName != string(DataTypeMessage) + return r.Optional && r.ProtoTypeName != DataTypeMessage } // RPCFields is a list of RPCFields that provides helper methods @@ -356,7 +359,7 @@ func formatRPCMessage(sb *strings.Builder, message RPCMessage, indent int) { for _, field := range message.Fields { fmt.Fprintf(sb, "%s - Name: %s\n", indentStr, field.Name) - fmt.Fprintf(sb, "%s TypeName: %s\n", indentStr, field.TypeName) + fmt.Fprintf(sb, "%s TypeName: %s\n", indentStr, field.ProtoTypeName) fmt.Fprintf(sb, "%s Repeated: %v\n", indentStr, field.Repeated) fmt.Fprintf(sb, "%s JSONPath: %s\n", indentStr, field.JSONPath) @@ -529,11 +532,11 @@ func (r *rpcPlanningContext) buildField(enclosingTypeNode ast.Node, fd int, fiel parentTypeName := enclosingTypeNode.NameString(r.definition) field := RPCField{ - Name: r.resolveFieldMapping(parentTypeName, fieldName), - Alias: fieldAlias, - Optional: !r.definition.TypeIsNonNull(fdt), - JSONPath: fieldName, - TypeName: typeName.String(), + Name: r.resolveFieldMapping(parentTypeName, fieldName), + Alias: fieldAlias, + Optional: !r.definition.TypeIsNonNull(fdt), + JSONPath: fieldName, + ProtoTypeName: typeName, } if r.definition.TypeIsList(fdt) { @@ -557,7 +560,7 @@ func (r *rpcPlanningContext) buildField(enclosingTypeNode ast.Node, fd int, fiel field.EnumName = r.definition.FieldDefinitionTypeNameString(fd) } - if fieldName == "__typename" { + if fieldName == typenameFieldName { field.StaticValue = parentTypeName } @@ -673,10 +676,10 @@ func (r *rpcPlanningContext) buildMessageFieldFromInputValueDefinition(ivdRef in func (r *rpcPlanningContext) buildInputMessageField(typeRef int, fieldName, jsonPath string, dt DataType) (RPCField, error) { field := RPCField{ - Name: fieldName, - Optional: !r.definition.TypeIsNonNull(typeRef), - TypeName: dt.String(), - JSONPath: jsonPath, + Name: fieldName, + Optional: !r.definition.TypeIsNonNull(typeRef), + ProtoTypeName: dt, + JSONPath: jsonPath, } if r.definition.TypeIsList(typeRef) { @@ -984,19 +987,19 @@ func (r *rpcPlanningContext) newResolveRPCCall(config resolveRPCCallConfig) (RPC Name: resolveConfig.Response, Fields: RPCFields{ { - Name: "result", - TypeName: string(DataTypeMessage), - JSONPath: "result", - Repeated: true, + Name: "result", + ProtoTypeName: DataTypeMessage, + JSONPath: "result", + Repeated: true, Message: &RPCMessage{ Name: resolveConfig.RPC + "Result", Fields: RPCFields{ { - Name: resolveConfig.FieldMappingData.TargetName, - TypeName: string(dataType), - JSONPath: config.fieldName, - Message: responseFieldsMessage, - Optional: !r.definition.TypeIsNonNull(resolvedField.fieldDefinitionTypeRef), + Name: resolveConfig.FieldMappingData.TargetName, + ProtoTypeName: dataType, + JSONPath: config.fieldName, + Message: responseFieldsMessage, + Optional: !r.definition.TypeIsNonNull(resolvedField.fieldDefinitionTypeRef), }, }, }, @@ -1014,16 +1017,16 @@ func (r *rpcPlanningContext) newResolveRPCCall(config resolveRPCCallConfig) (RPC Name: resolveConfig.Request, Fields: RPCFields{ { - Name: "context", - TypeName: string(DataTypeMessage), - Repeated: true, - Message: config.contextMessage, + Name: "context", + ProtoTypeName: DataTypeMessage, + Repeated: true, + Message: config.contextMessage, }, { - Name: "field_args", - TypeName: string(DataTypeMessage), - JSONPath: "", - Message: config.fieldArgsMessage, + Name: "field_args", + ProtoTypeName: DataTypeMessage, + JSONPath: "", + Message: config.fieldArgsMessage, }, }, }, diff --git a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_composite_test.go b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_composite_test.go index 45eec9b4a..1066d4700 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_composite_test.go +++ b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_composite_test.go @@ -33,9 +33,9 @@ func TestCompositeTypeExecutionPlan(t *testing.T) { Name: "QueryRandomPetResponse", Fields: []RPCField{ { - Name: "random_pet", - TypeName: string(DataTypeMessage), - JSONPath: "randomPet", + Name: "random_pet", + ProtoTypeName: DataTypeMessage, + JSONPath: "randomPet", Message: &RPCMessage{ Name: "Animal", OneOfType: OneOfTypeInterface, @@ -46,27 +46,27 @@ func TestCompositeTypeExecutionPlan(t *testing.T) { FieldSelectionSet: RPCFieldSelectionSet{ "Cat": { { - Name: "meow_volume", - TypeName: string(DataTypeInt32), - JSONPath: "meowVolume", + Name: "meow_volume", + ProtoTypeName: DataTypeInt32, + JSONPath: "meowVolume", }, }, }, Fields: []RPCField{ { - Name: "id", - TypeName: string(DataTypeString), - JSONPath: "id", + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", }, { - Name: "name", - TypeName: string(DataTypeString), - JSONPath: "name", + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", }, { - Name: "kind", - TypeName: string(DataTypeString), - JSONPath: "kind", + Name: "kind", + ProtoTypeName: DataTypeString, + JSONPath: "kind", }, }, }, @@ -92,9 +92,9 @@ func TestCompositeTypeExecutionPlan(t *testing.T) { Name: "QueryRandomPetResponse", Fields: []RPCField{ { - Name: "random_pet", - TypeName: string(DataTypeMessage), - JSONPath: "randomPet", + Name: "random_pet", + ProtoTypeName: DataTypeMessage, + JSONPath: "randomPet", Message: &RPCMessage{ Name: "Animal", OneOfType: OneOfTypeInterface, @@ -105,34 +105,34 @@ func TestCompositeTypeExecutionPlan(t *testing.T) { FieldSelectionSet: RPCFieldSelectionSet{ "Cat": { { - Name: "meow_volume", - TypeName: string(DataTypeInt32), - JSONPath: "meowVolume", + Name: "meow_volume", + ProtoTypeName: DataTypeInt32, + JSONPath: "meowVolume", }, }, "Dog": { { - Name: "bark_volume", - TypeName: string(DataTypeInt32), - JSONPath: "barkVolume", + Name: "bark_volume", + ProtoTypeName: DataTypeInt32, + JSONPath: "barkVolume", }, }, }, Fields: []RPCField{ { - Name: "id", - TypeName: string(DataTypeString), - JSONPath: "id", + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", }, { - Name: "name", - TypeName: string(DataTypeString), - JSONPath: "name", + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", }, { - Name: "kind", - TypeName: string(DataTypeString), - JSONPath: "kind", + Name: "kind", + ProtoTypeName: DataTypeString, + JSONPath: "kind", }, }, }, @@ -158,10 +158,10 @@ func TestCompositeTypeExecutionPlan(t *testing.T) { Name: "QueryAllPetsResponse", Fields: []RPCField{ { - Name: "all_pets", - TypeName: string(DataTypeMessage), - JSONPath: "allPets", - Repeated: true, + Name: "all_pets", + ProtoTypeName: DataTypeMessage, + JSONPath: "allPets", + Repeated: true, Message: &RPCMessage{ Name: "Animal", OneOfType: OneOfTypeInterface, @@ -172,34 +172,34 @@ func TestCompositeTypeExecutionPlan(t *testing.T) { FieldSelectionSet: RPCFieldSelectionSet{ "Cat": { { - Name: "meow_volume", - TypeName: string(DataTypeInt32), - JSONPath: "meowVolume", + Name: "meow_volume", + ProtoTypeName: DataTypeInt32, + JSONPath: "meowVolume", }, }, "Dog": { { - Name: "bark_volume", - TypeName: string(DataTypeInt32), - JSONPath: "barkVolume", + Name: "bark_volume", + ProtoTypeName: DataTypeInt32, + JSONPath: "barkVolume", }, }, }, Fields: []RPCField{ { - Name: "id", - TypeName: string(DataTypeString), - JSONPath: "id", + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", }, { - Name: "name", - TypeName: string(DataTypeString), - JSONPath: "name", + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", }, { - Name: "kind", - TypeName: string(DataTypeString), - JSONPath: "kind", + Name: "kind", + ProtoTypeName: DataTypeString, + JSONPath: "kind", }, }, }, @@ -225,10 +225,10 @@ func TestCompositeTypeExecutionPlan(t *testing.T) { Name: "QueryAllPetsResponse", Fields: []RPCField{ { - Name: "all_pets", - TypeName: string(DataTypeMessage), - JSONPath: "allPets", - Repeated: true, + Name: "all_pets", + ProtoTypeName: DataTypeMessage, + JSONPath: "allPets", + Repeated: true, Message: &RPCMessage{ Name: "Animal", OneOfType: OneOfTypeInterface, @@ -240,19 +240,19 @@ func TestCompositeTypeExecutionPlan(t *testing.T) { FieldSelectionSet: RPCFieldSelectionSet{ "Animal": { { - Name: "id", - TypeName: string(DataTypeString), - JSONPath: "id", + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", }, { - Name: "name", - TypeName: string(DataTypeString), - JSONPath: "name", + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", }, { - Name: "kind", - TypeName: string(DataTypeString), - JSONPath: "kind", + Name: "kind", + ProtoTypeName: DataTypeString, + JSONPath: "kind", }, }, }, @@ -279,9 +279,9 @@ func TestCompositeTypeExecutionPlan(t *testing.T) { Name: "QueryRandomPetResponse", Fields: []RPCField{ { - Name: "random_pet", - TypeName: string(DataTypeMessage), - JSONPath: "randomPet", + Name: "random_pet", + ProtoTypeName: DataTypeMessage, + JSONPath: "randomPet", Message: &RPCMessage{ Name: "Animal", OneOfType: OneOfTypeInterface, @@ -291,19 +291,19 @@ func TestCompositeTypeExecutionPlan(t *testing.T) { }, Fields: []RPCField{ { - Name: "id", - TypeName: string(DataTypeString), - JSONPath: "id", + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", }, { - Name: "name", - TypeName: string(DataTypeString), - JSONPath: "name", + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", }, { - Name: "kind", - TypeName: string(DataTypeString), - JSONPath: "kind", + Name: "kind", + ProtoTypeName: DataTypeString, + JSONPath: "kind", }, }, }, @@ -326,22 +326,22 @@ func TestCompositeTypeExecutionPlan(t *testing.T) { Name: "QuerySearchRequest", Fields: []RPCField{ { - Name: "input", - TypeName: string(DataTypeMessage), - JSONPath: "input", + Name: "input", + ProtoTypeName: DataTypeMessage, + JSONPath: "input", Message: &RPCMessage{ Name: "SearchInput", Fields: []RPCField{ { - Name: "query", - TypeName: string(DataTypeString), - JSONPath: "query", + Name: "query", + ProtoTypeName: DataTypeString, + JSONPath: "query", }, { - Name: "limit", - TypeName: string(DataTypeInt32), - JSONPath: "limit", - Optional: true, + Name: "limit", + ProtoTypeName: DataTypeInt32, + JSONPath: "limit", + Optional: true, }, }, }, @@ -352,10 +352,10 @@ func TestCompositeTypeExecutionPlan(t *testing.T) { Name: "QuerySearchResponse", Fields: []RPCField{ { - Name: "search", - TypeName: string(DataTypeMessage), - JSONPath: "search", - Repeated: true, + Name: "search", + ProtoTypeName: DataTypeMessage, + JSONPath: "search", + Repeated: true, Message: &RPCMessage{ Name: "SearchResult", OneOfType: OneOfTypeUnion, @@ -368,49 +368,49 @@ func TestCompositeTypeExecutionPlan(t *testing.T) { FieldSelectionSet: RPCFieldSelectionSet{ "Product": { { - Name: "id", - TypeName: string(DataTypeString), - JSONPath: "id", + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", }, { - Name: "name", - TypeName: string(DataTypeString), - JSONPath: "name", + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", }, { - Name: "price", - TypeName: string(DataTypeDouble), - JSONPath: "price", + Name: "price", + ProtoTypeName: DataTypeDouble, + JSONPath: "price", }, }, "User": { { - Name: "id", - TypeName: string(DataTypeString), - JSONPath: "id", + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", }, { - Name: "name", - TypeName: string(DataTypeString), - JSONPath: "name", + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", }, }, "Category": { { - Name: "id", - TypeName: string(DataTypeString), - JSONPath: "id", + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", }, { - Name: "name", - TypeName: string(DataTypeString), - JSONPath: "name", + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", }, { - Name: "kind", - TypeName: string(DataTypeEnum), - JSONPath: "kind", - EnumName: "CategoryKind", + Name: "kind", + ProtoTypeName: DataTypeEnum, + JSONPath: "kind", + EnumName: "CategoryKind", }, }, }, @@ -437,9 +437,9 @@ func TestCompositeTypeExecutionPlan(t *testing.T) { Name: "QueryRandomSearchResultResponse", Fields: []RPCField{ { - Name: "random_search_result", - TypeName: string(DataTypeMessage), - JSONPath: "randomSearchResult", + Name: "random_search_result", + ProtoTypeName: DataTypeMessage, + JSONPath: "randomSearchResult", Message: &RPCMessage{ Name: "SearchResult", OneOfType: OneOfTypeUnion, @@ -452,49 +452,49 @@ func TestCompositeTypeExecutionPlan(t *testing.T) { FieldSelectionSet: RPCFieldSelectionSet{ "Product": { { - Name: "id", - TypeName: string(DataTypeString), - JSONPath: "id", + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", }, { - Name: "name", - TypeName: string(DataTypeString), - JSONPath: "name", + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", }, { - Name: "price", - TypeName: string(DataTypeDouble), - JSONPath: "price", + Name: "price", + ProtoTypeName: DataTypeDouble, + JSONPath: "price", }, }, "User": { { - Name: "id", - TypeName: string(DataTypeString), - JSONPath: "id", + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", }, { - Name: "name", - TypeName: string(DataTypeString), - JSONPath: "name", + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", }, }, "Category": { { - Name: "id", - TypeName: string(DataTypeString), - JSONPath: "id", + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", }, { - Name: "name", - TypeName: string(DataTypeString), - JSONPath: "name", + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", }, { - Name: "kind", - TypeName: string(DataTypeEnum), - JSONPath: "kind", - EnumName: "CategoryKind", + Name: "kind", + ProtoTypeName: DataTypeEnum, + JSONPath: "kind", + EnumName: "CategoryKind", }, }, }, @@ -518,22 +518,22 @@ func TestCompositeTypeExecutionPlan(t *testing.T) { Name: "QuerySearchRequest", Fields: []RPCField{ { - Name: "input", - TypeName: string(DataTypeMessage), - JSONPath: "input", + Name: "input", + ProtoTypeName: DataTypeMessage, + JSONPath: "input", Message: &RPCMessage{ Name: "SearchInput", Fields: []RPCField{ { - Name: "query", - TypeName: string(DataTypeString), - JSONPath: "query", + Name: "query", + ProtoTypeName: DataTypeString, + JSONPath: "query", }, { - Name: "limit", - TypeName: string(DataTypeInt32), - JSONPath: "limit", - Optional: true, + Name: "limit", + ProtoTypeName: DataTypeInt32, + JSONPath: "limit", + Optional: true, }, }, }, @@ -544,10 +544,10 @@ func TestCompositeTypeExecutionPlan(t *testing.T) { Name: "QuerySearchResponse", Fields: []RPCField{ { - Name: "search", - TypeName: string(DataTypeMessage), - JSONPath: "search", - Repeated: true, + Name: "search", + ProtoTypeName: DataTypeMessage, + JSONPath: "search", + Repeated: true, Message: &RPCMessage{ Name: "SearchResult", OneOfType: OneOfTypeUnion, @@ -560,26 +560,26 @@ func TestCompositeTypeExecutionPlan(t *testing.T) { FieldSelectionSet: RPCFieldSelectionSet{ "Product": { { - Name: "id", - TypeName: string(DataTypeString), - JSONPath: "id", + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", }, { - Name: "name", - TypeName: string(DataTypeString), - JSONPath: "name", + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", }, }, "User": { { - Name: "id", - TypeName: string(DataTypeString), - JSONPath: "id", + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", }, { - Name: "name", - TypeName: string(DataTypeString), - JSONPath: "name", + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", }, }, }, @@ -647,21 +647,21 @@ func TestMutationUnionExecutionPlan(t *testing.T) { Name: "MutationPerformActionRequest", Fields: []RPCField{ { - Name: "input", - TypeName: string(DataTypeMessage), - JSONPath: "input", + Name: "input", + ProtoTypeName: DataTypeMessage, + JSONPath: "input", Message: &RPCMessage{ Name: "ActionInput", Fields: []RPCField{ { - Name: "type", - TypeName: string(DataTypeString), - JSONPath: "type", + Name: "type", + ProtoTypeName: DataTypeString, + JSONPath: "type", }, { - Name: "payload", - TypeName: string(DataTypeString), - JSONPath: "payload", + Name: "payload", + ProtoTypeName: DataTypeString, + JSONPath: "payload", }, }, }, @@ -672,9 +672,9 @@ func TestMutationUnionExecutionPlan(t *testing.T) { Name: "MutationPerformActionResponse", Fields: []RPCField{ { - Name: "perform_action", - TypeName: string(DataTypeMessage), - JSONPath: "performAction", + Name: "perform_action", + ProtoTypeName: DataTypeMessage, + JSONPath: "performAction", Message: &RPCMessage{ Name: "ActionResult", OneOfType: OneOfTypeUnion, @@ -686,26 +686,26 @@ func TestMutationUnionExecutionPlan(t *testing.T) { FieldSelectionSet: RPCFieldSelectionSet{ "ActionSuccess": { { - Name: "message", - TypeName: string(DataTypeString), - JSONPath: "message", + Name: "message", + ProtoTypeName: DataTypeString, + JSONPath: "message", }, { - Name: "timestamp", - TypeName: string(DataTypeString), - JSONPath: "timestamp", + Name: "timestamp", + ProtoTypeName: DataTypeString, + JSONPath: "timestamp", }, }, "ActionError": { { - Name: "message", - TypeName: string(DataTypeString), - JSONPath: "message", + Name: "message", + ProtoTypeName: DataTypeString, + JSONPath: "message", }, { - Name: "code", - TypeName: string(DataTypeString), - JSONPath: "code", + Name: "code", + ProtoTypeName: DataTypeString, + JSONPath: "code", }, }, }, @@ -729,21 +729,21 @@ func TestMutationUnionExecutionPlan(t *testing.T) { Name: "MutationPerformActionRequest", Fields: []RPCField{ { - Name: "input", - TypeName: string(DataTypeMessage), - JSONPath: "input", + Name: "input", + ProtoTypeName: DataTypeMessage, + JSONPath: "input", Message: &RPCMessage{ Name: "ActionInput", Fields: []RPCField{ { - Name: "type", - TypeName: string(DataTypeString), - JSONPath: "type", + Name: "type", + ProtoTypeName: DataTypeString, + JSONPath: "type", }, { - Name: "payload", - TypeName: string(DataTypeString), - JSONPath: "payload", + Name: "payload", + ProtoTypeName: DataTypeString, + JSONPath: "payload", }, }, }, @@ -754,9 +754,9 @@ func TestMutationUnionExecutionPlan(t *testing.T) { Name: "MutationPerformActionResponse", Fields: []RPCField{ { - Name: "perform_action", - TypeName: string(DataTypeMessage), - JSONPath: "performAction", + Name: "perform_action", + ProtoTypeName: DataTypeMessage, + JSONPath: "performAction", Message: &RPCMessage{ Name: "ActionResult", OneOfType: OneOfTypeUnion, @@ -768,14 +768,14 @@ func TestMutationUnionExecutionPlan(t *testing.T) { FieldSelectionSet: RPCFieldSelectionSet{ "ActionSuccess": { { - Name: "message", - TypeName: string(DataTypeString), - JSONPath: "message", + Name: "message", + ProtoTypeName: DataTypeString, + JSONPath: "message", }, { - Name: "timestamp", - TypeName: string(DataTypeString), - JSONPath: "timestamp", + Name: "timestamp", + ProtoTypeName: DataTypeString, + JSONPath: "timestamp", }, }, }, @@ -799,21 +799,21 @@ func TestMutationUnionExecutionPlan(t *testing.T) { Name: "MutationPerformActionRequest", Fields: []RPCField{ { - Name: "input", - TypeName: string(DataTypeMessage), - JSONPath: "input", + Name: "input", + ProtoTypeName: DataTypeMessage, + JSONPath: "input", Message: &RPCMessage{ Name: "ActionInput", Fields: []RPCField{ { - Name: "type", - TypeName: string(DataTypeString), - JSONPath: "type", + Name: "type", + ProtoTypeName: DataTypeString, + JSONPath: "type", }, { - Name: "payload", - TypeName: string(DataTypeString), - JSONPath: "payload", + Name: "payload", + ProtoTypeName: DataTypeString, + JSONPath: "payload", }, }, }, @@ -824,9 +824,9 @@ func TestMutationUnionExecutionPlan(t *testing.T) { Name: "MutationPerformActionResponse", Fields: []RPCField{ { - Name: "perform_action", - TypeName: string(DataTypeMessage), - JSONPath: "performAction", + Name: "perform_action", + ProtoTypeName: DataTypeMessage, + JSONPath: "performAction", Message: &RPCMessage{ Name: "ActionResult", OneOfType: OneOfTypeUnion, @@ -838,14 +838,14 @@ func TestMutationUnionExecutionPlan(t *testing.T) { FieldSelectionSet: RPCFieldSelectionSet{ "ActionError": { { - Name: "message", - TypeName: string(DataTypeString), - JSONPath: "message", + Name: "message", + ProtoTypeName: DataTypeString, + JSONPath: "message", }, { - Name: "code", - TypeName: string(DataTypeString), - JSONPath: "code", + Name: "code", + ProtoTypeName: DataTypeString, + JSONPath: "code", }, }, }, diff --git a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_federation_test.go b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_federation_test.go index a0fd7ce2d..06f0f0e76 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_federation_test.go +++ b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_federation_test.go @@ -60,18 +60,18 @@ func TestEntityLookup(t *testing.T) { Name: "LookupProductByIdRequest", Fields: []RPCField{ { - Name: "keys", - TypeName: string(DataTypeMessage), - Repeated: true, - JSONPath: "representations", + Name: "keys", + ProtoTypeName: DataTypeMessage, + Repeated: true, + JSONPath: "representations", Message: &RPCMessage{ Name: "LookupProductByIdKey", MemberTypes: []string{"Product"}, Fields: []RPCField{ { - Name: "id", - TypeName: string(DataTypeString), - JSONPath: "id", + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", }, }, }, @@ -83,33 +83,33 @@ func TestEntityLookup(t *testing.T) { Name: "LookupProductByIdResponse", Fields: []RPCField{ { - Name: "result", - TypeName: string(DataTypeMessage), - Repeated: true, - JSONPath: "_entities", + Name: "result", + ProtoTypeName: DataTypeMessage, + Repeated: true, + JSONPath: "_entities", Message: &RPCMessage{ Name: "Product", Fields: []RPCField{ { - Name: "__typename", - TypeName: string(DataTypeString), - JSONPath: "__typename", - StaticValue: "Product", + Name: "__typename", + ProtoTypeName: DataTypeString, + JSONPath: "__typename", + StaticValue: "Product", }, { - Name: "id", - TypeName: string(DataTypeString), - JSONPath: "id", + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", }, { - Name: "name", - TypeName: string(DataTypeString), - JSONPath: "name", + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", }, { - Name: "price", - TypeName: string(DataTypeDouble), - JSONPath: "price", + Name: "price", + ProtoTypeName: DataTypeDouble, + JSONPath: "price", }, }, }, @@ -144,18 +144,18 @@ func TestEntityLookup(t *testing.T) { Name: "LookupProductByIdRequest", Fields: []RPCField{ { - Name: "keys", - TypeName: string(DataTypeMessage), - Repeated: true, - JSONPath: "representations", + Name: "keys", + ProtoTypeName: DataTypeMessage, + Repeated: true, + JSONPath: "representations", Message: &RPCMessage{ Name: "LookupProductByIdKey", MemberTypes: []string{"Product"}, Fields: []RPCField{ { - Name: "id", - TypeName: string(DataTypeString), - JSONPath: "id", + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", }, }, }, @@ -166,34 +166,34 @@ func TestEntityLookup(t *testing.T) { Name: "LookupProductByIdResponse", Fields: []RPCField{ { - Name: "result", - TypeName: string(DataTypeMessage), - Repeated: true, - JSONPath: "_entities", + Name: "result", + ProtoTypeName: DataTypeMessage, + Repeated: true, + JSONPath: "_entities", Message: &RPCMessage{ Name: "Product", Fields: []RPCField{ { - Name: "__typename", - TypeName: string(DataTypeString), - JSONPath: "__typename", - StaticValue: "Product", + Name: "__typename", + ProtoTypeName: DataTypeString, + JSONPath: "__typename", + StaticValue: "Product", }, { - Name: "id", - TypeName: string(DataTypeString), - JSONPath: "id", + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", }, { - Name: "name", - TypeName: string(DataTypeString), - JSONPath: "name", + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", }, { - Name: "price", - TypeName: string(DataTypeDouble), - JSONPath: "price", + Name: "price", + ProtoTypeName: DataTypeDouble, + JSONPath: "price", }, }, }, @@ -209,18 +209,18 @@ func TestEntityLookup(t *testing.T) { Name: "LookupStorageByIdRequest", Fields: []RPCField{ { - Name: "keys", - TypeName: string(DataTypeMessage), - Repeated: true, - JSONPath: "representations", + Name: "keys", + ProtoTypeName: DataTypeMessage, + Repeated: true, + JSONPath: "representations", Message: &RPCMessage{ Name: "LookupStorageByIdKey", MemberTypes: []string{"Storage"}, Fields: []RPCField{ { - Name: "id", - TypeName: string(DataTypeString), - JSONPath: "id", + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", }, }, }, @@ -231,33 +231,33 @@ func TestEntityLookup(t *testing.T) { Name: "LookupStorageByIdResponse", Fields: []RPCField{ { - Name: "result", - TypeName: string(DataTypeMessage), - Repeated: true, - JSONPath: "_entities", + Name: "result", + ProtoTypeName: DataTypeMessage, + Repeated: true, + JSONPath: "_entities", Message: &RPCMessage{ Name: "Storage", Fields: []RPCField{ { - Name: "__typename", - TypeName: string(DataTypeString), - JSONPath: "__typename", - StaticValue: "Storage", + Name: "__typename", + ProtoTypeName: DataTypeString, + JSONPath: "__typename", + StaticValue: "Storage", }, { - Name: "id", - TypeName: string(DataTypeString), - JSONPath: "id", + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", }, { - Name: "name", - TypeName: string(DataTypeString), - JSONPath: "name", + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", }, { - Name: "location", - TypeName: string(DataTypeString), - JSONPath: "location", + Name: "location", + ProtoTypeName: DataTypeString, + JSONPath: "location", }, }, }, @@ -351,18 +351,18 @@ func TestEntityKeys(t *testing.T) { Name: "LookupUserByIdRequest", Fields: []RPCField{ { - Name: "keys", - TypeName: string(DataTypeMessage), - Repeated: true, - JSONPath: "representations", + Name: "keys", + ProtoTypeName: DataTypeMessage, + Repeated: true, + JSONPath: "representations", Message: &RPCMessage{ Name: "LookupUserByIdKey", MemberTypes: []string{"User"}, Fields: []RPCField{ { - Name: "id", - TypeName: string(DataTypeString), - JSONPath: "id", + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", }, }, }, @@ -374,28 +374,28 @@ func TestEntityKeys(t *testing.T) { Name: "LookupUserByIdResponse", Fields: []RPCField{ { - Name: "result", - TypeName: string(DataTypeMessage), - Repeated: true, - JSONPath: "_entities", + Name: "result", + ProtoTypeName: DataTypeMessage, + Repeated: true, + JSONPath: "_entities", Message: &RPCMessage{ Name: "User", Fields: []RPCField{ { - Name: "__typename", - TypeName: string(DataTypeString), - JSONPath: "__typename", - StaticValue: "User", + Name: "__typename", + ProtoTypeName: DataTypeString, + JSONPath: "__typename", + StaticValue: "User", }, { - Name: "id", - TypeName: string(DataTypeString), - JSONPath: "id", + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", }, { - Name: "name", - TypeName: string(DataTypeString), - JSONPath: "name", + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", }, }, }, @@ -461,30 +461,30 @@ func TestEntityKeys(t *testing.T) { Name: "LookupUserByIdAndAddressRequest", Fields: []RPCField{ { - Name: "keys", - TypeName: string(DataTypeMessage), - Repeated: true, - JSONPath: "representations", + Name: "keys", + ProtoTypeName: DataTypeMessage, + Repeated: true, + JSONPath: "representations", Message: &RPCMessage{ Name: "LookupUserByIdAndAddressKey", MemberTypes: []string{"User"}, Fields: []RPCField{ { - Name: "id", - TypeName: string(DataTypeString), - JSONPath: "id", + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", }, { - Name: "address", - TypeName: string(DataTypeMessage), - JSONPath: "address", + Name: "address", + ProtoTypeName: DataTypeMessage, + JSONPath: "address", Message: &RPCMessage{ Name: "Address", Fields: []RPCField{ { - Name: "id", - TypeName: string(DataTypeString), - JSONPath: "id", + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", }, }, }, @@ -499,28 +499,28 @@ func TestEntityKeys(t *testing.T) { Name: "LookupUserByIdAndAddressResponse", Fields: []RPCField{ { - Name: "result", - TypeName: string(DataTypeMessage), - Repeated: true, - JSONPath: "_entities", + Name: "result", + ProtoTypeName: DataTypeMessage, + Repeated: true, + JSONPath: "_entities", Message: &RPCMessage{ Name: "User", Fields: []RPCField{ { - Name: "__typename", - TypeName: string(DataTypeString), - JSONPath: "__typename", - StaticValue: "User", + Name: "__typename", + ProtoTypeName: DataTypeString, + JSONPath: "__typename", + StaticValue: "User", }, { - Name: "id", - TypeName: string(DataTypeString), - JSONPath: "id", + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", }, { - Name: "name", - TypeName: string(DataTypeString), - JSONPath: "name", + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", }, }, }, @@ -574,23 +574,23 @@ func TestEntityKeys(t *testing.T) { Name: "LookupUserByIdAndNameRequest", Fields: []RPCField{ { - Name: "keys", - TypeName: string(DataTypeMessage), - Repeated: true, - JSONPath: "representations", + Name: "keys", + ProtoTypeName: DataTypeMessage, + Repeated: true, + JSONPath: "representations", Message: &RPCMessage{ Name: "LookupUserByIdAndNameKey", MemberTypes: []string{"User"}, Fields: []RPCField{ { - Name: "id", - TypeName: string(DataTypeString), - JSONPath: "id", + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", }, { - Name: "name", - TypeName: string(DataTypeString), - JSONPath: "name", + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", }, }, }, @@ -601,28 +601,28 @@ func TestEntityKeys(t *testing.T) { Name: "LookupUserByIdAndNameResponse", Fields: []RPCField{ { - Name: "result", - TypeName: string(DataTypeMessage), - Repeated: true, - JSONPath: "_entities", + Name: "result", + ProtoTypeName: DataTypeMessage, + Repeated: true, + JSONPath: "_entities", Message: &RPCMessage{ Name: "User", Fields: []RPCField{ { - Name: "__typename", - TypeName: string(DataTypeString), - JSONPath: "__typename", - StaticValue: "User", + Name: "__typename", + ProtoTypeName: DataTypeString, + JSONPath: "__typename", + StaticValue: "User", }, { - Name: "id", - TypeName: string(DataTypeString), - JSONPath: "id", + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", }, { - Name: "name", - TypeName: string(DataTypeString), - JSONPath: "name", + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", }, }, }, @@ -676,23 +676,23 @@ func TestEntityKeys(t *testing.T) { Name: "LookupUserByIdAndNameRequest", Fields: []RPCField{ { - Name: "keys", - TypeName: string(DataTypeMessage), - Repeated: true, - JSONPath: "representations", + Name: "keys", + ProtoTypeName: DataTypeMessage, + Repeated: true, + JSONPath: "representations", Message: &RPCMessage{ Name: "LookupUserByIdAndNameKey", MemberTypes: []string{"User"}, Fields: []RPCField{ { - Name: "id", - TypeName: string(DataTypeString), - JSONPath: "id", + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", }, { - Name: "name", - TypeName: string(DataTypeString), - JSONPath: "name", + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", }, }, }, @@ -703,28 +703,28 @@ func TestEntityKeys(t *testing.T) { Name: "LookupUserByIdAndNameResponse", Fields: []RPCField{ { - Name: "result", - TypeName: string(DataTypeMessage), - Repeated: true, - JSONPath: "_entities", + Name: "result", + ProtoTypeName: DataTypeMessage, + Repeated: true, + JSONPath: "_entities", Message: &RPCMessage{ Name: "User", Fields: []RPCField{ { - Name: "__typename", - TypeName: string(DataTypeString), - JSONPath: "__typename", - StaticValue: "User", + Name: "__typename", + ProtoTypeName: DataTypeString, + JSONPath: "__typename", + StaticValue: "User", }, { - Name: "id", - TypeName: string(DataTypeString), - JSONPath: "id", + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", }, { - Name: "name", - TypeName: string(DataTypeString), - JSONPath: "name", + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", }, }, }, @@ -784,35 +784,35 @@ func TestEntityKeys(t *testing.T) { Name: "LookupUserByIdAndNameAndAddressRequest", Fields: []RPCField{ { - Name: "keys", - TypeName: string(DataTypeMessage), - Repeated: true, - JSONPath: "representations", + Name: "keys", + ProtoTypeName: DataTypeMessage, + Repeated: true, + JSONPath: "representations", Message: &RPCMessage{ Name: "LookupUserByIdAndNameAndAddressKey", MemberTypes: []string{"User"}, Fields: []RPCField{ { - Name: "id", - TypeName: string(DataTypeString), - JSONPath: "id", + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", }, { - Name: "name", - TypeName: string(DataTypeString), - JSONPath: "name", + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", }, { - Name: "address", - TypeName: string(DataTypeMessage), - JSONPath: "address", + Name: "address", + ProtoTypeName: DataTypeMessage, + JSONPath: "address", Message: &RPCMessage{ Name: "Address", Fields: []RPCField{ { - Name: "id", - TypeName: string(DataTypeString), - JSONPath: "id", + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", }, }, }, @@ -826,28 +826,28 @@ func TestEntityKeys(t *testing.T) { Name: "LookupUserByIdAndNameAndAddressResponse", Fields: []RPCField{ { - Name: "result", - TypeName: string(DataTypeMessage), - Repeated: true, - JSONPath: "_entities", + Name: "result", + ProtoTypeName: DataTypeMessage, + Repeated: true, + JSONPath: "_entities", Message: &RPCMessage{ Name: "User", Fields: []RPCField{ { - Name: "__typename", - TypeName: string(DataTypeString), - JSONPath: "__typename", - StaticValue: "User", + Name: "__typename", + ProtoTypeName: DataTypeString, + JSONPath: "__typename", + StaticValue: "User", }, { - Name: "id", - TypeName: string(DataTypeString), - JSONPath: "id", + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", }, { - Name: "name", - TypeName: string(DataTypeString), - JSONPath: "name", + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", }, }, }, @@ -895,18 +895,18 @@ func TestEntityLookupWithFieldResolvers(t *testing.T) { Name: "LookupProductByIdRequest", Fields: []RPCField{ { - Name: "keys", - TypeName: string(DataTypeMessage), - Repeated: true, - JSONPath: "representations", + Name: "keys", + ProtoTypeName: DataTypeMessage, + Repeated: true, + JSONPath: "representations", Message: &RPCMessage{ Name: "LookupProductByIdKey", MemberTypes: []string{"Product"}, Fields: []RPCField{ { - Name: "id", - TypeName: string(DataTypeString), - JSONPath: "id", + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", }, }, }, @@ -917,34 +917,34 @@ func TestEntityLookupWithFieldResolvers(t *testing.T) { Name: "LookupProductByIdResponse", Fields: []RPCField{ { - Name: "result", - TypeName: string(DataTypeMessage), - Repeated: true, - JSONPath: "_entities", + Name: "result", + ProtoTypeName: DataTypeMessage, + Repeated: true, + JSONPath: "_entities", Message: &RPCMessage{ Name: "Product", Fields: []RPCField{ { - Name: "__typename", - TypeName: string(DataTypeString), - JSONPath: "__typename", - StaticValue: "Product", + Name: "__typename", + ProtoTypeName: DataTypeString, + JSONPath: "__typename", + StaticValue: "Product", }, { - Name: "id", - TypeName: string(DataTypeString), - JSONPath: "id", + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", }, { - Name: "name", - TypeName: string(DataTypeString), - JSONPath: "name", + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", }, { - Name: "price", - TypeName: string(DataTypeDouble), - JSONPath: "price", + Name: "price", + ProtoTypeName: DataTypeDouble, + JSONPath: "price", }, }, }, @@ -962,58 +962,58 @@ func TestEntityLookupWithFieldResolvers(t *testing.T) { Name: "ResolveProductShippingEstimateRequest", Fields: []RPCField{ { - Name: "context", - TypeName: string(DataTypeMessage), - JSONPath: "", - Repeated: true, + Name: "context", + ProtoTypeName: DataTypeMessage, + JSONPath: "", + Repeated: true, Message: &RPCMessage{ Name: "ResolveProductShippingEstimateContext", Fields: []RPCField{ { - Name: "id", - TypeName: string(DataTypeString), - JSONPath: "id", - ResolvePath: buildPath("result.id"), + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", + ResolvePath: buildPath("result.id"), }, { - Name: "price", - TypeName: string(DataTypeDouble), - JSONPath: "price", - ResolvePath: buildPath("result.price"), + Name: "price", + ProtoTypeName: DataTypeDouble, + JSONPath: "price", + ResolvePath: buildPath("result.price"), }, }, }, }, { - Name: "field_args", - TypeName: string(DataTypeMessage), - JSONPath: "", + Name: "field_args", + ProtoTypeName: DataTypeMessage, + JSONPath: "", Message: &RPCMessage{ Name: "ResolveProductShippingEstimateArgs", Fields: []RPCField{ { - Name: "input", - TypeName: string(DataTypeMessage), - JSONPath: "input", + Name: "input", + ProtoTypeName: DataTypeMessage, + JSONPath: "input", Message: &RPCMessage{ Name: "ShippingEstimateInput", Fields: []RPCField{ { - Name: "destination", - TypeName: string(DataTypeEnum), - JSONPath: "destination", - EnumName: "ShippingDestination", + Name: "destination", + ProtoTypeName: DataTypeEnum, + JSONPath: "destination", + EnumName: "ShippingDestination", }, { - Name: "weight", - TypeName: string(DataTypeDouble), - JSONPath: "weight", + Name: "weight", + ProtoTypeName: DataTypeDouble, + JSONPath: "weight", }, { - Name: "expedited", - TypeName: string(DataTypeBool), - JSONPath: "expedited", - Optional: true, + Name: "expedited", + ProtoTypeName: DataTypeBool, + JSONPath: "expedited", + Optional: true, }, }, }, @@ -1027,17 +1027,17 @@ func TestEntityLookupWithFieldResolvers(t *testing.T) { Name: "ResolveProductShippingEstimateResponse", Fields: []RPCField{ { - Name: "result", - TypeName: string(DataTypeMessage), - JSONPath: "result", - Repeated: true, + Name: "result", + ProtoTypeName: DataTypeMessage, + JSONPath: "result", + Repeated: true, Message: &RPCMessage{ Name: "ResolveProductShippingEstimateResult", Fields: []RPCField{ { - Name: "shipping_estimate", - TypeName: string(DataTypeDouble), - JSONPath: "shippingEstimate", + Name: "shipping_estimate", + ProtoTypeName: DataTypeDouble, + JSONPath: "shippingEstimate", }, }, }, @@ -1072,18 +1072,18 @@ func TestEntityLookupWithFieldResolvers(t *testing.T) { Name: "LookupStorageByIdRequest", Fields: []RPCField{ { - Name: "keys", - TypeName: string(DataTypeMessage), - Repeated: true, - JSONPath: "representations", + Name: "keys", + ProtoTypeName: DataTypeMessage, + Repeated: true, + JSONPath: "representations", Message: &RPCMessage{ Name: "LookupStorageByIdKey", MemberTypes: []string{"Storage"}, Fields: []RPCField{ { - Name: "id", - TypeName: string(DataTypeString), - JSONPath: "id", + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", }, }, }, @@ -1094,33 +1094,33 @@ func TestEntityLookupWithFieldResolvers(t *testing.T) { Name: "LookupStorageByIdResponse", Fields: []RPCField{ { - Name: "result", - TypeName: string(DataTypeMessage), - Repeated: true, - JSONPath: "_entities", + Name: "result", + ProtoTypeName: DataTypeMessage, + Repeated: true, + JSONPath: "_entities", Message: &RPCMessage{ Name: "Storage", Fields: []RPCField{ { - Name: "__typename", - TypeName: string(DataTypeString), - JSONPath: "__typename", - StaticValue: "Storage", + Name: "__typename", + ProtoTypeName: DataTypeString, + JSONPath: "__typename", + StaticValue: "Storage", }, { - Name: "id", - TypeName: string(DataTypeString), - JSONPath: "id", + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", }, { - Name: "name", - TypeName: string(DataTypeString), - JSONPath: "name", + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", }, { - Name: "location", - TypeName: string(DataTypeString), - JSONPath: "location", + Name: "location", + ProtoTypeName: DataTypeString, + JSONPath: "location", }, }, }, @@ -1136,18 +1136,18 @@ func TestEntityLookupWithFieldResolvers(t *testing.T) { Name: "LookupProductByIdRequest", Fields: []RPCField{ { - Name: "keys", - TypeName: string(DataTypeMessage), - Repeated: true, - JSONPath: "representations", + Name: "keys", + ProtoTypeName: DataTypeMessage, + Repeated: true, + JSONPath: "representations", Message: &RPCMessage{ Name: "LookupProductByIdKey", MemberTypes: []string{"Product"}, Fields: []RPCField{ { - Name: "id", - TypeName: string(DataTypeString), - JSONPath: "id", + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", }, }, }, @@ -1158,33 +1158,33 @@ func TestEntityLookupWithFieldResolvers(t *testing.T) { Name: "LookupProductByIdResponse", Fields: []RPCField{ { - Name: "result", - TypeName: string(DataTypeMessage), - Repeated: true, - JSONPath: "_entities", + Name: "result", + ProtoTypeName: DataTypeMessage, + Repeated: true, + JSONPath: "_entities", Message: &RPCMessage{ Name: "Product", Fields: []RPCField{ { - Name: "__typename", - TypeName: string(DataTypeString), - JSONPath: "__typename", - StaticValue: "Product", + Name: "__typename", + ProtoTypeName: DataTypeString, + JSONPath: "__typename", + StaticValue: "Product", }, { - Name: "id", - TypeName: string(DataTypeString), - JSONPath: "id", + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", }, { - Name: "name", - TypeName: string(DataTypeString), - JSONPath: "name", + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", }, { - Name: "price", - TypeName: string(DataTypeDouble), - JSONPath: "price", + Name: "price", + ProtoTypeName: DataTypeDouble, + JSONPath: "price", }, }, }, @@ -1202,58 +1202,58 @@ func TestEntityLookupWithFieldResolvers(t *testing.T) { Name: "ResolveProductShippingEstimateRequest", Fields: []RPCField{ { - Name: "context", - TypeName: string(DataTypeMessage), - JSONPath: "", - Repeated: true, + Name: "context", + ProtoTypeName: DataTypeMessage, + JSONPath: "", + Repeated: true, Message: &RPCMessage{ Name: "ResolveProductShippingEstimateContext", Fields: []RPCField{ { - Name: "id", - TypeName: string(DataTypeString), - JSONPath: "id", - ResolvePath: buildPath("result.id"), + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", + ResolvePath: buildPath("result.id"), }, { - Name: "price", - TypeName: string(DataTypeDouble), - JSONPath: "price", - ResolvePath: buildPath("result.price"), + Name: "price", + ProtoTypeName: DataTypeDouble, + JSONPath: "price", + ResolvePath: buildPath("result.price"), }, }, }, }, { - Name: "field_args", - TypeName: string(DataTypeMessage), - JSONPath: "", + Name: "field_args", + ProtoTypeName: DataTypeMessage, + JSONPath: "", Message: &RPCMessage{ Name: "ResolveProductShippingEstimateArgs", Fields: []RPCField{ { - Name: "input", - TypeName: string(DataTypeMessage), - JSONPath: "input", + Name: "input", + ProtoTypeName: DataTypeMessage, + JSONPath: "input", Message: &RPCMessage{ Name: "ShippingEstimateInput", Fields: []RPCField{ { - Name: "destination", - TypeName: string(DataTypeEnum), - JSONPath: "destination", - EnumName: "ShippingDestination", + Name: "destination", + ProtoTypeName: DataTypeEnum, + JSONPath: "destination", + EnumName: "ShippingDestination", }, { - Name: "weight", - TypeName: string(DataTypeDouble), - JSONPath: "weight", + Name: "weight", + ProtoTypeName: DataTypeDouble, + JSONPath: "weight", }, { - Name: "expedited", - TypeName: string(DataTypeBool), - JSONPath: "expedited", - Optional: true, + Name: "expedited", + ProtoTypeName: DataTypeBool, + JSONPath: "expedited", + Optional: true, }, }, }, @@ -1267,17 +1267,17 @@ func TestEntityLookupWithFieldResolvers(t *testing.T) { Name: "ResolveProductShippingEstimateResponse", Fields: []RPCField{ { - Name: "result", - TypeName: string(DataTypeMessage), - JSONPath: "result", - Repeated: true, + Name: "result", + ProtoTypeName: DataTypeMessage, + JSONPath: "result", + Repeated: true, Message: &RPCMessage{ Name: "ResolveProductShippingEstimateResult", Fields: []RPCField{ { - Name: "shipping_estimate", - TypeName: string(DataTypeDouble), - JSONPath: "shippingEstimate", + Name: "shipping_estimate", + ProtoTypeName: DataTypeDouble, + JSONPath: "shippingEstimate", }, }, }, diff --git a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_field_resolvers_test.go b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_field_resolvers_test.go index 60f8f48db..a07573ba9 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_field_resolvers_test.go +++ b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_field_resolvers_test.go @@ -27,28 +27,28 @@ func TestExecutionPlanFieldResolvers(t *testing.T) { Name: "QueryCategoriesResponse", Fields: []RPCField{ { - Name: "categories", - TypeName: string(DataTypeMessage), - JSONPath: "categories", - Repeated: true, + Name: "categories", + ProtoTypeName: DataTypeMessage, + JSONPath: "categories", + Repeated: true, Message: &RPCMessage{ Name: "Category", Fields: []RPCField{ { - Name: "id", - TypeName: string(DataTypeString), - JSONPath: "id", + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", }, { - Name: "name", - TypeName: string(DataTypeString), - JSONPath: "name", + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", }, { - Name: "kind", - TypeName: string(DataTypeEnum), - JSONPath: "kind", - EnumName: "CategoryKind", + Name: "kind", + ProtoTypeName: DataTypeEnum, + JSONPath: "kind", + EnumName: "CategoryKind", }, }, }, @@ -66,66 +66,66 @@ func TestExecutionPlanFieldResolvers(t *testing.T) { Name: "ResolveCategoryProductCountRequest", Fields: []RPCField{ { - Name: "context", - TypeName: string(DataTypeMessage), - JSONPath: "", - Repeated: true, + Name: "context", + ProtoTypeName: DataTypeMessage, + JSONPath: "", + Repeated: true, Message: &RPCMessage{ Name: "ResolveCategoryProductCountContext", Fields: []RPCField{ { - Name: "id", - TypeName: string(DataTypeString), - JSONPath: "id", - ResolvePath: buildPath("categories.id"), + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", + ResolvePath: buildPath("categories.id"), }, { - Name: "name", - TypeName: string(DataTypeString), - JSONPath: "name", - ResolvePath: buildPath("categories.name"), + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", + ResolvePath: buildPath("categories.name"), }, }, }, }, { - Name: "field_args", - TypeName: string(DataTypeMessage), - JSONPath: "", + Name: "field_args", + ProtoTypeName: DataTypeMessage, + JSONPath: "", Message: &RPCMessage{ Name: "ResolveCategoryProductCountArgs", Fields: []RPCField{ { - Name: "filters", - TypeName: string(DataTypeMessage), - JSONPath: "whoop", - Optional: true, + Name: "filters", + ProtoTypeName: DataTypeMessage, + JSONPath: "whoop", + Optional: true, Message: &RPCMessage{ Name: "ProductCountFilter", Fields: []RPCField{ { - Name: "min_price", - TypeName: string(DataTypeDouble), - JSONPath: "minPrice", - Optional: true, + Name: "min_price", + ProtoTypeName: DataTypeDouble, + JSONPath: "minPrice", + Optional: true, }, { - Name: "max_price", - TypeName: string(DataTypeDouble), - JSONPath: "maxPrice", - Optional: true, + Name: "max_price", + ProtoTypeName: DataTypeDouble, + JSONPath: "maxPrice", + Optional: true, }, { - Name: "in_stock", - TypeName: string(DataTypeBool), - JSONPath: "inStock", - Optional: true, + Name: "in_stock", + ProtoTypeName: DataTypeBool, + JSONPath: "inStock", + Optional: true, }, { - Name: "search_term", - TypeName: string(DataTypeString), - JSONPath: "searchTerm", - Optional: true, + Name: "search_term", + ProtoTypeName: DataTypeString, + JSONPath: "searchTerm", + Optional: true, }, }, }, @@ -139,17 +139,17 @@ func TestExecutionPlanFieldResolvers(t *testing.T) { Name: "ResolveCategoryProductCountResponse", Fields: []RPCField{ { - Name: "result", - TypeName: string(DataTypeMessage), - JSONPath: "result", - Repeated: true, + Name: "result", + ProtoTypeName: DataTypeMessage, + JSONPath: "result", + Repeated: true, Message: &RPCMessage{ Name: "ResolveCategoryProductCountResult", Fields: []RPCField{ { - Name: "product_count", - TypeName: string(DataTypeInt32), - JSONPath: "productCount", + Name: "product_count", + ProtoTypeName: DataTypeInt32, + JSONPath: "productCount", }, }, }, @@ -175,10 +175,10 @@ func TestExecutionPlanFieldResolvers(t *testing.T) { Name: "QueryCategoriesResponse", Fields: []RPCField{ { - Name: "categories", - TypeName: string(DataTypeMessage), - JSONPath: "categories", - Repeated: true, + Name: "categories", + ProtoTypeName: DataTypeMessage, + JSONPath: "categories", + Repeated: true, Message: &RPCMessage{ Name: "Category", Fields: []RPCField{}, @@ -197,66 +197,66 @@ func TestExecutionPlanFieldResolvers(t *testing.T) { Name: "ResolveCategoryProductCountRequest", Fields: []RPCField{ { - Name: "context", - TypeName: string(DataTypeMessage), - JSONPath: "", - Repeated: true, + Name: "context", + ProtoTypeName: DataTypeMessage, + JSONPath: "", + Repeated: true, Message: &RPCMessage{ Name: "ResolveCategoryProductCountContext", Fields: []RPCField{ { - Name: "id", - TypeName: string(DataTypeString), - JSONPath: "id", - ResolvePath: buildPath("categories.id"), + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", + ResolvePath: buildPath("categories.id"), }, { - Name: "name", - TypeName: string(DataTypeString), - JSONPath: "name", - ResolvePath: buildPath("categories.name"), + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", + ResolvePath: buildPath("categories.name"), }, }, }, }, { - Name: "field_args", - TypeName: string(DataTypeMessage), - JSONPath: "", + Name: "field_args", + ProtoTypeName: DataTypeMessage, + JSONPath: "", Message: &RPCMessage{ Name: "ResolveCategoryProductCountArgs", Fields: []RPCField{ { - Name: "filters", - TypeName: string(DataTypeMessage), - JSONPath: "p1", - Optional: true, + Name: "filters", + ProtoTypeName: DataTypeMessage, + JSONPath: "p1", + Optional: true, Message: &RPCMessage{ Name: "ProductCountFilter", Fields: []RPCField{ { - Name: "min_price", - TypeName: string(DataTypeDouble), - JSONPath: "minPrice", - Optional: true, + Name: "min_price", + ProtoTypeName: DataTypeDouble, + JSONPath: "minPrice", + Optional: true, }, { - Name: "max_price", - TypeName: string(DataTypeDouble), - JSONPath: "maxPrice", - Optional: true, + Name: "max_price", + ProtoTypeName: DataTypeDouble, + JSONPath: "maxPrice", + Optional: true, }, { - Name: "in_stock", - TypeName: string(DataTypeBool), - JSONPath: "inStock", - Optional: true, + Name: "in_stock", + ProtoTypeName: DataTypeBool, + JSONPath: "inStock", + Optional: true, }, { - Name: "search_term", - TypeName: string(DataTypeString), - JSONPath: "searchTerm", - Optional: true, + Name: "search_term", + ProtoTypeName: DataTypeString, + JSONPath: "searchTerm", + Optional: true, }, }, }, @@ -270,17 +270,17 @@ func TestExecutionPlanFieldResolvers(t *testing.T) { Name: "ResolveCategoryProductCountResponse", Fields: []RPCField{ { - Name: "result", - TypeName: string(DataTypeMessage), - JSONPath: "result", - Repeated: true, + Name: "result", + ProtoTypeName: DataTypeMessage, + JSONPath: "result", + Repeated: true, Message: &RPCMessage{ Name: "ResolveCategoryProductCountResult", Fields: []RPCField{ { - Name: "product_count", - TypeName: string(DataTypeInt32), - JSONPath: "productCount1", + Name: "product_count", + ProtoTypeName: DataTypeInt32, + JSONPath: "productCount1", }, }, }, @@ -298,66 +298,66 @@ func TestExecutionPlanFieldResolvers(t *testing.T) { Name: "ResolveCategoryProductCountRequest", Fields: []RPCField{ { - Name: "context", - TypeName: string(DataTypeMessage), - JSONPath: "", - Repeated: true, + Name: "context", + ProtoTypeName: DataTypeMessage, + JSONPath: "", + Repeated: true, Message: &RPCMessage{ Name: "ResolveCategoryProductCountContext", Fields: []RPCField{ { - Name: "id", - TypeName: string(DataTypeString), - JSONPath: "id", - ResolvePath: buildPath("categories.id"), + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", + ResolvePath: buildPath("categories.id"), }, { - Name: "name", - TypeName: string(DataTypeString), - JSONPath: "name", - ResolvePath: buildPath("categories.name"), + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", + ResolvePath: buildPath("categories.name"), }, }, }, }, { - Name: "field_args", - TypeName: string(DataTypeMessage), - JSONPath: "", + Name: "field_args", + ProtoTypeName: DataTypeMessage, + JSONPath: "", Message: &RPCMessage{ Name: "ResolveCategoryProductCountArgs", Fields: []RPCField{ { - Name: "filters", - TypeName: string(DataTypeMessage), - JSONPath: "p2", - Optional: true, + Name: "filters", + ProtoTypeName: DataTypeMessage, + JSONPath: "p2", + Optional: true, Message: &RPCMessage{ Name: "ProductCountFilter", Fields: []RPCField{ { - Name: "min_price", - TypeName: string(DataTypeDouble), - JSONPath: "minPrice", - Optional: true, + Name: "min_price", + ProtoTypeName: DataTypeDouble, + JSONPath: "minPrice", + Optional: true, }, { - Name: "max_price", - TypeName: string(DataTypeDouble), - JSONPath: "maxPrice", - Optional: true, + Name: "max_price", + ProtoTypeName: DataTypeDouble, + JSONPath: "maxPrice", + Optional: true, }, { - Name: "in_stock", - TypeName: string(DataTypeBool), - JSONPath: "inStock", - Optional: true, + Name: "in_stock", + ProtoTypeName: DataTypeBool, + JSONPath: "inStock", + Optional: true, }, { - Name: "search_term", - TypeName: string(DataTypeString), - JSONPath: "searchTerm", - Optional: true, + Name: "search_term", + ProtoTypeName: DataTypeString, + JSONPath: "searchTerm", + Optional: true, }, }, }, @@ -371,17 +371,17 @@ func TestExecutionPlanFieldResolvers(t *testing.T) { Name: "ResolveCategoryProductCountResponse", Fields: []RPCField{ { - Name: "result", - TypeName: string(DataTypeMessage), - JSONPath: "result", - Repeated: true, + Name: "result", + ProtoTypeName: DataTypeMessage, + JSONPath: "result", + Repeated: true, Message: &RPCMessage{ Name: "ResolveCategoryProductCountResult", Fields: []RPCField{ { - Name: "product_count", - TypeName: string(DataTypeInt32), - JSONPath: "productCount2", + Name: "product_count", + ProtoTypeName: DataTypeInt32, + JSONPath: "productCount2", }, }, }, @@ -407,25 +407,25 @@ func TestExecutionPlanFieldResolvers(t *testing.T) { Name: "QueryCategoriesResponse", Fields: []RPCField{ { - Name: "categories", - TypeName: string(DataTypeMessage), - JSONPath: "categories", - Repeated: true, + Name: "categories", + ProtoTypeName: DataTypeMessage, + JSONPath: "categories", + Repeated: true, Message: &RPCMessage{ Name: "Category", Fields: []RPCField{ { - Name: "id", - TypeName: string(DataTypeString), - JSONPath: "id", + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", }, { - Name: "subcategories", - TypeName: string(DataTypeMessage), - JSONPath: "subcategories", - Repeated: false, - IsListType: true, - Optional: true, + Name: "subcategories", + ProtoTypeName: DataTypeMessage, + JSONPath: "subcategories", + Repeated: false, + IsListType: true, + Optional: true, ListMetadata: &ListMetadata{ NestingLevel: 1, LevelInfo: []LevelInfo{{Optional: true}}, @@ -434,25 +434,25 @@ func TestExecutionPlanFieldResolvers(t *testing.T) { Name: "Subcategory", Fields: []RPCField{ { - Name: "id", - TypeName: string(DataTypeString), - JSONPath: "id", + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", }, { - Name: "name", - TypeName: string(DataTypeString), - JSONPath: "name", + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", }, { - Name: "description", - TypeName: string(DataTypeString), - JSONPath: "description", - Optional: true, + Name: "description", + ProtoTypeName: DataTypeString, + JSONPath: "description", + Optional: true, }, { - Name: "is_active", - TypeName: string(DataTypeBool), - JSONPath: "isActive", + Name: "is_active", + ProtoTypeName: DataTypeBool, + JSONPath: "isActive", }, }, }, @@ -473,66 +473,66 @@ func TestExecutionPlanFieldResolvers(t *testing.T) { Name: "ResolveSubcategoryItemCountRequest", Fields: []RPCField{ { - Name: "context", - TypeName: string(DataTypeMessage), - JSONPath: "", - Repeated: true, + Name: "context", + ProtoTypeName: DataTypeMessage, + JSONPath: "", + Repeated: true, Message: &RPCMessage{ Name: "ResolveSubcategoryItemCountContext", Fields: []RPCField{ { - Name: "id", - TypeName: string(DataTypeString), - JSONPath: "id", - ResolvePath: buildPath("categories.@subcategories.id"), + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", + ResolvePath: buildPath("categories.@subcategories.id"), }, }, }, }, { - Name: "field_args", - TypeName: string(DataTypeMessage), - JSONPath: "", + Name: "field_args", + ProtoTypeName: DataTypeMessage, + JSONPath: "", Message: &RPCMessage{ Name: "ResolveSubcategoryItemCountArgs", Fields: []RPCField{ { - Name: "filters", - TypeName: string(DataTypeMessage), - JSONPath: "filter", - Optional: true, + Name: "filters", + ProtoTypeName: DataTypeMessage, + JSONPath: "filter", + Optional: true, Message: &RPCMessage{ Name: "SubcategoryItemFilter", Fields: []RPCField{ { - Name: "min_price", - TypeName: string(DataTypeDouble), - JSONPath: "minPrice", - Optional: true, + Name: "min_price", + ProtoTypeName: DataTypeDouble, + JSONPath: "minPrice", + Optional: true, }, { - Name: "max_price", - TypeName: string(DataTypeDouble), - JSONPath: "maxPrice", - Optional: true, + Name: "max_price", + ProtoTypeName: DataTypeDouble, + JSONPath: "maxPrice", + Optional: true, }, { - Name: "in_stock", - TypeName: string(DataTypeBool), - JSONPath: "inStock", - Optional: true, + Name: "in_stock", + ProtoTypeName: DataTypeBool, + JSONPath: "inStock", + Optional: true, }, { - Name: "is_active", - TypeName: string(DataTypeBool), - JSONPath: "isActive", - Optional: true, + Name: "is_active", + ProtoTypeName: DataTypeBool, + JSONPath: "isActive", + Optional: true, }, { - Name: "search_term", - TypeName: string(DataTypeString), - JSONPath: "searchTerm", - Optional: true, + Name: "search_term", + ProtoTypeName: DataTypeString, + JSONPath: "searchTerm", + Optional: true, }, }, }, @@ -546,17 +546,17 @@ func TestExecutionPlanFieldResolvers(t *testing.T) { Name: "ResolveSubcategoryItemCountResponse", Fields: []RPCField{ { - Name: "result", - TypeName: string(DataTypeMessage), - JSONPath: "result", - Repeated: true, + Name: "result", + ProtoTypeName: DataTypeMessage, + JSONPath: "result", + Repeated: true, Message: &RPCMessage{ Name: "ResolveSubcategoryItemCountResult", Fields: []RPCField{ { - Name: "item_count", - TypeName: string(DataTypeInt32), - JSONPath: "itemCount", + Name: "item_count", + ProtoTypeName: DataTypeInt32, + JSONPath: "itemCount", }, }, }, @@ -582,10 +582,10 @@ func TestExecutionPlanFieldResolvers(t *testing.T) { Name: "QueryCategoriesResponse", Fields: []RPCField{ { - Name: "categories", - TypeName: string(DataTypeMessage), - JSONPath: "categories", - Repeated: true, + Name: "categories", + ProtoTypeName: DataTypeMessage, + JSONPath: "categories", + Repeated: true, Message: &RPCMessage{ Name: "Category", Fields: []RPCField{}, @@ -604,40 +604,40 @@ func TestExecutionPlanFieldResolvers(t *testing.T) { Name: "ResolveCategoryCategoryMetricsRequest", Fields: []RPCField{ { - Name: "context", - TypeName: string(DataTypeMessage), - JSONPath: "", - Repeated: true, + Name: "context", + ProtoTypeName: DataTypeMessage, + JSONPath: "", + Repeated: true, Message: &RPCMessage{ Name: "ResolveCategoryCategoryMetricsContext", Fields: []RPCField{ { - Name: "id", - TypeName: string(DataTypeString), - JSONPath: "id", - ResolvePath: buildPath("categories.id"), + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", + ResolvePath: buildPath("categories.id"), }, { - Name: "name", - TypeName: string(DataTypeString), - JSONPath: "name", - ResolvePath: buildPath("categories.name"), + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", + ResolvePath: buildPath("categories.name"), }, }, }, }, { - Name: "field_args", - TypeName: string(DataTypeMessage), - JSONPath: "", + Name: "field_args", + ProtoTypeName: DataTypeMessage, + JSONPath: "", Message: &RPCMessage{ Name: "ResolveCategoryCategoryMetricsArgs", Fields: []RPCField{ { - Name: "metric_type", - TypeName: string(DataTypeString), - JSONPath: "metricType", - Optional: false, + Name: "metric_type", + ProtoTypeName: DataTypeString, + JSONPath: "metricType", + Optional: false, }, }, }, @@ -648,35 +648,35 @@ func TestExecutionPlanFieldResolvers(t *testing.T) { Name: "ResolveCategoryCategoryMetricsResponse", Fields: []RPCField{ { - Name: "result", - TypeName: string(DataTypeMessage), - JSONPath: "result", - Repeated: true, + Name: "result", + ProtoTypeName: DataTypeMessage, + JSONPath: "result", + Repeated: true, Message: &RPCMessage{ Name: "ResolveCategoryCategoryMetricsResult", Fields: []RPCField{ { - Name: "category_metrics", - TypeName: string(DataTypeMessage), - JSONPath: "categoryMetrics", - Optional: true, + Name: "category_metrics", + ProtoTypeName: DataTypeMessage, + JSONPath: "categoryMetrics", + Optional: true, Message: &RPCMessage{ Name: "CategoryMetrics", Fields: []RPCField{ { - Name: "id", - TypeName: string(DataTypeString), - JSONPath: "id", + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", }, { - Name: "metric_type", - TypeName: string(DataTypeString), - JSONPath: "metricType", + Name: "metric_type", + ProtoTypeName: DataTypeString, + JSONPath: "metricType", }, { - Name: "value", - TypeName: string(DataTypeDouble), - JSONPath: "value", + Name: "value", + ProtoTypeName: DataTypeDouble, + JSONPath: "value", }, }, }, @@ -705,10 +705,10 @@ func TestExecutionPlanFieldResolvers(t *testing.T) { Name: "QueryCategoriesResponse", Fields: []RPCField{ { - Name: "categories", - TypeName: string(DataTypeMessage), - JSONPath: "categories", - Repeated: true, + Name: "categories", + ProtoTypeName: DataTypeMessage, + JSONPath: "categories", + Repeated: true, Message: &RPCMessage{ Name: "Category", Fields: []RPCField{}, @@ -727,34 +727,34 @@ func TestExecutionPlanFieldResolvers(t *testing.T) { Name: "ResolveCategoryPopularityScoreRequest", Fields: []RPCField{ { - Name: "context", - TypeName: string(DataTypeMessage), - JSONPath: "", - Repeated: true, + Name: "context", + ProtoTypeName: DataTypeMessage, + JSONPath: "", + Repeated: true, Message: &RPCMessage{ Name: "ResolveCategoryPopularityScoreContext", Fields: []RPCField{ { - Name: "id", - TypeName: string(DataTypeString), - JSONPath: "id", - ResolvePath: buildPath("categories.id"), + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", + ResolvePath: buildPath("categories.id"), }, }, }, }, { - Name: "field_args", - TypeName: string(DataTypeMessage), - JSONPath: "", + Name: "field_args", + ProtoTypeName: DataTypeMessage, + JSONPath: "", Message: &RPCMessage{ Name: "ResolveCategoryPopularityScoreArgs", Fields: []RPCField{ { - Name: "threshold", - TypeName: string(DataTypeInt32), - JSONPath: "threshold", - Optional: true, + Name: "threshold", + ProtoTypeName: DataTypeInt32, + JSONPath: "threshold", + Optional: true, }, }, }, @@ -765,18 +765,18 @@ func TestExecutionPlanFieldResolvers(t *testing.T) { Name: "ResolveCategoryPopularityScoreResponse", Fields: []RPCField{ { - Name: "result", - TypeName: string(DataTypeMessage), - JSONPath: "result", - Repeated: true, + Name: "result", + ProtoTypeName: DataTypeMessage, + JSONPath: "result", + Repeated: true, Message: &RPCMessage{ Name: "ResolveCategoryPopularityScoreResult", Fields: []RPCField{ { - Name: "popularity_score", - TypeName: string(DataTypeInt32), - JSONPath: "popularityScore", - Optional: true, + Name: "popularity_score", + ProtoTypeName: DataTypeInt32, + JSONPath: "popularityScore", + Optional: true, }, }, }, @@ -794,39 +794,39 @@ func TestExecutionPlanFieldResolvers(t *testing.T) { Name: "ResolveCategoryCategoryMetricsRequest", Fields: []RPCField{ { - Name: "context", - TypeName: string(DataTypeMessage), - JSONPath: "", - Repeated: true, + Name: "context", + ProtoTypeName: DataTypeMessage, + JSONPath: "", + Repeated: true, Message: &RPCMessage{ Name: "ResolveCategoryCategoryMetricsContext", Fields: []RPCField{ { - Name: "id", - TypeName: string(DataTypeString), - JSONPath: "id", - ResolvePath: buildPath("categories.id"), + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", + ResolvePath: buildPath("categories.id"), }, { - Name: "name", - TypeName: string(DataTypeString), - JSONPath: "name", - ResolvePath: buildPath("categories.name"), + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", + ResolvePath: buildPath("categories.name"), }, }, }, }, { - Name: "field_args", - TypeName: string(DataTypeMessage), - JSONPath: "", + Name: "field_args", + ProtoTypeName: DataTypeMessage, + JSONPath: "", Message: &RPCMessage{ Name: "ResolveCategoryCategoryMetricsArgs", Fields: []RPCField{ { - Name: "metric_type", - TypeName: string(DataTypeString), - JSONPath: "metricType", + Name: "metric_type", + ProtoTypeName: DataTypeString, + JSONPath: "metricType", }, }, }, @@ -837,35 +837,35 @@ func TestExecutionPlanFieldResolvers(t *testing.T) { Name: "ResolveCategoryCategoryMetricsResponse", Fields: []RPCField{ { - Name: "result", - TypeName: string(DataTypeMessage), - JSONPath: "result", - Repeated: true, + Name: "result", + ProtoTypeName: DataTypeMessage, + JSONPath: "result", + Repeated: true, Message: &RPCMessage{ Name: "ResolveCategoryCategoryMetricsResult", Fields: []RPCField{ { - Name: "category_metrics", - TypeName: string(DataTypeMessage), - JSONPath: "categoryMetrics", - Optional: true, + Name: "category_metrics", + ProtoTypeName: DataTypeMessage, + JSONPath: "categoryMetrics", + Optional: true, Message: &RPCMessage{ Name: "CategoryMetrics", Fields: []RPCField{ { - Name: "id", - TypeName: string(DataTypeString), - JSONPath: "id", + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", }, { - Name: "metric_type", - TypeName: string(DataTypeString), - JSONPath: "metricType", + Name: "metric_type", + ProtoTypeName: DataTypeString, + JSONPath: "metricType", }, { - Name: "value", - TypeName: string(DataTypeDouble), - JSONPath: "value", + Name: "value", + ProtoTypeName: DataTypeDouble, + JSONPath: "value", }, }, }, diff --git a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_list_test.go b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_list_test.go index 2d0fbbffa..1957e88ba 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_list_test.go +++ b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_list_test.go @@ -25,22 +25,22 @@ func TestListExecutionPlan(t *testing.T) { Name: "QueryBlogPostResponse", Fields: RPCFields{ { - Name: "blog_post", - TypeName: string(DataTypeMessage), - JSONPath: "blogPost", + Name: "blog_post", + ProtoTypeName: DataTypeMessage, + JSONPath: "blogPost", Message: &RPCMessage{ Name: "BlogPost", Fields: RPCFields{ { - Name: "title", - TypeName: string(DataTypeString), - JSONPath: "title", + Name: "title", + ProtoTypeName: DataTypeString, + JSONPath: "title", }, { - Name: "tags", - TypeName: string(DataTypeString), - Repeated: true, - JSONPath: "tags", + Name: "tags", + ProtoTypeName: DataTypeString, + Repeated: true, + JSONPath: "tags", }, }, }, @@ -66,23 +66,23 @@ func TestListExecutionPlan(t *testing.T) { Name: "QueryBlogPostResponse", Fields: RPCFields{ { - Name: "blog_post", - TypeName: string(DataTypeMessage), - JSONPath: "blogPost", + Name: "blog_post", + ProtoTypeName: DataTypeMessage, + JSONPath: "blogPost", Message: &RPCMessage{ Name: "BlogPost", Fields: RPCFields{ { - Name: "title", - TypeName: string(DataTypeString), - JSONPath: "title", + Name: "title", + ProtoTypeName: DataTypeString, + JSONPath: "title", }, { - Name: "optional_tags", - TypeName: string(DataTypeString), - Optional: true, - JSONPath: "optionalTags", - IsListType: true, + Name: "optional_tags", + ProtoTypeName: DataTypeString, + Optional: true, + JSONPath: "optionalTags", + IsListType: true, ListMetadata: &ListMetadata{ NestingLevel: 1, LevelInfo: []LevelInfo{ @@ -116,18 +116,18 @@ func TestListExecutionPlan(t *testing.T) { Name: "QueryBlogPostResponse", Fields: RPCFields{ { - Name: "blog_post", - TypeName: string(DataTypeMessage), - JSONPath: "blogPost", + Name: "blog_post", + ProtoTypeName: DataTypeMessage, + JSONPath: "blogPost", Message: &RPCMessage{ Name: "BlogPost", Fields: RPCFields{ { - Name: "tag_groups", - TypeName: string(DataTypeString), - Repeated: false, - JSONPath: "tagGroups", - IsListType: true, + Name: "tag_groups", + ProtoTypeName: DataTypeString, + Repeated: false, + JSONPath: "tagGroups", + IsListType: true, ListMetadata: &ListMetadata{ NestingLevel: 2, LevelInfo: []LevelInfo{ @@ -183,11 +183,11 @@ func TestListParametersExecutionPlan(t *testing.T) { Name: "QueryCategoriesByKindsRequest", Fields: []RPCField{ { - Name: "kinds", - TypeName: string(DataTypeEnum), - JSONPath: "kinds", - EnumName: "CategoryKind", - Repeated: true, + Name: "kinds", + ProtoTypeName: DataTypeEnum, + JSONPath: "kinds", + EnumName: "CategoryKind", + Repeated: true, }, }, }, @@ -195,28 +195,28 @@ func TestListParametersExecutionPlan(t *testing.T) { Name: "QueryCategoriesByKindsResponse", Fields: []RPCField{ { - Name: "categories_by_kinds", - TypeName: string(DataTypeMessage), - JSONPath: "categoriesByKinds", - Repeated: true, + Name: "categories_by_kinds", + ProtoTypeName: DataTypeMessage, + JSONPath: "categoriesByKinds", + Repeated: true, Message: &RPCMessage{ Name: "Category", Fields: []RPCField{ { - Name: "id", - TypeName: string(DataTypeString), - JSONPath: "id", + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", }, { - Name: "name", - TypeName: string(DataTypeString), - JSONPath: "name", + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", }, { - Name: "kind", - TypeName: string(DataTypeEnum), - JSONPath: "kind", - EnumName: "CategoryKind", + Name: "kind", + ProtoTypeName: DataTypeEnum, + JSONPath: "kind", + EnumName: "CategoryKind", }, }, }, @@ -239,47 +239,47 @@ func TestListParametersExecutionPlan(t *testing.T) { Name: "QueryCalculateTotalsRequest", Fields: []RPCField{ { - Name: "orders", - TypeName: string(DataTypeMessage), - JSONPath: "orders", - Repeated: true, + Name: "orders", + ProtoTypeName: DataTypeMessage, + JSONPath: "orders", + Repeated: true, Message: &RPCMessage{ Name: "OrderInput", Fields: []RPCField{ { - Name: "order_id", - TypeName: string(DataTypeString), - JSONPath: "orderId", + Name: "order_id", + ProtoTypeName: DataTypeString, + JSONPath: "orderId", }, { - Name: "customer_name", - TypeName: string(DataTypeString), - JSONPath: "customerName", + Name: "customer_name", + ProtoTypeName: DataTypeString, + JSONPath: "customerName", }, { - Name: "lines", - TypeName: string(DataTypeMessage), - JSONPath: "lines", - Repeated: true, + Name: "lines", + ProtoTypeName: DataTypeMessage, + JSONPath: "lines", + Repeated: true, Message: &RPCMessage{ Name: "OrderLineInput", Fields: []RPCField{ { - Name: "product_id", - TypeName: string(DataTypeString), - JSONPath: "productId", + Name: "product_id", + ProtoTypeName: DataTypeString, + JSONPath: "productId", }, { - Name: "quantity", - TypeName: string(DataTypeInt32), - JSONPath: "quantity", + Name: "quantity", + ProtoTypeName: DataTypeInt32, + JSONPath: "quantity", }, { - Name: "modifiers", - TypeName: string(DataTypeString), - JSONPath: "modifiers", - Optional: true, - IsListType: true, + Name: "modifiers", + ProtoTypeName: DataTypeString, + JSONPath: "modifiers", + Optional: true, + IsListType: true, ListMetadata: &ListMetadata{ NestingLevel: 1, LevelInfo: []LevelInfo{ @@ -301,27 +301,27 @@ func TestListParametersExecutionPlan(t *testing.T) { Name: "QueryCalculateTotalsResponse", Fields: []RPCField{ { - Name: "calculate_totals", - TypeName: string(DataTypeMessage), - JSONPath: "calculateTotals", - Repeated: true, + Name: "calculate_totals", + ProtoTypeName: DataTypeMessage, + JSONPath: "calculateTotals", + Repeated: true, Message: &RPCMessage{ Name: "Order", Fields: []RPCField{ { - Name: "order_id", - TypeName: string(DataTypeString), - JSONPath: "orderId", + Name: "order_id", + ProtoTypeName: DataTypeString, + JSONPath: "orderId", }, { - Name: "customer_name", - TypeName: string(DataTypeString), - JSONPath: "customerName", + Name: "customer_name", + ProtoTypeName: DataTypeString, + JSONPath: "customerName", }, { - Name: "total_items", - TypeName: string(DataTypeInt32), - JSONPath: "totalItems", + Name: "total_items", + ProtoTypeName: DataTypeInt32, + JSONPath: "totalItems", }, }, }, @@ -344,34 +344,34 @@ func TestListParametersExecutionPlan(t *testing.T) { Name: "MutationCreateBlogPostRequest", Fields: []RPCField{ { - Name: "input", - TypeName: string(DataTypeMessage), - JSONPath: "input", + Name: "input", + ProtoTypeName: DataTypeMessage, + JSONPath: "input", Message: &RPCMessage{ Name: "BlogPostInput", Fields: []RPCField{ { - Name: "title", - TypeName: string(DataTypeString), - JSONPath: "title", + Name: "title", + ProtoTypeName: DataTypeString, + JSONPath: "title", }, { - Name: "content", - TypeName: string(DataTypeString), - JSONPath: "content", + Name: "content", + ProtoTypeName: DataTypeString, + JSONPath: "content", }, { - Name: "tags", - TypeName: string(DataTypeString), - JSONPath: "tags", - Repeated: true, + Name: "tags", + ProtoTypeName: DataTypeString, + JSONPath: "tags", + Repeated: true, }, { - Name: "optional_tags", - TypeName: string(DataTypeString), - JSONPath: "optionalTags", - Optional: true, - IsListType: true, + Name: "optional_tags", + ProtoTypeName: DataTypeString, + JSONPath: "optionalTags", + Optional: true, + IsListType: true, ListMetadata: &ListMetadata{ NestingLevel: 1, LevelInfo: []LevelInfo{ @@ -382,17 +382,17 @@ func TestListParametersExecutionPlan(t *testing.T) { }, }, { - Name: "categories", - TypeName: string(DataTypeString), - JSONPath: "categories", - Repeated: true, + Name: "categories", + ProtoTypeName: DataTypeString, + JSONPath: "categories", + Repeated: true, }, { - Name: "keywords", - TypeName: string(DataTypeString), - JSONPath: "keywords", - Optional: true, - IsListType: true, + Name: "keywords", + ProtoTypeName: DataTypeString, + JSONPath: "keywords", + Optional: true, + IsListType: true, ListMetadata: &ListMetadata{ NestingLevel: 1, LevelInfo: []LevelInfo{ @@ -403,17 +403,17 @@ func TestListParametersExecutionPlan(t *testing.T) { }, }, { - Name: "view_counts", - TypeName: string(DataTypeInt32), - JSONPath: "viewCounts", - Repeated: true, + Name: "view_counts", + ProtoTypeName: DataTypeInt32, + JSONPath: "viewCounts", + Repeated: true, }, { - Name: "ratings", - TypeName: string(DataTypeDouble), - JSONPath: "ratings", - Optional: true, - IsListType: true, + Name: "ratings", + ProtoTypeName: DataTypeDouble, + JSONPath: "ratings", + Optional: true, + IsListType: true, ListMetadata: &ListMetadata{ NestingLevel: 1, LevelInfo: []LevelInfo{ @@ -424,11 +424,11 @@ func TestListParametersExecutionPlan(t *testing.T) { }, }, { - Name: "is_published", - TypeName: string(DataTypeBool), - JSONPath: "isPublished", - Optional: true, - IsListType: true, + Name: "is_published", + ProtoTypeName: DataTypeBool, + JSONPath: "isPublished", + Optional: true, + IsListType: true, ListMetadata: &ListMetadata{ NestingLevel: 1, LevelInfo: []LevelInfo{ @@ -439,10 +439,10 @@ func TestListParametersExecutionPlan(t *testing.T) { }, }, { - Name: "tag_groups", - TypeName: string(DataTypeString), - JSONPath: "tagGroups", - IsListType: true, + Name: "tag_groups", + ProtoTypeName: DataTypeString, + JSONPath: "tagGroups", + IsListType: true, ListMetadata: &ListMetadata{ NestingLevel: 2, LevelInfo: []LevelInfo{ @@ -456,10 +456,10 @@ func TestListParametersExecutionPlan(t *testing.T) { }, }, { - Name: "related_topics", - TypeName: string(DataTypeString), - JSONPath: "relatedTopics", - IsListType: true, + Name: "related_topics", + ProtoTypeName: DataTypeString, + JSONPath: "relatedTopics", + IsListType: true, ListMetadata: &ListMetadata{ NestingLevel: 2, LevelInfo: []LevelInfo{ @@ -473,10 +473,10 @@ func TestListParametersExecutionPlan(t *testing.T) { }, }, { - Name: "comment_threads", - TypeName: string(DataTypeString), - JSONPath: "commentThreads", - IsListType: true, + Name: "comment_threads", + ProtoTypeName: DataTypeString, + JSONPath: "commentThreads", + IsListType: true, ListMetadata: &ListMetadata{ NestingLevel: 2, LevelInfo: []LevelInfo{ @@ -490,11 +490,11 @@ func TestListParametersExecutionPlan(t *testing.T) { }, }, { - Name: "suggestions", - TypeName: string(DataTypeString), - JSONPath: "suggestions", - Optional: true, - IsListType: true, + Name: "suggestions", + ProtoTypeName: DataTypeString, + JSONPath: "suggestions", + Optional: true, + IsListType: true, ListMetadata: &ListMetadata{ NestingLevel: 2, LevelInfo: []LevelInfo{ @@ -508,12 +508,12 @@ func TestListParametersExecutionPlan(t *testing.T) { }, }, { - Name: "related_categories", - TypeName: string(DataTypeMessage), - JSONPath: "relatedCategories", - Repeated: false, - Optional: true, - IsListType: true, + Name: "related_categories", + ProtoTypeName: DataTypeMessage, + JSONPath: "relatedCategories", + Repeated: false, + Optional: true, + IsListType: true, ListMetadata: &ListMetadata{ NestingLevel: 1, LevelInfo: []LevelInfo{ @@ -526,26 +526,26 @@ func TestListParametersExecutionPlan(t *testing.T) { Name: "CategoryInput", Fields: []RPCField{ { - Name: "name", - TypeName: string(DataTypeString), - JSONPath: "name", + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", }, { - Name: "kind", - TypeName: string(DataTypeEnum), - JSONPath: "kind", - EnumName: "CategoryKind", + Name: "kind", + ProtoTypeName: DataTypeEnum, + JSONPath: "kind", + EnumName: "CategoryKind", }, }, }, }, { - Name: "contributors", - TypeName: string(DataTypeMessage), - JSONPath: "contributors", - Repeated: false, - Optional: true, - IsListType: true, + Name: "contributors", + ProtoTypeName: DataTypeMessage, + JSONPath: "contributors", + Repeated: false, + Optional: true, + IsListType: true, ListMetadata: &ListMetadata{ NestingLevel: 1, LevelInfo: []LevelInfo{ @@ -558,20 +558,20 @@ func TestListParametersExecutionPlan(t *testing.T) { Name: "UserInput", Fields: []RPCField{ { - Name: "name", - TypeName: string(DataTypeString), - JSONPath: "name", + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", }, }, }, }, { - Name: "category_groups", - TypeName: string(DataTypeMessage), - JSONPath: "categoryGroups", - Repeated: false, - Optional: true, - IsListType: true, + Name: "category_groups", + ProtoTypeName: DataTypeMessage, + JSONPath: "categoryGroups", + Repeated: false, + Optional: true, + IsListType: true, ListMetadata: &ListMetadata{ NestingLevel: 2, LevelInfo: []LevelInfo{ @@ -587,15 +587,15 @@ func TestListParametersExecutionPlan(t *testing.T) { Name: "CategoryInput", Fields: []RPCField{ { - Name: "name", - TypeName: string(DataTypeString), - JSONPath: "name", + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", }, { - Name: "kind", - TypeName: string(DataTypeEnum), - JSONPath: "kind", - EnumName: "CategoryKind", + Name: "kind", + ProtoTypeName: DataTypeEnum, + JSONPath: "kind", + EnumName: "CategoryKind", }, }, }, @@ -609,34 +609,34 @@ func TestListParametersExecutionPlan(t *testing.T) { Name: "MutationCreateBlogPostResponse", Fields: []RPCField{ { - Name: "create_blog_post", - TypeName: string(DataTypeMessage), - JSONPath: "createBlogPost", + Name: "create_blog_post", + ProtoTypeName: DataTypeMessage, + JSONPath: "createBlogPost", Message: &RPCMessage{ Name: "BlogPost", Fields: []RPCField{ { - Name: "id", - TypeName: string(DataTypeString), - JSONPath: "id", + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", }, { - Name: "title", - TypeName: string(DataTypeString), - JSONPath: "title", + Name: "title", + ProtoTypeName: DataTypeString, + JSONPath: "title", }, { - Name: "tags", - TypeName: string(DataTypeString), - JSONPath: "tags", - Repeated: true, + Name: "tags", + ProtoTypeName: DataTypeString, + JSONPath: "tags", + Repeated: true, }, { - Name: "optional_tags", - TypeName: string(DataTypeString), - JSONPath: "optionalTags", - Optional: true, - IsListType: true, + Name: "optional_tags", + ProtoTypeName: DataTypeString, + JSONPath: "optionalTags", + Optional: true, + IsListType: true, ListMetadata: &ListMetadata{ NestingLevel: 1, LevelInfo: []LevelInfo{ @@ -647,17 +647,17 @@ func TestListParametersExecutionPlan(t *testing.T) { }, }, { - Name: "categories", - TypeName: string(DataTypeString), - JSONPath: "categories", - Repeated: true, + Name: "categories", + ProtoTypeName: DataTypeString, + JSONPath: "categories", + Repeated: true, }, { - Name: "keywords", - TypeName: string(DataTypeString), - JSONPath: "keywords", - Optional: true, - IsListType: true, + Name: "keywords", + ProtoTypeName: DataTypeString, + JSONPath: "keywords", + Optional: true, + IsListType: true, ListMetadata: &ListMetadata{ NestingLevel: 1, LevelInfo: []LevelInfo{ @@ -688,41 +688,41 @@ func TestListParametersExecutionPlan(t *testing.T) { Name: "MutationCreateAuthorRequest", Fields: []RPCField{ { - Name: "input", - TypeName: string(DataTypeMessage), - JSONPath: "input", + Name: "input", + ProtoTypeName: DataTypeMessage, + JSONPath: "input", Message: &RPCMessage{ Name: "AuthorInput", Fields: []RPCField{ { - Name: "name", - TypeName: string(DataTypeString), - JSONPath: "name", + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", }, { - Name: "email", - TypeName: string(DataTypeString), - JSONPath: "email", - Optional: true, + Name: "email", + ProtoTypeName: DataTypeString, + JSONPath: "email", + Optional: true, }, { - Name: "skills", - TypeName: string(DataTypeString), - JSONPath: "skills", - Repeated: true, + Name: "skills", + ProtoTypeName: DataTypeString, + JSONPath: "skills", + Repeated: true, }, { - Name: "languages", - TypeName: string(DataTypeString), - JSONPath: "languages", - Repeated: true, + Name: "languages", + ProtoTypeName: DataTypeString, + JSONPath: "languages", + Repeated: true, }, { - Name: "social_links", - TypeName: string(DataTypeString), - JSONPath: "socialLinks", - Optional: true, - IsListType: true, + Name: "social_links", + ProtoTypeName: DataTypeString, + JSONPath: "socialLinks", + Optional: true, + IsListType: true, ListMetadata: &ListMetadata{ NestingLevel: 1, LevelInfo: []LevelInfo{ @@ -733,10 +733,10 @@ func TestListParametersExecutionPlan(t *testing.T) { }, }, { - Name: "teams_by_project", - TypeName: string(DataTypeString), - JSONPath: "teamsByProject", - IsListType: true, + Name: "teams_by_project", + ProtoTypeName: DataTypeString, + JSONPath: "teamsByProject", + IsListType: true, ListMetadata: &ListMetadata{ NestingLevel: 2, LevelInfo: []LevelInfo{ @@ -750,11 +750,11 @@ func TestListParametersExecutionPlan(t *testing.T) { }, }, { - Name: "collaborations", - TypeName: string(DataTypeString), - JSONPath: "collaborations", - Optional: true, - IsListType: true, + Name: "collaborations", + ProtoTypeName: DataTypeString, + JSONPath: "collaborations", + Optional: true, + IsListType: true, ListMetadata: &ListMetadata{ NestingLevel: 2, LevelInfo: []LevelInfo{ @@ -768,34 +768,34 @@ func TestListParametersExecutionPlan(t *testing.T) { }, }, { - Name: "favorite_categories", - TypeName: string(DataTypeMessage), - JSONPath: "favoriteCategories", - Repeated: true, + Name: "favorite_categories", + ProtoTypeName: DataTypeMessage, + JSONPath: "favoriteCategories", + Repeated: true, Message: &RPCMessage{ Name: "CategoryInput", Fields: []RPCField{ { - Name: "name", - TypeName: string(DataTypeString), - JSONPath: "name", + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", }, { - Name: "kind", - TypeName: string(DataTypeEnum), - JSONPath: "kind", - EnumName: "CategoryKind", + Name: "kind", + ProtoTypeName: DataTypeEnum, + JSONPath: "kind", + EnumName: "CategoryKind", }, }, }, }, { - Name: "author_groups", - TypeName: string(DataTypeMessage), - JSONPath: "authorGroups", - Repeated: false, - Optional: true, - IsListType: true, + Name: "author_groups", + ProtoTypeName: DataTypeMessage, + JSONPath: "authorGroups", + Repeated: false, + Optional: true, + IsListType: true, ListMetadata: &ListMetadata{ NestingLevel: 2, LevelInfo: []LevelInfo{ @@ -811,20 +811,20 @@ func TestListParametersExecutionPlan(t *testing.T) { Name: "UserInput", Fields: []RPCField{ { - Name: "name", - TypeName: string(DataTypeString), - JSONPath: "name", + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", }, }, }, }, { - Name: "project_teams", - TypeName: string(DataTypeMessage), - JSONPath: "projectTeams", - Repeated: false, - Optional: true, - IsListType: true, + Name: "project_teams", + ProtoTypeName: DataTypeMessage, + JSONPath: "projectTeams", + Repeated: false, + Optional: true, + IsListType: true, ListMetadata: &ListMetadata{ NestingLevel: 2, LevelInfo: []LevelInfo{ @@ -840,9 +840,9 @@ func TestListParametersExecutionPlan(t *testing.T) { Name: "UserInput", Fields: []RPCField{ { - Name: "name", - TypeName: string(DataTypeString), - JSONPath: "name", + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", }, }, }, @@ -856,33 +856,33 @@ func TestListParametersExecutionPlan(t *testing.T) { Name: "MutationCreateAuthorResponse", Fields: []RPCField{ { - Name: "create_author", - TypeName: string(DataTypeMessage), - JSONPath: "createAuthor", + Name: "create_author", + ProtoTypeName: DataTypeMessage, + JSONPath: "createAuthor", Message: &RPCMessage{ Name: "Author", Fields: []RPCField{ { - Name: "id", - TypeName: string(DataTypeString), - JSONPath: "id", + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", }, { - Name: "name", - TypeName: string(DataTypeString), - JSONPath: "name", + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", }, { - Name: "skills", - TypeName: string(DataTypeString), - JSONPath: "skills", - Repeated: true, + Name: "skills", + ProtoTypeName: DataTypeString, + JSONPath: "skills", + Repeated: true, }, { - Name: "teams_by_project", - TypeName: string(DataTypeString), - JSONPath: "teamsByProject", - IsListType: true, + Name: "teams_by_project", + ProtoTypeName: DataTypeString, + JSONPath: "teamsByProject", + IsListType: true, ListMetadata: &ListMetadata{ NestingLevel: 2, LevelInfo: []LevelInfo{ @@ -896,11 +896,11 @@ func TestListParametersExecutionPlan(t *testing.T) { }, }, { - Name: "collaborations", - TypeName: string(DataTypeString), - JSONPath: "collaborations", - Optional: true, - IsListType: true, + Name: "collaborations", + ProtoTypeName: DataTypeString, + JSONPath: "collaborations", + Optional: true, + IsListType: true, ListMetadata: &ListMetadata{ NestingLevel: 2, LevelInfo: []LevelInfo{ @@ -934,29 +934,29 @@ func TestListParametersExecutionPlan(t *testing.T) { Name: "QueryBlogPostsWithFilterRequest", Fields: []RPCField{ { - Name: "filter", - TypeName: string(DataTypeMessage), - JSONPath: "filter", + Name: "filter", + ProtoTypeName: DataTypeMessage, + JSONPath: "filter", Message: &RPCMessage{ Name: "BlogPostFilter", Fields: []RPCField{ { - Name: "title", - TypeName: string(DataTypeString), - JSONPath: "title", - Optional: true, + Name: "title", + ProtoTypeName: DataTypeString, + JSONPath: "title", + Optional: true, }, { - Name: "has_categories", - TypeName: string(DataTypeBool), - JSONPath: "hasCategories", - Optional: true, + Name: "has_categories", + ProtoTypeName: DataTypeBool, + JSONPath: "hasCategories", + Optional: true, }, { - Name: "min_tags", - TypeName: string(DataTypeInt32), - JSONPath: "minTags", - Optional: true, + Name: "min_tags", + ProtoTypeName: DataTypeInt32, + JSONPath: "minTags", + Optional: true, }, }, }, @@ -967,28 +967,28 @@ func TestListParametersExecutionPlan(t *testing.T) { Name: "QueryBlogPostsWithFilterResponse", Fields: []RPCField{ { - Name: "blog_posts_with_filter", - TypeName: string(DataTypeMessage), - JSONPath: "blogPostsWithFilter", - Repeated: true, + Name: "blog_posts_with_filter", + ProtoTypeName: DataTypeMessage, + JSONPath: "blogPostsWithFilter", + Repeated: true, Message: &RPCMessage{ Name: "BlogPost", Fields: []RPCField{ { - Name: "id", - TypeName: string(DataTypeString), - JSONPath: "id", + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", }, { - Name: "title", - TypeName: string(DataTypeString), - JSONPath: "title", + Name: "title", + ProtoTypeName: DataTypeString, + JSONPath: "title", }, { - Name: "tags", - TypeName: string(DataTypeString), - JSONPath: "tags", - Repeated: true, + Name: "tags", + ProtoTypeName: DataTypeString, + JSONPath: "tags", + Repeated: true, }, }, }, @@ -1011,11 +1011,11 @@ func TestListParametersExecutionPlan(t *testing.T) { Name: "QueryBulkSearchAuthorsRequest", Fields: []RPCField{ { - Name: "filters", - TypeName: string(DataTypeMessage), - JSONPath: "filters", - IsListType: true, - Optional: true, + Name: "filters", + ProtoTypeName: DataTypeMessage, + JSONPath: "filters", + IsListType: true, + Optional: true, ListMetadata: &ListMetadata{ NestingLevel: 1, LevelInfo: []LevelInfo{ @@ -1028,22 +1028,22 @@ func TestListParametersExecutionPlan(t *testing.T) { Name: "AuthorFilter", Fields: []RPCField{ { - Name: "name", - TypeName: string(DataTypeString), - JSONPath: "name", - Optional: true, + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", + Optional: true, }, { - Name: "has_teams", - TypeName: string(DataTypeBool), - JSONPath: "hasTeams", - Optional: true, + Name: "has_teams", + ProtoTypeName: DataTypeBool, + JSONPath: "hasTeams", + Optional: true, }, { - Name: "skill_count", - TypeName: string(DataTypeInt32), - JSONPath: "skillCount", - Optional: true, + Name: "skill_count", + ProtoTypeName: DataTypeInt32, + JSONPath: "skillCount", + Optional: true, }, }, }, @@ -1054,34 +1054,34 @@ func TestListParametersExecutionPlan(t *testing.T) { Name: "QueryBulkSearchAuthorsResponse", Fields: []RPCField{ { - Name: "bulk_search_authors", - TypeName: string(DataTypeMessage), - JSONPath: "bulkSearchAuthors", - Repeated: true, + Name: "bulk_search_authors", + ProtoTypeName: DataTypeMessage, + JSONPath: "bulkSearchAuthors", + Repeated: true, Message: &RPCMessage{ Name: "Author", Fields: []RPCField{ { - Name: "id", - TypeName: string(DataTypeString), - JSONPath: "id", + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", }, { - Name: "name", - TypeName: string(DataTypeString), - JSONPath: "name", + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", }, { - Name: "email", - TypeName: string(DataTypeString), - JSONPath: "email", - Optional: true, + Name: "email", + ProtoTypeName: DataTypeString, + JSONPath: "email", + Optional: true, }, { - Name: "skills", - TypeName: string(DataTypeString), - JSONPath: "skills", - Repeated: true, + Name: "skills", + ProtoTypeName: DataTypeString, + JSONPath: "skills", + Repeated: true, }, }, }, diff --git a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_nullable_test.go b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_nullable_test.go index 100fed1af..d4f42596e 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_nullable_test.go +++ b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_nullable_test.go @@ -27,55 +27,55 @@ func TestNullableFieldsExecutionPlan(t *testing.T) { Name: "QueryNullableFieldsTypeResponse", Fields: []RPCField{ { - Name: "nullable_fields_type", - TypeName: string(DataTypeMessage), - JSONPath: "nullableFieldsType", + Name: "nullable_fields_type", + ProtoTypeName: DataTypeMessage, + JSONPath: "nullableFieldsType", Message: &RPCMessage{ Name: "NullableFieldsType", Fields: []RPCField{ { - Name: "id", - TypeName: string(DataTypeString), - JSONPath: "id", + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", }, { - Name: "name", - TypeName: string(DataTypeString), - JSONPath: "name", + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", }, { - Name: "optional_string", - TypeName: string(DataTypeString), - JSONPath: "optionalString", - Optional: true, + Name: "optional_string", + ProtoTypeName: DataTypeString, + JSONPath: "optionalString", + Optional: true, }, { - Name: "optional_int", - TypeName: string(DataTypeInt32), - JSONPath: "optionalInt", - Optional: true, + Name: "optional_int", + ProtoTypeName: DataTypeInt32, + JSONPath: "optionalInt", + Optional: true, }, { - Name: "optional_float", - TypeName: string(DataTypeDouble), - JSONPath: "optionalFloat", - Optional: true, + Name: "optional_float", + ProtoTypeName: DataTypeDouble, + JSONPath: "optionalFloat", + Optional: true, }, { - Name: "optional_boolean", - TypeName: string(DataTypeBool), - JSONPath: "optionalBoolean", - Optional: true, + Name: "optional_boolean", + ProtoTypeName: DataTypeBool, + JSONPath: "optionalBoolean", + Optional: true, }, { - Name: "required_string", - TypeName: string(DataTypeString), - JSONPath: "requiredString", + Name: "required_string", + ProtoTypeName: DataTypeString, + JSONPath: "requiredString", }, { - Name: "required_int", - TypeName: string(DataTypeInt32), - JSONPath: "requiredInt", + Name: "required_int", + ProtoTypeName: DataTypeInt32, + JSONPath: "requiredInt", }, }, }, @@ -98,29 +98,29 @@ func TestNullableFieldsExecutionPlan(t *testing.T) { Name: "QueryNullableFieldsTypeWithFilterRequest", Fields: []RPCField{ { - Name: "filter", - TypeName: string(DataTypeMessage), - JSONPath: "filter", + Name: "filter", + ProtoTypeName: DataTypeMessage, + JSONPath: "filter", Message: &RPCMessage{ Name: "NullableFieldsFilter", Fields: []RPCField{ { - Name: "name", - TypeName: string(DataTypeString), - JSONPath: "name", - Optional: true, + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", + Optional: true, }, { - Name: "optional_string", - TypeName: string(DataTypeString), - JSONPath: "optionalString", - Optional: true, + Name: "optional_string", + ProtoTypeName: DataTypeString, + JSONPath: "optionalString", + Optional: true, }, { - Name: "include_nulls", - TypeName: string(DataTypeBool), - JSONPath: "includeNulls", - Optional: true, + Name: "include_nulls", + ProtoTypeName: DataTypeBool, + JSONPath: "includeNulls", + Optional: true, }, }, }, @@ -131,46 +131,46 @@ func TestNullableFieldsExecutionPlan(t *testing.T) { Name: "QueryNullableFieldsTypeWithFilterResponse", Fields: []RPCField{ { - Name: "nullable_fields_type_with_filter", - TypeName: string(DataTypeMessage), - JSONPath: "nullableFieldsTypeWithFilter", - Repeated: true, + Name: "nullable_fields_type_with_filter", + ProtoTypeName: DataTypeMessage, + JSONPath: "nullableFieldsTypeWithFilter", + Repeated: true, Message: &RPCMessage{ Name: "NullableFieldsType", Fields: []RPCField{ { - Name: "id", - TypeName: string(DataTypeString), - JSONPath: "id", + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", }, { - Name: "name", - TypeName: string(DataTypeString), - JSONPath: "name", + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", }, { - Name: "optional_string", - TypeName: string(DataTypeString), - JSONPath: "optionalString", - Optional: true, + Name: "optional_string", + ProtoTypeName: DataTypeString, + JSONPath: "optionalString", + Optional: true, }, { - Name: "optional_int", - TypeName: string(DataTypeInt32), - JSONPath: "optionalInt", - Optional: true, + Name: "optional_int", + ProtoTypeName: DataTypeInt32, + JSONPath: "optionalInt", + Optional: true, }, { - Name: "optional_float", - TypeName: string(DataTypeDouble), - JSONPath: "optionalFloat", - Optional: true, + Name: "optional_float", + ProtoTypeName: DataTypeDouble, + JSONPath: "optionalFloat", + Optional: true, }, { - Name: "optional_boolean", - TypeName: string(DataTypeBool), - JSONPath: "optionalBoolean", - Optional: true, + Name: "optional_boolean", + ProtoTypeName: DataTypeBool, + JSONPath: "optionalBoolean", + Optional: true, }, }, }, @@ -193,9 +193,9 @@ func TestNullableFieldsExecutionPlan(t *testing.T) { Name: "QueryNullableFieldsTypeByIdRequest", Fields: []RPCField{ { - Name: "id", - TypeName: string(DataTypeString), - JSONPath: "id", + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", }, }, }, @@ -203,33 +203,33 @@ func TestNullableFieldsExecutionPlan(t *testing.T) { Name: "QueryNullableFieldsTypeByIdResponse", Fields: []RPCField{ { - Name: "nullable_fields_type_by_id", - TypeName: string(DataTypeMessage), - JSONPath: "nullableFieldsTypeById", - Optional: true, + Name: "nullable_fields_type_by_id", + ProtoTypeName: DataTypeMessage, + JSONPath: "nullableFieldsTypeById", + Optional: true, Message: &RPCMessage{ Name: "NullableFieldsType", Fields: []RPCField{ { - Name: "id", - TypeName: string(DataTypeString), - JSONPath: "id", + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", }, { - Name: "name", - TypeName: string(DataTypeString), - JSONPath: "name", + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", }, { - Name: "optional_string", - TypeName: string(DataTypeString), - JSONPath: "optionalString", - Optional: true, + Name: "optional_string", + ProtoTypeName: DataTypeString, + JSONPath: "optionalString", + Optional: true, }, { - Name: "required_string", - TypeName: string(DataTypeString), - JSONPath: "requiredString", + Name: "required_string", + ProtoTypeName: DataTypeString, + JSONPath: "requiredString", }, }, }, @@ -255,44 +255,44 @@ func TestNullableFieldsExecutionPlan(t *testing.T) { Name: "QueryAllNullableFieldsTypesResponse", Fields: []RPCField{ { - Name: "all_nullable_fields_types", - TypeName: string(DataTypeMessage), - JSONPath: "allNullableFieldsTypes", - Repeated: true, + Name: "all_nullable_fields_types", + ProtoTypeName: DataTypeMessage, + JSONPath: "allNullableFieldsTypes", + Repeated: true, Message: &RPCMessage{ Name: "NullableFieldsType", Fields: []RPCField{ { - Name: "id", - TypeName: string(DataTypeString), - JSONPath: "id", + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", }, { - Name: "name", - TypeName: string(DataTypeString), - JSONPath: "name", + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", }, { - Name: "optional_string", - TypeName: string(DataTypeString), - JSONPath: "optionalString", - Optional: true, + Name: "optional_string", + ProtoTypeName: DataTypeString, + JSONPath: "optionalString", + Optional: true, }, { - Name: "optional_int", - TypeName: string(DataTypeInt32), - JSONPath: "optionalInt", - Optional: true, + Name: "optional_int", + ProtoTypeName: DataTypeInt32, + JSONPath: "optionalInt", + Optional: true, }, { - Name: "required_string", - TypeName: string(DataTypeString), - JSONPath: "requiredString", + Name: "required_string", + ProtoTypeName: DataTypeString, + JSONPath: "requiredString", }, { - Name: "required_int", - TypeName: string(DataTypeInt32), - JSONPath: "requiredInt", + Name: "required_int", + ProtoTypeName: DataTypeInt32, + JSONPath: "requiredInt", }, }, }, @@ -315,50 +315,50 @@ func TestNullableFieldsExecutionPlan(t *testing.T) { Name: "MutationCreateNullableFieldsTypeRequest", Fields: []RPCField{ { - Name: "input", - TypeName: string(DataTypeMessage), - JSONPath: "input", + Name: "input", + ProtoTypeName: DataTypeMessage, + JSONPath: "input", Message: &RPCMessage{ Name: "NullableFieldsInput", Fields: []RPCField{ { - Name: "name", - TypeName: string(DataTypeString), - JSONPath: "name", + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", }, { - Name: "optional_string", - TypeName: string(DataTypeString), - JSONPath: "optionalString", - Optional: true, + Name: "optional_string", + ProtoTypeName: DataTypeString, + JSONPath: "optionalString", + Optional: true, }, { - Name: "optional_int", - TypeName: string(DataTypeInt32), - JSONPath: "optionalInt", - Optional: true, + Name: "optional_int", + ProtoTypeName: DataTypeInt32, + JSONPath: "optionalInt", + Optional: true, }, { - Name: "optional_float", - TypeName: string(DataTypeDouble), - JSONPath: "optionalFloat", - Optional: true, + Name: "optional_float", + ProtoTypeName: DataTypeDouble, + JSONPath: "optionalFloat", + Optional: true, }, { - Name: "optional_boolean", - TypeName: string(DataTypeBool), - JSONPath: "optionalBoolean", - Optional: true, + Name: "optional_boolean", + ProtoTypeName: DataTypeBool, + JSONPath: "optionalBoolean", + Optional: true, }, { - Name: "required_string", - TypeName: string(DataTypeString), - JSONPath: "requiredString", + Name: "required_string", + ProtoTypeName: DataTypeString, + JSONPath: "requiredString", }, { - Name: "required_int", - TypeName: string(DataTypeInt32), - JSONPath: "requiredInt", + Name: "required_int", + ProtoTypeName: DataTypeInt32, + JSONPath: "requiredInt", }, }, }, @@ -369,55 +369,55 @@ func TestNullableFieldsExecutionPlan(t *testing.T) { Name: "MutationCreateNullableFieldsTypeResponse", Fields: []RPCField{ { - Name: "create_nullable_fields_type", - TypeName: string(DataTypeMessage), - JSONPath: "createNullableFieldsType", + Name: "create_nullable_fields_type", + ProtoTypeName: DataTypeMessage, + JSONPath: "createNullableFieldsType", Message: &RPCMessage{ Name: "NullableFieldsType", Fields: []RPCField{ { - Name: "id", - TypeName: string(DataTypeString), - JSONPath: "id", + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", }, { - Name: "name", - TypeName: string(DataTypeString), - JSONPath: "name", + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", }, { - Name: "optional_string", - TypeName: string(DataTypeString), - JSONPath: "optionalString", - Optional: true, + Name: "optional_string", + ProtoTypeName: DataTypeString, + JSONPath: "optionalString", + Optional: true, }, { - Name: "optional_int", - TypeName: string(DataTypeInt32), - JSONPath: "optionalInt", - Optional: true, + Name: "optional_int", + ProtoTypeName: DataTypeInt32, + JSONPath: "optionalInt", + Optional: true, }, { - Name: "optional_float", - TypeName: string(DataTypeDouble), - JSONPath: "optionalFloat", - Optional: true, + Name: "optional_float", + ProtoTypeName: DataTypeDouble, + JSONPath: "optionalFloat", + Optional: true, }, { - Name: "optional_boolean", - TypeName: string(DataTypeBool), - JSONPath: "optionalBoolean", - Optional: true, + Name: "optional_boolean", + ProtoTypeName: DataTypeBool, + JSONPath: "optionalBoolean", + Optional: true, }, { - Name: "required_string", - TypeName: string(DataTypeString), - JSONPath: "requiredString", + Name: "required_string", + ProtoTypeName: DataTypeString, + JSONPath: "requiredString", }, { - Name: "required_int", - TypeName: string(DataTypeInt32), - JSONPath: "requiredInt", + Name: "required_int", + ProtoTypeName: DataTypeInt32, + JSONPath: "requiredInt", }, }, }, @@ -440,55 +440,55 @@ func TestNullableFieldsExecutionPlan(t *testing.T) { Name: "MutationUpdateNullableFieldsTypeRequest", Fields: []RPCField{ { - Name: "id", - TypeName: string(DataTypeString), - JSONPath: "id", + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", }, { - Name: "input", - TypeName: string(DataTypeMessage), - JSONPath: "input", + Name: "input", + ProtoTypeName: DataTypeMessage, + JSONPath: "input", Message: &RPCMessage{ Name: "NullableFieldsInput", Fields: []RPCField{ { - Name: "name", - TypeName: string(DataTypeString), - JSONPath: "name", + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", }, { - Name: "optional_string", - TypeName: string(DataTypeString), - JSONPath: "optionalString", - Optional: true, + Name: "optional_string", + ProtoTypeName: DataTypeString, + JSONPath: "optionalString", + Optional: true, }, { - Name: "optional_int", - TypeName: string(DataTypeInt32), - JSONPath: "optionalInt", - Optional: true, + Name: "optional_int", + ProtoTypeName: DataTypeInt32, + JSONPath: "optionalInt", + Optional: true, }, { - Name: "optional_float", - TypeName: string(DataTypeDouble), - JSONPath: "optionalFloat", - Optional: true, + Name: "optional_float", + ProtoTypeName: DataTypeDouble, + JSONPath: "optionalFloat", + Optional: true, }, { - Name: "optional_boolean", - TypeName: string(DataTypeBool), - JSONPath: "optionalBoolean", - Optional: true, + Name: "optional_boolean", + ProtoTypeName: DataTypeBool, + JSONPath: "optionalBoolean", + Optional: true, }, { - Name: "required_string", - TypeName: string(DataTypeString), - JSONPath: "requiredString", + Name: "required_string", + ProtoTypeName: DataTypeString, + JSONPath: "requiredString", }, { - Name: "required_int", - TypeName: string(DataTypeInt32), - JSONPath: "requiredInt", + Name: "required_int", + ProtoTypeName: DataTypeInt32, + JSONPath: "requiredInt", }, }, }, @@ -499,33 +499,33 @@ func TestNullableFieldsExecutionPlan(t *testing.T) { Name: "MutationUpdateNullableFieldsTypeResponse", Fields: []RPCField{ { - Name: "update_nullable_fields_type", - TypeName: string(DataTypeMessage), - JSONPath: "updateNullableFieldsType", - Optional: true, + Name: "update_nullable_fields_type", + ProtoTypeName: DataTypeMessage, + JSONPath: "updateNullableFieldsType", + Optional: true, Message: &RPCMessage{ Name: "NullableFieldsType", Fields: []RPCField{ { - Name: "id", - TypeName: string(DataTypeString), - JSONPath: "id", + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", }, { - Name: "name", - TypeName: string(DataTypeString), - JSONPath: "name", + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", }, { - Name: "optional_string", - TypeName: string(DataTypeString), - JSONPath: "optionalString", - Optional: true, + Name: "optional_string", + ProtoTypeName: DataTypeString, + JSONPath: "optionalString", + Optional: true, }, { - Name: "required_string", - TypeName: string(DataTypeString), - JSONPath: "requiredString", + Name: "required_string", + ProtoTypeName: DataTypeString, + JSONPath: "requiredString", }, }, }, @@ -551,22 +551,22 @@ func TestNullableFieldsExecutionPlan(t *testing.T) { Name: "QueryNullableFieldsTypeResponse", Fields: []RPCField{ { - Name: "nullable_fields_type", - TypeName: string(DataTypeMessage), - JSONPath: "nullableFieldsType", + Name: "nullable_fields_type", + ProtoTypeName: DataTypeMessage, + JSONPath: "nullableFieldsType", Message: &RPCMessage{ Name: "NullableFieldsType", Fields: []RPCField{ { - Name: "id", - TypeName: string(DataTypeString), - JSONPath: "id", + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", }, { - Name: "optional_string", - TypeName: string(DataTypeString), - JSONPath: "optionalString", - Optional: true, + Name: "optional_string", + ProtoTypeName: DataTypeString, + JSONPath: "optionalString", + Optional: true, }, }, }, @@ -592,35 +592,35 @@ func TestNullableFieldsExecutionPlan(t *testing.T) { Name: "QueryNullableFieldsTypeResponse", Fields: []RPCField{ { - Name: "nullable_fields_type", - TypeName: string(DataTypeMessage), - JSONPath: "nullableFieldsType", + Name: "nullable_fields_type", + ProtoTypeName: DataTypeMessage, + JSONPath: "nullableFieldsType", Message: &RPCMessage{ Name: "NullableFieldsType", Fields: []RPCField{ { - Name: "optional_string", - TypeName: string(DataTypeString), - JSONPath: "optionalString", - Optional: true, + Name: "optional_string", + ProtoTypeName: DataTypeString, + JSONPath: "optionalString", + Optional: true, }, { - Name: "optional_int", - TypeName: string(DataTypeInt32), - JSONPath: "optionalInt", - Optional: true, + Name: "optional_int", + ProtoTypeName: DataTypeInt32, + JSONPath: "optionalInt", + Optional: true, }, { - Name: "optional_float", - TypeName: string(DataTypeDouble), - JSONPath: "optionalFloat", - Optional: true, + Name: "optional_float", + ProtoTypeName: DataTypeDouble, + JSONPath: "optionalFloat", + Optional: true, }, { - Name: "optional_boolean", - TypeName: string(DataTypeBool), - JSONPath: "optionalBoolean", - Optional: true, + Name: "optional_boolean", + ProtoTypeName: DataTypeBool, + JSONPath: "optionalBoolean", + Optional: true, }, }, }, diff --git a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_test.go b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_test.go index d02860805..20c9e9a19 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_test.go +++ b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_test.go @@ -100,28 +100,28 @@ func TestQueryExecutionPlans(t *testing.T) { Name: "QueryUsersResponse", Fields: []RPCField{ { - Name: "users", - TypeName: string(DataTypeMessage), - Repeated: true, - JSONPath: "users", + Name: "users", + ProtoTypeName: DataTypeMessage, + Repeated: true, + JSONPath: "users", Message: &RPCMessage{ Name: "User", Fields: []RPCField{ { - Name: "__typename", - TypeName: string(DataTypeString), - JSONPath: "__typename", - StaticValue: "User", + Name: "__typename", + ProtoTypeName: DataTypeString, + JSONPath: "__typename", + StaticValue: "User", }, { - Name: "id", - TypeName: string(DataTypeString), - JSONPath: "id", + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", }, { - Name: "name", - TypeName: string(DataTypeString), - JSONPath: "name", + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", }, }, }, @@ -151,22 +151,22 @@ func TestQueryExecutionPlans(t *testing.T) { Name: "QueryUsersResponse", Fields: []RPCField{ { - Name: "users", - TypeName: string(DataTypeMessage), - Repeated: true, - JSONPath: "users", + Name: "users", + ProtoTypeName: DataTypeMessage, + Repeated: true, + JSONPath: "users", Message: &RPCMessage{ Name: "User", Fields: []RPCField{ { - Name: "id", - TypeName: string(DataTypeString), - JSONPath: "id", + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", }, { - Name: "name", - TypeName: string(DataTypeString), - JSONPath: "name", + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", }, }, }, @@ -181,9 +181,9 @@ func TestQueryExecutionPlans(t *testing.T) { Name: "QueryUserRequest", Fields: []RPCField{ { - Name: "id", - TypeName: string(DataTypeString), - JSONPath: "id", + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", }, }, }, @@ -191,22 +191,22 @@ func TestQueryExecutionPlans(t *testing.T) { Name: "QueryUserResponse", Fields: []RPCField{ { - Name: "user", - TypeName: string(DataTypeMessage), - JSONPath: "user", - Optional: true, + Name: "user", + ProtoTypeName: DataTypeMessage, + JSONPath: "user", + Optional: true, Message: &RPCMessage{ Name: "User", Fields: []RPCField{ { - Name: "id", - TypeName: string(DataTypeString), - JSONPath: "id", + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", }, { - Name: "name", - TypeName: string(DataTypeString), - JSONPath: "name", + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", }, }, }, @@ -230,14 +230,14 @@ func TestQueryExecutionPlans(t *testing.T) { Name: "QueryTypeFilterWithArgumentsRequest", Fields: []RPCField{ { - Name: "filter_field_1", - TypeName: string(DataTypeString), - JSONPath: "filterField1", + Name: "filter_field_1", + ProtoTypeName: DataTypeString, + JSONPath: "filterField1", }, { - Name: "filter_field_2", - TypeName: string(DataTypeString), - JSONPath: "filterField2", + Name: "filter_field_2", + ProtoTypeName: DataTypeString, + JSONPath: "filterField2", }, }, }, @@ -245,32 +245,32 @@ func TestQueryExecutionPlans(t *testing.T) { Name: "QueryTypeFilterWithArgumentsResponse", Fields: []RPCField{ { - Name: "type_filter_with_arguments", - TypeName: string(DataTypeMessage), - Repeated: true, - JSONPath: "typeFilterWithArguments", + Name: "type_filter_with_arguments", + ProtoTypeName: DataTypeMessage, + Repeated: true, + JSONPath: "typeFilterWithArguments", Message: &RPCMessage{ Name: "TypeWithMultipleFilterFields", Fields: []RPCField{ { - Name: "id", - TypeName: string(DataTypeString), - JSONPath: "id", + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", }, { - Name: "name", - TypeName: string(DataTypeString), - JSONPath: "name", + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", }, { - Name: "filter_field_1", - TypeName: string(DataTypeString), - JSONPath: "filterField1", + Name: "filter_field_1", + ProtoTypeName: DataTypeString, + JSONPath: "filterField1", }, { - Name: "filter_field_2", - TypeName: string(DataTypeString), - JSONPath: "filterField2", + Name: "filter_field_2", + ProtoTypeName: DataTypeString, + JSONPath: "filterField2", }, }, }, @@ -294,51 +294,51 @@ func TestQueryExecutionPlans(t *testing.T) { Name: "QueryComplexFilterTypeRequest", Fields: []RPCField{ { - Name: "filter", - TypeName: string(DataTypeMessage), - JSONPath: "filter", + Name: "filter", + ProtoTypeName: DataTypeMessage, + JSONPath: "filter", Message: &RPCMessage{ Name: "ComplexFilterTypeInput", Fields: []RPCField{ { - Name: "filter", - TypeName: string(DataTypeMessage), - JSONPath: "filter", + Name: "filter", + ProtoTypeName: DataTypeMessage, + JSONPath: "filter", Message: &RPCMessage{ Name: "FilterType", Fields: []RPCField{ { - Name: "name", - TypeName: string(DataTypeString), - JSONPath: "name", + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", }, { - Name: "filter_field_1", - TypeName: string(DataTypeString), - JSONPath: "filterField1", + Name: "filter_field_1", + ProtoTypeName: DataTypeString, + JSONPath: "filterField1", }, { - Name: "filter_field_2", - TypeName: string(DataTypeString), - JSONPath: "filterField2", + Name: "filter_field_2", + ProtoTypeName: DataTypeString, + JSONPath: "filterField2", }, { - Name: "pagination", - TypeName: string(DataTypeMessage), - JSONPath: "pagination", - Optional: true, + Name: "pagination", + ProtoTypeName: DataTypeMessage, + JSONPath: "pagination", + Optional: true, Message: &RPCMessage{ Name: "Pagination", Fields: []RPCField{ { - Name: "page", - TypeName: string(DataTypeInt32), - JSONPath: "page", + Name: "page", + ProtoTypeName: DataTypeInt32, + JSONPath: "page", }, { - Name: "per_page", - TypeName: string(DataTypeInt32), - JSONPath: "perPage", + Name: "per_page", + ProtoTypeName: DataTypeInt32, + JSONPath: "perPage", }, }, }, @@ -355,22 +355,22 @@ func TestQueryExecutionPlans(t *testing.T) { Name: "QueryComplexFilterTypeResponse", Fields: []RPCField{ { - Repeated: true, - Name: "complex_filter_type", - TypeName: string(DataTypeMessage), - JSONPath: "complexFilterType", + Repeated: true, + Name: "complex_filter_type", + ProtoTypeName: DataTypeMessage, + JSONPath: "complexFilterType", Message: &RPCMessage{ Name: "TypeWithComplexFilterInput", Fields: []RPCField{ { - Name: "id", - TypeName: string(DataTypeString), - JSONPath: "id", + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", }, { - Name: "name", - TypeName: string(DataTypeString), - JSONPath: "name", + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", }, }, }, @@ -394,51 +394,51 @@ func TestQueryExecutionPlans(t *testing.T) { Name: "QueryComplexFilterTypeRequest", Fields: []RPCField{ { - Name: "filter", - TypeName: string(DataTypeMessage), - JSONPath: "filter", + Name: "filter", + ProtoTypeName: DataTypeMessage, + JSONPath: "filter", Message: &RPCMessage{ Name: "ComplexFilterTypeInput", Fields: []RPCField{ { - Name: "filter", - TypeName: string(DataTypeMessage), - JSONPath: "filter", + Name: "filter", + ProtoTypeName: DataTypeMessage, + JSONPath: "filter", Message: &RPCMessage{ Name: "FilterType", Fields: []RPCField{ { - Name: "name", - TypeName: string(DataTypeString), - JSONPath: "name", + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", }, { - Name: "filter_field_1", - TypeName: string(DataTypeString), - JSONPath: "filterField1", + Name: "filter_field_1", + ProtoTypeName: DataTypeString, + JSONPath: "filterField1", }, { - Name: "filter_field_2", - TypeName: string(DataTypeString), - JSONPath: "filterField2", + Name: "filter_field_2", + ProtoTypeName: DataTypeString, + JSONPath: "filterField2", }, { - Name: "pagination", - TypeName: string(DataTypeMessage), - JSONPath: "pagination", - Optional: true, + Name: "pagination", + ProtoTypeName: DataTypeMessage, + JSONPath: "pagination", + Optional: true, Message: &RPCMessage{ Name: "Pagination", Fields: []RPCField{ { - Name: "page", - TypeName: string(DataTypeInt32), - JSONPath: "page", + Name: "page", + ProtoTypeName: DataTypeInt32, + JSONPath: "page", }, { - Name: "per_page", - TypeName: string(DataTypeInt32), - JSONPath: "perPage", + Name: "per_page", + ProtoTypeName: DataTypeInt32, + JSONPath: "perPage", }, }, }, @@ -455,22 +455,22 @@ func TestQueryExecutionPlans(t *testing.T) { Name: "QueryComplexFilterTypeResponse", Fields: []RPCField{ { - Repeated: true, - Name: "complex_filter_type", - TypeName: string(DataTypeMessage), - JSONPath: "complexFilterType", + Repeated: true, + Name: "complex_filter_type", + ProtoTypeName: DataTypeMessage, + JSONPath: "complexFilterType", Message: &RPCMessage{ Name: "TypeWithComplexFilterInput", Fields: []RPCField{ { - Name: "id", - TypeName: string(DataTypeString), - JSONPath: "id", + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", }, { - Name: "name", - TypeName: string(DataTypeString), - JSONPath: "name", + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", }, }, }, @@ -495,51 +495,51 @@ func TestQueryExecutionPlans(t *testing.T) { Name: "QueryComplexFilterTypeRequest", Fields: []RPCField{ { - Name: "filter", - TypeName: string(DataTypeMessage), - JSONPath: "foobar", + Name: "filter", + ProtoTypeName: DataTypeMessage, + JSONPath: "foobar", Message: &RPCMessage{ Name: "ComplexFilterTypeInput", Fields: []RPCField{ { - Name: "filter", - TypeName: string(DataTypeMessage), - JSONPath: "filter", + Name: "filter", + ProtoTypeName: DataTypeMessage, + JSONPath: "filter", Message: &RPCMessage{ Name: "FilterType", Fields: []RPCField{ { - Name: "name", - TypeName: string(DataTypeString), - JSONPath: "name", + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", }, { - Name: "filter_field_1", - TypeName: string(DataTypeString), - JSONPath: "filterField1", + Name: "filter_field_1", + ProtoTypeName: DataTypeString, + JSONPath: "filterField1", }, { - Name: "filter_field_2", - TypeName: string(DataTypeString), - JSONPath: "filterField2", + Name: "filter_field_2", + ProtoTypeName: DataTypeString, + JSONPath: "filterField2", }, { - Name: "pagination", - TypeName: string(DataTypeMessage), - JSONPath: "pagination", - Optional: true, + Name: "pagination", + ProtoTypeName: DataTypeMessage, + JSONPath: "pagination", + Optional: true, Message: &RPCMessage{ Name: "Pagination", Fields: []RPCField{ { - Name: "page", - TypeName: string(DataTypeInt32), - JSONPath: "page", + Name: "page", + ProtoTypeName: DataTypeInt32, + JSONPath: "page", }, { - Name: "per_page", - TypeName: string(DataTypeInt32), - JSONPath: "perPage", + Name: "per_page", + ProtoTypeName: DataTypeInt32, + JSONPath: "perPage", }, }, }, @@ -556,22 +556,22 @@ func TestQueryExecutionPlans(t *testing.T) { Name: "QueryComplexFilterTypeResponse", Fields: []RPCField{ { - Repeated: true, - Name: "complex_filter_type", - TypeName: string(DataTypeMessage), - JSONPath: "complexFilterType", + Repeated: true, + Name: "complex_filter_type", + ProtoTypeName: DataTypeMessage, + JSONPath: "complexFilterType", Message: &RPCMessage{ Name: "TypeWithComplexFilterInput", Fields: []RPCField{ { - Name: "id", - TypeName: string(DataTypeString), - JSONPath: "id", + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", }, { - Name: "name", - TypeName: string(DataTypeString), - JSONPath: "name", + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", }, }, }, @@ -595,23 +595,23 @@ func TestQueryExecutionPlans(t *testing.T) { Name: "QueryTypeWithMultipleFilterFieldsRequest", Fields: []RPCField{ { - Name: "filter", - TypeName: string(DataTypeMessage), - JSONPath: "filter", + Name: "filter", + ProtoTypeName: DataTypeMessage, + JSONPath: "filter", Message: &RPCMessage{ Name: "FilterTypeInput", Fields: []RPCField{ { - Repeated: false, - Name: "filter_field_1", - TypeName: string(DataTypeString), - JSONPath: "filterField1", + Repeated: false, + Name: "filter_field_1", + ProtoTypeName: DataTypeString, + JSONPath: "filterField1", }, { - Repeated: false, - Name: "filter_field_2", - TypeName: string(DataTypeString), - JSONPath: "filterField2", + Repeated: false, + Name: "filter_field_2", + ProtoTypeName: DataTypeString, + JSONPath: "filterField2", }, }, }, @@ -622,22 +622,22 @@ func TestQueryExecutionPlans(t *testing.T) { Name: "QueryTypeWithMultipleFilterFieldsResponse", Fields: []RPCField{ { - Name: "type_with_multiple_filter_fields", - TypeName: string(DataTypeMessage), - Repeated: true, - JSONPath: "typeWithMultipleFilterFields", + Name: "type_with_multiple_filter_fields", + ProtoTypeName: DataTypeMessage, + Repeated: true, + JSONPath: "typeWithMultipleFilterFields", Message: &RPCMessage{ Name: "TypeWithMultipleFilterFields", Fields: []RPCField{ { - Name: "id", - TypeName: string(DataTypeString), - JSONPath: "id", + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", }, { - Name: "name", - TypeName: string(DataTypeString), - JSONPath: "name", + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", }, }, }, @@ -664,22 +664,22 @@ func TestQueryExecutionPlans(t *testing.T) { Name: "QueryUsersResponse", Fields: []RPCField{ { - Name: "users", - TypeName: string(DataTypeMessage), - Repeated: true, - JSONPath: "users", + Name: "users", + ProtoTypeName: DataTypeMessage, + Repeated: true, + JSONPath: "users", Message: &RPCMessage{ Name: "User", Fields: []RPCField{ { - Name: "id", - TypeName: string(DataTypeString), - JSONPath: "id", + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", }, { - Name: "name", - TypeName: string(DataTypeString), - JSONPath: "name", + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", }, }, }, @@ -717,9 +717,9 @@ func TestQueryExecutionPlans(t *testing.T) { Name: "QueryUserRequest", Fields: []RPCField{ { - Name: "id", - TypeName: string(DataTypeString), - JSONPath: "id", + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", }, }, }, @@ -727,22 +727,22 @@ func TestQueryExecutionPlans(t *testing.T) { Name: "QueryUserResponse", Fields: []RPCField{ { - Name: "user", - TypeName: string(DataTypeMessage), - JSONPath: "user", - Optional: true, + Name: "user", + ProtoTypeName: DataTypeMessage, + JSONPath: "user", + Optional: true, Message: &RPCMessage{ Name: "User", Fields: []RPCField{ { - Name: "id", - TypeName: string(DataTypeString), - JSONPath: "id", + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", }, { - Name: "name", - TypeName: string(DataTypeString), - JSONPath: "name", + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", }, }, }, @@ -769,56 +769,56 @@ func TestQueryExecutionPlans(t *testing.T) { Name: "QueryNestedTypeResponse", Fields: []RPCField{ { - Name: "nested_type", - TypeName: string(DataTypeMessage), - Repeated: true, - JSONPath: "nestedType", + Name: "nested_type", + ProtoTypeName: DataTypeMessage, + Repeated: true, + JSONPath: "nestedType", Message: &RPCMessage{ Name: "NestedTypeA", Fields: []RPCField{ { - Name: "id", - TypeName: string(DataTypeString), - JSONPath: "id", + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", }, { - Name: "name", - TypeName: string(DataTypeString), - JSONPath: "name", + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", }, { - Name: "b", - TypeName: string(DataTypeMessage), - JSONPath: "b", + Name: "b", + ProtoTypeName: DataTypeMessage, + JSONPath: "b", Message: &RPCMessage{ Name: "NestedTypeB", Fields: []RPCField{ { - Name: "id", - TypeName: string(DataTypeString), - JSONPath: "id", + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", }, { - Name: "name", - TypeName: string(DataTypeString), - JSONPath: "name", + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", }, { - Name: "c", - TypeName: string(DataTypeMessage), - JSONPath: "c", + Name: "c", + ProtoTypeName: DataTypeMessage, + JSONPath: "c", Message: &RPCMessage{ Name: "NestedTypeC", Fields: []RPCField{ { - Name: "id", - TypeName: string(DataTypeString), - JSONPath: "id", + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", }, { - Name: "name", - TypeName: string(DataTypeString), - JSONPath: "name", + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", }, }, }, @@ -851,67 +851,67 @@ func TestQueryExecutionPlans(t *testing.T) { Name: "QueryRecursiveTypeResponse", Fields: []RPCField{ { - Name: "recursive_type", - TypeName: string(DataTypeMessage), - JSONPath: "recursiveType", + Name: "recursive_type", + ProtoTypeName: DataTypeMessage, + JSONPath: "recursiveType", Message: &RPCMessage{ Name: "RecursiveType", Fields: []RPCField{ { - Name: "id", - TypeName: string(DataTypeString), - JSONPath: "id", + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", }, { - Name: "name", - TypeName: string(DataTypeString), - JSONPath: "name", + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", }, { - Name: "recursive_type", - TypeName: string(DataTypeMessage), - JSONPath: "recursiveType", + Name: "recursive_type", + ProtoTypeName: DataTypeMessage, + JSONPath: "recursiveType", Message: &RPCMessage{ Name: "RecursiveType", Fields: []RPCField{ { - Name: "id", - TypeName: string(DataTypeString), - JSONPath: "id", + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", }, { - Name: "recursive_type", - TypeName: string(DataTypeMessage), - JSONPath: "recursiveType", + Name: "recursive_type", + ProtoTypeName: DataTypeMessage, + JSONPath: "recursiveType", Message: &RPCMessage{ Name: "RecursiveType", Fields: []RPCField{ { - Name: "id", - TypeName: string(DataTypeString), - JSONPath: "id", + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", }, { - Name: "name", - TypeName: string(DataTypeString), - JSONPath: "name", + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", }, { - Name: "recursive_type", - TypeName: string(DataTypeMessage), - JSONPath: "recursiveType", + Name: "recursive_type", + ProtoTypeName: DataTypeMessage, + JSONPath: "recursiveType", Message: &RPCMessage{ Name: "RecursiveType", Fields: []RPCField{ { - Name: "id", - TypeName: string(DataTypeString), - JSONPath: "id", + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", }, { - Name: "name", - TypeName: string(DataTypeString), - JSONPath: "name", + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", }, }, }, @@ -920,9 +920,9 @@ func TestQueryExecutionPlans(t *testing.T) { }, }, { - Name: "name", - TypeName: string(DataTypeString), - JSONPath: "name", + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", }, }, }, @@ -1010,48 +1010,48 @@ func TestQueryExecutionPlans(t *testing.T) { Name: "QueryCalculateTotalsRequest", Fields: []RPCField{ { - Name: "orders", - TypeName: string(DataTypeMessage), - JSONPath: "orders", - Repeated: true, + Name: "orders", + ProtoTypeName: DataTypeMessage, + JSONPath: "orders", + Repeated: true, Message: &RPCMessage{ Name: "OrderInput", Fields: []RPCField{ { - Name: "order_id", - TypeName: string(DataTypeString), - JSONPath: "orderId", + Name: "order_id", + ProtoTypeName: DataTypeString, + JSONPath: "orderId", }, { - Name: "customer_name", - TypeName: string(DataTypeString), - JSONPath: "customerName", + Name: "customer_name", + ProtoTypeName: DataTypeString, + JSONPath: "customerName", }, { - Name: "lines", - TypeName: string(DataTypeMessage), - JSONPath: "lines", - Repeated: true, + Name: "lines", + ProtoTypeName: DataTypeMessage, + JSONPath: "lines", + Repeated: true, Message: &RPCMessage{ Name: "OrderLineInput", Fields: []RPCField{ { - Name: "product_id", - TypeName: string(DataTypeString), - JSONPath: "productId", + Name: "product_id", + ProtoTypeName: DataTypeString, + JSONPath: "productId", }, { - Name: "quantity", - TypeName: string(DataTypeInt32), - JSONPath: "quantity", + Name: "quantity", + ProtoTypeName: DataTypeInt32, + JSONPath: "quantity", }, { - Name: "modifiers", - TypeName: string(DataTypeString), - Repeated: false, - Optional: true, - IsListType: true, - JSONPath: "modifiers", + Name: "modifiers", + ProtoTypeName: DataTypeString, + Repeated: false, + Optional: true, + IsListType: true, + JSONPath: "modifiers", ListMetadata: &ListMetadata{ NestingLevel: 1, LevelInfo: []LevelInfo{ @@ -1073,27 +1073,27 @@ func TestQueryExecutionPlans(t *testing.T) { Name: "QueryCalculateTotalsResponse", Fields: []RPCField{ { - Name: "calculate_totals", - TypeName: string(DataTypeMessage), - JSONPath: "calculateTotals", - Repeated: true, + Name: "calculate_totals", + ProtoTypeName: DataTypeMessage, + JSONPath: "calculateTotals", + Repeated: true, Message: &RPCMessage{ Name: "Order", Fields: []RPCField{ { - Name: "order_id", - TypeName: string(DataTypeString), - JSONPath: "orderId", + Name: "order_id", + ProtoTypeName: DataTypeString, + JSONPath: "orderId", }, { - Name: "customer_name", - TypeName: string(DataTypeString), - JSONPath: "customerName", + Name: "customer_name", + ProtoTypeName: DataTypeString, + JSONPath: "customerName", }, { - Name: "total_items", - TypeName: string(DataTypeInt32), - JSONPath: "totalItems", + Name: "total_items", + ProtoTypeName: DataTypeInt32, + JSONPath: "totalItems", }, }, }, @@ -1159,10 +1159,10 @@ func TestProductExecutionPlan(t *testing.T) { Name: "QueryCategoriesByKindRequest", Fields: []RPCField{ { - Name: "kind", - TypeName: string(DataTypeEnum), - JSONPath: "kind", - EnumName: "CategoryKind", + Name: "kind", + ProtoTypeName: DataTypeEnum, + JSONPath: "kind", + EnumName: "CategoryKind", }, }, }, @@ -1170,28 +1170,28 @@ func TestProductExecutionPlan(t *testing.T) { Name: "QueryCategoriesByKindResponse", Fields: []RPCField{ { - Name: "categories_by_kind", - TypeName: string(DataTypeMessage), - JSONPath: "categoriesByKind", - Repeated: true, + Name: "categories_by_kind", + ProtoTypeName: DataTypeMessage, + JSONPath: "categoriesByKind", + Repeated: true, Message: &RPCMessage{ Name: "Category", Fields: []RPCField{ { - Name: "id", - TypeName: string(DataTypeString), - JSONPath: "id", + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", }, { - Name: "name", - TypeName: string(DataTypeString), - JSONPath: "name", + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", }, { - Name: "kind", - TypeName: string(DataTypeEnum), - JSONPath: "kind", - EnumName: "CategoryKind", + Name: "kind", + ProtoTypeName: DataTypeEnum, + JSONPath: "kind", + EnumName: "CategoryKind", }, }, }, @@ -1214,11 +1214,11 @@ func TestProductExecutionPlan(t *testing.T) { Name: "QueryCategoriesByKindsRequest", Fields: []RPCField{ { - Name: "kinds", - TypeName: string(DataTypeEnum), - JSONPath: "kinds", - EnumName: "CategoryKind", - Repeated: true, + Name: "kinds", + ProtoTypeName: DataTypeEnum, + JSONPath: "kinds", + EnumName: "CategoryKind", + Repeated: true, }, }, }, @@ -1226,28 +1226,28 @@ func TestProductExecutionPlan(t *testing.T) { Name: "QueryCategoriesByKindsResponse", Fields: []RPCField{ { - Name: "categories_by_kinds", - TypeName: string(DataTypeMessage), - JSONPath: "categoriesByKinds", - Repeated: true, + Name: "categories_by_kinds", + ProtoTypeName: DataTypeMessage, + JSONPath: "categoriesByKinds", + Repeated: true, Message: &RPCMessage{ Name: "Category", Fields: []RPCField{ { - Name: "id", - TypeName: string(DataTypeString), - JSONPath: "id", + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", }, { - Name: "name", - TypeName: string(DataTypeString), - JSONPath: "name", + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", }, { - Name: "kind", - TypeName: string(DataTypeEnum), - JSONPath: "kind", - EnumName: "CategoryKind", + Name: "kind", + ProtoTypeName: DataTypeEnum, + JSONPath: "kind", + EnumName: "CategoryKind", }, }, }, @@ -1270,35 +1270,35 @@ func TestProductExecutionPlan(t *testing.T) { Name: "QueryFilterCategoriesRequest", Fields: []RPCField{ { - Name: "filter", - TypeName: string(DataTypeMessage), - JSONPath: "filter", + Name: "filter", + ProtoTypeName: DataTypeMessage, + JSONPath: "filter", Message: &RPCMessage{ Name: "CategoryFilter", Fields: []RPCField{ { - Name: "category", - TypeName: string(DataTypeEnum), - JSONPath: "category", - EnumName: "CategoryKind", + Name: "category", + ProtoTypeName: DataTypeEnum, + JSONPath: "category", + EnumName: "CategoryKind", }, { - Name: "pagination", - TypeName: string(DataTypeMessage), - JSONPath: "pagination", - Optional: true, + Name: "pagination", + ProtoTypeName: DataTypeMessage, + JSONPath: "pagination", + Optional: true, Message: &RPCMessage{ Name: "Pagination", Fields: []RPCField{ { - Name: "page", - TypeName: string(DataTypeInt32), - JSONPath: "page", + Name: "page", + ProtoTypeName: DataTypeInt32, + JSONPath: "page", }, { - Name: "per_page", - TypeName: string(DataTypeInt32), - JSONPath: "perPage", + Name: "per_page", + ProtoTypeName: DataTypeInt32, + JSONPath: "perPage", }, }, }, @@ -1312,28 +1312,28 @@ func TestProductExecutionPlan(t *testing.T) { Name: "QueryFilterCategoriesResponse", Fields: []RPCField{ { - Name: "filter_categories", - TypeName: string(DataTypeMessage), - JSONPath: "filterCategories", - Repeated: true, + Name: "filter_categories", + ProtoTypeName: DataTypeMessage, + JSONPath: "filterCategories", + Repeated: true, Message: &RPCMessage{ Name: "Category", Fields: []RPCField{ { - Name: "id", - TypeName: string(DataTypeString), - JSONPath: "id", + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", }, { - Name: "name", - TypeName: string(DataTypeString), - JSONPath: "name", + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", }, { - Name: "kind", - TypeName: string(DataTypeEnum), - JSONPath: "kind", - EnumName: "CategoryKind", + Name: "kind", + ProtoTypeName: DataTypeEnum, + JSONPath: "kind", + EnumName: "CategoryKind", }, }, }, @@ -1417,23 +1417,23 @@ func TestProductExecutionPlanWithAliases(t *testing.T) { Name: "QueryUsersResponse", Fields: RPCFields{ { - Name: "users", - TypeName: string(DataTypeMessage), - JSONPath: "users", - Alias: "foo", - Repeated: true, + Name: "users", + ProtoTypeName: DataTypeMessage, + JSONPath: "users", + Alias: "foo", + Repeated: true, Message: &RPCMessage{ Name: "User", Fields: RPCFields{ { - Name: "id", - TypeName: string(DataTypeString), - JSONPath: "id", + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", }, { - Name: "name", - TypeName: string(DataTypeString), - JSONPath: "name", + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", }, }, }, @@ -1456,9 +1456,9 @@ func TestProductExecutionPlanWithAliases(t *testing.T) { Name: "QueryUserRequest", Fields: []RPCField{ { - Name: "id", - TypeName: string(DataTypeString), - JSONPath: "id", + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", }, }, }, @@ -1466,25 +1466,25 @@ func TestProductExecutionPlanWithAliases(t *testing.T) { Name: "QueryUserResponse", Fields: RPCFields{ { - Name: "user", - TypeName: string(DataTypeMessage), - JSONPath: "user", - Alias: "specificUser", - Optional: true, + Name: "user", + ProtoTypeName: DataTypeMessage, + JSONPath: "user", + Alias: "specificUser", + Optional: true, Message: &RPCMessage{ Name: "User", Fields: RPCFields{ { - Name: "id", - TypeName: string(DataTypeString), - JSONPath: "id", - Alias: "userId", + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", + Alias: "userId", }, { - Name: "name", - TypeName: string(DataTypeString), - JSONPath: "name", - Alias: "userName", + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", + Alias: "userName", }, }, }, @@ -1510,23 +1510,23 @@ func TestProductExecutionPlanWithAliases(t *testing.T) { Name: "QueryUsersResponse", Fields: RPCFields{ { - Name: "users", - TypeName: string(DataTypeMessage), - JSONPath: "users", - Alias: "allUsers", - Repeated: true, + Name: "users", + ProtoTypeName: DataTypeMessage, + JSONPath: "users", + Alias: "allUsers", + Repeated: true, Message: &RPCMessage{ Name: "User", Fields: RPCFields{ { - Name: "id", - TypeName: string(DataTypeString), - JSONPath: "id", + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", }, { - Name: "name", - TypeName: string(DataTypeString), - JSONPath: "name", + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", }, }, }, @@ -1544,30 +1544,30 @@ func TestProductExecutionPlanWithAliases(t *testing.T) { Name: "QueryCategoriesResponse", Fields: RPCFields{ { - Name: "categories", - TypeName: string(DataTypeMessage), - JSONPath: "categories", - Alias: "allCategories", - Repeated: true, + Name: "categories", + ProtoTypeName: DataTypeMessage, + JSONPath: "categories", + Alias: "allCategories", + Repeated: true, Message: &RPCMessage{ Name: "Category", Fields: RPCFields{ { - Name: "id", - TypeName: string(DataTypeString), - JSONPath: "id", + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", }, { - Name: "name", - TypeName: string(DataTypeString), - JSONPath: "name", + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", }, { - Name: "kind", - TypeName: string(DataTypeEnum), - JSONPath: "kind", - Alias: "categoryType", - EnumName: "CategoryKind", + Name: "kind", + ProtoTypeName: DataTypeEnum, + JSONPath: "kind", + Alias: "categoryType", + EnumName: "CategoryKind", }, }, }, @@ -1593,65 +1593,65 @@ func TestProductExecutionPlanWithAliases(t *testing.T) { Name: "QueryNestedTypeResponse", Fields: RPCFields{ { - Name: "nested_type", - TypeName: string(DataTypeMessage), - JSONPath: "nestedType", - Alias: "nestedData", - Repeated: true, + Name: "nested_type", + ProtoTypeName: DataTypeMessage, + JSONPath: "nestedType", + Alias: "nestedData", + Repeated: true, Message: &RPCMessage{ Name: "NestedTypeA", Fields: RPCFields{ { - Name: "id", - TypeName: string(DataTypeString), - JSONPath: "id", - Alias: "identifier", + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", + Alias: "identifier", }, { - Name: "name", - TypeName: string(DataTypeString), - JSONPath: "name", - Alias: "title", + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", + Alias: "title", }, { - Name: "b", - TypeName: string(DataTypeMessage), - JSONPath: "b", - Alias: "childB", + Name: "b", + ProtoTypeName: DataTypeMessage, + JSONPath: "b", + Alias: "childB", Message: &RPCMessage{ Name: "NestedTypeB", Fields: RPCFields{ { - Name: "id", - TypeName: string(DataTypeString), - JSONPath: "id", - Alias: "identifier", + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", + Alias: "identifier", }, { - Name: "name", - TypeName: string(DataTypeString), - JSONPath: "name", - Alias: "title", + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", + Alias: "title", }, { - Name: "c", - TypeName: string(DataTypeMessage), - JSONPath: "c", - Alias: "grandChild", + Name: "c", + ProtoTypeName: DataTypeMessage, + JSONPath: "c", + Alias: "grandChild", Message: &RPCMessage{ Name: "NestedTypeC", Fields: RPCFields{ { - Name: "id", - TypeName: string(DataTypeString), - JSONPath: "id", - Alias: "identifier", + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", + Alias: "identifier", }, { - Name: "name", - TypeName: string(DataTypeString), - JSONPath: "name", - Alias: "title", + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", + Alias: "title", }, }, }, @@ -1683,10 +1683,10 @@ func TestProductExecutionPlanWithAliases(t *testing.T) { Name: "QueryRandomPetResponse", Fields: RPCFields{ { - Name: "random_pet", - TypeName: string(DataTypeMessage), - JSONPath: "randomPet", - Alias: "pet", + Name: "random_pet", + ProtoTypeName: DataTypeMessage, + JSONPath: "randomPet", + Alias: "pet", Message: &RPCMessage{ Name: "Animal", OneOfType: OneOfTypeInterface, @@ -1697,39 +1697,39 @@ func TestProductExecutionPlanWithAliases(t *testing.T) { FieldSelectionSet: RPCFieldSelectionSet{ "Cat": { { - Name: "meow_volume", - TypeName: string(DataTypeInt32), - JSONPath: "meowVolume", - Alias: "volumeLevel", + Name: "meow_volume", + ProtoTypeName: DataTypeInt32, + JSONPath: "meowVolume", + Alias: "volumeLevel", }, }, "Dog": { { - Name: "bark_volume", - TypeName: string(DataTypeInt32), - JSONPath: "barkVolume", - Alias: "volumeLevel", + Name: "bark_volume", + ProtoTypeName: DataTypeInt32, + JSONPath: "barkVolume", + Alias: "volumeLevel", }, }, }, Fields: RPCFields{ { - Name: "id", - TypeName: string(DataTypeString), - JSONPath: "id", - Alias: "identifier", + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", + Alias: "identifier", }, { - Name: "name", - TypeName: string(DataTypeString), - JSONPath: "name", - Alias: "petName", + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", + Alias: "petName", }, { - Name: "kind", - TypeName: string(DataTypeString), - JSONPath: "kind", - Alias: "animalKind", + Name: "kind", + ProtoTypeName: DataTypeString, + JSONPath: "kind", + Alias: "animalKind", }, }, }, @@ -1755,10 +1755,10 @@ func TestProductExecutionPlanWithAliases(t *testing.T) { Name: "QueryRandomSearchResultResponse", Fields: RPCFields{ { - Name: "random_search_result", - TypeName: string(DataTypeMessage), - JSONPath: "randomSearchResult", - Alias: "searchResults", + Name: "random_search_result", + ProtoTypeName: DataTypeMessage, + JSONPath: "randomSearchResult", + Alias: "searchResults", Message: &RPCMessage{ Name: "SearchResult", OneOfType: OneOfTypeUnion, @@ -1771,57 +1771,57 @@ func TestProductExecutionPlanWithAliases(t *testing.T) { FieldSelectionSet: RPCFieldSelectionSet{ "Product": { { - Name: "id", - TypeName: string(DataTypeString), - JSONPath: "id", - Alias: "productId", + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", + Alias: "productId", }, { - Name: "name", - TypeName: string(DataTypeString), - JSONPath: "name", - Alias: "productName", + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", + Alias: "productName", }, { - Name: "price", - TypeName: string(DataTypeDouble), - JSONPath: "price", - Alias: "cost", + Name: "price", + ProtoTypeName: DataTypeDouble, + JSONPath: "price", + Alias: "cost", }, }, "User": { { - Name: "id", - TypeName: string(DataTypeString), - JSONPath: "id", - Alias: "userId", + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", + Alias: "userId", }, { - Name: "name", - TypeName: string(DataTypeString), - JSONPath: "name", - Alias: "userName", + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", + Alias: "userName", }, }, "Category": { { - Name: "id", - TypeName: string(DataTypeString), - JSONPath: "id", - Alias: "categoryId", + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", + Alias: "categoryId", }, { - Name: "name", - TypeName: string(DataTypeString), - JSONPath: "name", - Alias: "categoryName", + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", + Alias: "categoryName", }, { - Name: "kind", - TypeName: string(DataTypeEnum), - JSONPath: "kind", - Alias: "categoryType", - EnumName: "CategoryKind", + Name: "kind", + ProtoTypeName: DataTypeEnum, + JSONPath: "kind", + Alias: "categoryType", + EnumName: "CategoryKind", }, }, }, @@ -1845,16 +1845,16 @@ func TestProductExecutionPlanWithAliases(t *testing.T) { Name: "MutationCreateUserRequest", Fields: []RPCField{ { - Name: "input", - TypeName: string(DataTypeMessage), - JSONPath: "input", + Name: "input", + ProtoTypeName: DataTypeMessage, + JSONPath: "input", Message: &RPCMessage{ Name: "UserInput", Fields: []RPCField{ { - Name: "name", - TypeName: string(DataTypeString), - JSONPath: "name", + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", }, }, }, @@ -1865,24 +1865,24 @@ func TestProductExecutionPlanWithAliases(t *testing.T) { Name: "MutationCreateUserResponse", Fields: RPCFields{ { - Name: "create_user", - TypeName: string(DataTypeMessage), - JSONPath: "createUser", - Alias: "newUser", + Name: "create_user", + ProtoTypeName: DataTypeMessage, + JSONPath: "createUser", + Alias: "newUser", Message: &RPCMessage{ Name: "User", Fields: RPCFields{ { - Name: "id", - TypeName: string(DataTypeString), - JSONPath: "id", - Alias: "userId", + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", + Alias: "userId", }, { - Name: "name", - TypeName: string(DataTypeString), - JSONPath: "name", - Alias: "fullName", + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", + Alias: "fullName", }, }, }, @@ -1905,10 +1905,10 @@ func TestProductExecutionPlanWithAliases(t *testing.T) { Name: "QueryCategoriesByKindRequest", Fields: []RPCField{ { - Name: "kind", - TypeName: string(DataTypeEnum), - JSONPath: "kind", - EnumName: "CategoryKind", + Name: "kind", + ProtoTypeName: DataTypeEnum, + JSONPath: "kind", + EnumName: "CategoryKind", }, }, }, @@ -1916,32 +1916,32 @@ func TestProductExecutionPlanWithAliases(t *testing.T) { Name: "QueryCategoriesByKindResponse", Fields: RPCFields{ { - Name: "categories_by_kind", - TypeName: string(DataTypeMessage), - JSONPath: "categoriesByKind", - Alias: "bookCategories", - Repeated: true, + Name: "categories_by_kind", + ProtoTypeName: DataTypeMessage, + JSONPath: "categoriesByKind", + Alias: "bookCategories", + Repeated: true, Message: &RPCMessage{ Name: "Category", Fields: RPCFields{ { - Name: "id", - TypeName: string(DataTypeString), - JSONPath: "id", - Alias: "identifier", + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", + Alias: "identifier", }, { - Name: "name", - TypeName: string(DataTypeString), - JSONPath: "name", - Alias: "title", + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", + Alias: "title", }, { - Name: "kind", - TypeName: string(DataTypeEnum), - JSONPath: "kind", - Alias: "type", - EnumName: "CategoryKind", + Name: "kind", + ProtoTypeName: DataTypeEnum, + JSONPath: "kind", + Alias: "type", + EnumName: "CategoryKind", }, }, }, @@ -1967,35 +1967,35 @@ func TestProductExecutionPlanWithAliases(t *testing.T) { Name: "QueryUsersResponse", Fields: RPCFields{ { - Name: "users", - TypeName: string(DataTypeMessage), - JSONPath: "users", - Repeated: true, + Name: "users", + ProtoTypeName: DataTypeMessage, + JSONPath: "users", + Repeated: true, Message: &RPCMessage{ Name: "User", Fields: RPCFields{ { - Name: "id", - TypeName: string(DataTypeString), - JSONPath: "id", + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", }, { - Name: "name", - TypeName: string(DataTypeString), - JSONPath: "name", - Alias: "name1", + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", + Alias: "name1", }, { - Name: "name", - TypeName: string(DataTypeString), - JSONPath: "name", - Alias: "name2", + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", + Alias: "name2", }, { - Name: "name", - TypeName: string(DataTypeString), - JSONPath: "name", - Alias: "name3", + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", + Alias: "name3", }, }, }, @@ -2018,9 +2018,9 @@ func TestProductExecutionPlanWithAliases(t *testing.T) { Name: "QueryUserRequest", Fields: []RPCField{ { - Name: "id", - TypeName: string(DataTypeString), - JSONPath: "id", + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", }, }, }, @@ -2028,23 +2028,23 @@ func TestProductExecutionPlanWithAliases(t *testing.T) { Name: "QueryUserResponse", Fields: RPCFields{ { - Name: "user", - TypeName: string(DataTypeMessage), - JSONPath: "user", - Alias: "user1", - Optional: true, + Name: "user", + ProtoTypeName: DataTypeMessage, + JSONPath: "user", + Alias: "user1", + Optional: true, Message: &RPCMessage{ Name: "User", Fields: RPCFields{ { - Name: "id", - TypeName: string(DataTypeString), - JSONPath: "id", + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", }, { - Name: "name", - TypeName: string(DataTypeString), - JSONPath: "name", + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", }, }, }, @@ -2059,9 +2059,9 @@ func TestProductExecutionPlanWithAliases(t *testing.T) { Name: "QueryUserRequest", Fields: []RPCField{ { - Name: "id", - TypeName: string(DataTypeString), - JSONPath: "id", + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", }, }, }, @@ -2069,23 +2069,23 @@ func TestProductExecutionPlanWithAliases(t *testing.T) { Name: "QueryUserResponse", Fields: RPCFields{ { - Name: "user", - TypeName: string(DataTypeMessage), - JSONPath: "user", - Alias: "user2", - Optional: true, + Name: "user", + ProtoTypeName: DataTypeMessage, + JSONPath: "user", + Alias: "user2", + Optional: true, Message: &RPCMessage{ Name: "User", Fields: RPCFields{ { - Name: "id", - TypeName: string(DataTypeString), - JSONPath: "id", + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", }, { - Name: "name", - TypeName: string(DataTypeString), - JSONPath: "name", + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", }, }, }, @@ -2100,9 +2100,9 @@ func TestProductExecutionPlanWithAliases(t *testing.T) { Name: "QueryUserRequest", Fields: []RPCField{ { - Name: "id", - TypeName: string(DataTypeString), - JSONPath: "id", + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", }, }, }, @@ -2110,25 +2110,25 @@ func TestProductExecutionPlanWithAliases(t *testing.T) { Name: "QueryUserResponse", Fields: RPCFields{ { - Name: "user", - TypeName: string(DataTypeMessage), - JSONPath: "user", - Alias: "sameUser", - Optional: true, + Name: "user", + ProtoTypeName: DataTypeMessage, + JSONPath: "user", + Alias: "sameUser", + Optional: true, Message: &RPCMessage{ Name: "User", Fields: RPCFields{ { - Name: "id", - TypeName: string(DataTypeString), - JSONPath: "id", - Alias: "userId", + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", + Alias: "userId", }, { - Name: "name", - TypeName: string(DataTypeString), - JSONPath: "name", - Alias: "userName", + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", + Alias: "userName", }, }, }, @@ -2154,77 +2154,77 @@ func TestProductExecutionPlanWithAliases(t *testing.T) { Name: "QueryNestedTypeResponse", Fields: RPCFields{ { - Name: "nested_type", - TypeName: string(DataTypeMessage), - JSONPath: "nestedType", - Repeated: true, + Name: "nested_type", + ProtoTypeName: DataTypeMessage, + JSONPath: "nestedType", + Repeated: true, Message: &RPCMessage{ Name: "NestedTypeA", Fields: RPCFields{ { - Name: "id", - TypeName: string(DataTypeString), - JSONPath: "id", + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", }, { - Name: "name", - TypeName: string(DataTypeString), - JSONPath: "name", - Alias: "name1", + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", + Alias: "name1", }, { - Name: "name", - TypeName: string(DataTypeString), - JSONPath: "name", - Alias: "name2", + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", + Alias: "name2", }, { - Name: "b", - TypeName: string(DataTypeMessage), - JSONPath: "b", + Name: "b", + ProtoTypeName: DataTypeMessage, + JSONPath: "b", Message: &RPCMessage{ Name: "NestedTypeB", Fields: RPCFields{ { - Name: "id", - TypeName: string(DataTypeString), - JSONPath: "id", + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", }, { - Name: "name", - TypeName: string(DataTypeString), - JSONPath: "name", - Alias: "title1", + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", + Alias: "title1", }, { - Name: "name", - TypeName: string(DataTypeString), - JSONPath: "name", - Alias: "title2", + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", + Alias: "title2", }, { - Name: "c", - TypeName: string(DataTypeMessage), - JSONPath: "c", + Name: "c", + ProtoTypeName: DataTypeMessage, + JSONPath: "c", Message: &RPCMessage{ Name: "NestedTypeC", Fields: RPCFields{ { - Name: "id", - TypeName: string(DataTypeString), - JSONPath: "id", + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", }, { - Name: "name", - TypeName: string(DataTypeString), - JSONPath: "name", - Alias: "label1", + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", + Alias: "label1", }, { - Name: "name", - TypeName: string(DataTypeString), - JSONPath: "name", - Alias: "label2", + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", + Alias: "label2", }, }, }, @@ -2256,9 +2256,9 @@ func TestProductExecutionPlanWithAliases(t *testing.T) { Name: "QueryRandomPetResponse", Fields: RPCFields{ { - Name: "random_pet", - TypeName: string(DataTypeMessage), - JSONPath: "randomPet", + Name: "random_pet", + ProtoTypeName: DataTypeMessage, + JSONPath: "randomPet", Message: &RPCMessage{ Name: "Animal", OneOfType: OneOfTypeInterface, @@ -2269,55 +2269,55 @@ func TestProductExecutionPlanWithAliases(t *testing.T) { FieldSelectionSet: RPCFieldSelectionSet{ "Cat": { { - Name: "meow_volume", - TypeName: string(DataTypeInt32), - JSONPath: "meowVolume", - Alias: "volume1", + Name: "meow_volume", + ProtoTypeName: DataTypeInt32, + JSONPath: "meowVolume", + Alias: "volume1", }, { - Name: "meow_volume", - TypeName: string(DataTypeInt32), - JSONPath: "meowVolume", - Alias: "volume2", + Name: "meow_volume", + ProtoTypeName: DataTypeInt32, + JSONPath: "meowVolume", + Alias: "volume2", }, }, "Dog": { { - Name: "bark_volume", - TypeName: string(DataTypeInt32), - JSONPath: "barkVolume", - Alias: "volume1", + Name: "bark_volume", + ProtoTypeName: DataTypeInt32, + JSONPath: "barkVolume", + Alias: "volume1", }, { - Name: "bark_volume", - TypeName: string(DataTypeInt32), - JSONPath: "barkVolume", - Alias: "volume2", + Name: "bark_volume", + ProtoTypeName: DataTypeInt32, + JSONPath: "barkVolume", + Alias: "volume2", }, }, }, Fields: RPCFields{ { - Name: "id", - TypeName: string(DataTypeString), - JSONPath: "id", + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", }, { - Name: "name", - TypeName: string(DataTypeString), - JSONPath: "name", - Alias: "name1", + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", + Alias: "name1", }, { - Name: "name", - TypeName: string(DataTypeString), - JSONPath: "name", - Alias: "name2", + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", + Alias: "name2", }, { - Name: "kind", - TypeName: string(DataTypeString), - JSONPath: "kind", + Name: "kind", + ProtoTypeName: DataTypeString, + JSONPath: "kind", }, }, }, @@ -2343,9 +2343,9 @@ func TestProductExecutionPlanWithAliases(t *testing.T) { Name: "QueryRandomSearchResultResponse", Fields: RPCFields{ { - Name: "random_search_result", - TypeName: string(DataTypeMessage), - JSONPath: "randomSearchResult", + Name: "random_search_result", + ProtoTypeName: DataTypeMessage, + JSONPath: "randomSearchResult", Message: &RPCMessage{ Name: "SearchResult", OneOfType: OneOfTypeUnion, @@ -2358,85 +2358,85 @@ func TestProductExecutionPlanWithAliases(t *testing.T) { FieldSelectionSet: RPCFieldSelectionSet{ "Product": { { - Name: "id", - TypeName: string(DataTypeString), - JSONPath: "id", + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", }, { - Name: "name", - TypeName: string(DataTypeString), - JSONPath: "name", - Alias: "name1", + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", + Alias: "name1", }, { - Name: "name", - TypeName: string(DataTypeString), - JSONPath: "name", - Alias: "name2", + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", + Alias: "name2", }, { - Name: "price", - TypeName: string(DataTypeDouble), - JSONPath: "price", - Alias: "price1", + Name: "price", + ProtoTypeName: DataTypeDouble, + JSONPath: "price", + Alias: "price1", }, { - Name: "price", - TypeName: string(DataTypeDouble), - JSONPath: "price", - Alias: "price2", + Name: "price", + ProtoTypeName: DataTypeDouble, + JSONPath: "price", + Alias: "price2", }, }, "User": { { - Name: "id", - TypeName: string(DataTypeString), - JSONPath: "id", + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", }, { - Name: "name", - TypeName: string(DataTypeString), - JSONPath: "name", - Alias: "name1", + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", + Alias: "name1", }, { - Name: "name", - TypeName: string(DataTypeString), - JSONPath: "name", - Alias: "name2", + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", + Alias: "name2", }, }, "Category": { { - Name: "id", - TypeName: string(DataTypeString), - JSONPath: "id", + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", }, { - Name: "name", - TypeName: string(DataTypeString), - JSONPath: "name", - Alias: "name1", + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", + Alias: "name1", }, { - Name: "name", - TypeName: string(DataTypeString), - JSONPath: "name", - Alias: "name2", + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", + Alias: "name2", }, { - Name: "kind", - TypeName: string(DataTypeEnum), - JSONPath: "kind", - Alias: "kind1", - EnumName: "CategoryKind", + Name: "kind", + ProtoTypeName: DataTypeEnum, + JSONPath: "kind", + Alias: "kind1", + EnumName: "CategoryKind", }, { - Name: "kind", - TypeName: string(DataTypeEnum), - JSONPath: "kind", - Alias: "kind2", - EnumName: "CategoryKind", + Name: "kind", + ProtoTypeName: DataTypeEnum, + JSONPath: "kind", + Alias: "kind2", + EnumName: "CategoryKind", }, }, }, diff --git a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor_federation.go b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor_federation.go index 58243b840..fad76e272 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor_federation.go +++ b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor_federation.go @@ -438,11 +438,11 @@ func (r *rpcPlanVisitorFederation) scaffoldEntityLookup(fc federationConfigData) r.planInfo.currentRequestMessage.Fields = []RPCField{ { - Name: "keys", - TypeName: DataTypeMessage.String(), - Repeated: true, // The inputs are always a list of objects - JSONPath: "representations", - Message: keyFieldMessage, + Name: "keys", + ProtoTypeName: DataTypeMessage, + Repeated: true, // The inputs are always a list of objects + JSONPath: "representations", + Message: keyFieldMessage, }, } @@ -450,10 +450,10 @@ func (r *rpcPlanVisitorFederation) scaffoldEntityLookup(fc federationConfigData) // As this is a special case we directly map it to _entities. r.planInfo.currentResponseMessage.Fields = []RPCField{ { - Name: "result", - TypeName: DataTypeMessage.String(), - JSONPath: "_entities", - Repeated: true, + Name: "result", + ProtoTypeName: DataTypeMessage, + JSONPath: "_entities", + Repeated: true, }, } } diff --git a/v2/pkg/engine/datasource/grpc_datasource/grpc_datasource_test.go b/v2/pkg/engine/datasource/grpc_datasource/grpc_datasource_test.go index 44bd9dc51..8191b5b08 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/grpc_datasource_test.go +++ b/v2/pkg/engine/datasource/grpc_datasource/grpc_datasource_test.go @@ -83,13 +83,14 @@ func Benchmark_DataSource_Load_WithFieldArguments(b *testing.B) { b.ReportAllocs() b.ResetTimer() + const subgraphName = "Products" mapping := testMapping() for b.Loop() { ds, err := NewDataSource(conn, DataSourceConfig{ Operation: &queryDoc, Definition: &schemaDoc, - SubgraphName: "Products", + SubgraphName: subgraphName, Compiler: compiler, Mapping: mapping, }) @@ -519,33 +520,33 @@ func TestMarshalResponseJSON(t *testing.T) { Name: "LookupProductByIdResponse", Fields: []RPCField{ { - Name: "result", - TypeName: string(DataTypeMessage), - Repeated: true, - JSONPath: "_entities", + Name: "result", + ProtoTypeName: DataTypeMessage, + Repeated: true, + JSONPath: "_entities", Message: &RPCMessage{ Name: "Product", Fields: []RPCField{ { - Name: "__typename", - TypeName: string(DataTypeString), - JSONPath: "__typename", - StaticValue: "Product", + Name: "__typename", + ProtoTypeName: DataTypeString, + JSONPath: "__typename", + StaticValue: "Product", }, { - Name: "id", - TypeName: string(DataTypeString), - JSONPath: "id", + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", }, { - Name: "name", - TypeName: string(DataTypeString), - JSONPath: "name_different", + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name_different", }, { - Name: "price", - TypeName: string(DataTypeDouble), - JSONPath: "price_different", + Name: "price", + ProtoTypeName: DataTypeDouble, + JSONPath: "price_different", }, }, }, From 6755e2b563f39535b24871d6e39c8de4d625a492 Mon Sep 17 00:00:00 2001 From: Ludwig Bedacht Date: Thu, 23 Oct 2025 14:38:01 +0200 Subject: [PATCH 25/72] chore: reduce number of allocations --- v2/pkg/ast/ast_selection.go | 11 ++- .../datasource/grpc_datasource/compiler.go | 80 +++++++------------ .../grpc_datasource/compiler_test.go | 19 ++++- .../grpc_datasource/configuration.go | 10 +-- .../grpc_datasource/execution_plan.go | 58 +++++++------- .../grpc_datasource/execution_plan_visitor.go | 2 +- .../execution_plan_visitor_federation.go | 6 +- .../grpc_datasource/grpc_datasource.go | 8 +- ..._test_helper.go => mapping_helper_test.go} | 0 9 files changed, 97 insertions(+), 97 deletions(-) rename v2/pkg/engine/datasource/grpc_datasource/{mapping_test_helper.go => mapping_helper_test.go} (100%) diff --git a/v2/pkg/ast/ast_selection.go b/v2/pkg/ast/ast_selection.go index fecd4229d..0c7b643c8 100644 --- a/v2/pkg/ast/ast_selection.go +++ b/v2/pkg/ast/ast_selection.go @@ -246,5 +246,14 @@ func (d *Document) SelectionSetFieldNames(set int) (fieldNames []string) { // SelectionSetFieldSetString returns a string of the field names in the selection set separated by a space // Example: "{ name status }" -> "name status" func (d *Document) SelectionSetFieldSetString(set int) string { - return strings.Join(d.SelectionSetFieldNames(set), " ") + fieldSelections := d.SelectionSetFieldSelections(set) + builder := strings.Builder{} + for i, fieldSelection := range fieldSelections { + builder.Write(d.FieldNameBytes(d.Selections[fieldSelection].Ref)) + if i != len(fieldSelections)-1 { + builder.WriteRune(' ') + } + } + + return builder.String() } diff --git a/v2/pkg/engine/datasource/grpc_datasource/compiler.go b/v2/pkg/engine/datasource/grpc_datasource/compiler.go index 3beb3090a..460b5c59f 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/compiler.go +++ b/v2/pkg/engine/datasource/grpc_datasource/compiler.go @@ -243,26 +243,26 @@ type RPCCompiler struct { // ServiceByName returns a Service by its name. // Returns an empty Service if no service with the given name exists. -func (d *Document) ServiceByName(name string) Service { +func (d *Document) ServiceByName(name string) *Service { for _, s := range d.Services { if s.Name == name { - return s + return &s } } - return Service{} + return nil } // MethodByName returns a Method by its name. // Returns an empty Method if no method with the given name exists. -func (d *Document) MethodByName(name string) Method { +func (d *Document) MethodByName(name string) *Method { for _, m := range d.Methods { if m.Name == name { - return m + return &m } } - return Method{} + return nil } // MethodRefByName returns the index of a Method in the Methods slice by its name. @@ -426,6 +426,18 @@ type ServiceCall struct { RPC *RPCCall } +func (s *ServiceCall) MethodFullName() string { + var builder strings.Builder + + builder.Grow(len(s.ServiceName) + len(s.MethodName) + 2) + builder.WriteRune('/') + builder.WriteString(s.ServiceName) + builder.WriteRune('/') + builder.WriteString(s.MethodName) + + return builder.String() +} + // func (p *RPCCompiler) CompileFetches(graph *DependencyGraph, fetches []FetchItem, inputData gjson.Result) ([]Invocation, error) { // invocations := make([]Invocation, 0, len(fetches)) @@ -515,7 +527,7 @@ func (p *RPCCompiler) CompileNode(graph *DependencyGraph, fetch FetchItem, input request = p.buildProtoMessageWithContext(inputMessage, &call.Request, inputData, context) } - serviceName, ok := p.resolveServiceName(call.MethodName) + serviceName, ok := p.resolveServiceName(call) if !ok { return ServiceCall{}, fmt.Errorf("failed to resolve service name for method %s from the protobuf definition", call.MethodName) } @@ -530,51 +542,15 @@ func (p *RPCCompiler) CompileNode(graph *DependencyGraph, fetch FetchItem, input } -// Compile processes an RPCExecutionPlan and builds protobuf messages from JSON data -// based on the compiled schema. -// Deprecated: Use CompileFetches instead. -func (p *RPCCompiler) Compile(executionPlan *RPCExecutionPlan, inputData gjson.Result) ([]ServiceCall, error) { - serviceCalls := make([]ServiceCall, 0, len(executionPlan.Calls)) - - for _, call := range executionPlan.Calls { - inputMessage, ok := p.doc.MessageByName(call.Request.Name) - if !ok { - return nil, fmt.Errorf("input message %s not found in document", call.Request.Name) - } - - outputMessage, ok := p.doc.MessageByName(call.Response.Name) - if !ok { - return nil, fmt.Errorf("output message %s not found in document", call.Response.Name) - } - - request := p.buildProtoMessage(inputMessage, &call.Request, inputData) - response := p.newEmptyMessage(outputMessage) - - if p.report.HasErrors() { - return nil, fmt.Errorf("failed to compile invocation: %w", p.report) - } - - serviceName, ok := p.resolveServiceName(call.MethodName) - if !ok { - return nil, fmt.Errorf("failed to resolve service name for method %s from the protobuf definition", call.MethodName) - } - - serviceCalls = append(serviceCalls, ServiceCall{ - ServiceName: serviceName, - MethodName: call.MethodName, - Input: request, - Output: response, - RPC: &call, - }) +func (p *RPCCompiler) resolveServiceName(call *RPCCall) (string, bool) { + if service := p.doc.ServiceByName(call.ServiceName); service != nil { + return service.FullName, true } - return serviceCalls, nil -} - -func (p *RPCCompiler) resolveServiceName(methodName string) (string, bool) { + // Fallback. Try to find the service by the method name. for _, service := range p.doc.Services { for _, methodRef := range service.MethodsRefs { - if p.doc.Methods[methodRef].Name == methodName { + if p.doc.Methods[methodRef].Name == call.MethodName { return service.FullName, true } } @@ -678,10 +654,10 @@ func (p *RPCCompiler) resolveContextData(context FetchItem, contextField *RPCFie } contextValues := make([]map[string]protoref.Value, 0) - outputMessage := context.ServiceCall.Output for _, field := range contextField.Message.Fields { - resolvePath := field.ResolvePath - values := p.resolveContextDataForPath(outputMessage, resolvePath) + values := p.resolveContextDataForPath(context.ServiceCall.Output, field.ResolvePath) + + contextValues = slices.Grow(contextValues, len(values)) for index, value := range values { if index >= len(contextValues) { @@ -861,7 +837,7 @@ func (p *RPCCompiler) resolveUnderlyingListItems(value protoref.Value, nestingLe return result } -func (p *RPCCompiler) newEmptyListMessageByName(msg *dynamicpb.Message, name string) protoref.List { +func (p *RPCCompiler) newEmptyListMessageByName(msg protoref.Message, name string) protoref.List { return msg.Mutable(msg.Descriptor().Fields().ByName(protoref.Name(name))).List() } diff --git a/v2/pkg/engine/datasource/grpc_datasource/compiler_test.go b/v2/pkg/engine/datasource/grpc_datasource/compiler_test.go index 4815ed1dd..1bd7dd880 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/compiler_test.go +++ b/v2/pkg/engine/datasource/grpc_datasource/compiler_test.go @@ -285,7 +285,14 @@ func TestBuildProtoMessage(t *testing.T) { // Compile the execution plan with the variables // This should build a protobuf message ready to be sent to the gRPC service - serviceCalls, err := compiler.Compile(executionPlan, gjson.ParseBytes(variables)) + graph := NewDependencyGraph(executionPlan) + fetches := make([]FetchItem, 0, len(executionPlan.Calls)) + graph.TopologicalSortResolve(func(nodes []FetchItem) error { + fetches = append(fetches, nodes...) + return nil + }) + + serviceCalls, err := compiler.CompileFetches(graph, fetches, gjson.ParseBytes(variables)) if err != nil { t.Fatalf("failed to compile proto: %v", err) } @@ -400,7 +407,15 @@ func TestCompileNestedLists(t *testing.T) { }, } - serviceCalls, err := compiler.Compile(plan, gjson.ParseBytes([]byte(`{"orders":[{"orderId":"123","customerName":"John Doe","lines":[{"productId":"123","quantity":1, "modifiers":["modifier1", "modifier2"]}]}]}`))) + graph := NewDependencyGraph(plan) + fetches := make([]FetchItem, 0, len(plan.Calls)) + + graph.TopologicalSortResolve(func(nodes []FetchItem) error { + fetches = append(fetches, nodes...) + return nil + }) + + serviceCalls, err := compiler.CompileFetches(graph, fetches, gjson.ParseBytes([]byte(`{"orders":[{"orderId":"123","customerName":"John Doe","lines":[{"productId":"123","quantity":1, "modifiers":["modifier1", "modifier2"]}]}]}`))) require.NoError(t, err) require.Equal(t, 1, len(serviceCalls)) diff --git a/v2/pkg/engine/datasource/grpc_datasource/configuration.go b/v2/pkg/engine/datasource/grpc_datasource/configuration.go index ad9863e09..2a607d4a0 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/configuration.go +++ b/v2/pkg/engine/datasource/grpc_datasource/configuration.go @@ -178,22 +178,22 @@ func (g *GRPCMapping) FindEntityRPCConfig(typeName, key string) (RPCConfig, bool } // FindResolveTypeFieldMapping finds the gRPC field name for a given GraphQL field name and type -func (g *GRPCMapping) FindResolveTypeFieldMapping(typeName, fieldName string) (ResolveRPCTypeField, bool) { +func (g *GRPCMapping) FindResolveTypeFieldMapping(typeName, fieldName string) *ResolveRPCTypeField { if g == nil || g.ResolveRPCs == nil { - return ResolveRPCTypeField{}, false + return nil } fieldMappings, ok := g.ResolveRPCs[typeName] if !ok { - return ResolveRPCTypeField{}, false + return nil } field, ok := fieldMappings[fieldName] if !ok { - return ResolveRPCTypeField{}, false + return nil } - return field, true + return &field } type keySet map[string]struct{} diff --git a/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go b/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go index 2d6cb2431..46e8b758c 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go +++ b/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go @@ -719,7 +719,7 @@ type resolvedField struct { parentTypeRef int fieldRef int fieldDefinitionTypeRef int - requiredFields string + requiredFieldSelection int responsePath ast.Path contextFields []contextField @@ -809,9 +809,10 @@ func (r *rpcPlanningContext) getFieldsFromContext(parentNode ast.Node, contextRe fieldsString := r.definition.ValueContentString(val) - walker := astvisitor.NewDefaultWalker() + walker := astvisitor.WalkerFromPool() + defer walker.Release() - v := newRequiredFieldsVisitor(&walker, &RPCMessage{}, r) + v := newRequiredFieldsVisitor(walker, &RPCMessage{}, r) if err := v.visitRequiredFields(r.definition, parentNode.NameString(r.definition), fieldsString); err != nil { return nil, err } @@ -866,16 +867,13 @@ func (r *rpcPlanningContext) nodeByTypeRef(typeRef int) (ast.Node, bool) { } type resolveRPCCallConfig struct { - serviceName string - typeName string - fieldName string - resolveConfig ResolveRPCTypeField - resolvedField resolvedField + resolveConfig *ResolveRPCTypeField + resolvedField *resolvedField contextMessage *RPCMessage fieldArgsMessage *RPCMessage } -func (r *rpcPlanningContext) resolveRequiredFields(typeName, requiredFields string) (*RPCMessage, error) { +func (r *rpcPlanningContext) resolveRequiredFields(typeName string, requiredFieldSelection int) (*RPCMessage, error) { walker := astvisitor.WalkerFromPool() defer walker.Release() message := &RPCMessage{ @@ -883,7 +881,7 @@ func (r *rpcPlanningContext) resolveRequiredFields(typeName, requiredFields stri } rfv := newRequiredFieldsVisitor(walker, message, r) - if err := rfv.visitWithMemberTypes(r.definition, typeName, requiredFields, nil); err != nil { + if err := rfv.visitWithMemberTypes(r.definition, typeName, r.operation.SelectionSetFieldSetString(requiredFieldSelection), nil); err != nil { return nil, err } return message, nil @@ -895,12 +893,12 @@ func (r *rpcPlanningContext) createResolverRPCCalls(subgraphName string, resolve calls := make([]RPCCall, 0, len(resolvedFields)) for _, resolvedField := range resolvedFields { - resolveConfig, exists := r.mapping.FindResolveTypeFieldMapping( + resolveConfig := r.mapping.FindResolveTypeFieldMapping( r.definition.ObjectTypeDefinitionNameString(resolvedField.parentTypeRef), r.operation.FieldNameString(resolvedField.fieldRef), ) - if !exists { + if resolveConfig == nil { return nil, fmt.Errorf("resolve config not found for type: %s, field: %s", r.definition.ResolveTypeNameString(resolvedField.parentTypeRef), r.operation.FieldAliasString(resolvedField.fieldRef)) } @@ -912,12 +910,9 @@ func (r *rpcPlanningContext) createResolverRPCCalls(subgraphName string, resolve Name: resolveConfig.RPC + "Args", } - call, err := r.newResolveRPCCall(resolveRPCCallConfig{ - serviceName: subgraphName, - typeName: r.definition.ResolveTypeNameString(resolvedField.parentTypeRef), - fieldName: r.operation.FieldAliasOrNameString(resolvedField.fieldRef), + call, err := r.newResolveRPCCall(&resolveRPCCallConfig{ resolveConfig: resolveConfig, - resolvedField: resolvedField, + resolvedField: &resolvedField, contextMessage: contextMessage, fieldArgsMessage: fieldArgsMessage, }) @@ -926,6 +921,8 @@ func (r *rpcPlanningContext) createResolverRPCCalls(subgraphName string, resolve return nil, err } + call.ServiceName = r.resolveServiceName(subgraphName) + contextMessage.Fields = make(RPCFields, len(resolvedField.contextFields)) for i := range resolvedField.contextFields { typeDefNode, found := r.definition.NodeByNameStr(r.definition.ResolveTypeNameString(resolvedField.parentTypeRef)) @@ -966,18 +963,27 @@ func (r *rpcPlanningContext) createResolverRPCCalls(subgraphName string, resolve return calls, nil } -func (r *rpcPlanningContext) newResolveRPCCall(config resolveRPCCallConfig) (RPCCall, error) { +const ( + resultFieldName = "result" + contextFieldName = "context" + fieldArgsFieldName = "field_args" +) + +func (r *rpcPlanningContext) newResolveRPCCall(config *resolveRPCCallConfig) (RPCCall, error) { resolveConfig := config.resolveConfig resolvedField := config.resolvedField underlyingTypeRef := r.definition.ResolveUnderlyingType(resolvedField.fieldDefinitionTypeRef) - fieldTypeName := r.definition.ResolveTypeNameString(underlyingTypeRef) dataType := r.toDataType(&r.definition.Types[underlyingTypeRef]) var responseFieldsMessage *RPCMessage if dataType == DataTypeMessage { var err error - responseFieldsMessage, err = r.resolveRequiredFields(fieldTypeName, resolvedField.requiredFields) + responseFieldsMessage, err = r.resolveRequiredFields( + r.definition.ResolveTypeNameString(underlyingTypeRef), + resolvedField.requiredFieldSelection, + ) + if err != nil { return RPCCall{}, err } @@ -987,9 +993,9 @@ func (r *rpcPlanningContext) newResolveRPCCall(config resolveRPCCallConfig) (RPC Name: resolveConfig.Response, Fields: RPCFields{ { - Name: "result", + Name: resultFieldName, ProtoTypeName: DataTypeMessage, - JSONPath: "result", + JSONPath: resultFieldName, Repeated: true, Message: &RPCMessage{ Name: resolveConfig.RPC + "Result", @@ -997,7 +1003,7 @@ func (r *rpcPlanningContext) newResolveRPCCall(config resolveRPCCallConfig) (RPC { Name: resolveConfig.FieldMappingData.TargetName, ProtoTypeName: dataType, - JSONPath: config.fieldName, + JSONPath: r.operation.FieldAliasOrNameString(resolvedField.fieldRef), Message: responseFieldsMessage, Optional: !r.definition.TypeIsNonNull(resolvedField.fieldDefinitionTypeRef), }, @@ -1010,22 +1016,20 @@ func (r *rpcPlanningContext) newResolveRPCCall(config resolveRPCCallConfig) (RPC return RPCCall{ DependentCalls: []int{resolvedField.callerRef}, ResponsePath: resolvedField.responsePath, - ServiceName: r.resolveServiceName(config.serviceName), MethodName: resolveConfig.RPC, Kind: CallKindResolve, Request: RPCMessage{ Name: resolveConfig.Request, Fields: RPCFields{ { - Name: "context", + Name: contextFieldName, ProtoTypeName: DataTypeMessage, Repeated: true, Message: config.contextMessage, }, { - Name: "field_args", + Name: fieldArgsFieldName, ProtoTypeName: DataTypeMessage, - JSONPath: "", Message: config.fieldArgsMessage, }, }, diff --git a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor.go b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor.go index e4b616d7d..04a784d66 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor.go +++ b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor.go @@ -200,7 +200,7 @@ func (r *rpcPlanVisitor) EnterSelectionSet(ref int) { // If we are inside of a resolved field that selects multiple fields, we get all the fields from the input and pass them to the required fields visitor. if r.resolvedFieldIndex != ast.InvalidRef { - r.resolvedFields[r.resolvedFieldIndex].requiredFields = r.operation.SelectionSetFieldSetString(ref) + r.resolvedFields[r.resolvedFieldIndex].requiredFieldSelection = ref r.walker.SkipNode() return } diff --git a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor_federation.go b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor_federation.go index fad76e272..f57de6212 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor_federation.go +++ b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor_federation.go @@ -11,7 +11,6 @@ import ( "github.com/wundergraph/graphql-go-tools/v2/pkg/ast" "github.com/wundergraph/graphql-go-tools/v2/pkg/astvisitor" "github.com/wundergraph/graphql-go-tools/v2/pkg/engine/plan" - "github.com/wundergraph/graphql-go-tools/v2/pkg/internal/unsafebytes" "github.com/wundergraph/graphql-go-tools/v2/pkg/operationreport" ) @@ -189,10 +188,7 @@ func (r *rpcPlanVisitorFederation) EnterSelectionSet(ref int) { // If we are inside of a resolved field that selects multiple fields, we get all the fields from the input and pass them to the required fields visitor. if r.resolvedFieldIndex != ast.InvalidRef { - lbrace := r.operation.SelectionSets[ref].LBrace.CharEnd - rbrace := r.operation.SelectionSets[ref].RBrace.CharStart - 1 - - r.resolvedFields[r.resolvedFieldIndex].requiredFields = unsafebytes.BytesToString(r.operation.Input.RawBytes[lbrace:rbrace]) + r.resolvedFields[r.resolvedFieldIndex].requiredFieldSelection = ref r.walker.SkipNode() return } diff --git a/v2/pkg/engine/datasource/grpc_datasource/grpc_datasource.go b/v2/pkg/engine/datasource/grpc_datasource/grpc_datasource.go index afee61fce..940d9a4ce 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/grpc_datasource.go +++ b/v2/pkg/engine/datasource/grpc_datasource/grpc_datasource.go @@ -9,7 +9,7 @@ package grpcdatasource import ( "bytes" "context" - "fmt" + "errors" "github.com/tidwall/gjson" "golang.org/x/sync/errgroup" @@ -91,11 +91,12 @@ func (d *DataSource) Load(ctx context.Context, input []byte, out *bytes.Buffer) builder := newJSONBuilder(d.mapping, variables) if d.disabled { - out.Write(builder.writeErrorBytes(fmt.Errorf("gRPC datasource needs to be enabled to be used"))) + out.Write(builder.writeErrorBytes(errors.New("gRPC datasource needs to be enabled to be used"))) return nil } arena := astjson.Arena{} + defer arena.Reset() root := arena.NewObject() failed := false @@ -114,9 +115,8 @@ func (d *DataSource) Load(ctx context.Context, input []byte, out *bytes.Buffer) errGrp.Go(func() error { a := astjson.Arena{} // Invoke the gRPC method - this will populate serviceCall.Output - methodName := fmt.Sprintf("/%s/%s", serviceCall.ServiceName, serviceCall.MethodName) - err := d.cc.Invoke(errGrpCtx, methodName, serviceCall.Input, serviceCall.Output) + err := d.cc.Invoke(errGrpCtx, serviceCall.MethodFullName(), serviceCall.Input, serviceCall.Output) if err != nil { return err } diff --git a/v2/pkg/engine/datasource/grpc_datasource/mapping_test_helper.go b/v2/pkg/engine/datasource/grpc_datasource/mapping_helper_test.go similarity index 100% rename from v2/pkg/engine/datasource/grpc_datasource/mapping_test_helper.go rename to v2/pkg/engine/datasource/grpc_datasource/mapping_helper_test.go From 218eb4cbbb9623a0a4fd3ffbd65740ad64a2f65f Mon Sep 17 00:00:00 2001 From: Ludwig Bedacht Date: Thu, 23 Oct 2025 14:42:27 +0200 Subject: [PATCH 26/72] chore: check error --- .../datasource/grpc_datasource/compiler_test.go | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/v2/pkg/engine/datasource/grpc_datasource/compiler_test.go b/v2/pkg/engine/datasource/grpc_datasource/compiler_test.go index 1bd7dd880..8a5fc6c9c 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/compiler_test.go +++ b/v2/pkg/engine/datasource/grpc_datasource/compiler_test.go @@ -287,18 +287,16 @@ func TestBuildProtoMessage(t *testing.T) { // This should build a protobuf message ready to be sent to the gRPC service graph := NewDependencyGraph(executionPlan) fetches := make([]FetchItem, 0, len(executionPlan.Calls)) - graph.TopologicalSortResolve(func(nodes []FetchItem) error { + err = graph.TopologicalSortResolve(func(nodes []FetchItem) error { fetches = append(fetches, nodes...) return nil }) - serviceCalls, err := compiler.CompileFetches(graph, fetches, gjson.ParseBytes(variables)) - if err != nil { - t.Fatalf("failed to compile proto: %v", err) - } + require.NoError(t, err) + serviceCalls, err := compiler.CompileFetches(graph, fetches, gjson.ParseBytes(variables)) + require.NoError(t, err) require.Equal(t, 1, len(serviceCalls)) - } func TestCompileNestedLists(t *testing.T) { @@ -410,11 +408,13 @@ func TestCompileNestedLists(t *testing.T) { graph := NewDependencyGraph(plan) fetches := make([]FetchItem, 0, len(plan.Calls)) - graph.TopologicalSortResolve(func(nodes []FetchItem) error { + err = graph.TopologicalSortResolve(func(nodes []FetchItem) error { fetches = append(fetches, nodes...) return nil }) + require.NoError(t, err) + serviceCalls, err := compiler.CompileFetches(graph, fetches, gjson.ParseBytes([]byte(`{"orders":[{"orderId":"123","customerName":"John Doe","lines":[{"productId":"123","quantity":1, "modifiers":["modifier1", "modifier2"]}]}]}`))) require.NoError(t, err) require.Equal(t, 1, len(serviceCalls)) From 077bfd30924bca3fd6f70d2bbb7b94899e73e474 Mon Sep 17 00:00:00 2001 From: Ludwig Bedacht Date: Thu, 23 Oct 2025 16:21:10 +0200 Subject: [PATCH 27/72] chore: rename file --- .../{mapping_helper_test.go => mapping_test_helper.go} | 0 1 file changed, 0 insertions(+), 0 deletions(-) rename v2/pkg/engine/datasource/grpc_datasource/{mapping_helper_test.go => mapping_test_helper.go} (100%) diff --git a/v2/pkg/engine/datasource/grpc_datasource/mapping_helper_test.go b/v2/pkg/engine/datasource/grpc_datasource/mapping_test_helper.go similarity index 100% rename from v2/pkg/engine/datasource/grpc_datasource/mapping_helper_test.go rename to v2/pkg/engine/datasource/grpc_datasource/mapping_test_helper.go From 37db946b097d2ee8f7c47251dc47691447486e63 Mon Sep 17 00:00:00 2001 From: Ludwig Bedacht Date: Fri, 24 Oct 2025 10:34:34 +0200 Subject: [PATCH 28/72] chore: improvements --- .../datasource/grpc_datasource/compiler.go | 61 ++++--------------- v2/pkg/grpctest/cmd/mapping_helper/main.go | 2 + 2 files changed, 14 insertions(+), 49 deletions(-) diff --git a/v2/pkg/engine/datasource/grpc_datasource/compiler.go b/v2/pkg/engine/datasource/grpc_datasource/compiler.go index 460b5c59f..fb11f71ef 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/compiler.go +++ b/v2/pkg/engine/datasource/grpc_datasource/compiler.go @@ -244,47 +244,12 @@ type RPCCompiler struct { // ServiceByName returns a Service by its name. // Returns an empty Service if no service with the given name exists. func (d *Document) ServiceByName(name string) *Service { - for _, s := range d.Services { - if s.Name == name { - return &s - } - } - - return nil -} - -// MethodByName returns a Method by its name. -// Returns an empty Method if no method with the given name exists. -func (d *Document) MethodByName(name string) *Method { - for _, m := range d.Methods { - if m.Name == name { - return &m - } - } - - return nil -} - -// MethodRefByName returns the index of a Method in the Methods slice by its name. -// Returns -1 if no method with the given name exists. -func (d *Document) MethodRefByName(name string) int { - for i, m := range d.Methods { - if m.Name == name { - return i - } + node, found := d.nodeByName(name) + if !found || node.kind != NodeKindService { + return nil } - return -1 -} - -// MethodByRef returns a Method by its reference index. -func (d *Document) MethodByRef(ref int) Method { - return d.Methods[ref] -} - -// ServiceByRef returns a Service by its reference index. -func (d *Document) ServiceByRef(ref int) Service { - return d.Services[ref] + return &d.Services[node.ref] } // MessageByName returns a Message by its name. @@ -292,13 +257,12 @@ func (d *Document) ServiceByRef(ref int) Service { // We only expect this function to return false if either the message name was provided incorrectly, // or the schema and mapping was not properly configured. func (d *Document) MessageByName(name string) (Message, bool) { - for _, m := range d.Messages { - if m.Name == name { - return m, true - } + node, found := d.nodeByName(name) + if !found || node.kind != NodeKindMessage { + return Message{}, false } - return Message{}, false + return d.Messages[node.ref], true } // MessageRefByName returns the index of a Message in the Messages slice by its name. @@ -320,13 +284,12 @@ func (d *Document) MessageByRef(ref int) Message { // EnumByName returns an Enum by its name. // Returns false if the enum does not exist. func (d *Document) EnumByName(name string) (Enum, bool) { - for _, e := range d.Enums { - if e.Name == name { - return e, true - } + node, found := d.nodeByName(name) + if !found || node.kind != NodeKindEnum { + return Enum{}, false } - return Enum{}, false + return d.Enums[node.ref], true } // NewProtoCompiler compiles the protobuf schema into a Document structure. diff --git a/v2/pkg/grpctest/cmd/mapping_helper/main.go b/v2/pkg/grpctest/cmd/mapping_helper/main.go index 5d654db33..390c27584 100644 --- a/v2/pkg/grpctest/cmd/mapping_helper/main.go +++ b/v2/pkg/grpctest/cmd/mapping_helper/main.go @@ -1,3 +1,5 @@ +// mapping_helper is a tool that generates a mapping.go file from a mapping.json file. +// The mapping.go file is used for testing the grpc_datasource package. package main import ( From 1ed5ae65fd34f392e1a98c581123571029b3dc48 Mon Sep 17 00:00:00 2001 From: Ludwig Bedacht Date: Fri, 24 Oct 2025 11:15:51 +0200 Subject: [PATCH 29/72] fix: remove unsafe operation report --- .../datasource/grpc_datasource/compiler.go | 217 +++++++++++------- .../grpc_datasource/compiler_test.go | 38 +-- 2 files changed, 150 insertions(+), 105 deletions(-) diff --git a/v2/pkg/engine/datasource/grpc_datasource/compiler.go b/v2/pkg/engine/datasource/grpc_datasource/compiler.go index fb11f71ef..44a7eb663 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/compiler.go +++ b/v2/pkg/engine/datasource/grpc_datasource/compiler.go @@ -2,6 +2,7 @@ package grpcdatasource import ( "context" + "errors" "fmt" "slices" "strings" @@ -12,7 +13,6 @@ import ( "google.golang.org/protobuf/types/dynamicpb" "github.com/wundergraph/graphql-go-tools/v2/pkg/ast" - "github.com/wundergraph/graphql-go-tools/v2/pkg/operationreport" "github.com/wundergraph/graphql-go-tools/v2/pkg/pool" ) @@ -180,21 +180,36 @@ type Method struct { // Message represents a protobuf message type with its fields. type Message struct { + Fields map[uint64]Field Name string // The name of the message - Fields []Field // The fields in the message Desc protoref.MessageDescriptor // The protobuf descriptor for the message } -// FieldByName returns a field by its name. +// GetField returns a field by its name. // Returns nil if no field with the given name exists. -func (m *Message) FieldByName(name string) *Field { - for index, field := range m.Fields { - if field.Name == name { - return &m.Fields[index] - } +func (m *Message) GetField(name string) *Field { + digest := pool.Hash64.Get() + defer pool.Hash64.Put(digest) + _, _ = digest.WriteString(name) + + field, found := m.Fields[digest.Sum64()] + if !found { + return nil } - return nil + return &field +} + +func (m *Message) SetField(f Field) { + digest := pool.Hash64.Get() + defer pool.Hash64.Put(digest) + _, _ = digest.WriteString(f.Name) + + if m.Fields == nil { + m.Fields = make(map[uint64]Field) + } + + m.Fields[digest.Sum64()] = f } // Field represents a field in a protobuf message. @@ -238,7 +253,6 @@ type EnumValue struct { type RPCCompiler struct { doc *Document // The compiled Document Ancestor []Message - report operationreport.Report } // ServiceByName returns a Service by its name. @@ -321,7 +335,6 @@ func NewProtoCompiler(schema string, mapping *GRPCMapping) (*RPCCompiler, error) nodes: make(map[uint64]node), Package: string(schemaFile.Package()), }, - report: operationreport.Report{}, } // Extract information from the compiled file descriptor @@ -382,9 +395,9 @@ type ServiceCall struct { // MethodName is the name of the method on the service to call MethodName string // Input is the input message for the gRPC call - Input *dynamicpb.Message + Input protoref.Message // Output is the output message for the gRPC call - Output *dynamicpb.Message + Output protoref.Message // RPC is the call that was made to the gRPC service RPC *RPCCall } @@ -472,11 +485,23 @@ func (p *RPCCompiler) CompileNode(graph *DependencyGraph, fetch FetchItem, input return ServiceCall{}, fmt.Errorf("output message %s not found in document", call.Response.Name) } - request, response := p.newEmptyMessage(inputMessage), p.newEmptyMessage(outputMessage) + request, err := p.newEmptyMessage(inputMessage) + if err != nil { + return ServiceCall{}, err + } + + response, err := p.newEmptyMessage(outputMessage) + if err != nil { + return ServiceCall{}, err + } switch call.Kind { case CallKindStandard, CallKindEntity: - request = p.buildProtoMessage(inputMessage, &call.Request, inputData) + request, err = p.buildProtoMessage(inputMessage, &call.Request, inputData) + if err != nil { + return ServiceCall{}, err + } + case CallKindResolve: context, err := graph.FetchDependencies(&fetch) if err != nil { @@ -487,7 +512,10 @@ func (p *RPCCompiler) CompileNode(graph *DependencyGraph, fetch FetchItem, input return ServiceCall{}, fmt.Errorf("context is required for resolve calls") } - request = p.buildProtoMessageWithContext(inputMessage, &call.Request, inputData, context) + request, err = p.buildProtoMessageWithContext(inputMessage, &call.Request, inputData, context) + if err != nil { + return ServiceCall{}, err + } } serviceName, ok := p.resolveServiceName(call) @@ -523,13 +551,12 @@ func (p *RPCCompiler) resolveServiceName(call *RPCCall) (string, bool) { } // newEmptyMessage creates a new empty dynamicpb.Message from a Message definition. -func (p *RPCCompiler) newEmptyMessage(message Message) *dynamicpb.Message { +func (p *RPCCompiler) newEmptyMessage(message Message) (protoref.Message, error) { if p.doc.MessageRefByName(message.Name) == InvalidRef { - p.report.AddInternalError(fmt.Errorf("message %s not found in document", message.Name)) - return nil + return nil, fmt.Errorf("message %s not found in document", message.Name) } - return dynamicpb.NewMessage(message.Desc) + return dynamicpb.NewMessage(message.Desc), nil } // buildProtoMessageWithContext builds a protobuf message from an RPCMessage definition @@ -541,39 +568,34 @@ func (p *RPCCompiler) newEmptyMessage(message Message) *dynamicpb.Message { // repeated CategoryProductCountContext context = 1; // CategoryProductCountArgs field_args = 2; // } -func (p *RPCCompiler) buildProtoMessageWithContext(inputMessage Message, rpcMessage *RPCMessage, data gjson.Result, context []FetchItem) *dynamicpb.Message { +func (p *RPCCompiler) buildProtoMessageWithContext(inputMessage Message, rpcMessage *RPCMessage, data gjson.Result, context []FetchItem) (protoref.Message, error) { if rpcMessage == nil { - return nil + return nil, fmt.Errorf("rpc message is nil") } if p.doc.MessageRefByName(rpcMessage.Name) == InvalidRef { - p.report.AddInternalError(fmt.Errorf("message %s not found in document", rpcMessage.Name)) - return nil + return nil, fmt.Errorf("message %s not found in document", rpcMessage.Name) } rootMessage := dynamicpb.NewMessage(inputMessage.Desc) if len(inputMessage.Fields) != 2 { - p.report.AddInternalError(fmt.Errorf("message %s must have exactly two fields: context and field_args", inputMessage.Name)) - return nil + return nil, fmt.Errorf("message %s must have exactly two fields: context and field_args", inputMessage.Name) } - contextSchemaField := inputMessage.FieldByName("context") + contextSchemaField := inputMessage.GetField("context") if contextSchemaField == nil { - p.report.AddInternalError(fmt.Errorf("context field not found in message %s", inputMessage.Name)) - return nil + return nil, fmt.Errorf("context field not found in message %s", inputMessage.Name) } contextRPCField := rpcMessage.Fields.ByName(contextSchemaField.Name) if contextRPCField == nil { - p.report.AddInternalError(fmt.Errorf("context field not found in message %s", rpcMessage.Name)) - return nil + return nil, fmt.Errorf("context field not found in message %s", rpcMessage.Name) } contextField := rootMessage.Descriptor().Fields().ByNumber(protoref.FieldNumber(contextSchemaField.Number)) if contextField == nil { - p.report.AddInternalError(fmt.Errorf("context field not found in message %s", inputMessage.Name)) - return nil + return nil, fmt.Errorf("context field not found in message %s", inputMessage.Name) } contextList := p.newEmptyListMessageByName(rootMessage, contextSchemaField.Name) @@ -589,26 +611,27 @@ func (p *RPCCompiler) buildProtoMessageWithContext(inputMessage Message, rpcMess contextList.Append(val) } - argsSchemaField := inputMessage.FieldByName("field_args") + argsSchemaField := inputMessage.GetField("field_args") if argsSchemaField == nil { - p.report.AddInternalError(fmt.Errorf("field_args field not found in message %s", inputMessage.Name)) - return nil + return nil, fmt.Errorf("field_args field not found in message %s", inputMessage.Name) } argsMessage := p.doc.Messages[argsSchemaField.MessageRef] argsRPCField := rpcMessage.Fields.ByName("field_args") if argsRPCField == nil { - p.report.AddInternalError(fmt.Errorf("field_args field not found in message %s", rpcMessage.Name)) - return nil + return nil, fmt.Errorf("field_args field not found in message %s", rpcMessage.Name) } - args := p.buildProtoMessage(argsMessage, argsRPCField.Message, data) + args, err := p.buildProtoMessage(argsMessage, argsRPCField.Message, data) + if err != nil { + return nil, err + } // // Set the key list p.setMessageValue(rootMessage, contextSchemaField.Name, protoref.ValueOfList(contextList)) p.setMessageValue(rootMessage, argsRPCField.Name, protoref.ValueOfMessage(args)) - return rootMessage + return rootMessage, nil } func (p *RPCCompiler) resolveContextData(context FetchItem, contextField *RPCField) []map[string]protoref.Value { @@ -810,25 +833,24 @@ func (p *RPCCompiler) setMessageValue(message protoref.Message, fieldName string // buildProtoMessage recursively builds a protobuf message from an RPCMessage definition // and JSON data. It handles nested messages and repeated fields. -func (p *RPCCompiler) buildProtoMessage(inputMessage Message, rpcMessage *RPCMessage, data gjson.Result) *dynamicpb.Message { +func (p *RPCCompiler) buildProtoMessage(inputMessage Message, rpcMessage *RPCMessage, data gjson.Result) (protoref.Message, error) { if rpcMessage == nil { - return nil + return nil, errors.New("rpc message is nil") } inputMessageRef := p.doc.MessageRefByName(inputMessage.Name) if inputMessageRef == InvalidRef { - p.report.AddInternalError(fmt.Errorf("message %s not found in document", inputMessage.Name)) - return nil + return nil, fmt.Errorf("message %s not found in document", inputMessage.Name) } message := dynamicpb.NewMessage(inputMessage.Desc) - for _, field := range inputMessage.Fields { + for _, rpcField := range rpcMessage.Fields { fd := inputMessage.Desc.Fields() // Look up the field in the RPC message definition - rpcField := rpcMessage.Fields.ByName(field.Name) - if rpcField == nil { + field := inputMessage.GetField(rpcField.Name) + if field == nil { continue } @@ -852,7 +874,11 @@ func (p *RPCCompiler) buildProtoMessage(inputMessage Message, rpcMessage *RPCMes continue } - fieldMsg := p.buildProtoMessage(p.doc.Messages[field.MessageRef], rpcField.Message, element) + fieldMsg, err := p.buildProtoMessage(p.doc.Messages[field.MessageRef], rpcField.Message, element) + if err != nil { + return nil, err + } + list.Append(protoref.ValueOfMessage(fieldMsg)) default: list.Append(p.setValueForKind(field.Type, element)) @@ -864,10 +890,12 @@ func (p *RPCCompiler) buildProtoMessage(inputMessage Message, rpcMessage *RPCMes // Handle nested message fields if field.MessageRef >= 0 { - var fieldMsg *dynamicpb.Message + var ( + fieldMsg protoref.Message + err error + ) switch { - case rpcField.IsListType: // Nested and nullable lists are wrapped in a message, therefore we need to handle them differently // than repeated fields. We need to do this because protobuf repeated fields are not nullable and cannot be nested. @@ -878,21 +906,21 @@ func (p *RPCCompiler) buildProtoMessage(inputMessage Message, rpcMessage *RPCMes // } if !data.Get(rpcField.JSONPath).Exists() { if !rpcField.Optional { - p.report.AddInternalError(fmt.Errorf("field %s is required but has no value", rpcField.JSONPath)) + return nil, fmt.Errorf("field %s is required but has no value", rpcField.JSONPath) } continue } if rpcField.ListMetadata == nil { - p.report.AddInternalError(fmt.Errorf("list metadata not found for field %s", rpcField.JSONPath)) - continue + return nil, fmt.Errorf("list metadata not found for field %s", rpcField.JSONPath) } - fieldMsg = p.buildListMessage(inputMessage.Desc, field, rpcField, data) + fieldMsg = p.buildListMessage(inputMessage.Desc, field, &rpcField, data) if fieldMsg == nil { continue } + case rpcField.IsOptionalScalar(): // If the field is optional, we are handling a scalar value that is wrapped in a message // as protobuf scalar types are not nullable. @@ -903,13 +931,20 @@ func (p *RPCCompiler) buildProtoMessage(inputMessage Message, rpcMessage *RPCMes } // As those optional messages are well known wrapper types, we can convert them to the underlying message definition. - fieldMsg = p.buildProtoMessage( + fieldMsg, err = p.buildProtoMessage( p.doc.Messages[field.MessageRef], rpcField.ToOptionalTypeMessage(p.doc.Messages[field.MessageRef].Name), data, ) + + if err != nil { + return nil, err + } default: - fieldMsg = p.buildProtoMessage(p.doc.Messages[field.MessageRef], rpcField.Message, data.Get(rpcField.JSONPath)) + fieldMsg, err = p.buildProtoMessage(p.doc.Messages[field.MessageRef], rpcField.Message, data.Get(rpcField.JSONPath)) + if err != nil { + return nil, err + } } message.Set(inputMessage.Desc.Fields().ByName(protoref.Name(field.Name)), protoref.ValueOfMessage(fieldMsg)) @@ -917,11 +952,13 @@ func (p *RPCCompiler) buildProtoMessage(inputMessage Message, rpcMessage *RPCMes } if field.Type == DataTypeEnum { - if val := p.getEnumValue(rpcField.EnumName, data.Get(rpcField.JSONPath)); val != nil { - message.Set( - fd.ByName(protoref.Name(field.Name)), - *val, - ) + val, err := p.getEnumValue(rpcField.EnumName, data.Get(rpcField.JSONPath)) + if err != nil { + return nil, err + } + + if val != nil { + message.Set(fd.ByName(protoref.Name(field.Name)), *val) } continue @@ -932,7 +969,7 @@ func (p *RPCCompiler) buildProtoMessage(inputMessage Message, rpcMessage *RPCMes message.Set(fd.ByName(protoref.Name(field.Name)), p.setValueForKind(field.Type, value)) } - return message + return message, nil } // buildListMessage creates a new protobuf message, which reflects a wrapper type to work with a list in GraphQL. @@ -945,7 +982,7 @@ func (p *RPCCompiler) buildProtoMessage(inputMessage Message, rpcMessage *RPCMes // } // List list = 1; // } -func (p *RPCCompiler) buildListMessage(desc protoref.MessageDescriptor, field Field, rpcField *RPCField, data gjson.Result) *dynamicpb.Message { +func (p *RPCCompiler) buildListMessage(desc protoref.MessageDescriptor, field *Field, rpcField *RPCField, data gjson.Result) *dynamicpb.Message { rootMsg := dynamicpb.NewMessage(desc.Fields().ByName(protoref.Name(field.Name)).Message()) p.traverseList(rootMsg, 1, field, rpcField, data.Get(rpcField.JSONPath)) return rootMsg @@ -960,30 +997,28 @@ func (p *RPCCompiler) buildListMessage(desc protoref.MessageDescriptor, field Fi // } // List list = 1; // } -func (p *RPCCompiler) traverseList(rootMsg protoref.Message, level int, field Field, rpcField *RPCField, data gjson.Result) protoref.Message { +func (p *RPCCompiler) traverseList(rootMsg protoref.Message, level int, field *Field, rpcField *RPCField, data gjson.Result) (protoref.Message, error) { listFieldDesc := rootMsg.Descriptor().Fields().ByNumber(1) if listFieldDesc == nil { - p.report.AddInternalError(fmt.Errorf("field with number %d not found in message %s", 1, rootMsg.Descriptor().Name())) - return nil + return nil, fmt.Errorf("field with number %d not found in message %s", 1, rootMsg.Descriptor().Name()) } elements := data.Array() newListField := rootMsg.NewField(listFieldDesc) if len(elements) == 0 { if rpcField.ListMetadata.LevelInfo[level-1].Optional { - return nil + return nil, nil } rootMsg.Set(listFieldDesc, newListField) - return rootMsg + return rootMsg, nil } // Inside of a List message type we expect a repeated "items" field with field number 1 itemsFieldMsg := newListField.Message() itemsFieldDesc := itemsFieldMsg.Descriptor().Fields().ByNumber(1) if itemsFieldDesc == nil { - p.report.AddInternalError(fmt.Errorf("field with number %d not found in message %s", 1, itemsFieldMsg.Descriptor().Name())) - return nil + return nil, fmt.Errorf("field with number %d not found in message %s", 1, itemsFieldMsg.Descriptor().Name()) } itemsField := itemsFieldMsg.Mutable(itemsFieldDesc).List() @@ -993,18 +1028,27 @@ func (p *RPCCompiler) traverseList(rootMsg protoref.Message, level int, field Fi case DataTypeMessage: itemsFieldMsg, ok := p.doc.MessageByName(rpcField.Message.Name) if !ok { - p.report.AddInternalError(fmt.Errorf("message %s not found in document", rpcField.Message.Name)) - return nil + return nil, fmt.Errorf("message %s not found in document", rpcField.Message.Name) } for _, element := range elements { - if msg := p.buildProtoMessage(itemsFieldMsg, rpcField.Message, element); msg != nil { + msg, err := p.buildProtoMessage(itemsFieldMsg, rpcField.Message, element) + if err != nil { + return nil, err + } + + if msg != nil { itemsField.Append(protoref.ValueOfMessage(msg)) } } case DataTypeEnum: for _, element := range elements { - if val := p.getEnumValue(rpcField.EnumName, element); val != nil { + val, err := p.getEnumValue(rpcField.EnumName, element) + if err != nil { + return nil, err + } + + if val != nil { itemsField.Append(*val) } } @@ -1016,35 +1060,39 @@ func (p *RPCCompiler) traverseList(rootMsg protoref.Message, level int, field Fi itemsFieldMsg.Set(itemsFieldDesc, protoref.ValueOfList(itemsField)) rootMsg.Set(listFieldDesc, newListField) - return rootMsg + return rootMsg, nil } for _, element := range elements { newElement := itemsField.NewElement() - if val := p.traverseList(newElement.Message(), level+1, field, rpcField, element); val != nil { + val, err := p.traverseList(newElement.Message(), level+1, field, rpcField, element) + if err != nil { + return nil, err + } + + if val != nil { itemsField.Append(protoref.ValueOfMessage(val)) } } rootMsg.Set(listFieldDesc, newListField) - return rootMsg + return rootMsg, nil } -func (p *RPCCompiler) getEnumValue(enumName string, data gjson.Result) *protoref.Value { +func (p *RPCCompiler) getEnumValue(enumName string, data gjson.Result) (*protoref.Value, error) { enum, ok := p.doc.EnumByName(enumName) if !ok { - p.report.AddInternalError(fmt.Errorf("enum %s not found in document", enumName)) - return nil + return nil, fmt.Errorf("enum %s not found in document", enumName) } for _, enumValue := range enum.Values { if enumValue.GraphqlValue == data.String() { v := protoref.ValueOfEnum(protoref.EnumNumber(enumValue.Number)) - return &v + return &v, nil } } - return nil + return nil, nil } // setValueForKind converts a gjson.Result value to the appropriate protobuf value @@ -1167,8 +1215,6 @@ func (p *RPCCompiler) parseMessageDefinitions(messages protoref.MessageDescripto // enrichMessageData enriches the message data with the field information. func (p *RPCCompiler) enrichMessageData(ref int, m protoref.MessageDescriptor) { - fields := []Field{} - msg := p.doc.Messages[ref] // Process all fields in the message for i := 0; i < m.Fields().Len(); i++ { @@ -1181,10 +1227,9 @@ func (p *RPCCompiler) enrichMessageData(ref int, m protoref.MessageDescriptor) { field.MessageRef = p.doc.MessageRefByName(string(f.Message().Name())) } - fields = append(fields, field) + msg.SetField(field) } - msg.Fields = fields p.doc.Messages[ref] = msg } diff --git a/v2/pkg/engine/datasource/grpc_datasource/compiler_test.go b/v2/pkg/engine/datasource/grpc_datasource/compiler_test.go index 8a5fc6c9c..e2e175c73 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/compiler_test.go +++ b/v2/pkg/engine/datasource/grpc_datasource/compiler_test.go @@ -129,12 +129,12 @@ func TestNewProtoCompilerRecursiveType(t *testing.T) { require.Equal(t, 1, len(compiler.doc.Messages)) require.Equal(t, "RecursiveMessage", compiler.doc.Messages[0].Name) require.Equal(t, 2, len(compiler.doc.Messages[0].Fields)) - require.Equal(t, "nested", compiler.doc.Messages[0].Fields[1].Name) - require.Equal(t, "RecursiveMessage", compiler.doc.Messages[0].Fields[1].ResolveUnderlyingMessage(compiler.doc).Name) - require.Equal(t, 2, len(compiler.doc.Messages[0].Fields[1].ResolveUnderlyingMessage(compiler.doc).Fields)) - require.Equal(t, "id", compiler.doc.Messages[0].Fields[1].ResolveUnderlyingMessage(compiler.doc).Fields[0].Name) - require.Equal(t, "nested", compiler.doc.Messages[0].Fields[1].ResolveUnderlyingMessage(compiler.doc).Fields[1].Name) - require.Equal(t, "RecursiveMessage", compiler.doc.Messages[0].Fields[1].ResolveUnderlyingMessage(compiler.doc).Fields[1].ResolveUnderlyingMessage(compiler.doc).Name) + require.Equal(t, "nested", compiler.doc.Messages[0].GetField("nested").Name) + require.Equal(t, "RecursiveMessage", compiler.doc.Messages[0].GetField("nested").ResolveUnderlyingMessage(compiler.doc).Name) + require.Equal(t, 2, len(compiler.doc.Messages[0].GetField("nested").ResolveUnderlyingMessage(compiler.doc).Fields)) + require.Equal(t, "id", compiler.doc.Messages[0].GetField("nested").ResolveUnderlyingMessage(compiler.doc).GetField("id").Name) + require.Equal(t, "nested", compiler.doc.Messages[0].GetField("nested").ResolveUnderlyingMessage(compiler.doc).GetField("nested").Name) + require.Equal(t, "RecursiveMessage", compiler.doc.Messages[0].GetField("nested").ResolveUnderlyingMessage(compiler.doc).GetField("nested").ResolveUnderlyingMessage(compiler.doc).Name) } func TestNewProtoCompilerNestedRecursiveType(t *testing.T) { @@ -162,30 +162,30 @@ message RecursiveMessage { require.Equal(t, "NestedRecursiveMessage", compiler.doc.Messages[0].Name) require.Equal(t, 2, len(compiler.doc.Messages[0].Fields)) - require.Equal(t, "id", compiler.doc.Messages[0].Fields[0].Name) - require.Equal(t, "nested", compiler.doc.Messages[0].Fields[1].Name) + require.Equal(t, "id", compiler.doc.Messages[0].GetField("id").Name) + require.Equal(t, "nested", compiler.doc.Messages[0].GetField("nested").Name) - nested := compiler.doc.Messages[0].Fields[1].ResolveUnderlyingMessage(compiler.doc) + nested := compiler.doc.Messages[0].GetField("nested").ResolveUnderlyingMessage(compiler.doc) require.Equal(t, "RecursiveMessage", nested.Name) require.Equal(t, 2, len(nested.Fields)) - require.Equal(t, "id", nested.Fields[0].Name) - require.Equal(t, "nested", nested.Fields[1].Name) + require.Equal(t, "id", nested.GetField("id").Name) + require.Equal(t, "nested", nested.GetField("nested").Name) - nested = nested.Fields[1].ResolveUnderlyingMessage(compiler.doc) + nested = nested.GetField("nested").ResolveUnderlyingMessage(compiler.doc) require.Equal(t, "NestedRecursiveMessage", nested.Name) require.Equal(t, 2, len(nested.Fields)) - require.Equal(t, "id", nested.Fields[0].Name) - require.Equal(t, "nested", nested.Fields[1].Name) + require.Equal(t, "id", nested.GetField("id").Name) + require.Equal(t, "nested", nested.GetField("nested").Name) - nested = nested.Fields[1].ResolveUnderlyingMessage(compiler.doc) + nested = nested.GetField("nested").ResolveUnderlyingMessage(compiler.doc) require.Equal(t, "RecursiveMessage", nested.Name) require.Equal(t, 2, len(nested.Fields)) - require.Equal(t, "id", nested.Fields[0].Name) - require.Equal(t, "nested", nested.Fields[1].Name) - require.Equal(t, "NestedRecursiveMessage", nested.Fields[1].ResolveUnderlyingMessage(compiler.doc).Name) + require.Equal(t, "id", nested.GetField("id").Name) + require.Equal(t, "nested", nested.GetField("nested").Name) + require.Equal(t, "NestedRecursiveMessage", nested.GetField("nested").ResolveUnderlyingMessage(compiler.doc).Name) } // TestBuildProtoMessage tests the ability to build a protobuf message @@ -419,7 +419,7 @@ func TestCompileNestedLists(t *testing.T) { require.NoError(t, err) require.Equal(t, 1, len(serviceCalls)) - proto := serviceCalls[0].Input.ProtoReflect() + proto := serviceCalls[0].Input msgDesc := proto.Descriptor() From a59091db49cf96da6d3d40ef9f194d9e43163e9f Mon Sep 17 00:00:00 2001 From: Ludwig Bedacht Date: Fri, 24 Oct 2025 11:25:56 +0200 Subject: [PATCH 30/72] fix: traverse list --- .../datasource/grpc_datasource/compiler.go | 23 +++++++++++++++---- 1 file changed, 18 insertions(+), 5 deletions(-) diff --git a/v2/pkg/engine/datasource/grpc_datasource/compiler.go b/v2/pkg/engine/datasource/grpc_datasource/compiler.go index 44a7eb663..f4b29fb19 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/compiler.go +++ b/v2/pkg/engine/datasource/grpc_datasource/compiler.go @@ -916,7 +916,11 @@ func (p *RPCCompiler) buildProtoMessage(inputMessage Message, rpcMessage *RPCMes return nil, fmt.Errorf("list metadata not found for field %s", rpcField.JSONPath) } - fieldMsg = p.buildListMessage(inputMessage.Desc, field, &rpcField, data) + fieldMsg, err = p.buildListMessage(inputMessage.Desc, field, &rpcField, data) + if err != nil { + return nil, err + } + if fieldMsg == nil { continue } @@ -982,10 +986,19 @@ func (p *RPCCompiler) buildProtoMessage(inputMessage Message, rpcMessage *RPCMes // } // List list = 1; // } -func (p *RPCCompiler) buildListMessage(desc protoref.MessageDescriptor, field *Field, rpcField *RPCField, data gjson.Result) *dynamicpb.Message { - rootMsg := dynamicpb.NewMessage(desc.Fields().ByName(protoref.Name(field.Name)).Message()) - p.traverseList(rootMsg, 1, field, rpcField, data.Get(rpcField.JSONPath)) - return rootMsg +func (p *RPCCompiler) buildListMessage(desc protoref.MessageDescriptor, field *Field, rpcField *RPCField, data gjson.Result) (protoref.Message, error) { + msg, err := p.traverseList( + dynamicpb.NewMessage(desc.Fields().ByName(protoref.Name(field.Name)).Message()), + 1, + field, + rpcField, + data.Get(rpcField.JSONPath), + ) + + if err != nil { + return nil, err + } + return msg, nil } // traverseList makes sure we can handle nested lists properly. From b6625fd7a2692ca87f2215491100dadafbcaf1aa Mon Sep 17 00:00:00 2001 From: Ludwig Bedacht Date: Mon, 27 Oct 2025 10:17:31 +0100 Subject: [PATCH 31/72] chore: improve memory allocation --- .../datasource/grpc_datasource/compiler.go | 16 +++++++++++++--- .../engine/datasource/grpc_datasource/fetch.go | 9 --------- v2/pkg/engine/datasource/grpc_datasource/util.go | 10 ++++++++++ 3 files changed, 23 insertions(+), 12 deletions(-) create mode 100644 v2/pkg/engine/datasource/grpc_datasource/util.go diff --git a/v2/pkg/engine/datasource/grpc_datasource/compiler.go b/v2/pkg/engine/datasource/grpc_datasource/compiler.go index f4b29fb19..baf170221 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/compiler.go +++ b/v2/pkg/engine/datasource/grpc_datasource/compiler.go @@ -212,6 +212,11 @@ func (m *Message) SetField(f Field) { m.Fields[digest.Sum64()] = f } +// AllocFields allocates a new map of fields with the given count. +func (m *Message) AllocFields(count int) { + m.Fields = make(map[uint64]Field, count) +} + // Field represents a field in a protobuf message. type Field struct { Name string // The name of the field @@ -1228,7 +1233,8 @@ func (p *RPCCompiler) parseMessageDefinitions(messages protoref.MessageDescripto // enrichMessageData enriches the message data with the field information. func (p *RPCCompiler) enrichMessageData(ref int, m protoref.MessageDescriptor) { - msg := p.doc.Messages[ref] + fields := make([]Field, m.Fields().Len()) + msg := &p.doc.Messages[ref] // Process all fields in the message for i := 0; i < m.Fields().Len(); i++ { f := m.Fields().Get(i) @@ -1240,10 +1246,14 @@ func (p *RPCCompiler) enrichMessageData(ref int, m protoref.MessageDescriptor) { field.MessageRef = p.doc.MessageRefByName(string(f.Message().Name())) } - msg.SetField(field) + fields[i] = field } - p.doc.Messages[ref] = msg + msg.AllocFields(len(fields)) + + for i := range fields { + msg.SetField(fields[i]) + } } // parseField extracts information from a protobuf field descriptor. diff --git a/v2/pkg/engine/datasource/grpc_datasource/fetch.go b/v2/pkg/engine/datasource/grpc_datasource/fetch.go index b241af2b2..475226857 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/fetch.go +++ b/v2/pkg/engine/datasource/grpc_datasource/fetch.go @@ -160,12 +160,3 @@ func (g *DependencyGraph) FetchDependencies(fetch *FetchItem) ([]FetchItem, erro func (g *DependencyGraph) SetFetchData(index int, serviceCall *ServiceCall) { g.fetches[index].ServiceCall = serviceCall } - -// initializeSlice initializes a slice with a given length and a given value. -func initializeSlice[T any](len int, zero T) []T { - s := make([]T, len) - for i := range s { - s[i] = zero - } - return s -} diff --git a/v2/pkg/engine/datasource/grpc_datasource/util.go b/v2/pkg/engine/datasource/grpc_datasource/util.go new file mode 100644 index 000000000..1a6a1f55d --- /dev/null +++ b/v2/pkg/engine/datasource/grpc_datasource/util.go @@ -0,0 +1,10 @@ +package grpcdatasource + +// initializeSlice initializes a slice with a given length and a given value. +func initializeSlice[T any](len int, zero T) []T { + s := make([]T, len) + for i := range s { + s[i] = zero + } + return s +} From 1418287443b42b6fb5978e24aadd1e46d7a76b16 Mon Sep 17 00:00:00 2001 From: Ludwig Bedacht Date: Tue, 28 Oct 2025 09:55:45 +0100 Subject: [PATCH 32/72] chore: addressing pr comments --- .../datasource/grpc_datasource/compiler.go | 23 +++++++++++++------ .../grpc_datasource/execution_plan.go | 8 +++---- .../execution_plan_federation_test.go | 16 ++++++++++--- .../grpc_datasource/execution_plan_test.go | 17 ++++++++++++-- .../grpc_datasource/grpc_datasource.go | 5 +++- 5 files changed, 52 insertions(+), 17 deletions(-) diff --git a/v2/pkg/engine/datasource/grpc_datasource/compiler.go b/v2/pkg/engine/datasource/grpc_datasource/compiler.go index baf170221..288aff3ab 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/compiler.go +++ b/v2/pkg/engine/datasource/grpc_datasource/compiler.go @@ -632,7 +632,7 @@ func (p *RPCCompiler) buildProtoMessageWithContext(inputMessage Message, rpcMess return nil, err } - // // Set the key list + // Set the context and args fields p.setMessageValue(rootMessage, contextSchemaField.Name, protoref.ValueOfList(contextList)) p.setMessageValue(rootMessage, argsRPCField.Name, protoref.ValueOfMessage(args)) @@ -648,8 +648,6 @@ func (p *RPCCompiler) resolveContextData(context FetchItem, contextField *RPCFie for _, field := range contextField.Message.Fields { values := p.resolveContextDataForPath(context.ServiceCall.Output, field.ResolvePath) - contextValues = slices.Grow(contextValues, len(values)) - for index, value := range values { if index >= len(contextValues) { contextValues = append(contextValues, make(map[string]protoref.Value)) @@ -663,6 +661,7 @@ func (p *RPCCompiler) resolveContextData(context FetchItem, contextField *RPCFie return contextValues } +// resolveContextDataForPath resolves the data for a given path in the context message. func (p *RPCCompiler) resolveContextDataForPath(message protoref.Message, path ast.Path) []protoref.Value { if path.Len() == 0 { return nil @@ -684,6 +683,7 @@ func (p *RPCCompiler) resolveContextDataForPath(message protoref.Message, path a } +// resolveListDataForPath resolves the data for a given path in a list message. func (p *RPCCompiler) resolveListDataForPath(message protoref.List, fd protoref.FieldDescriptor, path ast.Path) []protoref.Value { if path.Len() == 0 { return nil @@ -715,6 +715,7 @@ func (p *RPCCompiler) resolveListDataForPath(message protoref.List, fd protoref. return result } +// resolveDataForPath resolves the data for a given path in a message. func (p *RPCCompiler) resolveDataForPath(messsage protoref.Message, path ast.Path) []protoref.Value { if path.Len() == 0 { return nil @@ -751,9 +752,7 @@ func (p *RPCCompiler) resolveDataForPath(messsage protoref.Message, path ast.Pat } // getMessageField gets the field from the message by its name. -// It also handles nested lists and nullable lists. func (p *RPCCompiler) getMessageField(message protoref.Message, fieldName string) (protoref.Value, protoref.FieldDescriptor) { - fd := message.Descriptor().Fields().ByName(protoref.Name(fieldName)) if fd == nil { return protoref.Value{}, nil @@ -788,9 +787,19 @@ func (p *RPCCompiler) resolveUnderlyingList(msg protoref.Message, fieldName stri } +// resolveUnderlyingListItems resolves the items in a list message. +// +// message ListOfFloat { +// message List { +// repeated double items = 1; +// } +// List list = 1; +// } func (p *RPCCompiler) resolveUnderlyingListItems(value protoref.Value, nestingLevel int) []protoref.Value { + // The field number of the list and items field in the message + const listAndItemsFieldNumber = 1 msg := value.Message() - fd := msg.Descriptor().Fields().ByNumber(1) + fd := msg.Descriptor().Fields().ByNumber(listAndItemsFieldNumber) if fd == nil { return nil } @@ -800,7 +809,7 @@ func (p *RPCCompiler) resolveUnderlyingListItems(value protoref.Value, nestingLe return nil } - itemsValue := listMsg.Message().Get(listMsg.Message().Descriptor().Fields().ByNumber(1)) + itemsValue := listMsg.Message().Get(listMsg.Message().Descriptor().Fields().ByNumber(listAndItemsFieldNumber)) if !itemsValue.IsValid() { return nil } diff --git a/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go b/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go index 46e8b758c..b5db96937 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go +++ b/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go @@ -330,9 +330,9 @@ type PlanVisitor interface { // The planner is responsible for creating an RPCExecutionPlan from a given // GraphQL operation. It is used by the engine to execute operations against // gRPC services. -func NewPlanner(subgraphName string, mapping *GRPCMapping, federationConfigs plan.FederationFieldConfigurations) PlanVisitor { +func NewPlanner(subgraphName string, mapping *GRPCMapping, federationConfigs plan.FederationFieldConfigurations) (PlanVisitor, error) { if mapping == nil { - mapping = new(GRPCMapping) + return nil, fmt.Errorf("mapping is required") } if len(federationConfigs) > 0 { @@ -340,14 +340,14 @@ func NewPlanner(subgraphName string, mapping *GRPCMapping, federationConfigs pla subgraphName: subgraphName, mapping: mapping, federationConfigs: federationConfigs, - }) + }), nil } return newRPCPlanVisitor(rpcPlanVisitorConfig{ subgraphName: subgraphName, mapping: mapping, federationConfigs: federationConfigs, - }) + }), nil } // formatRPCMessage formats an RPCMessage and adds it to the string builder with the specified indentation diff --git a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_federation_test.go b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_federation_test.go index 06f0f0e76..21d0a8615 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_federation_test.go +++ b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_federation_test.go @@ -282,7 +282,11 @@ func TestEntityLookup(t *testing.T) { t.Fatalf("failed to parse query: %s", report.Error()) } - planner := NewPlanner("Products", tt.mapping, tt.federationConfigs) + planner, err := NewPlanner("Products", tt.mapping, tt.federationConfigs) + if err != nil { + t.Fatalf("failed to create planner %s", err) + } + plan, err := planner.PlanOperation(&queryDoc, &schemaDoc) if err != nil { t.Fatalf("failed to plan operation: %s", err) @@ -1302,7 +1306,10 @@ func TestEntityLookupWithFieldResolvers(t *testing.T) { t.Fatalf("failed to parse query: %s", report.Error()) } - planner := NewPlanner("Products", tt.mapping, tt.federationConfigs) + planner, err := NewPlanner("Products", tt.mapping, tt.federationConfigs) + if err != nil { + t.Fatalf("failed to create planner: %s", err) + } plan, err := planner.PlanOperation(&queryDoc, &schemaDoc) if err != nil { t.Fatalf("failed to plan operation: %s", err) @@ -1350,7 +1357,10 @@ func runFederationTest(t *testing.T, tt struct { t.Fatalf("failed to validate query: %s", report.Error()) } - planner := NewPlanner("Products", tt.mapping, tt.federationConfigs) + planner, err := NewPlanner("Products", tt.mapping, tt.federationConfigs) + if err != nil { + t.Fatalf("failed to create planner: %s", err) + } plan, err := planner.PlanOperation(&operation, &definition) if err != nil { t.Fatalf("failed to plan operation: %s", err) diff --git a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_test.go b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_test.go index 20c9e9a19..760a3e786 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_test.go +++ b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_test.go @@ -1369,8 +1369,14 @@ func TestProductExecutionPlan(t *testing.T) { t.Fatalf("failed to validate query: %s", report.Error()) } - planner := NewPlanner("Products", testMapping(), nil) + planner, err := NewPlanner("Products", testMapping(), nil) + if err != nil { + t.Fatalf("failed to create planner: %s", err) + } outPlan, err := planner.PlanOperation(&queryDoc, &schemaDoc) + if err != nil { + t.Fatalf("failed to plan operation: %s", err) + } if tt.expectedError != "" { if err == nil { @@ -2473,8 +2479,15 @@ func TestProductExecutionPlanWithAliases(t *testing.T) { t.Fatalf("failed to validate query: %s", report.Error()) } - planner := NewPlanner("Products", testMapping(), nil) + planner, err := NewPlanner("Products", testMapping(), nil) + if err != nil { + t.Fatalf("failed to create planner: %s", err) + } + outPlan, err := planner.PlanOperation(&queryDoc, &schemaDoc) + if err != nil { + t.Fatalf("failed to plan operation: %s", err) + } if tt.expectedError != "" { if err == nil { diff --git a/v2/pkg/engine/datasource/grpc_datasource/grpc_datasource.go b/v2/pkg/engine/datasource/grpc_datasource/grpc_datasource.go index 940d9a4ce..8a196cbc6 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/grpc_datasource.go +++ b/v2/pkg/engine/datasource/grpc_datasource/grpc_datasource.go @@ -62,7 +62,10 @@ type DataSourceConfig struct { // NewDataSource creates a new gRPC datasource func NewDataSource(client grpc.ClientConnInterface, config DataSourceConfig) (*DataSource, error) { - planner := NewPlanner(config.SubgraphName, config.Mapping, config.FederationConfigs) + planner, err := NewPlanner(config.SubgraphName, config.Mapping, config.FederationConfigs) + if err != nil { + return nil, err + } plan, err := planner.PlanOperation(config.Operation, config.Definition) if err != nil { return nil, err From 59fdcbd8abfb6a13f4d33637c165122eb362d89f Mon Sep 17 00:00:00 2001 From: Ludwig Bedacht Date: Tue, 28 Oct 2025 10:29:46 +0100 Subject: [PATCH 33/72] chore: improve variable names and function descriptions --- .../grpc_datasource/execution_plan.go | 77 +++++++++++-------- 1 file changed, 47 insertions(+), 30 deletions(-) diff --git a/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go b/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go index b5db96937..f349bd5b4 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go +++ b/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go @@ -617,7 +617,7 @@ func (r *rpcPlanningContext) createRPCFieldFromFieldArgument(fieldArg fieldArgum // buildMessageFromInputObjectType builds a message from an input object type definition. func (r *rpcPlanningContext) buildMessageFromInputObjectType(node *ast.Node) (*RPCMessage, error) { if node.Kind != ast.NodeKindInputObjectTypeDefinition { - return nil, fmt.Errorf("unsupported type: %s", node.Kind) + return nil, fmt.Errorf("unable to build message from input object type definition - incorrect type: %s", node.Kind) } inputObjectDefinition := r.definition.InputObjectTypeDefinitions[node.Ref] @@ -637,13 +637,15 @@ func (r *rpcPlanningContext) buildMessageFromInputObjectType(node *ast.Node) (*R return message, nil } +// buildMessageFieldFromInputValueDefinition builds an RPCField from an input value definition. func (r *rpcPlanningContext) buildMessageFieldFromInputValueDefinition(ivdRef int, node *ast.Node) (RPCField, error) { - ivd := r.definition.InputValueDefinitions[ivdRef] - ivdType := r.definition.Types[ivd.Type] + inputValueDef := r.definition.InputValueDefinitions[ivdRef] + inputValueDefType := r.definition.Types[inputValueDef.Type] - underlyingTypeNode, found := r.nodeByTypeRef(ivd.Type) + // We need to resolve the underlying type to determine whether we are building a nested message or a scalar type. + underlyingTypeNode, found := r.nodeByTypeRef(inputValueDef.Type) if !found { - return RPCField{}, fmt.Errorf("unable to resolve underlying type node for input value definition %s", r.definition.Input.ByteSliceString(ivd.Name)) + return RPCField{}, fmt.Errorf("unable to resolve underlying type node for input value definition %s", r.definition.Input.ByteSliceString(inputValueDef.Name)) } var ( @@ -651,6 +653,7 @@ func (r *rpcPlanningContext) buildMessageFieldFromInputValueDefinition(ivdRef in err error ) + // If the type is an input object type, we need to build a nested message. dt := DataTypeMessage switch underlyingTypeNode.Kind { case ast.NodeKindInputObjectTypeDefinition: @@ -659,13 +662,13 @@ func (r *rpcPlanningContext) buildMessageFieldFromInputValueDefinition(ivdRef in return RPCField{}, err } default: - dt = r.toDataType(&ivdType) + dt = r.toDataType(&inputValueDefType) } - fieldName := r.definition.Input.ByteSliceString(ivd.Name) + fieldName := r.definition.Input.ByteSliceString(inputValueDef.Name) mappedName := r.resolveFieldMapping(node.NameString(r.definition), fieldName) - field, err := r.buildInputMessageField(ivd.Type, mappedName, fieldName, dt) + field, err := r.buildInputMessageField(inputValueDef.Type, mappedName, fieldName, dt) if err != nil { return RPCField{}, err } @@ -674,6 +677,8 @@ func (r *rpcPlanningContext) buildMessageFieldFromInputValueDefinition(ivdRef in return field, nil } +// buildInputMessageField builds an RPCField from an input value definition. +// It handles scalar, enum and list types. func (r *rpcPlanningContext) buildInputMessageField(typeRef int, fieldName, jsonPath string, dt DataType) (RPCField, error) { field := RPCField{ Name: fieldName, @@ -726,8 +731,10 @@ type resolvedField struct { fieldArguments []fieldArgument } -func (r *rpcPlanningContext) setResolvedField(walker *astvisitor.Walker, fd int, fieldArgs []int, fieldPath ast.Path, resolvedField *resolvedField) error { - contextFields, err := r.resolveContextFields(walker, fd) +// setResolvedField sets the resolved field for a given field definition reference. +func (r *rpcPlanningContext) setResolvedField(walker *astvisitor.Walker, fieldDefRef int, fieldArgs []int, fieldPath ast.Path, resolvedField *resolvedField) error { + // We need to resolve the context fields for the given field definition reference. + contextFields, err := r.resolveContextFields(walker, fieldDefRef) if err != nil { return err } @@ -742,7 +749,7 @@ func (r *rpcPlanningContext) setResolvedField(walker *astvisitor.Walker, fd int, }) } - fieldArguments, err := r.parseFieldArguments(walker, fd, fieldArgs) + fieldArguments, err := r.parseFieldArguments(walker, fieldDefRef, fieldArgs) if err != nil { return err } @@ -751,8 +758,11 @@ func (r *rpcPlanningContext) setResolvedField(walker *astvisitor.Walker, fd int, return nil } -func (r *rpcPlanningContext) resolveContextFields(walker *astvisitor.Walker, fd int) ([]int, error) { - contextDirectiveRef, exists := r.definition.FieldDefinitionDirectiveByName(fd, ast.ByteSlice(resolverContextDirectiveName)) +// resolveContextFields resolves the context fields for a given field definition reference. +// The function attempts to resolve the context fields from the @connect__fieldResolver directive. +// If the directive is not present it instead attempts to resolve the ID field. +func (r *rpcPlanningContext) resolveContextFields(walker *astvisitor.Walker, fieldDefRef int) ([]int, error) { + contextDirectiveRef, exists := r.definition.FieldDefinitionDirectiveByName(fieldDefRef, ast.ByteSlice(resolverContextDirectiveName)) if exists { fields, err := r.getFieldsFromContext(walker.EnclosingTypeDefinition, contextDirectiveRef) if err != nil { @@ -762,37 +772,36 @@ func (r *rpcPlanningContext) resolveContextFields(walker *astvisitor.Walker, fd return fields, nil } - idFieldRef, err := r.findIDField(walker.EnclosingTypeDefinition, fd) + // If the directive is not present it instead attempts to resolve the ID field. + idFieldRef, err := r.findIDField(walker.EnclosingTypeDefinition, fieldDefRef) return []int{idFieldRef}, err } -func (r *rpcPlanningContext) parseFieldArguments(walker *astvisitor.Walker, fd int, fieldArgs []int) ([]fieldArgument, error) { +// parseFieldArguments parses the field arguments for a given field definition reference. +func (r *rpcPlanningContext) parseFieldArguments(walker *astvisitor.Walker, fieldDefRef int, fieldArgs []int) ([]fieldArgument, error) { result := make([]fieldArgument, 0, len(fieldArgs)) for _, fieldArgRef := range fieldArgs { arg := r.operation.Arguments[fieldArgRef] - fieldArg := r.operation.ArgumentNameString(fieldArgRef) - fieldType := arg.Value.Kind + + if arg.Value.Kind != ast.ValueKindVariable { + return nil, fmt.Errorf("unsupported argument value kind: %s", arg.Value.Kind) + } argDefRef := r.definition.NodeFieldDefinitionArgumentDefinitionByName( walker.EnclosingTypeDefinition, - r.definition.FieldDefinitionNameBytes(fd), + r.definition.FieldDefinitionNameBytes(fieldDefRef), r.operation.ArgumentNameBytes(fieldArgRef), ) if argDefRef == ast.InvalidRef { - return nil, fmt.Errorf("unable to resolve argument input value definition for argument %s", fieldArg) - } - - jsonValue := fieldArg - if fieldType == ast.ValueKindVariable { - jsonValue = r.operation.Input.ByteSliceString(r.operation.VariableValues[arg.Value.Ref].Name) + return nil, fmt.Errorf("unable to resolve argument input value definition for argument %s", r.operation.ArgumentNameString(fieldArgRef)) } result = append(result, fieldArgument{ - fieldDefinitionRef: fd, + fieldDefinitionRef: fieldDefRef, argumentDefinitionRef: argDefRef, parentTypeNode: walker.EnclosingTypeDefinition, - jsonPath: jsonValue, + jsonPath: r.operation.VariableValueNameString(arg.Value.Ref), }) } @@ -820,11 +829,16 @@ func (r *rpcPlanningContext) getFieldsFromContext(parentNode ast.Node, contextRe return v.fieldDefinitionRefs, nil } -func (r *rpcPlanningContext) findIDField(parentNode ast.Node, fd int) (int, error) { +// findIDField attempts to find the ID field for a given field definition reference. +// It fails if the parent node is not an object type definition. +// The functions checks whether an available ID field is present in the object type definition. +// If exactly one ID field is found, it returns the field definition reference. +// If none or multiple ID fields are found, it returns an error. +func (r *rpcPlanningContext) findIDField(parentNode ast.Node, fieldDefRef int) (int, error) { switch parentNode.Kind { case ast.NodeKindObjectTypeDefinition: o := r.definition.ObjectTypeDefinitions[parentNode.Ref] - result := slices.Collect(r.filterIDFieldsFunc(o, fd)) + result := slices.Collect(r.filterIDFieldsFunc(o, fieldDefRef)) if len(result) == 0 { return ast.InvalidRef, fmt.Errorf("unable to determine ID field in object type %s", parentNode.NameString(r.definition)) @@ -840,11 +854,14 @@ func (r *rpcPlanningContext) findIDField(parentNode ast.Node, fd int) (int, erro } } -func (r *rpcPlanningContext) filterIDFieldsFunc(o ast.ObjectTypeDefinition, fd int) func(yield func(int) bool) { +// filterIDFieldsFunc is a helper function to filter the ID fields from the object type definition. +// It yields the field definition references for the ID fields. +// It skips the field definition reference for the given field definition reference. +func (r *rpcPlanningContext) filterIDFieldsFunc(o ast.ObjectTypeDefinition, fieldDefRef int) func(yield func(int) bool) { fieldRefs := o.FieldsDefinition.Refs return func(yield func(int) bool) { for _, ref := range fieldRefs { - if ref == fd { + if ref == fieldDefRef { continue } From 49f0dc449721cfadd46cf93f6463f2f12aabc048 Mon Sep 17 00:00:00 2001 From: Ludwig Bedacht Date: Tue, 28 Oct 2025 10:32:28 +0100 Subject: [PATCH 34/72] chore: improve naming for directive --- .../datasource/grpc_datasource/execution_plan.go | 12 +++++++----- 1 file changed, 7 insertions(+), 5 deletions(-) diff --git a/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go b/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go index f349bd5b4..020b5e1a7 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go +++ b/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go @@ -15,8 +15,8 @@ const ( // in a message as protobuf scalar types are not nullable. knownTypeOptionalFieldValueName = "value" - // resolverContextDirectiveName is the name of the directive that is used to configure the resolver context. - resolverContextDirectiveName = "connect__fieldResolver" + // fieldResolverDirectiveName is the name of the directive that is used to configure the resolver context. + fieldResolverDirectiveName = "connect__fieldResolver" // typenameFieldName is the name of the field that is used to store the typename of the object. typenameFieldName = "__typename" @@ -762,9 +762,9 @@ func (r *rpcPlanningContext) setResolvedField(walker *astvisitor.Walker, fieldDe // The function attempts to resolve the context fields from the @connect__fieldResolver directive. // If the directive is not present it instead attempts to resolve the ID field. func (r *rpcPlanningContext) resolveContextFields(walker *astvisitor.Walker, fieldDefRef int) ([]int, error) { - contextDirectiveRef, exists := r.definition.FieldDefinitionDirectiveByName(fieldDefRef, ast.ByteSlice(resolverContextDirectiveName)) + resolverDirectiveRef, exists := r.definition.FieldDefinitionDirectiveByName(fieldDefRef, ast.ByteSlice(fieldResolverDirectiveName)) if exists { - fields, err := r.getFieldsFromContext(walker.EnclosingTypeDefinition, contextDirectiveRef) + fields, err := r.getFieldsFromFieldResolverDirective(walker.EnclosingTypeDefinition, resolverDirectiveRef) if err != nil { return nil, err } @@ -810,7 +810,9 @@ func (r *rpcPlanningContext) parseFieldArguments(walker *astvisitor.Walker, fiel } -func (r *rpcPlanningContext) getFieldsFromContext(parentNode ast.Node, contextRef int) ([]int, error) { +// getFieldsFromFieldResolverDirective gets the fields from the @connect__fieldResolver directive. +// It returns the field definition references for the fields in the context. +func (r *rpcPlanningContext) getFieldsFromFieldResolverDirective(parentNode ast.Node, contextRef int) ([]int, error) { val, exists := r.definition.DirectiveArgumentValueByName(contextRef, []byte("context")) if !exists { return nil, fmt.Errorf("context directive argument not found") From ca093fee82795ac2e2d767532eb0f57d048ed896 Mon Sep 17 00:00:00 2001 From: Ludwig Bedacht Date: Tue, 28 Oct 2025 10:41:01 +0100 Subject: [PATCH 35/72] chore: move function to ast package --- v2/pkg/ast/ast.go | 5 +++++ .../datasource/grpc_datasource/execution_plan.go | 10 ++-------- 2 files changed, 7 insertions(+), 8 deletions(-) diff --git a/v2/pkg/ast/ast.go b/v2/pkg/ast/ast.go index 1249e5082..e724209a1 100644 --- a/v2/pkg/ast/ast.go +++ b/v2/pkg/ast/ast.go @@ -236,6 +236,11 @@ func (d *Document) NodeByNameStr(name string) (Node, bool) { return d.Index.FirstNodeByNameStr(name) } +// ResolveNodeFromTypeRef returns the `ast.Node` for a given type reference. +func (d *Document) ResolveNodeFromTypeRef(typeRef int) (Node, bool) { + return d.Index.FirstNodeByNameStr(d.ResolveTypeNameString(typeRef)) +} + func (d *Document) TypeDefinitionContainsImplementsInterface(typeName, interfaceName ByteSlice) bool { typeDefinition, exists := d.Index.FirstNodeByNameBytes(typeName) if !exists { diff --git a/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go b/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go index 020b5e1a7..54ea545d4 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go +++ b/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go @@ -573,7 +573,7 @@ func (r *rpcPlanningContext) buildField(enclosingTypeNode ast.Node, fd int, fiel func (r *rpcPlanningContext) createRPCFieldFromFieldArgument(fieldArg fieldArgument) (RPCField, error) { argDef := r.definition.InputValueDefinitions[fieldArg.argumentDefinitionRef] argName := r.definition.Input.ByteSliceString(argDef.Name) - underlyingTypeNode, found := r.nodeByTypeRef(argDef.Type) + underlyingTypeNode, found := r.definition.ResolveNodeFromTypeRef(argDef.Type) if !found { return RPCField{}, fmt.Errorf("unable to resolve underlying type node for argument %s", argName) } @@ -643,7 +643,7 @@ func (r *rpcPlanningContext) buildMessageFieldFromInputValueDefinition(ivdRef in inputValueDefType := r.definition.Types[inputValueDef.Type] // We need to resolve the underlying type to determine whether we are building a nested message or a scalar type. - underlyingTypeNode, found := r.nodeByTypeRef(inputValueDef.Type) + underlyingTypeNode, found := r.definition.ResolveNodeFromTypeRef(inputValueDef.Type) if !found { return RPCField{}, fmt.Errorf("unable to resolve underlying type node for input value definition %s", r.definition.Input.ByteSliceString(inputValueDef.Name)) } @@ -879,12 +879,6 @@ func (r *rpcPlanningContext) filterIDFieldsFunc(o ast.ObjectTypeDefinition, fiel } } -// nodeByTypeRef is a helper function to resolve the underlying type node for a given type reference. -func (r *rpcPlanningContext) nodeByTypeRef(typeRef int) (ast.Node, bool) { - underlyingTypeName := r.definition.ResolveTypeNameString(typeRef) - return r.definition.NodeByNameStr(underlyingTypeName) -} - type resolveRPCCallConfig struct { resolveConfig *ResolveRPCTypeField resolvedField *resolvedField From b5225e8238ac1bb7657aa30c0897e813f286d336 Mon Sep 17 00:00:00 2001 From: Ludwig Bedacht Date: Tue, 28 Oct 2025 12:19:09 +0100 Subject: [PATCH 36/72] chore: improve naming --- v2/pkg/engine/datasource/grpc_datasource/execution_plan.go | 4 ++-- .../datasource/grpc_datasource/execution_plan_visitor.go | 4 +--- .../grpc_datasource/execution_plan_visitor_federation.go | 2 +- 3 files changed, 4 insertions(+), 6 deletions(-) diff --git a/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go b/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go index 54ea545d4..3346ae1f2 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go +++ b/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go @@ -724,7 +724,7 @@ type resolvedField struct { parentTypeRef int fieldRef int fieldDefinitionTypeRef int - requiredFieldSelection int + fieldsSelectionSetRef int responsePath ast.Path contextFields []contextField @@ -994,7 +994,7 @@ func (r *rpcPlanningContext) newResolveRPCCall(config *resolveRPCCallConfig) (RP var err error responseFieldsMessage, err = r.resolveRequiredFields( r.definition.ResolveTypeNameString(underlyingTypeRef), - resolvedField.requiredFieldSelection, + resolvedField.fieldsSelectionSetRef, ) if err != nil { diff --git a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor.go b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor.go index 04a784d66..574e9fb3c 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor.go +++ b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor.go @@ -144,8 +144,6 @@ func (r *rpcPlanVisitor) EnterOperationDefinition(ref int) { // EnterArgument implements astvisitor.EnterArgumentVisitor. // This method retrieves the input value definition for the argument // and builds the request message from the input argument. -// -// TODO handle field arguments to define resolvers func (r *rpcPlanVisitor) EnterArgument(ref int) { ancestor := r.walker.Ancestor() if ancestor.Kind != ast.NodeKindField || ancestor.Ref != r.operationFieldRef { @@ -200,7 +198,7 @@ func (r *rpcPlanVisitor) EnterSelectionSet(ref int) { // If we are inside of a resolved field that selects multiple fields, we get all the fields from the input and pass them to the required fields visitor. if r.resolvedFieldIndex != ast.InvalidRef { - r.resolvedFields[r.resolvedFieldIndex].requiredFieldSelection = ref + r.resolvedFields[r.resolvedFieldIndex].fieldsSelectionSetRef = ref r.walker.SkipNode() return } diff --git a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor_federation.go b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor_federation.go index f57de6212..c4f788dd4 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor_federation.go +++ b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor_federation.go @@ -188,7 +188,7 @@ func (r *rpcPlanVisitorFederation) EnterSelectionSet(ref int) { // If we are inside of a resolved field that selects multiple fields, we get all the fields from the input and pass them to the required fields visitor. if r.resolvedFieldIndex != ast.InvalidRef { - r.resolvedFields[r.resolvedFieldIndex].requiredFieldSelection = ref + r.resolvedFields[r.resolvedFieldIndex].fieldsSelectionSetRef = ref r.walker.SkipNode() return } From c9ebb5992808e09b52cb476711cfe267892fb674 Mon Sep 17 00:00:00 2001 From: Ludwig Bedacht Date: Tue, 28 Oct 2025 12:46:33 +0100 Subject: [PATCH 37/72] chore: more polishing --- v2/pkg/engine/datasource/grpc_datasource/compiler.go | 6 ------ .../datasource/grpc_datasource/execution_plan_visitor.go | 1 + v2/pkg/engine/datasource/grpc_datasource/json_builder.go | 6 ++++++ 3 files changed, 7 insertions(+), 6 deletions(-) diff --git a/v2/pkg/engine/datasource/grpc_datasource/compiler.go b/v2/pkg/engine/datasource/grpc_datasource/compiler.go index 288aff3ab..20ac0af8c 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/compiler.go +++ b/v2/pkg/engine/datasource/grpc_datasource/compiler.go @@ -387,12 +387,6 @@ func (p *RPCCompiler) processFile(f protoref.FileDescriptor, mapping *GRPCMappin } } -// ConstructExecutionPlan constructs an RPCExecutionPlan from a parsed GraphQL operation and schema. -// It will return an error if the operation does not match the protobuf definition provided to the compiler. -func (p *RPCCompiler) ConstructExecutionPlan(operation, schema *ast.Document) (*RPCExecutionPlan, error) { - return nil, nil -} - // ServiceCall represents a single gRPC service call with its input and output messages. type ServiceCall struct { // ServiceName is the name of the gRPC service to call diff --git a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor.go b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor.go index 574e9fb3c..5b35371ee 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor.go +++ b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor.go @@ -198,6 +198,7 @@ func (r *rpcPlanVisitor) EnterSelectionSet(ref int) { // If we are inside of a resolved field that selects multiple fields, we get all the fields from the input and pass them to the required fields visitor. if r.resolvedFieldIndex != ast.InvalidRef { + // TODO: handle nested resolved fields. r.resolvedFields[r.resolvedFieldIndex].fieldsSelectionSetRef = ref r.walker.SkipNode() return diff --git a/v2/pkg/engine/datasource/grpc_datasource/json_builder.go b/v2/pkg/engine/datasource/grpc_datasource/json_builder.go index 0dc12bc8f..6e521b041 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/json_builder.go +++ b/v2/pkg/engine/datasource/grpc_datasource/json_builder.go @@ -220,6 +220,7 @@ func (j *jsonBuilder) mergeEntities(left *astjson.Value, right *astjson.Value) ( return entities, nil } +// mergeWithPath merges a JSON value with a resolved value by its path. func (j *jsonBuilder) mergeWithPath(base *astjson.Value, resolved *astjson.Value, path ast.Path) error { if len(path) == 0 { return errors.New("path is empty") @@ -232,6 +233,7 @@ func (j *jsonBuilder) mergeWithPath(base *astjson.Value, resolved *astjson.Value responseValues := make([]*astjson.Value, 0, len(resolvedValues)) + // We retrieve a flat list of all the values that we need to merge with the resolved values. current := base current = current.Get(searchPath[0].FieldName.String()) switch current.Type() { @@ -261,6 +263,8 @@ func (j *jsonBuilder) mergeWithPath(base *astjson.Value, resolved *astjson.Value return nil } +// flattenObject flattens a JSON object into a list of values. +// This is needed because we want to get the values from the object by its path to merge them with the response values. func (j *jsonBuilder) flattenObject(value *astjson.Value, path ast.Path) ([]*astjson.Value, error) { if path.Len() == 0 { return []*astjson.Value{value}, nil @@ -289,6 +293,8 @@ func (j *jsonBuilder) flattenObject(value *astjson.Value, path ast.Path) ([]*ast return result, nil } +// flattenList flattens a list of JSON values into a list of values. +// This is needed because we want to get the values from the list by its path to merge them with the response values. func (j *jsonBuilder) flattenList(items []*astjson.Value, path ast.Path) ([]*astjson.Value, error) { if path.Len() == 0 { return items, nil From 3eb843926992c1f391411b1439669437f85490bf Mon Sep 17 00:00:00 2001 From: Ludwig Bedacht Date: Tue, 28 Oct 2025 13:36:17 +0100 Subject: [PATCH 38/72] chore: add more comments --- .../engine/datasource/grpc_datasource/execution_plan.go | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go b/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go index 3346ae1f2..3b7959e8d 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go +++ b/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go @@ -129,6 +129,7 @@ func (r *RPCMessage) SelectValidTypes(typeName string) []string { return []string{r.Name, typeName} } +// AppendTypeNameField appends a typename field to the message. func (r *RPCMessage) AppendTypeNameField(typeName string) { if r.Fields != nil && r.Fields.Exists(typenameFieldName, "") { return @@ -277,6 +278,7 @@ func (r RPCFields) ByName(name string) *RPCField { return nil } +// Exists checks if a field with the given name and alias exists in the collection of fields. func (r RPCFields) Exists(name, alias string) bool { for _, field := range r { if field.Name == name && field.Alias == alias { @@ -376,6 +378,7 @@ type rpcPlanningContext struct { mapping *GRPCMapping } +// newRPCPlanningContext creates a new RPCPlanningContext. func newRPCPlanningContext(operation *ast.Document, definition *ast.Document, mapping *GRPCMapping) *rpcPlanningContext { return &rpcPlanningContext{ operation: operation, @@ -425,6 +428,7 @@ func (r *rpcPlanningContext) parseGraphQLType(t *ast.Type) DataType { } } +// resolveRPCMethodMapping resolves the RPC method mapping for a given operation type and operation field name. func (r *rpcPlanningContext) resolveRPCMethodMapping(operationType ast.OperationType, operationFieldName string) (RPCConfig, error) { if r.mapping == nil { return RPCConfig{}, nil @@ -482,6 +486,7 @@ func (r *rpcPlanningContext) resolveFieldMapping(typeName, fieldName string) str return fieldName } +// resolveFieldArgumentMapping resolves the field argument mapping for a given type name, field name and argument name. func (r *rpcPlanningContext) resolveFieldArgumentMapping(typeName, fieldName, argumentName string) string { if grpcFieldName, ok := r.mapping.FindFieldArgumentMapping(typeName, fieldName, argumentName); ok { return grpcFieldName @@ -490,6 +495,7 @@ func (r *rpcPlanningContext) resolveFieldArgumentMapping(typeName, fieldName, ar return argumentName } +// typeIsNullableOrNestedList checks if a type is nullable or a nested list. func (r *rpcPlanningContext) typeIsNullableOrNestedList(typeRef int) bool { if !r.definition.TypeIsNonNull(typeRef) && r.definition.TypeIsList(typeRef) { return true @@ -502,6 +508,7 @@ func (r *rpcPlanningContext) typeIsNullableOrNestedList(typeRef int) bool { return false } +// createListMetadata creates a list metadata for a given type reference. func (r *rpcPlanningContext) createListMetadata(typeRef int) (*ListMetadata, error) { nestingLevel := r.definition.TypeNumberOfListWraps(typeRef) @@ -711,6 +718,7 @@ func (r *rpcPlanningContext) buildInputMessageField(typeRef int, fieldName, json return field, nil } +// resolveServiceName resolves the service name for a given subgraph name. func (r *rpcPlanningContext) resolveServiceName(subgraphName string) string { if r.mapping == nil || r.mapping.Service == "" { return subgraphName @@ -982,6 +990,7 @@ const ( fieldArgsFieldName = "field_args" ) +// newResolveRPCCall creates a new resolve RPC call for a given resolved field. func (r *rpcPlanningContext) newResolveRPCCall(config *resolveRPCCallConfig) (RPCCall, error) { resolveConfig := config.resolveConfig resolvedField := config.resolvedField From c178835d1946b8e4e294289b9ee06889acbbcdda Mon Sep 17 00:00:00 2001 From: Ludwig Bedacht Date: Tue, 28 Oct 2025 15:54:58 +0100 Subject: [PATCH 39/72] chore: use const --- v2/pkg/engine/datasource/grpc_datasource/execution_plan.go | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go b/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go index 3b7959e8d..e7b486280 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go +++ b/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go @@ -869,6 +869,7 @@ func (r *rpcPlanningContext) findIDField(parentNode ast.Node, fieldDefRef int) ( // It skips the field definition reference for the given field definition reference. func (r *rpcPlanningContext) filterIDFieldsFunc(o ast.ObjectTypeDefinition, fieldDefRef int) func(yield func(int) bool) { fieldRefs := o.FieldsDefinition.Refs + const idTypeName = "ID" return func(yield func(int) bool) { for _, ref := range fieldRefs { if ref == fieldDefRef { @@ -876,7 +877,7 @@ func (r *rpcPlanningContext) filterIDFieldsFunc(o ast.ObjectTypeDefinition, fiel } typeName := r.definition.FieldDefinitionTypeNameString(ref) - if typeName != "ID" { + if typeName != idTypeName { continue } From 555ad448af01c4e7cfdf5cda6eedc39047f5a7df Mon Sep 17 00:00:00 2001 From: Ludwig Bedacht Date: Wed, 29 Oct 2025 13:13:13 +0100 Subject: [PATCH 40/72] chore: resolve fields without required fields visitor --- .../grpc_datasource/execution_plan.go | 60 +- .../execution_plan_field_resolvers_test.go | 217 +++ .../grpc_datasource/execution_plan_visitor.go | 107 +- .../execution_plan_visitor_federation.go | 100 +- .../grpc_datasource/mapping_test_helper.go | 23 + .../required_fields_visitor.go | 39 +- .../engine/datasource/grpc_datasource/util.go | 26 + .../datasource/grpc_datasource/util_test.go | 19 + v2/pkg/grpctest/mapping/mapping.go | 22 + v2/pkg/grpctest/mockservice.go | 26 + v2/pkg/grpctest/product.proto | 27 + v2/pkg/grpctest/productv1/product.pb.go | 1709 ++++++++++------- v2/pkg/grpctest/productv1/product_grpc.pb.go | 140 +- v2/pkg/grpctest/testdata/products.graphqls | 3 + 14 files changed, 1643 insertions(+), 875 deletions(-) create mode 100644 v2/pkg/engine/datasource/grpc_datasource/util_test.go diff --git a/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go b/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go index e7b486280..13e59a3c0 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go +++ b/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go @@ -60,7 +60,7 @@ type RPCExecutionPlan struct { } // CallKind is the type of call operation to perform. -type CallKind int +type CallKind uint8 const ( // CallKindStandard is a basic fetch operation. @@ -729,7 +729,7 @@ func (r *rpcPlanningContext) resolveServiceName(subgraphName string) string { type resolvedField struct { callerRef int - parentTypeRef int + parentTypeNode ast.Node fieldRef int fieldDefinitionTypeRef int fieldsSelectionSetRef int @@ -739,6 +739,15 @@ type resolvedField struct { fieldArguments []fieldArgument } +// isFieldResolver checks if a field is a field resolver. +func (r *rpcPlanningContext) isFieldResolver(fieldRef int, isRootField bool) bool { + if isRootField { + return false + } + + return len(r.operation.FieldArguments(fieldRef)) > 0 +} + // setResolvedField sets the resolved field for a given field definition reference. func (r *rpcPlanningContext) setResolvedField(walker *astvisitor.Walker, fieldDefRef int, fieldArgs []int, fieldPath ast.Path, resolvedField *resolvedField) error { // We need to resolve the context fields for the given field definition reference. @@ -832,7 +841,7 @@ func (r *rpcPlanningContext) getFieldsFromFieldResolverDirective(parentNode ast. defer walker.Release() v := newRequiredFieldsVisitor(walker, &RPCMessage{}, r) - if err := v.visitRequiredFields(r.definition, parentNode.NameString(r.definition), fieldsString); err != nil { + if err := v.visitWithDefaults(r.definition, parentNode.NameString(r.definition), fieldsString); err != nil { return nil, err } @@ -896,16 +905,41 @@ type resolveRPCCallConfig struct { } func (r *rpcPlanningContext) resolveRequiredFields(typeName string, requiredFieldSelection int) (*RPCMessage, error) { - walker := astvisitor.WalkerFromPool() - defer walker.Release() message := &RPCMessage{ Name: typeName, } - rfv := newRequiredFieldsVisitor(walker, message, r) - if err := rfv.visitWithMemberTypes(r.definition, typeName, r.operation.SelectionSetFieldSetString(requiredFieldSelection), nil); err != nil { - return nil, err + parentTypeNode, found := r.definition.NodeByNameStr(typeName) + if !found { + return nil, fmt.Errorf("parent type node not found for type %s", typeName) + } + + fieldRefs := r.operation.SelectionSetFieldSelections(requiredFieldSelection) + message.Fields = make(RPCFields, 0, len(fieldRefs)) + + for _, fieldRef := range fieldRefs { + if r.isFieldResolver(fieldRef, false) { + continue + } + + if message.Fields.Exists(r.operation.FieldNameString(fieldRef), "") { + continue + } + + fieldDef, found := r.definition.NodeFieldDefinitionByName(parentTypeNode, r.operation.FieldNameBytes(fieldRef)) + if !found { + return nil, fmt.Errorf("field definition not found for field %s", r.operation.FieldNameString(fieldRef)) + } + + field, err := r.buildField(parentTypeNode, fieldDef, r.operation.FieldNameString(fieldRef), "") + if err != nil { + return nil, err + } + + message.Fields = append(message.Fields, field) } + + message.Fields = slices.Clip(message.Fields) return message, nil } @@ -916,12 +950,12 @@ func (r *rpcPlanningContext) createResolverRPCCalls(subgraphName string, resolve for _, resolvedField := range resolvedFields { resolveConfig := r.mapping.FindResolveTypeFieldMapping( - r.definition.ObjectTypeDefinitionNameString(resolvedField.parentTypeRef), + resolvedField.parentTypeNode.NameString(r.definition), r.operation.FieldNameString(resolvedField.fieldRef), ) if resolveConfig == nil { - return nil, fmt.Errorf("resolve config not found for type: %s, field: %s", r.definition.ResolveTypeNameString(resolvedField.parentTypeRef), r.operation.FieldAliasString(resolvedField.fieldRef)) + return nil, fmt.Errorf("resolve config not found for type: %s, field: %s", r.definition.NodeNameString(resolvedField.parentTypeNode), r.operation.FieldAliasString(resolvedField.fieldRef)) } contextMessage := &RPCMessage{ @@ -947,13 +981,9 @@ func (r *rpcPlanningContext) createResolverRPCCalls(subgraphName string, resolve contextMessage.Fields = make(RPCFields, len(resolvedField.contextFields)) for i := range resolvedField.contextFields { - typeDefNode, found := r.definition.NodeByNameStr(r.definition.ResolveTypeNameString(resolvedField.parentTypeRef)) - if !found { - return nil, fmt.Errorf("type definition node not found for type: %s", r.definition.ResolveTypeNameString(resolvedField.parentTypeRef)) - } field, err := r.buildField( - typeDefNode, + resolvedField.parentTypeNode, resolvedField.contextFields[i].fieldRef, r.definition.FieldDefinitionNameString(resolvedField.contextFields[i].fieldRef), "", diff --git a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_field_resolvers_test.go b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_field_resolvers_test.go index a07573ba9..658996d87 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_field_resolvers_test.go +++ b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_field_resolvers_test.go @@ -892,3 +892,220 @@ func TestExecutionPlanFieldResolvers(t *testing.T) { }) } } + +func TestExecutionPlanFieldResolvers_WithNestedResolvers(t *testing.T) { + t.Parallel() + tests := []struct { + name string + query string + expectedPlan *RPCExecutionPlan + expectedError string + }{ + { + name: "Should create an execution plan for a query with nested field resolvers", + query: "query CategoriesWithNestedResolvers($metricType: String, $baseline: Float!) { categories { categoryMetrics(metricType: $metricType) { id metricType value normalizedScore(baseline: $baseline) } } }", + expectedPlan: &RPCExecutionPlan{ + Calls: []RPCCall{ + { + ServiceName: "Products", + MethodName: "QueryCategories", + Request: RPCMessage{ + Name: "QueryCategoriesRequest", + }, + Response: RPCMessage{ + Name: "QueryCategoriesResponse", + Fields: []RPCField{ + { + Name: "categories", + ProtoTypeName: DataTypeMessage, + JSONPath: "categories", + Repeated: true, + Message: &RPCMessage{ + Name: "Category", + Fields: []RPCField{}, + }, + }, + }, + }, + }, + { + DependentCalls: []int{0}, + ServiceName: "Products", + MethodName: "ResolveCategoryCategoryMetrics", + Kind: CallKindResolve, + ResponsePath: buildPath("categories.categoryMetrics"), + Request: RPCMessage{ + Name: "ResolveCategoryCategoryMetricsRequest", + Fields: []RPCField{ + { + Name: "context", + ProtoTypeName: DataTypeMessage, + Repeated: true, + Message: &RPCMessage{ + Name: "ResolveCategoryCategoryMetricsContext", + Fields: []RPCField{ + { + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", + ResolvePath: buildPath("categories.id"), + }, + { + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", + ResolvePath: buildPath("categories.name"), + }, + }, + }, + }, + { + Name: "field_args", + ProtoTypeName: DataTypeMessage, + Message: &RPCMessage{ + Name: "ResolveCategoryCategoryMetricsArgs", + Fields: []RPCField{ + { + Name: "metric_type", + ProtoTypeName: DataTypeString, + JSONPath: "metricType", + }, + }, + }, + }, + }, + }, + Response: RPCMessage{ + Name: "ResolveCategoryCategoryMetricsResponse", + Fields: []RPCField{ + { + Name: "result", + ProtoTypeName: DataTypeMessage, + JSONPath: "result", + Repeated: true, + Message: &RPCMessage{ + Name: "ResolveCategoryCategoryMetricsResult", + Fields: []RPCField{ + { + Name: "category_metrics", + ProtoTypeName: DataTypeMessage, + JSONPath: "categoryMetrics", + Optional: true, + Message: &RPCMessage{ + Name: "CategoryMetrics", + Fields: []RPCField{ + { + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", + }, + { + Name: "metric_type", + ProtoTypeName: DataTypeString, + JSONPath: "metricType", + }, + { + Name: "value", + ProtoTypeName: DataTypeDouble, + JSONPath: "value", + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + { + DependentCalls: []int{1}, + ServiceName: "Products", + MethodName: "ResolveCategoryMetricsNormalizedScore", + Kind: CallKindResolve, + ResponsePath: buildPath("categories.categoryMetrics.normalizedScore"), + Request: RPCMessage{ + Name: "ResolveCategoryMetricsNormalizedScoreRequest", + Fields: []RPCField{ + { + Name: "context", + ProtoTypeName: DataTypeMessage, + Repeated: true, + Message: &RPCMessage{ + Name: "ResolveCategoryMetricsNormalizedScoreContext", + Fields: []RPCField{ + { + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", + ResolvePath: buildPath("categories.categoryMetrics.id"), + }, + { + Name: "value", + ProtoTypeName: DataTypeDouble, + JSONPath: "value", + ResolvePath: buildPath("categories.categoryMetrics.value"), + }, + { + Name: "metric_type", + ProtoTypeName: DataTypeString, + JSONPath: "metricType", + ResolvePath: buildPath("categories.categoryMetrics.metricType"), + }, + }, + }, + }, + { + Name: "field_args", + ProtoTypeName: DataTypeMessage, + Message: &RPCMessage{ + Name: "ResolveCategoryMetricsNormalizedScoreArgs", + Fields: []RPCField{ + { + Name: "baseline", + ProtoTypeName: DataTypeDouble, + JSONPath: "baseline", + }, + }, + }, + }, + }, + }, + Response: RPCMessage{ + Name: "ResolveCategoryMetricsNormalizedScoreResponse", + Fields: []RPCField{ + { + Name: "result", + ProtoTypeName: DataTypeMessage, + JSONPath: "result", + Repeated: true, + Message: &RPCMessage{ + Name: "ResolveCategoryMetricsNormalizedScoreResult", + Fields: []RPCField{ + { + Name: "normalized_score", + ProtoTypeName: DataTypeDouble, + JSONPath: "normalizedScore", + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + } + + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + t.Parallel() + runTest(t, testCase{ + query: tt.query, + expectedPlan: tt.expectedPlan, + expectedError: tt.expectedError, + }) + }) + } +} diff --git a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor.go b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor.go index 5b35371ee..eb83b9980 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor.go +++ b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor.go @@ -54,9 +54,9 @@ type rpcPlanVisitor struct { currentCall *RPCCall currentCallID int - relatedCallID int - resolvedFieldIndex int - resolvedFields []resolvedField + relatedCallID int + fieldResolverAncestors ancestor[int] + resolvedFields []resolvedField fieldPath ast.Path } @@ -72,15 +72,15 @@ type rpcPlanVisitorConfig struct { func newRPCPlanVisitor(config rpcPlanVisitorConfig) *rpcPlanVisitor { walker := astvisitor.NewWalker(48) visitor := &rpcPlanVisitor{ - walker: &walker, - plan: &RPCExecutionPlan{}, - subgraphName: cases.Title(language.Und, cases.NoLower).String(config.subgraphName), - mapping: config.mapping, - operationFieldRef: ast.InvalidRef, - resolvedFields: make([]resolvedField, 0), - relatedCallID: ast.InvalidRef, - resolvedFieldIndex: ast.InvalidRef, - fieldPath: make(ast.Path, 0), + walker: &walker, + plan: &RPCExecutionPlan{}, + subgraphName: cases.Title(language.Und, cases.NoLower).String(config.subgraphName), + mapping: config.mapping, + operationFieldRef: ast.InvalidRef, + resolvedFields: make([]resolvedField, 0), + relatedCallID: ast.InvalidRef, + fieldResolverAncestors: newAncestor[int](), + fieldPath: make(ast.Path, 0), } walker.RegisterDocumentVisitor(visitor) @@ -197,10 +197,9 @@ func (r *rpcPlanVisitor) EnterSelectionSet(ref int) { } // If we are inside of a resolved field that selects multiple fields, we get all the fields from the input and pass them to the required fields visitor. - if r.resolvedFieldIndex != ast.InvalidRef { + if r.fieldResolverAncestors.len() > 0 { // TODO: handle nested resolved fields. - r.resolvedFields[r.resolvedFieldIndex].fieldsSelectionSetRef = ref - r.walker.SkipNode() + r.resolvedFields[r.fieldResolverAncestors.peek()].fieldsSelectionSetRef = ref return } @@ -333,13 +332,7 @@ func (r *rpcPlanVisitor) EnterField(ref int) { return } - // prevent duplicate fields - fieldAlias := r.operation.FieldAliasString(ref) - if r.planInfo.currentResponseMessage.Fields.Exists(fieldName, fieldAlias) { - return - } - - fd, ok := r.walker.FieldDefinition(ref) + fieldDefRef, ok := r.walker.FieldDefinition(ref) if !ok { r.walker.Report.AddExternalError(operationreport.ExternalError{ Message: fmt.Sprintf("Field %s not found in definition %s", r.operation.FieldNameString(ref), r.walker.EnclosingTypeDefinition.NameString(r.definition)), @@ -347,34 +340,27 @@ func (r *rpcPlanVisitor) EnterField(ref int) { return } - // Field arguments for non root types will be handled as resolver calls. - // We need to make sure to handle a hierarchy of arguments in order to perform parallel calls in order to retrieve the data. - // TODO: this needs to be available for both visitors and added to the plancontext - if fieldArgs := r.operation.FieldArguments(ref); !inRootField && len(fieldArgs) > 0 { - // We don't want to add fields from the selection set to the actual call - resolvedField := resolvedField{ - callerRef: r.relatedCallID, - parentTypeRef: r.walker.EnclosingTypeDefinition.Ref, - fieldRef: ref, - responsePath: r.walker.Path[1:].WithoutInlineFragmentNames().WithFieldNameItem(r.operation.FieldAliasOrNameBytes(ref)), - fieldDefinitionTypeRef: r.definition.FieldDefinitionType(fd), - } - - if err := r.planCtx.setResolvedField(r.walker, fd, fieldArgs, r.fieldPath, &resolvedField); err != nil { - r.walker.StopWithInternalErr(err) - return - } + // If the field is a field resolver, we need to handle it later in a separate resolver call. + // We only store the information about the field and create the call later. + if r.planCtx.isFieldResolver(ref, inRootField) { + r.enterFieldResolver(ref, fieldDefRef) + return + } - r.resolvedFields = append(r.resolvedFields, resolvedField) - r.resolvedFieldIndex = len(r.resolvedFields) - 1 - r.fieldPath = r.fieldPath.WithFieldNameItem(r.operation.FieldNameBytes(ref)) + // Check if the field is inside of a resolver call. + if r.fieldResolverAncestors.len() > 0 { + // We don't want to call LeaveField here because we ignore the field entirely. + r.walker.SkipNode() + return + } - // In case of nested fields with arguments, we need to increment the related call ID. - r.relatedCallID++ + // prevent duplicate fields + fieldAlias := r.operation.FieldAliasString(ref) + if r.planInfo.currentResponseMessage.Fields.Exists(fieldName, fieldAlias) { return } - field, err := r.planCtx.buildField(r.walker.EnclosingTypeDefinition, fd, fieldName, fieldAlias) + field, err := r.planCtx.buildField(r.walker.EnclosingTypeDefinition, fieldDefRef, fieldName, fieldAlias) if err != nil { r.walker.StopWithInternalErr(err) return @@ -405,7 +391,7 @@ func (r *rpcPlanVisitor) EnterField(ref int) { // LeaveField implements astvisitor.FieldVisitor. func (r *rpcPlanVisitor) LeaveField(ref int) { r.fieldPath = r.fieldPath.RemoveLastItem() - r.resolvedFieldIndex = ast.InvalidRef + r.fieldResolverAncestors.pop() // If we are not in the operation field, we can increment the response field index. if !r.walker.InRootField() { @@ -430,3 +416,32 @@ func (r *rpcPlanVisitor) LeaveField(ref int) { r.planInfo.currentResponseFieldIndex = 0 } + +// enterFieldResolver enters a field resolver. +// ref is the field reference in the operation document. +// fieldDefRef is the field definition reference in the definition document. +func (r *rpcPlanVisitor) enterFieldResolver(ref int, fieldDefRef int) { + // Field arguments for non root types will be handled as resolver calls. + // We need to make sure to handle a hierarchy of arguments in order to perform parallel calls in order to retrieve the data. + fieldArgs := r.operation.FieldArguments(ref) + // We don't want to add fields from the selection set to the actual call + resolvedField := resolvedField{ + callerRef: r.relatedCallID, + parentTypeNode: r.walker.EnclosingTypeDefinition, + fieldRef: ref, + responsePath: r.walker.Path[1:].WithoutInlineFragmentNames().WithFieldNameItem(r.operation.FieldAliasOrNameBytes(ref)), + fieldDefinitionTypeRef: r.definition.FieldDefinitionType(fieldDefRef), + } + + if err := r.planCtx.setResolvedField(r.walker, fieldDefRef, fieldArgs, r.fieldPath, &resolvedField); err != nil { + r.walker.StopWithInternalErr(err) + return + } + + r.resolvedFields = append(r.resolvedFields, resolvedField) + r.fieldResolverAncestors.push(len(r.resolvedFields) - 1) + r.fieldPath = r.fieldPath.WithFieldNameItem(r.operation.FieldNameBytes(ref)) + + // In case of nested fields with arguments, we need to increment the related call ID. + r.relatedCallID++ +} diff --git a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor_federation.go b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor_federation.go index c4f788dd4..6e308b2c9 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor_federation.go +++ b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor_federation.go @@ -50,9 +50,9 @@ type rpcPlanVisitorFederation struct { subgraphName string currentCall *RPCCall - relatedCallID int - resolvedFieldIndex int - resolvedFields []resolvedField + relatedCallID int + fieldResolverAncestors ancestor[int] + resolvedFields []resolvedField fieldPath ast.Path } @@ -68,11 +68,11 @@ func newRPCPlanVisitorFederation(config rpcPlanVisitorConfig) *rpcPlanVisitorFed entityRootFieldRef: ast.InvalidRef, entityInlineFragmentRef: ast.InvalidRef, }, - federationConfigData: parseFederationConfigData(config.federationConfigs), - resolvedFields: make([]resolvedField, 0), - resolvedFieldIndex: ast.InvalidRef, - relatedCallID: ast.InvalidRef, - fieldPath: ast.Path{}.WithFieldNameItem([]byte("result")), + federationConfigData: parseFederationConfigData(config.federationConfigs), + resolvedFields: make([]resolvedField, 0), + fieldResolverAncestors: newAncestor[int](), + relatedCallID: ast.InvalidRef, + fieldPath: ast.Path{}.WithFieldNameItem([]byte("result")), } walker.RegisterDocumentVisitor(visitor) @@ -187,9 +187,8 @@ func (r *rpcPlanVisitorFederation) EnterSelectionSet(ref int) { } // If we are inside of a resolved field that selects multiple fields, we get all the fields from the input and pass them to the required fields visitor. - if r.resolvedFieldIndex != ast.InvalidRef { - r.resolvedFields[r.resolvedFieldIndex].fieldsSelectionSetRef = ref - r.walker.SkipNode() + if r.fieldResolverAncestors.len() > 0 { + r.resolvedFields[r.fieldResolverAncestors.peek()].fieldsSelectionSetRef = ref return } @@ -302,14 +301,7 @@ func (r *rpcPlanVisitorFederation) EnterField(ref int) { return } - // prevent duplicate fields - fieldAlias := r.operation.FieldAliasString(ref) - if r.planInfo.currentResponseMessage.Fields.Exists(fieldName, fieldAlias) { - r.fieldPath = r.fieldPath.WithFieldNameItem([]byte{}) - return - } - - fd, ok := r.walker.FieldDefinition(ref) + fieldDefRef, ok := r.walker.FieldDefinition(ref) if !ok { r.walker.Report.AddExternalError(operationreport.ExternalError{ Message: fmt.Sprintf("Field %s not found in definition %s", r.operation.FieldNameString(ref), r.walker.EnclosingTypeDefinition.NameString(r.definition)), @@ -317,31 +309,28 @@ func (r *rpcPlanVisitorFederation) EnterField(ref int) { return } - if fieldArgs := r.operation.FieldArguments(ref); !inRootField && len(fieldArgs) > 0 { - // We don't want to add fields from the selection set to the actual call - resolvedField := resolvedField{ - callerRef: r.relatedCallID, - parentTypeRef: r.walker.EnclosingTypeDefinition.Ref, - fieldRef: ref, - responsePath: r.walker.Path[1:].WithoutInlineFragmentNames().WithFieldNameItem(r.operation.FieldAliasOrNameBytes(ref)), - fieldDefinitionTypeRef: r.definition.FieldDefinitionType(fd), - } - - if err := r.planCtx.setResolvedField(r.walker, fd, fieldArgs, r.fieldPath, &resolvedField); err != nil { - r.walker.StopWithInternalErr(err) - return - } + // If the field is a field resolver, we need to handle it later in a separate resolver call. + // We only store the information about the field and create the call later. + if r.planCtx.isFieldResolver(ref, inRootField) { + r.enterFieldResolver(ref, fieldDefRef) + return + } - r.resolvedFields = append(r.resolvedFields, resolvedField) - r.resolvedFieldIndex = len(r.resolvedFields) - 1 - r.fieldPath = r.fieldPath.WithFieldNameItem(r.operation.FieldNameBytes(ref)) + // Check if the field is inside of a resolver call. + if r.fieldResolverAncestors.len() > 0 { + // We don't want to call LeaveField here because we ignore the field entirely. + r.walker.SkipNode() + return + } - // In case of nested fields with arguments, we need to increment the related call ID. - r.relatedCallID++ + // prevent duplicate fields + fieldAlias := r.operation.FieldAliasString(ref) + if r.planInfo.currentResponseMessage.Fields.Exists(fieldName, fieldAlias) { + r.fieldPath = r.fieldPath.WithFieldNameItem([]byte{}) return } - field, err := r.planCtx.buildField(r.walker.EnclosingTypeDefinition, fd, fieldName, fieldAlias) + field, err := r.planCtx.buildField(r.walker.EnclosingTypeDefinition, fieldDefRef, fieldName, fieldAlias) if err != nil { r.walker.StopWithInternalErr(err) return @@ -372,7 +361,7 @@ func (r *rpcPlanVisitorFederation) EnterField(ref int) { // LeaveField implements astvisitor.FieldVisitor. func (r *rpcPlanVisitorFederation) LeaveField(ref int) { r.fieldPath = r.fieldPath.RemoveLastItem() - r.resolvedFieldIndex = ast.InvalidRef + r.fieldResolverAncestors.pop() // If we are not in the operation field, we can increment the response field index. if !r.walker.InRootField() { // If the field has arguments, we need to decrement the related call ID. @@ -389,6 +378,35 @@ func (r *rpcPlanVisitorFederation) LeaveField(ref int) { r.planInfo.currentResponseFieldIndex = 0 } +// enterFieldResolver enters a field resolver. +// ref is the field reference in the operation document. +// fieldDefRef is the field definition reference in the definition document. +func (r *rpcPlanVisitorFederation) enterFieldResolver(ref int, fieldDefRef int) { + // Field arguments for non root types will be handled as resolver calls. + // We need to make sure to handle a hierarchy of arguments in order to perform parallel calls in order to retrieve the data. + fieldArgs := r.operation.FieldArguments(ref) + // We don't want to add fields from the selection set to the actual call + resolvedField := resolvedField{ + callerRef: r.relatedCallID, + parentTypeNode: r.walker.EnclosingTypeDefinition, + fieldRef: ref, + responsePath: r.walker.Path[1:].WithoutInlineFragmentNames().WithFieldNameItem(r.operation.FieldAliasOrNameBytes(ref)), + fieldDefinitionTypeRef: r.definition.FieldDefinitionType(fieldDefRef), + } + + if err := r.planCtx.setResolvedField(r.walker, fieldDefRef, fieldArgs, r.fieldPath, &resolvedField); err != nil { + r.walker.StopWithInternalErr(err) + return + } + + r.resolvedFields = append(r.resolvedFields, resolvedField) + r.fieldResolverAncestors.push(len(r.resolvedFields) - 1) + r.fieldPath = r.fieldPath.WithFieldNameItem(r.operation.FieldNameBytes(ref)) + + // In case of nested fields with arguments, we need to increment the related call ID. + r.relatedCallID++ +} + func (r *rpcPlanVisitorFederation) resolveEntityInformation(inlineFragmentRef int, fc federationConfigData) error { fragmentName := r.operation.InlineFragmentTypeConditionNameString(inlineFragmentRef) node, found := r.definition.NodeByNameStr(r.operation.InlineFragmentTypeConditionNameString(inlineFragmentRef)) @@ -426,7 +444,7 @@ func (r *rpcPlanVisitorFederation) scaffoldEntityLookup(fc federationConfigData) defer walker.Release() requiredFieldsVisitor := newRequiredFieldsVisitor(walker, keyFieldMessage, r.planCtx) - err := requiredFieldsVisitor.visitRequiredFields(r.definition, fc.entityTypeName, fc.keyFields) + err := requiredFieldsVisitor.visitWithDefaults(r.definition, fc.entityTypeName, fc.keyFields) if err != nil { r.walker.StopWithInternalErr(err) return diff --git a/v2/pkg/engine/datasource/grpc_datasource/mapping_test_helper.go b/v2/pkg/engine/datasource/grpc_datasource/mapping_test_helper.go index 148cd51dc..332730dac 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/mapping_test_helper.go +++ b/v2/pkg/engine/datasource/grpc_datasource/mapping_test_helper.go @@ -254,6 +254,19 @@ func testMapping() *GRPCMapping { Response: "ResolveCategoryCategoryMetricsResponse", }, }, + "CategoryMetrics": { + "normalizedScore": { + FieldMappingData: FieldMapData{ + TargetName: "normalized_score", + ArgumentMappings: FieldArgumentMap{ + "baseline": "baseline", + }, + }, + RPC: "ResolveCategoryMetricsNormalizedScore", + Request: "ResolveCategoryMetricsNormalizedScoreRequest", + Response: "ResolveCategoryMetricsNormalizedScoreResponse", + }, + }, "Product": { "shippingEstimate": { FieldMappingData: FieldMapData{ @@ -833,6 +846,15 @@ func testMapping() *GRPCMapping { "categoryId": { TargetName: "category_id", }, + "normalizedScore": { + TargetName: "normalized_score", + ArgumentMappings: FieldArgumentMap{ + "baseline": "baseline", + }, + }, + "relatedCategory": { + TargetName: "related_category", + }, }, "Cat": { "id": { @@ -1224,4 +1246,5 @@ func testMapping() *GRPCMapping { }, }, } + } diff --git a/v2/pkg/engine/datasource/grpc_datasource/required_fields_visitor.go b/v2/pkg/engine/datasource/grpc_datasource/required_fields_visitor.go index c78f4ea6a..09e531da9 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/required_fields_visitor.go +++ b/v2/pkg/engine/datasource/grpc_datasource/required_fields_visitor.go @@ -7,9 +7,15 @@ import ( "github.com/wundergraph/graphql-go-tools/v2/pkg/ast" "github.com/wundergraph/graphql-go-tools/v2/pkg/astvisitor" "github.com/wundergraph/graphql-go-tools/v2/pkg/engine/plan" - "github.com/wundergraph/graphql-go-tools/v2/pkg/operationreport" ) +type requiredFieldVisitorConfig struct { + // includeMemberType indicates if the member type should be included in the message. + includeMemberType bool + // skipFieldResolvers indicates if the field resolvers should be skipped. + skipFieldResolvers bool +} + // requiredFieldsVisitor is a visitor that visits the required fields of a message. type requiredFieldsVisitor struct { operation *ast.Document @@ -22,6 +28,8 @@ type requiredFieldsVisitor struct { planCtx *rpcPlanningContext messageAncestors []*RPCMessage + + skipFieldResolvers bool } // newRequiredFieldsVisitor creates a new requiredFieldsVisitor. @@ -42,20 +50,30 @@ func newRequiredFieldsVisitor(walker *astvisitor.Walker, message *RPCMessage, pl return visitor } -// visitRequiredFields visits the required fields of a message. +// visitWithDefaults visits the required fields of a message. // It creates a new document with the required fields and walks it. // To achieve that we create a fragment with the required fields and walk it. -func (r *requiredFieldsVisitor) visitRequiredFields(definition *ast.Document, typeName, requiredFields string) error { - return r.visitWithMemberTypes(definition, typeName, requiredFields, []string{typeName}) +func (r *requiredFieldsVisitor) visitWithDefaults(definition *ast.Document, typeName, requiredFields string) error { + return r.visit(definition, typeName, requiredFields, requiredFieldVisitorConfig{ + includeMemberType: true, + skipFieldResolvers: false, + }) } -func (r *requiredFieldsVisitor) visitWithMemberTypes(definition *ast.Document, typeName, requiredFields string, memberTypes []string) error { +// visit visits the required fields of a message. +// The function can be provided with options to customize the visitor. +func (r *requiredFieldsVisitor) visit(definition *ast.Document, typeName, requiredFields string, options requiredFieldVisitorConfig) error { doc, report := plan.RequiredFieldsFragment(typeName, requiredFields, false) if report.HasErrors() { return report } - r.message.MemberTypes = memberTypes + if options.includeMemberType { + r.message.MemberTypes = []string{typeName} + } + + r.skipFieldResolvers = options.skipFieldResolvers + r.walker.Walk(doc, definition, report) if report.HasErrors() { return report @@ -124,9 +142,12 @@ func (r *requiredFieldsVisitor) EnterField(ref int) { fd, ok := r.walker.FieldDefinition(ref) if !ok { - r.walker.Report.AddExternalError(operationreport.ExternalError{ - Message: fmt.Sprintf("Field %s not found in definition %s", fieldName, r.walker.EnclosingTypeDefinition.NameString(r.definition)), - }) + r.walker.StopWithInternalErr(fmt.Errorf("field definition not found for field %s", fieldName)) + return + } + + if r.planCtx.isFieldResolver(ref, r.walker.InRootField()) && r.skipFieldResolvers { + r.walker.SkipNode() return } diff --git a/v2/pkg/engine/datasource/grpc_datasource/util.go b/v2/pkg/engine/datasource/grpc_datasource/util.go index 1a6a1f55d..6d96f2a1a 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/util.go +++ b/v2/pkg/engine/datasource/grpc_datasource/util.go @@ -8,3 +8,29 @@ func initializeSlice[T any](len int, zero T) []T { } return s } + +type ancestor[T any] []T + +func newAncestor[T any]() ancestor[T] { + return make(ancestor[T], 0) +} + +func (a *ancestor[T]) push(value T) { + *a = append(*a, value) +} + +func (a *ancestor[T]) pop() { + if a.len() == 0 { + return + } + + *a = (*a)[:len(*a)-1] +} + +func (a *ancestor[T]) peek() T { + return (*a)[len(*a)-1] +} + +func (a *ancestor[T]) len() int { + return len(*a) +} diff --git a/v2/pkg/engine/datasource/grpc_datasource/util_test.go b/v2/pkg/engine/datasource/grpc_datasource/util_test.go new file mode 100644 index 000000000..aaf0e1cc0 --- /dev/null +++ b/v2/pkg/engine/datasource/grpc_datasource/util_test.go @@ -0,0 +1,19 @@ +package grpcdatasource + +import ( + "testing" + + "github.com/stretchr/testify/require" +) + +func TestAncestor(t *testing.T) { + ancestor := newAncestor[int]() + ancestor.push(1) + ancestor.push(2) + ancestor.push(3) + require.Equal(t, 3, ancestor.len()) + ancestor.pop() + require.Equal(t, 2, ancestor.len()) + require.Equal(t, 2, ancestor.peek()) + require.Equal(t, 2, ancestor.len()) +} diff --git a/v2/pkg/grpctest/mapping/mapping.go b/v2/pkg/grpctest/mapping/mapping.go index 6397adc31..c5087f12c 100644 --- a/v2/pkg/grpctest/mapping/mapping.go +++ b/v2/pkg/grpctest/mapping/mapping.go @@ -261,6 +261,19 @@ func DefaultGRPCMapping() *grpcdatasource.GRPCMapping { Response: "ResolveCategoryCategoryMetricsResponse", }, }, + "CategoryMetrics": { + "normalizedScore": { + FieldMappingData: grpcdatasource.FieldMapData{ + TargetName: "normalized_score", + ArgumentMappings: grpcdatasource.FieldArgumentMap{ + "baseline": "baseline", + }, + }, + RPC: "ResolveCategoryMetricsNormalizedScore", + Request: "ResolveCategoryMetricsNormalizedScoreRequest", + Response: "ResolveCategoryMetricsNormalizedScoreResponse", + }, + }, "Product": { "shippingEstimate": { FieldMappingData: grpcdatasource.FieldMapData{ @@ -840,6 +853,15 @@ func DefaultGRPCMapping() *grpcdatasource.GRPCMapping { "categoryId": { TargetName: "category_id", }, + "normalizedScore": { + TargetName: "normalized_score", + ArgumentMappings: grpcdatasource.FieldArgumentMap{ + "baseline": "baseline", + }, + }, + "relatedCategory": { + TargetName: "related_category", + }, }, "Cat": { "id": { diff --git a/v2/pkg/grpctest/mockservice.go b/v2/pkg/grpctest/mockservice.go index 3a50d1c1f..1bfce37ce 100644 --- a/v2/pkg/grpctest/mockservice.go +++ b/v2/pkg/grpctest/mockservice.go @@ -20,6 +20,32 @@ type MockService struct { productv1.UnimplementedProductServiceServer } +// ResolveCategoryMetricsNormalizedScore implements productv1.ProductServiceServer. +func (s *MockService) ResolveCategoryMetricsNormalizedScore(_ context.Context, req *productv1.ResolveCategoryMetricsNormalizedScoreRequest) (*productv1.ResolveCategoryMetricsNormalizedScoreResponse, error) { + results := make([]*productv1.ResolveCategoryMetricsNormalizedScoreResult, 0, len(req.GetContext())) + + baseline := req.GetFieldArgs().GetBaseline() + if baseline == 0 { + baseline = 1.0 // Avoid division by zero + } + + for _, ctx := range req.GetContext() { + // Calculate normalized score: (value / baseline) * 100 + // This gives a percentage relative to the baseline + normalizedScore := (ctx.GetValue() / baseline) * 100.0 + + results = append(results, &productv1.ResolveCategoryMetricsNormalizedScoreResult{ + NormalizedScore: normalizedScore, + }) + } + + resp := &productv1.ResolveCategoryMetricsNormalizedScoreResponse{ + Result: results, + } + + return resp, nil +} + // ResolveProductRecommendedCategory implements productv1.ProductServiceServer. func (s *MockService) ResolveProductRecommendedCategory(_ context.Context, req *productv1.ResolveProductRecommendedCategoryRequest) (*productv1.ResolveProductRecommendedCategoryResponse, error) { results := make([]*productv1.ResolveProductRecommendedCategoryResult, 0, len(req.GetContext())) diff --git a/v2/pkg/grpctest/product.proto b/v2/pkg/grpctest/product.proto index ac6ec65b2..4efeffbb2 100644 --- a/v2/pkg/grpctest/product.proto +++ b/v2/pkg/grpctest/product.proto @@ -56,6 +56,7 @@ service ProductService { rpc QueryUser(QueryUserRequest) returns (QueryUserResponse) {} rpc QueryUsers(QueryUsersRequest) returns (QueryUsersResponse) {} rpc ResolveCategoryCategoryMetrics(ResolveCategoryCategoryMetricsRequest) returns (ResolveCategoryCategoryMetricsResponse) {} + rpc ResolveCategoryMetricsNormalizedScore(ResolveCategoryMetricsNormalizedScoreRequest) returns (ResolveCategoryMetricsNormalizedScoreResponse) {} rpc ResolveCategoryPopularityScore(ResolveCategoryPopularityScoreRequest) returns (ResolveCategoryPopularityScoreResponse) {} rpc ResolveCategoryProductCount(ResolveCategoryProductCountRequest) returns (ResolveCategoryProductCountResponse) {} rpc ResolveProductRecommendedCategory(ResolveProductRecommendedCategoryRequest) returns (ResolveProductRecommendedCategoryResponse) {} @@ -775,6 +776,31 @@ message ResolveSubcategoryItemCountResponse { repeated ResolveSubcategoryItemCountResult result = 1; } +message ResolveCategoryMetricsNormalizedScoreArgs { + double baseline = 1; +} + +message ResolveCategoryMetricsNormalizedScoreContext { + string id = 1; + string metricType = 2; + double value = 3; +} + +message ResolveCategoryMetricsNormalizedScoreRequest { + // context provides the resolver context for the field normalizedScore of type CategoryMetrics. + repeated ResolveCategoryMetricsNormalizedScoreContext context = 1; + // field_args provides the arguments for the resolver field normalizedScore of type CategoryMetrics. + ResolveCategoryMetricsNormalizedScoreArgs field_args = 2; +} + +message ResolveCategoryMetricsNormalizedScoreResult { + double normalized_score = 1; +} + +message ResolveCategoryMetricsNormalizedScoreResponse { + repeated ResolveCategoryMetricsNormalizedScoreResult result = 1; +} + message Product { string id = 1; string name = 2; @@ -1052,6 +1078,7 @@ message CategoryMetrics { double value = 3; string timestamp = 4; string category_id = 5; + Category related_category = 6; } enum CategoryKind { diff --git a/v2/pkg/grpctest/productv1/product.pb.go b/v2/pkg/grpctest/productv1/product.pb.go index ed265f796..c8062850a 100644 --- a/v2/pkg/grpctest/productv1/product.pb.go +++ b/v2/pkg/grpctest/productv1/product.pb.go @@ -6610,6 +6610,252 @@ func (x *ResolveSubcategoryItemCountResponse) GetResult() []*ResolveSubcategoryI return nil } +type ResolveCategoryMetricsNormalizedScoreArgs struct { + state protoimpl.MessageState `protogen:"open.v1"` + Baseline float64 `protobuf:"fixed64,1,opt,name=baseline,proto3" json:"baseline,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ResolveCategoryMetricsNormalizedScoreArgs) Reset() { + *x = ResolveCategoryMetricsNormalizedScoreArgs{} + mi := &file_product_proto_msgTypes[143] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ResolveCategoryMetricsNormalizedScoreArgs) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ResolveCategoryMetricsNormalizedScoreArgs) ProtoMessage() {} + +func (x *ResolveCategoryMetricsNormalizedScoreArgs) ProtoReflect() protoreflect.Message { + mi := &file_product_proto_msgTypes[143] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ResolveCategoryMetricsNormalizedScoreArgs.ProtoReflect.Descriptor instead. +func (*ResolveCategoryMetricsNormalizedScoreArgs) Descriptor() ([]byte, []int) { + return file_product_proto_rawDescGZIP(), []int{143} +} + +func (x *ResolveCategoryMetricsNormalizedScoreArgs) GetBaseline() float64 { + if x != nil { + return x.Baseline + } + return 0 +} + +type ResolveCategoryMetricsNormalizedScoreContext struct { + state protoimpl.MessageState `protogen:"open.v1"` + Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` + MetricType string `protobuf:"bytes,2,opt,name=metricType,proto3" json:"metricType,omitempty"` + Value float64 `protobuf:"fixed64,3,opt,name=value,proto3" json:"value,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ResolveCategoryMetricsNormalizedScoreContext) Reset() { + *x = ResolveCategoryMetricsNormalizedScoreContext{} + mi := &file_product_proto_msgTypes[144] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ResolveCategoryMetricsNormalizedScoreContext) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ResolveCategoryMetricsNormalizedScoreContext) ProtoMessage() {} + +func (x *ResolveCategoryMetricsNormalizedScoreContext) ProtoReflect() protoreflect.Message { + mi := &file_product_proto_msgTypes[144] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ResolveCategoryMetricsNormalizedScoreContext.ProtoReflect.Descriptor instead. +func (*ResolveCategoryMetricsNormalizedScoreContext) Descriptor() ([]byte, []int) { + return file_product_proto_rawDescGZIP(), []int{144} +} + +func (x *ResolveCategoryMetricsNormalizedScoreContext) GetId() string { + if x != nil { + return x.Id + } + return "" +} + +func (x *ResolveCategoryMetricsNormalizedScoreContext) GetMetricType() string { + if x != nil { + return x.MetricType + } + return "" +} + +func (x *ResolveCategoryMetricsNormalizedScoreContext) GetValue() float64 { + if x != nil { + return x.Value + } + return 0 +} + +type ResolveCategoryMetricsNormalizedScoreRequest struct { + state protoimpl.MessageState `protogen:"open.v1"` + // context provides the resolver context for the field normalizedScore of type CategoryMetrics. + Context []*ResolveCategoryMetricsNormalizedScoreContext `protobuf:"bytes,1,rep,name=context,proto3" json:"context,omitempty"` + // field_args provides the arguments for the resolver field normalizedScore of type CategoryMetrics. + FieldArgs *ResolveCategoryMetricsNormalizedScoreArgs `protobuf:"bytes,2,opt,name=field_args,json=fieldArgs,proto3" json:"field_args,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ResolveCategoryMetricsNormalizedScoreRequest) Reset() { + *x = ResolveCategoryMetricsNormalizedScoreRequest{} + mi := &file_product_proto_msgTypes[145] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ResolveCategoryMetricsNormalizedScoreRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ResolveCategoryMetricsNormalizedScoreRequest) ProtoMessage() {} + +func (x *ResolveCategoryMetricsNormalizedScoreRequest) ProtoReflect() protoreflect.Message { + mi := &file_product_proto_msgTypes[145] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ResolveCategoryMetricsNormalizedScoreRequest.ProtoReflect.Descriptor instead. +func (*ResolveCategoryMetricsNormalizedScoreRequest) Descriptor() ([]byte, []int) { + return file_product_proto_rawDescGZIP(), []int{145} +} + +func (x *ResolveCategoryMetricsNormalizedScoreRequest) GetContext() []*ResolveCategoryMetricsNormalizedScoreContext { + if x != nil { + return x.Context + } + return nil +} + +func (x *ResolveCategoryMetricsNormalizedScoreRequest) GetFieldArgs() *ResolveCategoryMetricsNormalizedScoreArgs { + if x != nil { + return x.FieldArgs + } + return nil +} + +type ResolveCategoryMetricsNormalizedScoreResult struct { + state protoimpl.MessageState `protogen:"open.v1"` + NormalizedScore float64 `protobuf:"fixed64,1,opt,name=normalized_score,json=normalizedScore,proto3" json:"normalized_score,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ResolveCategoryMetricsNormalizedScoreResult) Reset() { + *x = ResolveCategoryMetricsNormalizedScoreResult{} + mi := &file_product_proto_msgTypes[146] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ResolveCategoryMetricsNormalizedScoreResult) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ResolveCategoryMetricsNormalizedScoreResult) ProtoMessage() {} + +func (x *ResolveCategoryMetricsNormalizedScoreResult) ProtoReflect() protoreflect.Message { + mi := &file_product_proto_msgTypes[146] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ResolveCategoryMetricsNormalizedScoreResult.ProtoReflect.Descriptor instead. +func (*ResolveCategoryMetricsNormalizedScoreResult) Descriptor() ([]byte, []int) { + return file_product_proto_rawDescGZIP(), []int{146} +} + +func (x *ResolveCategoryMetricsNormalizedScoreResult) GetNormalizedScore() float64 { + if x != nil { + return x.NormalizedScore + } + return 0 +} + +type ResolveCategoryMetricsNormalizedScoreResponse struct { + state protoimpl.MessageState `protogen:"open.v1"` + Result []*ResolveCategoryMetricsNormalizedScoreResult `protobuf:"bytes,1,rep,name=result,proto3" json:"result,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ResolveCategoryMetricsNormalizedScoreResponse) Reset() { + *x = ResolveCategoryMetricsNormalizedScoreResponse{} + mi := &file_product_proto_msgTypes[147] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ResolveCategoryMetricsNormalizedScoreResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ResolveCategoryMetricsNormalizedScoreResponse) ProtoMessage() {} + +func (x *ResolveCategoryMetricsNormalizedScoreResponse) ProtoReflect() protoreflect.Message { + mi := &file_product_proto_msgTypes[147] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ResolveCategoryMetricsNormalizedScoreResponse.ProtoReflect.Descriptor instead. +func (*ResolveCategoryMetricsNormalizedScoreResponse) Descriptor() ([]byte, []int) { + return file_product_proto_rawDescGZIP(), []int{147} +} + +func (x *ResolveCategoryMetricsNormalizedScoreResponse) GetResult() []*ResolveCategoryMetricsNormalizedScoreResult { + if x != nil { + return x.Result + } + return nil +} + type Product struct { state protoimpl.MessageState `protogen:"open.v1"` Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` @@ -6621,7 +6867,7 @@ type Product struct { func (x *Product) Reset() { *x = Product{} - mi := &file_product_proto_msgTypes[143] + mi := &file_product_proto_msgTypes[148] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6633,7 +6879,7 @@ func (x *Product) String() string { func (*Product) ProtoMessage() {} func (x *Product) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[143] + mi := &file_product_proto_msgTypes[148] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6646,7 +6892,7 @@ func (x *Product) ProtoReflect() protoreflect.Message { // Deprecated: Use Product.ProtoReflect.Descriptor instead. func (*Product) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{143} + return file_product_proto_rawDescGZIP(), []int{148} } func (x *Product) GetId() string { @@ -6681,7 +6927,7 @@ type Storage struct { func (x *Storage) Reset() { *x = Storage{} - mi := &file_product_proto_msgTypes[144] + mi := &file_product_proto_msgTypes[149] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6693,7 +6939,7 @@ func (x *Storage) String() string { func (*Storage) ProtoMessage() {} func (x *Storage) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[144] + mi := &file_product_proto_msgTypes[149] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6706,7 +6952,7 @@ func (x *Storage) ProtoReflect() protoreflect.Message { // Deprecated: Use Storage.ProtoReflect.Descriptor instead. func (*Storage) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{144} + return file_product_proto_rawDescGZIP(), []int{149} } func (x *Storage) GetId() string { @@ -6741,7 +6987,7 @@ type Warehouse struct { func (x *Warehouse) Reset() { *x = Warehouse{} - mi := &file_product_proto_msgTypes[145] + mi := &file_product_proto_msgTypes[150] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6753,7 +6999,7 @@ func (x *Warehouse) String() string { func (*Warehouse) ProtoMessage() {} func (x *Warehouse) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[145] + mi := &file_product_proto_msgTypes[150] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6766,7 +7012,7 @@ func (x *Warehouse) ProtoReflect() protoreflect.Message { // Deprecated: Use Warehouse.ProtoReflect.Descriptor instead. func (*Warehouse) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{145} + return file_product_proto_rawDescGZIP(), []int{150} } func (x *Warehouse) GetId() string { @@ -6800,7 +7046,7 @@ type User struct { func (x *User) Reset() { *x = User{} - mi := &file_product_proto_msgTypes[146] + mi := &file_product_proto_msgTypes[151] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6812,7 +7058,7 @@ func (x *User) String() string { func (*User) ProtoMessage() {} func (x *User) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[146] + mi := &file_product_proto_msgTypes[151] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6825,7 +7071,7 @@ func (x *User) ProtoReflect() protoreflect.Message { // Deprecated: Use User.ProtoReflect.Descriptor instead. func (*User) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{146} + return file_product_proto_rawDescGZIP(), []int{151} } func (x *User) GetId() string { @@ -6853,7 +7099,7 @@ type NestedTypeA struct { func (x *NestedTypeA) Reset() { *x = NestedTypeA{} - mi := &file_product_proto_msgTypes[147] + mi := &file_product_proto_msgTypes[152] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6865,7 +7111,7 @@ func (x *NestedTypeA) String() string { func (*NestedTypeA) ProtoMessage() {} func (x *NestedTypeA) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[147] + mi := &file_product_proto_msgTypes[152] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6878,7 +7124,7 @@ func (x *NestedTypeA) ProtoReflect() protoreflect.Message { // Deprecated: Use NestedTypeA.ProtoReflect.Descriptor instead. func (*NestedTypeA) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{147} + return file_product_proto_rawDescGZIP(), []int{152} } func (x *NestedTypeA) GetId() string { @@ -6913,7 +7159,7 @@ type RecursiveType struct { func (x *RecursiveType) Reset() { *x = RecursiveType{} - mi := &file_product_proto_msgTypes[148] + mi := &file_product_proto_msgTypes[153] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6925,7 +7171,7 @@ func (x *RecursiveType) String() string { func (*RecursiveType) ProtoMessage() {} func (x *RecursiveType) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[148] + mi := &file_product_proto_msgTypes[153] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6938,7 +7184,7 @@ func (x *RecursiveType) ProtoReflect() protoreflect.Message { // Deprecated: Use RecursiveType.ProtoReflect.Descriptor instead. func (*RecursiveType) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{148} + return file_product_proto_rawDescGZIP(), []int{153} } func (x *RecursiveType) GetId() string { @@ -6974,7 +7220,7 @@ type TypeWithMultipleFilterFields struct { func (x *TypeWithMultipleFilterFields) Reset() { *x = TypeWithMultipleFilterFields{} - mi := &file_product_proto_msgTypes[149] + mi := &file_product_proto_msgTypes[154] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6986,7 +7232,7 @@ func (x *TypeWithMultipleFilterFields) String() string { func (*TypeWithMultipleFilterFields) ProtoMessage() {} func (x *TypeWithMultipleFilterFields) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[149] + mi := &file_product_proto_msgTypes[154] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6999,7 +7245,7 @@ func (x *TypeWithMultipleFilterFields) ProtoReflect() protoreflect.Message { // Deprecated: Use TypeWithMultipleFilterFields.ProtoReflect.Descriptor instead. func (*TypeWithMultipleFilterFields) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{149} + return file_product_proto_rawDescGZIP(), []int{154} } func (x *TypeWithMultipleFilterFields) GetId() string { @@ -7040,7 +7286,7 @@ type FilterTypeInput struct { func (x *FilterTypeInput) Reset() { *x = FilterTypeInput{} - mi := &file_product_proto_msgTypes[150] + mi := &file_product_proto_msgTypes[155] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7052,7 +7298,7 @@ func (x *FilterTypeInput) String() string { func (*FilterTypeInput) ProtoMessage() {} func (x *FilterTypeInput) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[150] + mi := &file_product_proto_msgTypes[155] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7065,7 +7311,7 @@ func (x *FilterTypeInput) ProtoReflect() protoreflect.Message { // Deprecated: Use FilterTypeInput.ProtoReflect.Descriptor instead. func (*FilterTypeInput) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{150} + return file_product_proto_rawDescGZIP(), []int{155} } func (x *FilterTypeInput) GetFilterField_1() string { @@ -7091,7 +7337,7 @@ type ComplexFilterTypeInput struct { func (x *ComplexFilterTypeInput) Reset() { *x = ComplexFilterTypeInput{} - mi := &file_product_proto_msgTypes[151] + mi := &file_product_proto_msgTypes[156] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7103,7 +7349,7 @@ func (x *ComplexFilterTypeInput) String() string { func (*ComplexFilterTypeInput) ProtoMessage() {} func (x *ComplexFilterTypeInput) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[151] + mi := &file_product_proto_msgTypes[156] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7116,7 +7362,7 @@ func (x *ComplexFilterTypeInput) ProtoReflect() protoreflect.Message { // Deprecated: Use ComplexFilterTypeInput.ProtoReflect.Descriptor instead. func (*ComplexFilterTypeInput) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{151} + return file_product_proto_rawDescGZIP(), []int{156} } func (x *ComplexFilterTypeInput) GetFilter() *FilterType { @@ -7136,7 +7382,7 @@ type TypeWithComplexFilterInput struct { func (x *TypeWithComplexFilterInput) Reset() { *x = TypeWithComplexFilterInput{} - mi := &file_product_proto_msgTypes[152] + mi := &file_product_proto_msgTypes[157] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7148,7 +7394,7 @@ func (x *TypeWithComplexFilterInput) String() string { func (*TypeWithComplexFilterInput) ProtoMessage() {} func (x *TypeWithComplexFilterInput) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[152] + mi := &file_product_proto_msgTypes[157] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7161,7 +7407,7 @@ func (x *TypeWithComplexFilterInput) ProtoReflect() protoreflect.Message { // Deprecated: Use TypeWithComplexFilterInput.ProtoReflect.Descriptor instead. func (*TypeWithComplexFilterInput) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{152} + return file_product_proto_rawDescGZIP(), []int{157} } func (x *TypeWithComplexFilterInput) GetId() string { @@ -7189,7 +7435,7 @@ type OrderInput struct { func (x *OrderInput) Reset() { *x = OrderInput{} - mi := &file_product_proto_msgTypes[153] + mi := &file_product_proto_msgTypes[158] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7201,7 +7447,7 @@ func (x *OrderInput) String() string { func (*OrderInput) ProtoMessage() {} func (x *OrderInput) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[153] + mi := &file_product_proto_msgTypes[158] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7214,7 +7460,7 @@ func (x *OrderInput) ProtoReflect() protoreflect.Message { // Deprecated: Use OrderInput.ProtoReflect.Descriptor instead. func (*OrderInput) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{153} + return file_product_proto_rawDescGZIP(), []int{158} } func (x *OrderInput) GetOrderId() string { @@ -7250,7 +7496,7 @@ type Order struct { func (x *Order) Reset() { *x = Order{} - mi := &file_product_proto_msgTypes[154] + mi := &file_product_proto_msgTypes[159] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7262,7 +7508,7 @@ func (x *Order) String() string { func (*Order) ProtoMessage() {} func (x *Order) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[154] + mi := &file_product_proto_msgTypes[159] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7275,7 +7521,7 @@ func (x *Order) ProtoReflect() protoreflect.Message { // Deprecated: Use Order.ProtoReflect.Descriptor instead. func (*Order) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{154} + return file_product_proto_rawDescGZIP(), []int{159} } func (x *Order) GetOrderId() string { @@ -7318,7 +7564,7 @@ type Category struct { func (x *Category) Reset() { *x = Category{} - mi := &file_product_proto_msgTypes[155] + mi := &file_product_proto_msgTypes[160] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7330,7 +7576,7 @@ func (x *Category) String() string { func (*Category) ProtoMessage() {} func (x *Category) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[155] + mi := &file_product_proto_msgTypes[160] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7343,7 +7589,7 @@ func (x *Category) ProtoReflect() protoreflect.Message { // Deprecated: Use Category.ProtoReflect.Descriptor instead. func (*Category) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{155} + return file_product_proto_rawDescGZIP(), []int{160} } func (x *Category) GetId() string { @@ -7384,7 +7630,7 @@ type CategoryFilter struct { func (x *CategoryFilter) Reset() { *x = CategoryFilter{} - mi := &file_product_proto_msgTypes[156] + mi := &file_product_proto_msgTypes[161] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7396,7 +7642,7 @@ func (x *CategoryFilter) String() string { func (*CategoryFilter) ProtoMessage() {} func (x *CategoryFilter) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[156] + mi := &file_product_proto_msgTypes[161] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7409,7 +7655,7 @@ func (x *CategoryFilter) ProtoReflect() protoreflect.Message { // Deprecated: Use CategoryFilter.ProtoReflect.Descriptor instead. func (*CategoryFilter) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{156} + return file_product_proto_rawDescGZIP(), []int{161} } func (x *CategoryFilter) GetCategory() CategoryKind { @@ -7439,7 +7685,7 @@ type Animal struct { func (x *Animal) Reset() { *x = Animal{} - mi := &file_product_proto_msgTypes[157] + mi := &file_product_proto_msgTypes[162] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7451,7 +7697,7 @@ func (x *Animal) String() string { func (*Animal) ProtoMessage() {} func (x *Animal) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[157] + mi := &file_product_proto_msgTypes[162] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7464,7 +7710,7 @@ func (x *Animal) ProtoReflect() protoreflect.Message { // Deprecated: Use Animal.ProtoReflect.Descriptor instead. func (*Animal) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{157} + return file_product_proto_rawDescGZIP(), []int{162} } func (x *Animal) GetInstance() isAnimal_Instance { @@ -7518,7 +7764,7 @@ type SearchInput struct { func (x *SearchInput) Reset() { *x = SearchInput{} - mi := &file_product_proto_msgTypes[158] + mi := &file_product_proto_msgTypes[163] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7530,7 +7776,7 @@ func (x *SearchInput) String() string { func (*SearchInput) ProtoMessage() {} func (x *SearchInput) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[158] + mi := &file_product_proto_msgTypes[163] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7543,7 +7789,7 @@ func (x *SearchInput) ProtoReflect() protoreflect.Message { // Deprecated: Use SearchInput.ProtoReflect.Descriptor instead. func (*SearchInput) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{158} + return file_product_proto_rawDescGZIP(), []int{163} } func (x *SearchInput) GetQuery() string { @@ -7574,7 +7820,7 @@ type SearchResult struct { func (x *SearchResult) Reset() { *x = SearchResult{} - mi := &file_product_proto_msgTypes[159] + mi := &file_product_proto_msgTypes[164] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7586,7 +7832,7 @@ func (x *SearchResult) String() string { func (*SearchResult) ProtoMessage() {} func (x *SearchResult) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[159] + mi := &file_product_proto_msgTypes[164] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7599,7 +7845,7 @@ func (x *SearchResult) ProtoReflect() protoreflect.Message { // Deprecated: Use SearchResult.ProtoReflect.Descriptor instead. func (*SearchResult) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{159} + return file_product_proto_rawDescGZIP(), []int{164} } func (x *SearchResult) GetValue() isSearchResult_Value { @@ -7674,7 +7920,7 @@ type NullableFieldsType struct { func (x *NullableFieldsType) Reset() { *x = NullableFieldsType{} - mi := &file_product_proto_msgTypes[160] + mi := &file_product_proto_msgTypes[165] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7686,7 +7932,7 @@ func (x *NullableFieldsType) String() string { func (*NullableFieldsType) ProtoMessage() {} func (x *NullableFieldsType) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[160] + mi := &file_product_proto_msgTypes[165] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7699,7 +7945,7 @@ func (x *NullableFieldsType) ProtoReflect() protoreflect.Message { // Deprecated: Use NullableFieldsType.ProtoReflect.Descriptor instead. func (*NullableFieldsType) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{160} + return file_product_proto_rawDescGZIP(), []int{165} } func (x *NullableFieldsType) GetId() string { @@ -7769,7 +8015,7 @@ type NullableFieldsFilter struct { func (x *NullableFieldsFilter) Reset() { *x = NullableFieldsFilter{} - mi := &file_product_proto_msgTypes[161] + mi := &file_product_proto_msgTypes[166] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7781,7 +8027,7 @@ func (x *NullableFieldsFilter) String() string { func (*NullableFieldsFilter) ProtoMessage() {} func (x *NullableFieldsFilter) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[161] + mi := &file_product_proto_msgTypes[166] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7794,7 +8040,7 @@ func (x *NullableFieldsFilter) ProtoReflect() protoreflect.Message { // Deprecated: Use NullableFieldsFilter.ProtoReflect.Descriptor instead. func (*NullableFieldsFilter) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{161} + return file_product_proto_rawDescGZIP(), []int{166} } func (x *NullableFieldsFilter) GetName() *wrapperspb.StringValue { @@ -7846,7 +8092,7 @@ type BlogPost struct { func (x *BlogPost) Reset() { *x = BlogPost{} - mi := &file_product_proto_msgTypes[162] + mi := &file_product_proto_msgTypes[167] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7858,7 +8104,7 @@ func (x *BlogPost) String() string { func (*BlogPost) ProtoMessage() {} func (x *BlogPost) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[162] + mi := &file_product_proto_msgTypes[167] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7871,7 +8117,7 @@ func (x *BlogPost) ProtoReflect() protoreflect.Message { // Deprecated: Use BlogPost.ProtoReflect.Descriptor instead. func (*BlogPost) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{162} + return file_product_proto_rawDescGZIP(), []int{167} } func (x *BlogPost) GetId() string { @@ -8025,7 +8271,7 @@ type BlogPostFilter struct { func (x *BlogPostFilter) Reset() { *x = BlogPostFilter{} - mi := &file_product_proto_msgTypes[163] + mi := &file_product_proto_msgTypes[168] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8037,7 +8283,7 @@ func (x *BlogPostFilter) String() string { func (*BlogPostFilter) ProtoMessage() {} func (x *BlogPostFilter) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[163] + mi := &file_product_proto_msgTypes[168] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8050,7 +8296,7 @@ func (x *BlogPostFilter) ProtoReflect() protoreflect.Message { // Deprecated: Use BlogPostFilter.ProtoReflect.Descriptor instead. func (*BlogPostFilter) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{163} + return file_product_proto_rawDescGZIP(), []int{168} } func (x *BlogPostFilter) GetTitle() *wrapperspb.StringValue { @@ -8097,7 +8343,7 @@ type Author struct { func (x *Author) Reset() { *x = Author{} - mi := &file_product_proto_msgTypes[164] + mi := &file_product_proto_msgTypes[169] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8109,7 +8355,7 @@ func (x *Author) String() string { func (*Author) ProtoMessage() {} func (x *Author) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[164] + mi := &file_product_proto_msgTypes[169] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8122,7 +8368,7 @@ func (x *Author) ProtoReflect() protoreflect.Message { // Deprecated: Use Author.ProtoReflect.Descriptor instead. func (*Author) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{164} + return file_product_proto_rawDescGZIP(), []int{169} } func (x *Author) GetId() string { @@ -8241,7 +8487,7 @@ type AuthorFilter struct { func (x *AuthorFilter) Reset() { *x = AuthorFilter{} - mi := &file_product_proto_msgTypes[165] + mi := &file_product_proto_msgTypes[170] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8253,7 +8499,7 @@ func (x *AuthorFilter) String() string { func (*AuthorFilter) ProtoMessage() {} func (x *AuthorFilter) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[165] + mi := &file_product_proto_msgTypes[170] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8266,7 +8512,7 @@ func (x *AuthorFilter) ProtoReflect() protoreflect.Message { // Deprecated: Use AuthorFilter.ProtoReflect.Descriptor instead. func (*AuthorFilter) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{165} + return file_product_proto_rawDescGZIP(), []int{170} } func (x *AuthorFilter) GetName() *wrapperspb.StringValue { @@ -8299,7 +8545,7 @@ type UserInput struct { func (x *UserInput) Reset() { *x = UserInput{} - mi := &file_product_proto_msgTypes[166] + mi := &file_product_proto_msgTypes[171] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8311,7 +8557,7 @@ func (x *UserInput) String() string { func (*UserInput) ProtoMessage() {} func (x *UserInput) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[166] + mi := &file_product_proto_msgTypes[171] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8324,7 +8570,7 @@ func (x *UserInput) ProtoReflect() protoreflect.Message { // Deprecated: Use UserInput.ProtoReflect.Descriptor instead. func (*UserInput) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{166} + return file_product_proto_rawDescGZIP(), []int{171} } func (x *UserInput) GetName() string { @@ -8344,7 +8590,7 @@ type ActionInput struct { func (x *ActionInput) Reset() { *x = ActionInput{} - mi := &file_product_proto_msgTypes[167] + mi := &file_product_proto_msgTypes[172] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8356,7 +8602,7 @@ func (x *ActionInput) String() string { func (*ActionInput) ProtoMessage() {} func (x *ActionInput) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[167] + mi := &file_product_proto_msgTypes[172] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8369,7 +8615,7 @@ func (x *ActionInput) ProtoReflect() protoreflect.Message { // Deprecated: Use ActionInput.ProtoReflect.Descriptor instead. func (*ActionInput) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{167} + return file_product_proto_rawDescGZIP(), []int{172} } func (x *ActionInput) GetType() string { @@ -8399,7 +8645,7 @@ type ActionResult struct { func (x *ActionResult) Reset() { *x = ActionResult{} - mi := &file_product_proto_msgTypes[168] + mi := &file_product_proto_msgTypes[173] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8411,7 +8657,7 @@ func (x *ActionResult) String() string { func (*ActionResult) ProtoMessage() {} func (x *ActionResult) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[168] + mi := &file_product_proto_msgTypes[173] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8424,7 +8670,7 @@ func (x *ActionResult) ProtoReflect() protoreflect.Message { // Deprecated: Use ActionResult.ProtoReflect.Descriptor instead. func (*ActionResult) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{168} + return file_product_proto_rawDescGZIP(), []int{173} } func (x *ActionResult) GetValue() isActionResult_Value { @@ -8483,7 +8729,7 @@ type NullableFieldsInput struct { func (x *NullableFieldsInput) Reset() { *x = NullableFieldsInput{} - mi := &file_product_proto_msgTypes[169] + mi := &file_product_proto_msgTypes[174] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8495,7 +8741,7 @@ func (x *NullableFieldsInput) String() string { func (*NullableFieldsInput) ProtoMessage() {} func (x *NullableFieldsInput) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[169] + mi := &file_product_proto_msgTypes[174] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8508,7 +8754,7 @@ func (x *NullableFieldsInput) ProtoReflect() protoreflect.Message { // Deprecated: Use NullableFieldsInput.ProtoReflect.Descriptor instead. func (*NullableFieldsInput) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{169} + return file_product_proto_rawDescGZIP(), []int{174} } func (x *NullableFieldsInput) GetName() string { @@ -8584,7 +8830,7 @@ type BlogPostInput struct { func (x *BlogPostInput) Reset() { *x = BlogPostInput{} - mi := &file_product_proto_msgTypes[170] + mi := &file_product_proto_msgTypes[175] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8596,7 +8842,7 @@ func (x *BlogPostInput) String() string { func (*BlogPostInput) ProtoMessage() {} func (x *BlogPostInput) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[170] + mi := &file_product_proto_msgTypes[175] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8609,7 +8855,7 @@ func (x *BlogPostInput) ProtoReflect() protoreflect.Message { // Deprecated: Use BlogPostInput.ProtoReflect.Descriptor instead. func (*BlogPostInput) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{170} + return file_product_proto_rawDescGZIP(), []int{175} } func (x *BlogPostInput) GetTitle() string { @@ -8742,7 +8988,7 @@ type AuthorInput struct { func (x *AuthorInput) Reset() { *x = AuthorInput{} - mi := &file_product_proto_msgTypes[171] + mi := &file_product_proto_msgTypes[176] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8754,7 +9000,7 @@ func (x *AuthorInput) String() string { func (*AuthorInput) ProtoMessage() {} func (x *AuthorInput) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[171] + mi := &file_product_proto_msgTypes[176] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8767,7 +9013,7 @@ func (x *AuthorInput) ProtoReflect() protoreflect.Message { // Deprecated: Use AuthorInput.ProtoReflect.Descriptor instead. func (*AuthorInput) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{171} + return file_product_proto_rawDescGZIP(), []int{176} } func (x *AuthorInput) GetName() string { @@ -8851,7 +9097,7 @@ type NestedTypeB struct { func (x *NestedTypeB) Reset() { *x = NestedTypeB{} - mi := &file_product_proto_msgTypes[172] + mi := &file_product_proto_msgTypes[177] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8863,7 +9109,7 @@ func (x *NestedTypeB) String() string { func (*NestedTypeB) ProtoMessage() {} func (x *NestedTypeB) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[172] + mi := &file_product_proto_msgTypes[177] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8876,7 +9122,7 @@ func (x *NestedTypeB) ProtoReflect() protoreflect.Message { // Deprecated: Use NestedTypeB.ProtoReflect.Descriptor instead. func (*NestedTypeB) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{172} + return file_product_proto_rawDescGZIP(), []int{177} } func (x *NestedTypeB) GetId() string { @@ -8910,7 +9156,7 @@ type NestedTypeC struct { func (x *NestedTypeC) Reset() { *x = NestedTypeC{} - mi := &file_product_proto_msgTypes[173] + mi := &file_product_proto_msgTypes[178] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8922,7 +9168,7 @@ func (x *NestedTypeC) String() string { func (*NestedTypeC) ProtoMessage() {} func (x *NestedTypeC) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[173] + mi := &file_product_proto_msgTypes[178] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8935,7 +9181,7 @@ func (x *NestedTypeC) ProtoReflect() protoreflect.Message { // Deprecated: Use NestedTypeC.ProtoReflect.Descriptor instead. func (*NestedTypeC) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{173} + return file_product_proto_rawDescGZIP(), []int{178} } func (x *NestedTypeC) GetId() string { @@ -8964,7 +9210,7 @@ type FilterType struct { func (x *FilterType) Reset() { *x = FilterType{} - mi := &file_product_proto_msgTypes[174] + mi := &file_product_proto_msgTypes[179] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8976,7 +9222,7 @@ func (x *FilterType) String() string { func (*FilterType) ProtoMessage() {} func (x *FilterType) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[174] + mi := &file_product_proto_msgTypes[179] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8989,7 +9235,7 @@ func (x *FilterType) ProtoReflect() protoreflect.Message { // Deprecated: Use FilterType.ProtoReflect.Descriptor instead. func (*FilterType) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{174} + return file_product_proto_rawDescGZIP(), []int{179} } func (x *FilterType) GetName() string { @@ -9030,7 +9276,7 @@ type Pagination struct { func (x *Pagination) Reset() { *x = Pagination{} - mi := &file_product_proto_msgTypes[175] + mi := &file_product_proto_msgTypes[180] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9042,7 +9288,7 @@ func (x *Pagination) String() string { func (*Pagination) ProtoMessage() {} func (x *Pagination) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[175] + mi := &file_product_proto_msgTypes[180] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9055,7 +9301,7 @@ func (x *Pagination) ProtoReflect() protoreflect.Message { // Deprecated: Use Pagination.ProtoReflect.Descriptor instead. func (*Pagination) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{175} + return file_product_proto_rawDescGZIP(), []int{180} } func (x *Pagination) GetPage() int32 { @@ -9083,7 +9329,7 @@ type OrderLineInput struct { func (x *OrderLineInput) Reset() { *x = OrderLineInput{} - mi := &file_product_proto_msgTypes[176] + mi := &file_product_proto_msgTypes[181] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9095,7 +9341,7 @@ func (x *OrderLineInput) String() string { func (*OrderLineInput) ProtoMessage() {} func (x *OrderLineInput) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[176] + mi := &file_product_proto_msgTypes[181] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9108,7 +9354,7 @@ func (x *OrderLineInput) ProtoReflect() protoreflect.Message { // Deprecated: Use OrderLineInput.ProtoReflect.Descriptor instead. func (*OrderLineInput) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{176} + return file_product_proto_rawDescGZIP(), []int{181} } func (x *OrderLineInput) GetProductId() string { @@ -9143,7 +9389,7 @@ type OrderLine struct { func (x *OrderLine) Reset() { *x = OrderLine{} - mi := &file_product_proto_msgTypes[177] + mi := &file_product_proto_msgTypes[182] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9155,7 +9401,7 @@ func (x *OrderLine) String() string { func (*OrderLine) ProtoMessage() {} func (x *OrderLine) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[177] + mi := &file_product_proto_msgTypes[182] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9168,7 +9414,7 @@ func (x *OrderLine) ProtoReflect() protoreflect.Message { // Deprecated: Use OrderLine.ProtoReflect.Descriptor instead. func (*OrderLine) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{177} + return file_product_proto_rawDescGZIP(), []int{182} } func (x *OrderLine) GetProductId() string { @@ -9204,7 +9450,7 @@ type Subcategory struct { func (x *Subcategory) Reset() { *x = Subcategory{} - mi := &file_product_proto_msgTypes[178] + mi := &file_product_proto_msgTypes[183] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9216,7 +9462,7 @@ func (x *Subcategory) String() string { func (*Subcategory) ProtoMessage() {} func (x *Subcategory) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[178] + mi := &file_product_proto_msgTypes[183] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9229,7 +9475,7 @@ func (x *Subcategory) ProtoReflect() protoreflect.Message { // Deprecated: Use Subcategory.ProtoReflect.Descriptor instead. func (*Subcategory) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{178} + return file_product_proto_rawDescGZIP(), []int{183} } func (x *Subcategory) GetId() string { @@ -9261,19 +9507,20 @@ func (x *Subcategory) GetIsActive() bool { } type CategoryMetrics struct { - state protoimpl.MessageState `protogen:"open.v1"` - Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` - MetricType string `protobuf:"bytes,2,opt,name=metric_type,json=metricType,proto3" json:"metric_type,omitempty"` - Value float64 `protobuf:"fixed64,3,opt,name=value,proto3" json:"value,omitempty"` - Timestamp string `protobuf:"bytes,4,opt,name=timestamp,proto3" json:"timestamp,omitempty"` - CategoryId string `protobuf:"bytes,5,opt,name=category_id,json=categoryId,proto3" json:"category_id,omitempty"` - unknownFields protoimpl.UnknownFields - sizeCache protoimpl.SizeCache + state protoimpl.MessageState `protogen:"open.v1"` + Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` + MetricType string `protobuf:"bytes,2,opt,name=metric_type,json=metricType,proto3" json:"metric_type,omitempty"` + Value float64 `protobuf:"fixed64,3,opt,name=value,proto3" json:"value,omitempty"` + Timestamp string `protobuf:"bytes,4,opt,name=timestamp,proto3" json:"timestamp,omitempty"` + CategoryId string `protobuf:"bytes,5,opt,name=category_id,json=categoryId,proto3" json:"category_id,omitempty"` + RelatedCategory *Category `protobuf:"bytes,6,opt,name=related_category,json=relatedCategory,proto3" json:"related_category,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *CategoryMetrics) Reset() { *x = CategoryMetrics{} - mi := &file_product_proto_msgTypes[179] + mi := &file_product_proto_msgTypes[184] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9285,7 +9532,7 @@ func (x *CategoryMetrics) String() string { func (*CategoryMetrics) ProtoMessage() {} func (x *CategoryMetrics) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[179] + mi := &file_product_proto_msgTypes[184] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9298,7 +9545,7 @@ func (x *CategoryMetrics) ProtoReflect() protoreflect.Message { // Deprecated: Use CategoryMetrics.ProtoReflect.Descriptor instead. func (*CategoryMetrics) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{179} + return file_product_proto_rawDescGZIP(), []int{184} } func (x *CategoryMetrics) GetId() string { @@ -9336,6 +9583,13 @@ func (x *CategoryMetrics) GetCategoryId() string { return "" } +func (x *CategoryMetrics) GetRelatedCategory() *Category { + if x != nil { + return x.RelatedCategory + } + return nil +} + type Cat struct { state protoimpl.MessageState `protogen:"open.v1"` Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` @@ -9348,7 +9602,7 @@ type Cat struct { func (x *Cat) Reset() { *x = Cat{} - mi := &file_product_proto_msgTypes[180] + mi := &file_product_proto_msgTypes[185] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9360,7 +9614,7 @@ func (x *Cat) String() string { func (*Cat) ProtoMessage() {} func (x *Cat) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[180] + mi := &file_product_proto_msgTypes[185] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9373,7 +9627,7 @@ func (x *Cat) ProtoReflect() protoreflect.Message { // Deprecated: Use Cat.ProtoReflect.Descriptor instead. func (*Cat) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{180} + return file_product_proto_rawDescGZIP(), []int{185} } func (x *Cat) GetId() string { @@ -9416,7 +9670,7 @@ type Dog struct { func (x *Dog) Reset() { *x = Dog{} - mi := &file_product_proto_msgTypes[181] + mi := &file_product_proto_msgTypes[186] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9428,7 +9682,7 @@ func (x *Dog) String() string { func (*Dog) ProtoMessage() {} func (x *Dog) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[181] + mi := &file_product_proto_msgTypes[186] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9441,7 +9695,7 @@ func (x *Dog) ProtoReflect() protoreflect.Message { // Deprecated: Use Dog.ProtoReflect.Descriptor instead. func (*Dog) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{181} + return file_product_proto_rawDescGZIP(), []int{186} } func (x *Dog) GetId() string { @@ -9482,7 +9736,7 @@ type ActionSuccess struct { func (x *ActionSuccess) Reset() { *x = ActionSuccess{} - mi := &file_product_proto_msgTypes[182] + mi := &file_product_proto_msgTypes[187] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9494,7 +9748,7 @@ func (x *ActionSuccess) String() string { func (*ActionSuccess) ProtoMessage() {} func (x *ActionSuccess) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[182] + mi := &file_product_proto_msgTypes[187] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9507,7 +9761,7 @@ func (x *ActionSuccess) ProtoReflect() protoreflect.Message { // Deprecated: Use ActionSuccess.ProtoReflect.Descriptor instead. func (*ActionSuccess) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{182} + return file_product_proto_rawDescGZIP(), []int{187} } func (x *ActionSuccess) GetMessage() string { @@ -9534,7 +9788,7 @@ type ActionError struct { func (x *ActionError) Reset() { *x = ActionError{} - mi := &file_product_proto_msgTypes[183] + mi := &file_product_proto_msgTypes[188] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9546,7 +9800,7 @@ func (x *ActionError) String() string { func (*ActionError) ProtoMessage() {} func (x *ActionError) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[183] + mi := &file_product_proto_msgTypes[188] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9559,7 +9813,7 @@ func (x *ActionError) ProtoReflect() protoreflect.Message { // Deprecated: Use ActionError.ProtoReflect.Descriptor instead. func (*ActionError) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{183} + return file_product_proto_rawDescGZIP(), []int{188} } func (x *ActionError) GetMessage() string { @@ -9586,7 +9840,7 @@ type CategoryInput struct { func (x *CategoryInput) Reset() { *x = CategoryInput{} - mi := &file_product_proto_msgTypes[184] + mi := &file_product_proto_msgTypes[189] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9598,7 +9852,7 @@ func (x *CategoryInput) String() string { func (*CategoryInput) ProtoMessage() {} func (x *CategoryInput) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[184] + mi := &file_product_proto_msgTypes[189] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9611,7 +9865,7 @@ func (x *CategoryInput) ProtoReflect() protoreflect.Message { // Deprecated: Use CategoryInput.ProtoReflect.Descriptor instead. func (*CategoryInput) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{184} + return file_product_proto_rawDescGZIP(), []int{189} } func (x *CategoryInput) GetName() string { @@ -9640,7 +9894,7 @@ type ProductCountFilter struct { func (x *ProductCountFilter) Reset() { *x = ProductCountFilter{} - mi := &file_product_proto_msgTypes[185] + mi := &file_product_proto_msgTypes[190] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9652,7 +9906,7 @@ func (x *ProductCountFilter) String() string { func (*ProductCountFilter) ProtoMessage() {} func (x *ProductCountFilter) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[185] + mi := &file_product_proto_msgTypes[190] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9665,7 +9919,7 @@ func (x *ProductCountFilter) ProtoReflect() protoreflect.Message { // Deprecated: Use ProductCountFilter.ProtoReflect.Descriptor instead. func (*ProductCountFilter) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{185} + return file_product_proto_rawDescGZIP(), []int{190} } func (x *ProductCountFilter) GetMinPrice() *wrapperspb.DoubleValue { @@ -9709,7 +9963,7 @@ type SubcategoryItemFilter struct { func (x *SubcategoryItemFilter) Reset() { *x = SubcategoryItemFilter{} - mi := &file_product_proto_msgTypes[186] + mi := &file_product_proto_msgTypes[191] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9721,7 +9975,7 @@ func (x *SubcategoryItemFilter) String() string { func (*SubcategoryItemFilter) ProtoMessage() {} func (x *SubcategoryItemFilter) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[186] + mi := &file_product_proto_msgTypes[191] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9734,7 +9988,7 @@ func (x *SubcategoryItemFilter) ProtoReflect() protoreflect.Message { // Deprecated: Use SubcategoryItemFilter.ProtoReflect.Descriptor instead. func (*SubcategoryItemFilter) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{186} + return file_product_proto_rawDescGZIP(), []int{191} } func (x *SubcategoryItemFilter) GetMinPrice() *wrapperspb.DoubleValue { @@ -9783,7 +10037,7 @@ type ShippingEstimateInput struct { func (x *ShippingEstimateInput) Reset() { *x = ShippingEstimateInput{} - mi := &file_product_proto_msgTypes[187] + mi := &file_product_proto_msgTypes[192] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9795,7 +10049,7 @@ func (x *ShippingEstimateInput) String() string { func (*ShippingEstimateInput) ProtoMessage() {} func (x *ShippingEstimateInput) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[187] + mi := &file_product_proto_msgTypes[192] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9808,7 +10062,7 @@ func (x *ShippingEstimateInput) ProtoReflect() protoreflect.Message { // Deprecated: Use ShippingEstimateInput.ProtoReflect.Descriptor instead. func (*ShippingEstimateInput) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{187} + return file_product_proto_rawDescGZIP(), []int{192} } func (x *ShippingEstimateInput) GetDestination() ShippingDestination { @@ -9841,7 +10095,7 @@ type ListOfAuthorFilter_List struct { func (x *ListOfAuthorFilter_List) Reset() { *x = ListOfAuthorFilter_List{} - mi := &file_product_proto_msgTypes[188] + mi := &file_product_proto_msgTypes[193] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9853,7 +10107,7 @@ func (x *ListOfAuthorFilter_List) String() string { func (*ListOfAuthorFilter_List) ProtoMessage() {} func (x *ListOfAuthorFilter_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[188] + mi := &file_product_proto_msgTypes[193] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9885,7 +10139,7 @@ type ListOfAuthorInput_List struct { func (x *ListOfAuthorInput_List) Reset() { *x = ListOfAuthorInput_List{} - mi := &file_product_proto_msgTypes[189] + mi := &file_product_proto_msgTypes[194] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9897,7 +10151,7 @@ func (x *ListOfAuthorInput_List) String() string { func (*ListOfAuthorInput_List) ProtoMessage() {} func (x *ListOfAuthorInput_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[189] + mi := &file_product_proto_msgTypes[194] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9929,7 +10183,7 @@ type ListOfBlogPost_List struct { func (x *ListOfBlogPost_List) Reset() { *x = ListOfBlogPost_List{} - mi := &file_product_proto_msgTypes[190] + mi := &file_product_proto_msgTypes[195] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9941,7 +10195,7 @@ func (x *ListOfBlogPost_List) String() string { func (*ListOfBlogPost_List) ProtoMessage() {} func (x *ListOfBlogPost_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[190] + mi := &file_product_proto_msgTypes[195] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9973,7 +10227,7 @@ type ListOfBlogPostFilter_List struct { func (x *ListOfBlogPostFilter_List) Reset() { *x = ListOfBlogPostFilter_List{} - mi := &file_product_proto_msgTypes[191] + mi := &file_product_proto_msgTypes[196] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9985,7 +10239,7 @@ func (x *ListOfBlogPostFilter_List) String() string { func (*ListOfBlogPostFilter_List) ProtoMessage() {} func (x *ListOfBlogPostFilter_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[191] + mi := &file_product_proto_msgTypes[196] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -10017,7 +10271,7 @@ type ListOfBlogPostInput_List struct { func (x *ListOfBlogPostInput_List) Reset() { *x = ListOfBlogPostInput_List{} - mi := &file_product_proto_msgTypes[192] + mi := &file_product_proto_msgTypes[197] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -10029,7 +10283,7 @@ func (x *ListOfBlogPostInput_List) String() string { func (*ListOfBlogPostInput_List) ProtoMessage() {} func (x *ListOfBlogPostInput_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[192] + mi := &file_product_proto_msgTypes[197] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -10061,7 +10315,7 @@ type ListOfBoolean_List struct { func (x *ListOfBoolean_List) Reset() { *x = ListOfBoolean_List{} - mi := &file_product_proto_msgTypes[193] + mi := &file_product_proto_msgTypes[198] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -10073,7 +10327,7 @@ func (x *ListOfBoolean_List) String() string { func (*ListOfBoolean_List) ProtoMessage() {} func (x *ListOfBoolean_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[193] + mi := &file_product_proto_msgTypes[198] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -10105,7 +10359,7 @@ type ListOfCategory_List struct { func (x *ListOfCategory_List) Reset() { *x = ListOfCategory_List{} - mi := &file_product_proto_msgTypes[194] + mi := &file_product_proto_msgTypes[199] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -10117,7 +10371,7 @@ func (x *ListOfCategory_List) String() string { func (*ListOfCategory_List) ProtoMessage() {} func (x *ListOfCategory_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[194] + mi := &file_product_proto_msgTypes[199] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -10149,7 +10403,7 @@ type ListOfCategoryInput_List struct { func (x *ListOfCategoryInput_List) Reset() { *x = ListOfCategoryInput_List{} - mi := &file_product_proto_msgTypes[195] + mi := &file_product_proto_msgTypes[200] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -10161,7 +10415,7 @@ func (x *ListOfCategoryInput_List) String() string { func (*ListOfCategoryInput_List) ProtoMessage() {} func (x *ListOfCategoryInput_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[195] + mi := &file_product_proto_msgTypes[200] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -10193,7 +10447,7 @@ type ListOfFloat_List struct { func (x *ListOfFloat_List) Reset() { *x = ListOfFloat_List{} - mi := &file_product_proto_msgTypes[196] + mi := &file_product_proto_msgTypes[201] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -10205,7 +10459,7 @@ func (x *ListOfFloat_List) String() string { func (*ListOfFloat_List) ProtoMessage() {} func (x *ListOfFloat_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[196] + mi := &file_product_proto_msgTypes[201] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -10237,7 +10491,7 @@ type ListOfListOfCategory_List struct { func (x *ListOfListOfCategory_List) Reset() { *x = ListOfListOfCategory_List{} - mi := &file_product_proto_msgTypes[197] + mi := &file_product_proto_msgTypes[202] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -10249,7 +10503,7 @@ func (x *ListOfListOfCategory_List) String() string { func (*ListOfListOfCategory_List) ProtoMessage() {} func (x *ListOfListOfCategory_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[197] + mi := &file_product_proto_msgTypes[202] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -10281,7 +10535,7 @@ type ListOfListOfCategoryInput_List struct { func (x *ListOfListOfCategoryInput_List) Reset() { *x = ListOfListOfCategoryInput_List{} - mi := &file_product_proto_msgTypes[198] + mi := &file_product_proto_msgTypes[203] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -10293,7 +10547,7 @@ func (x *ListOfListOfCategoryInput_List) String() string { func (*ListOfListOfCategoryInput_List) ProtoMessage() {} func (x *ListOfListOfCategoryInput_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[198] + mi := &file_product_proto_msgTypes[203] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -10325,7 +10579,7 @@ type ListOfListOfString_List struct { func (x *ListOfListOfString_List) Reset() { *x = ListOfListOfString_List{} - mi := &file_product_proto_msgTypes[199] + mi := &file_product_proto_msgTypes[204] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -10337,7 +10591,7 @@ func (x *ListOfListOfString_List) String() string { func (*ListOfListOfString_List) ProtoMessage() {} func (x *ListOfListOfString_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[199] + mi := &file_product_proto_msgTypes[204] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -10369,7 +10623,7 @@ type ListOfListOfUser_List struct { func (x *ListOfListOfUser_List) Reset() { *x = ListOfListOfUser_List{} - mi := &file_product_proto_msgTypes[200] + mi := &file_product_proto_msgTypes[205] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -10381,7 +10635,7 @@ func (x *ListOfListOfUser_List) String() string { func (*ListOfListOfUser_List) ProtoMessage() {} func (x *ListOfListOfUser_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[200] + mi := &file_product_proto_msgTypes[205] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -10413,7 +10667,7 @@ type ListOfListOfUserInput_List struct { func (x *ListOfListOfUserInput_List) Reset() { *x = ListOfListOfUserInput_List{} - mi := &file_product_proto_msgTypes[201] + mi := &file_product_proto_msgTypes[206] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -10425,7 +10679,7 @@ func (x *ListOfListOfUserInput_List) String() string { func (*ListOfListOfUserInput_List) ProtoMessage() {} func (x *ListOfListOfUserInput_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[201] + mi := &file_product_proto_msgTypes[206] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -10457,7 +10711,7 @@ type ListOfOrderLine_List struct { func (x *ListOfOrderLine_List) Reset() { *x = ListOfOrderLine_List{} - mi := &file_product_proto_msgTypes[202] + mi := &file_product_proto_msgTypes[207] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -10469,7 +10723,7 @@ func (x *ListOfOrderLine_List) String() string { func (*ListOfOrderLine_List) ProtoMessage() {} func (x *ListOfOrderLine_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[202] + mi := &file_product_proto_msgTypes[207] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -10501,7 +10755,7 @@ type ListOfProduct_List struct { func (x *ListOfProduct_List) Reset() { *x = ListOfProduct_List{} - mi := &file_product_proto_msgTypes[203] + mi := &file_product_proto_msgTypes[208] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -10513,7 +10767,7 @@ func (x *ListOfProduct_List) String() string { func (*ListOfProduct_List) ProtoMessage() {} func (x *ListOfProduct_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[203] + mi := &file_product_proto_msgTypes[208] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -10545,7 +10799,7 @@ type ListOfString_List struct { func (x *ListOfString_List) Reset() { *x = ListOfString_List{} - mi := &file_product_proto_msgTypes[204] + mi := &file_product_proto_msgTypes[209] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -10557,7 +10811,7 @@ func (x *ListOfString_List) String() string { func (*ListOfString_List) ProtoMessage() {} func (x *ListOfString_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[204] + mi := &file_product_proto_msgTypes[209] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -10589,7 +10843,7 @@ type ListOfSubcategory_List struct { func (x *ListOfSubcategory_List) Reset() { *x = ListOfSubcategory_List{} - mi := &file_product_proto_msgTypes[205] + mi := &file_product_proto_msgTypes[210] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -10601,7 +10855,7 @@ func (x *ListOfSubcategory_List) String() string { func (*ListOfSubcategory_List) ProtoMessage() {} func (x *ListOfSubcategory_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[205] + mi := &file_product_proto_msgTypes[210] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -10633,7 +10887,7 @@ type ListOfUser_List struct { func (x *ListOfUser_List) Reset() { *x = ListOfUser_List{} - mi := &file_product_proto_msgTypes[206] + mi := &file_product_proto_msgTypes[211] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -10645,7 +10899,7 @@ func (x *ListOfUser_List) String() string { func (*ListOfUser_List) ProtoMessage() {} func (x *ListOfUser_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[206] + mi := &file_product_proto_msgTypes[211] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -10677,7 +10931,7 @@ type ListOfUserInput_List struct { func (x *ListOfUserInput_List) Reset() { *x = ListOfUserInput_List{} - mi := &file_product_proto_msgTypes[207] + mi := &file_product_proto_msgTypes[212] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -10689,7 +10943,7 @@ func (x *ListOfUserInput_List) String() string { func (*ListOfUserInput_List) ProtoMessage() {} func (x *ListOfUserInput_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[207] + mi := &file_product_proto_msgTypes[212] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -11062,7 +11316,23 @@ const file_product_proto_rawDesc = "" + "\n" + "item_count\x18\x01 \x01(\x05R\titemCount\"k\n" + "#ResolveSubcategoryItemCountResponse\x12D\n" + - "\x06result\x18\x01 \x03(\v2,.productv1.ResolveSubcategoryItemCountResultR\x06result\"C\n" + + "\x06result\x18\x01 \x03(\v2,.productv1.ResolveSubcategoryItemCountResultR\x06result\"G\n" + + ")ResolveCategoryMetricsNormalizedScoreArgs\x12\x1a\n" + + "\bbaseline\x18\x01 \x01(\x01R\bbaseline\"t\n" + + ",ResolveCategoryMetricsNormalizedScoreContext\x12\x0e\n" + + "\x02id\x18\x01 \x01(\tR\x02id\x12\x1e\n" + + "\n" + + "metricType\x18\x02 \x01(\tR\n" + + "metricType\x12\x14\n" + + "\x05value\x18\x03 \x01(\x01R\x05value\"\xd6\x01\n" + + ",ResolveCategoryMetricsNormalizedScoreRequest\x12Q\n" + + "\acontext\x18\x01 \x03(\v27.productv1.ResolveCategoryMetricsNormalizedScoreContextR\acontext\x12S\n" + + "\n" + + "field_args\x18\x02 \x01(\v24.productv1.ResolveCategoryMetricsNormalizedScoreArgsR\tfieldArgs\"X\n" + + "+ResolveCategoryMetricsNormalizedScoreResult\x12)\n" + + "\x10normalized_score\x18\x01 \x01(\x01R\x0fnormalizedScore\"\x7f\n" + + "-ResolveCategoryMetricsNormalizedScoreResponse\x12N\n" + + "\x06result\x18\x01 \x03(\v26.productv1.ResolveCategoryMetricsNormalizedScoreResultR\x06result\"C\n" + "\aProduct\x12\x0e\n" + "\x02id\x18\x01 \x01(\tR\x02id\x12\x12\n" + "\x04name\x18\x02 \x01(\tR\x04name\x12\x14\n" + @@ -11283,7 +11553,7 @@ const file_product_proto_rawDesc = "" + "\x02id\x18\x01 \x01(\tR\x02id\x12\x12\n" + "\x04name\x18\x02 \x01(\tR\x04name\x12>\n" + "\vdescription\x18\x03 \x01(\v2\x1c.google.protobuf.StringValueR\vdescription\x12\x1b\n" + - "\tis_active\x18\x04 \x01(\bR\bisActive\"\x97\x01\n" + + "\tis_active\x18\x04 \x01(\bR\bisActive\"\xd7\x01\n" + "\x0fCategoryMetrics\x12\x0e\n" + "\x02id\x18\x01 \x01(\tR\x02id\x12\x1f\n" + "\vmetric_type\x18\x02 \x01(\tR\n" + @@ -11291,7 +11561,8 @@ const file_product_proto_rawDesc = "" + "\x05value\x18\x03 \x01(\x01R\x05value\x12\x1c\n" + "\ttimestamp\x18\x04 \x01(\tR\ttimestamp\x12\x1f\n" + "\vcategory_id\x18\x05 \x01(\tR\n" + - "categoryId\"^\n" + + "categoryId\x12>\n" + + "\x10related_category\x18\x06 \x01(\v2\x13.productv1.CategoryR\x0frelatedCategory\"^\n" + "\x03Cat\x12\x0e\n" + "\x02id\x18\x01 \x01(\tR\x02id\x12\x12\n" + "\x04name\x18\x02 \x01(\tR\x04name\x12\x12\n" + @@ -11340,7 +11611,7 @@ const file_product_proto_rawDesc = "" + " SHIPPING_DESTINATION_UNSPECIFIED\x10\x00\x12!\n" + "\x1dSHIPPING_DESTINATION_DOMESTIC\x10\x01\x12 \n" + "\x1cSHIPPING_DESTINATION_EXPRESS\x10\x02\x12&\n" + - "\"SHIPPING_DESTINATION_INTERNATIONAL\x10\x032\xe9,\n" + + "\"SHIPPING_DESTINATION_INTERNATIONAL\x10\x032\x88.\n" + "\x0eProductService\x12`\n" + "\x11LookupProductById\x12#.productv1.LookupProductByIdRequest\x1a$.productv1.LookupProductByIdResponse\"\x00\x12`\n" + "\x11LookupStorageById\x12#.productv1.LookupStorageByIdRequest\x1a$.productv1.LookupStorageByIdResponse\"\x00\x12f\n" + @@ -11388,7 +11659,8 @@ const file_product_proto_rawDesc = "" + "\tQueryUser\x12\x1b.productv1.QueryUserRequest\x1a\x1c.productv1.QueryUserResponse\"\x00\x12K\n" + "\n" + "QueryUsers\x12\x1c.productv1.QueryUsersRequest\x1a\x1d.productv1.QueryUsersResponse\"\x00\x12\x87\x01\n" + - "\x1eResolveCategoryCategoryMetrics\x120.productv1.ResolveCategoryCategoryMetricsRequest\x1a1.productv1.ResolveCategoryCategoryMetricsResponse\"\x00\x12\x87\x01\n" + + "\x1eResolveCategoryCategoryMetrics\x120.productv1.ResolveCategoryCategoryMetricsRequest\x1a1.productv1.ResolveCategoryCategoryMetricsResponse\"\x00\x12\x9c\x01\n" + + "%ResolveCategoryMetricsNormalizedScore\x127.productv1.ResolveCategoryMetricsNormalizedScoreRequest\x1a8.productv1.ResolveCategoryMetricsNormalizedScoreResponse\"\x00\x12\x87\x01\n" + "\x1eResolveCategoryPopularityScore\x120.productv1.ResolveCategoryPopularityScoreRequest\x1a1.productv1.ResolveCategoryPopularityScoreResponse\"\x00\x12~\n" + "\x1bResolveCategoryProductCount\x12-.productv1.ResolveCategoryProductCountRequest\x1a..productv1.ResolveCategoryProductCountResponse\"\x00\x12\x90\x01\n" + "!ResolveProductRecommendedCategory\x123.productv1.ResolveProductRecommendedCategoryRequest\x1a4.productv1.ResolveProductRecommendedCategoryResponse\"\x00\x12\x87\x01\n" + @@ -11408,559 +11680,570 @@ func file_product_proto_rawDescGZIP() []byte { } var file_product_proto_enumTypes = make([]protoimpl.EnumInfo, 2) -var file_product_proto_msgTypes = make([]protoimpl.MessageInfo, 208) +var file_product_proto_msgTypes = make([]protoimpl.MessageInfo, 213) var file_product_proto_goTypes = []any{ - (CategoryKind)(0), // 0: productv1.CategoryKind - (ShippingDestination)(0), // 1: productv1.ShippingDestination - (*ListOfAuthorFilter)(nil), // 2: productv1.ListOfAuthorFilter - (*ListOfAuthorInput)(nil), // 3: productv1.ListOfAuthorInput - (*ListOfBlogPost)(nil), // 4: productv1.ListOfBlogPost - (*ListOfBlogPostFilter)(nil), // 5: productv1.ListOfBlogPostFilter - (*ListOfBlogPostInput)(nil), // 6: productv1.ListOfBlogPostInput - (*ListOfBoolean)(nil), // 7: productv1.ListOfBoolean - (*ListOfCategory)(nil), // 8: productv1.ListOfCategory - (*ListOfCategoryInput)(nil), // 9: productv1.ListOfCategoryInput - (*ListOfFloat)(nil), // 10: productv1.ListOfFloat - (*ListOfListOfCategory)(nil), // 11: productv1.ListOfListOfCategory - (*ListOfListOfCategoryInput)(nil), // 12: productv1.ListOfListOfCategoryInput - (*ListOfListOfString)(nil), // 13: productv1.ListOfListOfString - (*ListOfListOfUser)(nil), // 14: productv1.ListOfListOfUser - (*ListOfListOfUserInput)(nil), // 15: productv1.ListOfListOfUserInput - (*ListOfOrderLine)(nil), // 16: productv1.ListOfOrderLine - (*ListOfProduct)(nil), // 17: productv1.ListOfProduct - (*ListOfString)(nil), // 18: productv1.ListOfString - (*ListOfSubcategory)(nil), // 19: productv1.ListOfSubcategory - (*ListOfUser)(nil), // 20: productv1.ListOfUser - (*ListOfUserInput)(nil), // 21: productv1.ListOfUserInput - (*LookupProductByIdRequestKey)(nil), // 22: productv1.LookupProductByIdRequestKey - (*LookupProductByIdRequest)(nil), // 23: productv1.LookupProductByIdRequest - (*LookupProductByIdResponse)(nil), // 24: productv1.LookupProductByIdResponse - (*LookupStorageByIdRequestKey)(nil), // 25: productv1.LookupStorageByIdRequestKey - (*LookupStorageByIdRequest)(nil), // 26: productv1.LookupStorageByIdRequest - (*LookupStorageByIdResponse)(nil), // 27: productv1.LookupStorageByIdResponse - (*LookupWarehouseByIdRequestKey)(nil), // 28: productv1.LookupWarehouseByIdRequestKey - (*LookupWarehouseByIdRequest)(nil), // 29: productv1.LookupWarehouseByIdRequest - (*LookupWarehouseByIdResponse)(nil), // 30: productv1.LookupWarehouseByIdResponse - (*QueryUsersRequest)(nil), // 31: productv1.QueryUsersRequest - (*QueryUsersResponse)(nil), // 32: productv1.QueryUsersResponse - (*QueryUserRequest)(nil), // 33: productv1.QueryUserRequest - (*QueryUserResponse)(nil), // 34: productv1.QueryUserResponse - (*QueryNestedTypeRequest)(nil), // 35: productv1.QueryNestedTypeRequest - (*QueryNestedTypeResponse)(nil), // 36: productv1.QueryNestedTypeResponse - (*QueryRecursiveTypeRequest)(nil), // 37: productv1.QueryRecursiveTypeRequest - (*QueryRecursiveTypeResponse)(nil), // 38: productv1.QueryRecursiveTypeResponse - (*QueryTypeFilterWithArgumentsRequest)(nil), // 39: productv1.QueryTypeFilterWithArgumentsRequest - (*QueryTypeFilterWithArgumentsResponse)(nil), // 40: productv1.QueryTypeFilterWithArgumentsResponse - (*QueryTypeWithMultipleFilterFieldsRequest)(nil), // 41: productv1.QueryTypeWithMultipleFilterFieldsRequest - (*QueryTypeWithMultipleFilterFieldsResponse)(nil), // 42: productv1.QueryTypeWithMultipleFilterFieldsResponse - (*QueryComplexFilterTypeRequest)(nil), // 43: productv1.QueryComplexFilterTypeRequest - (*QueryComplexFilterTypeResponse)(nil), // 44: productv1.QueryComplexFilterTypeResponse - (*QueryCalculateTotalsRequest)(nil), // 45: productv1.QueryCalculateTotalsRequest - (*QueryCalculateTotalsResponse)(nil), // 46: productv1.QueryCalculateTotalsResponse - (*QueryCategoriesRequest)(nil), // 47: productv1.QueryCategoriesRequest - (*QueryCategoriesResponse)(nil), // 48: productv1.QueryCategoriesResponse - (*QueryCategoriesByKindRequest)(nil), // 49: productv1.QueryCategoriesByKindRequest - (*QueryCategoriesByKindResponse)(nil), // 50: productv1.QueryCategoriesByKindResponse - (*QueryCategoriesByKindsRequest)(nil), // 51: productv1.QueryCategoriesByKindsRequest - (*QueryCategoriesByKindsResponse)(nil), // 52: productv1.QueryCategoriesByKindsResponse - (*QueryFilterCategoriesRequest)(nil), // 53: productv1.QueryFilterCategoriesRequest - (*QueryFilterCategoriesResponse)(nil), // 54: productv1.QueryFilterCategoriesResponse - (*QueryRandomPetRequest)(nil), // 55: productv1.QueryRandomPetRequest - (*QueryRandomPetResponse)(nil), // 56: productv1.QueryRandomPetResponse - (*QueryAllPetsRequest)(nil), // 57: productv1.QueryAllPetsRequest - (*QueryAllPetsResponse)(nil), // 58: productv1.QueryAllPetsResponse - (*QuerySearchRequest)(nil), // 59: productv1.QuerySearchRequest - (*QuerySearchResponse)(nil), // 60: productv1.QuerySearchResponse - (*QueryRandomSearchResultRequest)(nil), // 61: productv1.QueryRandomSearchResultRequest - (*QueryRandomSearchResultResponse)(nil), // 62: productv1.QueryRandomSearchResultResponse - (*QueryNullableFieldsTypeRequest)(nil), // 63: productv1.QueryNullableFieldsTypeRequest - (*QueryNullableFieldsTypeResponse)(nil), // 64: productv1.QueryNullableFieldsTypeResponse - (*QueryNullableFieldsTypeByIdRequest)(nil), // 65: productv1.QueryNullableFieldsTypeByIdRequest - (*QueryNullableFieldsTypeByIdResponse)(nil), // 66: productv1.QueryNullableFieldsTypeByIdResponse - (*QueryNullableFieldsTypeWithFilterRequest)(nil), // 67: productv1.QueryNullableFieldsTypeWithFilterRequest - (*QueryNullableFieldsTypeWithFilterResponse)(nil), // 68: productv1.QueryNullableFieldsTypeWithFilterResponse - (*QueryAllNullableFieldsTypesRequest)(nil), // 69: productv1.QueryAllNullableFieldsTypesRequest - (*QueryAllNullableFieldsTypesResponse)(nil), // 70: productv1.QueryAllNullableFieldsTypesResponse - (*QueryBlogPostRequest)(nil), // 71: productv1.QueryBlogPostRequest - (*QueryBlogPostResponse)(nil), // 72: productv1.QueryBlogPostResponse - (*QueryBlogPostByIdRequest)(nil), // 73: productv1.QueryBlogPostByIdRequest - (*QueryBlogPostByIdResponse)(nil), // 74: productv1.QueryBlogPostByIdResponse - (*QueryBlogPostsWithFilterRequest)(nil), // 75: productv1.QueryBlogPostsWithFilterRequest - (*QueryBlogPostsWithFilterResponse)(nil), // 76: productv1.QueryBlogPostsWithFilterResponse - (*QueryAllBlogPostsRequest)(nil), // 77: productv1.QueryAllBlogPostsRequest - (*QueryAllBlogPostsResponse)(nil), // 78: productv1.QueryAllBlogPostsResponse - (*QueryAuthorRequest)(nil), // 79: productv1.QueryAuthorRequest - (*QueryAuthorResponse)(nil), // 80: productv1.QueryAuthorResponse - (*QueryAuthorByIdRequest)(nil), // 81: productv1.QueryAuthorByIdRequest - (*QueryAuthorByIdResponse)(nil), // 82: productv1.QueryAuthorByIdResponse - (*QueryAuthorsWithFilterRequest)(nil), // 83: productv1.QueryAuthorsWithFilterRequest - (*QueryAuthorsWithFilterResponse)(nil), // 84: productv1.QueryAuthorsWithFilterResponse - (*QueryAllAuthorsRequest)(nil), // 85: productv1.QueryAllAuthorsRequest - (*QueryAllAuthorsResponse)(nil), // 86: productv1.QueryAllAuthorsResponse - (*QueryBulkSearchAuthorsRequest)(nil), // 87: productv1.QueryBulkSearchAuthorsRequest - (*QueryBulkSearchAuthorsResponse)(nil), // 88: productv1.QueryBulkSearchAuthorsResponse - (*QueryBulkSearchBlogPostsRequest)(nil), // 89: productv1.QueryBulkSearchBlogPostsRequest - (*QueryBulkSearchBlogPostsResponse)(nil), // 90: productv1.QueryBulkSearchBlogPostsResponse - (*MutationCreateUserRequest)(nil), // 91: productv1.MutationCreateUserRequest - (*MutationCreateUserResponse)(nil), // 92: productv1.MutationCreateUserResponse - (*MutationPerformActionRequest)(nil), // 93: productv1.MutationPerformActionRequest - (*MutationPerformActionResponse)(nil), // 94: productv1.MutationPerformActionResponse - (*MutationCreateNullableFieldsTypeRequest)(nil), // 95: productv1.MutationCreateNullableFieldsTypeRequest - (*MutationCreateNullableFieldsTypeResponse)(nil), // 96: productv1.MutationCreateNullableFieldsTypeResponse - (*MutationUpdateNullableFieldsTypeRequest)(nil), // 97: productv1.MutationUpdateNullableFieldsTypeRequest - (*MutationUpdateNullableFieldsTypeResponse)(nil), // 98: productv1.MutationUpdateNullableFieldsTypeResponse - (*MutationCreateBlogPostRequest)(nil), // 99: productv1.MutationCreateBlogPostRequest - (*MutationCreateBlogPostResponse)(nil), // 100: productv1.MutationCreateBlogPostResponse - (*MutationUpdateBlogPostRequest)(nil), // 101: productv1.MutationUpdateBlogPostRequest - (*MutationUpdateBlogPostResponse)(nil), // 102: productv1.MutationUpdateBlogPostResponse - (*MutationCreateAuthorRequest)(nil), // 103: productv1.MutationCreateAuthorRequest - (*MutationCreateAuthorResponse)(nil), // 104: productv1.MutationCreateAuthorResponse - (*MutationUpdateAuthorRequest)(nil), // 105: productv1.MutationUpdateAuthorRequest - (*MutationUpdateAuthorResponse)(nil), // 106: productv1.MutationUpdateAuthorResponse - (*MutationBulkCreateAuthorsRequest)(nil), // 107: productv1.MutationBulkCreateAuthorsRequest - (*MutationBulkCreateAuthorsResponse)(nil), // 108: productv1.MutationBulkCreateAuthorsResponse - (*MutationBulkUpdateAuthorsRequest)(nil), // 109: productv1.MutationBulkUpdateAuthorsRequest - (*MutationBulkUpdateAuthorsResponse)(nil), // 110: productv1.MutationBulkUpdateAuthorsResponse - (*MutationBulkCreateBlogPostsRequest)(nil), // 111: productv1.MutationBulkCreateBlogPostsRequest - (*MutationBulkCreateBlogPostsResponse)(nil), // 112: productv1.MutationBulkCreateBlogPostsResponse - (*MutationBulkUpdateBlogPostsRequest)(nil), // 113: productv1.MutationBulkUpdateBlogPostsRequest - (*MutationBulkUpdateBlogPostsResponse)(nil), // 114: productv1.MutationBulkUpdateBlogPostsResponse - (*ResolveProductShippingEstimateArgs)(nil), // 115: productv1.ResolveProductShippingEstimateArgs - (*ResolveProductShippingEstimateContext)(nil), // 116: productv1.ResolveProductShippingEstimateContext - (*ResolveProductShippingEstimateRequest)(nil), // 117: productv1.ResolveProductShippingEstimateRequest - (*ResolveProductShippingEstimateResult)(nil), // 118: productv1.ResolveProductShippingEstimateResult - (*ResolveProductShippingEstimateResponse)(nil), // 119: productv1.ResolveProductShippingEstimateResponse - (*ResolveProductRecommendedCategoryArgs)(nil), // 120: productv1.ResolveProductRecommendedCategoryArgs - (*ResolveProductRecommendedCategoryContext)(nil), // 121: productv1.ResolveProductRecommendedCategoryContext - (*ResolveProductRecommendedCategoryRequest)(nil), // 122: productv1.ResolveProductRecommendedCategoryRequest - (*ResolveProductRecommendedCategoryResult)(nil), // 123: productv1.ResolveProductRecommendedCategoryResult - (*ResolveProductRecommendedCategoryResponse)(nil), // 124: productv1.ResolveProductRecommendedCategoryResponse - (*ResolveCategoryProductCountArgs)(nil), // 125: productv1.ResolveCategoryProductCountArgs - (*ResolveCategoryProductCountContext)(nil), // 126: productv1.ResolveCategoryProductCountContext - (*ResolveCategoryProductCountRequest)(nil), // 127: productv1.ResolveCategoryProductCountRequest - (*ResolveCategoryProductCountResult)(nil), // 128: productv1.ResolveCategoryProductCountResult - (*ResolveCategoryProductCountResponse)(nil), // 129: productv1.ResolveCategoryProductCountResponse - (*ResolveCategoryPopularityScoreArgs)(nil), // 130: productv1.ResolveCategoryPopularityScoreArgs - (*ResolveCategoryPopularityScoreContext)(nil), // 131: productv1.ResolveCategoryPopularityScoreContext - (*ResolveCategoryPopularityScoreRequest)(nil), // 132: productv1.ResolveCategoryPopularityScoreRequest - (*ResolveCategoryPopularityScoreResult)(nil), // 133: productv1.ResolveCategoryPopularityScoreResult - (*ResolveCategoryPopularityScoreResponse)(nil), // 134: productv1.ResolveCategoryPopularityScoreResponse - (*ResolveCategoryCategoryMetricsArgs)(nil), // 135: productv1.ResolveCategoryCategoryMetricsArgs - (*ResolveCategoryCategoryMetricsContext)(nil), // 136: productv1.ResolveCategoryCategoryMetricsContext - (*ResolveCategoryCategoryMetricsRequest)(nil), // 137: productv1.ResolveCategoryCategoryMetricsRequest - (*ResolveCategoryCategoryMetricsResult)(nil), // 138: productv1.ResolveCategoryCategoryMetricsResult - (*ResolveCategoryCategoryMetricsResponse)(nil), // 139: productv1.ResolveCategoryCategoryMetricsResponse - (*ResolveSubcategoryItemCountArgs)(nil), // 140: productv1.ResolveSubcategoryItemCountArgs - (*ResolveSubcategoryItemCountContext)(nil), // 141: productv1.ResolveSubcategoryItemCountContext - (*ResolveSubcategoryItemCountRequest)(nil), // 142: productv1.ResolveSubcategoryItemCountRequest - (*ResolveSubcategoryItemCountResult)(nil), // 143: productv1.ResolveSubcategoryItemCountResult - (*ResolveSubcategoryItemCountResponse)(nil), // 144: productv1.ResolveSubcategoryItemCountResponse - (*Product)(nil), // 145: productv1.Product - (*Storage)(nil), // 146: productv1.Storage - (*Warehouse)(nil), // 147: productv1.Warehouse - (*User)(nil), // 148: productv1.User - (*NestedTypeA)(nil), // 149: productv1.NestedTypeA - (*RecursiveType)(nil), // 150: productv1.RecursiveType - (*TypeWithMultipleFilterFields)(nil), // 151: productv1.TypeWithMultipleFilterFields - (*FilterTypeInput)(nil), // 152: productv1.FilterTypeInput - (*ComplexFilterTypeInput)(nil), // 153: productv1.ComplexFilterTypeInput - (*TypeWithComplexFilterInput)(nil), // 154: productv1.TypeWithComplexFilterInput - (*OrderInput)(nil), // 155: productv1.OrderInput - (*Order)(nil), // 156: productv1.Order - (*Category)(nil), // 157: productv1.Category - (*CategoryFilter)(nil), // 158: productv1.CategoryFilter - (*Animal)(nil), // 159: productv1.Animal - (*SearchInput)(nil), // 160: productv1.SearchInput - (*SearchResult)(nil), // 161: productv1.SearchResult - (*NullableFieldsType)(nil), // 162: productv1.NullableFieldsType - (*NullableFieldsFilter)(nil), // 163: productv1.NullableFieldsFilter - (*BlogPost)(nil), // 164: productv1.BlogPost - (*BlogPostFilter)(nil), // 165: productv1.BlogPostFilter - (*Author)(nil), // 166: productv1.Author - (*AuthorFilter)(nil), // 167: productv1.AuthorFilter - (*UserInput)(nil), // 168: productv1.UserInput - (*ActionInput)(nil), // 169: productv1.ActionInput - (*ActionResult)(nil), // 170: productv1.ActionResult - (*NullableFieldsInput)(nil), // 171: productv1.NullableFieldsInput - (*BlogPostInput)(nil), // 172: productv1.BlogPostInput - (*AuthorInput)(nil), // 173: productv1.AuthorInput - (*NestedTypeB)(nil), // 174: productv1.NestedTypeB - (*NestedTypeC)(nil), // 175: productv1.NestedTypeC - (*FilterType)(nil), // 176: productv1.FilterType - (*Pagination)(nil), // 177: productv1.Pagination - (*OrderLineInput)(nil), // 178: productv1.OrderLineInput - (*OrderLine)(nil), // 179: productv1.OrderLine - (*Subcategory)(nil), // 180: productv1.Subcategory - (*CategoryMetrics)(nil), // 181: productv1.CategoryMetrics - (*Cat)(nil), // 182: productv1.Cat - (*Dog)(nil), // 183: productv1.Dog - (*ActionSuccess)(nil), // 184: productv1.ActionSuccess - (*ActionError)(nil), // 185: productv1.ActionError - (*CategoryInput)(nil), // 186: productv1.CategoryInput - (*ProductCountFilter)(nil), // 187: productv1.ProductCountFilter - (*SubcategoryItemFilter)(nil), // 188: productv1.SubcategoryItemFilter - (*ShippingEstimateInput)(nil), // 189: productv1.ShippingEstimateInput - (*ListOfAuthorFilter_List)(nil), // 190: productv1.ListOfAuthorFilter.List - (*ListOfAuthorInput_List)(nil), // 191: productv1.ListOfAuthorInput.List - (*ListOfBlogPost_List)(nil), // 192: productv1.ListOfBlogPost.List - (*ListOfBlogPostFilter_List)(nil), // 193: productv1.ListOfBlogPostFilter.List - (*ListOfBlogPostInput_List)(nil), // 194: productv1.ListOfBlogPostInput.List - (*ListOfBoolean_List)(nil), // 195: productv1.ListOfBoolean.List - (*ListOfCategory_List)(nil), // 196: productv1.ListOfCategory.List - (*ListOfCategoryInput_List)(nil), // 197: productv1.ListOfCategoryInput.List - (*ListOfFloat_List)(nil), // 198: productv1.ListOfFloat.List - (*ListOfListOfCategory_List)(nil), // 199: productv1.ListOfListOfCategory.List - (*ListOfListOfCategoryInput_List)(nil), // 200: productv1.ListOfListOfCategoryInput.List - (*ListOfListOfString_List)(nil), // 201: productv1.ListOfListOfString.List - (*ListOfListOfUser_List)(nil), // 202: productv1.ListOfListOfUser.List - (*ListOfListOfUserInput_List)(nil), // 203: productv1.ListOfListOfUserInput.List - (*ListOfOrderLine_List)(nil), // 204: productv1.ListOfOrderLine.List - (*ListOfProduct_List)(nil), // 205: productv1.ListOfProduct.List - (*ListOfString_List)(nil), // 206: productv1.ListOfString.List - (*ListOfSubcategory_List)(nil), // 207: productv1.ListOfSubcategory.List - (*ListOfUser_List)(nil), // 208: productv1.ListOfUser.List - (*ListOfUserInput_List)(nil), // 209: productv1.ListOfUserInput.List - (*wrapperspb.Int32Value)(nil), // 210: google.protobuf.Int32Value - (*wrapperspb.StringValue)(nil), // 211: google.protobuf.StringValue - (*wrapperspb.DoubleValue)(nil), // 212: google.protobuf.DoubleValue - (*wrapperspb.BoolValue)(nil), // 213: google.protobuf.BoolValue + (CategoryKind)(0), // 0: productv1.CategoryKind + (ShippingDestination)(0), // 1: productv1.ShippingDestination + (*ListOfAuthorFilter)(nil), // 2: productv1.ListOfAuthorFilter + (*ListOfAuthorInput)(nil), // 3: productv1.ListOfAuthorInput + (*ListOfBlogPost)(nil), // 4: productv1.ListOfBlogPost + (*ListOfBlogPostFilter)(nil), // 5: productv1.ListOfBlogPostFilter + (*ListOfBlogPostInput)(nil), // 6: productv1.ListOfBlogPostInput + (*ListOfBoolean)(nil), // 7: productv1.ListOfBoolean + (*ListOfCategory)(nil), // 8: productv1.ListOfCategory + (*ListOfCategoryInput)(nil), // 9: productv1.ListOfCategoryInput + (*ListOfFloat)(nil), // 10: productv1.ListOfFloat + (*ListOfListOfCategory)(nil), // 11: productv1.ListOfListOfCategory + (*ListOfListOfCategoryInput)(nil), // 12: productv1.ListOfListOfCategoryInput + (*ListOfListOfString)(nil), // 13: productv1.ListOfListOfString + (*ListOfListOfUser)(nil), // 14: productv1.ListOfListOfUser + (*ListOfListOfUserInput)(nil), // 15: productv1.ListOfListOfUserInput + (*ListOfOrderLine)(nil), // 16: productv1.ListOfOrderLine + (*ListOfProduct)(nil), // 17: productv1.ListOfProduct + (*ListOfString)(nil), // 18: productv1.ListOfString + (*ListOfSubcategory)(nil), // 19: productv1.ListOfSubcategory + (*ListOfUser)(nil), // 20: productv1.ListOfUser + (*ListOfUserInput)(nil), // 21: productv1.ListOfUserInput + (*LookupProductByIdRequestKey)(nil), // 22: productv1.LookupProductByIdRequestKey + (*LookupProductByIdRequest)(nil), // 23: productv1.LookupProductByIdRequest + (*LookupProductByIdResponse)(nil), // 24: productv1.LookupProductByIdResponse + (*LookupStorageByIdRequestKey)(nil), // 25: productv1.LookupStorageByIdRequestKey + (*LookupStorageByIdRequest)(nil), // 26: productv1.LookupStorageByIdRequest + (*LookupStorageByIdResponse)(nil), // 27: productv1.LookupStorageByIdResponse + (*LookupWarehouseByIdRequestKey)(nil), // 28: productv1.LookupWarehouseByIdRequestKey + (*LookupWarehouseByIdRequest)(nil), // 29: productv1.LookupWarehouseByIdRequest + (*LookupWarehouseByIdResponse)(nil), // 30: productv1.LookupWarehouseByIdResponse + (*QueryUsersRequest)(nil), // 31: productv1.QueryUsersRequest + (*QueryUsersResponse)(nil), // 32: productv1.QueryUsersResponse + (*QueryUserRequest)(nil), // 33: productv1.QueryUserRequest + (*QueryUserResponse)(nil), // 34: productv1.QueryUserResponse + (*QueryNestedTypeRequest)(nil), // 35: productv1.QueryNestedTypeRequest + (*QueryNestedTypeResponse)(nil), // 36: productv1.QueryNestedTypeResponse + (*QueryRecursiveTypeRequest)(nil), // 37: productv1.QueryRecursiveTypeRequest + (*QueryRecursiveTypeResponse)(nil), // 38: productv1.QueryRecursiveTypeResponse + (*QueryTypeFilterWithArgumentsRequest)(nil), // 39: productv1.QueryTypeFilterWithArgumentsRequest + (*QueryTypeFilterWithArgumentsResponse)(nil), // 40: productv1.QueryTypeFilterWithArgumentsResponse + (*QueryTypeWithMultipleFilterFieldsRequest)(nil), // 41: productv1.QueryTypeWithMultipleFilterFieldsRequest + (*QueryTypeWithMultipleFilterFieldsResponse)(nil), // 42: productv1.QueryTypeWithMultipleFilterFieldsResponse + (*QueryComplexFilterTypeRequest)(nil), // 43: productv1.QueryComplexFilterTypeRequest + (*QueryComplexFilterTypeResponse)(nil), // 44: productv1.QueryComplexFilterTypeResponse + (*QueryCalculateTotalsRequest)(nil), // 45: productv1.QueryCalculateTotalsRequest + (*QueryCalculateTotalsResponse)(nil), // 46: productv1.QueryCalculateTotalsResponse + (*QueryCategoriesRequest)(nil), // 47: productv1.QueryCategoriesRequest + (*QueryCategoriesResponse)(nil), // 48: productv1.QueryCategoriesResponse + (*QueryCategoriesByKindRequest)(nil), // 49: productv1.QueryCategoriesByKindRequest + (*QueryCategoriesByKindResponse)(nil), // 50: productv1.QueryCategoriesByKindResponse + (*QueryCategoriesByKindsRequest)(nil), // 51: productv1.QueryCategoriesByKindsRequest + (*QueryCategoriesByKindsResponse)(nil), // 52: productv1.QueryCategoriesByKindsResponse + (*QueryFilterCategoriesRequest)(nil), // 53: productv1.QueryFilterCategoriesRequest + (*QueryFilterCategoriesResponse)(nil), // 54: productv1.QueryFilterCategoriesResponse + (*QueryRandomPetRequest)(nil), // 55: productv1.QueryRandomPetRequest + (*QueryRandomPetResponse)(nil), // 56: productv1.QueryRandomPetResponse + (*QueryAllPetsRequest)(nil), // 57: productv1.QueryAllPetsRequest + (*QueryAllPetsResponse)(nil), // 58: productv1.QueryAllPetsResponse + (*QuerySearchRequest)(nil), // 59: productv1.QuerySearchRequest + (*QuerySearchResponse)(nil), // 60: productv1.QuerySearchResponse + (*QueryRandomSearchResultRequest)(nil), // 61: productv1.QueryRandomSearchResultRequest + (*QueryRandomSearchResultResponse)(nil), // 62: productv1.QueryRandomSearchResultResponse + (*QueryNullableFieldsTypeRequest)(nil), // 63: productv1.QueryNullableFieldsTypeRequest + (*QueryNullableFieldsTypeResponse)(nil), // 64: productv1.QueryNullableFieldsTypeResponse + (*QueryNullableFieldsTypeByIdRequest)(nil), // 65: productv1.QueryNullableFieldsTypeByIdRequest + (*QueryNullableFieldsTypeByIdResponse)(nil), // 66: productv1.QueryNullableFieldsTypeByIdResponse + (*QueryNullableFieldsTypeWithFilterRequest)(nil), // 67: productv1.QueryNullableFieldsTypeWithFilterRequest + (*QueryNullableFieldsTypeWithFilterResponse)(nil), // 68: productv1.QueryNullableFieldsTypeWithFilterResponse + (*QueryAllNullableFieldsTypesRequest)(nil), // 69: productv1.QueryAllNullableFieldsTypesRequest + (*QueryAllNullableFieldsTypesResponse)(nil), // 70: productv1.QueryAllNullableFieldsTypesResponse + (*QueryBlogPostRequest)(nil), // 71: productv1.QueryBlogPostRequest + (*QueryBlogPostResponse)(nil), // 72: productv1.QueryBlogPostResponse + (*QueryBlogPostByIdRequest)(nil), // 73: productv1.QueryBlogPostByIdRequest + (*QueryBlogPostByIdResponse)(nil), // 74: productv1.QueryBlogPostByIdResponse + (*QueryBlogPostsWithFilterRequest)(nil), // 75: productv1.QueryBlogPostsWithFilterRequest + (*QueryBlogPostsWithFilterResponse)(nil), // 76: productv1.QueryBlogPostsWithFilterResponse + (*QueryAllBlogPostsRequest)(nil), // 77: productv1.QueryAllBlogPostsRequest + (*QueryAllBlogPostsResponse)(nil), // 78: productv1.QueryAllBlogPostsResponse + (*QueryAuthorRequest)(nil), // 79: productv1.QueryAuthorRequest + (*QueryAuthorResponse)(nil), // 80: productv1.QueryAuthorResponse + (*QueryAuthorByIdRequest)(nil), // 81: productv1.QueryAuthorByIdRequest + (*QueryAuthorByIdResponse)(nil), // 82: productv1.QueryAuthorByIdResponse + (*QueryAuthorsWithFilterRequest)(nil), // 83: productv1.QueryAuthorsWithFilterRequest + (*QueryAuthorsWithFilterResponse)(nil), // 84: productv1.QueryAuthorsWithFilterResponse + (*QueryAllAuthorsRequest)(nil), // 85: productv1.QueryAllAuthorsRequest + (*QueryAllAuthorsResponse)(nil), // 86: productv1.QueryAllAuthorsResponse + (*QueryBulkSearchAuthorsRequest)(nil), // 87: productv1.QueryBulkSearchAuthorsRequest + (*QueryBulkSearchAuthorsResponse)(nil), // 88: productv1.QueryBulkSearchAuthorsResponse + (*QueryBulkSearchBlogPostsRequest)(nil), // 89: productv1.QueryBulkSearchBlogPostsRequest + (*QueryBulkSearchBlogPostsResponse)(nil), // 90: productv1.QueryBulkSearchBlogPostsResponse + (*MutationCreateUserRequest)(nil), // 91: productv1.MutationCreateUserRequest + (*MutationCreateUserResponse)(nil), // 92: productv1.MutationCreateUserResponse + (*MutationPerformActionRequest)(nil), // 93: productv1.MutationPerformActionRequest + (*MutationPerformActionResponse)(nil), // 94: productv1.MutationPerformActionResponse + (*MutationCreateNullableFieldsTypeRequest)(nil), // 95: productv1.MutationCreateNullableFieldsTypeRequest + (*MutationCreateNullableFieldsTypeResponse)(nil), // 96: productv1.MutationCreateNullableFieldsTypeResponse + (*MutationUpdateNullableFieldsTypeRequest)(nil), // 97: productv1.MutationUpdateNullableFieldsTypeRequest + (*MutationUpdateNullableFieldsTypeResponse)(nil), // 98: productv1.MutationUpdateNullableFieldsTypeResponse + (*MutationCreateBlogPostRequest)(nil), // 99: productv1.MutationCreateBlogPostRequest + (*MutationCreateBlogPostResponse)(nil), // 100: productv1.MutationCreateBlogPostResponse + (*MutationUpdateBlogPostRequest)(nil), // 101: productv1.MutationUpdateBlogPostRequest + (*MutationUpdateBlogPostResponse)(nil), // 102: productv1.MutationUpdateBlogPostResponse + (*MutationCreateAuthorRequest)(nil), // 103: productv1.MutationCreateAuthorRequest + (*MutationCreateAuthorResponse)(nil), // 104: productv1.MutationCreateAuthorResponse + (*MutationUpdateAuthorRequest)(nil), // 105: productv1.MutationUpdateAuthorRequest + (*MutationUpdateAuthorResponse)(nil), // 106: productv1.MutationUpdateAuthorResponse + (*MutationBulkCreateAuthorsRequest)(nil), // 107: productv1.MutationBulkCreateAuthorsRequest + (*MutationBulkCreateAuthorsResponse)(nil), // 108: productv1.MutationBulkCreateAuthorsResponse + (*MutationBulkUpdateAuthorsRequest)(nil), // 109: productv1.MutationBulkUpdateAuthorsRequest + (*MutationBulkUpdateAuthorsResponse)(nil), // 110: productv1.MutationBulkUpdateAuthorsResponse + (*MutationBulkCreateBlogPostsRequest)(nil), // 111: productv1.MutationBulkCreateBlogPostsRequest + (*MutationBulkCreateBlogPostsResponse)(nil), // 112: productv1.MutationBulkCreateBlogPostsResponse + (*MutationBulkUpdateBlogPostsRequest)(nil), // 113: productv1.MutationBulkUpdateBlogPostsRequest + (*MutationBulkUpdateBlogPostsResponse)(nil), // 114: productv1.MutationBulkUpdateBlogPostsResponse + (*ResolveProductShippingEstimateArgs)(nil), // 115: productv1.ResolveProductShippingEstimateArgs + (*ResolveProductShippingEstimateContext)(nil), // 116: productv1.ResolveProductShippingEstimateContext + (*ResolveProductShippingEstimateRequest)(nil), // 117: productv1.ResolveProductShippingEstimateRequest + (*ResolveProductShippingEstimateResult)(nil), // 118: productv1.ResolveProductShippingEstimateResult + (*ResolveProductShippingEstimateResponse)(nil), // 119: productv1.ResolveProductShippingEstimateResponse + (*ResolveProductRecommendedCategoryArgs)(nil), // 120: productv1.ResolveProductRecommendedCategoryArgs + (*ResolveProductRecommendedCategoryContext)(nil), // 121: productv1.ResolveProductRecommendedCategoryContext + (*ResolveProductRecommendedCategoryRequest)(nil), // 122: productv1.ResolveProductRecommendedCategoryRequest + (*ResolveProductRecommendedCategoryResult)(nil), // 123: productv1.ResolveProductRecommendedCategoryResult + (*ResolveProductRecommendedCategoryResponse)(nil), // 124: productv1.ResolveProductRecommendedCategoryResponse + (*ResolveCategoryProductCountArgs)(nil), // 125: productv1.ResolveCategoryProductCountArgs + (*ResolveCategoryProductCountContext)(nil), // 126: productv1.ResolveCategoryProductCountContext + (*ResolveCategoryProductCountRequest)(nil), // 127: productv1.ResolveCategoryProductCountRequest + (*ResolveCategoryProductCountResult)(nil), // 128: productv1.ResolveCategoryProductCountResult + (*ResolveCategoryProductCountResponse)(nil), // 129: productv1.ResolveCategoryProductCountResponse + (*ResolveCategoryPopularityScoreArgs)(nil), // 130: productv1.ResolveCategoryPopularityScoreArgs + (*ResolveCategoryPopularityScoreContext)(nil), // 131: productv1.ResolveCategoryPopularityScoreContext + (*ResolveCategoryPopularityScoreRequest)(nil), // 132: productv1.ResolveCategoryPopularityScoreRequest + (*ResolveCategoryPopularityScoreResult)(nil), // 133: productv1.ResolveCategoryPopularityScoreResult + (*ResolveCategoryPopularityScoreResponse)(nil), // 134: productv1.ResolveCategoryPopularityScoreResponse + (*ResolveCategoryCategoryMetricsArgs)(nil), // 135: productv1.ResolveCategoryCategoryMetricsArgs + (*ResolveCategoryCategoryMetricsContext)(nil), // 136: productv1.ResolveCategoryCategoryMetricsContext + (*ResolveCategoryCategoryMetricsRequest)(nil), // 137: productv1.ResolveCategoryCategoryMetricsRequest + (*ResolveCategoryCategoryMetricsResult)(nil), // 138: productv1.ResolveCategoryCategoryMetricsResult + (*ResolveCategoryCategoryMetricsResponse)(nil), // 139: productv1.ResolveCategoryCategoryMetricsResponse + (*ResolveSubcategoryItemCountArgs)(nil), // 140: productv1.ResolveSubcategoryItemCountArgs + (*ResolveSubcategoryItemCountContext)(nil), // 141: productv1.ResolveSubcategoryItemCountContext + (*ResolveSubcategoryItemCountRequest)(nil), // 142: productv1.ResolveSubcategoryItemCountRequest + (*ResolveSubcategoryItemCountResult)(nil), // 143: productv1.ResolveSubcategoryItemCountResult + (*ResolveSubcategoryItemCountResponse)(nil), // 144: productv1.ResolveSubcategoryItemCountResponse + (*ResolveCategoryMetricsNormalizedScoreArgs)(nil), // 145: productv1.ResolveCategoryMetricsNormalizedScoreArgs + (*ResolveCategoryMetricsNormalizedScoreContext)(nil), // 146: productv1.ResolveCategoryMetricsNormalizedScoreContext + (*ResolveCategoryMetricsNormalizedScoreRequest)(nil), // 147: productv1.ResolveCategoryMetricsNormalizedScoreRequest + (*ResolveCategoryMetricsNormalizedScoreResult)(nil), // 148: productv1.ResolveCategoryMetricsNormalizedScoreResult + (*ResolveCategoryMetricsNormalizedScoreResponse)(nil), // 149: productv1.ResolveCategoryMetricsNormalizedScoreResponse + (*Product)(nil), // 150: productv1.Product + (*Storage)(nil), // 151: productv1.Storage + (*Warehouse)(nil), // 152: productv1.Warehouse + (*User)(nil), // 153: productv1.User + (*NestedTypeA)(nil), // 154: productv1.NestedTypeA + (*RecursiveType)(nil), // 155: productv1.RecursiveType + (*TypeWithMultipleFilterFields)(nil), // 156: productv1.TypeWithMultipleFilterFields + (*FilterTypeInput)(nil), // 157: productv1.FilterTypeInput + (*ComplexFilterTypeInput)(nil), // 158: productv1.ComplexFilterTypeInput + (*TypeWithComplexFilterInput)(nil), // 159: productv1.TypeWithComplexFilterInput + (*OrderInput)(nil), // 160: productv1.OrderInput + (*Order)(nil), // 161: productv1.Order + (*Category)(nil), // 162: productv1.Category + (*CategoryFilter)(nil), // 163: productv1.CategoryFilter + (*Animal)(nil), // 164: productv1.Animal + (*SearchInput)(nil), // 165: productv1.SearchInput + (*SearchResult)(nil), // 166: productv1.SearchResult + (*NullableFieldsType)(nil), // 167: productv1.NullableFieldsType + (*NullableFieldsFilter)(nil), // 168: productv1.NullableFieldsFilter + (*BlogPost)(nil), // 169: productv1.BlogPost + (*BlogPostFilter)(nil), // 170: productv1.BlogPostFilter + (*Author)(nil), // 171: productv1.Author + (*AuthorFilter)(nil), // 172: productv1.AuthorFilter + (*UserInput)(nil), // 173: productv1.UserInput + (*ActionInput)(nil), // 174: productv1.ActionInput + (*ActionResult)(nil), // 175: productv1.ActionResult + (*NullableFieldsInput)(nil), // 176: productv1.NullableFieldsInput + (*BlogPostInput)(nil), // 177: productv1.BlogPostInput + (*AuthorInput)(nil), // 178: productv1.AuthorInput + (*NestedTypeB)(nil), // 179: productv1.NestedTypeB + (*NestedTypeC)(nil), // 180: productv1.NestedTypeC + (*FilterType)(nil), // 181: productv1.FilterType + (*Pagination)(nil), // 182: productv1.Pagination + (*OrderLineInput)(nil), // 183: productv1.OrderLineInput + (*OrderLine)(nil), // 184: productv1.OrderLine + (*Subcategory)(nil), // 185: productv1.Subcategory + (*CategoryMetrics)(nil), // 186: productv1.CategoryMetrics + (*Cat)(nil), // 187: productv1.Cat + (*Dog)(nil), // 188: productv1.Dog + (*ActionSuccess)(nil), // 189: productv1.ActionSuccess + (*ActionError)(nil), // 190: productv1.ActionError + (*CategoryInput)(nil), // 191: productv1.CategoryInput + (*ProductCountFilter)(nil), // 192: productv1.ProductCountFilter + (*SubcategoryItemFilter)(nil), // 193: productv1.SubcategoryItemFilter + (*ShippingEstimateInput)(nil), // 194: productv1.ShippingEstimateInput + (*ListOfAuthorFilter_List)(nil), // 195: productv1.ListOfAuthorFilter.List + (*ListOfAuthorInput_List)(nil), // 196: productv1.ListOfAuthorInput.List + (*ListOfBlogPost_List)(nil), // 197: productv1.ListOfBlogPost.List + (*ListOfBlogPostFilter_List)(nil), // 198: productv1.ListOfBlogPostFilter.List + (*ListOfBlogPostInput_List)(nil), // 199: productv1.ListOfBlogPostInput.List + (*ListOfBoolean_List)(nil), // 200: productv1.ListOfBoolean.List + (*ListOfCategory_List)(nil), // 201: productv1.ListOfCategory.List + (*ListOfCategoryInput_List)(nil), // 202: productv1.ListOfCategoryInput.List + (*ListOfFloat_List)(nil), // 203: productv1.ListOfFloat.List + (*ListOfListOfCategory_List)(nil), // 204: productv1.ListOfListOfCategory.List + (*ListOfListOfCategoryInput_List)(nil), // 205: productv1.ListOfListOfCategoryInput.List + (*ListOfListOfString_List)(nil), // 206: productv1.ListOfListOfString.List + (*ListOfListOfUser_List)(nil), // 207: productv1.ListOfListOfUser.List + (*ListOfListOfUserInput_List)(nil), // 208: productv1.ListOfListOfUserInput.List + (*ListOfOrderLine_List)(nil), // 209: productv1.ListOfOrderLine.List + (*ListOfProduct_List)(nil), // 210: productv1.ListOfProduct.List + (*ListOfString_List)(nil), // 211: productv1.ListOfString.List + (*ListOfSubcategory_List)(nil), // 212: productv1.ListOfSubcategory.List + (*ListOfUser_List)(nil), // 213: productv1.ListOfUser.List + (*ListOfUserInput_List)(nil), // 214: productv1.ListOfUserInput.List + (*wrapperspb.Int32Value)(nil), // 215: google.protobuf.Int32Value + (*wrapperspb.StringValue)(nil), // 216: google.protobuf.StringValue + (*wrapperspb.DoubleValue)(nil), // 217: google.protobuf.DoubleValue + (*wrapperspb.BoolValue)(nil), // 218: google.protobuf.BoolValue } var file_product_proto_depIdxs = []int32{ - 190, // 0: productv1.ListOfAuthorFilter.list:type_name -> productv1.ListOfAuthorFilter.List - 191, // 1: productv1.ListOfAuthorInput.list:type_name -> productv1.ListOfAuthorInput.List - 192, // 2: productv1.ListOfBlogPost.list:type_name -> productv1.ListOfBlogPost.List - 193, // 3: productv1.ListOfBlogPostFilter.list:type_name -> productv1.ListOfBlogPostFilter.List - 194, // 4: productv1.ListOfBlogPostInput.list:type_name -> productv1.ListOfBlogPostInput.List - 195, // 5: productv1.ListOfBoolean.list:type_name -> productv1.ListOfBoolean.List - 196, // 6: productv1.ListOfCategory.list:type_name -> productv1.ListOfCategory.List - 197, // 7: productv1.ListOfCategoryInput.list:type_name -> productv1.ListOfCategoryInput.List - 198, // 8: productv1.ListOfFloat.list:type_name -> productv1.ListOfFloat.List - 199, // 9: productv1.ListOfListOfCategory.list:type_name -> productv1.ListOfListOfCategory.List - 200, // 10: productv1.ListOfListOfCategoryInput.list:type_name -> productv1.ListOfListOfCategoryInput.List - 201, // 11: productv1.ListOfListOfString.list:type_name -> productv1.ListOfListOfString.List - 202, // 12: productv1.ListOfListOfUser.list:type_name -> productv1.ListOfListOfUser.List - 203, // 13: productv1.ListOfListOfUserInput.list:type_name -> productv1.ListOfListOfUserInput.List - 204, // 14: productv1.ListOfOrderLine.list:type_name -> productv1.ListOfOrderLine.List - 205, // 15: productv1.ListOfProduct.list:type_name -> productv1.ListOfProduct.List - 206, // 16: productv1.ListOfString.list:type_name -> productv1.ListOfString.List - 207, // 17: productv1.ListOfSubcategory.list:type_name -> productv1.ListOfSubcategory.List - 208, // 18: productv1.ListOfUser.list:type_name -> productv1.ListOfUser.List - 209, // 19: productv1.ListOfUserInput.list:type_name -> productv1.ListOfUserInput.List + 195, // 0: productv1.ListOfAuthorFilter.list:type_name -> productv1.ListOfAuthorFilter.List + 196, // 1: productv1.ListOfAuthorInput.list:type_name -> productv1.ListOfAuthorInput.List + 197, // 2: productv1.ListOfBlogPost.list:type_name -> productv1.ListOfBlogPost.List + 198, // 3: productv1.ListOfBlogPostFilter.list:type_name -> productv1.ListOfBlogPostFilter.List + 199, // 4: productv1.ListOfBlogPostInput.list:type_name -> productv1.ListOfBlogPostInput.List + 200, // 5: productv1.ListOfBoolean.list:type_name -> productv1.ListOfBoolean.List + 201, // 6: productv1.ListOfCategory.list:type_name -> productv1.ListOfCategory.List + 202, // 7: productv1.ListOfCategoryInput.list:type_name -> productv1.ListOfCategoryInput.List + 203, // 8: productv1.ListOfFloat.list:type_name -> productv1.ListOfFloat.List + 204, // 9: productv1.ListOfListOfCategory.list:type_name -> productv1.ListOfListOfCategory.List + 205, // 10: productv1.ListOfListOfCategoryInput.list:type_name -> productv1.ListOfListOfCategoryInput.List + 206, // 11: productv1.ListOfListOfString.list:type_name -> productv1.ListOfListOfString.List + 207, // 12: productv1.ListOfListOfUser.list:type_name -> productv1.ListOfListOfUser.List + 208, // 13: productv1.ListOfListOfUserInput.list:type_name -> productv1.ListOfListOfUserInput.List + 209, // 14: productv1.ListOfOrderLine.list:type_name -> productv1.ListOfOrderLine.List + 210, // 15: productv1.ListOfProduct.list:type_name -> productv1.ListOfProduct.List + 211, // 16: productv1.ListOfString.list:type_name -> productv1.ListOfString.List + 212, // 17: productv1.ListOfSubcategory.list:type_name -> productv1.ListOfSubcategory.List + 213, // 18: productv1.ListOfUser.list:type_name -> productv1.ListOfUser.List + 214, // 19: productv1.ListOfUserInput.list:type_name -> productv1.ListOfUserInput.List 22, // 20: productv1.LookupProductByIdRequest.keys:type_name -> productv1.LookupProductByIdRequestKey - 145, // 21: productv1.LookupProductByIdResponse.result:type_name -> productv1.Product + 150, // 21: productv1.LookupProductByIdResponse.result:type_name -> productv1.Product 25, // 22: productv1.LookupStorageByIdRequest.keys:type_name -> productv1.LookupStorageByIdRequestKey - 146, // 23: productv1.LookupStorageByIdResponse.result:type_name -> productv1.Storage + 151, // 23: productv1.LookupStorageByIdResponse.result:type_name -> productv1.Storage 28, // 24: productv1.LookupWarehouseByIdRequest.keys:type_name -> productv1.LookupWarehouseByIdRequestKey - 147, // 25: productv1.LookupWarehouseByIdResponse.result:type_name -> productv1.Warehouse - 148, // 26: productv1.QueryUsersResponse.users:type_name -> productv1.User - 148, // 27: productv1.QueryUserResponse.user:type_name -> productv1.User - 149, // 28: productv1.QueryNestedTypeResponse.nested_type:type_name -> productv1.NestedTypeA - 150, // 29: productv1.QueryRecursiveTypeResponse.recursive_type:type_name -> productv1.RecursiveType - 151, // 30: productv1.QueryTypeFilterWithArgumentsResponse.type_filter_with_arguments:type_name -> productv1.TypeWithMultipleFilterFields - 152, // 31: productv1.QueryTypeWithMultipleFilterFieldsRequest.filter:type_name -> productv1.FilterTypeInput - 151, // 32: productv1.QueryTypeWithMultipleFilterFieldsResponse.type_with_multiple_filter_fields:type_name -> productv1.TypeWithMultipleFilterFields - 153, // 33: productv1.QueryComplexFilterTypeRequest.filter:type_name -> productv1.ComplexFilterTypeInput - 154, // 34: productv1.QueryComplexFilterTypeResponse.complex_filter_type:type_name -> productv1.TypeWithComplexFilterInput - 155, // 35: productv1.QueryCalculateTotalsRequest.orders:type_name -> productv1.OrderInput - 156, // 36: productv1.QueryCalculateTotalsResponse.calculate_totals:type_name -> productv1.Order - 157, // 37: productv1.QueryCategoriesResponse.categories:type_name -> productv1.Category + 152, // 25: productv1.LookupWarehouseByIdResponse.result:type_name -> productv1.Warehouse + 153, // 26: productv1.QueryUsersResponse.users:type_name -> productv1.User + 153, // 27: productv1.QueryUserResponse.user:type_name -> productv1.User + 154, // 28: productv1.QueryNestedTypeResponse.nested_type:type_name -> productv1.NestedTypeA + 155, // 29: productv1.QueryRecursiveTypeResponse.recursive_type:type_name -> productv1.RecursiveType + 156, // 30: productv1.QueryTypeFilterWithArgumentsResponse.type_filter_with_arguments:type_name -> productv1.TypeWithMultipleFilterFields + 157, // 31: productv1.QueryTypeWithMultipleFilterFieldsRequest.filter:type_name -> productv1.FilterTypeInput + 156, // 32: productv1.QueryTypeWithMultipleFilterFieldsResponse.type_with_multiple_filter_fields:type_name -> productv1.TypeWithMultipleFilterFields + 158, // 33: productv1.QueryComplexFilterTypeRequest.filter:type_name -> productv1.ComplexFilterTypeInput + 159, // 34: productv1.QueryComplexFilterTypeResponse.complex_filter_type:type_name -> productv1.TypeWithComplexFilterInput + 160, // 35: productv1.QueryCalculateTotalsRequest.orders:type_name -> productv1.OrderInput + 161, // 36: productv1.QueryCalculateTotalsResponse.calculate_totals:type_name -> productv1.Order + 162, // 37: productv1.QueryCategoriesResponse.categories:type_name -> productv1.Category 0, // 38: productv1.QueryCategoriesByKindRequest.kind:type_name -> productv1.CategoryKind - 157, // 39: productv1.QueryCategoriesByKindResponse.categories_by_kind:type_name -> productv1.Category + 162, // 39: productv1.QueryCategoriesByKindResponse.categories_by_kind:type_name -> productv1.Category 0, // 40: productv1.QueryCategoriesByKindsRequest.kinds:type_name -> productv1.CategoryKind - 157, // 41: productv1.QueryCategoriesByKindsResponse.categories_by_kinds:type_name -> productv1.Category - 158, // 42: productv1.QueryFilterCategoriesRequest.filter:type_name -> productv1.CategoryFilter - 157, // 43: productv1.QueryFilterCategoriesResponse.filter_categories:type_name -> productv1.Category - 159, // 44: productv1.QueryRandomPetResponse.random_pet:type_name -> productv1.Animal - 159, // 45: productv1.QueryAllPetsResponse.all_pets:type_name -> productv1.Animal - 160, // 46: productv1.QuerySearchRequest.input:type_name -> productv1.SearchInput - 161, // 47: productv1.QuerySearchResponse.search:type_name -> productv1.SearchResult - 161, // 48: productv1.QueryRandomSearchResultResponse.random_search_result:type_name -> productv1.SearchResult - 162, // 49: productv1.QueryNullableFieldsTypeResponse.nullable_fields_type:type_name -> productv1.NullableFieldsType - 162, // 50: productv1.QueryNullableFieldsTypeByIdResponse.nullable_fields_type_by_id:type_name -> productv1.NullableFieldsType - 163, // 51: productv1.QueryNullableFieldsTypeWithFilterRequest.filter:type_name -> productv1.NullableFieldsFilter - 162, // 52: productv1.QueryNullableFieldsTypeWithFilterResponse.nullable_fields_type_with_filter:type_name -> productv1.NullableFieldsType - 162, // 53: productv1.QueryAllNullableFieldsTypesResponse.all_nullable_fields_types:type_name -> productv1.NullableFieldsType - 164, // 54: productv1.QueryBlogPostResponse.blog_post:type_name -> productv1.BlogPost - 164, // 55: productv1.QueryBlogPostByIdResponse.blog_post_by_id:type_name -> productv1.BlogPost - 165, // 56: productv1.QueryBlogPostsWithFilterRequest.filter:type_name -> productv1.BlogPostFilter - 164, // 57: productv1.QueryBlogPostsWithFilterResponse.blog_posts_with_filter:type_name -> productv1.BlogPost - 164, // 58: productv1.QueryAllBlogPostsResponse.all_blog_posts:type_name -> productv1.BlogPost - 166, // 59: productv1.QueryAuthorResponse.author:type_name -> productv1.Author - 166, // 60: productv1.QueryAuthorByIdResponse.author_by_id:type_name -> productv1.Author - 167, // 61: productv1.QueryAuthorsWithFilterRequest.filter:type_name -> productv1.AuthorFilter - 166, // 62: productv1.QueryAuthorsWithFilterResponse.authors_with_filter:type_name -> productv1.Author - 166, // 63: productv1.QueryAllAuthorsResponse.all_authors:type_name -> productv1.Author + 162, // 41: productv1.QueryCategoriesByKindsResponse.categories_by_kinds:type_name -> productv1.Category + 163, // 42: productv1.QueryFilterCategoriesRequest.filter:type_name -> productv1.CategoryFilter + 162, // 43: productv1.QueryFilterCategoriesResponse.filter_categories:type_name -> productv1.Category + 164, // 44: productv1.QueryRandomPetResponse.random_pet:type_name -> productv1.Animal + 164, // 45: productv1.QueryAllPetsResponse.all_pets:type_name -> productv1.Animal + 165, // 46: productv1.QuerySearchRequest.input:type_name -> productv1.SearchInput + 166, // 47: productv1.QuerySearchResponse.search:type_name -> productv1.SearchResult + 166, // 48: productv1.QueryRandomSearchResultResponse.random_search_result:type_name -> productv1.SearchResult + 167, // 49: productv1.QueryNullableFieldsTypeResponse.nullable_fields_type:type_name -> productv1.NullableFieldsType + 167, // 50: productv1.QueryNullableFieldsTypeByIdResponse.nullable_fields_type_by_id:type_name -> productv1.NullableFieldsType + 168, // 51: productv1.QueryNullableFieldsTypeWithFilterRequest.filter:type_name -> productv1.NullableFieldsFilter + 167, // 52: productv1.QueryNullableFieldsTypeWithFilterResponse.nullable_fields_type_with_filter:type_name -> productv1.NullableFieldsType + 167, // 53: productv1.QueryAllNullableFieldsTypesResponse.all_nullable_fields_types:type_name -> productv1.NullableFieldsType + 169, // 54: productv1.QueryBlogPostResponse.blog_post:type_name -> productv1.BlogPost + 169, // 55: productv1.QueryBlogPostByIdResponse.blog_post_by_id:type_name -> productv1.BlogPost + 170, // 56: productv1.QueryBlogPostsWithFilterRequest.filter:type_name -> productv1.BlogPostFilter + 169, // 57: productv1.QueryBlogPostsWithFilterResponse.blog_posts_with_filter:type_name -> productv1.BlogPost + 169, // 58: productv1.QueryAllBlogPostsResponse.all_blog_posts:type_name -> productv1.BlogPost + 171, // 59: productv1.QueryAuthorResponse.author:type_name -> productv1.Author + 171, // 60: productv1.QueryAuthorByIdResponse.author_by_id:type_name -> productv1.Author + 172, // 61: productv1.QueryAuthorsWithFilterRequest.filter:type_name -> productv1.AuthorFilter + 171, // 62: productv1.QueryAuthorsWithFilterResponse.authors_with_filter:type_name -> productv1.Author + 171, // 63: productv1.QueryAllAuthorsResponse.all_authors:type_name -> productv1.Author 2, // 64: productv1.QueryBulkSearchAuthorsRequest.filters:type_name -> productv1.ListOfAuthorFilter - 166, // 65: productv1.QueryBulkSearchAuthorsResponse.bulk_search_authors:type_name -> productv1.Author + 171, // 65: productv1.QueryBulkSearchAuthorsResponse.bulk_search_authors:type_name -> productv1.Author 5, // 66: productv1.QueryBulkSearchBlogPostsRequest.filters:type_name -> productv1.ListOfBlogPostFilter - 164, // 67: productv1.QueryBulkSearchBlogPostsResponse.bulk_search_blog_posts:type_name -> productv1.BlogPost - 168, // 68: productv1.MutationCreateUserRequest.input:type_name -> productv1.UserInput - 148, // 69: productv1.MutationCreateUserResponse.create_user:type_name -> productv1.User - 169, // 70: productv1.MutationPerformActionRequest.input:type_name -> productv1.ActionInput - 170, // 71: productv1.MutationPerformActionResponse.perform_action:type_name -> productv1.ActionResult - 171, // 72: productv1.MutationCreateNullableFieldsTypeRequest.input:type_name -> productv1.NullableFieldsInput - 162, // 73: productv1.MutationCreateNullableFieldsTypeResponse.create_nullable_fields_type:type_name -> productv1.NullableFieldsType - 171, // 74: productv1.MutationUpdateNullableFieldsTypeRequest.input:type_name -> productv1.NullableFieldsInput - 162, // 75: productv1.MutationUpdateNullableFieldsTypeResponse.update_nullable_fields_type:type_name -> productv1.NullableFieldsType - 172, // 76: productv1.MutationCreateBlogPostRequest.input:type_name -> productv1.BlogPostInput - 164, // 77: productv1.MutationCreateBlogPostResponse.create_blog_post:type_name -> productv1.BlogPost - 172, // 78: productv1.MutationUpdateBlogPostRequest.input:type_name -> productv1.BlogPostInput - 164, // 79: productv1.MutationUpdateBlogPostResponse.update_blog_post:type_name -> productv1.BlogPost - 173, // 80: productv1.MutationCreateAuthorRequest.input:type_name -> productv1.AuthorInput - 166, // 81: productv1.MutationCreateAuthorResponse.create_author:type_name -> productv1.Author - 173, // 82: productv1.MutationUpdateAuthorRequest.input:type_name -> productv1.AuthorInput - 166, // 83: productv1.MutationUpdateAuthorResponse.update_author:type_name -> productv1.Author + 169, // 67: productv1.QueryBulkSearchBlogPostsResponse.bulk_search_blog_posts:type_name -> productv1.BlogPost + 173, // 68: productv1.MutationCreateUserRequest.input:type_name -> productv1.UserInput + 153, // 69: productv1.MutationCreateUserResponse.create_user:type_name -> productv1.User + 174, // 70: productv1.MutationPerformActionRequest.input:type_name -> productv1.ActionInput + 175, // 71: productv1.MutationPerformActionResponse.perform_action:type_name -> productv1.ActionResult + 176, // 72: productv1.MutationCreateNullableFieldsTypeRequest.input:type_name -> productv1.NullableFieldsInput + 167, // 73: productv1.MutationCreateNullableFieldsTypeResponse.create_nullable_fields_type:type_name -> productv1.NullableFieldsType + 176, // 74: productv1.MutationUpdateNullableFieldsTypeRequest.input:type_name -> productv1.NullableFieldsInput + 167, // 75: productv1.MutationUpdateNullableFieldsTypeResponse.update_nullable_fields_type:type_name -> productv1.NullableFieldsType + 177, // 76: productv1.MutationCreateBlogPostRequest.input:type_name -> productv1.BlogPostInput + 169, // 77: productv1.MutationCreateBlogPostResponse.create_blog_post:type_name -> productv1.BlogPost + 177, // 78: productv1.MutationUpdateBlogPostRequest.input:type_name -> productv1.BlogPostInput + 169, // 79: productv1.MutationUpdateBlogPostResponse.update_blog_post:type_name -> productv1.BlogPost + 178, // 80: productv1.MutationCreateAuthorRequest.input:type_name -> productv1.AuthorInput + 171, // 81: productv1.MutationCreateAuthorResponse.create_author:type_name -> productv1.Author + 178, // 82: productv1.MutationUpdateAuthorRequest.input:type_name -> productv1.AuthorInput + 171, // 83: productv1.MutationUpdateAuthorResponse.update_author:type_name -> productv1.Author 3, // 84: productv1.MutationBulkCreateAuthorsRequest.authors:type_name -> productv1.ListOfAuthorInput - 166, // 85: productv1.MutationBulkCreateAuthorsResponse.bulk_create_authors:type_name -> productv1.Author + 171, // 85: productv1.MutationBulkCreateAuthorsResponse.bulk_create_authors:type_name -> productv1.Author 3, // 86: productv1.MutationBulkUpdateAuthorsRequest.authors:type_name -> productv1.ListOfAuthorInput - 166, // 87: productv1.MutationBulkUpdateAuthorsResponse.bulk_update_authors:type_name -> productv1.Author + 171, // 87: productv1.MutationBulkUpdateAuthorsResponse.bulk_update_authors:type_name -> productv1.Author 6, // 88: productv1.MutationBulkCreateBlogPostsRequest.blog_posts:type_name -> productv1.ListOfBlogPostInput - 164, // 89: productv1.MutationBulkCreateBlogPostsResponse.bulk_create_blog_posts:type_name -> productv1.BlogPost + 169, // 89: productv1.MutationBulkCreateBlogPostsResponse.bulk_create_blog_posts:type_name -> productv1.BlogPost 6, // 90: productv1.MutationBulkUpdateBlogPostsRequest.blog_posts:type_name -> productv1.ListOfBlogPostInput - 164, // 91: productv1.MutationBulkUpdateBlogPostsResponse.bulk_update_blog_posts:type_name -> productv1.BlogPost - 189, // 92: productv1.ResolveProductShippingEstimateArgs.input:type_name -> productv1.ShippingEstimateInput + 169, // 91: productv1.MutationBulkUpdateBlogPostsResponse.bulk_update_blog_posts:type_name -> productv1.BlogPost + 194, // 92: productv1.ResolveProductShippingEstimateArgs.input:type_name -> productv1.ShippingEstimateInput 116, // 93: productv1.ResolveProductShippingEstimateRequest.context:type_name -> productv1.ResolveProductShippingEstimateContext 115, // 94: productv1.ResolveProductShippingEstimateRequest.field_args:type_name -> productv1.ResolveProductShippingEstimateArgs 118, // 95: productv1.ResolveProductShippingEstimateResponse.result:type_name -> productv1.ResolveProductShippingEstimateResult 121, // 96: productv1.ResolveProductRecommendedCategoryRequest.context:type_name -> productv1.ResolveProductRecommendedCategoryContext 120, // 97: productv1.ResolveProductRecommendedCategoryRequest.field_args:type_name -> productv1.ResolveProductRecommendedCategoryArgs - 157, // 98: productv1.ResolveProductRecommendedCategoryResult.recommended_category:type_name -> productv1.Category + 162, // 98: productv1.ResolveProductRecommendedCategoryResult.recommended_category:type_name -> productv1.Category 123, // 99: productv1.ResolveProductRecommendedCategoryResponse.result:type_name -> productv1.ResolveProductRecommendedCategoryResult - 187, // 100: productv1.ResolveCategoryProductCountArgs.filters:type_name -> productv1.ProductCountFilter + 192, // 100: productv1.ResolveCategoryProductCountArgs.filters:type_name -> productv1.ProductCountFilter 126, // 101: productv1.ResolveCategoryProductCountRequest.context:type_name -> productv1.ResolveCategoryProductCountContext 125, // 102: productv1.ResolveCategoryProductCountRequest.field_args:type_name -> productv1.ResolveCategoryProductCountArgs 128, // 103: productv1.ResolveCategoryProductCountResponse.result:type_name -> productv1.ResolveCategoryProductCountResult - 210, // 104: productv1.ResolveCategoryPopularityScoreArgs.threshold:type_name -> google.protobuf.Int32Value + 215, // 104: productv1.ResolveCategoryPopularityScoreArgs.threshold:type_name -> google.protobuf.Int32Value 131, // 105: productv1.ResolveCategoryPopularityScoreRequest.context:type_name -> productv1.ResolveCategoryPopularityScoreContext 130, // 106: productv1.ResolveCategoryPopularityScoreRequest.field_args:type_name -> productv1.ResolveCategoryPopularityScoreArgs - 210, // 107: productv1.ResolveCategoryPopularityScoreResult.popularity_score:type_name -> google.protobuf.Int32Value + 215, // 107: productv1.ResolveCategoryPopularityScoreResult.popularity_score:type_name -> google.protobuf.Int32Value 133, // 108: productv1.ResolveCategoryPopularityScoreResponse.result:type_name -> productv1.ResolveCategoryPopularityScoreResult 136, // 109: productv1.ResolveCategoryCategoryMetricsRequest.context:type_name -> productv1.ResolveCategoryCategoryMetricsContext 135, // 110: productv1.ResolveCategoryCategoryMetricsRequest.field_args:type_name -> productv1.ResolveCategoryCategoryMetricsArgs - 181, // 111: productv1.ResolveCategoryCategoryMetricsResult.category_metrics:type_name -> productv1.CategoryMetrics + 186, // 111: productv1.ResolveCategoryCategoryMetricsResult.category_metrics:type_name -> productv1.CategoryMetrics 138, // 112: productv1.ResolveCategoryCategoryMetricsResponse.result:type_name -> productv1.ResolveCategoryCategoryMetricsResult - 188, // 113: productv1.ResolveSubcategoryItemCountArgs.filters:type_name -> productv1.SubcategoryItemFilter + 193, // 113: productv1.ResolveSubcategoryItemCountArgs.filters:type_name -> productv1.SubcategoryItemFilter 141, // 114: productv1.ResolveSubcategoryItemCountRequest.context:type_name -> productv1.ResolveSubcategoryItemCountContext 140, // 115: productv1.ResolveSubcategoryItemCountRequest.field_args:type_name -> productv1.ResolveSubcategoryItemCountArgs 143, // 116: productv1.ResolveSubcategoryItemCountResponse.result:type_name -> productv1.ResolveSubcategoryItemCountResult - 174, // 117: productv1.NestedTypeA.b:type_name -> productv1.NestedTypeB - 150, // 118: productv1.RecursiveType.recursive_type:type_name -> productv1.RecursiveType - 176, // 119: productv1.ComplexFilterTypeInput.filter:type_name -> productv1.FilterType - 178, // 120: productv1.OrderInput.lines:type_name -> productv1.OrderLineInput - 16, // 121: productv1.Order.order_lines:type_name -> productv1.ListOfOrderLine - 0, // 122: productv1.Category.kind:type_name -> productv1.CategoryKind - 19, // 123: productv1.Category.subcategories:type_name -> productv1.ListOfSubcategory - 0, // 124: productv1.CategoryFilter.category:type_name -> productv1.CategoryKind - 177, // 125: productv1.CategoryFilter.pagination:type_name -> productv1.Pagination - 182, // 126: productv1.Animal.cat:type_name -> productv1.Cat - 183, // 127: productv1.Animal.dog:type_name -> productv1.Dog - 210, // 128: productv1.SearchInput.limit:type_name -> google.protobuf.Int32Value - 145, // 129: productv1.SearchResult.product:type_name -> productv1.Product - 148, // 130: productv1.SearchResult.user:type_name -> productv1.User - 157, // 131: productv1.SearchResult.category:type_name -> productv1.Category - 211, // 132: productv1.NullableFieldsType.optional_string:type_name -> google.protobuf.StringValue - 210, // 133: productv1.NullableFieldsType.optional_int:type_name -> google.protobuf.Int32Value - 212, // 134: productv1.NullableFieldsType.optional_float:type_name -> google.protobuf.DoubleValue - 213, // 135: productv1.NullableFieldsType.optional_boolean:type_name -> google.protobuf.BoolValue - 211, // 136: productv1.NullableFieldsFilter.name:type_name -> google.protobuf.StringValue - 211, // 137: productv1.NullableFieldsFilter.optional_string:type_name -> google.protobuf.StringValue - 213, // 138: productv1.NullableFieldsFilter.include_nulls:type_name -> google.protobuf.BoolValue - 18, // 139: productv1.BlogPost.optional_tags:type_name -> productv1.ListOfString - 18, // 140: productv1.BlogPost.keywords:type_name -> productv1.ListOfString - 10, // 141: productv1.BlogPost.ratings:type_name -> productv1.ListOfFloat - 7, // 142: productv1.BlogPost.is_published:type_name -> productv1.ListOfBoolean - 13, // 143: productv1.BlogPost.tag_groups:type_name -> productv1.ListOfListOfString - 13, // 144: productv1.BlogPost.related_topics:type_name -> productv1.ListOfListOfString - 13, // 145: productv1.BlogPost.comment_threads:type_name -> productv1.ListOfListOfString - 13, // 146: productv1.BlogPost.suggestions:type_name -> productv1.ListOfListOfString - 157, // 147: productv1.BlogPost.related_categories:type_name -> productv1.Category - 148, // 148: productv1.BlogPost.contributors:type_name -> productv1.User - 17, // 149: productv1.BlogPost.mentioned_products:type_name -> productv1.ListOfProduct - 20, // 150: productv1.BlogPost.mentioned_users:type_name -> productv1.ListOfUser - 11, // 151: productv1.BlogPost.category_groups:type_name -> productv1.ListOfListOfCategory - 14, // 152: productv1.BlogPost.contributor_teams:type_name -> productv1.ListOfListOfUser - 211, // 153: productv1.BlogPostFilter.title:type_name -> google.protobuf.StringValue - 213, // 154: productv1.BlogPostFilter.has_categories:type_name -> google.protobuf.BoolValue - 210, // 155: productv1.BlogPostFilter.min_tags:type_name -> google.protobuf.Int32Value - 211, // 156: productv1.Author.email:type_name -> google.protobuf.StringValue - 18, // 157: productv1.Author.social_links:type_name -> productv1.ListOfString - 13, // 158: productv1.Author.teams_by_project:type_name -> productv1.ListOfListOfString - 13, // 159: productv1.Author.collaborations:type_name -> productv1.ListOfListOfString - 4, // 160: productv1.Author.written_posts:type_name -> productv1.ListOfBlogPost - 157, // 161: productv1.Author.favorite_categories:type_name -> productv1.Category - 20, // 162: productv1.Author.related_authors:type_name -> productv1.ListOfUser - 17, // 163: productv1.Author.product_reviews:type_name -> productv1.ListOfProduct - 14, // 164: productv1.Author.author_groups:type_name -> productv1.ListOfListOfUser - 11, // 165: productv1.Author.category_preferences:type_name -> productv1.ListOfListOfCategory - 14, // 166: productv1.Author.project_teams:type_name -> productv1.ListOfListOfUser - 211, // 167: productv1.AuthorFilter.name:type_name -> google.protobuf.StringValue - 213, // 168: productv1.AuthorFilter.has_teams:type_name -> google.protobuf.BoolValue - 210, // 169: productv1.AuthorFilter.skill_count:type_name -> google.protobuf.Int32Value - 184, // 170: productv1.ActionResult.action_success:type_name -> productv1.ActionSuccess - 185, // 171: productv1.ActionResult.action_error:type_name -> productv1.ActionError - 211, // 172: productv1.NullableFieldsInput.optional_string:type_name -> google.protobuf.StringValue - 210, // 173: productv1.NullableFieldsInput.optional_int:type_name -> google.protobuf.Int32Value - 212, // 174: productv1.NullableFieldsInput.optional_float:type_name -> google.protobuf.DoubleValue - 213, // 175: productv1.NullableFieldsInput.optional_boolean:type_name -> google.protobuf.BoolValue - 18, // 176: productv1.BlogPostInput.optional_tags:type_name -> productv1.ListOfString - 18, // 177: productv1.BlogPostInput.keywords:type_name -> productv1.ListOfString - 10, // 178: productv1.BlogPostInput.ratings:type_name -> productv1.ListOfFloat - 7, // 179: productv1.BlogPostInput.is_published:type_name -> productv1.ListOfBoolean - 13, // 180: productv1.BlogPostInput.tag_groups:type_name -> productv1.ListOfListOfString - 13, // 181: productv1.BlogPostInput.related_topics:type_name -> productv1.ListOfListOfString - 13, // 182: productv1.BlogPostInput.comment_threads:type_name -> productv1.ListOfListOfString - 13, // 183: productv1.BlogPostInput.suggestions:type_name -> productv1.ListOfListOfString - 9, // 184: productv1.BlogPostInput.related_categories:type_name -> productv1.ListOfCategoryInput - 21, // 185: productv1.BlogPostInput.contributors:type_name -> productv1.ListOfUserInput - 12, // 186: productv1.BlogPostInput.category_groups:type_name -> productv1.ListOfListOfCategoryInput - 211, // 187: productv1.AuthorInput.email:type_name -> google.protobuf.StringValue - 18, // 188: productv1.AuthorInput.social_links:type_name -> productv1.ListOfString - 13, // 189: productv1.AuthorInput.teams_by_project:type_name -> productv1.ListOfListOfString - 13, // 190: productv1.AuthorInput.collaborations:type_name -> productv1.ListOfListOfString - 186, // 191: productv1.AuthorInput.favorite_categories:type_name -> productv1.CategoryInput - 15, // 192: productv1.AuthorInput.author_groups:type_name -> productv1.ListOfListOfUserInput - 15, // 193: productv1.AuthorInput.project_teams:type_name -> productv1.ListOfListOfUserInput - 175, // 194: productv1.NestedTypeB.c:type_name -> productv1.NestedTypeC - 177, // 195: productv1.FilterType.pagination:type_name -> productv1.Pagination - 18, // 196: productv1.OrderLineInput.modifiers:type_name -> productv1.ListOfString - 18, // 197: productv1.OrderLine.modifiers:type_name -> productv1.ListOfString - 211, // 198: productv1.Subcategory.description:type_name -> google.protobuf.StringValue - 0, // 199: productv1.CategoryInput.kind:type_name -> productv1.CategoryKind - 212, // 200: productv1.ProductCountFilter.min_price:type_name -> google.protobuf.DoubleValue - 212, // 201: productv1.ProductCountFilter.max_price:type_name -> google.protobuf.DoubleValue - 213, // 202: productv1.ProductCountFilter.in_stock:type_name -> google.protobuf.BoolValue - 211, // 203: productv1.ProductCountFilter.search_term:type_name -> google.protobuf.StringValue - 212, // 204: productv1.SubcategoryItemFilter.min_price:type_name -> google.protobuf.DoubleValue - 212, // 205: productv1.SubcategoryItemFilter.max_price:type_name -> google.protobuf.DoubleValue - 213, // 206: productv1.SubcategoryItemFilter.in_stock:type_name -> google.protobuf.BoolValue - 213, // 207: productv1.SubcategoryItemFilter.is_active:type_name -> google.protobuf.BoolValue - 211, // 208: productv1.SubcategoryItemFilter.search_term:type_name -> google.protobuf.StringValue - 1, // 209: productv1.ShippingEstimateInput.destination:type_name -> productv1.ShippingDestination - 213, // 210: productv1.ShippingEstimateInput.expedited:type_name -> google.protobuf.BoolValue - 167, // 211: productv1.ListOfAuthorFilter.List.items:type_name -> productv1.AuthorFilter - 173, // 212: productv1.ListOfAuthorInput.List.items:type_name -> productv1.AuthorInput - 164, // 213: productv1.ListOfBlogPost.List.items:type_name -> productv1.BlogPost - 165, // 214: productv1.ListOfBlogPostFilter.List.items:type_name -> productv1.BlogPostFilter - 172, // 215: productv1.ListOfBlogPostInput.List.items:type_name -> productv1.BlogPostInput - 157, // 216: productv1.ListOfCategory.List.items:type_name -> productv1.Category - 186, // 217: productv1.ListOfCategoryInput.List.items:type_name -> productv1.CategoryInput - 8, // 218: productv1.ListOfListOfCategory.List.items:type_name -> productv1.ListOfCategory - 9, // 219: productv1.ListOfListOfCategoryInput.List.items:type_name -> productv1.ListOfCategoryInput - 18, // 220: productv1.ListOfListOfString.List.items:type_name -> productv1.ListOfString - 20, // 221: productv1.ListOfListOfUser.List.items:type_name -> productv1.ListOfUser - 21, // 222: productv1.ListOfListOfUserInput.List.items:type_name -> productv1.ListOfUserInput - 179, // 223: productv1.ListOfOrderLine.List.items:type_name -> productv1.OrderLine - 145, // 224: productv1.ListOfProduct.List.items:type_name -> productv1.Product - 180, // 225: productv1.ListOfSubcategory.List.items:type_name -> productv1.Subcategory - 148, // 226: productv1.ListOfUser.List.items:type_name -> productv1.User - 168, // 227: productv1.ListOfUserInput.List.items:type_name -> productv1.UserInput - 23, // 228: productv1.ProductService.LookupProductById:input_type -> productv1.LookupProductByIdRequest - 26, // 229: productv1.ProductService.LookupStorageById:input_type -> productv1.LookupStorageByIdRequest - 29, // 230: productv1.ProductService.LookupWarehouseById:input_type -> productv1.LookupWarehouseByIdRequest - 107, // 231: productv1.ProductService.MutationBulkCreateAuthors:input_type -> productv1.MutationBulkCreateAuthorsRequest - 111, // 232: productv1.ProductService.MutationBulkCreateBlogPosts:input_type -> productv1.MutationBulkCreateBlogPostsRequest - 109, // 233: productv1.ProductService.MutationBulkUpdateAuthors:input_type -> productv1.MutationBulkUpdateAuthorsRequest - 113, // 234: productv1.ProductService.MutationBulkUpdateBlogPosts:input_type -> productv1.MutationBulkUpdateBlogPostsRequest - 103, // 235: productv1.ProductService.MutationCreateAuthor:input_type -> productv1.MutationCreateAuthorRequest - 99, // 236: productv1.ProductService.MutationCreateBlogPost:input_type -> productv1.MutationCreateBlogPostRequest - 95, // 237: productv1.ProductService.MutationCreateNullableFieldsType:input_type -> productv1.MutationCreateNullableFieldsTypeRequest - 91, // 238: productv1.ProductService.MutationCreateUser:input_type -> productv1.MutationCreateUserRequest - 93, // 239: productv1.ProductService.MutationPerformAction:input_type -> productv1.MutationPerformActionRequest - 105, // 240: productv1.ProductService.MutationUpdateAuthor:input_type -> productv1.MutationUpdateAuthorRequest - 101, // 241: productv1.ProductService.MutationUpdateBlogPost:input_type -> productv1.MutationUpdateBlogPostRequest - 97, // 242: productv1.ProductService.MutationUpdateNullableFieldsType:input_type -> productv1.MutationUpdateNullableFieldsTypeRequest - 85, // 243: productv1.ProductService.QueryAllAuthors:input_type -> productv1.QueryAllAuthorsRequest - 77, // 244: productv1.ProductService.QueryAllBlogPosts:input_type -> productv1.QueryAllBlogPostsRequest - 69, // 245: productv1.ProductService.QueryAllNullableFieldsTypes:input_type -> productv1.QueryAllNullableFieldsTypesRequest - 57, // 246: productv1.ProductService.QueryAllPets:input_type -> productv1.QueryAllPetsRequest - 79, // 247: productv1.ProductService.QueryAuthor:input_type -> productv1.QueryAuthorRequest - 81, // 248: productv1.ProductService.QueryAuthorById:input_type -> productv1.QueryAuthorByIdRequest - 83, // 249: productv1.ProductService.QueryAuthorsWithFilter:input_type -> productv1.QueryAuthorsWithFilterRequest - 71, // 250: productv1.ProductService.QueryBlogPost:input_type -> productv1.QueryBlogPostRequest - 73, // 251: productv1.ProductService.QueryBlogPostById:input_type -> productv1.QueryBlogPostByIdRequest - 75, // 252: productv1.ProductService.QueryBlogPostsWithFilter:input_type -> productv1.QueryBlogPostsWithFilterRequest - 87, // 253: productv1.ProductService.QueryBulkSearchAuthors:input_type -> productv1.QueryBulkSearchAuthorsRequest - 89, // 254: productv1.ProductService.QueryBulkSearchBlogPosts:input_type -> productv1.QueryBulkSearchBlogPostsRequest - 45, // 255: productv1.ProductService.QueryCalculateTotals:input_type -> productv1.QueryCalculateTotalsRequest - 47, // 256: productv1.ProductService.QueryCategories:input_type -> productv1.QueryCategoriesRequest - 49, // 257: productv1.ProductService.QueryCategoriesByKind:input_type -> productv1.QueryCategoriesByKindRequest - 51, // 258: productv1.ProductService.QueryCategoriesByKinds:input_type -> productv1.QueryCategoriesByKindsRequest - 43, // 259: productv1.ProductService.QueryComplexFilterType:input_type -> productv1.QueryComplexFilterTypeRequest - 53, // 260: productv1.ProductService.QueryFilterCategories:input_type -> productv1.QueryFilterCategoriesRequest - 35, // 261: productv1.ProductService.QueryNestedType:input_type -> productv1.QueryNestedTypeRequest - 63, // 262: productv1.ProductService.QueryNullableFieldsType:input_type -> productv1.QueryNullableFieldsTypeRequest - 65, // 263: productv1.ProductService.QueryNullableFieldsTypeById:input_type -> productv1.QueryNullableFieldsTypeByIdRequest - 67, // 264: productv1.ProductService.QueryNullableFieldsTypeWithFilter:input_type -> productv1.QueryNullableFieldsTypeWithFilterRequest - 55, // 265: productv1.ProductService.QueryRandomPet:input_type -> productv1.QueryRandomPetRequest - 61, // 266: productv1.ProductService.QueryRandomSearchResult:input_type -> productv1.QueryRandomSearchResultRequest - 37, // 267: productv1.ProductService.QueryRecursiveType:input_type -> productv1.QueryRecursiveTypeRequest - 59, // 268: productv1.ProductService.QuerySearch:input_type -> productv1.QuerySearchRequest - 39, // 269: productv1.ProductService.QueryTypeFilterWithArguments:input_type -> productv1.QueryTypeFilterWithArgumentsRequest - 41, // 270: productv1.ProductService.QueryTypeWithMultipleFilterFields:input_type -> productv1.QueryTypeWithMultipleFilterFieldsRequest - 33, // 271: productv1.ProductService.QueryUser:input_type -> productv1.QueryUserRequest - 31, // 272: productv1.ProductService.QueryUsers:input_type -> productv1.QueryUsersRequest - 137, // 273: productv1.ProductService.ResolveCategoryCategoryMetrics:input_type -> productv1.ResolveCategoryCategoryMetricsRequest - 132, // 274: productv1.ProductService.ResolveCategoryPopularityScore:input_type -> productv1.ResolveCategoryPopularityScoreRequest - 127, // 275: productv1.ProductService.ResolveCategoryProductCount:input_type -> productv1.ResolveCategoryProductCountRequest - 122, // 276: productv1.ProductService.ResolveProductRecommendedCategory:input_type -> productv1.ResolveProductRecommendedCategoryRequest - 117, // 277: productv1.ProductService.ResolveProductShippingEstimate:input_type -> productv1.ResolveProductShippingEstimateRequest - 142, // 278: productv1.ProductService.ResolveSubcategoryItemCount:input_type -> productv1.ResolveSubcategoryItemCountRequest - 24, // 279: productv1.ProductService.LookupProductById:output_type -> productv1.LookupProductByIdResponse - 27, // 280: productv1.ProductService.LookupStorageById:output_type -> productv1.LookupStorageByIdResponse - 30, // 281: productv1.ProductService.LookupWarehouseById:output_type -> productv1.LookupWarehouseByIdResponse - 108, // 282: productv1.ProductService.MutationBulkCreateAuthors:output_type -> productv1.MutationBulkCreateAuthorsResponse - 112, // 283: productv1.ProductService.MutationBulkCreateBlogPosts:output_type -> productv1.MutationBulkCreateBlogPostsResponse - 110, // 284: productv1.ProductService.MutationBulkUpdateAuthors:output_type -> productv1.MutationBulkUpdateAuthorsResponse - 114, // 285: productv1.ProductService.MutationBulkUpdateBlogPosts:output_type -> productv1.MutationBulkUpdateBlogPostsResponse - 104, // 286: productv1.ProductService.MutationCreateAuthor:output_type -> productv1.MutationCreateAuthorResponse - 100, // 287: productv1.ProductService.MutationCreateBlogPost:output_type -> productv1.MutationCreateBlogPostResponse - 96, // 288: productv1.ProductService.MutationCreateNullableFieldsType:output_type -> productv1.MutationCreateNullableFieldsTypeResponse - 92, // 289: productv1.ProductService.MutationCreateUser:output_type -> productv1.MutationCreateUserResponse - 94, // 290: productv1.ProductService.MutationPerformAction:output_type -> productv1.MutationPerformActionResponse - 106, // 291: productv1.ProductService.MutationUpdateAuthor:output_type -> productv1.MutationUpdateAuthorResponse - 102, // 292: productv1.ProductService.MutationUpdateBlogPost:output_type -> productv1.MutationUpdateBlogPostResponse - 98, // 293: productv1.ProductService.MutationUpdateNullableFieldsType:output_type -> productv1.MutationUpdateNullableFieldsTypeResponse - 86, // 294: productv1.ProductService.QueryAllAuthors:output_type -> productv1.QueryAllAuthorsResponse - 78, // 295: productv1.ProductService.QueryAllBlogPosts:output_type -> productv1.QueryAllBlogPostsResponse - 70, // 296: productv1.ProductService.QueryAllNullableFieldsTypes:output_type -> productv1.QueryAllNullableFieldsTypesResponse - 58, // 297: productv1.ProductService.QueryAllPets:output_type -> productv1.QueryAllPetsResponse - 80, // 298: productv1.ProductService.QueryAuthor:output_type -> productv1.QueryAuthorResponse - 82, // 299: productv1.ProductService.QueryAuthorById:output_type -> productv1.QueryAuthorByIdResponse - 84, // 300: productv1.ProductService.QueryAuthorsWithFilter:output_type -> productv1.QueryAuthorsWithFilterResponse - 72, // 301: productv1.ProductService.QueryBlogPost:output_type -> productv1.QueryBlogPostResponse - 74, // 302: productv1.ProductService.QueryBlogPostById:output_type -> productv1.QueryBlogPostByIdResponse - 76, // 303: productv1.ProductService.QueryBlogPostsWithFilter:output_type -> productv1.QueryBlogPostsWithFilterResponse - 88, // 304: productv1.ProductService.QueryBulkSearchAuthors:output_type -> productv1.QueryBulkSearchAuthorsResponse - 90, // 305: productv1.ProductService.QueryBulkSearchBlogPosts:output_type -> productv1.QueryBulkSearchBlogPostsResponse - 46, // 306: productv1.ProductService.QueryCalculateTotals:output_type -> productv1.QueryCalculateTotalsResponse - 48, // 307: productv1.ProductService.QueryCategories:output_type -> productv1.QueryCategoriesResponse - 50, // 308: productv1.ProductService.QueryCategoriesByKind:output_type -> productv1.QueryCategoriesByKindResponse - 52, // 309: productv1.ProductService.QueryCategoriesByKinds:output_type -> productv1.QueryCategoriesByKindsResponse - 44, // 310: productv1.ProductService.QueryComplexFilterType:output_type -> productv1.QueryComplexFilterTypeResponse - 54, // 311: productv1.ProductService.QueryFilterCategories:output_type -> productv1.QueryFilterCategoriesResponse - 36, // 312: productv1.ProductService.QueryNestedType:output_type -> productv1.QueryNestedTypeResponse - 64, // 313: productv1.ProductService.QueryNullableFieldsType:output_type -> productv1.QueryNullableFieldsTypeResponse - 66, // 314: productv1.ProductService.QueryNullableFieldsTypeById:output_type -> productv1.QueryNullableFieldsTypeByIdResponse - 68, // 315: productv1.ProductService.QueryNullableFieldsTypeWithFilter:output_type -> productv1.QueryNullableFieldsTypeWithFilterResponse - 56, // 316: productv1.ProductService.QueryRandomPet:output_type -> productv1.QueryRandomPetResponse - 62, // 317: productv1.ProductService.QueryRandomSearchResult:output_type -> productv1.QueryRandomSearchResultResponse - 38, // 318: productv1.ProductService.QueryRecursiveType:output_type -> productv1.QueryRecursiveTypeResponse - 60, // 319: productv1.ProductService.QuerySearch:output_type -> productv1.QuerySearchResponse - 40, // 320: productv1.ProductService.QueryTypeFilterWithArguments:output_type -> productv1.QueryTypeFilterWithArgumentsResponse - 42, // 321: productv1.ProductService.QueryTypeWithMultipleFilterFields:output_type -> productv1.QueryTypeWithMultipleFilterFieldsResponse - 34, // 322: productv1.ProductService.QueryUser:output_type -> productv1.QueryUserResponse - 32, // 323: productv1.ProductService.QueryUsers:output_type -> productv1.QueryUsersResponse - 139, // 324: productv1.ProductService.ResolveCategoryCategoryMetrics:output_type -> productv1.ResolveCategoryCategoryMetricsResponse - 134, // 325: productv1.ProductService.ResolveCategoryPopularityScore:output_type -> productv1.ResolveCategoryPopularityScoreResponse - 129, // 326: productv1.ProductService.ResolveCategoryProductCount:output_type -> productv1.ResolveCategoryProductCountResponse - 124, // 327: productv1.ProductService.ResolveProductRecommendedCategory:output_type -> productv1.ResolveProductRecommendedCategoryResponse - 119, // 328: productv1.ProductService.ResolveProductShippingEstimate:output_type -> productv1.ResolveProductShippingEstimateResponse - 144, // 329: productv1.ProductService.ResolveSubcategoryItemCount:output_type -> productv1.ResolveSubcategoryItemCountResponse - 279, // [279:330] is the sub-list for method output_type - 228, // [228:279] is the sub-list for method input_type - 228, // [228:228] is the sub-list for extension type_name - 228, // [228:228] is the sub-list for extension extendee - 0, // [0:228] is the sub-list for field type_name + 146, // 117: productv1.ResolveCategoryMetricsNormalizedScoreRequest.context:type_name -> productv1.ResolveCategoryMetricsNormalizedScoreContext + 145, // 118: productv1.ResolveCategoryMetricsNormalizedScoreRequest.field_args:type_name -> productv1.ResolveCategoryMetricsNormalizedScoreArgs + 148, // 119: productv1.ResolveCategoryMetricsNormalizedScoreResponse.result:type_name -> productv1.ResolveCategoryMetricsNormalizedScoreResult + 179, // 120: productv1.NestedTypeA.b:type_name -> productv1.NestedTypeB + 155, // 121: productv1.RecursiveType.recursive_type:type_name -> productv1.RecursiveType + 181, // 122: productv1.ComplexFilterTypeInput.filter:type_name -> productv1.FilterType + 183, // 123: productv1.OrderInput.lines:type_name -> productv1.OrderLineInput + 16, // 124: productv1.Order.order_lines:type_name -> productv1.ListOfOrderLine + 0, // 125: productv1.Category.kind:type_name -> productv1.CategoryKind + 19, // 126: productv1.Category.subcategories:type_name -> productv1.ListOfSubcategory + 0, // 127: productv1.CategoryFilter.category:type_name -> productv1.CategoryKind + 182, // 128: productv1.CategoryFilter.pagination:type_name -> productv1.Pagination + 187, // 129: productv1.Animal.cat:type_name -> productv1.Cat + 188, // 130: productv1.Animal.dog:type_name -> productv1.Dog + 215, // 131: productv1.SearchInput.limit:type_name -> google.protobuf.Int32Value + 150, // 132: productv1.SearchResult.product:type_name -> productv1.Product + 153, // 133: productv1.SearchResult.user:type_name -> productv1.User + 162, // 134: productv1.SearchResult.category:type_name -> productv1.Category + 216, // 135: productv1.NullableFieldsType.optional_string:type_name -> google.protobuf.StringValue + 215, // 136: productv1.NullableFieldsType.optional_int:type_name -> google.protobuf.Int32Value + 217, // 137: productv1.NullableFieldsType.optional_float:type_name -> google.protobuf.DoubleValue + 218, // 138: productv1.NullableFieldsType.optional_boolean:type_name -> google.protobuf.BoolValue + 216, // 139: productv1.NullableFieldsFilter.name:type_name -> google.protobuf.StringValue + 216, // 140: productv1.NullableFieldsFilter.optional_string:type_name -> google.protobuf.StringValue + 218, // 141: productv1.NullableFieldsFilter.include_nulls:type_name -> google.protobuf.BoolValue + 18, // 142: productv1.BlogPost.optional_tags:type_name -> productv1.ListOfString + 18, // 143: productv1.BlogPost.keywords:type_name -> productv1.ListOfString + 10, // 144: productv1.BlogPost.ratings:type_name -> productv1.ListOfFloat + 7, // 145: productv1.BlogPost.is_published:type_name -> productv1.ListOfBoolean + 13, // 146: productv1.BlogPost.tag_groups:type_name -> productv1.ListOfListOfString + 13, // 147: productv1.BlogPost.related_topics:type_name -> productv1.ListOfListOfString + 13, // 148: productv1.BlogPost.comment_threads:type_name -> productv1.ListOfListOfString + 13, // 149: productv1.BlogPost.suggestions:type_name -> productv1.ListOfListOfString + 162, // 150: productv1.BlogPost.related_categories:type_name -> productv1.Category + 153, // 151: productv1.BlogPost.contributors:type_name -> productv1.User + 17, // 152: productv1.BlogPost.mentioned_products:type_name -> productv1.ListOfProduct + 20, // 153: productv1.BlogPost.mentioned_users:type_name -> productv1.ListOfUser + 11, // 154: productv1.BlogPost.category_groups:type_name -> productv1.ListOfListOfCategory + 14, // 155: productv1.BlogPost.contributor_teams:type_name -> productv1.ListOfListOfUser + 216, // 156: productv1.BlogPostFilter.title:type_name -> google.protobuf.StringValue + 218, // 157: productv1.BlogPostFilter.has_categories:type_name -> google.protobuf.BoolValue + 215, // 158: productv1.BlogPostFilter.min_tags:type_name -> google.protobuf.Int32Value + 216, // 159: productv1.Author.email:type_name -> google.protobuf.StringValue + 18, // 160: productv1.Author.social_links:type_name -> productv1.ListOfString + 13, // 161: productv1.Author.teams_by_project:type_name -> productv1.ListOfListOfString + 13, // 162: productv1.Author.collaborations:type_name -> productv1.ListOfListOfString + 4, // 163: productv1.Author.written_posts:type_name -> productv1.ListOfBlogPost + 162, // 164: productv1.Author.favorite_categories:type_name -> productv1.Category + 20, // 165: productv1.Author.related_authors:type_name -> productv1.ListOfUser + 17, // 166: productv1.Author.product_reviews:type_name -> productv1.ListOfProduct + 14, // 167: productv1.Author.author_groups:type_name -> productv1.ListOfListOfUser + 11, // 168: productv1.Author.category_preferences:type_name -> productv1.ListOfListOfCategory + 14, // 169: productv1.Author.project_teams:type_name -> productv1.ListOfListOfUser + 216, // 170: productv1.AuthorFilter.name:type_name -> google.protobuf.StringValue + 218, // 171: productv1.AuthorFilter.has_teams:type_name -> google.protobuf.BoolValue + 215, // 172: productv1.AuthorFilter.skill_count:type_name -> google.protobuf.Int32Value + 189, // 173: productv1.ActionResult.action_success:type_name -> productv1.ActionSuccess + 190, // 174: productv1.ActionResult.action_error:type_name -> productv1.ActionError + 216, // 175: productv1.NullableFieldsInput.optional_string:type_name -> google.protobuf.StringValue + 215, // 176: productv1.NullableFieldsInput.optional_int:type_name -> google.protobuf.Int32Value + 217, // 177: productv1.NullableFieldsInput.optional_float:type_name -> google.protobuf.DoubleValue + 218, // 178: productv1.NullableFieldsInput.optional_boolean:type_name -> google.protobuf.BoolValue + 18, // 179: productv1.BlogPostInput.optional_tags:type_name -> productv1.ListOfString + 18, // 180: productv1.BlogPostInput.keywords:type_name -> productv1.ListOfString + 10, // 181: productv1.BlogPostInput.ratings:type_name -> productv1.ListOfFloat + 7, // 182: productv1.BlogPostInput.is_published:type_name -> productv1.ListOfBoolean + 13, // 183: productv1.BlogPostInput.tag_groups:type_name -> productv1.ListOfListOfString + 13, // 184: productv1.BlogPostInput.related_topics:type_name -> productv1.ListOfListOfString + 13, // 185: productv1.BlogPostInput.comment_threads:type_name -> productv1.ListOfListOfString + 13, // 186: productv1.BlogPostInput.suggestions:type_name -> productv1.ListOfListOfString + 9, // 187: productv1.BlogPostInput.related_categories:type_name -> productv1.ListOfCategoryInput + 21, // 188: productv1.BlogPostInput.contributors:type_name -> productv1.ListOfUserInput + 12, // 189: productv1.BlogPostInput.category_groups:type_name -> productv1.ListOfListOfCategoryInput + 216, // 190: productv1.AuthorInput.email:type_name -> google.protobuf.StringValue + 18, // 191: productv1.AuthorInput.social_links:type_name -> productv1.ListOfString + 13, // 192: productv1.AuthorInput.teams_by_project:type_name -> productv1.ListOfListOfString + 13, // 193: productv1.AuthorInput.collaborations:type_name -> productv1.ListOfListOfString + 191, // 194: productv1.AuthorInput.favorite_categories:type_name -> productv1.CategoryInput + 15, // 195: productv1.AuthorInput.author_groups:type_name -> productv1.ListOfListOfUserInput + 15, // 196: productv1.AuthorInput.project_teams:type_name -> productv1.ListOfListOfUserInput + 180, // 197: productv1.NestedTypeB.c:type_name -> productv1.NestedTypeC + 182, // 198: productv1.FilterType.pagination:type_name -> productv1.Pagination + 18, // 199: productv1.OrderLineInput.modifiers:type_name -> productv1.ListOfString + 18, // 200: productv1.OrderLine.modifiers:type_name -> productv1.ListOfString + 216, // 201: productv1.Subcategory.description:type_name -> google.protobuf.StringValue + 162, // 202: productv1.CategoryMetrics.related_category:type_name -> productv1.Category + 0, // 203: productv1.CategoryInput.kind:type_name -> productv1.CategoryKind + 217, // 204: productv1.ProductCountFilter.min_price:type_name -> google.protobuf.DoubleValue + 217, // 205: productv1.ProductCountFilter.max_price:type_name -> google.protobuf.DoubleValue + 218, // 206: productv1.ProductCountFilter.in_stock:type_name -> google.protobuf.BoolValue + 216, // 207: productv1.ProductCountFilter.search_term:type_name -> google.protobuf.StringValue + 217, // 208: productv1.SubcategoryItemFilter.min_price:type_name -> google.protobuf.DoubleValue + 217, // 209: productv1.SubcategoryItemFilter.max_price:type_name -> google.protobuf.DoubleValue + 218, // 210: productv1.SubcategoryItemFilter.in_stock:type_name -> google.protobuf.BoolValue + 218, // 211: productv1.SubcategoryItemFilter.is_active:type_name -> google.protobuf.BoolValue + 216, // 212: productv1.SubcategoryItemFilter.search_term:type_name -> google.protobuf.StringValue + 1, // 213: productv1.ShippingEstimateInput.destination:type_name -> productv1.ShippingDestination + 218, // 214: productv1.ShippingEstimateInput.expedited:type_name -> google.protobuf.BoolValue + 172, // 215: productv1.ListOfAuthorFilter.List.items:type_name -> productv1.AuthorFilter + 178, // 216: productv1.ListOfAuthorInput.List.items:type_name -> productv1.AuthorInput + 169, // 217: productv1.ListOfBlogPost.List.items:type_name -> productv1.BlogPost + 170, // 218: productv1.ListOfBlogPostFilter.List.items:type_name -> productv1.BlogPostFilter + 177, // 219: productv1.ListOfBlogPostInput.List.items:type_name -> productv1.BlogPostInput + 162, // 220: productv1.ListOfCategory.List.items:type_name -> productv1.Category + 191, // 221: productv1.ListOfCategoryInput.List.items:type_name -> productv1.CategoryInput + 8, // 222: productv1.ListOfListOfCategory.List.items:type_name -> productv1.ListOfCategory + 9, // 223: productv1.ListOfListOfCategoryInput.List.items:type_name -> productv1.ListOfCategoryInput + 18, // 224: productv1.ListOfListOfString.List.items:type_name -> productv1.ListOfString + 20, // 225: productv1.ListOfListOfUser.List.items:type_name -> productv1.ListOfUser + 21, // 226: productv1.ListOfListOfUserInput.List.items:type_name -> productv1.ListOfUserInput + 184, // 227: productv1.ListOfOrderLine.List.items:type_name -> productv1.OrderLine + 150, // 228: productv1.ListOfProduct.List.items:type_name -> productv1.Product + 185, // 229: productv1.ListOfSubcategory.List.items:type_name -> productv1.Subcategory + 153, // 230: productv1.ListOfUser.List.items:type_name -> productv1.User + 173, // 231: productv1.ListOfUserInput.List.items:type_name -> productv1.UserInput + 23, // 232: productv1.ProductService.LookupProductById:input_type -> productv1.LookupProductByIdRequest + 26, // 233: productv1.ProductService.LookupStorageById:input_type -> productv1.LookupStorageByIdRequest + 29, // 234: productv1.ProductService.LookupWarehouseById:input_type -> productv1.LookupWarehouseByIdRequest + 107, // 235: productv1.ProductService.MutationBulkCreateAuthors:input_type -> productv1.MutationBulkCreateAuthorsRequest + 111, // 236: productv1.ProductService.MutationBulkCreateBlogPosts:input_type -> productv1.MutationBulkCreateBlogPostsRequest + 109, // 237: productv1.ProductService.MutationBulkUpdateAuthors:input_type -> productv1.MutationBulkUpdateAuthorsRequest + 113, // 238: productv1.ProductService.MutationBulkUpdateBlogPosts:input_type -> productv1.MutationBulkUpdateBlogPostsRequest + 103, // 239: productv1.ProductService.MutationCreateAuthor:input_type -> productv1.MutationCreateAuthorRequest + 99, // 240: productv1.ProductService.MutationCreateBlogPost:input_type -> productv1.MutationCreateBlogPostRequest + 95, // 241: productv1.ProductService.MutationCreateNullableFieldsType:input_type -> productv1.MutationCreateNullableFieldsTypeRequest + 91, // 242: productv1.ProductService.MutationCreateUser:input_type -> productv1.MutationCreateUserRequest + 93, // 243: productv1.ProductService.MutationPerformAction:input_type -> productv1.MutationPerformActionRequest + 105, // 244: productv1.ProductService.MutationUpdateAuthor:input_type -> productv1.MutationUpdateAuthorRequest + 101, // 245: productv1.ProductService.MutationUpdateBlogPost:input_type -> productv1.MutationUpdateBlogPostRequest + 97, // 246: productv1.ProductService.MutationUpdateNullableFieldsType:input_type -> productv1.MutationUpdateNullableFieldsTypeRequest + 85, // 247: productv1.ProductService.QueryAllAuthors:input_type -> productv1.QueryAllAuthorsRequest + 77, // 248: productv1.ProductService.QueryAllBlogPosts:input_type -> productv1.QueryAllBlogPostsRequest + 69, // 249: productv1.ProductService.QueryAllNullableFieldsTypes:input_type -> productv1.QueryAllNullableFieldsTypesRequest + 57, // 250: productv1.ProductService.QueryAllPets:input_type -> productv1.QueryAllPetsRequest + 79, // 251: productv1.ProductService.QueryAuthor:input_type -> productv1.QueryAuthorRequest + 81, // 252: productv1.ProductService.QueryAuthorById:input_type -> productv1.QueryAuthorByIdRequest + 83, // 253: productv1.ProductService.QueryAuthorsWithFilter:input_type -> productv1.QueryAuthorsWithFilterRequest + 71, // 254: productv1.ProductService.QueryBlogPost:input_type -> productv1.QueryBlogPostRequest + 73, // 255: productv1.ProductService.QueryBlogPostById:input_type -> productv1.QueryBlogPostByIdRequest + 75, // 256: productv1.ProductService.QueryBlogPostsWithFilter:input_type -> productv1.QueryBlogPostsWithFilterRequest + 87, // 257: productv1.ProductService.QueryBulkSearchAuthors:input_type -> productv1.QueryBulkSearchAuthorsRequest + 89, // 258: productv1.ProductService.QueryBulkSearchBlogPosts:input_type -> productv1.QueryBulkSearchBlogPostsRequest + 45, // 259: productv1.ProductService.QueryCalculateTotals:input_type -> productv1.QueryCalculateTotalsRequest + 47, // 260: productv1.ProductService.QueryCategories:input_type -> productv1.QueryCategoriesRequest + 49, // 261: productv1.ProductService.QueryCategoriesByKind:input_type -> productv1.QueryCategoriesByKindRequest + 51, // 262: productv1.ProductService.QueryCategoriesByKinds:input_type -> productv1.QueryCategoriesByKindsRequest + 43, // 263: productv1.ProductService.QueryComplexFilterType:input_type -> productv1.QueryComplexFilterTypeRequest + 53, // 264: productv1.ProductService.QueryFilterCategories:input_type -> productv1.QueryFilterCategoriesRequest + 35, // 265: productv1.ProductService.QueryNestedType:input_type -> productv1.QueryNestedTypeRequest + 63, // 266: productv1.ProductService.QueryNullableFieldsType:input_type -> productv1.QueryNullableFieldsTypeRequest + 65, // 267: productv1.ProductService.QueryNullableFieldsTypeById:input_type -> productv1.QueryNullableFieldsTypeByIdRequest + 67, // 268: productv1.ProductService.QueryNullableFieldsTypeWithFilter:input_type -> productv1.QueryNullableFieldsTypeWithFilterRequest + 55, // 269: productv1.ProductService.QueryRandomPet:input_type -> productv1.QueryRandomPetRequest + 61, // 270: productv1.ProductService.QueryRandomSearchResult:input_type -> productv1.QueryRandomSearchResultRequest + 37, // 271: productv1.ProductService.QueryRecursiveType:input_type -> productv1.QueryRecursiveTypeRequest + 59, // 272: productv1.ProductService.QuerySearch:input_type -> productv1.QuerySearchRequest + 39, // 273: productv1.ProductService.QueryTypeFilterWithArguments:input_type -> productv1.QueryTypeFilterWithArgumentsRequest + 41, // 274: productv1.ProductService.QueryTypeWithMultipleFilterFields:input_type -> productv1.QueryTypeWithMultipleFilterFieldsRequest + 33, // 275: productv1.ProductService.QueryUser:input_type -> productv1.QueryUserRequest + 31, // 276: productv1.ProductService.QueryUsers:input_type -> productv1.QueryUsersRequest + 137, // 277: productv1.ProductService.ResolveCategoryCategoryMetrics:input_type -> productv1.ResolveCategoryCategoryMetricsRequest + 147, // 278: productv1.ProductService.ResolveCategoryMetricsNormalizedScore:input_type -> productv1.ResolveCategoryMetricsNormalizedScoreRequest + 132, // 279: productv1.ProductService.ResolveCategoryPopularityScore:input_type -> productv1.ResolveCategoryPopularityScoreRequest + 127, // 280: productv1.ProductService.ResolveCategoryProductCount:input_type -> productv1.ResolveCategoryProductCountRequest + 122, // 281: productv1.ProductService.ResolveProductRecommendedCategory:input_type -> productv1.ResolveProductRecommendedCategoryRequest + 117, // 282: productv1.ProductService.ResolveProductShippingEstimate:input_type -> productv1.ResolveProductShippingEstimateRequest + 142, // 283: productv1.ProductService.ResolveSubcategoryItemCount:input_type -> productv1.ResolveSubcategoryItemCountRequest + 24, // 284: productv1.ProductService.LookupProductById:output_type -> productv1.LookupProductByIdResponse + 27, // 285: productv1.ProductService.LookupStorageById:output_type -> productv1.LookupStorageByIdResponse + 30, // 286: productv1.ProductService.LookupWarehouseById:output_type -> productv1.LookupWarehouseByIdResponse + 108, // 287: productv1.ProductService.MutationBulkCreateAuthors:output_type -> productv1.MutationBulkCreateAuthorsResponse + 112, // 288: productv1.ProductService.MutationBulkCreateBlogPosts:output_type -> productv1.MutationBulkCreateBlogPostsResponse + 110, // 289: productv1.ProductService.MutationBulkUpdateAuthors:output_type -> productv1.MutationBulkUpdateAuthorsResponse + 114, // 290: productv1.ProductService.MutationBulkUpdateBlogPosts:output_type -> productv1.MutationBulkUpdateBlogPostsResponse + 104, // 291: productv1.ProductService.MutationCreateAuthor:output_type -> productv1.MutationCreateAuthorResponse + 100, // 292: productv1.ProductService.MutationCreateBlogPost:output_type -> productv1.MutationCreateBlogPostResponse + 96, // 293: productv1.ProductService.MutationCreateNullableFieldsType:output_type -> productv1.MutationCreateNullableFieldsTypeResponse + 92, // 294: productv1.ProductService.MutationCreateUser:output_type -> productv1.MutationCreateUserResponse + 94, // 295: productv1.ProductService.MutationPerformAction:output_type -> productv1.MutationPerformActionResponse + 106, // 296: productv1.ProductService.MutationUpdateAuthor:output_type -> productv1.MutationUpdateAuthorResponse + 102, // 297: productv1.ProductService.MutationUpdateBlogPost:output_type -> productv1.MutationUpdateBlogPostResponse + 98, // 298: productv1.ProductService.MutationUpdateNullableFieldsType:output_type -> productv1.MutationUpdateNullableFieldsTypeResponse + 86, // 299: productv1.ProductService.QueryAllAuthors:output_type -> productv1.QueryAllAuthorsResponse + 78, // 300: productv1.ProductService.QueryAllBlogPosts:output_type -> productv1.QueryAllBlogPostsResponse + 70, // 301: productv1.ProductService.QueryAllNullableFieldsTypes:output_type -> productv1.QueryAllNullableFieldsTypesResponse + 58, // 302: productv1.ProductService.QueryAllPets:output_type -> productv1.QueryAllPetsResponse + 80, // 303: productv1.ProductService.QueryAuthor:output_type -> productv1.QueryAuthorResponse + 82, // 304: productv1.ProductService.QueryAuthorById:output_type -> productv1.QueryAuthorByIdResponse + 84, // 305: productv1.ProductService.QueryAuthorsWithFilter:output_type -> productv1.QueryAuthorsWithFilterResponse + 72, // 306: productv1.ProductService.QueryBlogPost:output_type -> productv1.QueryBlogPostResponse + 74, // 307: productv1.ProductService.QueryBlogPostById:output_type -> productv1.QueryBlogPostByIdResponse + 76, // 308: productv1.ProductService.QueryBlogPostsWithFilter:output_type -> productv1.QueryBlogPostsWithFilterResponse + 88, // 309: productv1.ProductService.QueryBulkSearchAuthors:output_type -> productv1.QueryBulkSearchAuthorsResponse + 90, // 310: productv1.ProductService.QueryBulkSearchBlogPosts:output_type -> productv1.QueryBulkSearchBlogPostsResponse + 46, // 311: productv1.ProductService.QueryCalculateTotals:output_type -> productv1.QueryCalculateTotalsResponse + 48, // 312: productv1.ProductService.QueryCategories:output_type -> productv1.QueryCategoriesResponse + 50, // 313: productv1.ProductService.QueryCategoriesByKind:output_type -> productv1.QueryCategoriesByKindResponse + 52, // 314: productv1.ProductService.QueryCategoriesByKinds:output_type -> productv1.QueryCategoriesByKindsResponse + 44, // 315: productv1.ProductService.QueryComplexFilterType:output_type -> productv1.QueryComplexFilterTypeResponse + 54, // 316: productv1.ProductService.QueryFilterCategories:output_type -> productv1.QueryFilterCategoriesResponse + 36, // 317: productv1.ProductService.QueryNestedType:output_type -> productv1.QueryNestedTypeResponse + 64, // 318: productv1.ProductService.QueryNullableFieldsType:output_type -> productv1.QueryNullableFieldsTypeResponse + 66, // 319: productv1.ProductService.QueryNullableFieldsTypeById:output_type -> productv1.QueryNullableFieldsTypeByIdResponse + 68, // 320: productv1.ProductService.QueryNullableFieldsTypeWithFilter:output_type -> productv1.QueryNullableFieldsTypeWithFilterResponse + 56, // 321: productv1.ProductService.QueryRandomPet:output_type -> productv1.QueryRandomPetResponse + 62, // 322: productv1.ProductService.QueryRandomSearchResult:output_type -> productv1.QueryRandomSearchResultResponse + 38, // 323: productv1.ProductService.QueryRecursiveType:output_type -> productv1.QueryRecursiveTypeResponse + 60, // 324: productv1.ProductService.QuerySearch:output_type -> productv1.QuerySearchResponse + 40, // 325: productv1.ProductService.QueryTypeFilterWithArguments:output_type -> productv1.QueryTypeFilterWithArgumentsResponse + 42, // 326: productv1.ProductService.QueryTypeWithMultipleFilterFields:output_type -> productv1.QueryTypeWithMultipleFilterFieldsResponse + 34, // 327: productv1.ProductService.QueryUser:output_type -> productv1.QueryUserResponse + 32, // 328: productv1.ProductService.QueryUsers:output_type -> productv1.QueryUsersResponse + 139, // 329: productv1.ProductService.ResolveCategoryCategoryMetrics:output_type -> productv1.ResolveCategoryCategoryMetricsResponse + 149, // 330: productv1.ProductService.ResolveCategoryMetricsNormalizedScore:output_type -> productv1.ResolveCategoryMetricsNormalizedScoreResponse + 134, // 331: productv1.ProductService.ResolveCategoryPopularityScore:output_type -> productv1.ResolveCategoryPopularityScoreResponse + 129, // 332: productv1.ProductService.ResolveCategoryProductCount:output_type -> productv1.ResolveCategoryProductCountResponse + 124, // 333: productv1.ProductService.ResolveProductRecommendedCategory:output_type -> productv1.ResolveProductRecommendedCategoryResponse + 119, // 334: productv1.ProductService.ResolveProductShippingEstimate:output_type -> productv1.ResolveProductShippingEstimateResponse + 144, // 335: productv1.ProductService.ResolveSubcategoryItemCount:output_type -> productv1.ResolveSubcategoryItemCountResponse + 284, // [284:336] is the sub-list for method output_type + 232, // [232:284] is the sub-list for method input_type + 232, // [232:232] is the sub-list for extension type_name + 232, // [232:232] is the sub-list for extension extendee + 0, // [0:232] is the sub-list for field type_name } func init() { file_product_proto_init() } @@ -11968,16 +12251,16 @@ func file_product_proto_init() { if File_product_proto != nil { return } - file_product_proto_msgTypes[157].OneofWrappers = []any{ + file_product_proto_msgTypes[162].OneofWrappers = []any{ (*Animal_Cat)(nil), (*Animal_Dog)(nil), } - file_product_proto_msgTypes[159].OneofWrappers = []any{ + file_product_proto_msgTypes[164].OneofWrappers = []any{ (*SearchResult_Product)(nil), (*SearchResult_User)(nil), (*SearchResult_Category)(nil), } - file_product_proto_msgTypes[168].OneofWrappers = []any{ + file_product_proto_msgTypes[173].OneofWrappers = []any{ (*ActionResult_ActionSuccess)(nil), (*ActionResult_ActionError)(nil), } @@ -11987,7 +12270,7 @@ func file_product_proto_init() { GoPackagePath: reflect.TypeOf(x{}).PkgPath(), RawDescriptor: unsafe.Slice(unsafe.StringData(file_product_proto_rawDesc), len(file_product_proto_rawDesc)), NumEnums: 2, - NumMessages: 208, + NumMessages: 213, NumExtensions: 0, NumServices: 1, }, diff --git a/v2/pkg/grpctest/productv1/product_grpc.pb.go b/v2/pkg/grpctest/productv1/product_grpc.pb.go index 9f856d858..5fc55ddba 100644 --- a/v2/pkg/grpctest/productv1/product_grpc.pb.go +++ b/v2/pkg/grpctest/productv1/product_grpc.pb.go @@ -19,57 +19,58 @@ import ( const _ = grpc.SupportPackageIsVersion9 const ( - ProductService_LookupProductById_FullMethodName = "/productv1.ProductService/LookupProductById" - ProductService_LookupStorageById_FullMethodName = "/productv1.ProductService/LookupStorageById" - ProductService_LookupWarehouseById_FullMethodName = "/productv1.ProductService/LookupWarehouseById" - ProductService_MutationBulkCreateAuthors_FullMethodName = "/productv1.ProductService/MutationBulkCreateAuthors" - ProductService_MutationBulkCreateBlogPosts_FullMethodName = "/productv1.ProductService/MutationBulkCreateBlogPosts" - ProductService_MutationBulkUpdateAuthors_FullMethodName = "/productv1.ProductService/MutationBulkUpdateAuthors" - ProductService_MutationBulkUpdateBlogPosts_FullMethodName = "/productv1.ProductService/MutationBulkUpdateBlogPosts" - ProductService_MutationCreateAuthor_FullMethodName = "/productv1.ProductService/MutationCreateAuthor" - ProductService_MutationCreateBlogPost_FullMethodName = "/productv1.ProductService/MutationCreateBlogPost" - ProductService_MutationCreateNullableFieldsType_FullMethodName = "/productv1.ProductService/MutationCreateNullableFieldsType" - ProductService_MutationCreateUser_FullMethodName = "/productv1.ProductService/MutationCreateUser" - ProductService_MutationPerformAction_FullMethodName = "/productv1.ProductService/MutationPerformAction" - ProductService_MutationUpdateAuthor_FullMethodName = "/productv1.ProductService/MutationUpdateAuthor" - ProductService_MutationUpdateBlogPost_FullMethodName = "/productv1.ProductService/MutationUpdateBlogPost" - ProductService_MutationUpdateNullableFieldsType_FullMethodName = "/productv1.ProductService/MutationUpdateNullableFieldsType" - ProductService_QueryAllAuthors_FullMethodName = "/productv1.ProductService/QueryAllAuthors" - ProductService_QueryAllBlogPosts_FullMethodName = "/productv1.ProductService/QueryAllBlogPosts" - ProductService_QueryAllNullableFieldsTypes_FullMethodName = "/productv1.ProductService/QueryAllNullableFieldsTypes" - ProductService_QueryAllPets_FullMethodName = "/productv1.ProductService/QueryAllPets" - ProductService_QueryAuthor_FullMethodName = "/productv1.ProductService/QueryAuthor" - ProductService_QueryAuthorById_FullMethodName = "/productv1.ProductService/QueryAuthorById" - ProductService_QueryAuthorsWithFilter_FullMethodName = "/productv1.ProductService/QueryAuthorsWithFilter" - ProductService_QueryBlogPost_FullMethodName = "/productv1.ProductService/QueryBlogPost" - ProductService_QueryBlogPostById_FullMethodName = "/productv1.ProductService/QueryBlogPostById" - ProductService_QueryBlogPostsWithFilter_FullMethodName = "/productv1.ProductService/QueryBlogPostsWithFilter" - ProductService_QueryBulkSearchAuthors_FullMethodName = "/productv1.ProductService/QueryBulkSearchAuthors" - ProductService_QueryBulkSearchBlogPosts_FullMethodName = "/productv1.ProductService/QueryBulkSearchBlogPosts" - ProductService_QueryCalculateTotals_FullMethodName = "/productv1.ProductService/QueryCalculateTotals" - ProductService_QueryCategories_FullMethodName = "/productv1.ProductService/QueryCategories" - ProductService_QueryCategoriesByKind_FullMethodName = "/productv1.ProductService/QueryCategoriesByKind" - ProductService_QueryCategoriesByKinds_FullMethodName = "/productv1.ProductService/QueryCategoriesByKinds" - ProductService_QueryComplexFilterType_FullMethodName = "/productv1.ProductService/QueryComplexFilterType" - ProductService_QueryFilterCategories_FullMethodName = "/productv1.ProductService/QueryFilterCategories" - ProductService_QueryNestedType_FullMethodName = "/productv1.ProductService/QueryNestedType" - ProductService_QueryNullableFieldsType_FullMethodName = "/productv1.ProductService/QueryNullableFieldsType" - ProductService_QueryNullableFieldsTypeById_FullMethodName = "/productv1.ProductService/QueryNullableFieldsTypeById" - ProductService_QueryNullableFieldsTypeWithFilter_FullMethodName = "/productv1.ProductService/QueryNullableFieldsTypeWithFilter" - ProductService_QueryRandomPet_FullMethodName = "/productv1.ProductService/QueryRandomPet" - ProductService_QueryRandomSearchResult_FullMethodName = "/productv1.ProductService/QueryRandomSearchResult" - ProductService_QueryRecursiveType_FullMethodName = "/productv1.ProductService/QueryRecursiveType" - ProductService_QuerySearch_FullMethodName = "/productv1.ProductService/QuerySearch" - ProductService_QueryTypeFilterWithArguments_FullMethodName = "/productv1.ProductService/QueryTypeFilterWithArguments" - ProductService_QueryTypeWithMultipleFilterFields_FullMethodName = "/productv1.ProductService/QueryTypeWithMultipleFilterFields" - ProductService_QueryUser_FullMethodName = "/productv1.ProductService/QueryUser" - ProductService_QueryUsers_FullMethodName = "/productv1.ProductService/QueryUsers" - ProductService_ResolveCategoryCategoryMetrics_FullMethodName = "/productv1.ProductService/ResolveCategoryCategoryMetrics" - ProductService_ResolveCategoryPopularityScore_FullMethodName = "/productv1.ProductService/ResolveCategoryPopularityScore" - ProductService_ResolveCategoryProductCount_FullMethodName = "/productv1.ProductService/ResolveCategoryProductCount" - ProductService_ResolveProductRecommendedCategory_FullMethodName = "/productv1.ProductService/ResolveProductRecommendedCategory" - ProductService_ResolveProductShippingEstimate_FullMethodName = "/productv1.ProductService/ResolveProductShippingEstimate" - ProductService_ResolveSubcategoryItemCount_FullMethodName = "/productv1.ProductService/ResolveSubcategoryItemCount" + ProductService_LookupProductById_FullMethodName = "/productv1.ProductService/LookupProductById" + ProductService_LookupStorageById_FullMethodName = "/productv1.ProductService/LookupStorageById" + ProductService_LookupWarehouseById_FullMethodName = "/productv1.ProductService/LookupWarehouseById" + ProductService_MutationBulkCreateAuthors_FullMethodName = "/productv1.ProductService/MutationBulkCreateAuthors" + ProductService_MutationBulkCreateBlogPosts_FullMethodName = "/productv1.ProductService/MutationBulkCreateBlogPosts" + ProductService_MutationBulkUpdateAuthors_FullMethodName = "/productv1.ProductService/MutationBulkUpdateAuthors" + ProductService_MutationBulkUpdateBlogPosts_FullMethodName = "/productv1.ProductService/MutationBulkUpdateBlogPosts" + ProductService_MutationCreateAuthor_FullMethodName = "/productv1.ProductService/MutationCreateAuthor" + ProductService_MutationCreateBlogPost_FullMethodName = "/productv1.ProductService/MutationCreateBlogPost" + ProductService_MutationCreateNullableFieldsType_FullMethodName = "/productv1.ProductService/MutationCreateNullableFieldsType" + ProductService_MutationCreateUser_FullMethodName = "/productv1.ProductService/MutationCreateUser" + ProductService_MutationPerformAction_FullMethodName = "/productv1.ProductService/MutationPerformAction" + ProductService_MutationUpdateAuthor_FullMethodName = "/productv1.ProductService/MutationUpdateAuthor" + ProductService_MutationUpdateBlogPost_FullMethodName = "/productv1.ProductService/MutationUpdateBlogPost" + ProductService_MutationUpdateNullableFieldsType_FullMethodName = "/productv1.ProductService/MutationUpdateNullableFieldsType" + ProductService_QueryAllAuthors_FullMethodName = "/productv1.ProductService/QueryAllAuthors" + ProductService_QueryAllBlogPosts_FullMethodName = "/productv1.ProductService/QueryAllBlogPosts" + ProductService_QueryAllNullableFieldsTypes_FullMethodName = "/productv1.ProductService/QueryAllNullableFieldsTypes" + ProductService_QueryAllPets_FullMethodName = "/productv1.ProductService/QueryAllPets" + ProductService_QueryAuthor_FullMethodName = "/productv1.ProductService/QueryAuthor" + ProductService_QueryAuthorById_FullMethodName = "/productv1.ProductService/QueryAuthorById" + ProductService_QueryAuthorsWithFilter_FullMethodName = "/productv1.ProductService/QueryAuthorsWithFilter" + ProductService_QueryBlogPost_FullMethodName = "/productv1.ProductService/QueryBlogPost" + ProductService_QueryBlogPostById_FullMethodName = "/productv1.ProductService/QueryBlogPostById" + ProductService_QueryBlogPostsWithFilter_FullMethodName = "/productv1.ProductService/QueryBlogPostsWithFilter" + ProductService_QueryBulkSearchAuthors_FullMethodName = "/productv1.ProductService/QueryBulkSearchAuthors" + ProductService_QueryBulkSearchBlogPosts_FullMethodName = "/productv1.ProductService/QueryBulkSearchBlogPosts" + ProductService_QueryCalculateTotals_FullMethodName = "/productv1.ProductService/QueryCalculateTotals" + ProductService_QueryCategories_FullMethodName = "/productv1.ProductService/QueryCategories" + ProductService_QueryCategoriesByKind_FullMethodName = "/productv1.ProductService/QueryCategoriesByKind" + ProductService_QueryCategoriesByKinds_FullMethodName = "/productv1.ProductService/QueryCategoriesByKinds" + ProductService_QueryComplexFilterType_FullMethodName = "/productv1.ProductService/QueryComplexFilterType" + ProductService_QueryFilterCategories_FullMethodName = "/productv1.ProductService/QueryFilterCategories" + ProductService_QueryNestedType_FullMethodName = "/productv1.ProductService/QueryNestedType" + ProductService_QueryNullableFieldsType_FullMethodName = "/productv1.ProductService/QueryNullableFieldsType" + ProductService_QueryNullableFieldsTypeById_FullMethodName = "/productv1.ProductService/QueryNullableFieldsTypeById" + ProductService_QueryNullableFieldsTypeWithFilter_FullMethodName = "/productv1.ProductService/QueryNullableFieldsTypeWithFilter" + ProductService_QueryRandomPet_FullMethodName = "/productv1.ProductService/QueryRandomPet" + ProductService_QueryRandomSearchResult_FullMethodName = "/productv1.ProductService/QueryRandomSearchResult" + ProductService_QueryRecursiveType_FullMethodName = "/productv1.ProductService/QueryRecursiveType" + ProductService_QuerySearch_FullMethodName = "/productv1.ProductService/QuerySearch" + ProductService_QueryTypeFilterWithArguments_FullMethodName = "/productv1.ProductService/QueryTypeFilterWithArguments" + ProductService_QueryTypeWithMultipleFilterFields_FullMethodName = "/productv1.ProductService/QueryTypeWithMultipleFilterFields" + ProductService_QueryUser_FullMethodName = "/productv1.ProductService/QueryUser" + ProductService_QueryUsers_FullMethodName = "/productv1.ProductService/QueryUsers" + ProductService_ResolveCategoryCategoryMetrics_FullMethodName = "/productv1.ProductService/ResolveCategoryCategoryMetrics" + ProductService_ResolveCategoryMetricsNormalizedScore_FullMethodName = "/productv1.ProductService/ResolveCategoryMetricsNormalizedScore" + ProductService_ResolveCategoryPopularityScore_FullMethodName = "/productv1.ProductService/ResolveCategoryPopularityScore" + ProductService_ResolveCategoryProductCount_FullMethodName = "/productv1.ProductService/ResolveCategoryProductCount" + ProductService_ResolveProductRecommendedCategory_FullMethodName = "/productv1.ProductService/ResolveProductRecommendedCategory" + ProductService_ResolveProductShippingEstimate_FullMethodName = "/productv1.ProductService/ResolveProductShippingEstimate" + ProductService_ResolveSubcategoryItemCount_FullMethodName = "/productv1.ProductService/ResolveSubcategoryItemCount" ) // ProductServiceClient is the client API for ProductService service. @@ -127,6 +128,7 @@ type ProductServiceClient interface { QueryUser(ctx context.Context, in *QueryUserRequest, opts ...grpc.CallOption) (*QueryUserResponse, error) QueryUsers(ctx context.Context, in *QueryUsersRequest, opts ...grpc.CallOption) (*QueryUsersResponse, error) ResolveCategoryCategoryMetrics(ctx context.Context, in *ResolveCategoryCategoryMetricsRequest, opts ...grpc.CallOption) (*ResolveCategoryCategoryMetricsResponse, error) + ResolveCategoryMetricsNormalizedScore(ctx context.Context, in *ResolveCategoryMetricsNormalizedScoreRequest, opts ...grpc.CallOption) (*ResolveCategoryMetricsNormalizedScoreResponse, error) ResolveCategoryPopularityScore(ctx context.Context, in *ResolveCategoryPopularityScoreRequest, opts ...grpc.CallOption) (*ResolveCategoryPopularityScoreResponse, error) ResolveCategoryProductCount(ctx context.Context, in *ResolveCategoryProductCountRequest, opts ...grpc.CallOption) (*ResolveCategoryProductCountResponse, error) ResolveProductRecommendedCategory(ctx context.Context, in *ResolveProductRecommendedCategoryRequest, opts ...grpc.CallOption) (*ResolveProductRecommendedCategoryResponse, error) @@ -602,6 +604,16 @@ func (c *productServiceClient) ResolveCategoryCategoryMetrics(ctx context.Contex return out, nil } +func (c *productServiceClient) ResolveCategoryMetricsNormalizedScore(ctx context.Context, in *ResolveCategoryMetricsNormalizedScoreRequest, opts ...grpc.CallOption) (*ResolveCategoryMetricsNormalizedScoreResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(ResolveCategoryMetricsNormalizedScoreResponse) + err := c.cc.Invoke(ctx, ProductService_ResolveCategoryMetricsNormalizedScore_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + func (c *productServiceClient) ResolveCategoryPopularityScore(ctx context.Context, in *ResolveCategoryPopularityScoreRequest, opts ...grpc.CallOption) (*ResolveCategoryPopularityScoreResponse, error) { cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) out := new(ResolveCategoryPopularityScoreResponse) @@ -707,6 +719,7 @@ type ProductServiceServer interface { QueryUser(context.Context, *QueryUserRequest) (*QueryUserResponse, error) QueryUsers(context.Context, *QueryUsersRequest) (*QueryUsersResponse, error) ResolveCategoryCategoryMetrics(context.Context, *ResolveCategoryCategoryMetricsRequest) (*ResolveCategoryCategoryMetricsResponse, error) + ResolveCategoryMetricsNormalizedScore(context.Context, *ResolveCategoryMetricsNormalizedScoreRequest) (*ResolveCategoryMetricsNormalizedScoreResponse, error) ResolveCategoryPopularityScore(context.Context, *ResolveCategoryPopularityScoreRequest) (*ResolveCategoryPopularityScoreResponse, error) ResolveCategoryProductCount(context.Context, *ResolveCategoryProductCountRequest) (*ResolveCategoryProductCountResponse, error) ResolveProductRecommendedCategory(context.Context, *ResolveProductRecommendedCategoryRequest) (*ResolveProductRecommendedCategoryResponse, error) @@ -860,6 +873,9 @@ func (UnimplementedProductServiceServer) QueryUsers(context.Context, *QueryUsers func (UnimplementedProductServiceServer) ResolveCategoryCategoryMetrics(context.Context, *ResolveCategoryCategoryMetricsRequest) (*ResolveCategoryCategoryMetricsResponse, error) { return nil, status.Errorf(codes.Unimplemented, "method ResolveCategoryCategoryMetrics not implemented") } +func (UnimplementedProductServiceServer) ResolveCategoryMetricsNormalizedScore(context.Context, *ResolveCategoryMetricsNormalizedScoreRequest) (*ResolveCategoryMetricsNormalizedScoreResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method ResolveCategoryMetricsNormalizedScore not implemented") +} func (UnimplementedProductServiceServer) ResolveCategoryPopularityScore(context.Context, *ResolveCategoryPopularityScoreRequest) (*ResolveCategoryPopularityScoreResponse, error) { return nil, status.Errorf(codes.Unimplemented, "method ResolveCategoryPopularityScore not implemented") } @@ -1724,6 +1740,24 @@ func _ProductService_ResolveCategoryCategoryMetrics_Handler(srv interface{}, ctx return interceptor(ctx, in, info, handler) } +func _ProductService_ResolveCategoryMetricsNormalizedScore_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(ResolveCategoryMetricsNormalizedScoreRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(ProductServiceServer).ResolveCategoryMetricsNormalizedScore(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: ProductService_ResolveCategoryMetricsNormalizedScore_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(ProductServiceServer).ResolveCategoryMetricsNormalizedScore(ctx, req.(*ResolveCategoryMetricsNormalizedScoreRequest)) + } + return interceptor(ctx, in, info, handler) +} + func _ProductService_ResolveCategoryPopularityScore_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { in := new(ResolveCategoryPopularityScoreRequest) if err := dec(in); err != nil { @@ -2005,6 +2039,10 @@ var ProductService_ServiceDesc = grpc.ServiceDesc{ MethodName: "ResolveCategoryCategoryMetrics", Handler: _ProductService_ResolveCategoryCategoryMetrics_Handler, }, + { + MethodName: "ResolveCategoryMetricsNormalizedScore", + Handler: _ProductService_ResolveCategoryMetricsNormalizedScore_Handler, + }, { MethodName: "ResolveCategoryPopularityScore", Handler: _ProductService_ResolveCategoryPopularityScore_Handler, diff --git a/v2/pkg/grpctest/testdata/products.graphqls b/v2/pkg/grpctest/testdata/products.graphqls index d2b6007d5..989bbd4de 100644 --- a/v2/pkg/grpctest/testdata/products.graphqls +++ b/v2/pkg/grpctest/testdata/products.graphqls @@ -134,6 +134,9 @@ type CategoryMetrics { value: Float! timestamp: String! categoryId: ID! + normalizedScore(baseline: Float!): Float! @connect__fieldResolver(context: "id value metricType") + # This will currently not generate a resolver as it is missing arguments. + relatedCategory: Category @connect__fieldResolver(context: "categoryId") } enum CategoryKind { From 503be721465cccda10e287e51b0ca047f5026216 Mon Sep 17 00:00:00 2001 From: Ludwig Bedacht Date: Thu, 30 Oct 2025 11:49:16 +0100 Subject: [PATCH 41/72] feat: add support for composite types in the planner --- v2/pkg/ast/ast_selection.go | 12 + .../grpc_datasource/execution_plan.go | 97 +- .../execution_plan_field_resolvers_test.go | 144 ++ .../grpc_datasource/execution_plan_visitor.go | 44 +- .../grpc_datasource/mapping_test_helper.go | 34 + v2/pkg/grpctest/mapping/mapping.go | 34 + v2/pkg/grpctest/mockservice.go | 108 + v2/pkg/grpctest/product.proto | 50 + v2/pkg/grpctest/productv1/product.pb.go | 1741 +++++++++++------ v2/pkg/grpctest/productv1/product_grpc.pb.go | 76 + v2/pkg/grpctest/testdata/products.graphqls | 2 + 11 files changed, 1729 insertions(+), 613 deletions(-) diff --git a/v2/pkg/ast/ast_selection.go b/v2/pkg/ast/ast_selection.go index 0c7b643c8..2ef6a482b 100644 --- a/v2/pkg/ast/ast_selection.go +++ b/v2/pkg/ast/ast_selection.go @@ -212,6 +212,18 @@ func (d *Document) SelectionSetHasFieldSelectionWithExactName(set int, name []by return false, InvalidRef } +// SelectionSetFieldRefs returns a list of field refs in the selection set. +// It traverses through the field selections of the selection set and returns the actual field refs. +func (d *Document) SelectionSetFieldRefs(set int) (refs []int) { + for _, selectionRef := range d.SelectionSets[set].SelectionRefs { + if d.Selections[selectionRef].Kind == SelectionKindField { + refs = append(refs, d.Selections[selectionRef].Ref) + } + } + return +} + +// SelectionSetFieldSelections returns a list of field selection refs in the selection set. func (d *Document) SelectionSetFieldSelections(set int) (refs []int) { for _, selectionRef := range d.SelectionSets[set].SelectionRefs { if d.Selections[selectionRef].Kind == SelectionKindField { diff --git a/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go b/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go index 13e59a3c0..2aac75f80 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go +++ b/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go @@ -1,6 +1,7 @@ package grpcdatasource import ( + "errors" "fmt" "slices" "strings" @@ -735,8 +736,16 @@ type resolvedField struct { fieldsSelectionSetRef int responsePath ast.Path - contextFields []contextField - fieldArguments []fieldArgument + contextFields []contextField + fieldArguments []fieldArgument + fragmentSelections []fragmentSelection + fragmentType OneOfType + memberTypes []string +} + +type fragmentSelection struct { + typeName string + selectionSetRef int } // isFieldResolver checks if a field is a field resolver. @@ -748,6 +757,48 @@ func (r *rpcPlanningContext) isFieldResolver(fieldRef int, isRootField bool) boo return len(r.operation.FieldArguments(fieldRef)) > 0 } +// getCompositeType returns the most recent composite type from the type definitions. +func (r *rpcPlanningContext) getCompositeType(node ast.Node) OneOfType { + switch node.Kind { + case ast.NodeKindInterfaceTypeDefinition: + return OneOfTypeInterface + case ast.NodeKindUnionTypeDefinition: + return OneOfTypeUnion + default: + return OneOfTypeNone + } +} + +func (r *rpcPlanningContext) getMemberTypes(node ast.Node) ([]string, error) { + switch node.Kind { + case ast.NodeKindInterfaceTypeDefinition: + memberTypes, ok := r.definition.InterfaceTypeDefinitionImplementedByObjectWithNames(node.Ref) + if !ok { + return nil, fmt.Errorf("interface type %s is not implemented by any object", r.definition.InterfaceTypeDefinitionNameString(node.Ref)) + } + return memberTypes, nil + case ast.NodeKindUnionTypeDefinition: + memberTypes, ok := r.definition.UnionTypeDefinitionMemberTypeNames(node.Ref) + if !ok { + return nil, fmt.Errorf("union type %s is not defined", r.definition.UnionTypeDefinitionNameString(node.Ref)) + } + return memberTypes, nil + default: + return nil, fmt.Errorf("invalid node kind: %s", node.Kind) + } +} + +func (r *rpcPlanningContext) getFragmentSelection(node ast.Node, selectionSetRef int) (*fragmentSelection, error) { + if node.Ref == ast.InvalidRef { + return nil, nil + } + + return &fragmentSelection{ + typeName: node.NameString(r.definition), + selectionSetRef: selectionSetRef, + }, nil +} + // setResolvedField sets the resolved field for a given field definition reference. func (r *rpcPlanningContext) setResolvedField(walker *astvisitor.Walker, fieldDefRef int, fieldArgs []int, fieldPath ast.Path, resolvedField *resolvedField) error { // We need to resolve the context fields for the given field definition reference. @@ -904,7 +955,7 @@ type resolveRPCCallConfig struct { fieldArgsMessage *RPCMessage } -func (r *rpcPlanningContext) resolveRequiredFields(typeName string, requiredFieldSelection int) (*RPCMessage, error) { +func (r *rpcPlanningContext) resolveRequiredFields(typeName string, resolvedField *resolvedField) (*RPCMessage, error) { message := &RPCMessage{ Name: typeName, } @@ -914,7 +965,29 @@ func (r *rpcPlanningContext) resolveRequiredFields(typeName string, requiredFiel return nil, fmt.Errorf("parent type node not found for type %s", typeName) } - fieldRefs := r.operation.SelectionSetFieldSelections(requiredFieldSelection) + // TODO: handle composite types. + if len(resolvedField.fragmentSelections) > 0 { + message.FieldSelectionSet = make(RPCFieldSelectionSet, len(resolvedField.fragmentSelections)) + message.OneOfType = resolvedField.fragmentType + message.MemberTypes = resolvedField.memberTypes + + for _, fragmentSelection := range resolvedField.fragmentSelections { + fields, err := r.buildCompositeField(parentTypeNode, fragmentSelection) + if err != nil { + return nil, err + } + + message.FieldSelectionSet[fragmentSelection.typeName] = fields + } + + return message, nil + } + + if resolvedField.fieldsSelectionSetRef == ast.InvalidRef { + return nil, errors.New("unable to resolve required fields: no fields selection set found") + } + + fieldRefs := r.operation.SelectionSetFieldSelections(resolvedField.fieldsSelectionSetRef) message.Fields = make(RPCFields, 0, len(fieldRefs)) for _, fieldRef := range fieldRefs { @@ -943,6 +1016,20 @@ func (r *rpcPlanningContext) resolveRequiredFields(typeName string, requiredFiel return message, nil } +func (r *rpcPlanningContext) buildCompositeField(parentNode ast.Node, fragmentSelection fragmentSelection) ([]RPCField, error) { + fieldRefs := r.operation.SelectionSetFieldRefs(fragmentSelection.selectionSetRef) + result := make([]RPCField, 0, len(fieldRefs)) + + for _, fieldRef := range fieldRefs { + field, err := r.buildField(parentNode, fieldRef, r.operation.FieldNameString(fieldRef), "") + if err != nil { + return nil, err + } + result = append(result, field) + } + return result, nil +} + // createResolverRPCCalls creates a new call for each resolved field. func (r *rpcPlanningContext) createResolverRPCCalls(subgraphName string, resolvedFields []resolvedField) ([]RPCCall, error) { // We need to create a new call for each resolved field. @@ -1034,7 +1121,7 @@ func (r *rpcPlanningContext) newResolveRPCCall(config *resolveRPCCallConfig) (RP var err error responseFieldsMessage, err = r.resolveRequiredFields( r.definition.ResolveTypeNameString(underlyingTypeRef), - resolvedField.fieldsSelectionSetRef, + resolvedField, ) if err != nil { diff --git a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_field_resolvers_test.go b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_field_resolvers_test.go index 658996d87..c4c16a3cc 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_field_resolvers_test.go +++ b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_field_resolvers_test.go @@ -1109,3 +1109,147 @@ func TestExecutionPlanFieldResolvers_WithNestedResolvers(t *testing.T) { }) } } + +func TestExecutionPlanFieldResolvers_WithOneOfTypes(t *testing.T) { + t.Parallel() + tests := []struct { + name string + query string + expectedPlan *RPCExecutionPlan + expectedError string + }{ + { + name: "Should create an execution plan for a query with interface type", + query: "query CategoriesWithNestedResolvers($includeValue: Boolean!) { categories { mascot(includeVolume: $includeVolume) { ... on Cat { name } ... on Dog { name } } } }", + expectedPlan: &RPCExecutionPlan{ + Calls: []RPCCall{ + { + ServiceName: "Products", + MethodName: "QueryCategories", + Request: RPCMessage{ + Name: "QueryCategoriesRequest", + }, + Response: RPCMessage{ + Name: "QueryCategoriesResponse", + Fields: []RPCField{ + { + Name: "categories", + ProtoTypeName: DataTypeMessage, + JSONPath: "categories", + Repeated: true, + Message: &RPCMessage{ + Name: "Category", + Fields: []RPCField{}, + }, + }, + }, + }, + }, + { + DependentCalls: []int{0}, + ServiceName: "Products", + MethodName: "ResolveCategoryMascot", + Kind: CallKindResolve, + ResponsePath: buildPath("categories.mascot"), + Request: RPCMessage{ + Name: "ResolveCategoryMascotRequest", + Fields: []RPCField{ + { + Name: "context", + ProtoTypeName: DataTypeMessage, + Repeated: true, + Message: &RPCMessage{ + Name: "ResolveCategoryMascotContext", + Fields: []RPCField{ + { + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", + ResolvePath: buildPath("categories.id"), + }, + { + Name: "kind", + ProtoTypeName: DataTypeEnum, + JSONPath: "kind", + EnumName: "CategoryKind", + ResolvePath: buildPath("categories.kind"), + }, + }, + }, + }, + { + Name: "field_args", + ProtoTypeName: DataTypeMessage, + Message: &RPCMessage{ + Name: "ResolveCategoryMascotArgs", + Fields: []RPCField{ + { + Name: "include_volume", + ProtoTypeName: DataTypeBool, + JSONPath: "includeVolume", + }, + }, + }, + }, + }, + }, + Response: RPCMessage{ + Name: "ResolveCategoryMascotResponse", + Fields: []RPCField{ + { + Name: "result", + ProtoTypeName: DataTypeMessage, + JSONPath: "result", + Repeated: true, + Message: &RPCMessage{ + Name: "ResolveCategoryMascotResult", + Fields: []RPCField{ + { + Name: "mascot", + ProtoTypeName: DataTypeMessage, + JSONPath: "mascot", + Optional: true, + Message: &RPCMessage{ + Name: "Animal", + OneOfType: OneOfTypeInterface, + MemberTypes: []string{"Cat", "Dog"}, + FieldSelectionSet: RPCFieldSelectionSet{ + "Cat": { + { + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", + }, + }, + "Dog": { + { + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + } + + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + t.Parallel() + runTest(t, testCase{ + query: tt.query, + expectedPlan: tt.expectedPlan, + expectedError: tt.expectedError, + }) + }) + } +} diff --git a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor.go b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor.go index eb83b9980..869b656a3 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor.go +++ b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor.go @@ -189,6 +189,30 @@ func (r *rpcPlanVisitor) EnterArgument(ref int) { } +func (r *rpcPlanVisitor) enterResolverCompositeSelectionSet(oneOfType OneOfType, selectionSetRef int) { + inlineFragmentSelections := r.operation.SelectionSetInlineFragmentSelections(selectionSetRef) + if len(inlineFragmentSelections) == 0 { + return + } + + ancestor := r.fieldResolverAncestors.peek() + for _, inlineFragmentSelectionRef := range inlineFragmentSelections { + inlineFragmentRef := r.operation.Selections[inlineFragmentSelectionRef].Ref + ss, ok := r.operation.InlineFragmentSelectionSet(inlineFragmentRef) + if !ok { + continue + } + + r.resolvedFields[ancestor].fragmentSelections = append(r.resolvedFields[ancestor].fragmentSelections, fragmentSelection{ + typeName: r.operation.InlineFragmentTypeConditionNameString(inlineFragmentRef), + selectionSetRef: ss, + }) + } + + // r.resolvedFields[ancestor].fragmentSelections = append(r.resolvedFields[ancestor].fragmentSelections, *fragmentSelection) + r.resolvedFields[ancestor].fragmentType = oneOfType +} + // EnterSelectionSet implements astvisitor.EnterSelectionSetVisitor. // Checks if this is in the root level below the operation definition. func (r *rpcPlanVisitor) EnterSelectionSet(ref int) { @@ -198,8 +222,26 @@ func (r *rpcPlanVisitor) EnterSelectionSet(ref int) { // If we are inside of a resolved field that selects multiple fields, we get all the fields from the input and pass them to the required fields visitor. if r.fieldResolverAncestors.len() > 0 { + if r.walker.Ancestor().Kind == ast.NodeKindInlineFragment { + return + } + + resolvedFieldAncestor := r.fieldResolverAncestors.peek() + if compositType := r.planCtx.getCompositeType(r.walker.EnclosingTypeDefinition); compositType != OneOfTypeNone { + memberTypes, err := r.planCtx.getMemberTypes(r.walker.EnclosingTypeDefinition) + if err != nil { + r.walker.StopWithInternalErr(err) + return + } + r.resolvedFields[resolvedFieldAncestor].memberTypes = memberTypes + r.resolvedFields[resolvedFieldAncestor].fieldsSelectionSetRef = ast.InvalidRef + + r.enterResolverCompositeSelectionSet(compositType, ref) + return + } + // TODO: handle nested resolved fields. - r.resolvedFields[r.fieldResolverAncestors.peek()].fieldsSelectionSetRef = ref + r.resolvedFields[resolvedFieldAncestor].fieldsSelectionSetRef = ref return } diff --git a/v2/pkg/engine/datasource/grpc_datasource/mapping_test_helper.go b/v2/pkg/engine/datasource/grpc_datasource/mapping_test_helper.go index 332730dac..d4039f4f9 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/mapping_test_helper.go +++ b/v2/pkg/engine/datasource/grpc_datasource/mapping_test_helper.go @@ -253,6 +253,28 @@ func testMapping() *GRPCMapping { Request: "ResolveCategoryCategoryMetricsRequest", Response: "ResolveCategoryCategoryMetricsResponse", }, + "mascot": { + FieldMappingData: FieldMapData{ + TargetName: "mascot", + ArgumentMappings: FieldArgumentMap{ + "includeVolume": "include_volume", + }, + }, + RPC: "ResolveCategoryMascot", + Request: "ResolveCategoryMascotRequest", + Response: "ResolveCategoryMascotResponse", + }, + "categoryStatus": { + FieldMappingData: FieldMapData{ + TargetName: "category_status", + ArgumentMappings: FieldArgumentMap{ + "checkHealth": "check_health", + }, + }, + RPC: "ResolveCategoryCategoryStatus", + Request: "ResolveCategoryCategoryStatusRequest", + Response: "ResolveCategoryCategoryStatusResponse", + }, }, "CategoryMetrics": { "normalizedScore": { @@ -809,6 +831,18 @@ func testMapping() *GRPCMapping { "metricType": "metric_type", }, }, + "mascot": { + TargetName: "mascot", + ArgumentMappings: FieldArgumentMap{ + "includeVolume": "include_volume", + }, + }, + "categoryStatus": { + TargetName: "category_status", + ArgumentMappings: FieldArgumentMap{ + "checkHealth": "check_health", + }, + }, }, "Subcategory": { "id": { diff --git a/v2/pkg/grpctest/mapping/mapping.go b/v2/pkg/grpctest/mapping/mapping.go index c5087f12c..95031912a 100644 --- a/v2/pkg/grpctest/mapping/mapping.go +++ b/v2/pkg/grpctest/mapping/mapping.go @@ -260,6 +260,28 @@ func DefaultGRPCMapping() *grpcdatasource.GRPCMapping { Request: "ResolveCategoryCategoryMetricsRequest", Response: "ResolveCategoryCategoryMetricsResponse", }, + "mascot": { + FieldMappingData: grpcdatasource.FieldMapData{ + TargetName: "mascot", + ArgumentMappings: grpcdatasource.FieldArgumentMap{ + "includeVolume": "include_volume", + }, + }, + RPC: "ResolveCategoryMascot", + Request: "ResolveCategoryMascotRequest", + Response: "ResolveCategoryMascotResponse", + }, + "categoryStatus": { + FieldMappingData: grpcdatasource.FieldMapData{ + TargetName: "category_status", + ArgumentMappings: grpcdatasource.FieldArgumentMap{ + "checkHealth": "check_health", + }, + }, + RPC: "ResolveCategoryCategoryStatus", + Request: "ResolveCategoryCategoryStatusRequest", + Response: "ResolveCategoryCategoryStatusResponse", + }, }, "CategoryMetrics": { "normalizedScore": { @@ -816,6 +838,18 @@ func DefaultGRPCMapping() *grpcdatasource.GRPCMapping { "metricType": "metric_type", }, }, + "mascot": { + TargetName: "mascot", + ArgumentMappings: grpcdatasource.FieldArgumentMap{ + "includeVolume": "include_volume", + }, + }, + "categoryStatus": { + TargetName: "category_status", + ArgumentMappings: grpcdatasource.FieldArgumentMap{ + "checkHealth": "check_health", + }, + }, }, "Subcategory": { "id": { diff --git a/v2/pkg/grpctest/mockservice.go b/v2/pkg/grpctest/mockservice.go index 1bfce37ce..9c5ddc413 100644 --- a/v2/pkg/grpctest/mockservice.go +++ b/v2/pkg/grpctest/mockservice.go @@ -46,6 +46,114 @@ func (s *MockService) ResolveCategoryMetricsNormalizedScore(_ context.Context, r return resp, nil } +// ResolveCategoryMascot implements productv1.ProductServiceServer. +func (s *MockService) ResolveCategoryMascot(_ context.Context, req *productv1.ResolveCategoryMascotRequest) (*productv1.ResolveCategoryMascotResponse, error) { + results := make([]*productv1.ResolveCategoryMascotResult, 0, len(req.GetContext())) + + includeVolume := false + if req.GetFieldArgs() != nil { + includeVolume = req.GetFieldArgs().GetIncludeVolume() + } + + for i, ctx := range req.GetContext() { + // Return nil for certain categories to test optional return + if ctx.GetKind() == productv1.CategoryKind_CATEGORY_KIND_OTHER { + results = append(results, &productv1.ResolveCategoryMascotResult{ + Mascot: nil, + }) + } else { + // Alternate between Cat and Dog based on category kind + var animal *productv1.Animal + if ctx.GetKind() == productv1.CategoryKind_CATEGORY_KIND_BOOK || ctx.GetKind() == productv1.CategoryKind_CATEGORY_KIND_ELECTRONICS { + volume := int32(0) + if includeVolume { + volume = int32(i*10 + 5) + } + animal = &productv1.Animal{ + Instance: &productv1.Animal_Cat{ + Cat: &productv1.Cat{ + Id: fmt.Sprintf("cat-mascot-%s", ctx.GetId()), + Name: fmt.Sprintf("Whiskers-%s", ctx.GetId()), + Kind: "Cat", + MeowVolume: volume, + }, + }, + } + } else { + volume := int32(0) + if includeVolume { + volume = int32(i*10 + 10) + } + animal = &productv1.Animal{ + Instance: &productv1.Animal_Dog{ + Dog: &productv1.Dog{ + Id: fmt.Sprintf("dog-mascot-%s", ctx.GetId()), + Name: fmt.Sprintf("Buddy-%s", ctx.GetId()), + Kind: "Dog", + BarkVolume: volume, + }, + }, + } + } + results = append(results, &productv1.ResolveCategoryMascotResult{ + Mascot: animal, + }) + } + } + + resp := &productv1.ResolveCategoryMascotResponse{ + Result: results, + } + + return resp, nil +} + +// ResolveCategoryCategoryStatus implements productv1.ProductServiceServer. +func (s *MockService) ResolveCategoryCategoryStatus(_ context.Context, req *productv1.ResolveCategoryCategoryStatusRequest) (*productv1.ResolveCategoryCategoryStatusResponse, error) { + results := make([]*productv1.ResolveCategoryCategoryStatusResult, 0, len(req.GetContext())) + + checkHealth := false + if req.GetFieldArgs() != nil { + checkHealth = req.GetFieldArgs().GetCheckHealth() + } + + for i, ctx := range req.GetContext() { + var actionResult *productv1.ActionResult + + if checkHealth && i%3 == 0 { + // Return error status for health check failures + actionResult = &productv1.ActionResult{ + Value: &productv1.ActionResult_ActionError{ + ActionError: &productv1.ActionError{ + Message: fmt.Sprintf("Health check failed for category %s", ctx.GetName()), + Code: "HEALTH_CHECK_FAILED", + }, + }, + } + } else { + // Return success status + actionResult = &productv1.ActionResult{ + Value: &productv1.ActionResult_ActionSuccess{ + ActionSuccess: &productv1.ActionSuccess{ + Message: fmt.Sprintf("Category %s is healthy", ctx.GetName()), + Timestamp: "2024-01-01T00:00:00Z", + }, + }, + } + } + + results = append(results, &productv1.ResolveCategoryCategoryStatusResult{ + CategoryStatus: actionResult, + }) + } + + resp := &productv1.ResolveCategoryCategoryStatusResponse{ + Result: results, + } + + return resp, nil +} + // ResolveProductRecommendedCategory implements productv1.ProductServiceServer. func (s *MockService) ResolveProductRecommendedCategory(_ context.Context, req *productv1.ResolveProductRecommendedCategoryRequest) (*productv1.ResolveProductRecommendedCategoryResponse, error) { results := make([]*productv1.ResolveProductRecommendedCategoryResult, 0, len(req.GetContext())) diff --git a/v2/pkg/grpctest/product.proto b/v2/pkg/grpctest/product.proto index 4efeffbb2..f4792611b 100644 --- a/v2/pkg/grpctest/product.proto +++ b/v2/pkg/grpctest/product.proto @@ -56,6 +56,8 @@ service ProductService { rpc QueryUser(QueryUserRequest) returns (QueryUserResponse) {} rpc QueryUsers(QueryUsersRequest) returns (QueryUsersResponse) {} rpc ResolveCategoryCategoryMetrics(ResolveCategoryCategoryMetricsRequest) returns (ResolveCategoryCategoryMetricsResponse) {} + rpc ResolveCategoryCategoryStatus(ResolveCategoryCategoryStatusRequest) returns (ResolveCategoryCategoryStatusResponse) {} + rpc ResolveCategoryMascot(ResolveCategoryMascotRequest) returns (ResolveCategoryMascotResponse) {} rpc ResolveCategoryMetricsNormalizedScore(ResolveCategoryMetricsNormalizedScoreRequest) returns (ResolveCategoryMetricsNormalizedScoreResponse) {} rpc ResolveCategoryPopularityScore(ResolveCategoryPopularityScoreRequest) returns (ResolveCategoryPopularityScoreResponse) {} rpc ResolveCategoryProductCount(ResolveCategoryProductCountRequest) returns (ResolveCategoryProductCountResponse) {} @@ -753,6 +755,54 @@ message ResolveCategoryCategoryMetricsResponse { repeated ResolveCategoryCategoryMetricsResult result = 1; } +message ResolveCategoryMascotArgs { + bool include_volume = 1; +} + +message ResolveCategoryMascotContext { + string id = 1; + CategoryKind kind = 2; +} + +message ResolveCategoryMascotRequest { + // context provides the resolver context for the field mascot of type Category. + repeated ResolveCategoryMascotContext context = 1; + // field_args provides the arguments for the resolver field mascot of type Category. + ResolveCategoryMascotArgs field_args = 2; +} + +message ResolveCategoryMascotResult { + Animal mascot = 1; +} + +message ResolveCategoryMascotResponse { + repeated ResolveCategoryMascotResult result = 1; +} + +message ResolveCategoryCategoryStatusArgs { + bool check_health = 1; +} + +message ResolveCategoryCategoryStatusContext { + string id = 1; + string name = 2; +} + +message ResolveCategoryCategoryStatusRequest { + // context provides the resolver context for the field categoryStatus of type Category. + repeated ResolveCategoryCategoryStatusContext context = 1; + // field_args provides the arguments for the resolver field categoryStatus of type Category. + ResolveCategoryCategoryStatusArgs field_args = 2; +} + +message ResolveCategoryCategoryStatusResult { + ActionResult category_status = 1; +} + +message ResolveCategoryCategoryStatusResponse { + repeated ResolveCategoryCategoryStatusResult result = 1; +} + message ResolveSubcategoryItemCountArgs { SubcategoryItemFilter filters = 1; } diff --git a/v2/pkg/grpctest/productv1/product.pb.go b/v2/pkg/grpctest/productv1/product.pb.go index c8062850a..c4e4f4fea 100644 --- a/v2/pkg/grpctest/productv1/product.pb.go +++ b/v2/pkg/grpctest/productv1/product.pb.go @@ -6380,6 +6380,482 @@ func (x *ResolveCategoryCategoryMetricsResponse) GetResult() []*ResolveCategoryC return nil } +type ResolveCategoryMascotArgs struct { + state protoimpl.MessageState `protogen:"open.v1"` + IncludeVolume bool `protobuf:"varint,1,opt,name=include_volume,json=includeVolume,proto3" json:"include_volume,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ResolveCategoryMascotArgs) Reset() { + *x = ResolveCategoryMascotArgs{} + mi := &file_product_proto_msgTypes[138] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ResolveCategoryMascotArgs) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ResolveCategoryMascotArgs) ProtoMessage() {} + +func (x *ResolveCategoryMascotArgs) ProtoReflect() protoreflect.Message { + mi := &file_product_proto_msgTypes[138] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ResolveCategoryMascotArgs.ProtoReflect.Descriptor instead. +func (*ResolveCategoryMascotArgs) Descriptor() ([]byte, []int) { + return file_product_proto_rawDescGZIP(), []int{138} +} + +func (x *ResolveCategoryMascotArgs) GetIncludeVolume() bool { + if x != nil { + return x.IncludeVolume + } + return false +} + +type ResolveCategoryMascotContext struct { + state protoimpl.MessageState `protogen:"open.v1"` + Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` + Kind CategoryKind `protobuf:"varint,2,opt,name=kind,proto3,enum=productv1.CategoryKind" json:"kind,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ResolveCategoryMascotContext) Reset() { + *x = ResolveCategoryMascotContext{} + mi := &file_product_proto_msgTypes[139] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ResolveCategoryMascotContext) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ResolveCategoryMascotContext) ProtoMessage() {} + +func (x *ResolveCategoryMascotContext) ProtoReflect() protoreflect.Message { + mi := &file_product_proto_msgTypes[139] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ResolveCategoryMascotContext.ProtoReflect.Descriptor instead. +func (*ResolveCategoryMascotContext) Descriptor() ([]byte, []int) { + return file_product_proto_rawDescGZIP(), []int{139} +} + +func (x *ResolveCategoryMascotContext) GetId() string { + if x != nil { + return x.Id + } + return "" +} + +func (x *ResolveCategoryMascotContext) GetKind() CategoryKind { + if x != nil { + return x.Kind + } + return CategoryKind_CATEGORY_KIND_UNSPECIFIED +} + +type ResolveCategoryMascotRequest struct { + state protoimpl.MessageState `protogen:"open.v1"` + // context provides the resolver context for the field mascot of type Category. + Context []*ResolveCategoryMascotContext `protobuf:"bytes,1,rep,name=context,proto3" json:"context,omitempty"` + // field_args provides the arguments for the resolver field mascot of type Category. + FieldArgs *ResolveCategoryMascotArgs `protobuf:"bytes,2,opt,name=field_args,json=fieldArgs,proto3" json:"field_args,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ResolveCategoryMascotRequest) Reset() { + *x = ResolveCategoryMascotRequest{} + mi := &file_product_proto_msgTypes[140] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ResolveCategoryMascotRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ResolveCategoryMascotRequest) ProtoMessage() {} + +func (x *ResolveCategoryMascotRequest) ProtoReflect() protoreflect.Message { + mi := &file_product_proto_msgTypes[140] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ResolveCategoryMascotRequest.ProtoReflect.Descriptor instead. +func (*ResolveCategoryMascotRequest) Descriptor() ([]byte, []int) { + return file_product_proto_rawDescGZIP(), []int{140} +} + +func (x *ResolveCategoryMascotRequest) GetContext() []*ResolveCategoryMascotContext { + if x != nil { + return x.Context + } + return nil +} + +func (x *ResolveCategoryMascotRequest) GetFieldArgs() *ResolveCategoryMascotArgs { + if x != nil { + return x.FieldArgs + } + return nil +} + +type ResolveCategoryMascotResult struct { + state protoimpl.MessageState `protogen:"open.v1"` + Mascot *Animal `protobuf:"bytes,1,opt,name=mascot,proto3" json:"mascot,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ResolveCategoryMascotResult) Reset() { + *x = ResolveCategoryMascotResult{} + mi := &file_product_proto_msgTypes[141] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ResolveCategoryMascotResult) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ResolveCategoryMascotResult) ProtoMessage() {} + +func (x *ResolveCategoryMascotResult) ProtoReflect() protoreflect.Message { + mi := &file_product_proto_msgTypes[141] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ResolveCategoryMascotResult.ProtoReflect.Descriptor instead. +func (*ResolveCategoryMascotResult) Descriptor() ([]byte, []int) { + return file_product_proto_rawDescGZIP(), []int{141} +} + +func (x *ResolveCategoryMascotResult) GetMascot() *Animal { + if x != nil { + return x.Mascot + } + return nil +} + +type ResolveCategoryMascotResponse struct { + state protoimpl.MessageState `protogen:"open.v1"` + Result []*ResolveCategoryMascotResult `protobuf:"bytes,1,rep,name=result,proto3" json:"result,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ResolveCategoryMascotResponse) Reset() { + *x = ResolveCategoryMascotResponse{} + mi := &file_product_proto_msgTypes[142] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ResolveCategoryMascotResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ResolveCategoryMascotResponse) ProtoMessage() {} + +func (x *ResolveCategoryMascotResponse) ProtoReflect() protoreflect.Message { + mi := &file_product_proto_msgTypes[142] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ResolveCategoryMascotResponse.ProtoReflect.Descriptor instead. +func (*ResolveCategoryMascotResponse) Descriptor() ([]byte, []int) { + return file_product_proto_rawDescGZIP(), []int{142} +} + +func (x *ResolveCategoryMascotResponse) GetResult() []*ResolveCategoryMascotResult { + if x != nil { + return x.Result + } + return nil +} + +type ResolveCategoryCategoryStatusArgs struct { + state protoimpl.MessageState `protogen:"open.v1"` + CheckHealth bool `protobuf:"varint,1,opt,name=check_health,json=checkHealth,proto3" json:"check_health,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ResolveCategoryCategoryStatusArgs) Reset() { + *x = ResolveCategoryCategoryStatusArgs{} + mi := &file_product_proto_msgTypes[143] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ResolveCategoryCategoryStatusArgs) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ResolveCategoryCategoryStatusArgs) ProtoMessage() {} + +func (x *ResolveCategoryCategoryStatusArgs) ProtoReflect() protoreflect.Message { + mi := &file_product_proto_msgTypes[143] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ResolveCategoryCategoryStatusArgs.ProtoReflect.Descriptor instead. +func (*ResolveCategoryCategoryStatusArgs) Descriptor() ([]byte, []int) { + return file_product_proto_rawDescGZIP(), []int{143} +} + +func (x *ResolveCategoryCategoryStatusArgs) GetCheckHealth() bool { + if x != nil { + return x.CheckHealth + } + return false +} + +type ResolveCategoryCategoryStatusContext struct { + state protoimpl.MessageState `protogen:"open.v1"` + Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` + Name string `protobuf:"bytes,2,opt,name=name,proto3" json:"name,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ResolveCategoryCategoryStatusContext) Reset() { + *x = ResolveCategoryCategoryStatusContext{} + mi := &file_product_proto_msgTypes[144] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ResolveCategoryCategoryStatusContext) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ResolveCategoryCategoryStatusContext) ProtoMessage() {} + +func (x *ResolveCategoryCategoryStatusContext) ProtoReflect() protoreflect.Message { + mi := &file_product_proto_msgTypes[144] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ResolveCategoryCategoryStatusContext.ProtoReflect.Descriptor instead. +func (*ResolveCategoryCategoryStatusContext) Descriptor() ([]byte, []int) { + return file_product_proto_rawDescGZIP(), []int{144} +} + +func (x *ResolveCategoryCategoryStatusContext) GetId() string { + if x != nil { + return x.Id + } + return "" +} + +func (x *ResolveCategoryCategoryStatusContext) GetName() string { + if x != nil { + return x.Name + } + return "" +} + +type ResolveCategoryCategoryStatusRequest struct { + state protoimpl.MessageState `protogen:"open.v1"` + // context provides the resolver context for the field categoryStatus of type Category. + Context []*ResolveCategoryCategoryStatusContext `protobuf:"bytes,1,rep,name=context,proto3" json:"context,omitempty"` + // field_args provides the arguments for the resolver field categoryStatus of type Category. + FieldArgs *ResolveCategoryCategoryStatusArgs `protobuf:"bytes,2,opt,name=field_args,json=fieldArgs,proto3" json:"field_args,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ResolveCategoryCategoryStatusRequest) Reset() { + *x = ResolveCategoryCategoryStatusRequest{} + mi := &file_product_proto_msgTypes[145] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ResolveCategoryCategoryStatusRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ResolveCategoryCategoryStatusRequest) ProtoMessage() {} + +func (x *ResolveCategoryCategoryStatusRequest) ProtoReflect() protoreflect.Message { + mi := &file_product_proto_msgTypes[145] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ResolveCategoryCategoryStatusRequest.ProtoReflect.Descriptor instead. +func (*ResolveCategoryCategoryStatusRequest) Descriptor() ([]byte, []int) { + return file_product_proto_rawDescGZIP(), []int{145} +} + +func (x *ResolveCategoryCategoryStatusRequest) GetContext() []*ResolveCategoryCategoryStatusContext { + if x != nil { + return x.Context + } + return nil +} + +func (x *ResolveCategoryCategoryStatusRequest) GetFieldArgs() *ResolveCategoryCategoryStatusArgs { + if x != nil { + return x.FieldArgs + } + return nil +} + +type ResolveCategoryCategoryStatusResult struct { + state protoimpl.MessageState `protogen:"open.v1"` + CategoryStatus *ActionResult `protobuf:"bytes,1,opt,name=category_status,json=categoryStatus,proto3" json:"category_status,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ResolveCategoryCategoryStatusResult) Reset() { + *x = ResolveCategoryCategoryStatusResult{} + mi := &file_product_proto_msgTypes[146] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ResolveCategoryCategoryStatusResult) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ResolveCategoryCategoryStatusResult) ProtoMessage() {} + +func (x *ResolveCategoryCategoryStatusResult) ProtoReflect() protoreflect.Message { + mi := &file_product_proto_msgTypes[146] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ResolveCategoryCategoryStatusResult.ProtoReflect.Descriptor instead. +func (*ResolveCategoryCategoryStatusResult) Descriptor() ([]byte, []int) { + return file_product_proto_rawDescGZIP(), []int{146} +} + +func (x *ResolveCategoryCategoryStatusResult) GetCategoryStatus() *ActionResult { + if x != nil { + return x.CategoryStatus + } + return nil +} + +type ResolveCategoryCategoryStatusResponse struct { + state protoimpl.MessageState `protogen:"open.v1"` + Result []*ResolveCategoryCategoryStatusResult `protobuf:"bytes,1,rep,name=result,proto3" json:"result,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ResolveCategoryCategoryStatusResponse) Reset() { + *x = ResolveCategoryCategoryStatusResponse{} + mi := &file_product_proto_msgTypes[147] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ResolveCategoryCategoryStatusResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ResolveCategoryCategoryStatusResponse) ProtoMessage() {} + +func (x *ResolveCategoryCategoryStatusResponse) ProtoReflect() protoreflect.Message { + mi := &file_product_proto_msgTypes[147] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ResolveCategoryCategoryStatusResponse.ProtoReflect.Descriptor instead. +func (*ResolveCategoryCategoryStatusResponse) Descriptor() ([]byte, []int) { + return file_product_proto_rawDescGZIP(), []int{147} +} + +func (x *ResolveCategoryCategoryStatusResponse) GetResult() []*ResolveCategoryCategoryStatusResult { + if x != nil { + return x.Result + } + return nil +} + type ResolveSubcategoryItemCountArgs struct { state protoimpl.MessageState `protogen:"open.v1"` Filters *SubcategoryItemFilter `protobuf:"bytes,1,opt,name=filters,proto3" json:"filters,omitempty"` @@ -6389,7 +6865,7 @@ type ResolveSubcategoryItemCountArgs struct { func (x *ResolveSubcategoryItemCountArgs) Reset() { *x = ResolveSubcategoryItemCountArgs{} - mi := &file_product_proto_msgTypes[138] + mi := &file_product_proto_msgTypes[148] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6401,7 +6877,7 @@ func (x *ResolveSubcategoryItemCountArgs) String() string { func (*ResolveSubcategoryItemCountArgs) ProtoMessage() {} func (x *ResolveSubcategoryItemCountArgs) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[138] + mi := &file_product_proto_msgTypes[148] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6414,7 +6890,7 @@ func (x *ResolveSubcategoryItemCountArgs) ProtoReflect() protoreflect.Message { // Deprecated: Use ResolveSubcategoryItemCountArgs.ProtoReflect.Descriptor instead. func (*ResolveSubcategoryItemCountArgs) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{138} + return file_product_proto_rawDescGZIP(), []int{148} } func (x *ResolveSubcategoryItemCountArgs) GetFilters() *SubcategoryItemFilter { @@ -6433,7 +6909,7 @@ type ResolveSubcategoryItemCountContext struct { func (x *ResolveSubcategoryItemCountContext) Reset() { *x = ResolveSubcategoryItemCountContext{} - mi := &file_product_proto_msgTypes[139] + mi := &file_product_proto_msgTypes[149] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6445,7 +6921,7 @@ func (x *ResolveSubcategoryItemCountContext) String() string { func (*ResolveSubcategoryItemCountContext) ProtoMessage() {} func (x *ResolveSubcategoryItemCountContext) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[139] + mi := &file_product_proto_msgTypes[149] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6458,7 +6934,7 @@ func (x *ResolveSubcategoryItemCountContext) ProtoReflect() protoreflect.Message // Deprecated: Use ResolveSubcategoryItemCountContext.ProtoReflect.Descriptor instead. func (*ResolveSubcategoryItemCountContext) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{139} + return file_product_proto_rawDescGZIP(), []int{149} } func (x *ResolveSubcategoryItemCountContext) GetId() string { @@ -6480,7 +6956,7 @@ type ResolveSubcategoryItemCountRequest struct { func (x *ResolveSubcategoryItemCountRequest) Reset() { *x = ResolveSubcategoryItemCountRequest{} - mi := &file_product_proto_msgTypes[140] + mi := &file_product_proto_msgTypes[150] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6492,7 +6968,7 @@ func (x *ResolveSubcategoryItemCountRequest) String() string { func (*ResolveSubcategoryItemCountRequest) ProtoMessage() {} func (x *ResolveSubcategoryItemCountRequest) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[140] + mi := &file_product_proto_msgTypes[150] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6505,7 +6981,7 @@ func (x *ResolveSubcategoryItemCountRequest) ProtoReflect() protoreflect.Message // Deprecated: Use ResolveSubcategoryItemCountRequest.ProtoReflect.Descriptor instead. func (*ResolveSubcategoryItemCountRequest) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{140} + return file_product_proto_rawDescGZIP(), []int{150} } func (x *ResolveSubcategoryItemCountRequest) GetContext() []*ResolveSubcategoryItemCountContext { @@ -6531,7 +7007,7 @@ type ResolveSubcategoryItemCountResult struct { func (x *ResolveSubcategoryItemCountResult) Reset() { *x = ResolveSubcategoryItemCountResult{} - mi := &file_product_proto_msgTypes[141] + mi := &file_product_proto_msgTypes[151] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6543,7 +7019,7 @@ func (x *ResolveSubcategoryItemCountResult) String() string { func (*ResolveSubcategoryItemCountResult) ProtoMessage() {} func (x *ResolveSubcategoryItemCountResult) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[141] + mi := &file_product_proto_msgTypes[151] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6556,7 +7032,7 @@ func (x *ResolveSubcategoryItemCountResult) ProtoReflect() protoreflect.Message // Deprecated: Use ResolveSubcategoryItemCountResult.ProtoReflect.Descriptor instead. func (*ResolveSubcategoryItemCountResult) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{141} + return file_product_proto_rawDescGZIP(), []int{151} } func (x *ResolveSubcategoryItemCountResult) GetItemCount() int32 { @@ -6575,7 +7051,7 @@ type ResolveSubcategoryItemCountResponse struct { func (x *ResolveSubcategoryItemCountResponse) Reset() { *x = ResolveSubcategoryItemCountResponse{} - mi := &file_product_proto_msgTypes[142] + mi := &file_product_proto_msgTypes[152] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6587,7 +7063,7 @@ func (x *ResolveSubcategoryItemCountResponse) String() string { func (*ResolveSubcategoryItemCountResponse) ProtoMessage() {} func (x *ResolveSubcategoryItemCountResponse) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[142] + mi := &file_product_proto_msgTypes[152] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6600,7 +7076,7 @@ func (x *ResolveSubcategoryItemCountResponse) ProtoReflect() protoreflect.Messag // Deprecated: Use ResolveSubcategoryItemCountResponse.ProtoReflect.Descriptor instead. func (*ResolveSubcategoryItemCountResponse) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{142} + return file_product_proto_rawDescGZIP(), []int{152} } func (x *ResolveSubcategoryItemCountResponse) GetResult() []*ResolveSubcategoryItemCountResult { @@ -6619,7 +7095,7 @@ type ResolveCategoryMetricsNormalizedScoreArgs struct { func (x *ResolveCategoryMetricsNormalizedScoreArgs) Reset() { *x = ResolveCategoryMetricsNormalizedScoreArgs{} - mi := &file_product_proto_msgTypes[143] + mi := &file_product_proto_msgTypes[153] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6631,7 +7107,7 @@ func (x *ResolveCategoryMetricsNormalizedScoreArgs) String() string { func (*ResolveCategoryMetricsNormalizedScoreArgs) ProtoMessage() {} func (x *ResolveCategoryMetricsNormalizedScoreArgs) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[143] + mi := &file_product_proto_msgTypes[153] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6644,7 +7120,7 @@ func (x *ResolveCategoryMetricsNormalizedScoreArgs) ProtoReflect() protoreflect. // Deprecated: Use ResolveCategoryMetricsNormalizedScoreArgs.ProtoReflect.Descriptor instead. func (*ResolveCategoryMetricsNormalizedScoreArgs) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{143} + return file_product_proto_rawDescGZIP(), []int{153} } func (x *ResolveCategoryMetricsNormalizedScoreArgs) GetBaseline() float64 { @@ -6665,7 +7141,7 @@ type ResolveCategoryMetricsNormalizedScoreContext struct { func (x *ResolveCategoryMetricsNormalizedScoreContext) Reset() { *x = ResolveCategoryMetricsNormalizedScoreContext{} - mi := &file_product_proto_msgTypes[144] + mi := &file_product_proto_msgTypes[154] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6677,7 +7153,7 @@ func (x *ResolveCategoryMetricsNormalizedScoreContext) String() string { func (*ResolveCategoryMetricsNormalizedScoreContext) ProtoMessage() {} func (x *ResolveCategoryMetricsNormalizedScoreContext) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[144] + mi := &file_product_proto_msgTypes[154] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6690,7 +7166,7 @@ func (x *ResolveCategoryMetricsNormalizedScoreContext) ProtoReflect() protorefle // Deprecated: Use ResolveCategoryMetricsNormalizedScoreContext.ProtoReflect.Descriptor instead. func (*ResolveCategoryMetricsNormalizedScoreContext) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{144} + return file_product_proto_rawDescGZIP(), []int{154} } func (x *ResolveCategoryMetricsNormalizedScoreContext) GetId() string { @@ -6726,7 +7202,7 @@ type ResolveCategoryMetricsNormalizedScoreRequest struct { func (x *ResolveCategoryMetricsNormalizedScoreRequest) Reset() { *x = ResolveCategoryMetricsNormalizedScoreRequest{} - mi := &file_product_proto_msgTypes[145] + mi := &file_product_proto_msgTypes[155] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6738,7 +7214,7 @@ func (x *ResolveCategoryMetricsNormalizedScoreRequest) String() string { func (*ResolveCategoryMetricsNormalizedScoreRequest) ProtoMessage() {} func (x *ResolveCategoryMetricsNormalizedScoreRequest) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[145] + mi := &file_product_proto_msgTypes[155] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6751,7 +7227,7 @@ func (x *ResolveCategoryMetricsNormalizedScoreRequest) ProtoReflect() protorefle // Deprecated: Use ResolveCategoryMetricsNormalizedScoreRequest.ProtoReflect.Descriptor instead. func (*ResolveCategoryMetricsNormalizedScoreRequest) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{145} + return file_product_proto_rawDescGZIP(), []int{155} } func (x *ResolveCategoryMetricsNormalizedScoreRequest) GetContext() []*ResolveCategoryMetricsNormalizedScoreContext { @@ -6777,7 +7253,7 @@ type ResolveCategoryMetricsNormalizedScoreResult struct { func (x *ResolveCategoryMetricsNormalizedScoreResult) Reset() { *x = ResolveCategoryMetricsNormalizedScoreResult{} - mi := &file_product_proto_msgTypes[146] + mi := &file_product_proto_msgTypes[156] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6789,7 +7265,7 @@ func (x *ResolveCategoryMetricsNormalizedScoreResult) String() string { func (*ResolveCategoryMetricsNormalizedScoreResult) ProtoMessage() {} func (x *ResolveCategoryMetricsNormalizedScoreResult) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[146] + mi := &file_product_proto_msgTypes[156] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6802,7 +7278,7 @@ func (x *ResolveCategoryMetricsNormalizedScoreResult) ProtoReflect() protoreflec // Deprecated: Use ResolveCategoryMetricsNormalizedScoreResult.ProtoReflect.Descriptor instead. func (*ResolveCategoryMetricsNormalizedScoreResult) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{146} + return file_product_proto_rawDescGZIP(), []int{156} } func (x *ResolveCategoryMetricsNormalizedScoreResult) GetNormalizedScore() float64 { @@ -6821,7 +7297,7 @@ type ResolveCategoryMetricsNormalizedScoreResponse struct { func (x *ResolveCategoryMetricsNormalizedScoreResponse) Reset() { *x = ResolveCategoryMetricsNormalizedScoreResponse{} - mi := &file_product_proto_msgTypes[147] + mi := &file_product_proto_msgTypes[157] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6833,7 +7309,7 @@ func (x *ResolveCategoryMetricsNormalizedScoreResponse) String() string { func (*ResolveCategoryMetricsNormalizedScoreResponse) ProtoMessage() {} func (x *ResolveCategoryMetricsNormalizedScoreResponse) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[147] + mi := &file_product_proto_msgTypes[157] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6846,7 +7322,7 @@ func (x *ResolveCategoryMetricsNormalizedScoreResponse) ProtoReflect() protorefl // Deprecated: Use ResolveCategoryMetricsNormalizedScoreResponse.ProtoReflect.Descriptor instead. func (*ResolveCategoryMetricsNormalizedScoreResponse) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{147} + return file_product_proto_rawDescGZIP(), []int{157} } func (x *ResolveCategoryMetricsNormalizedScoreResponse) GetResult() []*ResolveCategoryMetricsNormalizedScoreResult { @@ -6867,7 +7343,7 @@ type Product struct { func (x *Product) Reset() { *x = Product{} - mi := &file_product_proto_msgTypes[148] + mi := &file_product_proto_msgTypes[158] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6879,7 +7355,7 @@ func (x *Product) String() string { func (*Product) ProtoMessage() {} func (x *Product) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[148] + mi := &file_product_proto_msgTypes[158] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6892,7 +7368,7 @@ func (x *Product) ProtoReflect() protoreflect.Message { // Deprecated: Use Product.ProtoReflect.Descriptor instead. func (*Product) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{148} + return file_product_proto_rawDescGZIP(), []int{158} } func (x *Product) GetId() string { @@ -6927,7 +7403,7 @@ type Storage struct { func (x *Storage) Reset() { *x = Storage{} - mi := &file_product_proto_msgTypes[149] + mi := &file_product_proto_msgTypes[159] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6939,7 +7415,7 @@ func (x *Storage) String() string { func (*Storage) ProtoMessage() {} func (x *Storage) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[149] + mi := &file_product_proto_msgTypes[159] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6952,7 +7428,7 @@ func (x *Storage) ProtoReflect() protoreflect.Message { // Deprecated: Use Storage.ProtoReflect.Descriptor instead. func (*Storage) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{149} + return file_product_proto_rawDescGZIP(), []int{159} } func (x *Storage) GetId() string { @@ -6987,7 +7463,7 @@ type Warehouse struct { func (x *Warehouse) Reset() { *x = Warehouse{} - mi := &file_product_proto_msgTypes[150] + mi := &file_product_proto_msgTypes[160] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6999,7 +7475,7 @@ func (x *Warehouse) String() string { func (*Warehouse) ProtoMessage() {} func (x *Warehouse) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[150] + mi := &file_product_proto_msgTypes[160] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7012,7 +7488,7 @@ func (x *Warehouse) ProtoReflect() protoreflect.Message { // Deprecated: Use Warehouse.ProtoReflect.Descriptor instead. func (*Warehouse) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{150} + return file_product_proto_rawDescGZIP(), []int{160} } func (x *Warehouse) GetId() string { @@ -7046,7 +7522,7 @@ type User struct { func (x *User) Reset() { *x = User{} - mi := &file_product_proto_msgTypes[151] + mi := &file_product_proto_msgTypes[161] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7058,7 +7534,7 @@ func (x *User) String() string { func (*User) ProtoMessage() {} func (x *User) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[151] + mi := &file_product_proto_msgTypes[161] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7071,7 +7547,7 @@ func (x *User) ProtoReflect() protoreflect.Message { // Deprecated: Use User.ProtoReflect.Descriptor instead. func (*User) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{151} + return file_product_proto_rawDescGZIP(), []int{161} } func (x *User) GetId() string { @@ -7099,7 +7575,7 @@ type NestedTypeA struct { func (x *NestedTypeA) Reset() { *x = NestedTypeA{} - mi := &file_product_proto_msgTypes[152] + mi := &file_product_proto_msgTypes[162] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7111,7 +7587,7 @@ func (x *NestedTypeA) String() string { func (*NestedTypeA) ProtoMessage() {} func (x *NestedTypeA) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[152] + mi := &file_product_proto_msgTypes[162] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7124,7 +7600,7 @@ func (x *NestedTypeA) ProtoReflect() protoreflect.Message { // Deprecated: Use NestedTypeA.ProtoReflect.Descriptor instead. func (*NestedTypeA) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{152} + return file_product_proto_rawDescGZIP(), []int{162} } func (x *NestedTypeA) GetId() string { @@ -7159,7 +7635,7 @@ type RecursiveType struct { func (x *RecursiveType) Reset() { *x = RecursiveType{} - mi := &file_product_proto_msgTypes[153] + mi := &file_product_proto_msgTypes[163] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7171,7 +7647,7 @@ func (x *RecursiveType) String() string { func (*RecursiveType) ProtoMessage() {} func (x *RecursiveType) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[153] + mi := &file_product_proto_msgTypes[163] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7184,7 +7660,7 @@ func (x *RecursiveType) ProtoReflect() protoreflect.Message { // Deprecated: Use RecursiveType.ProtoReflect.Descriptor instead. func (*RecursiveType) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{153} + return file_product_proto_rawDescGZIP(), []int{163} } func (x *RecursiveType) GetId() string { @@ -7220,7 +7696,7 @@ type TypeWithMultipleFilterFields struct { func (x *TypeWithMultipleFilterFields) Reset() { *x = TypeWithMultipleFilterFields{} - mi := &file_product_proto_msgTypes[154] + mi := &file_product_proto_msgTypes[164] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7232,7 +7708,7 @@ func (x *TypeWithMultipleFilterFields) String() string { func (*TypeWithMultipleFilterFields) ProtoMessage() {} func (x *TypeWithMultipleFilterFields) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[154] + mi := &file_product_proto_msgTypes[164] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7245,7 +7721,7 @@ func (x *TypeWithMultipleFilterFields) ProtoReflect() protoreflect.Message { // Deprecated: Use TypeWithMultipleFilterFields.ProtoReflect.Descriptor instead. func (*TypeWithMultipleFilterFields) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{154} + return file_product_proto_rawDescGZIP(), []int{164} } func (x *TypeWithMultipleFilterFields) GetId() string { @@ -7286,7 +7762,7 @@ type FilterTypeInput struct { func (x *FilterTypeInput) Reset() { *x = FilterTypeInput{} - mi := &file_product_proto_msgTypes[155] + mi := &file_product_proto_msgTypes[165] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7298,7 +7774,7 @@ func (x *FilterTypeInput) String() string { func (*FilterTypeInput) ProtoMessage() {} func (x *FilterTypeInput) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[155] + mi := &file_product_proto_msgTypes[165] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7311,7 +7787,7 @@ func (x *FilterTypeInput) ProtoReflect() protoreflect.Message { // Deprecated: Use FilterTypeInput.ProtoReflect.Descriptor instead. func (*FilterTypeInput) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{155} + return file_product_proto_rawDescGZIP(), []int{165} } func (x *FilterTypeInput) GetFilterField_1() string { @@ -7337,7 +7813,7 @@ type ComplexFilterTypeInput struct { func (x *ComplexFilterTypeInput) Reset() { *x = ComplexFilterTypeInput{} - mi := &file_product_proto_msgTypes[156] + mi := &file_product_proto_msgTypes[166] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7349,7 +7825,7 @@ func (x *ComplexFilterTypeInput) String() string { func (*ComplexFilterTypeInput) ProtoMessage() {} func (x *ComplexFilterTypeInput) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[156] + mi := &file_product_proto_msgTypes[166] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7362,7 +7838,7 @@ func (x *ComplexFilterTypeInput) ProtoReflect() protoreflect.Message { // Deprecated: Use ComplexFilterTypeInput.ProtoReflect.Descriptor instead. func (*ComplexFilterTypeInput) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{156} + return file_product_proto_rawDescGZIP(), []int{166} } func (x *ComplexFilterTypeInput) GetFilter() *FilterType { @@ -7382,7 +7858,7 @@ type TypeWithComplexFilterInput struct { func (x *TypeWithComplexFilterInput) Reset() { *x = TypeWithComplexFilterInput{} - mi := &file_product_proto_msgTypes[157] + mi := &file_product_proto_msgTypes[167] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7394,7 +7870,7 @@ func (x *TypeWithComplexFilterInput) String() string { func (*TypeWithComplexFilterInput) ProtoMessage() {} func (x *TypeWithComplexFilterInput) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[157] + mi := &file_product_proto_msgTypes[167] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7407,7 +7883,7 @@ func (x *TypeWithComplexFilterInput) ProtoReflect() protoreflect.Message { // Deprecated: Use TypeWithComplexFilterInput.ProtoReflect.Descriptor instead. func (*TypeWithComplexFilterInput) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{157} + return file_product_proto_rawDescGZIP(), []int{167} } func (x *TypeWithComplexFilterInput) GetId() string { @@ -7435,7 +7911,7 @@ type OrderInput struct { func (x *OrderInput) Reset() { *x = OrderInput{} - mi := &file_product_proto_msgTypes[158] + mi := &file_product_proto_msgTypes[168] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7447,7 +7923,7 @@ func (x *OrderInput) String() string { func (*OrderInput) ProtoMessage() {} func (x *OrderInput) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[158] + mi := &file_product_proto_msgTypes[168] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7460,7 +7936,7 @@ func (x *OrderInput) ProtoReflect() protoreflect.Message { // Deprecated: Use OrderInput.ProtoReflect.Descriptor instead. func (*OrderInput) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{158} + return file_product_proto_rawDescGZIP(), []int{168} } func (x *OrderInput) GetOrderId() string { @@ -7496,7 +7972,7 @@ type Order struct { func (x *Order) Reset() { *x = Order{} - mi := &file_product_proto_msgTypes[159] + mi := &file_product_proto_msgTypes[169] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7508,7 +7984,7 @@ func (x *Order) String() string { func (*Order) ProtoMessage() {} func (x *Order) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[159] + mi := &file_product_proto_msgTypes[169] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7521,7 +7997,7 @@ func (x *Order) ProtoReflect() protoreflect.Message { // Deprecated: Use Order.ProtoReflect.Descriptor instead. func (*Order) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{159} + return file_product_proto_rawDescGZIP(), []int{169} } func (x *Order) GetOrderId() string { @@ -7564,7 +8040,7 @@ type Category struct { func (x *Category) Reset() { *x = Category{} - mi := &file_product_proto_msgTypes[160] + mi := &file_product_proto_msgTypes[170] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7576,7 +8052,7 @@ func (x *Category) String() string { func (*Category) ProtoMessage() {} func (x *Category) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[160] + mi := &file_product_proto_msgTypes[170] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7589,7 +8065,7 @@ func (x *Category) ProtoReflect() protoreflect.Message { // Deprecated: Use Category.ProtoReflect.Descriptor instead. func (*Category) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{160} + return file_product_proto_rawDescGZIP(), []int{170} } func (x *Category) GetId() string { @@ -7630,7 +8106,7 @@ type CategoryFilter struct { func (x *CategoryFilter) Reset() { *x = CategoryFilter{} - mi := &file_product_proto_msgTypes[161] + mi := &file_product_proto_msgTypes[171] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7642,7 +8118,7 @@ func (x *CategoryFilter) String() string { func (*CategoryFilter) ProtoMessage() {} func (x *CategoryFilter) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[161] + mi := &file_product_proto_msgTypes[171] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7655,7 +8131,7 @@ func (x *CategoryFilter) ProtoReflect() protoreflect.Message { // Deprecated: Use CategoryFilter.ProtoReflect.Descriptor instead. func (*CategoryFilter) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{161} + return file_product_proto_rawDescGZIP(), []int{171} } func (x *CategoryFilter) GetCategory() CategoryKind { @@ -7685,7 +8161,7 @@ type Animal struct { func (x *Animal) Reset() { *x = Animal{} - mi := &file_product_proto_msgTypes[162] + mi := &file_product_proto_msgTypes[172] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7697,7 +8173,7 @@ func (x *Animal) String() string { func (*Animal) ProtoMessage() {} func (x *Animal) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[162] + mi := &file_product_proto_msgTypes[172] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7710,7 +8186,7 @@ func (x *Animal) ProtoReflect() protoreflect.Message { // Deprecated: Use Animal.ProtoReflect.Descriptor instead. func (*Animal) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{162} + return file_product_proto_rawDescGZIP(), []int{172} } func (x *Animal) GetInstance() isAnimal_Instance { @@ -7764,7 +8240,7 @@ type SearchInput struct { func (x *SearchInput) Reset() { *x = SearchInput{} - mi := &file_product_proto_msgTypes[163] + mi := &file_product_proto_msgTypes[173] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7776,7 +8252,7 @@ func (x *SearchInput) String() string { func (*SearchInput) ProtoMessage() {} func (x *SearchInput) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[163] + mi := &file_product_proto_msgTypes[173] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7789,7 +8265,7 @@ func (x *SearchInput) ProtoReflect() protoreflect.Message { // Deprecated: Use SearchInput.ProtoReflect.Descriptor instead. func (*SearchInput) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{163} + return file_product_proto_rawDescGZIP(), []int{173} } func (x *SearchInput) GetQuery() string { @@ -7820,7 +8296,7 @@ type SearchResult struct { func (x *SearchResult) Reset() { *x = SearchResult{} - mi := &file_product_proto_msgTypes[164] + mi := &file_product_proto_msgTypes[174] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7832,7 +8308,7 @@ func (x *SearchResult) String() string { func (*SearchResult) ProtoMessage() {} func (x *SearchResult) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[164] + mi := &file_product_proto_msgTypes[174] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7845,7 +8321,7 @@ func (x *SearchResult) ProtoReflect() protoreflect.Message { // Deprecated: Use SearchResult.ProtoReflect.Descriptor instead. func (*SearchResult) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{164} + return file_product_proto_rawDescGZIP(), []int{174} } func (x *SearchResult) GetValue() isSearchResult_Value { @@ -7920,7 +8396,7 @@ type NullableFieldsType struct { func (x *NullableFieldsType) Reset() { *x = NullableFieldsType{} - mi := &file_product_proto_msgTypes[165] + mi := &file_product_proto_msgTypes[175] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7932,7 +8408,7 @@ func (x *NullableFieldsType) String() string { func (*NullableFieldsType) ProtoMessage() {} func (x *NullableFieldsType) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[165] + mi := &file_product_proto_msgTypes[175] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7945,7 +8421,7 @@ func (x *NullableFieldsType) ProtoReflect() protoreflect.Message { // Deprecated: Use NullableFieldsType.ProtoReflect.Descriptor instead. func (*NullableFieldsType) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{165} + return file_product_proto_rawDescGZIP(), []int{175} } func (x *NullableFieldsType) GetId() string { @@ -8015,7 +8491,7 @@ type NullableFieldsFilter struct { func (x *NullableFieldsFilter) Reset() { *x = NullableFieldsFilter{} - mi := &file_product_proto_msgTypes[166] + mi := &file_product_proto_msgTypes[176] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8027,7 +8503,7 @@ func (x *NullableFieldsFilter) String() string { func (*NullableFieldsFilter) ProtoMessage() {} func (x *NullableFieldsFilter) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[166] + mi := &file_product_proto_msgTypes[176] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8040,7 +8516,7 @@ func (x *NullableFieldsFilter) ProtoReflect() protoreflect.Message { // Deprecated: Use NullableFieldsFilter.ProtoReflect.Descriptor instead. func (*NullableFieldsFilter) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{166} + return file_product_proto_rawDescGZIP(), []int{176} } func (x *NullableFieldsFilter) GetName() *wrapperspb.StringValue { @@ -8092,7 +8568,7 @@ type BlogPost struct { func (x *BlogPost) Reset() { *x = BlogPost{} - mi := &file_product_proto_msgTypes[167] + mi := &file_product_proto_msgTypes[177] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8104,7 +8580,7 @@ func (x *BlogPost) String() string { func (*BlogPost) ProtoMessage() {} func (x *BlogPost) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[167] + mi := &file_product_proto_msgTypes[177] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8117,7 +8593,7 @@ func (x *BlogPost) ProtoReflect() protoreflect.Message { // Deprecated: Use BlogPost.ProtoReflect.Descriptor instead. func (*BlogPost) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{167} + return file_product_proto_rawDescGZIP(), []int{177} } func (x *BlogPost) GetId() string { @@ -8271,7 +8747,7 @@ type BlogPostFilter struct { func (x *BlogPostFilter) Reset() { *x = BlogPostFilter{} - mi := &file_product_proto_msgTypes[168] + mi := &file_product_proto_msgTypes[178] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8283,7 +8759,7 @@ func (x *BlogPostFilter) String() string { func (*BlogPostFilter) ProtoMessage() {} func (x *BlogPostFilter) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[168] + mi := &file_product_proto_msgTypes[178] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8296,7 +8772,7 @@ func (x *BlogPostFilter) ProtoReflect() protoreflect.Message { // Deprecated: Use BlogPostFilter.ProtoReflect.Descriptor instead. func (*BlogPostFilter) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{168} + return file_product_proto_rawDescGZIP(), []int{178} } func (x *BlogPostFilter) GetTitle() *wrapperspb.StringValue { @@ -8343,7 +8819,7 @@ type Author struct { func (x *Author) Reset() { *x = Author{} - mi := &file_product_proto_msgTypes[169] + mi := &file_product_proto_msgTypes[179] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8355,7 +8831,7 @@ func (x *Author) String() string { func (*Author) ProtoMessage() {} func (x *Author) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[169] + mi := &file_product_proto_msgTypes[179] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8368,7 +8844,7 @@ func (x *Author) ProtoReflect() protoreflect.Message { // Deprecated: Use Author.ProtoReflect.Descriptor instead. func (*Author) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{169} + return file_product_proto_rawDescGZIP(), []int{179} } func (x *Author) GetId() string { @@ -8487,7 +8963,7 @@ type AuthorFilter struct { func (x *AuthorFilter) Reset() { *x = AuthorFilter{} - mi := &file_product_proto_msgTypes[170] + mi := &file_product_proto_msgTypes[180] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8499,7 +8975,7 @@ func (x *AuthorFilter) String() string { func (*AuthorFilter) ProtoMessage() {} func (x *AuthorFilter) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[170] + mi := &file_product_proto_msgTypes[180] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8512,7 +8988,7 @@ func (x *AuthorFilter) ProtoReflect() protoreflect.Message { // Deprecated: Use AuthorFilter.ProtoReflect.Descriptor instead. func (*AuthorFilter) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{170} + return file_product_proto_rawDescGZIP(), []int{180} } func (x *AuthorFilter) GetName() *wrapperspb.StringValue { @@ -8545,7 +9021,7 @@ type UserInput struct { func (x *UserInput) Reset() { *x = UserInput{} - mi := &file_product_proto_msgTypes[171] + mi := &file_product_proto_msgTypes[181] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8557,7 +9033,7 @@ func (x *UserInput) String() string { func (*UserInput) ProtoMessage() {} func (x *UserInput) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[171] + mi := &file_product_proto_msgTypes[181] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8570,7 +9046,7 @@ func (x *UserInput) ProtoReflect() protoreflect.Message { // Deprecated: Use UserInput.ProtoReflect.Descriptor instead. func (*UserInput) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{171} + return file_product_proto_rawDescGZIP(), []int{181} } func (x *UserInput) GetName() string { @@ -8590,7 +9066,7 @@ type ActionInput struct { func (x *ActionInput) Reset() { *x = ActionInput{} - mi := &file_product_proto_msgTypes[172] + mi := &file_product_proto_msgTypes[182] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8602,7 +9078,7 @@ func (x *ActionInput) String() string { func (*ActionInput) ProtoMessage() {} func (x *ActionInput) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[172] + mi := &file_product_proto_msgTypes[182] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8615,7 +9091,7 @@ func (x *ActionInput) ProtoReflect() protoreflect.Message { // Deprecated: Use ActionInput.ProtoReflect.Descriptor instead. func (*ActionInput) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{172} + return file_product_proto_rawDescGZIP(), []int{182} } func (x *ActionInput) GetType() string { @@ -8645,7 +9121,7 @@ type ActionResult struct { func (x *ActionResult) Reset() { *x = ActionResult{} - mi := &file_product_proto_msgTypes[173] + mi := &file_product_proto_msgTypes[183] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8657,7 +9133,7 @@ func (x *ActionResult) String() string { func (*ActionResult) ProtoMessage() {} func (x *ActionResult) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[173] + mi := &file_product_proto_msgTypes[183] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8670,7 +9146,7 @@ func (x *ActionResult) ProtoReflect() protoreflect.Message { // Deprecated: Use ActionResult.ProtoReflect.Descriptor instead. func (*ActionResult) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{173} + return file_product_proto_rawDescGZIP(), []int{183} } func (x *ActionResult) GetValue() isActionResult_Value { @@ -8729,7 +9205,7 @@ type NullableFieldsInput struct { func (x *NullableFieldsInput) Reset() { *x = NullableFieldsInput{} - mi := &file_product_proto_msgTypes[174] + mi := &file_product_proto_msgTypes[184] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8741,7 +9217,7 @@ func (x *NullableFieldsInput) String() string { func (*NullableFieldsInput) ProtoMessage() {} func (x *NullableFieldsInput) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[174] + mi := &file_product_proto_msgTypes[184] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8754,7 +9230,7 @@ func (x *NullableFieldsInput) ProtoReflect() protoreflect.Message { // Deprecated: Use NullableFieldsInput.ProtoReflect.Descriptor instead. func (*NullableFieldsInput) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{174} + return file_product_proto_rawDescGZIP(), []int{184} } func (x *NullableFieldsInput) GetName() string { @@ -8830,7 +9306,7 @@ type BlogPostInput struct { func (x *BlogPostInput) Reset() { *x = BlogPostInput{} - mi := &file_product_proto_msgTypes[175] + mi := &file_product_proto_msgTypes[185] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8842,7 +9318,7 @@ func (x *BlogPostInput) String() string { func (*BlogPostInput) ProtoMessage() {} func (x *BlogPostInput) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[175] + mi := &file_product_proto_msgTypes[185] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8855,7 +9331,7 @@ func (x *BlogPostInput) ProtoReflect() protoreflect.Message { // Deprecated: Use BlogPostInput.ProtoReflect.Descriptor instead. func (*BlogPostInput) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{175} + return file_product_proto_rawDescGZIP(), []int{185} } func (x *BlogPostInput) GetTitle() string { @@ -8988,7 +9464,7 @@ type AuthorInput struct { func (x *AuthorInput) Reset() { *x = AuthorInput{} - mi := &file_product_proto_msgTypes[176] + mi := &file_product_proto_msgTypes[186] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9000,7 +9476,7 @@ func (x *AuthorInput) String() string { func (*AuthorInput) ProtoMessage() {} func (x *AuthorInput) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[176] + mi := &file_product_proto_msgTypes[186] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9013,7 +9489,7 @@ func (x *AuthorInput) ProtoReflect() protoreflect.Message { // Deprecated: Use AuthorInput.ProtoReflect.Descriptor instead. func (*AuthorInput) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{176} + return file_product_proto_rawDescGZIP(), []int{186} } func (x *AuthorInput) GetName() string { @@ -9097,7 +9573,7 @@ type NestedTypeB struct { func (x *NestedTypeB) Reset() { *x = NestedTypeB{} - mi := &file_product_proto_msgTypes[177] + mi := &file_product_proto_msgTypes[187] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9109,7 +9585,7 @@ func (x *NestedTypeB) String() string { func (*NestedTypeB) ProtoMessage() {} func (x *NestedTypeB) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[177] + mi := &file_product_proto_msgTypes[187] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9122,7 +9598,7 @@ func (x *NestedTypeB) ProtoReflect() protoreflect.Message { // Deprecated: Use NestedTypeB.ProtoReflect.Descriptor instead. func (*NestedTypeB) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{177} + return file_product_proto_rawDescGZIP(), []int{187} } func (x *NestedTypeB) GetId() string { @@ -9156,7 +9632,7 @@ type NestedTypeC struct { func (x *NestedTypeC) Reset() { *x = NestedTypeC{} - mi := &file_product_proto_msgTypes[178] + mi := &file_product_proto_msgTypes[188] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9168,7 +9644,7 @@ func (x *NestedTypeC) String() string { func (*NestedTypeC) ProtoMessage() {} func (x *NestedTypeC) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[178] + mi := &file_product_proto_msgTypes[188] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9181,7 +9657,7 @@ func (x *NestedTypeC) ProtoReflect() protoreflect.Message { // Deprecated: Use NestedTypeC.ProtoReflect.Descriptor instead. func (*NestedTypeC) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{178} + return file_product_proto_rawDescGZIP(), []int{188} } func (x *NestedTypeC) GetId() string { @@ -9210,7 +9686,7 @@ type FilterType struct { func (x *FilterType) Reset() { *x = FilterType{} - mi := &file_product_proto_msgTypes[179] + mi := &file_product_proto_msgTypes[189] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9222,7 +9698,7 @@ func (x *FilterType) String() string { func (*FilterType) ProtoMessage() {} func (x *FilterType) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[179] + mi := &file_product_proto_msgTypes[189] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9235,7 +9711,7 @@ func (x *FilterType) ProtoReflect() protoreflect.Message { // Deprecated: Use FilterType.ProtoReflect.Descriptor instead. func (*FilterType) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{179} + return file_product_proto_rawDescGZIP(), []int{189} } func (x *FilterType) GetName() string { @@ -9276,7 +9752,7 @@ type Pagination struct { func (x *Pagination) Reset() { *x = Pagination{} - mi := &file_product_proto_msgTypes[180] + mi := &file_product_proto_msgTypes[190] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9288,7 +9764,7 @@ func (x *Pagination) String() string { func (*Pagination) ProtoMessage() {} func (x *Pagination) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[180] + mi := &file_product_proto_msgTypes[190] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9301,7 +9777,7 @@ func (x *Pagination) ProtoReflect() protoreflect.Message { // Deprecated: Use Pagination.ProtoReflect.Descriptor instead. func (*Pagination) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{180} + return file_product_proto_rawDescGZIP(), []int{190} } func (x *Pagination) GetPage() int32 { @@ -9329,7 +9805,7 @@ type OrderLineInput struct { func (x *OrderLineInput) Reset() { *x = OrderLineInput{} - mi := &file_product_proto_msgTypes[181] + mi := &file_product_proto_msgTypes[191] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9341,7 +9817,7 @@ func (x *OrderLineInput) String() string { func (*OrderLineInput) ProtoMessage() {} func (x *OrderLineInput) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[181] + mi := &file_product_proto_msgTypes[191] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9354,7 +9830,7 @@ func (x *OrderLineInput) ProtoReflect() protoreflect.Message { // Deprecated: Use OrderLineInput.ProtoReflect.Descriptor instead. func (*OrderLineInput) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{181} + return file_product_proto_rawDescGZIP(), []int{191} } func (x *OrderLineInput) GetProductId() string { @@ -9389,7 +9865,7 @@ type OrderLine struct { func (x *OrderLine) Reset() { *x = OrderLine{} - mi := &file_product_proto_msgTypes[182] + mi := &file_product_proto_msgTypes[192] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9401,7 +9877,7 @@ func (x *OrderLine) String() string { func (*OrderLine) ProtoMessage() {} func (x *OrderLine) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[182] + mi := &file_product_proto_msgTypes[192] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9414,7 +9890,7 @@ func (x *OrderLine) ProtoReflect() protoreflect.Message { // Deprecated: Use OrderLine.ProtoReflect.Descriptor instead. func (*OrderLine) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{182} + return file_product_proto_rawDescGZIP(), []int{192} } func (x *OrderLine) GetProductId() string { @@ -9450,7 +9926,7 @@ type Subcategory struct { func (x *Subcategory) Reset() { *x = Subcategory{} - mi := &file_product_proto_msgTypes[183] + mi := &file_product_proto_msgTypes[193] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9462,7 +9938,7 @@ func (x *Subcategory) String() string { func (*Subcategory) ProtoMessage() {} func (x *Subcategory) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[183] + mi := &file_product_proto_msgTypes[193] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9475,7 +9951,7 @@ func (x *Subcategory) ProtoReflect() protoreflect.Message { // Deprecated: Use Subcategory.ProtoReflect.Descriptor instead. func (*Subcategory) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{183} + return file_product_proto_rawDescGZIP(), []int{193} } func (x *Subcategory) GetId() string { @@ -9520,7 +9996,7 @@ type CategoryMetrics struct { func (x *CategoryMetrics) Reset() { *x = CategoryMetrics{} - mi := &file_product_proto_msgTypes[184] + mi := &file_product_proto_msgTypes[194] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9532,7 +10008,7 @@ func (x *CategoryMetrics) String() string { func (*CategoryMetrics) ProtoMessage() {} func (x *CategoryMetrics) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[184] + mi := &file_product_proto_msgTypes[194] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9545,7 +10021,7 @@ func (x *CategoryMetrics) ProtoReflect() protoreflect.Message { // Deprecated: Use CategoryMetrics.ProtoReflect.Descriptor instead. func (*CategoryMetrics) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{184} + return file_product_proto_rawDescGZIP(), []int{194} } func (x *CategoryMetrics) GetId() string { @@ -9602,7 +10078,7 @@ type Cat struct { func (x *Cat) Reset() { *x = Cat{} - mi := &file_product_proto_msgTypes[185] + mi := &file_product_proto_msgTypes[195] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9614,7 +10090,7 @@ func (x *Cat) String() string { func (*Cat) ProtoMessage() {} func (x *Cat) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[185] + mi := &file_product_proto_msgTypes[195] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9627,7 +10103,7 @@ func (x *Cat) ProtoReflect() protoreflect.Message { // Deprecated: Use Cat.ProtoReflect.Descriptor instead. func (*Cat) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{185} + return file_product_proto_rawDescGZIP(), []int{195} } func (x *Cat) GetId() string { @@ -9670,7 +10146,7 @@ type Dog struct { func (x *Dog) Reset() { *x = Dog{} - mi := &file_product_proto_msgTypes[186] + mi := &file_product_proto_msgTypes[196] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9682,7 +10158,7 @@ func (x *Dog) String() string { func (*Dog) ProtoMessage() {} func (x *Dog) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[186] + mi := &file_product_proto_msgTypes[196] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9695,7 +10171,7 @@ func (x *Dog) ProtoReflect() protoreflect.Message { // Deprecated: Use Dog.ProtoReflect.Descriptor instead. func (*Dog) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{186} + return file_product_proto_rawDescGZIP(), []int{196} } func (x *Dog) GetId() string { @@ -9736,7 +10212,7 @@ type ActionSuccess struct { func (x *ActionSuccess) Reset() { *x = ActionSuccess{} - mi := &file_product_proto_msgTypes[187] + mi := &file_product_proto_msgTypes[197] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9748,7 +10224,7 @@ func (x *ActionSuccess) String() string { func (*ActionSuccess) ProtoMessage() {} func (x *ActionSuccess) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[187] + mi := &file_product_proto_msgTypes[197] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9761,7 +10237,7 @@ func (x *ActionSuccess) ProtoReflect() protoreflect.Message { // Deprecated: Use ActionSuccess.ProtoReflect.Descriptor instead. func (*ActionSuccess) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{187} + return file_product_proto_rawDescGZIP(), []int{197} } func (x *ActionSuccess) GetMessage() string { @@ -9788,7 +10264,7 @@ type ActionError struct { func (x *ActionError) Reset() { *x = ActionError{} - mi := &file_product_proto_msgTypes[188] + mi := &file_product_proto_msgTypes[198] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9800,7 +10276,7 @@ func (x *ActionError) String() string { func (*ActionError) ProtoMessage() {} func (x *ActionError) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[188] + mi := &file_product_proto_msgTypes[198] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9813,7 +10289,7 @@ func (x *ActionError) ProtoReflect() protoreflect.Message { // Deprecated: Use ActionError.ProtoReflect.Descriptor instead. func (*ActionError) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{188} + return file_product_proto_rawDescGZIP(), []int{198} } func (x *ActionError) GetMessage() string { @@ -9840,7 +10316,7 @@ type CategoryInput struct { func (x *CategoryInput) Reset() { *x = CategoryInput{} - mi := &file_product_proto_msgTypes[189] + mi := &file_product_proto_msgTypes[199] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9852,7 +10328,7 @@ func (x *CategoryInput) String() string { func (*CategoryInput) ProtoMessage() {} func (x *CategoryInput) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[189] + mi := &file_product_proto_msgTypes[199] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9865,7 +10341,7 @@ func (x *CategoryInput) ProtoReflect() protoreflect.Message { // Deprecated: Use CategoryInput.ProtoReflect.Descriptor instead. func (*CategoryInput) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{189} + return file_product_proto_rawDescGZIP(), []int{199} } func (x *CategoryInput) GetName() string { @@ -9894,7 +10370,7 @@ type ProductCountFilter struct { func (x *ProductCountFilter) Reset() { *x = ProductCountFilter{} - mi := &file_product_proto_msgTypes[190] + mi := &file_product_proto_msgTypes[200] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9906,7 +10382,7 @@ func (x *ProductCountFilter) String() string { func (*ProductCountFilter) ProtoMessage() {} func (x *ProductCountFilter) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[190] + mi := &file_product_proto_msgTypes[200] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9919,7 +10395,7 @@ func (x *ProductCountFilter) ProtoReflect() protoreflect.Message { // Deprecated: Use ProductCountFilter.ProtoReflect.Descriptor instead. func (*ProductCountFilter) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{190} + return file_product_proto_rawDescGZIP(), []int{200} } func (x *ProductCountFilter) GetMinPrice() *wrapperspb.DoubleValue { @@ -9963,7 +10439,7 @@ type SubcategoryItemFilter struct { func (x *SubcategoryItemFilter) Reset() { *x = SubcategoryItemFilter{} - mi := &file_product_proto_msgTypes[191] + mi := &file_product_proto_msgTypes[201] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9975,7 +10451,7 @@ func (x *SubcategoryItemFilter) String() string { func (*SubcategoryItemFilter) ProtoMessage() {} func (x *SubcategoryItemFilter) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[191] + mi := &file_product_proto_msgTypes[201] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9988,7 +10464,7 @@ func (x *SubcategoryItemFilter) ProtoReflect() protoreflect.Message { // Deprecated: Use SubcategoryItemFilter.ProtoReflect.Descriptor instead. func (*SubcategoryItemFilter) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{191} + return file_product_proto_rawDescGZIP(), []int{201} } func (x *SubcategoryItemFilter) GetMinPrice() *wrapperspb.DoubleValue { @@ -10037,7 +10513,7 @@ type ShippingEstimateInput struct { func (x *ShippingEstimateInput) Reset() { *x = ShippingEstimateInput{} - mi := &file_product_proto_msgTypes[192] + mi := &file_product_proto_msgTypes[202] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -10049,7 +10525,7 @@ func (x *ShippingEstimateInput) String() string { func (*ShippingEstimateInput) ProtoMessage() {} func (x *ShippingEstimateInput) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[192] + mi := &file_product_proto_msgTypes[202] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -10062,7 +10538,7 @@ func (x *ShippingEstimateInput) ProtoReflect() protoreflect.Message { // Deprecated: Use ShippingEstimateInput.ProtoReflect.Descriptor instead. func (*ShippingEstimateInput) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{192} + return file_product_proto_rawDescGZIP(), []int{202} } func (x *ShippingEstimateInput) GetDestination() ShippingDestination { @@ -10095,7 +10571,7 @@ type ListOfAuthorFilter_List struct { func (x *ListOfAuthorFilter_List) Reset() { *x = ListOfAuthorFilter_List{} - mi := &file_product_proto_msgTypes[193] + mi := &file_product_proto_msgTypes[203] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -10107,7 +10583,7 @@ func (x *ListOfAuthorFilter_List) String() string { func (*ListOfAuthorFilter_List) ProtoMessage() {} func (x *ListOfAuthorFilter_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[193] + mi := &file_product_proto_msgTypes[203] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -10139,7 +10615,7 @@ type ListOfAuthorInput_List struct { func (x *ListOfAuthorInput_List) Reset() { *x = ListOfAuthorInput_List{} - mi := &file_product_proto_msgTypes[194] + mi := &file_product_proto_msgTypes[204] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -10151,7 +10627,7 @@ func (x *ListOfAuthorInput_List) String() string { func (*ListOfAuthorInput_List) ProtoMessage() {} func (x *ListOfAuthorInput_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[194] + mi := &file_product_proto_msgTypes[204] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -10183,7 +10659,7 @@ type ListOfBlogPost_List struct { func (x *ListOfBlogPost_List) Reset() { *x = ListOfBlogPost_List{} - mi := &file_product_proto_msgTypes[195] + mi := &file_product_proto_msgTypes[205] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -10195,7 +10671,7 @@ func (x *ListOfBlogPost_List) String() string { func (*ListOfBlogPost_List) ProtoMessage() {} func (x *ListOfBlogPost_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[195] + mi := &file_product_proto_msgTypes[205] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -10227,7 +10703,7 @@ type ListOfBlogPostFilter_List struct { func (x *ListOfBlogPostFilter_List) Reset() { *x = ListOfBlogPostFilter_List{} - mi := &file_product_proto_msgTypes[196] + mi := &file_product_proto_msgTypes[206] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -10239,7 +10715,7 @@ func (x *ListOfBlogPostFilter_List) String() string { func (*ListOfBlogPostFilter_List) ProtoMessage() {} func (x *ListOfBlogPostFilter_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[196] + mi := &file_product_proto_msgTypes[206] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -10271,7 +10747,7 @@ type ListOfBlogPostInput_List struct { func (x *ListOfBlogPostInput_List) Reset() { *x = ListOfBlogPostInput_List{} - mi := &file_product_proto_msgTypes[197] + mi := &file_product_proto_msgTypes[207] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -10283,7 +10759,7 @@ func (x *ListOfBlogPostInput_List) String() string { func (*ListOfBlogPostInput_List) ProtoMessage() {} func (x *ListOfBlogPostInput_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[197] + mi := &file_product_proto_msgTypes[207] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -10315,7 +10791,7 @@ type ListOfBoolean_List struct { func (x *ListOfBoolean_List) Reset() { *x = ListOfBoolean_List{} - mi := &file_product_proto_msgTypes[198] + mi := &file_product_proto_msgTypes[208] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -10327,7 +10803,7 @@ func (x *ListOfBoolean_List) String() string { func (*ListOfBoolean_List) ProtoMessage() {} func (x *ListOfBoolean_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[198] + mi := &file_product_proto_msgTypes[208] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -10359,7 +10835,7 @@ type ListOfCategory_List struct { func (x *ListOfCategory_List) Reset() { *x = ListOfCategory_List{} - mi := &file_product_proto_msgTypes[199] + mi := &file_product_proto_msgTypes[209] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -10371,7 +10847,7 @@ func (x *ListOfCategory_List) String() string { func (*ListOfCategory_List) ProtoMessage() {} func (x *ListOfCategory_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[199] + mi := &file_product_proto_msgTypes[209] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -10403,7 +10879,7 @@ type ListOfCategoryInput_List struct { func (x *ListOfCategoryInput_List) Reset() { *x = ListOfCategoryInput_List{} - mi := &file_product_proto_msgTypes[200] + mi := &file_product_proto_msgTypes[210] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -10415,7 +10891,7 @@ func (x *ListOfCategoryInput_List) String() string { func (*ListOfCategoryInput_List) ProtoMessage() {} func (x *ListOfCategoryInput_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[200] + mi := &file_product_proto_msgTypes[210] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -10447,7 +10923,7 @@ type ListOfFloat_List struct { func (x *ListOfFloat_List) Reset() { *x = ListOfFloat_List{} - mi := &file_product_proto_msgTypes[201] + mi := &file_product_proto_msgTypes[211] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -10459,7 +10935,7 @@ func (x *ListOfFloat_List) String() string { func (*ListOfFloat_List) ProtoMessage() {} func (x *ListOfFloat_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[201] + mi := &file_product_proto_msgTypes[211] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -10491,7 +10967,7 @@ type ListOfListOfCategory_List struct { func (x *ListOfListOfCategory_List) Reset() { *x = ListOfListOfCategory_List{} - mi := &file_product_proto_msgTypes[202] + mi := &file_product_proto_msgTypes[212] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -10503,7 +10979,7 @@ func (x *ListOfListOfCategory_List) String() string { func (*ListOfListOfCategory_List) ProtoMessage() {} func (x *ListOfListOfCategory_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[202] + mi := &file_product_proto_msgTypes[212] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -10535,7 +11011,7 @@ type ListOfListOfCategoryInput_List struct { func (x *ListOfListOfCategoryInput_List) Reset() { *x = ListOfListOfCategoryInput_List{} - mi := &file_product_proto_msgTypes[203] + mi := &file_product_proto_msgTypes[213] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -10547,7 +11023,7 @@ func (x *ListOfListOfCategoryInput_List) String() string { func (*ListOfListOfCategoryInput_List) ProtoMessage() {} func (x *ListOfListOfCategoryInput_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[203] + mi := &file_product_proto_msgTypes[213] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -10579,7 +11055,7 @@ type ListOfListOfString_List struct { func (x *ListOfListOfString_List) Reset() { *x = ListOfListOfString_List{} - mi := &file_product_proto_msgTypes[204] + mi := &file_product_proto_msgTypes[214] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -10591,7 +11067,7 @@ func (x *ListOfListOfString_List) String() string { func (*ListOfListOfString_List) ProtoMessage() {} func (x *ListOfListOfString_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[204] + mi := &file_product_proto_msgTypes[214] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -10623,7 +11099,7 @@ type ListOfListOfUser_List struct { func (x *ListOfListOfUser_List) Reset() { *x = ListOfListOfUser_List{} - mi := &file_product_proto_msgTypes[205] + mi := &file_product_proto_msgTypes[215] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -10635,7 +11111,7 @@ func (x *ListOfListOfUser_List) String() string { func (*ListOfListOfUser_List) ProtoMessage() {} func (x *ListOfListOfUser_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[205] + mi := &file_product_proto_msgTypes[215] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -10667,7 +11143,7 @@ type ListOfListOfUserInput_List struct { func (x *ListOfListOfUserInput_List) Reset() { *x = ListOfListOfUserInput_List{} - mi := &file_product_proto_msgTypes[206] + mi := &file_product_proto_msgTypes[216] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -10679,7 +11155,7 @@ func (x *ListOfListOfUserInput_List) String() string { func (*ListOfListOfUserInput_List) ProtoMessage() {} func (x *ListOfListOfUserInput_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[206] + mi := &file_product_proto_msgTypes[216] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -10711,7 +11187,7 @@ type ListOfOrderLine_List struct { func (x *ListOfOrderLine_List) Reset() { *x = ListOfOrderLine_List{} - mi := &file_product_proto_msgTypes[207] + mi := &file_product_proto_msgTypes[217] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -10723,7 +11199,7 @@ func (x *ListOfOrderLine_List) String() string { func (*ListOfOrderLine_List) ProtoMessage() {} func (x *ListOfOrderLine_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[207] + mi := &file_product_proto_msgTypes[217] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -10755,7 +11231,7 @@ type ListOfProduct_List struct { func (x *ListOfProduct_List) Reset() { *x = ListOfProduct_List{} - mi := &file_product_proto_msgTypes[208] + mi := &file_product_proto_msgTypes[218] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -10767,7 +11243,7 @@ func (x *ListOfProduct_List) String() string { func (*ListOfProduct_List) ProtoMessage() {} func (x *ListOfProduct_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[208] + mi := &file_product_proto_msgTypes[218] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -10799,7 +11275,7 @@ type ListOfString_List struct { func (x *ListOfString_List) Reset() { *x = ListOfString_List{} - mi := &file_product_proto_msgTypes[209] + mi := &file_product_proto_msgTypes[219] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -10811,7 +11287,7 @@ func (x *ListOfString_List) String() string { func (*ListOfString_List) ProtoMessage() {} func (x *ListOfString_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[209] + mi := &file_product_proto_msgTypes[219] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -10843,7 +11319,7 @@ type ListOfSubcategory_List struct { func (x *ListOfSubcategory_List) Reset() { *x = ListOfSubcategory_List{} - mi := &file_product_proto_msgTypes[210] + mi := &file_product_proto_msgTypes[220] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -10855,7 +11331,7 @@ func (x *ListOfSubcategory_List) String() string { func (*ListOfSubcategory_List) ProtoMessage() {} func (x *ListOfSubcategory_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[210] + mi := &file_product_proto_msgTypes[220] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -10887,7 +11363,7 @@ type ListOfUser_List struct { func (x *ListOfUser_List) Reset() { *x = ListOfUser_List{} - mi := &file_product_proto_msgTypes[211] + mi := &file_product_proto_msgTypes[221] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -10899,7 +11375,7 @@ func (x *ListOfUser_List) String() string { func (*ListOfUser_List) ProtoMessage() {} func (x *ListOfUser_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[211] + mi := &file_product_proto_msgTypes[221] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -10931,7 +11407,7 @@ type ListOfUserInput_List struct { func (x *ListOfUserInput_List) Reset() { *x = ListOfUserInput_List{} - mi := &file_product_proto_msgTypes[212] + mi := &file_product_proto_msgTypes[222] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -10943,7 +11419,7 @@ func (x *ListOfUserInput_List) String() string { func (*ListOfUserInput_List) ProtoMessage() {} func (x *ListOfUserInput_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[212] + mi := &file_product_proto_msgTypes[222] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -11303,7 +11779,33 @@ const file_product_proto_rawDesc = "" + "$ResolveCategoryCategoryMetricsResult\x12E\n" + "\x10category_metrics\x18\x01 \x01(\v2\x1a.productv1.CategoryMetricsR\x0fcategoryMetrics\"q\n" + "&ResolveCategoryCategoryMetricsResponse\x12G\n" + - "\x06result\x18\x01 \x03(\v2/.productv1.ResolveCategoryCategoryMetricsResultR\x06result\"]\n" + + "\x06result\x18\x01 \x03(\v2/.productv1.ResolveCategoryCategoryMetricsResultR\x06result\"B\n" + + "\x19ResolveCategoryMascotArgs\x12%\n" + + "\x0einclude_volume\x18\x01 \x01(\bR\rincludeVolume\"[\n" + + "\x1cResolveCategoryMascotContext\x12\x0e\n" + + "\x02id\x18\x01 \x01(\tR\x02id\x12+\n" + + "\x04kind\x18\x02 \x01(\x0e2\x17.productv1.CategoryKindR\x04kind\"\xa6\x01\n" + + "\x1cResolveCategoryMascotRequest\x12A\n" + + "\acontext\x18\x01 \x03(\v2'.productv1.ResolveCategoryMascotContextR\acontext\x12C\n" + + "\n" + + "field_args\x18\x02 \x01(\v2$.productv1.ResolveCategoryMascotArgsR\tfieldArgs\"H\n" + + "\x1bResolveCategoryMascotResult\x12)\n" + + "\x06mascot\x18\x01 \x01(\v2\x11.productv1.AnimalR\x06mascot\"_\n" + + "\x1dResolveCategoryMascotResponse\x12>\n" + + "\x06result\x18\x01 \x03(\v2&.productv1.ResolveCategoryMascotResultR\x06result\"F\n" + + "!ResolveCategoryCategoryStatusArgs\x12!\n" + + "\fcheck_health\x18\x01 \x01(\bR\vcheckHealth\"J\n" + + "$ResolveCategoryCategoryStatusContext\x12\x0e\n" + + "\x02id\x18\x01 \x01(\tR\x02id\x12\x12\n" + + "\x04name\x18\x02 \x01(\tR\x04name\"\xbe\x01\n" + + "$ResolveCategoryCategoryStatusRequest\x12I\n" + + "\acontext\x18\x01 \x03(\v2/.productv1.ResolveCategoryCategoryStatusContextR\acontext\x12K\n" + + "\n" + + "field_args\x18\x02 \x01(\v2,.productv1.ResolveCategoryCategoryStatusArgsR\tfieldArgs\"g\n" + + "#ResolveCategoryCategoryStatusResult\x12@\n" + + "\x0fcategory_status\x18\x01 \x01(\v2\x17.productv1.ActionResultR\x0ecategoryStatus\"o\n" + + "%ResolveCategoryCategoryStatusResponse\x12F\n" + + "\x06result\x18\x01 \x03(\v2..productv1.ResolveCategoryCategoryStatusResultR\x06result\"]\n" + "\x1fResolveSubcategoryItemCountArgs\x12:\n" + "\afilters\x18\x01 \x01(\v2 .productv1.SubcategoryItemFilterR\afilters\"4\n" + "\"ResolveSubcategoryItemCountContext\x12\x0e\n" + @@ -11611,7 +12113,7 @@ const file_product_proto_rawDesc = "" + " SHIPPING_DESTINATION_UNSPECIFIED\x10\x00\x12!\n" + "\x1dSHIPPING_DESTINATION_DOMESTIC\x10\x01\x12 \n" + "\x1cSHIPPING_DESTINATION_EXPRESS\x10\x02\x12&\n" + - "\"SHIPPING_DESTINATION_INTERNATIONAL\x10\x032\x88.\n" + + "\"SHIPPING_DESTINATION_INTERNATIONAL\x10\x032\xfd/\n" + "\x0eProductService\x12`\n" + "\x11LookupProductById\x12#.productv1.LookupProductByIdRequest\x1a$.productv1.LookupProductByIdResponse\"\x00\x12`\n" + "\x11LookupStorageById\x12#.productv1.LookupStorageByIdRequest\x1a$.productv1.LookupStorageByIdResponse\"\x00\x12f\n" + @@ -11659,7 +12161,9 @@ const file_product_proto_rawDesc = "" + "\tQueryUser\x12\x1b.productv1.QueryUserRequest\x1a\x1c.productv1.QueryUserResponse\"\x00\x12K\n" + "\n" + "QueryUsers\x12\x1c.productv1.QueryUsersRequest\x1a\x1d.productv1.QueryUsersResponse\"\x00\x12\x87\x01\n" + - "\x1eResolveCategoryCategoryMetrics\x120.productv1.ResolveCategoryCategoryMetricsRequest\x1a1.productv1.ResolveCategoryCategoryMetricsResponse\"\x00\x12\x9c\x01\n" + + "\x1eResolveCategoryCategoryMetrics\x120.productv1.ResolveCategoryCategoryMetricsRequest\x1a1.productv1.ResolveCategoryCategoryMetricsResponse\"\x00\x12\x84\x01\n" + + "\x1dResolveCategoryCategoryStatus\x12/.productv1.ResolveCategoryCategoryStatusRequest\x1a0.productv1.ResolveCategoryCategoryStatusResponse\"\x00\x12l\n" + + "\x15ResolveCategoryMascot\x12'.productv1.ResolveCategoryMascotRequest\x1a(.productv1.ResolveCategoryMascotResponse\"\x00\x12\x9c\x01\n" + "%ResolveCategoryMetricsNormalizedScore\x127.productv1.ResolveCategoryMetricsNormalizedScoreRequest\x1a8.productv1.ResolveCategoryMetricsNormalizedScoreResponse\"\x00\x12\x87\x01\n" + "\x1eResolveCategoryPopularityScore\x120.productv1.ResolveCategoryPopularityScoreRequest\x1a1.productv1.ResolveCategoryPopularityScoreResponse\"\x00\x12~\n" + "\x1bResolveCategoryProductCount\x12-.productv1.ResolveCategoryProductCountRequest\x1a..productv1.ResolveCategoryProductCountResponse\"\x00\x12\x90\x01\n" + @@ -11680,7 +12184,7 @@ func file_product_proto_rawDescGZIP() []byte { } var file_product_proto_enumTypes = make([]protoimpl.EnumInfo, 2) -var file_product_proto_msgTypes = make([]protoimpl.MessageInfo, 213) +var file_product_proto_msgTypes = make([]protoimpl.MessageInfo, 223) var file_product_proto_goTypes = []any{ (CategoryKind)(0), // 0: productv1.CategoryKind (ShippingDestination)(0), // 1: productv1.ShippingDestination @@ -11822,428 +12326,451 @@ var file_product_proto_goTypes = []any{ (*ResolveCategoryCategoryMetricsRequest)(nil), // 137: productv1.ResolveCategoryCategoryMetricsRequest (*ResolveCategoryCategoryMetricsResult)(nil), // 138: productv1.ResolveCategoryCategoryMetricsResult (*ResolveCategoryCategoryMetricsResponse)(nil), // 139: productv1.ResolveCategoryCategoryMetricsResponse - (*ResolveSubcategoryItemCountArgs)(nil), // 140: productv1.ResolveSubcategoryItemCountArgs - (*ResolveSubcategoryItemCountContext)(nil), // 141: productv1.ResolveSubcategoryItemCountContext - (*ResolveSubcategoryItemCountRequest)(nil), // 142: productv1.ResolveSubcategoryItemCountRequest - (*ResolveSubcategoryItemCountResult)(nil), // 143: productv1.ResolveSubcategoryItemCountResult - (*ResolveSubcategoryItemCountResponse)(nil), // 144: productv1.ResolveSubcategoryItemCountResponse - (*ResolveCategoryMetricsNormalizedScoreArgs)(nil), // 145: productv1.ResolveCategoryMetricsNormalizedScoreArgs - (*ResolveCategoryMetricsNormalizedScoreContext)(nil), // 146: productv1.ResolveCategoryMetricsNormalizedScoreContext - (*ResolveCategoryMetricsNormalizedScoreRequest)(nil), // 147: productv1.ResolveCategoryMetricsNormalizedScoreRequest - (*ResolveCategoryMetricsNormalizedScoreResult)(nil), // 148: productv1.ResolveCategoryMetricsNormalizedScoreResult - (*ResolveCategoryMetricsNormalizedScoreResponse)(nil), // 149: productv1.ResolveCategoryMetricsNormalizedScoreResponse - (*Product)(nil), // 150: productv1.Product - (*Storage)(nil), // 151: productv1.Storage - (*Warehouse)(nil), // 152: productv1.Warehouse - (*User)(nil), // 153: productv1.User - (*NestedTypeA)(nil), // 154: productv1.NestedTypeA - (*RecursiveType)(nil), // 155: productv1.RecursiveType - (*TypeWithMultipleFilterFields)(nil), // 156: productv1.TypeWithMultipleFilterFields - (*FilterTypeInput)(nil), // 157: productv1.FilterTypeInput - (*ComplexFilterTypeInput)(nil), // 158: productv1.ComplexFilterTypeInput - (*TypeWithComplexFilterInput)(nil), // 159: productv1.TypeWithComplexFilterInput - (*OrderInput)(nil), // 160: productv1.OrderInput - (*Order)(nil), // 161: productv1.Order - (*Category)(nil), // 162: productv1.Category - (*CategoryFilter)(nil), // 163: productv1.CategoryFilter - (*Animal)(nil), // 164: productv1.Animal - (*SearchInput)(nil), // 165: productv1.SearchInput - (*SearchResult)(nil), // 166: productv1.SearchResult - (*NullableFieldsType)(nil), // 167: productv1.NullableFieldsType - (*NullableFieldsFilter)(nil), // 168: productv1.NullableFieldsFilter - (*BlogPost)(nil), // 169: productv1.BlogPost - (*BlogPostFilter)(nil), // 170: productv1.BlogPostFilter - (*Author)(nil), // 171: productv1.Author - (*AuthorFilter)(nil), // 172: productv1.AuthorFilter - (*UserInput)(nil), // 173: productv1.UserInput - (*ActionInput)(nil), // 174: productv1.ActionInput - (*ActionResult)(nil), // 175: productv1.ActionResult - (*NullableFieldsInput)(nil), // 176: productv1.NullableFieldsInput - (*BlogPostInput)(nil), // 177: productv1.BlogPostInput - (*AuthorInput)(nil), // 178: productv1.AuthorInput - (*NestedTypeB)(nil), // 179: productv1.NestedTypeB - (*NestedTypeC)(nil), // 180: productv1.NestedTypeC - (*FilterType)(nil), // 181: productv1.FilterType - (*Pagination)(nil), // 182: productv1.Pagination - (*OrderLineInput)(nil), // 183: productv1.OrderLineInput - (*OrderLine)(nil), // 184: productv1.OrderLine - (*Subcategory)(nil), // 185: productv1.Subcategory - (*CategoryMetrics)(nil), // 186: productv1.CategoryMetrics - (*Cat)(nil), // 187: productv1.Cat - (*Dog)(nil), // 188: productv1.Dog - (*ActionSuccess)(nil), // 189: productv1.ActionSuccess - (*ActionError)(nil), // 190: productv1.ActionError - (*CategoryInput)(nil), // 191: productv1.CategoryInput - (*ProductCountFilter)(nil), // 192: productv1.ProductCountFilter - (*SubcategoryItemFilter)(nil), // 193: productv1.SubcategoryItemFilter - (*ShippingEstimateInput)(nil), // 194: productv1.ShippingEstimateInput - (*ListOfAuthorFilter_List)(nil), // 195: productv1.ListOfAuthorFilter.List - (*ListOfAuthorInput_List)(nil), // 196: productv1.ListOfAuthorInput.List - (*ListOfBlogPost_List)(nil), // 197: productv1.ListOfBlogPost.List - (*ListOfBlogPostFilter_List)(nil), // 198: productv1.ListOfBlogPostFilter.List - (*ListOfBlogPostInput_List)(nil), // 199: productv1.ListOfBlogPostInput.List - (*ListOfBoolean_List)(nil), // 200: productv1.ListOfBoolean.List - (*ListOfCategory_List)(nil), // 201: productv1.ListOfCategory.List - (*ListOfCategoryInput_List)(nil), // 202: productv1.ListOfCategoryInput.List - (*ListOfFloat_List)(nil), // 203: productv1.ListOfFloat.List - (*ListOfListOfCategory_List)(nil), // 204: productv1.ListOfListOfCategory.List - (*ListOfListOfCategoryInput_List)(nil), // 205: productv1.ListOfListOfCategoryInput.List - (*ListOfListOfString_List)(nil), // 206: productv1.ListOfListOfString.List - (*ListOfListOfUser_List)(nil), // 207: productv1.ListOfListOfUser.List - (*ListOfListOfUserInput_List)(nil), // 208: productv1.ListOfListOfUserInput.List - (*ListOfOrderLine_List)(nil), // 209: productv1.ListOfOrderLine.List - (*ListOfProduct_List)(nil), // 210: productv1.ListOfProduct.List - (*ListOfString_List)(nil), // 211: productv1.ListOfString.List - (*ListOfSubcategory_List)(nil), // 212: productv1.ListOfSubcategory.List - (*ListOfUser_List)(nil), // 213: productv1.ListOfUser.List - (*ListOfUserInput_List)(nil), // 214: productv1.ListOfUserInput.List - (*wrapperspb.Int32Value)(nil), // 215: google.protobuf.Int32Value - (*wrapperspb.StringValue)(nil), // 216: google.protobuf.StringValue - (*wrapperspb.DoubleValue)(nil), // 217: google.protobuf.DoubleValue - (*wrapperspb.BoolValue)(nil), // 218: google.protobuf.BoolValue + (*ResolveCategoryMascotArgs)(nil), // 140: productv1.ResolveCategoryMascotArgs + (*ResolveCategoryMascotContext)(nil), // 141: productv1.ResolveCategoryMascotContext + (*ResolveCategoryMascotRequest)(nil), // 142: productv1.ResolveCategoryMascotRequest + (*ResolveCategoryMascotResult)(nil), // 143: productv1.ResolveCategoryMascotResult + (*ResolveCategoryMascotResponse)(nil), // 144: productv1.ResolveCategoryMascotResponse + (*ResolveCategoryCategoryStatusArgs)(nil), // 145: productv1.ResolveCategoryCategoryStatusArgs + (*ResolveCategoryCategoryStatusContext)(nil), // 146: productv1.ResolveCategoryCategoryStatusContext + (*ResolveCategoryCategoryStatusRequest)(nil), // 147: productv1.ResolveCategoryCategoryStatusRequest + (*ResolveCategoryCategoryStatusResult)(nil), // 148: productv1.ResolveCategoryCategoryStatusResult + (*ResolveCategoryCategoryStatusResponse)(nil), // 149: productv1.ResolveCategoryCategoryStatusResponse + (*ResolveSubcategoryItemCountArgs)(nil), // 150: productv1.ResolveSubcategoryItemCountArgs + (*ResolveSubcategoryItemCountContext)(nil), // 151: productv1.ResolveSubcategoryItemCountContext + (*ResolveSubcategoryItemCountRequest)(nil), // 152: productv1.ResolveSubcategoryItemCountRequest + (*ResolveSubcategoryItemCountResult)(nil), // 153: productv1.ResolveSubcategoryItemCountResult + (*ResolveSubcategoryItemCountResponse)(nil), // 154: productv1.ResolveSubcategoryItemCountResponse + (*ResolveCategoryMetricsNormalizedScoreArgs)(nil), // 155: productv1.ResolveCategoryMetricsNormalizedScoreArgs + (*ResolveCategoryMetricsNormalizedScoreContext)(nil), // 156: productv1.ResolveCategoryMetricsNormalizedScoreContext + (*ResolveCategoryMetricsNormalizedScoreRequest)(nil), // 157: productv1.ResolveCategoryMetricsNormalizedScoreRequest + (*ResolveCategoryMetricsNormalizedScoreResult)(nil), // 158: productv1.ResolveCategoryMetricsNormalizedScoreResult + (*ResolveCategoryMetricsNormalizedScoreResponse)(nil), // 159: productv1.ResolveCategoryMetricsNormalizedScoreResponse + (*Product)(nil), // 160: productv1.Product + (*Storage)(nil), // 161: productv1.Storage + (*Warehouse)(nil), // 162: productv1.Warehouse + (*User)(nil), // 163: productv1.User + (*NestedTypeA)(nil), // 164: productv1.NestedTypeA + (*RecursiveType)(nil), // 165: productv1.RecursiveType + (*TypeWithMultipleFilterFields)(nil), // 166: productv1.TypeWithMultipleFilterFields + (*FilterTypeInput)(nil), // 167: productv1.FilterTypeInput + (*ComplexFilterTypeInput)(nil), // 168: productv1.ComplexFilterTypeInput + (*TypeWithComplexFilterInput)(nil), // 169: productv1.TypeWithComplexFilterInput + (*OrderInput)(nil), // 170: productv1.OrderInput + (*Order)(nil), // 171: productv1.Order + (*Category)(nil), // 172: productv1.Category + (*CategoryFilter)(nil), // 173: productv1.CategoryFilter + (*Animal)(nil), // 174: productv1.Animal + (*SearchInput)(nil), // 175: productv1.SearchInput + (*SearchResult)(nil), // 176: productv1.SearchResult + (*NullableFieldsType)(nil), // 177: productv1.NullableFieldsType + (*NullableFieldsFilter)(nil), // 178: productv1.NullableFieldsFilter + (*BlogPost)(nil), // 179: productv1.BlogPost + (*BlogPostFilter)(nil), // 180: productv1.BlogPostFilter + (*Author)(nil), // 181: productv1.Author + (*AuthorFilter)(nil), // 182: productv1.AuthorFilter + (*UserInput)(nil), // 183: productv1.UserInput + (*ActionInput)(nil), // 184: productv1.ActionInput + (*ActionResult)(nil), // 185: productv1.ActionResult + (*NullableFieldsInput)(nil), // 186: productv1.NullableFieldsInput + (*BlogPostInput)(nil), // 187: productv1.BlogPostInput + (*AuthorInput)(nil), // 188: productv1.AuthorInput + (*NestedTypeB)(nil), // 189: productv1.NestedTypeB + (*NestedTypeC)(nil), // 190: productv1.NestedTypeC + (*FilterType)(nil), // 191: productv1.FilterType + (*Pagination)(nil), // 192: productv1.Pagination + (*OrderLineInput)(nil), // 193: productv1.OrderLineInput + (*OrderLine)(nil), // 194: productv1.OrderLine + (*Subcategory)(nil), // 195: productv1.Subcategory + (*CategoryMetrics)(nil), // 196: productv1.CategoryMetrics + (*Cat)(nil), // 197: productv1.Cat + (*Dog)(nil), // 198: productv1.Dog + (*ActionSuccess)(nil), // 199: productv1.ActionSuccess + (*ActionError)(nil), // 200: productv1.ActionError + (*CategoryInput)(nil), // 201: productv1.CategoryInput + (*ProductCountFilter)(nil), // 202: productv1.ProductCountFilter + (*SubcategoryItemFilter)(nil), // 203: productv1.SubcategoryItemFilter + (*ShippingEstimateInput)(nil), // 204: productv1.ShippingEstimateInput + (*ListOfAuthorFilter_List)(nil), // 205: productv1.ListOfAuthorFilter.List + (*ListOfAuthorInput_List)(nil), // 206: productv1.ListOfAuthorInput.List + (*ListOfBlogPost_List)(nil), // 207: productv1.ListOfBlogPost.List + (*ListOfBlogPostFilter_List)(nil), // 208: productv1.ListOfBlogPostFilter.List + (*ListOfBlogPostInput_List)(nil), // 209: productv1.ListOfBlogPostInput.List + (*ListOfBoolean_List)(nil), // 210: productv1.ListOfBoolean.List + (*ListOfCategory_List)(nil), // 211: productv1.ListOfCategory.List + (*ListOfCategoryInput_List)(nil), // 212: productv1.ListOfCategoryInput.List + (*ListOfFloat_List)(nil), // 213: productv1.ListOfFloat.List + (*ListOfListOfCategory_List)(nil), // 214: productv1.ListOfListOfCategory.List + (*ListOfListOfCategoryInput_List)(nil), // 215: productv1.ListOfListOfCategoryInput.List + (*ListOfListOfString_List)(nil), // 216: productv1.ListOfListOfString.List + (*ListOfListOfUser_List)(nil), // 217: productv1.ListOfListOfUser.List + (*ListOfListOfUserInput_List)(nil), // 218: productv1.ListOfListOfUserInput.List + (*ListOfOrderLine_List)(nil), // 219: productv1.ListOfOrderLine.List + (*ListOfProduct_List)(nil), // 220: productv1.ListOfProduct.List + (*ListOfString_List)(nil), // 221: productv1.ListOfString.List + (*ListOfSubcategory_List)(nil), // 222: productv1.ListOfSubcategory.List + (*ListOfUser_List)(nil), // 223: productv1.ListOfUser.List + (*ListOfUserInput_List)(nil), // 224: productv1.ListOfUserInput.List + (*wrapperspb.Int32Value)(nil), // 225: google.protobuf.Int32Value + (*wrapperspb.StringValue)(nil), // 226: google.protobuf.StringValue + (*wrapperspb.DoubleValue)(nil), // 227: google.protobuf.DoubleValue + (*wrapperspb.BoolValue)(nil), // 228: google.protobuf.BoolValue } var file_product_proto_depIdxs = []int32{ - 195, // 0: productv1.ListOfAuthorFilter.list:type_name -> productv1.ListOfAuthorFilter.List - 196, // 1: productv1.ListOfAuthorInput.list:type_name -> productv1.ListOfAuthorInput.List - 197, // 2: productv1.ListOfBlogPost.list:type_name -> productv1.ListOfBlogPost.List - 198, // 3: productv1.ListOfBlogPostFilter.list:type_name -> productv1.ListOfBlogPostFilter.List - 199, // 4: productv1.ListOfBlogPostInput.list:type_name -> productv1.ListOfBlogPostInput.List - 200, // 5: productv1.ListOfBoolean.list:type_name -> productv1.ListOfBoolean.List - 201, // 6: productv1.ListOfCategory.list:type_name -> productv1.ListOfCategory.List - 202, // 7: productv1.ListOfCategoryInput.list:type_name -> productv1.ListOfCategoryInput.List - 203, // 8: productv1.ListOfFloat.list:type_name -> productv1.ListOfFloat.List - 204, // 9: productv1.ListOfListOfCategory.list:type_name -> productv1.ListOfListOfCategory.List - 205, // 10: productv1.ListOfListOfCategoryInput.list:type_name -> productv1.ListOfListOfCategoryInput.List - 206, // 11: productv1.ListOfListOfString.list:type_name -> productv1.ListOfListOfString.List - 207, // 12: productv1.ListOfListOfUser.list:type_name -> productv1.ListOfListOfUser.List - 208, // 13: productv1.ListOfListOfUserInput.list:type_name -> productv1.ListOfListOfUserInput.List - 209, // 14: productv1.ListOfOrderLine.list:type_name -> productv1.ListOfOrderLine.List - 210, // 15: productv1.ListOfProduct.list:type_name -> productv1.ListOfProduct.List - 211, // 16: productv1.ListOfString.list:type_name -> productv1.ListOfString.List - 212, // 17: productv1.ListOfSubcategory.list:type_name -> productv1.ListOfSubcategory.List - 213, // 18: productv1.ListOfUser.list:type_name -> productv1.ListOfUser.List - 214, // 19: productv1.ListOfUserInput.list:type_name -> productv1.ListOfUserInput.List + 205, // 0: productv1.ListOfAuthorFilter.list:type_name -> productv1.ListOfAuthorFilter.List + 206, // 1: productv1.ListOfAuthorInput.list:type_name -> productv1.ListOfAuthorInput.List + 207, // 2: productv1.ListOfBlogPost.list:type_name -> productv1.ListOfBlogPost.List + 208, // 3: productv1.ListOfBlogPostFilter.list:type_name -> productv1.ListOfBlogPostFilter.List + 209, // 4: productv1.ListOfBlogPostInput.list:type_name -> productv1.ListOfBlogPostInput.List + 210, // 5: productv1.ListOfBoolean.list:type_name -> productv1.ListOfBoolean.List + 211, // 6: productv1.ListOfCategory.list:type_name -> productv1.ListOfCategory.List + 212, // 7: productv1.ListOfCategoryInput.list:type_name -> productv1.ListOfCategoryInput.List + 213, // 8: productv1.ListOfFloat.list:type_name -> productv1.ListOfFloat.List + 214, // 9: productv1.ListOfListOfCategory.list:type_name -> productv1.ListOfListOfCategory.List + 215, // 10: productv1.ListOfListOfCategoryInput.list:type_name -> productv1.ListOfListOfCategoryInput.List + 216, // 11: productv1.ListOfListOfString.list:type_name -> productv1.ListOfListOfString.List + 217, // 12: productv1.ListOfListOfUser.list:type_name -> productv1.ListOfListOfUser.List + 218, // 13: productv1.ListOfListOfUserInput.list:type_name -> productv1.ListOfListOfUserInput.List + 219, // 14: productv1.ListOfOrderLine.list:type_name -> productv1.ListOfOrderLine.List + 220, // 15: productv1.ListOfProduct.list:type_name -> productv1.ListOfProduct.List + 221, // 16: productv1.ListOfString.list:type_name -> productv1.ListOfString.List + 222, // 17: productv1.ListOfSubcategory.list:type_name -> productv1.ListOfSubcategory.List + 223, // 18: productv1.ListOfUser.list:type_name -> productv1.ListOfUser.List + 224, // 19: productv1.ListOfUserInput.list:type_name -> productv1.ListOfUserInput.List 22, // 20: productv1.LookupProductByIdRequest.keys:type_name -> productv1.LookupProductByIdRequestKey - 150, // 21: productv1.LookupProductByIdResponse.result:type_name -> productv1.Product + 160, // 21: productv1.LookupProductByIdResponse.result:type_name -> productv1.Product 25, // 22: productv1.LookupStorageByIdRequest.keys:type_name -> productv1.LookupStorageByIdRequestKey - 151, // 23: productv1.LookupStorageByIdResponse.result:type_name -> productv1.Storage + 161, // 23: productv1.LookupStorageByIdResponse.result:type_name -> productv1.Storage 28, // 24: productv1.LookupWarehouseByIdRequest.keys:type_name -> productv1.LookupWarehouseByIdRequestKey - 152, // 25: productv1.LookupWarehouseByIdResponse.result:type_name -> productv1.Warehouse - 153, // 26: productv1.QueryUsersResponse.users:type_name -> productv1.User - 153, // 27: productv1.QueryUserResponse.user:type_name -> productv1.User - 154, // 28: productv1.QueryNestedTypeResponse.nested_type:type_name -> productv1.NestedTypeA - 155, // 29: productv1.QueryRecursiveTypeResponse.recursive_type:type_name -> productv1.RecursiveType - 156, // 30: productv1.QueryTypeFilterWithArgumentsResponse.type_filter_with_arguments:type_name -> productv1.TypeWithMultipleFilterFields - 157, // 31: productv1.QueryTypeWithMultipleFilterFieldsRequest.filter:type_name -> productv1.FilterTypeInput - 156, // 32: productv1.QueryTypeWithMultipleFilterFieldsResponse.type_with_multiple_filter_fields:type_name -> productv1.TypeWithMultipleFilterFields - 158, // 33: productv1.QueryComplexFilterTypeRequest.filter:type_name -> productv1.ComplexFilterTypeInput - 159, // 34: productv1.QueryComplexFilterTypeResponse.complex_filter_type:type_name -> productv1.TypeWithComplexFilterInput - 160, // 35: productv1.QueryCalculateTotalsRequest.orders:type_name -> productv1.OrderInput - 161, // 36: productv1.QueryCalculateTotalsResponse.calculate_totals:type_name -> productv1.Order - 162, // 37: productv1.QueryCategoriesResponse.categories:type_name -> productv1.Category + 162, // 25: productv1.LookupWarehouseByIdResponse.result:type_name -> productv1.Warehouse + 163, // 26: productv1.QueryUsersResponse.users:type_name -> productv1.User + 163, // 27: productv1.QueryUserResponse.user:type_name -> productv1.User + 164, // 28: productv1.QueryNestedTypeResponse.nested_type:type_name -> productv1.NestedTypeA + 165, // 29: productv1.QueryRecursiveTypeResponse.recursive_type:type_name -> productv1.RecursiveType + 166, // 30: productv1.QueryTypeFilterWithArgumentsResponse.type_filter_with_arguments:type_name -> productv1.TypeWithMultipleFilterFields + 167, // 31: productv1.QueryTypeWithMultipleFilterFieldsRequest.filter:type_name -> productv1.FilterTypeInput + 166, // 32: productv1.QueryTypeWithMultipleFilterFieldsResponse.type_with_multiple_filter_fields:type_name -> productv1.TypeWithMultipleFilterFields + 168, // 33: productv1.QueryComplexFilterTypeRequest.filter:type_name -> productv1.ComplexFilterTypeInput + 169, // 34: productv1.QueryComplexFilterTypeResponse.complex_filter_type:type_name -> productv1.TypeWithComplexFilterInput + 170, // 35: productv1.QueryCalculateTotalsRequest.orders:type_name -> productv1.OrderInput + 171, // 36: productv1.QueryCalculateTotalsResponse.calculate_totals:type_name -> productv1.Order + 172, // 37: productv1.QueryCategoriesResponse.categories:type_name -> productv1.Category 0, // 38: productv1.QueryCategoriesByKindRequest.kind:type_name -> productv1.CategoryKind - 162, // 39: productv1.QueryCategoriesByKindResponse.categories_by_kind:type_name -> productv1.Category + 172, // 39: productv1.QueryCategoriesByKindResponse.categories_by_kind:type_name -> productv1.Category 0, // 40: productv1.QueryCategoriesByKindsRequest.kinds:type_name -> productv1.CategoryKind - 162, // 41: productv1.QueryCategoriesByKindsResponse.categories_by_kinds:type_name -> productv1.Category - 163, // 42: productv1.QueryFilterCategoriesRequest.filter:type_name -> productv1.CategoryFilter - 162, // 43: productv1.QueryFilterCategoriesResponse.filter_categories:type_name -> productv1.Category - 164, // 44: productv1.QueryRandomPetResponse.random_pet:type_name -> productv1.Animal - 164, // 45: productv1.QueryAllPetsResponse.all_pets:type_name -> productv1.Animal - 165, // 46: productv1.QuerySearchRequest.input:type_name -> productv1.SearchInput - 166, // 47: productv1.QuerySearchResponse.search:type_name -> productv1.SearchResult - 166, // 48: productv1.QueryRandomSearchResultResponse.random_search_result:type_name -> productv1.SearchResult - 167, // 49: productv1.QueryNullableFieldsTypeResponse.nullable_fields_type:type_name -> productv1.NullableFieldsType - 167, // 50: productv1.QueryNullableFieldsTypeByIdResponse.nullable_fields_type_by_id:type_name -> productv1.NullableFieldsType - 168, // 51: productv1.QueryNullableFieldsTypeWithFilterRequest.filter:type_name -> productv1.NullableFieldsFilter - 167, // 52: productv1.QueryNullableFieldsTypeWithFilterResponse.nullable_fields_type_with_filter:type_name -> productv1.NullableFieldsType - 167, // 53: productv1.QueryAllNullableFieldsTypesResponse.all_nullable_fields_types:type_name -> productv1.NullableFieldsType - 169, // 54: productv1.QueryBlogPostResponse.blog_post:type_name -> productv1.BlogPost - 169, // 55: productv1.QueryBlogPostByIdResponse.blog_post_by_id:type_name -> productv1.BlogPost - 170, // 56: productv1.QueryBlogPostsWithFilterRequest.filter:type_name -> productv1.BlogPostFilter - 169, // 57: productv1.QueryBlogPostsWithFilterResponse.blog_posts_with_filter:type_name -> productv1.BlogPost - 169, // 58: productv1.QueryAllBlogPostsResponse.all_blog_posts:type_name -> productv1.BlogPost - 171, // 59: productv1.QueryAuthorResponse.author:type_name -> productv1.Author - 171, // 60: productv1.QueryAuthorByIdResponse.author_by_id:type_name -> productv1.Author - 172, // 61: productv1.QueryAuthorsWithFilterRequest.filter:type_name -> productv1.AuthorFilter - 171, // 62: productv1.QueryAuthorsWithFilterResponse.authors_with_filter:type_name -> productv1.Author - 171, // 63: productv1.QueryAllAuthorsResponse.all_authors:type_name -> productv1.Author + 172, // 41: productv1.QueryCategoriesByKindsResponse.categories_by_kinds:type_name -> productv1.Category + 173, // 42: productv1.QueryFilterCategoriesRequest.filter:type_name -> productv1.CategoryFilter + 172, // 43: productv1.QueryFilterCategoriesResponse.filter_categories:type_name -> productv1.Category + 174, // 44: productv1.QueryRandomPetResponse.random_pet:type_name -> productv1.Animal + 174, // 45: productv1.QueryAllPetsResponse.all_pets:type_name -> productv1.Animal + 175, // 46: productv1.QuerySearchRequest.input:type_name -> productv1.SearchInput + 176, // 47: productv1.QuerySearchResponse.search:type_name -> productv1.SearchResult + 176, // 48: productv1.QueryRandomSearchResultResponse.random_search_result:type_name -> productv1.SearchResult + 177, // 49: productv1.QueryNullableFieldsTypeResponse.nullable_fields_type:type_name -> productv1.NullableFieldsType + 177, // 50: productv1.QueryNullableFieldsTypeByIdResponse.nullable_fields_type_by_id:type_name -> productv1.NullableFieldsType + 178, // 51: productv1.QueryNullableFieldsTypeWithFilterRequest.filter:type_name -> productv1.NullableFieldsFilter + 177, // 52: productv1.QueryNullableFieldsTypeWithFilterResponse.nullable_fields_type_with_filter:type_name -> productv1.NullableFieldsType + 177, // 53: productv1.QueryAllNullableFieldsTypesResponse.all_nullable_fields_types:type_name -> productv1.NullableFieldsType + 179, // 54: productv1.QueryBlogPostResponse.blog_post:type_name -> productv1.BlogPost + 179, // 55: productv1.QueryBlogPostByIdResponse.blog_post_by_id:type_name -> productv1.BlogPost + 180, // 56: productv1.QueryBlogPostsWithFilterRequest.filter:type_name -> productv1.BlogPostFilter + 179, // 57: productv1.QueryBlogPostsWithFilterResponse.blog_posts_with_filter:type_name -> productv1.BlogPost + 179, // 58: productv1.QueryAllBlogPostsResponse.all_blog_posts:type_name -> productv1.BlogPost + 181, // 59: productv1.QueryAuthorResponse.author:type_name -> productv1.Author + 181, // 60: productv1.QueryAuthorByIdResponse.author_by_id:type_name -> productv1.Author + 182, // 61: productv1.QueryAuthorsWithFilterRequest.filter:type_name -> productv1.AuthorFilter + 181, // 62: productv1.QueryAuthorsWithFilterResponse.authors_with_filter:type_name -> productv1.Author + 181, // 63: productv1.QueryAllAuthorsResponse.all_authors:type_name -> productv1.Author 2, // 64: productv1.QueryBulkSearchAuthorsRequest.filters:type_name -> productv1.ListOfAuthorFilter - 171, // 65: productv1.QueryBulkSearchAuthorsResponse.bulk_search_authors:type_name -> productv1.Author + 181, // 65: productv1.QueryBulkSearchAuthorsResponse.bulk_search_authors:type_name -> productv1.Author 5, // 66: productv1.QueryBulkSearchBlogPostsRequest.filters:type_name -> productv1.ListOfBlogPostFilter - 169, // 67: productv1.QueryBulkSearchBlogPostsResponse.bulk_search_blog_posts:type_name -> productv1.BlogPost - 173, // 68: productv1.MutationCreateUserRequest.input:type_name -> productv1.UserInput - 153, // 69: productv1.MutationCreateUserResponse.create_user:type_name -> productv1.User - 174, // 70: productv1.MutationPerformActionRequest.input:type_name -> productv1.ActionInput - 175, // 71: productv1.MutationPerformActionResponse.perform_action:type_name -> productv1.ActionResult - 176, // 72: productv1.MutationCreateNullableFieldsTypeRequest.input:type_name -> productv1.NullableFieldsInput - 167, // 73: productv1.MutationCreateNullableFieldsTypeResponse.create_nullable_fields_type:type_name -> productv1.NullableFieldsType - 176, // 74: productv1.MutationUpdateNullableFieldsTypeRequest.input:type_name -> productv1.NullableFieldsInput - 167, // 75: productv1.MutationUpdateNullableFieldsTypeResponse.update_nullable_fields_type:type_name -> productv1.NullableFieldsType - 177, // 76: productv1.MutationCreateBlogPostRequest.input:type_name -> productv1.BlogPostInput - 169, // 77: productv1.MutationCreateBlogPostResponse.create_blog_post:type_name -> productv1.BlogPost - 177, // 78: productv1.MutationUpdateBlogPostRequest.input:type_name -> productv1.BlogPostInput - 169, // 79: productv1.MutationUpdateBlogPostResponse.update_blog_post:type_name -> productv1.BlogPost - 178, // 80: productv1.MutationCreateAuthorRequest.input:type_name -> productv1.AuthorInput - 171, // 81: productv1.MutationCreateAuthorResponse.create_author:type_name -> productv1.Author - 178, // 82: productv1.MutationUpdateAuthorRequest.input:type_name -> productv1.AuthorInput - 171, // 83: productv1.MutationUpdateAuthorResponse.update_author:type_name -> productv1.Author + 179, // 67: productv1.QueryBulkSearchBlogPostsResponse.bulk_search_blog_posts:type_name -> productv1.BlogPost + 183, // 68: productv1.MutationCreateUserRequest.input:type_name -> productv1.UserInput + 163, // 69: productv1.MutationCreateUserResponse.create_user:type_name -> productv1.User + 184, // 70: productv1.MutationPerformActionRequest.input:type_name -> productv1.ActionInput + 185, // 71: productv1.MutationPerformActionResponse.perform_action:type_name -> productv1.ActionResult + 186, // 72: productv1.MutationCreateNullableFieldsTypeRequest.input:type_name -> productv1.NullableFieldsInput + 177, // 73: productv1.MutationCreateNullableFieldsTypeResponse.create_nullable_fields_type:type_name -> productv1.NullableFieldsType + 186, // 74: productv1.MutationUpdateNullableFieldsTypeRequest.input:type_name -> productv1.NullableFieldsInput + 177, // 75: productv1.MutationUpdateNullableFieldsTypeResponse.update_nullable_fields_type:type_name -> productv1.NullableFieldsType + 187, // 76: productv1.MutationCreateBlogPostRequest.input:type_name -> productv1.BlogPostInput + 179, // 77: productv1.MutationCreateBlogPostResponse.create_blog_post:type_name -> productv1.BlogPost + 187, // 78: productv1.MutationUpdateBlogPostRequest.input:type_name -> productv1.BlogPostInput + 179, // 79: productv1.MutationUpdateBlogPostResponse.update_blog_post:type_name -> productv1.BlogPost + 188, // 80: productv1.MutationCreateAuthorRequest.input:type_name -> productv1.AuthorInput + 181, // 81: productv1.MutationCreateAuthorResponse.create_author:type_name -> productv1.Author + 188, // 82: productv1.MutationUpdateAuthorRequest.input:type_name -> productv1.AuthorInput + 181, // 83: productv1.MutationUpdateAuthorResponse.update_author:type_name -> productv1.Author 3, // 84: productv1.MutationBulkCreateAuthorsRequest.authors:type_name -> productv1.ListOfAuthorInput - 171, // 85: productv1.MutationBulkCreateAuthorsResponse.bulk_create_authors:type_name -> productv1.Author + 181, // 85: productv1.MutationBulkCreateAuthorsResponse.bulk_create_authors:type_name -> productv1.Author 3, // 86: productv1.MutationBulkUpdateAuthorsRequest.authors:type_name -> productv1.ListOfAuthorInput - 171, // 87: productv1.MutationBulkUpdateAuthorsResponse.bulk_update_authors:type_name -> productv1.Author + 181, // 87: productv1.MutationBulkUpdateAuthorsResponse.bulk_update_authors:type_name -> productv1.Author 6, // 88: productv1.MutationBulkCreateBlogPostsRequest.blog_posts:type_name -> productv1.ListOfBlogPostInput - 169, // 89: productv1.MutationBulkCreateBlogPostsResponse.bulk_create_blog_posts:type_name -> productv1.BlogPost + 179, // 89: productv1.MutationBulkCreateBlogPostsResponse.bulk_create_blog_posts:type_name -> productv1.BlogPost 6, // 90: productv1.MutationBulkUpdateBlogPostsRequest.blog_posts:type_name -> productv1.ListOfBlogPostInput - 169, // 91: productv1.MutationBulkUpdateBlogPostsResponse.bulk_update_blog_posts:type_name -> productv1.BlogPost - 194, // 92: productv1.ResolveProductShippingEstimateArgs.input:type_name -> productv1.ShippingEstimateInput + 179, // 91: productv1.MutationBulkUpdateBlogPostsResponse.bulk_update_blog_posts:type_name -> productv1.BlogPost + 204, // 92: productv1.ResolveProductShippingEstimateArgs.input:type_name -> productv1.ShippingEstimateInput 116, // 93: productv1.ResolveProductShippingEstimateRequest.context:type_name -> productv1.ResolveProductShippingEstimateContext 115, // 94: productv1.ResolveProductShippingEstimateRequest.field_args:type_name -> productv1.ResolveProductShippingEstimateArgs 118, // 95: productv1.ResolveProductShippingEstimateResponse.result:type_name -> productv1.ResolveProductShippingEstimateResult 121, // 96: productv1.ResolveProductRecommendedCategoryRequest.context:type_name -> productv1.ResolveProductRecommendedCategoryContext 120, // 97: productv1.ResolveProductRecommendedCategoryRequest.field_args:type_name -> productv1.ResolveProductRecommendedCategoryArgs - 162, // 98: productv1.ResolveProductRecommendedCategoryResult.recommended_category:type_name -> productv1.Category + 172, // 98: productv1.ResolveProductRecommendedCategoryResult.recommended_category:type_name -> productv1.Category 123, // 99: productv1.ResolveProductRecommendedCategoryResponse.result:type_name -> productv1.ResolveProductRecommendedCategoryResult - 192, // 100: productv1.ResolveCategoryProductCountArgs.filters:type_name -> productv1.ProductCountFilter + 202, // 100: productv1.ResolveCategoryProductCountArgs.filters:type_name -> productv1.ProductCountFilter 126, // 101: productv1.ResolveCategoryProductCountRequest.context:type_name -> productv1.ResolveCategoryProductCountContext 125, // 102: productv1.ResolveCategoryProductCountRequest.field_args:type_name -> productv1.ResolveCategoryProductCountArgs 128, // 103: productv1.ResolveCategoryProductCountResponse.result:type_name -> productv1.ResolveCategoryProductCountResult - 215, // 104: productv1.ResolveCategoryPopularityScoreArgs.threshold:type_name -> google.protobuf.Int32Value + 225, // 104: productv1.ResolveCategoryPopularityScoreArgs.threshold:type_name -> google.protobuf.Int32Value 131, // 105: productv1.ResolveCategoryPopularityScoreRequest.context:type_name -> productv1.ResolveCategoryPopularityScoreContext 130, // 106: productv1.ResolveCategoryPopularityScoreRequest.field_args:type_name -> productv1.ResolveCategoryPopularityScoreArgs - 215, // 107: productv1.ResolveCategoryPopularityScoreResult.popularity_score:type_name -> google.protobuf.Int32Value + 225, // 107: productv1.ResolveCategoryPopularityScoreResult.popularity_score:type_name -> google.protobuf.Int32Value 133, // 108: productv1.ResolveCategoryPopularityScoreResponse.result:type_name -> productv1.ResolveCategoryPopularityScoreResult 136, // 109: productv1.ResolveCategoryCategoryMetricsRequest.context:type_name -> productv1.ResolveCategoryCategoryMetricsContext 135, // 110: productv1.ResolveCategoryCategoryMetricsRequest.field_args:type_name -> productv1.ResolveCategoryCategoryMetricsArgs - 186, // 111: productv1.ResolveCategoryCategoryMetricsResult.category_metrics:type_name -> productv1.CategoryMetrics + 196, // 111: productv1.ResolveCategoryCategoryMetricsResult.category_metrics:type_name -> productv1.CategoryMetrics 138, // 112: productv1.ResolveCategoryCategoryMetricsResponse.result:type_name -> productv1.ResolveCategoryCategoryMetricsResult - 193, // 113: productv1.ResolveSubcategoryItemCountArgs.filters:type_name -> productv1.SubcategoryItemFilter - 141, // 114: productv1.ResolveSubcategoryItemCountRequest.context:type_name -> productv1.ResolveSubcategoryItemCountContext - 140, // 115: productv1.ResolveSubcategoryItemCountRequest.field_args:type_name -> productv1.ResolveSubcategoryItemCountArgs - 143, // 116: productv1.ResolveSubcategoryItemCountResponse.result:type_name -> productv1.ResolveSubcategoryItemCountResult - 146, // 117: productv1.ResolveCategoryMetricsNormalizedScoreRequest.context:type_name -> productv1.ResolveCategoryMetricsNormalizedScoreContext - 145, // 118: productv1.ResolveCategoryMetricsNormalizedScoreRequest.field_args:type_name -> productv1.ResolveCategoryMetricsNormalizedScoreArgs - 148, // 119: productv1.ResolveCategoryMetricsNormalizedScoreResponse.result:type_name -> productv1.ResolveCategoryMetricsNormalizedScoreResult - 179, // 120: productv1.NestedTypeA.b:type_name -> productv1.NestedTypeB - 155, // 121: productv1.RecursiveType.recursive_type:type_name -> productv1.RecursiveType - 181, // 122: productv1.ComplexFilterTypeInput.filter:type_name -> productv1.FilterType - 183, // 123: productv1.OrderInput.lines:type_name -> productv1.OrderLineInput - 16, // 124: productv1.Order.order_lines:type_name -> productv1.ListOfOrderLine - 0, // 125: productv1.Category.kind:type_name -> productv1.CategoryKind - 19, // 126: productv1.Category.subcategories:type_name -> productv1.ListOfSubcategory - 0, // 127: productv1.CategoryFilter.category:type_name -> productv1.CategoryKind - 182, // 128: productv1.CategoryFilter.pagination:type_name -> productv1.Pagination - 187, // 129: productv1.Animal.cat:type_name -> productv1.Cat - 188, // 130: productv1.Animal.dog:type_name -> productv1.Dog - 215, // 131: productv1.SearchInput.limit:type_name -> google.protobuf.Int32Value - 150, // 132: productv1.SearchResult.product:type_name -> productv1.Product - 153, // 133: productv1.SearchResult.user:type_name -> productv1.User - 162, // 134: productv1.SearchResult.category:type_name -> productv1.Category - 216, // 135: productv1.NullableFieldsType.optional_string:type_name -> google.protobuf.StringValue - 215, // 136: productv1.NullableFieldsType.optional_int:type_name -> google.protobuf.Int32Value - 217, // 137: productv1.NullableFieldsType.optional_float:type_name -> google.protobuf.DoubleValue - 218, // 138: productv1.NullableFieldsType.optional_boolean:type_name -> google.protobuf.BoolValue - 216, // 139: productv1.NullableFieldsFilter.name:type_name -> google.protobuf.StringValue - 216, // 140: productv1.NullableFieldsFilter.optional_string:type_name -> google.protobuf.StringValue - 218, // 141: productv1.NullableFieldsFilter.include_nulls:type_name -> google.protobuf.BoolValue - 18, // 142: productv1.BlogPost.optional_tags:type_name -> productv1.ListOfString - 18, // 143: productv1.BlogPost.keywords:type_name -> productv1.ListOfString - 10, // 144: productv1.BlogPost.ratings:type_name -> productv1.ListOfFloat - 7, // 145: productv1.BlogPost.is_published:type_name -> productv1.ListOfBoolean - 13, // 146: productv1.BlogPost.tag_groups:type_name -> productv1.ListOfListOfString - 13, // 147: productv1.BlogPost.related_topics:type_name -> productv1.ListOfListOfString - 13, // 148: productv1.BlogPost.comment_threads:type_name -> productv1.ListOfListOfString - 13, // 149: productv1.BlogPost.suggestions:type_name -> productv1.ListOfListOfString - 162, // 150: productv1.BlogPost.related_categories:type_name -> productv1.Category - 153, // 151: productv1.BlogPost.contributors:type_name -> productv1.User - 17, // 152: productv1.BlogPost.mentioned_products:type_name -> productv1.ListOfProduct - 20, // 153: productv1.BlogPost.mentioned_users:type_name -> productv1.ListOfUser - 11, // 154: productv1.BlogPost.category_groups:type_name -> productv1.ListOfListOfCategory - 14, // 155: productv1.BlogPost.contributor_teams:type_name -> productv1.ListOfListOfUser - 216, // 156: productv1.BlogPostFilter.title:type_name -> google.protobuf.StringValue - 218, // 157: productv1.BlogPostFilter.has_categories:type_name -> google.protobuf.BoolValue - 215, // 158: productv1.BlogPostFilter.min_tags:type_name -> google.protobuf.Int32Value - 216, // 159: productv1.Author.email:type_name -> google.protobuf.StringValue - 18, // 160: productv1.Author.social_links:type_name -> productv1.ListOfString - 13, // 161: productv1.Author.teams_by_project:type_name -> productv1.ListOfListOfString - 13, // 162: productv1.Author.collaborations:type_name -> productv1.ListOfListOfString - 4, // 163: productv1.Author.written_posts:type_name -> productv1.ListOfBlogPost - 162, // 164: productv1.Author.favorite_categories:type_name -> productv1.Category - 20, // 165: productv1.Author.related_authors:type_name -> productv1.ListOfUser - 17, // 166: productv1.Author.product_reviews:type_name -> productv1.ListOfProduct - 14, // 167: productv1.Author.author_groups:type_name -> productv1.ListOfListOfUser - 11, // 168: productv1.Author.category_preferences:type_name -> productv1.ListOfListOfCategory - 14, // 169: productv1.Author.project_teams:type_name -> productv1.ListOfListOfUser - 216, // 170: productv1.AuthorFilter.name:type_name -> google.protobuf.StringValue - 218, // 171: productv1.AuthorFilter.has_teams:type_name -> google.protobuf.BoolValue - 215, // 172: productv1.AuthorFilter.skill_count:type_name -> google.protobuf.Int32Value - 189, // 173: productv1.ActionResult.action_success:type_name -> productv1.ActionSuccess - 190, // 174: productv1.ActionResult.action_error:type_name -> productv1.ActionError - 216, // 175: productv1.NullableFieldsInput.optional_string:type_name -> google.protobuf.StringValue - 215, // 176: productv1.NullableFieldsInput.optional_int:type_name -> google.protobuf.Int32Value - 217, // 177: productv1.NullableFieldsInput.optional_float:type_name -> google.protobuf.DoubleValue - 218, // 178: productv1.NullableFieldsInput.optional_boolean:type_name -> google.protobuf.BoolValue - 18, // 179: productv1.BlogPostInput.optional_tags:type_name -> productv1.ListOfString - 18, // 180: productv1.BlogPostInput.keywords:type_name -> productv1.ListOfString - 10, // 181: productv1.BlogPostInput.ratings:type_name -> productv1.ListOfFloat - 7, // 182: productv1.BlogPostInput.is_published:type_name -> productv1.ListOfBoolean - 13, // 183: productv1.BlogPostInput.tag_groups:type_name -> productv1.ListOfListOfString - 13, // 184: productv1.BlogPostInput.related_topics:type_name -> productv1.ListOfListOfString - 13, // 185: productv1.BlogPostInput.comment_threads:type_name -> productv1.ListOfListOfString - 13, // 186: productv1.BlogPostInput.suggestions:type_name -> productv1.ListOfListOfString - 9, // 187: productv1.BlogPostInput.related_categories:type_name -> productv1.ListOfCategoryInput - 21, // 188: productv1.BlogPostInput.contributors:type_name -> productv1.ListOfUserInput - 12, // 189: productv1.BlogPostInput.category_groups:type_name -> productv1.ListOfListOfCategoryInput - 216, // 190: productv1.AuthorInput.email:type_name -> google.protobuf.StringValue - 18, // 191: productv1.AuthorInput.social_links:type_name -> productv1.ListOfString - 13, // 192: productv1.AuthorInput.teams_by_project:type_name -> productv1.ListOfListOfString - 13, // 193: productv1.AuthorInput.collaborations:type_name -> productv1.ListOfListOfString - 191, // 194: productv1.AuthorInput.favorite_categories:type_name -> productv1.CategoryInput - 15, // 195: productv1.AuthorInput.author_groups:type_name -> productv1.ListOfListOfUserInput - 15, // 196: productv1.AuthorInput.project_teams:type_name -> productv1.ListOfListOfUserInput - 180, // 197: productv1.NestedTypeB.c:type_name -> productv1.NestedTypeC - 182, // 198: productv1.FilterType.pagination:type_name -> productv1.Pagination - 18, // 199: productv1.OrderLineInput.modifiers:type_name -> productv1.ListOfString - 18, // 200: productv1.OrderLine.modifiers:type_name -> productv1.ListOfString - 216, // 201: productv1.Subcategory.description:type_name -> google.protobuf.StringValue - 162, // 202: productv1.CategoryMetrics.related_category:type_name -> productv1.Category - 0, // 203: productv1.CategoryInput.kind:type_name -> productv1.CategoryKind - 217, // 204: productv1.ProductCountFilter.min_price:type_name -> google.protobuf.DoubleValue - 217, // 205: productv1.ProductCountFilter.max_price:type_name -> google.protobuf.DoubleValue - 218, // 206: productv1.ProductCountFilter.in_stock:type_name -> google.protobuf.BoolValue - 216, // 207: productv1.ProductCountFilter.search_term:type_name -> google.protobuf.StringValue - 217, // 208: productv1.SubcategoryItemFilter.min_price:type_name -> google.protobuf.DoubleValue - 217, // 209: productv1.SubcategoryItemFilter.max_price:type_name -> google.protobuf.DoubleValue - 218, // 210: productv1.SubcategoryItemFilter.in_stock:type_name -> google.protobuf.BoolValue - 218, // 211: productv1.SubcategoryItemFilter.is_active:type_name -> google.protobuf.BoolValue - 216, // 212: productv1.SubcategoryItemFilter.search_term:type_name -> google.protobuf.StringValue - 1, // 213: productv1.ShippingEstimateInput.destination:type_name -> productv1.ShippingDestination - 218, // 214: productv1.ShippingEstimateInput.expedited:type_name -> google.protobuf.BoolValue - 172, // 215: productv1.ListOfAuthorFilter.List.items:type_name -> productv1.AuthorFilter - 178, // 216: productv1.ListOfAuthorInput.List.items:type_name -> productv1.AuthorInput - 169, // 217: productv1.ListOfBlogPost.List.items:type_name -> productv1.BlogPost - 170, // 218: productv1.ListOfBlogPostFilter.List.items:type_name -> productv1.BlogPostFilter - 177, // 219: productv1.ListOfBlogPostInput.List.items:type_name -> productv1.BlogPostInput - 162, // 220: productv1.ListOfCategory.List.items:type_name -> productv1.Category - 191, // 221: productv1.ListOfCategoryInput.List.items:type_name -> productv1.CategoryInput - 8, // 222: productv1.ListOfListOfCategory.List.items:type_name -> productv1.ListOfCategory - 9, // 223: productv1.ListOfListOfCategoryInput.List.items:type_name -> productv1.ListOfCategoryInput - 18, // 224: productv1.ListOfListOfString.List.items:type_name -> productv1.ListOfString - 20, // 225: productv1.ListOfListOfUser.List.items:type_name -> productv1.ListOfUser - 21, // 226: productv1.ListOfListOfUserInput.List.items:type_name -> productv1.ListOfUserInput - 184, // 227: productv1.ListOfOrderLine.List.items:type_name -> productv1.OrderLine - 150, // 228: productv1.ListOfProduct.List.items:type_name -> productv1.Product - 185, // 229: productv1.ListOfSubcategory.List.items:type_name -> productv1.Subcategory - 153, // 230: productv1.ListOfUser.List.items:type_name -> productv1.User - 173, // 231: productv1.ListOfUserInput.List.items:type_name -> productv1.UserInput - 23, // 232: productv1.ProductService.LookupProductById:input_type -> productv1.LookupProductByIdRequest - 26, // 233: productv1.ProductService.LookupStorageById:input_type -> productv1.LookupStorageByIdRequest - 29, // 234: productv1.ProductService.LookupWarehouseById:input_type -> productv1.LookupWarehouseByIdRequest - 107, // 235: productv1.ProductService.MutationBulkCreateAuthors:input_type -> productv1.MutationBulkCreateAuthorsRequest - 111, // 236: productv1.ProductService.MutationBulkCreateBlogPosts:input_type -> productv1.MutationBulkCreateBlogPostsRequest - 109, // 237: productv1.ProductService.MutationBulkUpdateAuthors:input_type -> productv1.MutationBulkUpdateAuthorsRequest - 113, // 238: productv1.ProductService.MutationBulkUpdateBlogPosts:input_type -> productv1.MutationBulkUpdateBlogPostsRequest - 103, // 239: productv1.ProductService.MutationCreateAuthor:input_type -> productv1.MutationCreateAuthorRequest - 99, // 240: productv1.ProductService.MutationCreateBlogPost:input_type -> productv1.MutationCreateBlogPostRequest - 95, // 241: productv1.ProductService.MutationCreateNullableFieldsType:input_type -> productv1.MutationCreateNullableFieldsTypeRequest - 91, // 242: productv1.ProductService.MutationCreateUser:input_type -> productv1.MutationCreateUserRequest - 93, // 243: productv1.ProductService.MutationPerformAction:input_type -> productv1.MutationPerformActionRequest - 105, // 244: productv1.ProductService.MutationUpdateAuthor:input_type -> productv1.MutationUpdateAuthorRequest - 101, // 245: productv1.ProductService.MutationUpdateBlogPost:input_type -> productv1.MutationUpdateBlogPostRequest - 97, // 246: productv1.ProductService.MutationUpdateNullableFieldsType:input_type -> productv1.MutationUpdateNullableFieldsTypeRequest - 85, // 247: productv1.ProductService.QueryAllAuthors:input_type -> productv1.QueryAllAuthorsRequest - 77, // 248: productv1.ProductService.QueryAllBlogPosts:input_type -> productv1.QueryAllBlogPostsRequest - 69, // 249: productv1.ProductService.QueryAllNullableFieldsTypes:input_type -> productv1.QueryAllNullableFieldsTypesRequest - 57, // 250: productv1.ProductService.QueryAllPets:input_type -> productv1.QueryAllPetsRequest - 79, // 251: productv1.ProductService.QueryAuthor:input_type -> productv1.QueryAuthorRequest - 81, // 252: productv1.ProductService.QueryAuthorById:input_type -> productv1.QueryAuthorByIdRequest - 83, // 253: productv1.ProductService.QueryAuthorsWithFilter:input_type -> productv1.QueryAuthorsWithFilterRequest - 71, // 254: productv1.ProductService.QueryBlogPost:input_type -> productv1.QueryBlogPostRequest - 73, // 255: productv1.ProductService.QueryBlogPostById:input_type -> productv1.QueryBlogPostByIdRequest - 75, // 256: productv1.ProductService.QueryBlogPostsWithFilter:input_type -> productv1.QueryBlogPostsWithFilterRequest - 87, // 257: productv1.ProductService.QueryBulkSearchAuthors:input_type -> productv1.QueryBulkSearchAuthorsRequest - 89, // 258: productv1.ProductService.QueryBulkSearchBlogPosts:input_type -> productv1.QueryBulkSearchBlogPostsRequest - 45, // 259: productv1.ProductService.QueryCalculateTotals:input_type -> productv1.QueryCalculateTotalsRequest - 47, // 260: productv1.ProductService.QueryCategories:input_type -> productv1.QueryCategoriesRequest - 49, // 261: productv1.ProductService.QueryCategoriesByKind:input_type -> productv1.QueryCategoriesByKindRequest - 51, // 262: productv1.ProductService.QueryCategoriesByKinds:input_type -> productv1.QueryCategoriesByKindsRequest - 43, // 263: productv1.ProductService.QueryComplexFilterType:input_type -> productv1.QueryComplexFilterTypeRequest - 53, // 264: productv1.ProductService.QueryFilterCategories:input_type -> productv1.QueryFilterCategoriesRequest - 35, // 265: productv1.ProductService.QueryNestedType:input_type -> productv1.QueryNestedTypeRequest - 63, // 266: productv1.ProductService.QueryNullableFieldsType:input_type -> productv1.QueryNullableFieldsTypeRequest - 65, // 267: productv1.ProductService.QueryNullableFieldsTypeById:input_type -> productv1.QueryNullableFieldsTypeByIdRequest - 67, // 268: productv1.ProductService.QueryNullableFieldsTypeWithFilter:input_type -> productv1.QueryNullableFieldsTypeWithFilterRequest - 55, // 269: productv1.ProductService.QueryRandomPet:input_type -> productv1.QueryRandomPetRequest - 61, // 270: productv1.ProductService.QueryRandomSearchResult:input_type -> productv1.QueryRandomSearchResultRequest - 37, // 271: productv1.ProductService.QueryRecursiveType:input_type -> productv1.QueryRecursiveTypeRequest - 59, // 272: productv1.ProductService.QuerySearch:input_type -> productv1.QuerySearchRequest - 39, // 273: productv1.ProductService.QueryTypeFilterWithArguments:input_type -> productv1.QueryTypeFilterWithArgumentsRequest - 41, // 274: productv1.ProductService.QueryTypeWithMultipleFilterFields:input_type -> productv1.QueryTypeWithMultipleFilterFieldsRequest - 33, // 275: productv1.ProductService.QueryUser:input_type -> productv1.QueryUserRequest - 31, // 276: productv1.ProductService.QueryUsers:input_type -> productv1.QueryUsersRequest - 137, // 277: productv1.ProductService.ResolveCategoryCategoryMetrics:input_type -> productv1.ResolveCategoryCategoryMetricsRequest - 147, // 278: productv1.ProductService.ResolveCategoryMetricsNormalizedScore:input_type -> productv1.ResolveCategoryMetricsNormalizedScoreRequest - 132, // 279: productv1.ProductService.ResolveCategoryPopularityScore:input_type -> productv1.ResolveCategoryPopularityScoreRequest - 127, // 280: productv1.ProductService.ResolveCategoryProductCount:input_type -> productv1.ResolveCategoryProductCountRequest - 122, // 281: productv1.ProductService.ResolveProductRecommendedCategory:input_type -> productv1.ResolveProductRecommendedCategoryRequest - 117, // 282: productv1.ProductService.ResolveProductShippingEstimate:input_type -> productv1.ResolveProductShippingEstimateRequest - 142, // 283: productv1.ProductService.ResolveSubcategoryItemCount:input_type -> productv1.ResolveSubcategoryItemCountRequest - 24, // 284: productv1.ProductService.LookupProductById:output_type -> productv1.LookupProductByIdResponse - 27, // 285: productv1.ProductService.LookupStorageById:output_type -> productv1.LookupStorageByIdResponse - 30, // 286: productv1.ProductService.LookupWarehouseById:output_type -> productv1.LookupWarehouseByIdResponse - 108, // 287: productv1.ProductService.MutationBulkCreateAuthors:output_type -> productv1.MutationBulkCreateAuthorsResponse - 112, // 288: productv1.ProductService.MutationBulkCreateBlogPosts:output_type -> productv1.MutationBulkCreateBlogPostsResponse - 110, // 289: productv1.ProductService.MutationBulkUpdateAuthors:output_type -> productv1.MutationBulkUpdateAuthorsResponse - 114, // 290: productv1.ProductService.MutationBulkUpdateBlogPosts:output_type -> productv1.MutationBulkUpdateBlogPostsResponse - 104, // 291: productv1.ProductService.MutationCreateAuthor:output_type -> productv1.MutationCreateAuthorResponse - 100, // 292: productv1.ProductService.MutationCreateBlogPost:output_type -> productv1.MutationCreateBlogPostResponse - 96, // 293: productv1.ProductService.MutationCreateNullableFieldsType:output_type -> productv1.MutationCreateNullableFieldsTypeResponse - 92, // 294: productv1.ProductService.MutationCreateUser:output_type -> productv1.MutationCreateUserResponse - 94, // 295: productv1.ProductService.MutationPerformAction:output_type -> productv1.MutationPerformActionResponse - 106, // 296: productv1.ProductService.MutationUpdateAuthor:output_type -> productv1.MutationUpdateAuthorResponse - 102, // 297: productv1.ProductService.MutationUpdateBlogPost:output_type -> productv1.MutationUpdateBlogPostResponse - 98, // 298: productv1.ProductService.MutationUpdateNullableFieldsType:output_type -> productv1.MutationUpdateNullableFieldsTypeResponse - 86, // 299: productv1.ProductService.QueryAllAuthors:output_type -> productv1.QueryAllAuthorsResponse - 78, // 300: productv1.ProductService.QueryAllBlogPosts:output_type -> productv1.QueryAllBlogPostsResponse - 70, // 301: productv1.ProductService.QueryAllNullableFieldsTypes:output_type -> productv1.QueryAllNullableFieldsTypesResponse - 58, // 302: productv1.ProductService.QueryAllPets:output_type -> productv1.QueryAllPetsResponse - 80, // 303: productv1.ProductService.QueryAuthor:output_type -> productv1.QueryAuthorResponse - 82, // 304: productv1.ProductService.QueryAuthorById:output_type -> productv1.QueryAuthorByIdResponse - 84, // 305: productv1.ProductService.QueryAuthorsWithFilter:output_type -> productv1.QueryAuthorsWithFilterResponse - 72, // 306: productv1.ProductService.QueryBlogPost:output_type -> productv1.QueryBlogPostResponse - 74, // 307: productv1.ProductService.QueryBlogPostById:output_type -> productv1.QueryBlogPostByIdResponse - 76, // 308: productv1.ProductService.QueryBlogPostsWithFilter:output_type -> productv1.QueryBlogPostsWithFilterResponse - 88, // 309: productv1.ProductService.QueryBulkSearchAuthors:output_type -> productv1.QueryBulkSearchAuthorsResponse - 90, // 310: productv1.ProductService.QueryBulkSearchBlogPosts:output_type -> productv1.QueryBulkSearchBlogPostsResponse - 46, // 311: productv1.ProductService.QueryCalculateTotals:output_type -> productv1.QueryCalculateTotalsResponse - 48, // 312: productv1.ProductService.QueryCategories:output_type -> productv1.QueryCategoriesResponse - 50, // 313: productv1.ProductService.QueryCategoriesByKind:output_type -> productv1.QueryCategoriesByKindResponse - 52, // 314: productv1.ProductService.QueryCategoriesByKinds:output_type -> productv1.QueryCategoriesByKindsResponse - 44, // 315: productv1.ProductService.QueryComplexFilterType:output_type -> productv1.QueryComplexFilterTypeResponse - 54, // 316: productv1.ProductService.QueryFilterCategories:output_type -> productv1.QueryFilterCategoriesResponse - 36, // 317: productv1.ProductService.QueryNestedType:output_type -> productv1.QueryNestedTypeResponse - 64, // 318: productv1.ProductService.QueryNullableFieldsType:output_type -> productv1.QueryNullableFieldsTypeResponse - 66, // 319: productv1.ProductService.QueryNullableFieldsTypeById:output_type -> productv1.QueryNullableFieldsTypeByIdResponse - 68, // 320: productv1.ProductService.QueryNullableFieldsTypeWithFilter:output_type -> productv1.QueryNullableFieldsTypeWithFilterResponse - 56, // 321: productv1.ProductService.QueryRandomPet:output_type -> productv1.QueryRandomPetResponse - 62, // 322: productv1.ProductService.QueryRandomSearchResult:output_type -> productv1.QueryRandomSearchResultResponse - 38, // 323: productv1.ProductService.QueryRecursiveType:output_type -> productv1.QueryRecursiveTypeResponse - 60, // 324: productv1.ProductService.QuerySearch:output_type -> productv1.QuerySearchResponse - 40, // 325: productv1.ProductService.QueryTypeFilterWithArguments:output_type -> productv1.QueryTypeFilterWithArgumentsResponse - 42, // 326: productv1.ProductService.QueryTypeWithMultipleFilterFields:output_type -> productv1.QueryTypeWithMultipleFilterFieldsResponse - 34, // 327: productv1.ProductService.QueryUser:output_type -> productv1.QueryUserResponse - 32, // 328: productv1.ProductService.QueryUsers:output_type -> productv1.QueryUsersResponse - 139, // 329: productv1.ProductService.ResolveCategoryCategoryMetrics:output_type -> productv1.ResolveCategoryCategoryMetricsResponse - 149, // 330: productv1.ProductService.ResolveCategoryMetricsNormalizedScore:output_type -> productv1.ResolveCategoryMetricsNormalizedScoreResponse - 134, // 331: productv1.ProductService.ResolveCategoryPopularityScore:output_type -> productv1.ResolveCategoryPopularityScoreResponse - 129, // 332: productv1.ProductService.ResolveCategoryProductCount:output_type -> productv1.ResolveCategoryProductCountResponse - 124, // 333: productv1.ProductService.ResolveProductRecommendedCategory:output_type -> productv1.ResolveProductRecommendedCategoryResponse - 119, // 334: productv1.ProductService.ResolveProductShippingEstimate:output_type -> productv1.ResolveProductShippingEstimateResponse - 144, // 335: productv1.ProductService.ResolveSubcategoryItemCount:output_type -> productv1.ResolveSubcategoryItemCountResponse - 284, // [284:336] is the sub-list for method output_type - 232, // [232:284] is the sub-list for method input_type - 232, // [232:232] is the sub-list for extension type_name - 232, // [232:232] is the sub-list for extension extendee - 0, // [0:232] is the sub-list for field type_name + 0, // 113: productv1.ResolveCategoryMascotContext.kind:type_name -> productv1.CategoryKind + 141, // 114: productv1.ResolveCategoryMascotRequest.context:type_name -> productv1.ResolveCategoryMascotContext + 140, // 115: productv1.ResolveCategoryMascotRequest.field_args:type_name -> productv1.ResolveCategoryMascotArgs + 174, // 116: productv1.ResolveCategoryMascotResult.mascot:type_name -> productv1.Animal + 143, // 117: productv1.ResolveCategoryMascotResponse.result:type_name -> productv1.ResolveCategoryMascotResult + 146, // 118: productv1.ResolveCategoryCategoryStatusRequest.context:type_name -> productv1.ResolveCategoryCategoryStatusContext + 145, // 119: productv1.ResolveCategoryCategoryStatusRequest.field_args:type_name -> productv1.ResolveCategoryCategoryStatusArgs + 185, // 120: productv1.ResolveCategoryCategoryStatusResult.category_status:type_name -> productv1.ActionResult + 148, // 121: productv1.ResolveCategoryCategoryStatusResponse.result:type_name -> productv1.ResolveCategoryCategoryStatusResult + 203, // 122: productv1.ResolveSubcategoryItemCountArgs.filters:type_name -> productv1.SubcategoryItemFilter + 151, // 123: productv1.ResolveSubcategoryItemCountRequest.context:type_name -> productv1.ResolveSubcategoryItemCountContext + 150, // 124: productv1.ResolveSubcategoryItemCountRequest.field_args:type_name -> productv1.ResolveSubcategoryItemCountArgs + 153, // 125: productv1.ResolveSubcategoryItemCountResponse.result:type_name -> productv1.ResolveSubcategoryItemCountResult + 156, // 126: productv1.ResolveCategoryMetricsNormalizedScoreRequest.context:type_name -> productv1.ResolveCategoryMetricsNormalizedScoreContext + 155, // 127: productv1.ResolveCategoryMetricsNormalizedScoreRequest.field_args:type_name -> productv1.ResolveCategoryMetricsNormalizedScoreArgs + 158, // 128: productv1.ResolveCategoryMetricsNormalizedScoreResponse.result:type_name -> productv1.ResolveCategoryMetricsNormalizedScoreResult + 189, // 129: productv1.NestedTypeA.b:type_name -> productv1.NestedTypeB + 165, // 130: productv1.RecursiveType.recursive_type:type_name -> productv1.RecursiveType + 191, // 131: productv1.ComplexFilterTypeInput.filter:type_name -> productv1.FilterType + 193, // 132: productv1.OrderInput.lines:type_name -> productv1.OrderLineInput + 16, // 133: productv1.Order.order_lines:type_name -> productv1.ListOfOrderLine + 0, // 134: productv1.Category.kind:type_name -> productv1.CategoryKind + 19, // 135: productv1.Category.subcategories:type_name -> productv1.ListOfSubcategory + 0, // 136: productv1.CategoryFilter.category:type_name -> productv1.CategoryKind + 192, // 137: productv1.CategoryFilter.pagination:type_name -> productv1.Pagination + 197, // 138: productv1.Animal.cat:type_name -> productv1.Cat + 198, // 139: productv1.Animal.dog:type_name -> productv1.Dog + 225, // 140: productv1.SearchInput.limit:type_name -> google.protobuf.Int32Value + 160, // 141: productv1.SearchResult.product:type_name -> productv1.Product + 163, // 142: productv1.SearchResult.user:type_name -> productv1.User + 172, // 143: productv1.SearchResult.category:type_name -> productv1.Category + 226, // 144: productv1.NullableFieldsType.optional_string:type_name -> google.protobuf.StringValue + 225, // 145: productv1.NullableFieldsType.optional_int:type_name -> google.protobuf.Int32Value + 227, // 146: productv1.NullableFieldsType.optional_float:type_name -> google.protobuf.DoubleValue + 228, // 147: productv1.NullableFieldsType.optional_boolean:type_name -> google.protobuf.BoolValue + 226, // 148: productv1.NullableFieldsFilter.name:type_name -> google.protobuf.StringValue + 226, // 149: productv1.NullableFieldsFilter.optional_string:type_name -> google.protobuf.StringValue + 228, // 150: productv1.NullableFieldsFilter.include_nulls:type_name -> google.protobuf.BoolValue + 18, // 151: productv1.BlogPost.optional_tags:type_name -> productv1.ListOfString + 18, // 152: productv1.BlogPost.keywords:type_name -> productv1.ListOfString + 10, // 153: productv1.BlogPost.ratings:type_name -> productv1.ListOfFloat + 7, // 154: productv1.BlogPost.is_published:type_name -> productv1.ListOfBoolean + 13, // 155: productv1.BlogPost.tag_groups:type_name -> productv1.ListOfListOfString + 13, // 156: productv1.BlogPost.related_topics:type_name -> productv1.ListOfListOfString + 13, // 157: productv1.BlogPost.comment_threads:type_name -> productv1.ListOfListOfString + 13, // 158: productv1.BlogPost.suggestions:type_name -> productv1.ListOfListOfString + 172, // 159: productv1.BlogPost.related_categories:type_name -> productv1.Category + 163, // 160: productv1.BlogPost.contributors:type_name -> productv1.User + 17, // 161: productv1.BlogPost.mentioned_products:type_name -> productv1.ListOfProduct + 20, // 162: productv1.BlogPost.mentioned_users:type_name -> productv1.ListOfUser + 11, // 163: productv1.BlogPost.category_groups:type_name -> productv1.ListOfListOfCategory + 14, // 164: productv1.BlogPost.contributor_teams:type_name -> productv1.ListOfListOfUser + 226, // 165: productv1.BlogPostFilter.title:type_name -> google.protobuf.StringValue + 228, // 166: productv1.BlogPostFilter.has_categories:type_name -> google.protobuf.BoolValue + 225, // 167: productv1.BlogPostFilter.min_tags:type_name -> google.protobuf.Int32Value + 226, // 168: productv1.Author.email:type_name -> google.protobuf.StringValue + 18, // 169: productv1.Author.social_links:type_name -> productv1.ListOfString + 13, // 170: productv1.Author.teams_by_project:type_name -> productv1.ListOfListOfString + 13, // 171: productv1.Author.collaborations:type_name -> productv1.ListOfListOfString + 4, // 172: productv1.Author.written_posts:type_name -> productv1.ListOfBlogPost + 172, // 173: productv1.Author.favorite_categories:type_name -> productv1.Category + 20, // 174: productv1.Author.related_authors:type_name -> productv1.ListOfUser + 17, // 175: productv1.Author.product_reviews:type_name -> productv1.ListOfProduct + 14, // 176: productv1.Author.author_groups:type_name -> productv1.ListOfListOfUser + 11, // 177: productv1.Author.category_preferences:type_name -> productv1.ListOfListOfCategory + 14, // 178: productv1.Author.project_teams:type_name -> productv1.ListOfListOfUser + 226, // 179: productv1.AuthorFilter.name:type_name -> google.protobuf.StringValue + 228, // 180: productv1.AuthorFilter.has_teams:type_name -> google.protobuf.BoolValue + 225, // 181: productv1.AuthorFilter.skill_count:type_name -> google.protobuf.Int32Value + 199, // 182: productv1.ActionResult.action_success:type_name -> productv1.ActionSuccess + 200, // 183: productv1.ActionResult.action_error:type_name -> productv1.ActionError + 226, // 184: productv1.NullableFieldsInput.optional_string:type_name -> google.protobuf.StringValue + 225, // 185: productv1.NullableFieldsInput.optional_int:type_name -> google.protobuf.Int32Value + 227, // 186: productv1.NullableFieldsInput.optional_float:type_name -> google.protobuf.DoubleValue + 228, // 187: productv1.NullableFieldsInput.optional_boolean:type_name -> google.protobuf.BoolValue + 18, // 188: productv1.BlogPostInput.optional_tags:type_name -> productv1.ListOfString + 18, // 189: productv1.BlogPostInput.keywords:type_name -> productv1.ListOfString + 10, // 190: productv1.BlogPostInput.ratings:type_name -> productv1.ListOfFloat + 7, // 191: productv1.BlogPostInput.is_published:type_name -> productv1.ListOfBoolean + 13, // 192: productv1.BlogPostInput.tag_groups:type_name -> productv1.ListOfListOfString + 13, // 193: productv1.BlogPostInput.related_topics:type_name -> productv1.ListOfListOfString + 13, // 194: productv1.BlogPostInput.comment_threads:type_name -> productv1.ListOfListOfString + 13, // 195: productv1.BlogPostInput.suggestions:type_name -> productv1.ListOfListOfString + 9, // 196: productv1.BlogPostInput.related_categories:type_name -> productv1.ListOfCategoryInput + 21, // 197: productv1.BlogPostInput.contributors:type_name -> productv1.ListOfUserInput + 12, // 198: productv1.BlogPostInput.category_groups:type_name -> productv1.ListOfListOfCategoryInput + 226, // 199: productv1.AuthorInput.email:type_name -> google.protobuf.StringValue + 18, // 200: productv1.AuthorInput.social_links:type_name -> productv1.ListOfString + 13, // 201: productv1.AuthorInput.teams_by_project:type_name -> productv1.ListOfListOfString + 13, // 202: productv1.AuthorInput.collaborations:type_name -> productv1.ListOfListOfString + 201, // 203: productv1.AuthorInput.favorite_categories:type_name -> productv1.CategoryInput + 15, // 204: productv1.AuthorInput.author_groups:type_name -> productv1.ListOfListOfUserInput + 15, // 205: productv1.AuthorInput.project_teams:type_name -> productv1.ListOfListOfUserInput + 190, // 206: productv1.NestedTypeB.c:type_name -> productv1.NestedTypeC + 192, // 207: productv1.FilterType.pagination:type_name -> productv1.Pagination + 18, // 208: productv1.OrderLineInput.modifiers:type_name -> productv1.ListOfString + 18, // 209: productv1.OrderLine.modifiers:type_name -> productv1.ListOfString + 226, // 210: productv1.Subcategory.description:type_name -> google.protobuf.StringValue + 172, // 211: productv1.CategoryMetrics.related_category:type_name -> productv1.Category + 0, // 212: productv1.CategoryInput.kind:type_name -> productv1.CategoryKind + 227, // 213: productv1.ProductCountFilter.min_price:type_name -> google.protobuf.DoubleValue + 227, // 214: productv1.ProductCountFilter.max_price:type_name -> google.protobuf.DoubleValue + 228, // 215: productv1.ProductCountFilter.in_stock:type_name -> google.protobuf.BoolValue + 226, // 216: productv1.ProductCountFilter.search_term:type_name -> google.protobuf.StringValue + 227, // 217: productv1.SubcategoryItemFilter.min_price:type_name -> google.protobuf.DoubleValue + 227, // 218: productv1.SubcategoryItemFilter.max_price:type_name -> google.protobuf.DoubleValue + 228, // 219: productv1.SubcategoryItemFilter.in_stock:type_name -> google.protobuf.BoolValue + 228, // 220: productv1.SubcategoryItemFilter.is_active:type_name -> google.protobuf.BoolValue + 226, // 221: productv1.SubcategoryItemFilter.search_term:type_name -> google.protobuf.StringValue + 1, // 222: productv1.ShippingEstimateInput.destination:type_name -> productv1.ShippingDestination + 228, // 223: productv1.ShippingEstimateInput.expedited:type_name -> google.protobuf.BoolValue + 182, // 224: productv1.ListOfAuthorFilter.List.items:type_name -> productv1.AuthorFilter + 188, // 225: productv1.ListOfAuthorInput.List.items:type_name -> productv1.AuthorInput + 179, // 226: productv1.ListOfBlogPost.List.items:type_name -> productv1.BlogPost + 180, // 227: productv1.ListOfBlogPostFilter.List.items:type_name -> productv1.BlogPostFilter + 187, // 228: productv1.ListOfBlogPostInput.List.items:type_name -> productv1.BlogPostInput + 172, // 229: productv1.ListOfCategory.List.items:type_name -> productv1.Category + 201, // 230: productv1.ListOfCategoryInput.List.items:type_name -> productv1.CategoryInput + 8, // 231: productv1.ListOfListOfCategory.List.items:type_name -> productv1.ListOfCategory + 9, // 232: productv1.ListOfListOfCategoryInput.List.items:type_name -> productv1.ListOfCategoryInput + 18, // 233: productv1.ListOfListOfString.List.items:type_name -> productv1.ListOfString + 20, // 234: productv1.ListOfListOfUser.List.items:type_name -> productv1.ListOfUser + 21, // 235: productv1.ListOfListOfUserInput.List.items:type_name -> productv1.ListOfUserInput + 194, // 236: productv1.ListOfOrderLine.List.items:type_name -> productv1.OrderLine + 160, // 237: productv1.ListOfProduct.List.items:type_name -> productv1.Product + 195, // 238: productv1.ListOfSubcategory.List.items:type_name -> productv1.Subcategory + 163, // 239: productv1.ListOfUser.List.items:type_name -> productv1.User + 183, // 240: productv1.ListOfUserInput.List.items:type_name -> productv1.UserInput + 23, // 241: productv1.ProductService.LookupProductById:input_type -> productv1.LookupProductByIdRequest + 26, // 242: productv1.ProductService.LookupStorageById:input_type -> productv1.LookupStorageByIdRequest + 29, // 243: productv1.ProductService.LookupWarehouseById:input_type -> productv1.LookupWarehouseByIdRequest + 107, // 244: productv1.ProductService.MutationBulkCreateAuthors:input_type -> productv1.MutationBulkCreateAuthorsRequest + 111, // 245: productv1.ProductService.MutationBulkCreateBlogPosts:input_type -> productv1.MutationBulkCreateBlogPostsRequest + 109, // 246: productv1.ProductService.MutationBulkUpdateAuthors:input_type -> productv1.MutationBulkUpdateAuthorsRequest + 113, // 247: productv1.ProductService.MutationBulkUpdateBlogPosts:input_type -> productv1.MutationBulkUpdateBlogPostsRequest + 103, // 248: productv1.ProductService.MutationCreateAuthor:input_type -> productv1.MutationCreateAuthorRequest + 99, // 249: productv1.ProductService.MutationCreateBlogPost:input_type -> productv1.MutationCreateBlogPostRequest + 95, // 250: productv1.ProductService.MutationCreateNullableFieldsType:input_type -> productv1.MutationCreateNullableFieldsTypeRequest + 91, // 251: productv1.ProductService.MutationCreateUser:input_type -> productv1.MutationCreateUserRequest + 93, // 252: productv1.ProductService.MutationPerformAction:input_type -> productv1.MutationPerformActionRequest + 105, // 253: productv1.ProductService.MutationUpdateAuthor:input_type -> productv1.MutationUpdateAuthorRequest + 101, // 254: productv1.ProductService.MutationUpdateBlogPost:input_type -> productv1.MutationUpdateBlogPostRequest + 97, // 255: productv1.ProductService.MutationUpdateNullableFieldsType:input_type -> productv1.MutationUpdateNullableFieldsTypeRequest + 85, // 256: productv1.ProductService.QueryAllAuthors:input_type -> productv1.QueryAllAuthorsRequest + 77, // 257: productv1.ProductService.QueryAllBlogPosts:input_type -> productv1.QueryAllBlogPostsRequest + 69, // 258: productv1.ProductService.QueryAllNullableFieldsTypes:input_type -> productv1.QueryAllNullableFieldsTypesRequest + 57, // 259: productv1.ProductService.QueryAllPets:input_type -> productv1.QueryAllPetsRequest + 79, // 260: productv1.ProductService.QueryAuthor:input_type -> productv1.QueryAuthorRequest + 81, // 261: productv1.ProductService.QueryAuthorById:input_type -> productv1.QueryAuthorByIdRequest + 83, // 262: productv1.ProductService.QueryAuthorsWithFilter:input_type -> productv1.QueryAuthorsWithFilterRequest + 71, // 263: productv1.ProductService.QueryBlogPost:input_type -> productv1.QueryBlogPostRequest + 73, // 264: productv1.ProductService.QueryBlogPostById:input_type -> productv1.QueryBlogPostByIdRequest + 75, // 265: productv1.ProductService.QueryBlogPostsWithFilter:input_type -> productv1.QueryBlogPostsWithFilterRequest + 87, // 266: productv1.ProductService.QueryBulkSearchAuthors:input_type -> productv1.QueryBulkSearchAuthorsRequest + 89, // 267: productv1.ProductService.QueryBulkSearchBlogPosts:input_type -> productv1.QueryBulkSearchBlogPostsRequest + 45, // 268: productv1.ProductService.QueryCalculateTotals:input_type -> productv1.QueryCalculateTotalsRequest + 47, // 269: productv1.ProductService.QueryCategories:input_type -> productv1.QueryCategoriesRequest + 49, // 270: productv1.ProductService.QueryCategoriesByKind:input_type -> productv1.QueryCategoriesByKindRequest + 51, // 271: productv1.ProductService.QueryCategoriesByKinds:input_type -> productv1.QueryCategoriesByKindsRequest + 43, // 272: productv1.ProductService.QueryComplexFilterType:input_type -> productv1.QueryComplexFilterTypeRequest + 53, // 273: productv1.ProductService.QueryFilterCategories:input_type -> productv1.QueryFilterCategoriesRequest + 35, // 274: productv1.ProductService.QueryNestedType:input_type -> productv1.QueryNestedTypeRequest + 63, // 275: productv1.ProductService.QueryNullableFieldsType:input_type -> productv1.QueryNullableFieldsTypeRequest + 65, // 276: productv1.ProductService.QueryNullableFieldsTypeById:input_type -> productv1.QueryNullableFieldsTypeByIdRequest + 67, // 277: productv1.ProductService.QueryNullableFieldsTypeWithFilter:input_type -> productv1.QueryNullableFieldsTypeWithFilterRequest + 55, // 278: productv1.ProductService.QueryRandomPet:input_type -> productv1.QueryRandomPetRequest + 61, // 279: productv1.ProductService.QueryRandomSearchResult:input_type -> productv1.QueryRandomSearchResultRequest + 37, // 280: productv1.ProductService.QueryRecursiveType:input_type -> productv1.QueryRecursiveTypeRequest + 59, // 281: productv1.ProductService.QuerySearch:input_type -> productv1.QuerySearchRequest + 39, // 282: productv1.ProductService.QueryTypeFilterWithArguments:input_type -> productv1.QueryTypeFilterWithArgumentsRequest + 41, // 283: productv1.ProductService.QueryTypeWithMultipleFilterFields:input_type -> productv1.QueryTypeWithMultipleFilterFieldsRequest + 33, // 284: productv1.ProductService.QueryUser:input_type -> productv1.QueryUserRequest + 31, // 285: productv1.ProductService.QueryUsers:input_type -> productv1.QueryUsersRequest + 137, // 286: productv1.ProductService.ResolveCategoryCategoryMetrics:input_type -> productv1.ResolveCategoryCategoryMetricsRequest + 147, // 287: productv1.ProductService.ResolveCategoryCategoryStatus:input_type -> productv1.ResolveCategoryCategoryStatusRequest + 142, // 288: productv1.ProductService.ResolveCategoryMascot:input_type -> productv1.ResolveCategoryMascotRequest + 157, // 289: productv1.ProductService.ResolveCategoryMetricsNormalizedScore:input_type -> productv1.ResolveCategoryMetricsNormalizedScoreRequest + 132, // 290: productv1.ProductService.ResolveCategoryPopularityScore:input_type -> productv1.ResolveCategoryPopularityScoreRequest + 127, // 291: productv1.ProductService.ResolveCategoryProductCount:input_type -> productv1.ResolveCategoryProductCountRequest + 122, // 292: productv1.ProductService.ResolveProductRecommendedCategory:input_type -> productv1.ResolveProductRecommendedCategoryRequest + 117, // 293: productv1.ProductService.ResolveProductShippingEstimate:input_type -> productv1.ResolveProductShippingEstimateRequest + 152, // 294: productv1.ProductService.ResolveSubcategoryItemCount:input_type -> productv1.ResolveSubcategoryItemCountRequest + 24, // 295: productv1.ProductService.LookupProductById:output_type -> productv1.LookupProductByIdResponse + 27, // 296: productv1.ProductService.LookupStorageById:output_type -> productv1.LookupStorageByIdResponse + 30, // 297: productv1.ProductService.LookupWarehouseById:output_type -> productv1.LookupWarehouseByIdResponse + 108, // 298: productv1.ProductService.MutationBulkCreateAuthors:output_type -> productv1.MutationBulkCreateAuthorsResponse + 112, // 299: productv1.ProductService.MutationBulkCreateBlogPosts:output_type -> productv1.MutationBulkCreateBlogPostsResponse + 110, // 300: productv1.ProductService.MutationBulkUpdateAuthors:output_type -> productv1.MutationBulkUpdateAuthorsResponse + 114, // 301: productv1.ProductService.MutationBulkUpdateBlogPosts:output_type -> productv1.MutationBulkUpdateBlogPostsResponse + 104, // 302: productv1.ProductService.MutationCreateAuthor:output_type -> productv1.MutationCreateAuthorResponse + 100, // 303: productv1.ProductService.MutationCreateBlogPost:output_type -> productv1.MutationCreateBlogPostResponse + 96, // 304: productv1.ProductService.MutationCreateNullableFieldsType:output_type -> productv1.MutationCreateNullableFieldsTypeResponse + 92, // 305: productv1.ProductService.MutationCreateUser:output_type -> productv1.MutationCreateUserResponse + 94, // 306: productv1.ProductService.MutationPerformAction:output_type -> productv1.MutationPerformActionResponse + 106, // 307: productv1.ProductService.MutationUpdateAuthor:output_type -> productv1.MutationUpdateAuthorResponse + 102, // 308: productv1.ProductService.MutationUpdateBlogPost:output_type -> productv1.MutationUpdateBlogPostResponse + 98, // 309: productv1.ProductService.MutationUpdateNullableFieldsType:output_type -> productv1.MutationUpdateNullableFieldsTypeResponse + 86, // 310: productv1.ProductService.QueryAllAuthors:output_type -> productv1.QueryAllAuthorsResponse + 78, // 311: productv1.ProductService.QueryAllBlogPosts:output_type -> productv1.QueryAllBlogPostsResponse + 70, // 312: productv1.ProductService.QueryAllNullableFieldsTypes:output_type -> productv1.QueryAllNullableFieldsTypesResponse + 58, // 313: productv1.ProductService.QueryAllPets:output_type -> productv1.QueryAllPetsResponse + 80, // 314: productv1.ProductService.QueryAuthor:output_type -> productv1.QueryAuthorResponse + 82, // 315: productv1.ProductService.QueryAuthorById:output_type -> productv1.QueryAuthorByIdResponse + 84, // 316: productv1.ProductService.QueryAuthorsWithFilter:output_type -> productv1.QueryAuthorsWithFilterResponse + 72, // 317: productv1.ProductService.QueryBlogPost:output_type -> productv1.QueryBlogPostResponse + 74, // 318: productv1.ProductService.QueryBlogPostById:output_type -> productv1.QueryBlogPostByIdResponse + 76, // 319: productv1.ProductService.QueryBlogPostsWithFilter:output_type -> productv1.QueryBlogPostsWithFilterResponse + 88, // 320: productv1.ProductService.QueryBulkSearchAuthors:output_type -> productv1.QueryBulkSearchAuthorsResponse + 90, // 321: productv1.ProductService.QueryBulkSearchBlogPosts:output_type -> productv1.QueryBulkSearchBlogPostsResponse + 46, // 322: productv1.ProductService.QueryCalculateTotals:output_type -> productv1.QueryCalculateTotalsResponse + 48, // 323: productv1.ProductService.QueryCategories:output_type -> productv1.QueryCategoriesResponse + 50, // 324: productv1.ProductService.QueryCategoriesByKind:output_type -> productv1.QueryCategoriesByKindResponse + 52, // 325: productv1.ProductService.QueryCategoriesByKinds:output_type -> productv1.QueryCategoriesByKindsResponse + 44, // 326: productv1.ProductService.QueryComplexFilterType:output_type -> productv1.QueryComplexFilterTypeResponse + 54, // 327: productv1.ProductService.QueryFilterCategories:output_type -> productv1.QueryFilterCategoriesResponse + 36, // 328: productv1.ProductService.QueryNestedType:output_type -> productv1.QueryNestedTypeResponse + 64, // 329: productv1.ProductService.QueryNullableFieldsType:output_type -> productv1.QueryNullableFieldsTypeResponse + 66, // 330: productv1.ProductService.QueryNullableFieldsTypeById:output_type -> productv1.QueryNullableFieldsTypeByIdResponse + 68, // 331: productv1.ProductService.QueryNullableFieldsTypeWithFilter:output_type -> productv1.QueryNullableFieldsTypeWithFilterResponse + 56, // 332: productv1.ProductService.QueryRandomPet:output_type -> productv1.QueryRandomPetResponse + 62, // 333: productv1.ProductService.QueryRandomSearchResult:output_type -> productv1.QueryRandomSearchResultResponse + 38, // 334: productv1.ProductService.QueryRecursiveType:output_type -> productv1.QueryRecursiveTypeResponse + 60, // 335: productv1.ProductService.QuerySearch:output_type -> productv1.QuerySearchResponse + 40, // 336: productv1.ProductService.QueryTypeFilterWithArguments:output_type -> productv1.QueryTypeFilterWithArgumentsResponse + 42, // 337: productv1.ProductService.QueryTypeWithMultipleFilterFields:output_type -> productv1.QueryTypeWithMultipleFilterFieldsResponse + 34, // 338: productv1.ProductService.QueryUser:output_type -> productv1.QueryUserResponse + 32, // 339: productv1.ProductService.QueryUsers:output_type -> productv1.QueryUsersResponse + 139, // 340: productv1.ProductService.ResolveCategoryCategoryMetrics:output_type -> productv1.ResolveCategoryCategoryMetricsResponse + 149, // 341: productv1.ProductService.ResolveCategoryCategoryStatus:output_type -> productv1.ResolveCategoryCategoryStatusResponse + 144, // 342: productv1.ProductService.ResolveCategoryMascot:output_type -> productv1.ResolveCategoryMascotResponse + 159, // 343: productv1.ProductService.ResolveCategoryMetricsNormalizedScore:output_type -> productv1.ResolveCategoryMetricsNormalizedScoreResponse + 134, // 344: productv1.ProductService.ResolveCategoryPopularityScore:output_type -> productv1.ResolveCategoryPopularityScoreResponse + 129, // 345: productv1.ProductService.ResolveCategoryProductCount:output_type -> productv1.ResolveCategoryProductCountResponse + 124, // 346: productv1.ProductService.ResolveProductRecommendedCategory:output_type -> productv1.ResolveProductRecommendedCategoryResponse + 119, // 347: productv1.ProductService.ResolveProductShippingEstimate:output_type -> productv1.ResolveProductShippingEstimateResponse + 154, // 348: productv1.ProductService.ResolveSubcategoryItemCount:output_type -> productv1.ResolveSubcategoryItemCountResponse + 295, // [295:349] is the sub-list for method output_type + 241, // [241:295] is the sub-list for method input_type + 241, // [241:241] is the sub-list for extension type_name + 241, // [241:241] is the sub-list for extension extendee + 0, // [0:241] is the sub-list for field type_name } func init() { file_product_proto_init() } @@ -12251,16 +12778,16 @@ func file_product_proto_init() { if File_product_proto != nil { return } - file_product_proto_msgTypes[162].OneofWrappers = []any{ + file_product_proto_msgTypes[172].OneofWrappers = []any{ (*Animal_Cat)(nil), (*Animal_Dog)(nil), } - file_product_proto_msgTypes[164].OneofWrappers = []any{ + file_product_proto_msgTypes[174].OneofWrappers = []any{ (*SearchResult_Product)(nil), (*SearchResult_User)(nil), (*SearchResult_Category)(nil), } - file_product_proto_msgTypes[173].OneofWrappers = []any{ + file_product_proto_msgTypes[183].OneofWrappers = []any{ (*ActionResult_ActionSuccess)(nil), (*ActionResult_ActionError)(nil), } @@ -12270,7 +12797,7 @@ func file_product_proto_init() { GoPackagePath: reflect.TypeOf(x{}).PkgPath(), RawDescriptor: unsafe.Slice(unsafe.StringData(file_product_proto_rawDesc), len(file_product_proto_rawDesc)), NumEnums: 2, - NumMessages: 213, + NumMessages: 223, NumExtensions: 0, NumServices: 1, }, diff --git a/v2/pkg/grpctest/productv1/product_grpc.pb.go b/v2/pkg/grpctest/productv1/product_grpc.pb.go index 5fc55ddba..1d5a28249 100644 --- a/v2/pkg/grpctest/productv1/product_grpc.pb.go +++ b/v2/pkg/grpctest/productv1/product_grpc.pb.go @@ -65,6 +65,8 @@ const ( ProductService_QueryUser_FullMethodName = "/productv1.ProductService/QueryUser" ProductService_QueryUsers_FullMethodName = "/productv1.ProductService/QueryUsers" ProductService_ResolveCategoryCategoryMetrics_FullMethodName = "/productv1.ProductService/ResolveCategoryCategoryMetrics" + ProductService_ResolveCategoryCategoryStatus_FullMethodName = "/productv1.ProductService/ResolveCategoryCategoryStatus" + ProductService_ResolveCategoryMascot_FullMethodName = "/productv1.ProductService/ResolveCategoryMascot" ProductService_ResolveCategoryMetricsNormalizedScore_FullMethodName = "/productv1.ProductService/ResolveCategoryMetricsNormalizedScore" ProductService_ResolveCategoryPopularityScore_FullMethodName = "/productv1.ProductService/ResolveCategoryPopularityScore" ProductService_ResolveCategoryProductCount_FullMethodName = "/productv1.ProductService/ResolveCategoryProductCount" @@ -128,6 +130,8 @@ type ProductServiceClient interface { QueryUser(ctx context.Context, in *QueryUserRequest, opts ...grpc.CallOption) (*QueryUserResponse, error) QueryUsers(ctx context.Context, in *QueryUsersRequest, opts ...grpc.CallOption) (*QueryUsersResponse, error) ResolveCategoryCategoryMetrics(ctx context.Context, in *ResolveCategoryCategoryMetricsRequest, opts ...grpc.CallOption) (*ResolveCategoryCategoryMetricsResponse, error) + ResolveCategoryCategoryStatus(ctx context.Context, in *ResolveCategoryCategoryStatusRequest, opts ...grpc.CallOption) (*ResolveCategoryCategoryStatusResponse, error) + ResolveCategoryMascot(ctx context.Context, in *ResolveCategoryMascotRequest, opts ...grpc.CallOption) (*ResolveCategoryMascotResponse, error) ResolveCategoryMetricsNormalizedScore(ctx context.Context, in *ResolveCategoryMetricsNormalizedScoreRequest, opts ...grpc.CallOption) (*ResolveCategoryMetricsNormalizedScoreResponse, error) ResolveCategoryPopularityScore(ctx context.Context, in *ResolveCategoryPopularityScoreRequest, opts ...grpc.CallOption) (*ResolveCategoryPopularityScoreResponse, error) ResolveCategoryProductCount(ctx context.Context, in *ResolveCategoryProductCountRequest, opts ...grpc.CallOption) (*ResolveCategoryProductCountResponse, error) @@ -604,6 +608,26 @@ func (c *productServiceClient) ResolveCategoryCategoryMetrics(ctx context.Contex return out, nil } +func (c *productServiceClient) ResolveCategoryCategoryStatus(ctx context.Context, in *ResolveCategoryCategoryStatusRequest, opts ...grpc.CallOption) (*ResolveCategoryCategoryStatusResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(ResolveCategoryCategoryStatusResponse) + err := c.cc.Invoke(ctx, ProductService_ResolveCategoryCategoryStatus_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *productServiceClient) ResolveCategoryMascot(ctx context.Context, in *ResolveCategoryMascotRequest, opts ...grpc.CallOption) (*ResolveCategoryMascotResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(ResolveCategoryMascotResponse) + err := c.cc.Invoke(ctx, ProductService_ResolveCategoryMascot_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + func (c *productServiceClient) ResolveCategoryMetricsNormalizedScore(ctx context.Context, in *ResolveCategoryMetricsNormalizedScoreRequest, opts ...grpc.CallOption) (*ResolveCategoryMetricsNormalizedScoreResponse, error) { cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) out := new(ResolveCategoryMetricsNormalizedScoreResponse) @@ -719,6 +743,8 @@ type ProductServiceServer interface { QueryUser(context.Context, *QueryUserRequest) (*QueryUserResponse, error) QueryUsers(context.Context, *QueryUsersRequest) (*QueryUsersResponse, error) ResolveCategoryCategoryMetrics(context.Context, *ResolveCategoryCategoryMetricsRequest) (*ResolveCategoryCategoryMetricsResponse, error) + ResolveCategoryCategoryStatus(context.Context, *ResolveCategoryCategoryStatusRequest) (*ResolveCategoryCategoryStatusResponse, error) + ResolveCategoryMascot(context.Context, *ResolveCategoryMascotRequest) (*ResolveCategoryMascotResponse, error) ResolveCategoryMetricsNormalizedScore(context.Context, *ResolveCategoryMetricsNormalizedScoreRequest) (*ResolveCategoryMetricsNormalizedScoreResponse, error) ResolveCategoryPopularityScore(context.Context, *ResolveCategoryPopularityScoreRequest) (*ResolveCategoryPopularityScoreResponse, error) ResolveCategoryProductCount(context.Context, *ResolveCategoryProductCountRequest) (*ResolveCategoryProductCountResponse, error) @@ -873,6 +899,12 @@ func (UnimplementedProductServiceServer) QueryUsers(context.Context, *QueryUsers func (UnimplementedProductServiceServer) ResolveCategoryCategoryMetrics(context.Context, *ResolveCategoryCategoryMetricsRequest) (*ResolveCategoryCategoryMetricsResponse, error) { return nil, status.Errorf(codes.Unimplemented, "method ResolveCategoryCategoryMetrics not implemented") } +func (UnimplementedProductServiceServer) ResolveCategoryCategoryStatus(context.Context, *ResolveCategoryCategoryStatusRequest) (*ResolveCategoryCategoryStatusResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method ResolveCategoryCategoryStatus not implemented") +} +func (UnimplementedProductServiceServer) ResolveCategoryMascot(context.Context, *ResolveCategoryMascotRequest) (*ResolveCategoryMascotResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method ResolveCategoryMascot not implemented") +} func (UnimplementedProductServiceServer) ResolveCategoryMetricsNormalizedScore(context.Context, *ResolveCategoryMetricsNormalizedScoreRequest) (*ResolveCategoryMetricsNormalizedScoreResponse, error) { return nil, status.Errorf(codes.Unimplemented, "method ResolveCategoryMetricsNormalizedScore not implemented") } @@ -1740,6 +1772,42 @@ func _ProductService_ResolveCategoryCategoryMetrics_Handler(srv interface{}, ctx return interceptor(ctx, in, info, handler) } +func _ProductService_ResolveCategoryCategoryStatus_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(ResolveCategoryCategoryStatusRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(ProductServiceServer).ResolveCategoryCategoryStatus(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: ProductService_ResolveCategoryCategoryStatus_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(ProductServiceServer).ResolveCategoryCategoryStatus(ctx, req.(*ResolveCategoryCategoryStatusRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _ProductService_ResolveCategoryMascot_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(ResolveCategoryMascotRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(ProductServiceServer).ResolveCategoryMascot(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: ProductService_ResolveCategoryMascot_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(ProductServiceServer).ResolveCategoryMascot(ctx, req.(*ResolveCategoryMascotRequest)) + } + return interceptor(ctx, in, info, handler) +} + func _ProductService_ResolveCategoryMetricsNormalizedScore_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { in := new(ResolveCategoryMetricsNormalizedScoreRequest) if err := dec(in); err != nil { @@ -2039,6 +2107,14 @@ var ProductService_ServiceDesc = grpc.ServiceDesc{ MethodName: "ResolveCategoryCategoryMetrics", Handler: _ProductService_ResolveCategoryCategoryMetrics_Handler, }, + { + MethodName: "ResolveCategoryCategoryStatus", + Handler: _ProductService_ResolveCategoryCategoryStatus_Handler, + }, + { + MethodName: "ResolveCategoryMascot", + Handler: _ProductService_ResolveCategoryMascot_Handler, + }, { MethodName: "ResolveCategoryMetricsNormalizedScore", Handler: _ProductService_ResolveCategoryMetricsNormalizedScore_Handler, diff --git a/v2/pkg/grpctest/testdata/products.graphqls b/v2/pkg/grpctest/testdata/products.graphqls index 989bbd4de..b28ddbdf4 100644 --- a/v2/pkg/grpctest/testdata/products.graphqls +++ b/v2/pkg/grpctest/testdata/products.graphqls @@ -118,6 +118,8 @@ type Category { subcategories: [Subcategory!] popularityScore(threshold: Int): Int @connect__fieldResolver(context: "id") categoryMetrics(metricType: String!): CategoryMetrics @connect__fieldResolver(context: "id name") + mascot(includeVolume: Boolean!): Animal @connect__fieldResolver(context: "id kind") + categoryStatus(checkHealth: Boolean!): ActionResult! @connect__fieldResolver(context: "id name") } type Subcategory { From ca0e63d72e54bb778ac3d9ca70efe5ea09d9c35f Mon Sep 17 00:00:00 2001 From: Ludwig Bedacht Date: Thu, 30 Oct 2025 11:51:02 +0100 Subject: [PATCH 42/72] chore: rename field --- .../grpc_datasource/execution_plan_visitor.go | 12 ++++++------ .../execution_plan_visitor_federation.go | 12 ++++++------ 2 files changed, 12 insertions(+), 12 deletions(-) diff --git a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor.go b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor.go index 5b35371ee..6fbc030b9 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor.go +++ b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor.go @@ -54,7 +54,7 @@ type rpcPlanVisitor struct { currentCall *RPCCall currentCallID int - relatedCallID int + parentCallID int resolvedFieldIndex int resolvedFields []resolvedField @@ -78,7 +78,7 @@ func newRPCPlanVisitor(config rpcPlanVisitorConfig) *rpcPlanVisitor { mapping: config.mapping, operationFieldRef: ast.InvalidRef, resolvedFields: make([]resolvedField, 0), - relatedCallID: ast.InvalidRef, + parentCallID: ast.InvalidRef, resolvedFieldIndex: ast.InvalidRef, fieldPath: make(ast.Path, 0), } @@ -301,7 +301,7 @@ func (r *rpcPlanVisitor) handleRootField(isRootField bool, ref int) error { ServiceName: r.planCtx.resolveServiceName(r.subgraphName), } - r.relatedCallID = r.currentCallID + r.parentCallID = r.currentCallID r.planInfo.currentRequestMessage = &r.currentCall.Request r.planInfo.currentResponseMessage = &r.currentCall.Response @@ -353,7 +353,7 @@ func (r *rpcPlanVisitor) EnterField(ref int) { if fieldArgs := r.operation.FieldArguments(ref); !inRootField && len(fieldArgs) > 0 { // We don't want to add fields from the selection set to the actual call resolvedField := resolvedField{ - callerRef: r.relatedCallID, + callerRef: r.parentCallID, parentTypeRef: r.walker.EnclosingTypeDefinition.Ref, fieldRef: ref, responsePath: r.walker.Path[1:].WithoutInlineFragmentNames().WithFieldNameItem(r.operation.FieldAliasOrNameBytes(ref)), @@ -370,7 +370,7 @@ func (r *rpcPlanVisitor) EnterField(ref int) { r.fieldPath = r.fieldPath.WithFieldNameItem(r.operation.FieldNameBytes(ref)) // In case of nested fields with arguments, we need to increment the related call ID. - r.relatedCallID++ + r.parentCallID++ return } @@ -413,7 +413,7 @@ func (r *rpcPlanVisitor) LeaveField(ref int) { // This is because we can also have nested arguments, which require the underlying field to be resolved // by values provided by the parent call. if r.operation.FieldHasArguments(ref) { - r.relatedCallID-- + r.parentCallID-- } r.planInfo.currentResponseFieldIndex++ diff --git a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor_federation.go b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor_federation.go index c4f788dd4..ef0a2644f 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor_federation.go +++ b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor_federation.go @@ -50,7 +50,7 @@ type rpcPlanVisitorFederation struct { subgraphName string currentCall *RPCCall - relatedCallID int + parentCallID int resolvedFieldIndex int resolvedFields []resolvedField @@ -71,7 +71,7 @@ func newRPCPlanVisitorFederation(config rpcPlanVisitorConfig) *rpcPlanVisitorFed federationConfigData: parseFederationConfigData(config.federationConfigs), resolvedFields: make([]resolvedField, 0), resolvedFieldIndex: ast.InvalidRef, - relatedCallID: ast.InvalidRef, + parentCallID: ast.InvalidRef, fieldPath: ast.Path{}.WithFieldNameItem([]byte("result")), } @@ -142,7 +142,7 @@ func (r *rpcPlanVisitorFederation) EnterInlineFragment(ref int) { Kind: CallKindEntity, } - r.relatedCallID = len(r.plan.Calls) + r.parentCallID = len(r.plan.Calls) r.planInfo.currentRequestMessage = &r.currentCall.Request r.planInfo.currentResponseMessage = &r.currentCall.Response @@ -320,7 +320,7 @@ func (r *rpcPlanVisitorFederation) EnterField(ref int) { if fieldArgs := r.operation.FieldArguments(ref); !inRootField && len(fieldArgs) > 0 { // We don't want to add fields from the selection set to the actual call resolvedField := resolvedField{ - callerRef: r.relatedCallID, + callerRef: r.parentCallID, parentTypeRef: r.walker.EnclosingTypeDefinition.Ref, fieldRef: ref, responsePath: r.walker.Path[1:].WithoutInlineFragmentNames().WithFieldNameItem(r.operation.FieldAliasOrNameBytes(ref)), @@ -337,7 +337,7 @@ func (r *rpcPlanVisitorFederation) EnterField(ref int) { r.fieldPath = r.fieldPath.WithFieldNameItem(r.operation.FieldNameBytes(ref)) // In case of nested fields with arguments, we need to increment the related call ID. - r.relatedCallID++ + r.parentCallID++ return } @@ -379,7 +379,7 @@ func (r *rpcPlanVisitorFederation) LeaveField(ref int) { // This is because we can also have nested arguments, which require the underlying field to be resolved // by values provided by the parent call. if r.operation.FieldHasArguments(ref) { - r.relatedCallID-- + r.parentCallID-- } r.planInfo.currentResponseFieldIndex++ From 3eed05062ebc96660cedf5cd5bcdaa594f94a50f Mon Sep 17 00:00:00 2001 From: Ludwig Bedacht Date: Thu, 30 Oct 2025 11:53:00 +0100 Subject: [PATCH 43/72] chore: move length check --- v2/pkg/engine/datasource/grpc_datasource/compiler.go | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/v2/pkg/engine/datasource/grpc_datasource/compiler.go b/v2/pkg/engine/datasource/grpc_datasource/compiler.go index 20ac0af8c..a22e6a6b9 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/compiler.go +++ b/v2/pkg/engine/datasource/grpc_datasource/compiler.go @@ -507,10 +507,6 @@ func (p *RPCCompiler) CompileNode(graph *DependencyGraph, fetch FetchItem, input return ServiceCall{}, err } - if len(context) == 0 { - return ServiceCall{}, fmt.Errorf("context is required for resolve calls") - } - request, err = p.buildProtoMessageWithContext(inputMessage, &call.Request, inputData, context) if err != nil { return ServiceCall{}, err @@ -572,6 +568,10 @@ func (p *RPCCompiler) buildProtoMessageWithContext(inputMessage Message, rpcMess return nil, fmt.Errorf("rpc message is nil") } + if len(context) == 0 { + return nil, fmt.Errorf("context is required for resolve calls") + } + if p.doc.MessageRefByName(rpcMessage.Name) == InvalidRef { return nil, fmt.Errorf("message %s not found in document", rpcMessage.Name) } From f6318c7cebf3c0b43ae661772636a0a52ceab887 Mon Sep 17 00:00:00 2001 From: Ludwig Bedacht Date: Fri, 31 Oct 2025 10:03:03 +0100 Subject: [PATCH 44/72] chore: rename type --- .../grpc_datasource/execution_plan_visitor.go | 4 +-- .../execution_plan_visitor_federation.go | 4 +-- .../engine/datasource/grpc_datasource/util.go | 27 ++++++++++++++----- .../datasource/grpc_datasource/util_test.go | 3 ++- 4 files changed, 26 insertions(+), 12 deletions(-) diff --git a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor.go b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor.go index 86abe9bab..d50f1c548 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor.go +++ b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor.go @@ -55,7 +55,7 @@ type rpcPlanVisitor struct { currentCallID int parentCallID int - fieldResolverAncestors ancestor[int] + fieldResolverAncestors stack[int] resolvedFields []resolvedField fieldPath ast.Path @@ -79,7 +79,7 @@ func newRPCPlanVisitor(config rpcPlanVisitorConfig) *rpcPlanVisitor { operationFieldRef: ast.InvalidRef, resolvedFields: make([]resolvedField, 0), parentCallID: ast.InvalidRef, - fieldResolverAncestors: newAncestor[int](), + fieldResolverAncestors: newStack[int](0), fieldPath: make(ast.Path, 0), } diff --git a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor_federation.go b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor_federation.go index f0bfe3fbc..01e74e043 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor_federation.go +++ b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor_federation.go @@ -51,7 +51,7 @@ type rpcPlanVisitorFederation struct { currentCall *RPCCall parentCallID int - fieldResolverAncestors ancestor[int] + fieldResolverAncestors stack[int] resolvedFields []resolvedField fieldPath ast.Path @@ -70,7 +70,7 @@ func newRPCPlanVisitorFederation(config rpcPlanVisitorConfig) *rpcPlanVisitorFed }, federationConfigData: parseFederationConfigData(config.federationConfigs), resolvedFields: make([]resolvedField, 0), - fieldResolverAncestors: newAncestor[int](), + fieldResolverAncestors: newStack[int](0), parentCallID: ast.InvalidRef, fieldPath: ast.Path{}.WithFieldNameItem([]byte("result")), } diff --git a/v2/pkg/engine/datasource/grpc_datasource/util.go b/v2/pkg/engine/datasource/grpc_datasource/util.go index 6d96f2a1a..48dc37d44 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/util.go +++ b/v2/pkg/engine/datasource/grpc_datasource/util.go @@ -9,17 +9,22 @@ func initializeSlice[T any](len int, zero T) []T { return s } -type ancestor[T any] []T +// stack is a generic LIFO (Last In First Out) data structure that stores elements of type T. +type stack[T any] []T -func newAncestor[T any]() ancestor[T] { - return make(ancestor[T], 0) +// newStack creates and returns a new empty stack for elements of type T. +func newStack[T any](size int) stack[T] { + return make(stack[T], 0, size) } -func (a *ancestor[T]) push(value T) { +// push adds a new element to the top of the stack. +func (a *stack[T]) push(value T) { *a = append(*a, value) } -func (a *ancestor[T]) pop() { +// pop removes the top element from the stack. +// If the stack is empty, this operation is a no-op. +func (a *stack[T]) pop() { if a.len() == 0 { return } @@ -27,10 +32,18 @@ func (a *ancestor[T]) pop() { *a = (*a)[:len(*a)-1] } -func (a *ancestor[T]) peek() T { +// peek returns the top element of the stack without removing it. +// Note: This function will panic if called on an empty stack. +func (a *stack[T]) peek() T { return (*a)[len(*a)-1] } -func (a *ancestor[T]) len() int { +// len returns the number of elements currently in the stack. +func (a *stack[T]) len() int { return len(*a) } + +// capacity returns the capacity of the stack. +func (a *stack[T]) capacity() int { + return cap(*a) +} diff --git a/v2/pkg/engine/datasource/grpc_datasource/util_test.go b/v2/pkg/engine/datasource/grpc_datasource/util_test.go index aaf0e1cc0..44762e2be 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/util_test.go +++ b/v2/pkg/engine/datasource/grpc_datasource/util_test.go @@ -7,10 +7,11 @@ import ( ) func TestAncestor(t *testing.T) { - ancestor := newAncestor[int]() + ancestor := newStack[int](10) ancestor.push(1) ancestor.push(2) ancestor.push(3) + require.Equal(t, 10, ancestor.capacity()) require.Equal(t, 3, ancestor.len()) ancestor.pop() require.Equal(t, 2, ancestor.len()) From 22b2b0140f70943c8c9aabeefdbd385c56e65c99 Mon Sep 17 00:00:00 2001 From: Ludwig Bedacht Date: Fri, 31 Oct 2025 15:24:54 +0100 Subject: [PATCH 45/72] feat: implement handling for nested composite types --- .../grpc_datasource/execution_plan.go | 150 +- .../execution_plan_field_resolvers_test.go | 652 +++++ .../grpc_datasource/grpc_datasource_test.go | 247 ++ .../grpc_datasource/mapping_test_helper.go | 63 + .../required_fields_visitor.go | 2 +- v2/pkg/grpctest/mapping/mapping.go | 63 + v2/pkg/grpctest/mockservice.go | 111 + v2/pkg/grpctest/product.proto | 55 + v2/pkg/grpctest/productv1/product.pb.go | 2460 ++++++++++------- v2/pkg/grpctest/productv1/product_grpc.pb.go | 114 + v2/pkg/grpctest/testdata/products.graphqls | 19 + 11 files changed, 2978 insertions(+), 958 deletions(-) diff --git a/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go b/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go index 2aac75f80..d46ef6909 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go +++ b/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go @@ -9,6 +9,7 @@ import ( "github.com/wundergraph/graphql-go-tools/v2/pkg/ast" "github.com/wundergraph/graphql-go-tools/v2/pkg/astvisitor" "github.com/wundergraph/graphql-go-tools/v2/pkg/engine/plan" + "github.com/wundergraph/graphql-go-tools/v2/pkg/internal/unsafebytes" ) const ( @@ -148,12 +149,8 @@ func (r *RPCMessage) AppendTypeNameField(typeName string) { type RPCFieldSelectionSet map[string]RPCFields // Add adds a field selection set to the map -func (r RPCFieldSelectionSet) Add(fragmentName string, field RPCField) { - if r[fragmentName] == nil { - r[fragmentName] = make(RPCFields, 0) - } - - r[fragmentName] = append(r[fragmentName], field) +func (r RPCFieldSelectionSet) Add(fragmentName string, field ...RPCField) { + r[fragmentName] = append(r[fragmentName], field...) } // SelectFieldsForTypes returns the fields for the given valid types. @@ -719,6 +716,81 @@ func (r *rpcPlanningContext) buildInputMessageField(typeRef int, fieldName, json return field, nil } +func (r *rpcPlanningContext) buildFieldMessage(fieldTypeNode ast.Node, fieldRef int) (*RPCMessage, error) { + field := r.operation.Fields[fieldRef] + if !field.HasSelections { + return nil, nil + } + + fieldRefs := make([]int, 0) + inlineFragmentRefs := make([]int, 0) + selections := r.operation.SelectionSets[field.SelectionSet].SelectionRefs + for i := range selections { + selection := r.operation.Selections[selections[i]] + switch selection.Kind { + case ast.SelectionKindField: + fieldRefs = append(fieldRefs, selection.Ref) + case ast.SelectionKindInlineFragment: + inlineFragmentRefs = append(inlineFragmentRefs, selection.Ref) + } + } + + fmt.Println("fieldName", r.operation.FieldNameString(fieldRef)) + + fmt.Println("fieldRefs", fieldRefs, "inlineFragmentRefs", inlineFragmentRefs) + + message := &RPCMessage{ + Name: fieldTypeNode.NameString(r.definition), + } + + if compositeType := r.getCompositeType(fieldTypeNode); compositeType != OneOfTypeNone { + memberTypes, err := r.getMemberTypes(fieldTypeNode) + if err != nil { + return nil, err + } + message.MemberTypes = memberTypes + message.OneOfType = compositeType + } + + for _, inlineFragmentRef := range inlineFragmentRefs { + selectionSetRef, ok := r.operation.InlineFragmentSelectionSet(inlineFragmentRef) + if !ok { + continue + } + + typeName := r.operation.InlineFragmentTypeConditionNameString(inlineFragmentRef) + inlineFragmentTypeNode, found := r.definition.NodeByNameStr(typeName) + if !found { + return nil, fmt.Errorf("unable to resolve type node for inline fragment %s", typeName) + } + + fields, err := r.buildCompositeField(inlineFragmentTypeNode, fragmentSelection{ + typeName: typeName, + selectionSetRef: selectionSetRef, + }) + + if err != nil { + return nil, err + } + + if message.FieldSelectionSet == nil { + message.FieldSelectionSet = make(RPCFieldSelectionSet) + } + + message.FieldSelectionSet.Add(typeName, fields...) + } + + for _, fieldRef := range fieldRefs { + field, err := r.buildRequiredField(fieldTypeNode, fieldRef) + if err != nil { + return nil, err + } + message.Fields = append(message.Fields, field) + } + + return message, nil +} + // resolveServiceName resolves the service name for a given subgraph name. func (r *rpcPlanningContext) resolveServiceName(subgraphName string) string { if r.mapping == nil || r.mapping.Service == "" { @@ -788,17 +860,6 @@ func (r *rpcPlanningContext) getMemberTypes(node ast.Node) ([]string, error) { } } -func (r *rpcPlanningContext) getFragmentSelection(node ast.Node, selectionSetRef int) (*fragmentSelection, error) { - if node.Ref == ast.InvalidRef { - return nil, nil - } - - return &fragmentSelection{ - typeName: node.NameString(r.definition), - selectionSetRef: selectionSetRef, - }, nil -} - // setResolvedField sets the resolved field for a given field definition reference. func (r *rpcPlanningContext) setResolvedField(walker *astvisitor.Walker, fieldDefRef int, fieldArgs []int, fieldPath ast.Path, resolvedField *resolvedField) error { // We need to resolve the context fields for the given field definition reference. @@ -987,7 +1048,7 @@ func (r *rpcPlanningContext) resolveRequiredFields(typeName string, resolvedFiel return nil, errors.New("unable to resolve required fields: no fields selection set found") } - fieldRefs := r.operation.SelectionSetFieldSelections(resolvedField.fieldsSelectionSetRef) + fieldRefs := r.operation.SelectionSetFieldRefs(resolvedField.fieldsSelectionSetRef) message.Fields = make(RPCFields, 0, len(fieldRefs)) for _, fieldRef := range fieldRefs { @@ -999,12 +1060,7 @@ func (r *rpcPlanningContext) resolveRequiredFields(typeName string, resolvedFiel continue } - fieldDef, found := r.definition.NodeFieldDefinitionByName(parentTypeNode, r.operation.FieldNameBytes(fieldRef)) - if !found { - return nil, fmt.Errorf("field definition not found for field %s", r.operation.FieldNameString(fieldRef)) - } - - field, err := r.buildField(parentTypeNode, fieldDef, r.operation.FieldNameString(fieldRef), "") + field, err := r.buildRequiredField(parentTypeNode, fieldRef) if err != nil { return nil, err } @@ -1016,12 +1072,47 @@ func (r *rpcPlanningContext) resolveRequiredFields(typeName string, resolvedFiel return message, nil } +func (r *rpcPlanningContext) buildRequiredField(typeNode ast.Node, fieldRef int) (RPCField, error) { + fieldName := r.operation.FieldNameString(fieldRef) + fieldDef, found := r.definition.NodeFieldDefinitionByName(typeNode, r.operation.FieldNameBytes(fieldRef)) + if !found { + return RPCField{}, fmt.Errorf("unable to build required field: field definition not found for field %s", fieldName) + } + + field, err := r.buildField(typeNode, fieldDef, r.operation.FieldNameString(fieldRef), "") + if err != nil { + return RPCField{}, err + } + + // If the field is a message type and has selections, we need to build a nested message. + if field.ProtoTypeName == DataTypeMessage && r.operation.FieldHasSelections(fieldRef) { + fieldTypeNode, found := r.definition.ResolveNodeFromTypeRef(r.definition.FieldDefinitionType(fieldDef)) + if !found { + return RPCField{}, fmt.Errorf("unable to build required field: unable to resolve field type node for field %s", fieldName) + } + + message, err := r.buildFieldMessage(fieldTypeNode, fieldRef) + if err != nil { + return RPCField{}, err + } + + field.Message = message + } + + return field, nil +} + func (r *rpcPlanningContext) buildCompositeField(parentNode ast.Node, fragmentSelection fragmentSelection) ([]RPCField, error) { fieldRefs := r.operation.SelectionSetFieldRefs(fragmentSelection.selectionSetRef) result := make([]RPCField, 0, len(fieldRefs)) for _, fieldRef := range fieldRefs { - field, err := r.buildField(parentNode, fieldRef, r.operation.FieldNameString(fieldRef), "") + fieldDef, found := r.fieldDefinitionForType(r.operation.FieldNameString(fieldRef), fragmentSelection.typeName) + if !found { + return nil, fmt.Errorf("unable to build composite field: field definition not found for field %s", r.operation.FieldNameString(fieldRef)) + } + + field, err := r.buildField(parentNode, fieldDef, r.operation.FieldNameString(fieldRef), "") if err != nil { return nil, err } @@ -1030,6 +1121,15 @@ func (r *rpcPlanningContext) buildCompositeField(parentNode ast.Node, fragmentSe return result, nil } +func (r *rpcPlanningContext) fieldDefinitionForType(fieldName, typeName string) (ref int, exists bool) { + node, found := r.definition.NodeByNameStr(typeName) + if !found { + return ast.InvalidRef, false + } + + return r.definition.NodeFieldDefinitionByName(node, unsafebytes.StringToBytes(fieldName)) +} + // createResolverRPCCalls creates a new call for each resolved field. func (r *rpcPlanningContext) createResolverRPCCalls(subgraphName string, resolvedFields []resolvedField) ([]RPCCall, error) { // We need to create a new call for each resolved field. diff --git a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_field_resolvers_test.go b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_field_resolvers_test.go index c4c16a3cc..d98060295 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_field_resolvers_test.go +++ b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_field_resolvers_test.go @@ -1240,6 +1240,658 @@ func TestExecutionPlanFieldResolvers_WithOneOfTypes(t *testing.T) { }, }, }, + { + name: "Should create an execution plan for a query with union type", + query: "query CategoriesWithUnionResolver($checkHealth: Boolean!) { categories { categoryStatus(checkHealth: $checkHealth) { ... on ActionSuccess { message timestamp } ... on ActionError { message code } } } }", + expectedPlan: &RPCExecutionPlan{ + Calls: []RPCCall{ + { + ServiceName: "Products", + MethodName: "QueryCategories", + Request: RPCMessage{ + Name: "QueryCategoriesRequest", + }, + Response: RPCMessage{ + Name: "QueryCategoriesResponse", + Fields: []RPCField{ + { + Name: "categories", + ProtoTypeName: DataTypeMessage, + JSONPath: "categories", + Repeated: true, + Message: &RPCMessage{ + Name: "Category", + Fields: []RPCField{}, + }, + }, + }, + }, + }, + { + DependentCalls: []int{0}, + ServiceName: "Products", + MethodName: "ResolveCategoryCategoryStatus", + Kind: CallKindResolve, + ResponsePath: buildPath("categories.categoryStatus"), + Request: RPCMessage{ + Name: "ResolveCategoryCategoryStatusRequest", + Fields: []RPCField{ + { + Name: "context", + ProtoTypeName: DataTypeMessage, + Repeated: true, + Message: &RPCMessage{ + Name: "ResolveCategoryCategoryStatusContext", + Fields: []RPCField{ + { + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", + ResolvePath: buildPath("categories.id"), + }, + { + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", + ResolvePath: buildPath("categories.name"), + }, + }, + }, + }, + { + Name: "field_args", + ProtoTypeName: DataTypeMessage, + Message: &RPCMessage{ + Name: "ResolveCategoryCategoryStatusArgs", + Fields: []RPCField{ + { + Name: "check_health", + ProtoTypeName: DataTypeBool, + JSONPath: "checkHealth", + }, + }, + }, + }, + }, + }, + Response: RPCMessage{ + Name: "ResolveCategoryCategoryStatusResponse", + Fields: []RPCField{ + { + Name: "result", + ProtoTypeName: DataTypeMessage, + JSONPath: "result", + Repeated: true, + Message: &RPCMessage{ + Name: "ResolveCategoryCategoryStatusResult", + Fields: []RPCField{ + { + Name: "category_status", + ProtoTypeName: DataTypeMessage, + JSONPath: "categoryStatus", + Message: &RPCMessage{ + Name: "ActionResult", + OneOfType: OneOfTypeUnion, + MemberTypes: []string{"ActionSuccess", "ActionError"}, + FieldSelectionSet: RPCFieldSelectionSet{ + "ActionSuccess": { + { + Name: "message", + ProtoTypeName: DataTypeString, + JSONPath: "message", + }, + { + Name: "timestamp", + ProtoTypeName: DataTypeString, + JSONPath: "timestamp", + }, + }, + "ActionError": { + { + Name: "message", + ProtoTypeName: DataTypeString, + JSONPath: "message", + }, + { + Name: "code", + ProtoTypeName: DataTypeString, + JSONPath: "code", + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + { + name: "Should create an execution plan for a query with nested interface type", + query: "query TestContainersWithInterface($includeExtended: Boolean!) { testContainers { id name details(includeExtended: $includeExtended) { id summary pet { ... on Cat { name meowVolume } ... on Dog { name barkVolume } } } } }", + expectedPlan: &RPCExecutionPlan{ + Calls: []RPCCall{ + { + ServiceName: "Products", + MethodName: "QueryTestContainers", + Request: RPCMessage{ + Name: "QueryTestContainersRequest", + }, + Response: RPCMessage{ + Name: "QueryTestContainersResponse", + Fields: []RPCField{ + { + Name: "test_containers", + ProtoTypeName: DataTypeMessage, + JSONPath: "testContainers", + Repeated: true, + Message: &RPCMessage{ + Name: "TestContainer", + Fields: []RPCField{ + { + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", + }, + { + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", + }, + }, + }, + }, + }, + }, + }, + { + DependentCalls: []int{0}, + ServiceName: "Products", + MethodName: "ResolveTestContainerDetails", + Kind: CallKindResolve, + ResponsePath: buildPath("testContainers.details"), + Request: RPCMessage{ + Name: "ResolveTestContainerDetailsRequest", + Fields: []RPCField{ + { + Name: "context", + ProtoTypeName: DataTypeMessage, + Repeated: true, + Message: &RPCMessage{ + Name: "ResolveTestContainerDetailsContext", + Fields: []RPCField{ + { + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", + ResolvePath: buildPath("test_containers.id"), + }, + { + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", + ResolvePath: buildPath("test_containers.name"), + }, + }, + }, + }, + { + Name: "field_args", + ProtoTypeName: DataTypeMessage, + Message: &RPCMessage{ + Name: "ResolveTestContainerDetailsArgs", + Fields: []RPCField{ + { + Name: "include_extended", + ProtoTypeName: DataTypeBool, + JSONPath: "includeExtended", + }, + }, + }, + }, + }, + }, + Response: RPCMessage{ + Name: "ResolveTestContainerDetailsResponse", + Fields: []RPCField{ + { + Name: "result", + ProtoTypeName: DataTypeMessage, + JSONPath: "result", + Repeated: true, + Message: &RPCMessage{ + Name: "ResolveTestContainerDetailsResult", + Fields: []RPCField{ + { + Name: "details", + ProtoTypeName: DataTypeMessage, + JSONPath: "details", + Optional: true, + Message: &RPCMessage{ + Name: "TestDetails", + Fields: []RPCField{ + { + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", + }, + { + Name: "summary", + ProtoTypeName: DataTypeString, + JSONPath: "summary", + }, + { + Name: "pet", + ProtoTypeName: DataTypeMessage, + JSONPath: "pet", + Message: &RPCMessage{ + Name: "Animal", + OneOfType: OneOfTypeInterface, + MemberTypes: []string{"Cat", "Dog"}, + FieldSelectionSet: RPCFieldSelectionSet{ + "Cat": { + { + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", + }, + { + Name: "meow_volume", + ProtoTypeName: DataTypeInt32, + JSONPath: "meowVolume", + }, + }, + "Dog": { + { + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", + }, + { + Name: "bark_volume", + ProtoTypeName: DataTypeInt32, + JSONPath: "barkVolume", + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + { + name: "Should create an execution plan for a query with nested union type", + query: "query TestContainersWithUnion($includeExtended: Boolean!) { testContainers { id name details(includeExtended: $includeExtended) { id summary status { ... on ActionSuccess { message timestamp } ... on ActionError { message code } } } } }", + expectedPlan: &RPCExecutionPlan{ + Calls: []RPCCall{ + { + ServiceName: "Products", + MethodName: "QueryTestContainers", + Request: RPCMessage{ + Name: "QueryTestContainersRequest", + }, + Response: RPCMessage{ + Name: "QueryTestContainersResponse", + Fields: []RPCField{ + { + Name: "test_containers", + ProtoTypeName: DataTypeMessage, + JSONPath: "testContainers", + Repeated: true, + Message: &RPCMessage{ + Name: "TestContainer", + Fields: []RPCField{ + { + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", + }, + { + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", + }, + }, + }, + }, + }, + }, + }, + { + DependentCalls: []int{0}, + ServiceName: "Products", + MethodName: "ResolveTestContainerDetails", + Kind: CallKindResolve, + ResponsePath: buildPath("testContainers.details"), + Request: RPCMessage{ + Name: "ResolveTestContainerDetailsRequest", + Fields: []RPCField{ + { + Name: "context", + ProtoTypeName: DataTypeMessage, + Repeated: true, + Message: &RPCMessage{ + Name: "ResolveTestContainerDetailsContext", + Fields: []RPCField{ + { + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", + ResolvePath: buildPath("test_containers.id"), + }, + { + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", + ResolvePath: buildPath("test_containers.name"), + }, + }, + }, + }, + { + Name: "field_args", + ProtoTypeName: DataTypeMessage, + Message: &RPCMessage{ + Name: "ResolveTestContainerDetailsArgs", + Fields: []RPCField{ + { + Name: "include_extended", + ProtoTypeName: DataTypeBool, + JSONPath: "includeExtended", + }, + }, + }, + }, + }, + }, + Response: RPCMessage{ + Name: "ResolveTestContainerDetailsResponse", + Fields: []RPCField{ + { + Name: "result", + ProtoTypeName: DataTypeMessage, + JSONPath: "result", + Repeated: true, + Message: &RPCMessage{ + Name: "ResolveTestContainerDetailsResult", + Fields: []RPCField{ + { + Name: "details", + ProtoTypeName: DataTypeMessage, + JSONPath: "details", + Optional: true, + Message: &RPCMessage{ + Name: "TestDetails", + Fields: []RPCField{ + { + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", + }, + { + Name: "summary", + ProtoTypeName: DataTypeString, + JSONPath: "summary", + }, + { + Name: "status", + ProtoTypeName: DataTypeMessage, + JSONPath: "status", + Message: &RPCMessage{ + Name: "ActionResult", + OneOfType: OneOfTypeUnion, + MemberTypes: []string{"ActionSuccess", "ActionError"}, + FieldSelectionSet: RPCFieldSelectionSet{ + "ActionSuccess": { + { + Name: "message", + ProtoTypeName: DataTypeString, + JSONPath: "message", + }, + { + Name: "timestamp", + ProtoTypeName: DataTypeString, + JSONPath: "timestamp", + }, + }, + "ActionError": { + { + Name: "message", + ProtoTypeName: DataTypeString, + JSONPath: "message", + }, + { + Name: "code", + ProtoTypeName: DataTypeString, + JSONPath: "code", + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + { + name: "Should create an execution plan for a query with both nested interface and union types", + query: "query TestContainersWithBoth($includeExtended: Boolean!) { testContainers { id name details(includeExtended: $includeExtended) { id summary pet { ... on Cat { name meowVolume } ... on Dog { name barkVolume } } status { ... on ActionSuccess { message timestamp } ... on ActionError { message code } } } } }", + expectedPlan: &RPCExecutionPlan{ + Calls: []RPCCall{ + { + ServiceName: "Products", + MethodName: "QueryTestContainers", + Request: RPCMessage{ + Name: "QueryTestContainersRequest", + }, + Response: RPCMessage{ + Name: "QueryTestContainersResponse", + Fields: []RPCField{ + { + Name: "test_containers", + ProtoTypeName: DataTypeMessage, + JSONPath: "testContainers", + Repeated: true, + Message: &RPCMessage{ + Name: "TestContainer", + Fields: []RPCField{ + { + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", + }, + { + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", + }, + }, + }, + }, + }, + }, + }, + { + DependentCalls: []int{0}, + ServiceName: "Products", + MethodName: "ResolveTestContainerDetails", + Kind: CallKindResolve, + ResponsePath: buildPath("testContainers.details"), + Request: RPCMessage{ + Name: "ResolveTestContainerDetailsRequest", + Fields: []RPCField{ + { + Name: "context", + ProtoTypeName: DataTypeMessage, + Repeated: true, + Message: &RPCMessage{ + Name: "ResolveTestContainerDetailsContext", + Fields: []RPCField{ + { + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", + ResolvePath: buildPath("test_containers.id"), + }, + { + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", + ResolvePath: buildPath("test_containers.name"), + }, + }, + }, + }, + { + Name: "field_args", + ProtoTypeName: DataTypeMessage, + Message: &RPCMessage{ + Name: "ResolveTestContainerDetailsArgs", + Fields: []RPCField{ + { + Name: "include_extended", + ProtoTypeName: DataTypeBool, + JSONPath: "includeExtended", + }, + }, + }, + }, + }, + }, + Response: RPCMessage{ + Name: "ResolveTestContainerDetailsResponse", + Fields: []RPCField{ + { + Name: "result", + ProtoTypeName: DataTypeMessage, + JSONPath: "result", + Repeated: true, + Message: &RPCMessage{ + Name: "ResolveTestContainerDetailsResult", + Fields: []RPCField{ + { + Name: "details", + ProtoTypeName: DataTypeMessage, + JSONPath: "details", + Optional: true, + Message: &RPCMessage{ + Name: "TestDetails", + Fields: []RPCField{ + { + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", + }, + { + Name: "summary", + ProtoTypeName: DataTypeString, + JSONPath: "summary", + }, + { + Name: "pet", + ProtoTypeName: DataTypeMessage, + JSONPath: "pet", + Message: &RPCMessage{ + Name: "Animal", + OneOfType: OneOfTypeInterface, + MemberTypes: []string{"Cat", "Dog"}, + FieldSelectionSet: RPCFieldSelectionSet{ + "Cat": { + { + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", + }, + { + Name: "meow_volume", + ProtoTypeName: DataTypeInt32, + JSONPath: "meowVolume", + }, + }, + "Dog": { + { + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", + }, + { + Name: "bark_volume", + ProtoTypeName: DataTypeInt32, + JSONPath: "barkVolume", + }, + }, + }, + }, + }, + { + Name: "status", + ProtoTypeName: DataTypeMessage, + JSONPath: "status", + Message: &RPCMessage{ + Name: "ActionResult", + OneOfType: OneOfTypeUnion, + MemberTypes: []string{"ActionSuccess", "ActionError"}, + FieldSelectionSet: RPCFieldSelectionSet{ + "ActionSuccess": { + { + Name: "message", + ProtoTypeName: DataTypeString, + JSONPath: "message", + }, + { + Name: "timestamp", + ProtoTypeName: DataTypeString, + JSONPath: "timestamp", + }, + }, + "ActionError": { + { + Name: "message", + ProtoTypeName: DataTypeString, + JSONPath: "message", + }, + { + Name: "code", + ProtoTypeName: DataTypeString, + JSONPath: "code", + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, } for _, tt := range tests { diff --git a/v2/pkg/engine/datasource/grpc_datasource/grpc_datasource_test.go b/v2/pkg/engine/datasource/grpc_datasource/grpc_datasource_test.go index 8191b5b08..97893033a 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/grpc_datasource_test.go +++ b/v2/pkg/engine/datasource/grpc_datasource/grpc_datasource_test.go @@ -3898,6 +3898,253 @@ func Test_Datasource_Load_WithFieldResolvers(t *testing.T) { require.Empty(t, errData) }, }, + { + name: "Query with field resolvers and Interface type", + query: "query CategoriesWithInterfaceType($includeVolume: Boolean!) { categories { kind mascot(includeVolume: $includeVolume) { ... on Cat { name } ... on Dog { name } } } }", + vars: `{"variables":{"includeVolume":true}}`, + validate: func(t *testing.T, data map[string]interface{}) { + require.NotEmpty(t, data) + + categories, ok := data["categories"].([]interface{}) + require.True(t, ok, "categories should be an array") + require.NotEmpty(t, categories, "categories should not be empty") + + for _, category := range categories { + category, ok := category.(map[string]interface{}) + require.True(t, ok, "category should be an object") + require.NotEmpty(t, category["kind"]) + if category["kind"] == "OTHER" { + require.Empty(t, category["mascot"]) + continue + } + + require.NotEmpty(t, category["mascot"]) + mascot, ok := category["mascot"].(map[string]interface{}) + require.True(t, ok, "mascot should be an object") + require.NotEmpty(t, mascot["name"]) + } + }, + validateError: func(t *testing.T, errData []graphqlError) { + require.Empty(t, errData) + }, + }, + { + name: "Query with field resolvers and Union type", + query: "query CategoriesWithUnionType($checkHealth: Boolean!) { categories { id name categoryStatus(checkHealth: $checkHealth) { ... on ActionSuccess { message timestamp } ... on ActionError { message code } } } }", + vars: `{"variables":{"checkHealth":true}}`, + validate: func(t *testing.T, data map[string]interface{}) { + require.NotEmpty(t, data) + + categories, ok := data["categories"].([]interface{}) + require.True(t, ok, "categories should be an array") + require.NotEmpty(t, categories, "categories should not be empty") + require.Len(t, categories, 4, "Should return 4 categories") + + // Based on mockservice.go implementation: + // - If checkHealth && i%3 == 0, returns ActionError + // - Otherwise, returns ActionSuccess + for i, category := range categories { + category, ok := category.(map[string]interface{}) + require.True(t, ok, "category should be an object") + require.NotEmpty(t, category["id"]) + require.NotEmpty(t, category["name"]) + require.NotEmpty(t, category["categoryStatus"]) + + categoryStatus, ok := category["categoryStatus"].(map[string]interface{}) + require.True(t, ok, "categoryStatus should be an object") + + if i%3 == 0 { + // Should be ActionError + require.NotEmpty(t, categoryStatus["message"], "ActionError should have message") + require.NotEmpty(t, categoryStatus["code"], "ActionError should have code") + require.Empty(t, categoryStatus["timestamp"], "ActionError should not have timestamp") + require.Contains(t, categoryStatus["message"], "Health check failed", "ActionError message should contain 'Health check failed'") + require.Equal(t, "HEALTH_CHECK_FAILED", categoryStatus["code"], "ActionError code should be HEALTH_CHECK_FAILED") + } else { + // Should be ActionSuccess + require.NotEmpty(t, categoryStatus["message"], "ActionSuccess should have message") + require.NotEmpty(t, categoryStatus["timestamp"], "ActionSuccess should have timestamp") + require.Empty(t, categoryStatus["code"], "ActionSuccess should not have code") + require.Contains(t, categoryStatus["message"], "is healthy", "ActionSuccess message should contain 'is healthy'") + require.Equal(t, "2024-01-01T00:00:00Z", categoryStatus["timestamp"], "ActionSuccess timestamp should match") + } + } + }, + validateError: func(t *testing.T, errData []graphqlError) { + require.Empty(t, errData) + }, + }, + { + name: "Query with nested field resolver returning interface type", + query: "query TestContainersWithInterface($includeExtended: Boolean!) { testContainers { id name details(includeExtended: $includeExtended) { id summary pet { ... on Cat { name meowVolume } ... on Dog { name barkVolume } } } } }", + vars: `{"variables":{"includeExtended":false}}`, + validate: func(t *testing.T, data map[string]interface{}) { + require.NotEmpty(t, data) + + containers, ok := data["testContainers"].([]interface{}) + require.True(t, ok, "testContainers should be an array") + require.NotEmpty(t, containers, "testContainers should not be empty") + require.Len(t, containers, 3, "Should return 3 test containers") + + // Based on mockservice.go implementation: + // - Even indices (0, 2) return Cat + // - Odd indices (1) return Dog + for i, container := range containers { + container, ok := container.(map[string]interface{}) + require.True(t, ok, "container should be an object") + require.NotEmpty(t, container["id"]) + require.NotEmpty(t, container["name"]) + require.NotEmpty(t, container["details"]) + + details, ok := container["details"].(map[string]interface{}) + require.True(t, ok, "details should be an object") + require.NotEmpty(t, details["id"]) + require.NotEmpty(t, details["summary"]) + require.NotEmpty(t, details["pet"]) + + pet, ok := details["pet"].(map[string]interface{}) + require.True(t, ok, "pet should be an object") + require.NotEmpty(t, pet["name"]) + + if i%2 == 0 { + // Should be Cat + require.NotEmpty(t, pet["meowVolume"], "Cat should have meowVolume") + require.Empty(t, pet["barkVolume"], "Cat should not have barkVolume") + require.Contains(t, pet["name"], "TestCat", "Cat name should contain 'TestCat'") + } else { + // Should be Dog + require.NotEmpty(t, pet["barkVolume"], "Dog should have barkVolume") + require.Empty(t, pet["meowVolume"], "Dog should not have meowVolume") + require.Contains(t, pet["name"], "TestDog", "Dog name should contain 'TestDog'") + } + } + }, + validateError: func(t *testing.T, errData []graphqlError) { + require.Empty(t, errData) + }, + }, + { + name: "Query with nested field resolver returning union type", + query: "query TestContainersWithUnion($includeExtended: Boolean!) { testContainers { id name details(includeExtended: $includeExtended) { id summary status { ... on ActionSuccess { message timestamp } ... on ActionError { message code } } } } }", + vars: `{"variables":{"includeExtended":true}}`, + validate: func(t *testing.T, data map[string]interface{}) { + require.NotEmpty(t, data) + + containers, ok := data["testContainers"].([]interface{}) + require.True(t, ok, "testContainers should be an array") + require.NotEmpty(t, containers, "testContainers should not be empty") + require.Len(t, containers, 3, "Should return 3 test containers") + + // Based on mockservice.go implementation: + // - When includeExtended=true && i%3 == 0, returns ActionError + // - Otherwise, returns ActionSuccess + for i, container := range containers { + container, ok := container.(map[string]interface{}) + require.True(t, ok, "container should be an object") + require.NotEmpty(t, container["id"]) + require.NotEmpty(t, container["name"]) + require.NotEmpty(t, container["details"]) + + details, ok := container["details"].(map[string]interface{}) + require.True(t, ok, "details should be an object") + require.NotEmpty(t, details["id"]) + require.NotEmpty(t, details["summary"]) + require.Contains(t, details["summary"], "Extended summary", "Summary should contain 'Extended summary'") + require.NotEmpty(t, details["status"]) + + status, ok := details["status"].(map[string]interface{}) + require.True(t, ok, "status should be an object") + + if i%3 == 0 { + // Should be ActionError + require.NotEmpty(t, status["message"], "ActionError should have message") + require.NotEmpty(t, status["code"], "ActionError should have code") + require.Empty(t, status["timestamp"], "ActionError should not have timestamp") + require.Contains(t, status["message"], "Extended check failed", "ActionError message should contain 'Extended check failed'") + require.Equal(t, "EXTENDED_CHECK_FAILED", status["code"], "ActionError code should be EXTENDED_CHECK_FAILED") + } else { + // Should be ActionSuccess + require.NotEmpty(t, status["message"], "ActionSuccess should have message") + require.NotEmpty(t, status["timestamp"], "ActionSuccess should have timestamp") + require.Empty(t, status["code"], "ActionSuccess should not have code") + require.Contains(t, status["message"], "details loaded successfully", "ActionSuccess message should contain 'details loaded successfully'") + require.Equal(t, "2024-01-01T12:00:00Z", status["timestamp"], "ActionSuccess timestamp should match") + } + } + }, + validateError: func(t *testing.T, errData []graphqlError) { + require.Empty(t, errData) + }, + }, + { + name: "Query with nested field resolver returning both interface and union types", + query: "query TestContainersWithBoth($includeExtended: Boolean!) { testContainers { id name details(includeExtended: $includeExtended) { id summary pet { ... on Cat { name meowVolume } ... on Dog { name barkVolume } } status { ... on ActionSuccess { message timestamp } ... on ActionError { message code } } } } }", + vars: `{"variables":{"includeExtended":true}}`, + validate: func(t *testing.T, data map[string]interface{}) { + require.NotEmpty(t, data) + + containers, ok := data["testContainers"].([]interface{}) + require.True(t, ok, "testContainers should be an array") + require.NotEmpty(t, containers, "testContainers should not be empty") + require.Len(t, containers, 3, "Should return 3 test containers") + + // Validate both pet (interface) and status (union) fields + for i, container := range containers { + container, ok := container.(map[string]interface{}) + require.True(t, ok, "container should be an object") + require.NotEmpty(t, container["id"]) + require.NotEmpty(t, container["name"]) + require.NotEmpty(t, container["details"]) + + details, ok := container["details"].(map[string]interface{}) + require.True(t, ok, "details should be an object") + require.NotEmpty(t, details["id"]) + require.NotEmpty(t, details["summary"]) + require.NotEmpty(t, details["pet"]) + require.NotEmpty(t, details["status"]) + + // Validate pet (Animal interface) + pet, ok := details["pet"].(map[string]interface{}) + require.True(t, ok, "pet should be an object") + require.NotEmpty(t, pet["name"]) + + if i%2 == 0 { + // Should be Cat + require.NotEmpty(t, pet["meowVolume"], "Cat should have meowVolume") + require.Empty(t, pet["barkVolume"], "Cat should not have barkVolume") + require.Contains(t, pet["name"], "TestCat", "Cat name should contain 'TestCat'") + } else { + // Should be Dog + require.NotEmpty(t, pet["barkVolume"], "Dog should have barkVolume") + require.Empty(t, pet["meowVolume"], "Dog should not have meowVolume") + require.Contains(t, pet["name"], "TestDog", "Dog name should contain 'TestDog'") + } + + // Validate status (ActionResult union) + status, ok := details["status"].(map[string]interface{}) + require.True(t, ok, "status should be an object") + + if i%3 == 0 { + // Should be ActionError + require.NotEmpty(t, status["message"], "ActionError should have message") + require.NotEmpty(t, status["code"], "ActionError should have code") + require.Empty(t, status["timestamp"], "ActionError should not have timestamp") + require.Contains(t, status["message"], "Extended check failed", "ActionError message should contain 'Extended check failed'") + require.Equal(t, "EXTENDED_CHECK_FAILED", status["code"], "ActionError code should be EXTENDED_CHECK_FAILED") + } else { + // Should be ActionSuccess + require.NotEmpty(t, status["message"], "ActionSuccess should have message") + require.NotEmpty(t, status["timestamp"], "ActionSuccess should have timestamp") + require.Empty(t, status["code"], "ActionSuccess should not have code") + require.Contains(t, status["message"], "details loaded successfully", "ActionSuccess message should contain 'details loaded successfully'") + require.Equal(t, "2024-01-01T12:00:00Z", status["timestamp"], "ActionSuccess timestamp should match") + } + } + }, + validateError: func(t *testing.T, errData []graphqlError) { + require.Empty(t, errData) + }, + }, } for _, tc := range testCases { diff --git a/v2/pkg/engine/datasource/grpc_datasource/mapping_test_helper.go b/v2/pkg/engine/datasource/grpc_datasource/mapping_test_helper.go index d4039f4f9..e5387a130 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/mapping_test_helper.go +++ b/v2/pkg/engine/datasource/grpc_datasource/mapping_test_helper.go @@ -154,6 +154,16 @@ func testMapping() *GRPCMapping { Request: "QueryBulkSearchBlogPostsRequest", Response: "QueryBulkSearchBlogPostsResponse", }, + "testContainer": { + RPC: "QueryTestContainer", + Request: "QueryTestContainerRequest", + Response: "QueryTestContainerResponse", + }, + "testContainers": { + RPC: "QueryTestContainers", + Request: "QueryTestContainersRequest", + Response: "QueryTestContainersResponse", + }, }, MutationRPCs: RPCConfigMap[RPCConfig]{ "createUser": { @@ -326,6 +336,19 @@ func testMapping() *GRPCMapping { Response: "ResolveSubcategoryItemCountResponse", }, }, + "TestContainer": { + "details": { + FieldMappingData: FieldMapData{ + TargetName: "details", + ArgumentMappings: FieldArgumentMap{ + "includeExtended": "include_extended", + }, + }, + RPC: "ResolveTestContainerDetails", + Request: "ResolveTestContainerDetailsRequest", + Response: "ResolveTestContainerDetailsResponse", + }, + }, }, EntityRPCs: map[string][]EntityRPCConfig{ "Product": { @@ -516,6 +539,15 @@ func testMapping() *GRPCMapping { "filters": "filters", }, }, + "testContainer": { + TargetName: "test_container", + ArgumentMappings: FieldArgumentMap{ + "id": "id", + }, + }, + "testContainers": { + TargetName: "test_containers", + }, }, "Mutation": { "createUser": { @@ -934,6 +966,37 @@ func testMapping() *GRPCMapping { TargetName: "code", }, }, + "TestContainer": { + "id": { + TargetName: "id", + }, + "name": { + TargetName: "name", + }, + "description": { + TargetName: "description", + }, + "details": { + TargetName: "details", + ArgumentMappings: FieldArgumentMap{ + "includeExtended": "include_extended", + }, + }, + }, + "TestDetails": { + "id": { + TargetName: "id", + }, + "summary": { + TargetName: "summary", + }, + "pet": { + TargetName: "pet", + }, + "status": { + TargetName: "status", + }, + }, "SearchInput": { "query": { TargetName: "query", diff --git a/v2/pkg/engine/datasource/grpc_datasource/required_fields_visitor.go b/v2/pkg/engine/datasource/grpc_datasource/required_fields_visitor.go index 09e531da9..42c132ec2 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/required_fields_visitor.go +++ b/v2/pkg/engine/datasource/grpc_datasource/required_fields_visitor.go @@ -142,7 +142,7 @@ func (r *requiredFieldsVisitor) EnterField(ref int) { fd, ok := r.walker.FieldDefinition(ref) if !ok { - r.walker.StopWithInternalErr(fmt.Errorf("field definition not found for field %s", fieldName)) + r.walker.StopWithInternalErr(fmt.Errorf("RequiredFieldsVisitor: field definition not found for field %s", fieldName)) return } diff --git a/v2/pkg/grpctest/mapping/mapping.go b/v2/pkg/grpctest/mapping/mapping.go index 95031912a..26898ea22 100644 --- a/v2/pkg/grpctest/mapping/mapping.go +++ b/v2/pkg/grpctest/mapping/mapping.go @@ -161,6 +161,16 @@ func DefaultGRPCMapping() *grpcdatasource.GRPCMapping { Request: "QueryBulkSearchBlogPostsRequest", Response: "QueryBulkSearchBlogPostsResponse", }, + "testContainer": { + RPC: "QueryTestContainer", + Request: "QueryTestContainerRequest", + Response: "QueryTestContainerResponse", + }, + "testContainers": { + RPC: "QueryTestContainers", + Request: "QueryTestContainersRequest", + Response: "QueryTestContainersResponse", + }, }, MutationRPCs: grpcdatasource.RPCConfigMap[grpcdatasource.RPCConfig]{ "createUser": { @@ -333,6 +343,19 @@ func DefaultGRPCMapping() *grpcdatasource.GRPCMapping { Response: "ResolveSubcategoryItemCountResponse", }, }, + "TestContainer": { + "details": { + FieldMappingData: grpcdatasource.FieldMapData{ + TargetName: "details", + ArgumentMappings: grpcdatasource.FieldArgumentMap{ + "includeExtended": "include_extended", + }, + }, + RPC: "ResolveTestContainerDetails", + Request: "ResolveTestContainerDetailsRequest", + Response: "ResolveTestContainerDetailsResponse", + }, + }, }, EntityRPCs: map[string][]grpcdatasource.EntityRPCConfig{ "Product": { @@ -523,6 +546,15 @@ func DefaultGRPCMapping() *grpcdatasource.GRPCMapping { "filters": "filters", }, }, + "testContainer": { + TargetName: "test_container", + ArgumentMappings: grpcdatasource.FieldArgumentMap{ + "id": "id", + }, + }, + "testContainers": { + TargetName: "test_containers", + }, }, "Mutation": { "createUser": { @@ -941,6 +973,37 @@ func DefaultGRPCMapping() *grpcdatasource.GRPCMapping { TargetName: "code", }, }, + "TestContainer": { + "id": { + TargetName: "id", + }, + "name": { + TargetName: "name", + }, + "description": { + TargetName: "description", + }, + "details": { + TargetName: "details", + ArgumentMappings: grpcdatasource.FieldArgumentMap{ + "includeExtended": "include_extended", + }, + }, + }, + "TestDetails": { + "id": { + TargetName: "id", + }, + "summary": { + TargetName: "summary", + }, + "pet": { + TargetName: "pet", + }, + "status": { + TargetName: "status", + }, + }, "SearchInput": { "query": { TargetName: "query", diff --git a/v2/pkg/grpctest/mockservice.go b/v2/pkg/grpctest/mockservice.go index 9c5ddc413..91dbaf434 100644 --- a/v2/pkg/grpctest/mockservice.go +++ b/v2/pkg/grpctest/mockservice.go @@ -20,6 +20,117 @@ type MockService struct { productv1.UnimplementedProductServiceServer } +// QueryTestContainer implements productv1.ProductServiceServer. +func (s *MockService) QueryTestContainer(_ context.Context, req *productv1.QueryTestContainerRequest) (*productv1.QueryTestContainerResponse, error) { + id := req.GetId() + + return &productv1.QueryTestContainerResponse{ + TestContainer: &productv1.TestContainer{ + Id: id, + Name: fmt.Sprintf("TestContainer-%s", id), + Description: &wrapperspb.StringValue{Value: fmt.Sprintf("Description for TestContainer %s", id)}, + }, + }, nil +} + +// QueryTestContainers implements productv1.ProductServiceServer. +func (s *MockService) QueryTestContainers(_ context.Context, _ *productv1.QueryTestContainersRequest) (*productv1.QueryTestContainersResponse, error) { + var containers []*productv1.TestContainer + + // Generate 3 test containers + for i := 1; i <= 3; i++ { + containers = append(containers, &productv1.TestContainer{ + Id: fmt.Sprintf("container-%d", i), + Name: fmt.Sprintf("TestContainer %d", i), + Description: &wrapperspb.StringValue{Value: fmt.Sprintf("Description for container %d", i)}, + }) + } + + return &productv1.QueryTestContainersResponse{ + TestContainers: containers, + }, nil +} + +// ResolveTestContainerDetails implements productv1.ProductServiceServer. +func (s *MockService) ResolveTestContainerDetails(_ context.Context, req *productv1.ResolveTestContainerDetailsRequest) (*productv1.ResolveTestContainerDetailsResponse, error) { + results := make([]*productv1.ResolveTestContainerDetailsResult, 0, len(req.GetContext())) + + includeExtended := false + if req.GetFieldArgs() != nil { + includeExtended = req.GetFieldArgs().GetIncludeExtended() + } + + for i, ctx := range req.GetContext() { + // Alternate between Cat and Dog for the pet field (Animal interface) + var pet *productv1.Animal + if i%2 == 0 { + pet = &productv1.Animal{ + Instance: &productv1.Animal_Cat{ + Cat: &productv1.Cat{ + Id: fmt.Sprintf("test-cat-%s", ctx.GetId()), + Name: fmt.Sprintf("TestCat-%s", ctx.GetName()), + Kind: "Cat", + MeowVolume: int32((i + 1) * 5), + }, + }, + } + } else { + pet = &productv1.Animal{ + Instance: &productv1.Animal_Dog{ + Dog: &productv1.Dog{ + Id: fmt.Sprintf("test-dog-%s", ctx.GetId()), + Name: fmt.Sprintf("TestDog-%s", ctx.GetName()), + Kind: "Dog", + BarkVolume: int32((i + 1) * 7), + }, + }, + } + } + + // Alternate between ActionSuccess and ActionError for the status field (ActionResult union) + var status *productv1.ActionResult + if includeExtended && i%3 == 0 { + // Return error status for extended mode on certain items + status = &productv1.ActionResult{ + Value: &productv1.ActionResult_ActionError{ + ActionError: &productv1.ActionError{ + Message: fmt.Sprintf("Extended check failed for %s", ctx.GetName()), + Code: "EXTENDED_CHECK_FAILED", + }, + }, + } + } else { + // Return success status + status = &productv1.ActionResult{ + Value: &productv1.ActionResult_ActionSuccess{ + ActionSuccess: &productv1.ActionSuccess{ + Message: fmt.Sprintf("TestContainer %s details loaded successfully", ctx.GetName()), + Timestamp: "2024-01-01T12:00:00Z", + }, + }, + } + } + + summary := fmt.Sprintf("Summary for %s", ctx.GetName()) + if includeExtended { + summary = fmt.Sprintf("Extended summary for %s with additional details", ctx.GetName()) + } + + results = append(results, &productv1.ResolveTestContainerDetailsResult{ + Details: &productv1.TestDetails{ + Id: fmt.Sprintf("details-%s-%d", ctx.GetId(), i), + Summary: summary, + Pet: pet, + Status: status, + }, + }) + } + + return &productv1.ResolveTestContainerDetailsResponse{ + Result: results, + }, nil +} + // ResolveCategoryMetricsNormalizedScore implements productv1.ProductServiceServer. func (s *MockService) ResolveCategoryMetricsNormalizedScore(_ context.Context, req *productv1.ResolveCategoryMetricsNormalizedScoreRequest) (*productv1.ResolveCategoryMetricsNormalizedScoreResponse, error) { results := make([]*productv1.ResolveCategoryMetricsNormalizedScoreResult, 0, len(req.GetContext())) diff --git a/v2/pkg/grpctest/product.proto b/v2/pkg/grpctest/product.proto index f4792611b..7f1fdb9f8 100644 --- a/v2/pkg/grpctest/product.proto +++ b/v2/pkg/grpctest/product.proto @@ -51,6 +51,8 @@ service ProductService { rpc QueryRandomSearchResult(QueryRandomSearchResultRequest) returns (QueryRandomSearchResultResponse) {} rpc QueryRecursiveType(QueryRecursiveTypeRequest) returns (QueryRecursiveTypeResponse) {} rpc QuerySearch(QuerySearchRequest) returns (QuerySearchResponse) {} + rpc QueryTestContainer(QueryTestContainerRequest) returns (QueryTestContainerResponse) {} + rpc QueryTestContainers(QueryTestContainersRequest) returns (QueryTestContainersResponse) {} rpc QueryTypeFilterWithArguments(QueryTypeFilterWithArgumentsRequest) returns (QueryTypeFilterWithArgumentsResponse) {} rpc QueryTypeWithMultipleFilterFields(QueryTypeWithMultipleFilterFieldsRequest) returns (QueryTypeWithMultipleFilterFieldsResponse) {} rpc QueryUser(QueryUserRequest) returns (QueryUserResponse) {} @@ -64,6 +66,7 @@ service ProductService { rpc ResolveProductRecommendedCategory(ResolveProductRecommendedCategoryRequest) returns (ResolveProductRecommendedCategoryResponse) {} rpc ResolveProductShippingEstimate(ResolveProductShippingEstimateRequest) returns (ResolveProductShippingEstimateResponse) {} rpc ResolveSubcategoryItemCount(ResolveSubcategoryItemCountRequest) returns (ResolveSubcategoryItemCountResponse) {} + rpc ResolveTestContainerDetails(ResolveTestContainerDetailsRequest) returns (ResolveTestContainerDetailsResponse) {} } // Wrapper message for a list of AuthorFilter. @@ -536,6 +539,21 @@ message QueryBulkSearchBlogPostsRequest { message QueryBulkSearchBlogPostsResponse { repeated BlogPost bulk_search_blog_posts = 1; } +// Request message for testContainer operation. +message QueryTestContainerRequest { + string id = 1; +} +// Response message for testContainer operation. +message QueryTestContainerResponse { + TestContainer test_container = 1; +} +// Request message for testContainers operation. +message QueryTestContainersRequest { +} +// Response message for testContainers operation. +message QueryTestContainersResponse { + repeated TestContainer test_containers = 1; +} // Request message for createUser operation. message MutationCreateUserRequest { UserInput input = 1; @@ -851,6 +869,30 @@ message ResolveCategoryMetricsNormalizedScoreResponse { repeated ResolveCategoryMetricsNormalizedScoreResult result = 1; } +message ResolveTestContainerDetailsArgs { + bool include_extended = 1; +} + +message ResolveTestContainerDetailsContext { + string id = 1; + string name = 2; +} + +message ResolveTestContainerDetailsRequest { + // context provides the resolver context for the field details of type TestContainer. + repeated ResolveTestContainerDetailsContext context = 1; + // field_args provides the arguments for the resolver field details of type TestContainer. + ResolveTestContainerDetailsArgs field_args = 2; +} + +message ResolveTestContainerDetailsResult { + TestDetails details = 1; +} + +message ResolveTestContainerDetailsResponse { + repeated ResolveTestContainerDetailsResult result = 1; +} + message Product { string id = 1; string name = 2; @@ -1022,6 +1064,12 @@ message AuthorFilter { google.protobuf.Int32Value skill_count = 3; } +message TestContainer { + string id = 1; + string name = 2; + google.protobuf.StringValue description = 3; +} + message UserInput { string name = 1; } @@ -1163,6 +1211,13 @@ message ActionError { string code = 2; } +message TestDetails { + string id = 1; + string summary = 2; + Animal pet = 3; + ActionResult status = 4; +} + message CategoryInput { string name = 1; CategoryKind kind = 2; diff --git a/v2/pkg/grpctest/productv1/product.pb.go b/v2/pkg/grpctest/productv1/product.pb.go index c4e4f4fea..7b48740ba 100644 --- a/v2/pkg/grpctest/productv1/product.pb.go +++ b/v2/pkg/grpctest/productv1/product.pb.go @@ -4086,6 +4086,178 @@ func (x *QueryBulkSearchBlogPostsResponse) GetBulkSearchBlogPosts() []*BlogPost return nil } +// Request message for testContainer operation. +type QueryTestContainerRequest struct { + state protoimpl.MessageState `protogen:"open.v1"` + Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *QueryTestContainerRequest) Reset() { + *x = QueryTestContainerRequest{} + mi := &file_product_proto_msgTypes[89] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *QueryTestContainerRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*QueryTestContainerRequest) ProtoMessage() {} + +func (x *QueryTestContainerRequest) ProtoReflect() protoreflect.Message { + mi := &file_product_proto_msgTypes[89] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use QueryTestContainerRequest.ProtoReflect.Descriptor instead. +func (*QueryTestContainerRequest) Descriptor() ([]byte, []int) { + return file_product_proto_rawDescGZIP(), []int{89} +} + +func (x *QueryTestContainerRequest) GetId() string { + if x != nil { + return x.Id + } + return "" +} + +// Response message for testContainer operation. +type QueryTestContainerResponse struct { + state protoimpl.MessageState `protogen:"open.v1"` + TestContainer *TestContainer `protobuf:"bytes,1,opt,name=test_container,json=testContainer,proto3" json:"test_container,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *QueryTestContainerResponse) Reset() { + *x = QueryTestContainerResponse{} + mi := &file_product_proto_msgTypes[90] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *QueryTestContainerResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*QueryTestContainerResponse) ProtoMessage() {} + +func (x *QueryTestContainerResponse) ProtoReflect() protoreflect.Message { + mi := &file_product_proto_msgTypes[90] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use QueryTestContainerResponse.ProtoReflect.Descriptor instead. +func (*QueryTestContainerResponse) Descriptor() ([]byte, []int) { + return file_product_proto_rawDescGZIP(), []int{90} +} + +func (x *QueryTestContainerResponse) GetTestContainer() *TestContainer { + if x != nil { + return x.TestContainer + } + return nil +} + +// Request message for testContainers operation. +type QueryTestContainersRequest struct { + state protoimpl.MessageState `protogen:"open.v1"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *QueryTestContainersRequest) Reset() { + *x = QueryTestContainersRequest{} + mi := &file_product_proto_msgTypes[91] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *QueryTestContainersRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*QueryTestContainersRequest) ProtoMessage() {} + +func (x *QueryTestContainersRequest) ProtoReflect() protoreflect.Message { + mi := &file_product_proto_msgTypes[91] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use QueryTestContainersRequest.ProtoReflect.Descriptor instead. +func (*QueryTestContainersRequest) Descriptor() ([]byte, []int) { + return file_product_proto_rawDescGZIP(), []int{91} +} + +// Response message for testContainers operation. +type QueryTestContainersResponse struct { + state protoimpl.MessageState `protogen:"open.v1"` + TestContainers []*TestContainer `protobuf:"bytes,1,rep,name=test_containers,json=testContainers,proto3" json:"test_containers,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *QueryTestContainersResponse) Reset() { + *x = QueryTestContainersResponse{} + mi := &file_product_proto_msgTypes[92] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *QueryTestContainersResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*QueryTestContainersResponse) ProtoMessage() {} + +func (x *QueryTestContainersResponse) ProtoReflect() protoreflect.Message { + mi := &file_product_proto_msgTypes[92] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use QueryTestContainersResponse.ProtoReflect.Descriptor instead. +func (*QueryTestContainersResponse) Descriptor() ([]byte, []int) { + return file_product_proto_rawDescGZIP(), []int{92} +} + +func (x *QueryTestContainersResponse) GetTestContainers() []*TestContainer { + if x != nil { + return x.TestContainers + } + return nil +} + // Request message for createUser operation. type MutationCreateUserRequest struct { state protoimpl.MessageState `protogen:"open.v1"` @@ -4096,7 +4268,7 @@ type MutationCreateUserRequest struct { func (x *MutationCreateUserRequest) Reset() { *x = MutationCreateUserRequest{} - mi := &file_product_proto_msgTypes[89] + mi := &file_product_proto_msgTypes[93] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -4108,7 +4280,7 @@ func (x *MutationCreateUserRequest) String() string { func (*MutationCreateUserRequest) ProtoMessage() {} func (x *MutationCreateUserRequest) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[89] + mi := &file_product_proto_msgTypes[93] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -4121,7 +4293,7 @@ func (x *MutationCreateUserRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use MutationCreateUserRequest.ProtoReflect.Descriptor instead. func (*MutationCreateUserRequest) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{89} + return file_product_proto_rawDescGZIP(), []int{93} } func (x *MutationCreateUserRequest) GetInput() *UserInput { @@ -4141,7 +4313,7 @@ type MutationCreateUserResponse struct { func (x *MutationCreateUserResponse) Reset() { *x = MutationCreateUserResponse{} - mi := &file_product_proto_msgTypes[90] + mi := &file_product_proto_msgTypes[94] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -4153,7 +4325,7 @@ func (x *MutationCreateUserResponse) String() string { func (*MutationCreateUserResponse) ProtoMessage() {} func (x *MutationCreateUserResponse) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[90] + mi := &file_product_proto_msgTypes[94] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -4166,7 +4338,7 @@ func (x *MutationCreateUserResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use MutationCreateUserResponse.ProtoReflect.Descriptor instead. func (*MutationCreateUserResponse) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{90} + return file_product_proto_rawDescGZIP(), []int{94} } func (x *MutationCreateUserResponse) GetCreateUser() *User { @@ -4186,7 +4358,7 @@ type MutationPerformActionRequest struct { func (x *MutationPerformActionRequest) Reset() { *x = MutationPerformActionRequest{} - mi := &file_product_proto_msgTypes[91] + mi := &file_product_proto_msgTypes[95] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -4198,7 +4370,7 @@ func (x *MutationPerformActionRequest) String() string { func (*MutationPerformActionRequest) ProtoMessage() {} func (x *MutationPerformActionRequest) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[91] + mi := &file_product_proto_msgTypes[95] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -4211,7 +4383,7 @@ func (x *MutationPerformActionRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use MutationPerformActionRequest.ProtoReflect.Descriptor instead. func (*MutationPerformActionRequest) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{91} + return file_product_proto_rawDescGZIP(), []int{95} } func (x *MutationPerformActionRequest) GetInput() *ActionInput { @@ -4231,7 +4403,7 @@ type MutationPerformActionResponse struct { func (x *MutationPerformActionResponse) Reset() { *x = MutationPerformActionResponse{} - mi := &file_product_proto_msgTypes[92] + mi := &file_product_proto_msgTypes[96] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -4243,7 +4415,7 @@ func (x *MutationPerformActionResponse) String() string { func (*MutationPerformActionResponse) ProtoMessage() {} func (x *MutationPerformActionResponse) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[92] + mi := &file_product_proto_msgTypes[96] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -4256,7 +4428,7 @@ func (x *MutationPerformActionResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use MutationPerformActionResponse.ProtoReflect.Descriptor instead. func (*MutationPerformActionResponse) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{92} + return file_product_proto_rawDescGZIP(), []int{96} } func (x *MutationPerformActionResponse) GetPerformAction() *ActionResult { @@ -4276,7 +4448,7 @@ type MutationCreateNullableFieldsTypeRequest struct { func (x *MutationCreateNullableFieldsTypeRequest) Reset() { *x = MutationCreateNullableFieldsTypeRequest{} - mi := &file_product_proto_msgTypes[93] + mi := &file_product_proto_msgTypes[97] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -4288,7 +4460,7 @@ func (x *MutationCreateNullableFieldsTypeRequest) String() string { func (*MutationCreateNullableFieldsTypeRequest) ProtoMessage() {} func (x *MutationCreateNullableFieldsTypeRequest) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[93] + mi := &file_product_proto_msgTypes[97] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -4301,7 +4473,7 @@ func (x *MutationCreateNullableFieldsTypeRequest) ProtoReflect() protoreflect.Me // Deprecated: Use MutationCreateNullableFieldsTypeRequest.ProtoReflect.Descriptor instead. func (*MutationCreateNullableFieldsTypeRequest) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{93} + return file_product_proto_rawDescGZIP(), []int{97} } func (x *MutationCreateNullableFieldsTypeRequest) GetInput() *NullableFieldsInput { @@ -4321,7 +4493,7 @@ type MutationCreateNullableFieldsTypeResponse struct { func (x *MutationCreateNullableFieldsTypeResponse) Reset() { *x = MutationCreateNullableFieldsTypeResponse{} - mi := &file_product_proto_msgTypes[94] + mi := &file_product_proto_msgTypes[98] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -4333,7 +4505,7 @@ func (x *MutationCreateNullableFieldsTypeResponse) String() string { func (*MutationCreateNullableFieldsTypeResponse) ProtoMessage() {} func (x *MutationCreateNullableFieldsTypeResponse) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[94] + mi := &file_product_proto_msgTypes[98] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -4346,7 +4518,7 @@ func (x *MutationCreateNullableFieldsTypeResponse) ProtoReflect() protoreflect.M // Deprecated: Use MutationCreateNullableFieldsTypeResponse.ProtoReflect.Descriptor instead. func (*MutationCreateNullableFieldsTypeResponse) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{94} + return file_product_proto_rawDescGZIP(), []int{98} } func (x *MutationCreateNullableFieldsTypeResponse) GetCreateNullableFieldsType() *NullableFieldsType { @@ -4367,7 +4539,7 @@ type MutationUpdateNullableFieldsTypeRequest struct { func (x *MutationUpdateNullableFieldsTypeRequest) Reset() { *x = MutationUpdateNullableFieldsTypeRequest{} - mi := &file_product_proto_msgTypes[95] + mi := &file_product_proto_msgTypes[99] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -4379,7 +4551,7 @@ func (x *MutationUpdateNullableFieldsTypeRequest) String() string { func (*MutationUpdateNullableFieldsTypeRequest) ProtoMessage() {} func (x *MutationUpdateNullableFieldsTypeRequest) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[95] + mi := &file_product_proto_msgTypes[99] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -4392,7 +4564,7 @@ func (x *MutationUpdateNullableFieldsTypeRequest) ProtoReflect() protoreflect.Me // Deprecated: Use MutationUpdateNullableFieldsTypeRequest.ProtoReflect.Descriptor instead. func (*MutationUpdateNullableFieldsTypeRequest) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{95} + return file_product_proto_rawDescGZIP(), []int{99} } func (x *MutationUpdateNullableFieldsTypeRequest) GetId() string { @@ -4419,7 +4591,7 @@ type MutationUpdateNullableFieldsTypeResponse struct { func (x *MutationUpdateNullableFieldsTypeResponse) Reset() { *x = MutationUpdateNullableFieldsTypeResponse{} - mi := &file_product_proto_msgTypes[96] + mi := &file_product_proto_msgTypes[100] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -4431,7 +4603,7 @@ func (x *MutationUpdateNullableFieldsTypeResponse) String() string { func (*MutationUpdateNullableFieldsTypeResponse) ProtoMessage() {} func (x *MutationUpdateNullableFieldsTypeResponse) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[96] + mi := &file_product_proto_msgTypes[100] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -4444,7 +4616,7 @@ func (x *MutationUpdateNullableFieldsTypeResponse) ProtoReflect() protoreflect.M // Deprecated: Use MutationUpdateNullableFieldsTypeResponse.ProtoReflect.Descriptor instead. func (*MutationUpdateNullableFieldsTypeResponse) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{96} + return file_product_proto_rawDescGZIP(), []int{100} } func (x *MutationUpdateNullableFieldsTypeResponse) GetUpdateNullableFieldsType() *NullableFieldsType { @@ -4464,7 +4636,7 @@ type MutationCreateBlogPostRequest struct { func (x *MutationCreateBlogPostRequest) Reset() { *x = MutationCreateBlogPostRequest{} - mi := &file_product_proto_msgTypes[97] + mi := &file_product_proto_msgTypes[101] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -4476,7 +4648,7 @@ func (x *MutationCreateBlogPostRequest) String() string { func (*MutationCreateBlogPostRequest) ProtoMessage() {} func (x *MutationCreateBlogPostRequest) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[97] + mi := &file_product_proto_msgTypes[101] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -4489,7 +4661,7 @@ func (x *MutationCreateBlogPostRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use MutationCreateBlogPostRequest.ProtoReflect.Descriptor instead. func (*MutationCreateBlogPostRequest) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{97} + return file_product_proto_rawDescGZIP(), []int{101} } func (x *MutationCreateBlogPostRequest) GetInput() *BlogPostInput { @@ -4509,7 +4681,7 @@ type MutationCreateBlogPostResponse struct { func (x *MutationCreateBlogPostResponse) Reset() { *x = MutationCreateBlogPostResponse{} - mi := &file_product_proto_msgTypes[98] + mi := &file_product_proto_msgTypes[102] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -4521,7 +4693,7 @@ func (x *MutationCreateBlogPostResponse) String() string { func (*MutationCreateBlogPostResponse) ProtoMessage() {} func (x *MutationCreateBlogPostResponse) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[98] + mi := &file_product_proto_msgTypes[102] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -4534,7 +4706,7 @@ func (x *MutationCreateBlogPostResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use MutationCreateBlogPostResponse.ProtoReflect.Descriptor instead. func (*MutationCreateBlogPostResponse) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{98} + return file_product_proto_rawDescGZIP(), []int{102} } func (x *MutationCreateBlogPostResponse) GetCreateBlogPost() *BlogPost { @@ -4555,7 +4727,7 @@ type MutationUpdateBlogPostRequest struct { func (x *MutationUpdateBlogPostRequest) Reset() { *x = MutationUpdateBlogPostRequest{} - mi := &file_product_proto_msgTypes[99] + mi := &file_product_proto_msgTypes[103] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -4567,7 +4739,7 @@ func (x *MutationUpdateBlogPostRequest) String() string { func (*MutationUpdateBlogPostRequest) ProtoMessage() {} func (x *MutationUpdateBlogPostRequest) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[99] + mi := &file_product_proto_msgTypes[103] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -4580,7 +4752,7 @@ func (x *MutationUpdateBlogPostRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use MutationUpdateBlogPostRequest.ProtoReflect.Descriptor instead. func (*MutationUpdateBlogPostRequest) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{99} + return file_product_proto_rawDescGZIP(), []int{103} } func (x *MutationUpdateBlogPostRequest) GetId() string { @@ -4607,7 +4779,7 @@ type MutationUpdateBlogPostResponse struct { func (x *MutationUpdateBlogPostResponse) Reset() { *x = MutationUpdateBlogPostResponse{} - mi := &file_product_proto_msgTypes[100] + mi := &file_product_proto_msgTypes[104] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -4619,7 +4791,7 @@ func (x *MutationUpdateBlogPostResponse) String() string { func (*MutationUpdateBlogPostResponse) ProtoMessage() {} func (x *MutationUpdateBlogPostResponse) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[100] + mi := &file_product_proto_msgTypes[104] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -4632,7 +4804,7 @@ func (x *MutationUpdateBlogPostResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use MutationUpdateBlogPostResponse.ProtoReflect.Descriptor instead. func (*MutationUpdateBlogPostResponse) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{100} + return file_product_proto_rawDescGZIP(), []int{104} } func (x *MutationUpdateBlogPostResponse) GetUpdateBlogPost() *BlogPost { @@ -4652,7 +4824,7 @@ type MutationCreateAuthorRequest struct { func (x *MutationCreateAuthorRequest) Reset() { *x = MutationCreateAuthorRequest{} - mi := &file_product_proto_msgTypes[101] + mi := &file_product_proto_msgTypes[105] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -4664,7 +4836,7 @@ func (x *MutationCreateAuthorRequest) String() string { func (*MutationCreateAuthorRequest) ProtoMessage() {} func (x *MutationCreateAuthorRequest) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[101] + mi := &file_product_proto_msgTypes[105] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -4677,7 +4849,7 @@ func (x *MutationCreateAuthorRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use MutationCreateAuthorRequest.ProtoReflect.Descriptor instead. func (*MutationCreateAuthorRequest) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{101} + return file_product_proto_rawDescGZIP(), []int{105} } func (x *MutationCreateAuthorRequest) GetInput() *AuthorInput { @@ -4697,7 +4869,7 @@ type MutationCreateAuthorResponse struct { func (x *MutationCreateAuthorResponse) Reset() { *x = MutationCreateAuthorResponse{} - mi := &file_product_proto_msgTypes[102] + mi := &file_product_proto_msgTypes[106] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -4709,7 +4881,7 @@ func (x *MutationCreateAuthorResponse) String() string { func (*MutationCreateAuthorResponse) ProtoMessage() {} func (x *MutationCreateAuthorResponse) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[102] + mi := &file_product_proto_msgTypes[106] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -4722,7 +4894,7 @@ func (x *MutationCreateAuthorResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use MutationCreateAuthorResponse.ProtoReflect.Descriptor instead. func (*MutationCreateAuthorResponse) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{102} + return file_product_proto_rawDescGZIP(), []int{106} } func (x *MutationCreateAuthorResponse) GetCreateAuthor() *Author { @@ -4743,7 +4915,7 @@ type MutationUpdateAuthorRequest struct { func (x *MutationUpdateAuthorRequest) Reset() { *x = MutationUpdateAuthorRequest{} - mi := &file_product_proto_msgTypes[103] + mi := &file_product_proto_msgTypes[107] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -4755,7 +4927,7 @@ func (x *MutationUpdateAuthorRequest) String() string { func (*MutationUpdateAuthorRequest) ProtoMessage() {} func (x *MutationUpdateAuthorRequest) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[103] + mi := &file_product_proto_msgTypes[107] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -4768,7 +4940,7 @@ func (x *MutationUpdateAuthorRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use MutationUpdateAuthorRequest.ProtoReflect.Descriptor instead. func (*MutationUpdateAuthorRequest) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{103} + return file_product_proto_rawDescGZIP(), []int{107} } func (x *MutationUpdateAuthorRequest) GetId() string { @@ -4795,7 +4967,7 @@ type MutationUpdateAuthorResponse struct { func (x *MutationUpdateAuthorResponse) Reset() { *x = MutationUpdateAuthorResponse{} - mi := &file_product_proto_msgTypes[104] + mi := &file_product_proto_msgTypes[108] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -4807,7 +4979,7 @@ func (x *MutationUpdateAuthorResponse) String() string { func (*MutationUpdateAuthorResponse) ProtoMessage() {} func (x *MutationUpdateAuthorResponse) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[104] + mi := &file_product_proto_msgTypes[108] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -4820,7 +4992,7 @@ func (x *MutationUpdateAuthorResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use MutationUpdateAuthorResponse.ProtoReflect.Descriptor instead. func (*MutationUpdateAuthorResponse) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{104} + return file_product_proto_rawDescGZIP(), []int{108} } func (x *MutationUpdateAuthorResponse) GetUpdateAuthor() *Author { @@ -4840,7 +5012,7 @@ type MutationBulkCreateAuthorsRequest struct { func (x *MutationBulkCreateAuthorsRequest) Reset() { *x = MutationBulkCreateAuthorsRequest{} - mi := &file_product_proto_msgTypes[105] + mi := &file_product_proto_msgTypes[109] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -4852,7 +5024,7 @@ func (x *MutationBulkCreateAuthorsRequest) String() string { func (*MutationBulkCreateAuthorsRequest) ProtoMessage() {} func (x *MutationBulkCreateAuthorsRequest) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[105] + mi := &file_product_proto_msgTypes[109] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -4865,7 +5037,7 @@ func (x *MutationBulkCreateAuthorsRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use MutationBulkCreateAuthorsRequest.ProtoReflect.Descriptor instead. func (*MutationBulkCreateAuthorsRequest) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{105} + return file_product_proto_rawDescGZIP(), []int{109} } func (x *MutationBulkCreateAuthorsRequest) GetAuthors() *ListOfAuthorInput { @@ -4885,7 +5057,7 @@ type MutationBulkCreateAuthorsResponse struct { func (x *MutationBulkCreateAuthorsResponse) Reset() { *x = MutationBulkCreateAuthorsResponse{} - mi := &file_product_proto_msgTypes[106] + mi := &file_product_proto_msgTypes[110] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -4897,7 +5069,7 @@ func (x *MutationBulkCreateAuthorsResponse) String() string { func (*MutationBulkCreateAuthorsResponse) ProtoMessage() {} func (x *MutationBulkCreateAuthorsResponse) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[106] + mi := &file_product_proto_msgTypes[110] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -4910,7 +5082,7 @@ func (x *MutationBulkCreateAuthorsResponse) ProtoReflect() protoreflect.Message // Deprecated: Use MutationBulkCreateAuthorsResponse.ProtoReflect.Descriptor instead. func (*MutationBulkCreateAuthorsResponse) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{106} + return file_product_proto_rawDescGZIP(), []int{110} } func (x *MutationBulkCreateAuthorsResponse) GetBulkCreateAuthors() []*Author { @@ -4930,7 +5102,7 @@ type MutationBulkUpdateAuthorsRequest struct { func (x *MutationBulkUpdateAuthorsRequest) Reset() { *x = MutationBulkUpdateAuthorsRequest{} - mi := &file_product_proto_msgTypes[107] + mi := &file_product_proto_msgTypes[111] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -4942,7 +5114,7 @@ func (x *MutationBulkUpdateAuthorsRequest) String() string { func (*MutationBulkUpdateAuthorsRequest) ProtoMessage() {} func (x *MutationBulkUpdateAuthorsRequest) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[107] + mi := &file_product_proto_msgTypes[111] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -4955,7 +5127,7 @@ func (x *MutationBulkUpdateAuthorsRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use MutationBulkUpdateAuthorsRequest.ProtoReflect.Descriptor instead. func (*MutationBulkUpdateAuthorsRequest) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{107} + return file_product_proto_rawDescGZIP(), []int{111} } func (x *MutationBulkUpdateAuthorsRequest) GetAuthors() *ListOfAuthorInput { @@ -4975,7 +5147,7 @@ type MutationBulkUpdateAuthorsResponse struct { func (x *MutationBulkUpdateAuthorsResponse) Reset() { *x = MutationBulkUpdateAuthorsResponse{} - mi := &file_product_proto_msgTypes[108] + mi := &file_product_proto_msgTypes[112] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -4987,7 +5159,7 @@ func (x *MutationBulkUpdateAuthorsResponse) String() string { func (*MutationBulkUpdateAuthorsResponse) ProtoMessage() {} func (x *MutationBulkUpdateAuthorsResponse) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[108] + mi := &file_product_proto_msgTypes[112] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5000,7 +5172,7 @@ func (x *MutationBulkUpdateAuthorsResponse) ProtoReflect() protoreflect.Message // Deprecated: Use MutationBulkUpdateAuthorsResponse.ProtoReflect.Descriptor instead. func (*MutationBulkUpdateAuthorsResponse) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{108} + return file_product_proto_rawDescGZIP(), []int{112} } func (x *MutationBulkUpdateAuthorsResponse) GetBulkUpdateAuthors() []*Author { @@ -5020,7 +5192,7 @@ type MutationBulkCreateBlogPostsRequest struct { func (x *MutationBulkCreateBlogPostsRequest) Reset() { *x = MutationBulkCreateBlogPostsRequest{} - mi := &file_product_proto_msgTypes[109] + mi := &file_product_proto_msgTypes[113] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5032,7 +5204,7 @@ func (x *MutationBulkCreateBlogPostsRequest) String() string { func (*MutationBulkCreateBlogPostsRequest) ProtoMessage() {} func (x *MutationBulkCreateBlogPostsRequest) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[109] + mi := &file_product_proto_msgTypes[113] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5045,7 +5217,7 @@ func (x *MutationBulkCreateBlogPostsRequest) ProtoReflect() protoreflect.Message // Deprecated: Use MutationBulkCreateBlogPostsRequest.ProtoReflect.Descriptor instead. func (*MutationBulkCreateBlogPostsRequest) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{109} + return file_product_proto_rawDescGZIP(), []int{113} } func (x *MutationBulkCreateBlogPostsRequest) GetBlogPosts() *ListOfBlogPostInput { @@ -5065,7 +5237,7 @@ type MutationBulkCreateBlogPostsResponse struct { func (x *MutationBulkCreateBlogPostsResponse) Reset() { *x = MutationBulkCreateBlogPostsResponse{} - mi := &file_product_proto_msgTypes[110] + mi := &file_product_proto_msgTypes[114] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5077,7 +5249,7 @@ func (x *MutationBulkCreateBlogPostsResponse) String() string { func (*MutationBulkCreateBlogPostsResponse) ProtoMessage() {} func (x *MutationBulkCreateBlogPostsResponse) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[110] + mi := &file_product_proto_msgTypes[114] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5090,7 +5262,7 @@ func (x *MutationBulkCreateBlogPostsResponse) ProtoReflect() protoreflect.Messag // Deprecated: Use MutationBulkCreateBlogPostsResponse.ProtoReflect.Descriptor instead. func (*MutationBulkCreateBlogPostsResponse) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{110} + return file_product_proto_rawDescGZIP(), []int{114} } func (x *MutationBulkCreateBlogPostsResponse) GetBulkCreateBlogPosts() []*BlogPost { @@ -5110,7 +5282,7 @@ type MutationBulkUpdateBlogPostsRequest struct { func (x *MutationBulkUpdateBlogPostsRequest) Reset() { *x = MutationBulkUpdateBlogPostsRequest{} - mi := &file_product_proto_msgTypes[111] + mi := &file_product_proto_msgTypes[115] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5122,7 +5294,7 @@ func (x *MutationBulkUpdateBlogPostsRequest) String() string { func (*MutationBulkUpdateBlogPostsRequest) ProtoMessage() {} func (x *MutationBulkUpdateBlogPostsRequest) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[111] + mi := &file_product_proto_msgTypes[115] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5135,7 +5307,7 @@ func (x *MutationBulkUpdateBlogPostsRequest) ProtoReflect() protoreflect.Message // Deprecated: Use MutationBulkUpdateBlogPostsRequest.ProtoReflect.Descriptor instead. func (*MutationBulkUpdateBlogPostsRequest) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{111} + return file_product_proto_rawDescGZIP(), []int{115} } func (x *MutationBulkUpdateBlogPostsRequest) GetBlogPosts() *ListOfBlogPostInput { @@ -5155,7 +5327,7 @@ type MutationBulkUpdateBlogPostsResponse struct { func (x *MutationBulkUpdateBlogPostsResponse) Reset() { *x = MutationBulkUpdateBlogPostsResponse{} - mi := &file_product_proto_msgTypes[112] + mi := &file_product_proto_msgTypes[116] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5167,7 +5339,7 @@ func (x *MutationBulkUpdateBlogPostsResponse) String() string { func (*MutationBulkUpdateBlogPostsResponse) ProtoMessage() {} func (x *MutationBulkUpdateBlogPostsResponse) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[112] + mi := &file_product_proto_msgTypes[116] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5180,7 +5352,7 @@ func (x *MutationBulkUpdateBlogPostsResponse) ProtoReflect() protoreflect.Messag // Deprecated: Use MutationBulkUpdateBlogPostsResponse.ProtoReflect.Descriptor instead. func (*MutationBulkUpdateBlogPostsResponse) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{112} + return file_product_proto_rawDescGZIP(), []int{116} } func (x *MutationBulkUpdateBlogPostsResponse) GetBulkUpdateBlogPosts() []*BlogPost { @@ -5199,7 +5371,7 @@ type ResolveProductShippingEstimateArgs struct { func (x *ResolveProductShippingEstimateArgs) Reset() { *x = ResolveProductShippingEstimateArgs{} - mi := &file_product_proto_msgTypes[113] + mi := &file_product_proto_msgTypes[117] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5211,7 +5383,7 @@ func (x *ResolveProductShippingEstimateArgs) String() string { func (*ResolveProductShippingEstimateArgs) ProtoMessage() {} func (x *ResolveProductShippingEstimateArgs) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[113] + mi := &file_product_proto_msgTypes[117] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5224,7 +5396,7 @@ func (x *ResolveProductShippingEstimateArgs) ProtoReflect() protoreflect.Message // Deprecated: Use ResolveProductShippingEstimateArgs.ProtoReflect.Descriptor instead. func (*ResolveProductShippingEstimateArgs) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{113} + return file_product_proto_rawDescGZIP(), []int{117} } func (x *ResolveProductShippingEstimateArgs) GetInput() *ShippingEstimateInput { @@ -5244,7 +5416,7 @@ type ResolveProductShippingEstimateContext struct { func (x *ResolveProductShippingEstimateContext) Reset() { *x = ResolveProductShippingEstimateContext{} - mi := &file_product_proto_msgTypes[114] + mi := &file_product_proto_msgTypes[118] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5256,7 +5428,7 @@ func (x *ResolveProductShippingEstimateContext) String() string { func (*ResolveProductShippingEstimateContext) ProtoMessage() {} func (x *ResolveProductShippingEstimateContext) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[114] + mi := &file_product_proto_msgTypes[118] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5269,7 +5441,7 @@ func (x *ResolveProductShippingEstimateContext) ProtoReflect() protoreflect.Mess // Deprecated: Use ResolveProductShippingEstimateContext.ProtoReflect.Descriptor instead. func (*ResolveProductShippingEstimateContext) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{114} + return file_product_proto_rawDescGZIP(), []int{118} } func (x *ResolveProductShippingEstimateContext) GetId() string { @@ -5298,7 +5470,7 @@ type ResolveProductShippingEstimateRequest struct { func (x *ResolveProductShippingEstimateRequest) Reset() { *x = ResolveProductShippingEstimateRequest{} - mi := &file_product_proto_msgTypes[115] + mi := &file_product_proto_msgTypes[119] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5310,7 +5482,7 @@ func (x *ResolveProductShippingEstimateRequest) String() string { func (*ResolveProductShippingEstimateRequest) ProtoMessage() {} func (x *ResolveProductShippingEstimateRequest) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[115] + mi := &file_product_proto_msgTypes[119] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5323,7 +5495,7 @@ func (x *ResolveProductShippingEstimateRequest) ProtoReflect() protoreflect.Mess // Deprecated: Use ResolveProductShippingEstimateRequest.ProtoReflect.Descriptor instead. func (*ResolveProductShippingEstimateRequest) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{115} + return file_product_proto_rawDescGZIP(), []int{119} } func (x *ResolveProductShippingEstimateRequest) GetContext() []*ResolveProductShippingEstimateContext { @@ -5349,7 +5521,7 @@ type ResolveProductShippingEstimateResult struct { func (x *ResolveProductShippingEstimateResult) Reset() { *x = ResolveProductShippingEstimateResult{} - mi := &file_product_proto_msgTypes[116] + mi := &file_product_proto_msgTypes[120] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5361,7 +5533,7 @@ func (x *ResolveProductShippingEstimateResult) String() string { func (*ResolveProductShippingEstimateResult) ProtoMessage() {} func (x *ResolveProductShippingEstimateResult) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[116] + mi := &file_product_proto_msgTypes[120] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5374,7 +5546,7 @@ func (x *ResolveProductShippingEstimateResult) ProtoReflect() protoreflect.Messa // Deprecated: Use ResolveProductShippingEstimateResult.ProtoReflect.Descriptor instead. func (*ResolveProductShippingEstimateResult) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{116} + return file_product_proto_rawDescGZIP(), []int{120} } func (x *ResolveProductShippingEstimateResult) GetShippingEstimate() float64 { @@ -5393,7 +5565,7 @@ type ResolveProductShippingEstimateResponse struct { func (x *ResolveProductShippingEstimateResponse) Reset() { *x = ResolveProductShippingEstimateResponse{} - mi := &file_product_proto_msgTypes[117] + mi := &file_product_proto_msgTypes[121] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5405,7 +5577,7 @@ func (x *ResolveProductShippingEstimateResponse) String() string { func (*ResolveProductShippingEstimateResponse) ProtoMessage() {} func (x *ResolveProductShippingEstimateResponse) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[117] + mi := &file_product_proto_msgTypes[121] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5418,7 +5590,7 @@ func (x *ResolveProductShippingEstimateResponse) ProtoReflect() protoreflect.Mes // Deprecated: Use ResolveProductShippingEstimateResponse.ProtoReflect.Descriptor instead. func (*ResolveProductShippingEstimateResponse) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{117} + return file_product_proto_rawDescGZIP(), []int{121} } func (x *ResolveProductShippingEstimateResponse) GetResult() []*ResolveProductShippingEstimateResult { @@ -5437,7 +5609,7 @@ type ResolveProductRecommendedCategoryArgs struct { func (x *ResolveProductRecommendedCategoryArgs) Reset() { *x = ResolveProductRecommendedCategoryArgs{} - mi := &file_product_proto_msgTypes[118] + mi := &file_product_proto_msgTypes[122] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5449,7 +5621,7 @@ func (x *ResolveProductRecommendedCategoryArgs) String() string { func (*ResolveProductRecommendedCategoryArgs) ProtoMessage() {} func (x *ResolveProductRecommendedCategoryArgs) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[118] + mi := &file_product_proto_msgTypes[122] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5462,7 +5634,7 @@ func (x *ResolveProductRecommendedCategoryArgs) ProtoReflect() protoreflect.Mess // Deprecated: Use ResolveProductRecommendedCategoryArgs.ProtoReflect.Descriptor instead. func (*ResolveProductRecommendedCategoryArgs) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{118} + return file_product_proto_rawDescGZIP(), []int{122} } func (x *ResolveProductRecommendedCategoryArgs) GetMaxPrice() int32 { @@ -5483,7 +5655,7 @@ type ResolveProductRecommendedCategoryContext struct { func (x *ResolveProductRecommendedCategoryContext) Reset() { *x = ResolveProductRecommendedCategoryContext{} - mi := &file_product_proto_msgTypes[119] + mi := &file_product_proto_msgTypes[123] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5495,7 +5667,7 @@ func (x *ResolveProductRecommendedCategoryContext) String() string { func (*ResolveProductRecommendedCategoryContext) ProtoMessage() {} func (x *ResolveProductRecommendedCategoryContext) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[119] + mi := &file_product_proto_msgTypes[123] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5508,7 +5680,7 @@ func (x *ResolveProductRecommendedCategoryContext) ProtoReflect() protoreflect.M // Deprecated: Use ResolveProductRecommendedCategoryContext.ProtoReflect.Descriptor instead. func (*ResolveProductRecommendedCategoryContext) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{119} + return file_product_proto_rawDescGZIP(), []int{123} } func (x *ResolveProductRecommendedCategoryContext) GetId() string { @@ -5544,7 +5716,7 @@ type ResolveProductRecommendedCategoryRequest struct { func (x *ResolveProductRecommendedCategoryRequest) Reset() { *x = ResolveProductRecommendedCategoryRequest{} - mi := &file_product_proto_msgTypes[120] + mi := &file_product_proto_msgTypes[124] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5556,7 +5728,7 @@ func (x *ResolveProductRecommendedCategoryRequest) String() string { func (*ResolveProductRecommendedCategoryRequest) ProtoMessage() {} func (x *ResolveProductRecommendedCategoryRequest) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[120] + mi := &file_product_proto_msgTypes[124] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5569,7 +5741,7 @@ func (x *ResolveProductRecommendedCategoryRequest) ProtoReflect() protoreflect.M // Deprecated: Use ResolveProductRecommendedCategoryRequest.ProtoReflect.Descriptor instead. func (*ResolveProductRecommendedCategoryRequest) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{120} + return file_product_proto_rawDescGZIP(), []int{124} } func (x *ResolveProductRecommendedCategoryRequest) GetContext() []*ResolveProductRecommendedCategoryContext { @@ -5595,7 +5767,7 @@ type ResolveProductRecommendedCategoryResult struct { func (x *ResolveProductRecommendedCategoryResult) Reset() { *x = ResolveProductRecommendedCategoryResult{} - mi := &file_product_proto_msgTypes[121] + mi := &file_product_proto_msgTypes[125] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5607,7 +5779,7 @@ func (x *ResolveProductRecommendedCategoryResult) String() string { func (*ResolveProductRecommendedCategoryResult) ProtoMessage() {} func (x *ResolveProductRecommendedCategoryResult) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[121] + mi := &file_product_proto_msgTypes[125] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5620,7 +5792,7 @@ func (x *ResolveProductRecommendedCategoryResult) ProtoReflect() protoreflect.Me // Deprecated: Use ResolveProductRecommendedCategoryResult.ProtoReflect.Descriptor instead. func (*ResolveProductRecommendedCategoryResult) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{121} + return file_product_proto_rawDescGZIP(), []int{125} } func (x *ResolveProductRecommendedCategoryResult) GetRecommendedCategory() *Category { @@ -5639,7 +5811,7 @@ type ResolveProductRecommendedCategoryResponse struct { func (x *ResolveProductRecommendedCategoryResponse) Reset() { *x = ResolveProductRecommendedCategoryResponse{} - mi := &file_product_proto_msgTypes[122] + mi := &file_product_proto_msgTypes[126] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5651,7 +5823,7 @@ func (x *ResolveProductRecommendedCategoryResponse) String() string { func (*ResolveProductRecommendedCategoryResponse) ProtoMessage() {} func (x *ResolveProductRecommendedCategoryResponse) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[122] + mi := &file_product_proto_msgTypes[126] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5664,7 +5836,7 @@ func (x *ResolveProductRecommendedCategoryResponse) ProtoReflect() protoreflect. // Deprecated: Use ResolveProductRecommendedCategoryResponse.ProtoReflect.Descriptor instead. func (*ResolveProductRecommendedCategoryResponse) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{122} + return file_product_proto_rawDescGZIP(), []int{126} } func (x *ResolveProductRecommendedCategoryResponse) GetResult() []*ResolveProductRecommendedCategoryResult { @@ -5683,7 +5855,7 @@ type ResolveCategoryProductCountArgs struct { func (x *ResolveCategoryProductCountArgs) Reset() { *x = ResolveCategoryProductCountArgs{} - mi := &file_product_proto_msgTypes[123] + mi := &file_product_proto_msgTypes[127] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5695,7 +5867,7 @@ func (x *ResolveCategoryProductCountArgs) String() string { func (*ResolveCategoryProductCountArgs) ProtoMessage() {} func (x *ResolveCategoryProductCountArgs) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[123] + mi := &file_product_proto_msgTypes[127] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5708,7 +5880,7 @@ func (x *ResolveCategoryProductCountArgs) ProtoReflect() protoreflect.Message { // Deprecated: Use ResolveCategoryProductCountArgs.ProtoReflect.Descriptor instead. func (*ResolveCategoryProductCountArgs) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{123} + return file_product_proto_rawDescGZIP(), []int{127} } func (x *ResolveCategoryProductCountArgs) GetFilters() *ProductCountFilter { @@ -5728,7 +5900,7 @@ type ResolveCategoryProductCountContext struct { func (x *ResolveCategoryProductCountContext) Reset() { *x = ResolveCategoryProductCountContext{} - mi := &file_product_proto_msgTypes[124] + mi := &file_product_proto_msgTypes[128] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5740,7 +5912,7 @@ func (x *ResolveCategoryProductCountContext) String() string { func (*ResolveCategoryProductCountContext) ProtoMessage() {} func (x *ResolveCategoryProductCountContext) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[124] + mi := &file_product_proto_msgTypes[128] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5753,7 +5925,7 @@ func (x *ResolveCategoryProductCountContext) ProtoReflect() protoreflect.Message // Deprecated: Use ResolveCategoryProductCountContext.ProtoReflect.Descriptor instead. func (*ResolveCategoryProductCountContext) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{124} + return file_product_proto_rawDescGZIP(), []int{128} } func (x *ResolveCategoryProductCountContext) GetId() string { @@ -5782,7 +5954,7 @@ type ResolveCategoryProductCountRequest struct { func (x *ResolveCategoryProductCountRequest) Reset() { *x = ResolveCategoryProductCountRequest{} - mi := &file_product_proto_msgTypes[125] + mi := &file_product_proto_msgTypes[129] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5794,7 +5966,7 @@ func (x *ResolveCategoryProductCountRequest) String() string { func (*ResolveCategoryProductCountRequest) ProtoMessage() {} func (x *ResolveCategoryProductCountRequest) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[125] + mi := &file_product_proto_msgTypes[129] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5807,7 +5979,7 @@ func (x *ResolveCategoryProductCountRequest) ProtoReflect() protoreflect.Message // Deprecated: Use ResolveCategoryProductCountRequest.ProtoReflect.Descriptor instead. func (*ResolveCategoryProductCountRequest) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{125} + return file_product_proto_rawDescGZIP(), []int{129} } func (x *ResolveCategoryProductCountRequest) GetContext() []*ResolveCategoryProductCountContext { @@ -5833,7 +6005,7 @@ type ResolveCategoryProductCountResult struct { func (x *ResolveCategoryProductCountResult) Reset() { *x = ResolveCategoryProductCountResult{} - mi := &file_product_proto_msgTypes[126] + mi := &file_product_proto_msgTypes[130] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5845,7 +6017,7 @@ func (x *ResolveCategoryProductCountResult) String() string { func (*ResolveCategoryProductCountResult) ProtoMessage() {} func (x *ResolveCategoryProductCountResult) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[126] + mi := &file_product_proto_msgTypes[130] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5858,7 +6030,7 @@ func (x *ResolveCategoryProductCountResult) ProtoReflect() protoreflect.Message // Deprecated: Use ResolveCategoryProductCountResult.ProtoReflect.Descriptor instead. func (*ResolveCategoryProductCountResult) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{126} + return file_product_proto_rawDescGZIP(), []int{130} } func (x *ResolveCategoryProductCountResult) GetProductCount() int32 { @@ -5877,7 +6049,7 @@ type ResolveCategoryProductCountResponse struct { func (x *ResolveCategoryProductCountResponse) Reset() { *x = ResolveCategoryProductCountResponse{} - mi := &file_product_proto_msgTypes[127] + mi := &file_product_proto_msgTypes[131] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5889,7 +6061,7 @@ func (x *ResolveCategoryProductCountResponse) String() string { func (*ResolveCategoryProductCountResponse) ProtoMessage() {} func (x *ResolveCategoryProductCountResponse) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[127] + mi := &file_product_proto_msgTypes[131] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5902,7 +6074,7 @@ func (x *ResolveCategoryProductCountResponse) ProtoReflect() protoreflect.Messag // Deprecated: Use ResolveCategoryProductCountResponse.ProtoReflect.Descriptor instead. func (*ResolveCategoryProductCountResponse) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{127} + return file_product_proto_rawDescGZIP(), []int{131} } func (x *ResolveCategoryProductCountResponse) GetResult() []*ResolveCategoryProductCountResult { @@ -5921,7 +6093,7 @@ type ResolveCategoryPopularityScoreArgs struct { func (x *ResolveCategoryPopularityScoreArgs) Reset() { *x = ResolveCategoryPopularityScoreArgs{} - mi := &file_product_proto_msgTypes[128] + mi := &file_product_proto_msgTypes[132] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5933,7 +6105,7 @@ func (x *ResolveCategoryPopularityScoreArgs) String() string { func (*ResolveCategoryPopularityScoreArgs) ProtoMessage() {} func (x *ResolveCategoryPopularityScoreArgs) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[128] + mi := &file_product_proto_msgTypes[132] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5946,7 +6118,7 @@ func (x *ResolveCategoryPopularityScoreArgs) ProtoReflect() protoreflect.Message // Deprecated: Use ResolveCategoryPopularityScoreArgs.ProtoReflect.Descriptor instead. func (*ResolveCategoryPopularityScoreArgs) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{128} + return file_product_proto_rawDescGZIP(), []int{132} } func (x *ResolveCategoryPopularityScoreArgs) GetThreshold() *wrapperspb.Int32Value { @@ -5965,7 +6137,7 @@ type ResolveCategoryPopularityScoreContext struct { func (x *ResolveCategoryPopularityScoreContext) Reset() { *x = ResolveCategoryPopularityScoreContext{} - mi := &file_product_proto_msgTypes[129] + mi := &file_product_proto_msgTypes[133] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5977,7 +6149,7 @@ func (x *ResolveCategoryPopularityScoreContext) String() string { func (*ResolveCategoryPopularityScoreContext) ProtoMessage() {} func (x *ResolveCategoryPopularityScoreContext) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[129] + mi := &file_product_proto_msgTypes[133] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5990,7 +6162,7 @@ func (x *ResolveCategoryPopularityScoreContext) ProtoReflect() protoreflect.Mess // Deprecated: Use ResolveCategoryPopularityScoreContext.ProtoReflect.Descriptor instead. func (*ResolveCategoryPopularityScoreContext) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{129} + return file_product_proto_rawDescGZIP(), []int{133} } func (x *ResolveCategoryPopularityScoreContext) GetId() string { @@ -6012,7 +6184,7 @@ type ResolveCategoryPopularityScoreRequest struct { func (x *ResolveCategoryPopularityScoreRequest) Reset() { *x = ResolveCategoryPopularityScoreRequest{} - mi := &file_product_proto_msgTypes[130] + mi := &file_product_proto_msgTypes[134] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6024,7 +6196,7 @@ func (x *ResolveCategoryPopularityScoreRequest) String() string { func (*ResolveCategoryPopularityScoreRequest) ProtoMessage() {} func (x *ResolveCategoryPopularityScoreRequest) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[130] + mi := &file_product_proto_msgTypes[134] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6037,7 +6209,7 @@ func (x *ResolveCategoryPopularityScoreRequest) ProtoReflect() protoreflect.Mess // Deprecated: Use ResolveCategoryPopularityScoreRequest.ProtoReflect.Descriptor instead. func (*ResolveCategoryPopularityScoreRequest) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{130} + return file_product_proto_rawDescGZIP(), []int{134} } func (x *ResolveCategoryPopularityScoreRequest) GetContext() []*ResolveCategoryPopularityScoreContext { @@ -6063,7 +6235,7 @@ type ResolveCategoryPopularityScoreResult struct { func (x *ResolveCategoryPopularityScoreResult) Reset() { *x = ResolveCategoryPopularityScoreResult{} - mi := &file_product_proto_msgTypes[131] + mi := &file_product_proto_msgTypes[135] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6075,7 +6247,7 @@ func (x *ResolveCategoryPopularityScoreResult) String() string { func (*ResolveCategoryPopularityScoreResult) ProtoMessage() {} func (x *ResolveCategoryPopularityScoreResult) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[131] + mi := &file_product_proto_msgTypes[135] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6088,7 +6260,7 @@ func (x *ResolveCategoryPopularityScoreResult) ProtoReflect() protoreflect.Messa // Deprecated: Use ResolveCategoryPopularityScoreResult.ProtoReflect.Descriptor instead. func (*ResolveCategoryPopularityScoreResult) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{131} + return file_product_proto_rawDescGZIP(), []int{135} } func (x *ResolveCategoryPopularityScoreResult) GetPopularityScore() *wrapperspb.Int32Value { @@ -6107,7 +6279,7 @@ type ResolveCategoryPopularityScoreResponse struct { func (x *ResolveCategoryPopularityScoreResponse) Reset() { *x = ResolveCategoryPopularityScoreResponse{} - mi := &file_product_proto_msgTypes[132] + mi := &file_product_proto_msgTypes[136] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6119,7 +6291,7 @@ func (x *ResolveCategoryPopularityScoreResponse) String() string { func (*ResolveCategoryPopularityScoreResponse) ProtoMessage() {} func (x *ResolveCategoryPopularityScoreResponse) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[132] + mi := &file_product_proto_msgTypes[136] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6132,7 +6304,7 @@ func (x *ResolveCategoryPopularityScoreResponse) ProtoReflect() protoreflect.Mes // Deprecated: Use ResolveCategoryPopularityScoreResponse.ProtoReflect.Descriptor instead. func (*ResolveCategoryPopularityScoreResponse) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{132} + return file_product_proto_rawDescGZIP(), []int{136} } func (x *ResolveCategoryPopularityScoreResponse) GetResult() []*ResolveCategoryPopularityScoreResult { @@ -6151,7 +6323,7 @@ type ResolveCategoryCategoryMetricsArgs struct { func (x *ResolveCategoryCategoryMetricsArgs) Reset() { *x = ResolveCategoryCategoryMetricsArgs{} - mi := &file_product_proto_msgTypes[133] + mi := &file_product_proto_msgTypes[137] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6163,7 +6335,7 @@ func (x *ResolveCategoryCategoryMetricsArgs) String() string { func (*ResolveCategoryCategoryMetricsArgs) ProtoMessage() {} func (x *ResolveCategoryCategoryMetricsArgs) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[133] + mi := &file_product_proto_msgTypes[137] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6176,7 +6348,7 @@ func (x *ResolveCategoryCategoryMetricsArgs) ProtoReflect() protoreflect.Message // Deprecated: Use ResolveCategoryCategoryMetricsArgs.ProtoReflect.Descriptor instead. func (*ResolveCategoryCategoryMetricsArgs) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{133} + return file_product_proto_rawDescGZIP(), []int{137} } func (x *ResolveCategoryCategoryMetricsArgs) GetMetricType() string { @@ -6196,7 +6368,7 @@ type ResolveCategoryCategoryMetricsContext struct { func (x *ResolveCategoryCategoryMetricsContext) Reset() { *x = ResolveCategoryCategoryMetricsContext{} - mi := &file_product_proto_msgTypes[134] + mi := &file_product_proto_msgTypes[138] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6208,7 +6380,7 @@ func (x *ResolveCategoryCategoryMetricsContext) String() string { func (*ResolveCategoryCategoryMetricsContext) ProtoMessage() {} func (x *ResolveCategoryCategoryMetricsContext) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[134] + mi := &file_product_proto_msgTypes[138] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6221,7 +6393,7 @@ func (x *ResolveCategoryCategoryMetricsContext) ProtoReflect() protoreflect.Mess // Deprecated: Use ResolveCategoryCategoryMetricsContext.ProtoReflect.Descriptor instead. func (*ResolveCategoryCategoryMetricsContext) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{134} + return file_product_proto_rawDescGZIP(), []int{138} } func (x *ResolveCategoryCategoryMetricsContext) GetId() string { @@ -6250,7 +6422,7 @@ type ResolveCategoryCategoryMetricsRequest struct { func (x *ResolveCategoryCategoryMetricsRequest) Reset() { *x = ResolveCategoryCategoryMetricsRequest{} - mi := &file_product_proto_msgTypes[135] + mi := &file_product_proto_msgTypes[139] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6262,7 +6434,7 @@ func (x *ResolveCategoryCategoryMetricsRequest) String() string { func (*ResolveCategoryCategoryMetricsRequest) ProtoMessage() {} func (x *ResolveCategoryCategoryMetricsRequest) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[135] + mi := &file_product_proto_msgTypes[139] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6275,7 +6447,7 @@ func (x *ResolveCategoryCategoryMetricsRequest) ProtoReflect() protoreflect.Mess // Deprecated: Use ResolveCategoryCategoryMetricsRequest.ProtoReflect.Descriptor instead. func (*ResolveCategoryCategoryMetricsRequest) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{135} + return file_product_proto_rawDescGZIP(), []int{139} } func (x *ResolveCategoryCategoryMetricsRequest) GetContext() []*ResolveCategoryCategoryMetricsContext { @@ -6301,7 +6473,7 @@ type ResolveCategoryCategoryMetricsResult struct { func (x *ResolveCategoryCategoryMetricsResult) Reset() { *x = ResolveCategoryCategoryMetricsResult{} - mi := &file_product_proto_msgTypes[136] + mi := &file_product_proto_msgTypes[140] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6313,7 +6485,7 @@ func (x *ResolveCategoryCategoryMetricsResult) String() string { func (*ResolveCategoryCategoryMetricsResult) ProtoMessage() {} func (x *ResolveCategoryCategoryMetricsResult) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[136] + mi := &file_product_proto_msgTypes[140] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6326,7 +6498,7 @@ func (x *ResolveCategoryCategoryMetricsResult) ProtoReflect() protoreflect.Messa // Deprecated: Use ResolveCategoryCategoryMetricsResult.ProtoReflect.Descriptor instead. func (*ResolveCategoryCategoryMetricsResult) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{136} + return file_product_proto_rawDescGZIP(), []int{140} } func (x *ResolveCategoryCategoryMetricsResult) GetCategoryMetrics() *CategoryMetrics { @@ -6345,7 +6517,7 @@ type ResolveCategoryCategoryMetricsResponse struct { func (x *ResolveCategoryCategoryMetricsResponse) Reset() { *x = ResolveCategoryCategoryMetricsResponse{} - mi := &file_product_proto_msgTypes[137] + mi := &file_product_proto_msgTypes[141] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6357,7 +6529,7 @@ func (x *ResolveCategoryCategoryMetricsResponse) String() string { func (*ResolveCategoryCategoryMetricsResponse) ProtoMessage() {} func (x *ResolveCategoryCategoryMetricsResponse) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[137] + mi := &file_product_proto_msgTypes[141] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6370,7 +6542,7 @@ func (x *ResolveCategoryCategoryMetricsResponse) ProtoReflect() protoreflect.Mes // Deprecated: Use ResolveCategoryCategoryMetricsResponse.ProtoReflect.Descriptor instead. func (*ResolveCategoryCategoryMetricsResponse) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{137} + return file_product_proto_rawDescGZIP(), []int{141} } func (x *ResolveCategoryCategoryMetricsResponse) GetResult() []*ResolveCategoryCategoryMetricsResult { @@ -6389,7 +6561,7 @@ type ResolveCategoryMascotArgs struct { func (x *ResolveCategoryMascotArgs) Reset() { *x = ResolveCategoryMascotArgs{} - mi := &file_product_proto_msgTypes[138] + mi := &file_product_proto_msgTypes[142] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6401,7 +6573,7 @@ func (x *ResolveCategoryMascotArgs) String() string { func (*ResolveCategoryMascotArgs) ProtoMessage() {} func (x *ResolveCategoryMascotArgs) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[138] + mi := &file_product_proto_msgTypes[142] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6414,7 +6586,7 @@ func (x *ResolveCategoryMascotArgs) ProtoReflect() protoreflect.Message { // Deprecated: Use ResolveCategoryMascotArgs.ProtoReflect.Descriptor instead. func (*ResolveCategoryMascotArgs) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{138} + return file_product_proto_rawDescGZIP(), []int{142} } func (x *ResolveCategoryMascotArgs) GetIncludeVolume() bool { @@ -6434,7 +6606,7 @@ type ResolveCategoryMascotContext struct { func (x *ResolveCategoryMascotContext) Reset() { *x = ResolveCategoryMascotContext{} - mi := &file_product_proto_msgTypes[139] + mi := &file_product_proto_msgTypes[143] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6446,7 +6618,7 @@ func (x *ResolveCategoryMascotContext) String() string { func (*ResolveCategoryMascotContext) ProtoMessage() {} func (x *ResolveCategoryMascotContext) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[139] + mi := &file_product_proto_msgTypes[143] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6459,7 +6631,7 @@ func (x *ResolveCategoryMascotContext) ProtoReflect() protoreflect.Message { // Deprecated: Use ResolveCategoryMascotContext.ProtoReflect.Descriptor instead. func (*ResolveCategoryMascotContext) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{139} + return file_product_proto_rawDescGZIP(), []int{143} } func (x *ResolveCategoryMascotContext) GetId() string { @@ -6488,7 +6660,7 @@ type ResolveCategoryMascotRequest struct { func (x *ResolveCategoryMascotRequest) Reset() { *x = ResolveCategoryMascotRequest{} - mi := &file_product_proto_msgTypes[140] + mi := &file_product_proto_msgTypes[144] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6500,7 +6672,7 @@ func (x *ResolveCategoryMascotRequest) String() string { func (*ResolveCategoryMascotRequest) ProtoMessage() {} func (x *ResolveCategoryMascotRequest) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[140] + mi := &file_product_proto_msgTypes[144] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6513,7 +6685,7 @@ func (x *ResolveCategoryMascotRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use ResolveCategoryMascotRequest.ProtoReflect.Descriptor instead. func (*ResolveCategoryMascotRequest) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{140} + return file_product_proto_rawDescGZIP(), []int{144} } func (x *ResolveCategoryMascotRequest) GetContext() []*ResolveCategoryMascotContext { @@ -6539,7 +6711,7 @@ type ResolveCategoryMascotResult struct { func (x *ResolveCategoryMascotResult) Reset() { *x = ResolveCategoryMascotResult{} - mi := &file_product_proto_msgTypes[141] + mi := &file_product_proto_msgTypes[145] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6551,7 +6723,7 @@ func (x *ResolveCategoryMascotResult) String() string { func (*ResolveCategoryMascotResult) ProtoMessage() {} func (x *ResolveCategoryMascotResult) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[141] + mi := &file_product_proto_msgTypes[145] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6564,7 +6736,7 @@ func (x *ResolveCategoryMascotResult) ProtoReflect() protoreflect.Message { // Deprecated: Use ResolveCategoryMascotResult.ProtoReflect.Descriptor instead. func (*ResolveCategoryMascotResult) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{141} + return file_product_proto_rawDescGZIP(), []int{145} } func (x *ResolveCategoryMascotResult) GetMascot() *Animal { @@ -6583,7 +6755,7 @@ type ResolveCategoryMascotResponse struct { func (x *ResolveCategoryMascotResponse) Reset() { *x = ResolveCategoryMascotResponse{} - mi := &file_product_proto_msgTypes[142] + mi := &file_product_proto_msgTypes[146] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6595,7 +6767,7 @@ func (x *ResolveCategoryMascotResponse) String() string { func (*ResolveCategoryMascotResponse) ProtoMessage() {} func (x *ResolveCategoryMascotResponse) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[142] + mi := &file_product_proto_msgTypes[146] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6608,7 +6780,7 @@ func (x *ResolveCategoryMascotResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use ResolveCategoryMascotResponse.ProtoReflect.Descriptor instead. func (*ResolveCategoryMascotResponse) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{142} + return file_product_proto_rawDescGZIP(), []int{146} } func (x *ResolveCategoryMascotResponse) GetResult() []*ResolveCategoryMascotResult { @@ -6627,7 +6799,7 @@ type ResolveCategoryCategoryStatusArgs struct { func (x *ResolveCategoryCategoryStatusArgs) Reset() { *x = ResolveCategoryCategoryStatusArgs{} - mi := &file_product_proto_msgTypes[143] + mi := &file_product_proto_msgTypes[147] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6639,7 +6811,7 @@ func (x *ResolveCategoryCategoryStatusArgs) String() string { func (*ResolveCategoryCategoryStatusArgs) ProtoMessage() {} func (x *ResolveCategoryCategoryStatusArgs) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[143] + mi := &file_product_proto_msgTypes[147] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6652,7 +6824,7 @@ func (x *ResolveCategoryCategoryStatusArgs) ProtoReflect() protoreflect.Message // Deprecated: Use ResolveCategoryCategoryStatusArgs.ProtoReflect.Descriptor instead. func (*ResolveCategoryCategoryStatusArgs) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{143} + return file_product_proto_rawDescGZIP(), []int{147} } func (x *ResolveCategoryCategoryStatusArgs) GetCheckHealth() bool { @@ -6672,7 +6844,7 @@ type ResolveCategoryCategoryStatusContext struct { func (x *ResolveCategoryCategoryStatusContext) Reset() { *x = ResolveCategoryCategoryStatusContext{} - mi := &file_product_proto_msgTypes[144] + mi := &file_product_proto_msgTypes[148] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6684,7 +6856,7 @@ func (x *ResolveCategoryCategoryStatusContext) String() string { func (*ResolveCategoryCategoryStatusContext) ProtoMessage() {} func (x *ResolveCategoryCategoryStatusContext) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[144] + mi := &file_product_proto_msgTypes[148] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6697,7 +6869,7 @@ func (x *ResolveCategoryCategoryStatusContext) ProtoReflect() protoreflect.Messa // Deprecated: Use ResolveCategoryCategoryStatusContext.ProtoReflect.Descriptor instead. func (*ResolveCategoryCategoryStatusContext) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{144} + return file_product_proto_rawDescGZIP(), []int{148} } func (x *ResolveCategoryCategoryStatusContext) GetId() string { @@ -6726,7 +6898,7 @@ type ResolveCategoryCategoryStatusRequest struct { func (x *ResolveCategoryCategoryStatusRequest) Reset() { *x = ResolveCategoryCategoryStatusRequest{} - mi := &file_product_proto_msgTypes[145] + mi := &file_product_proto_msgTypes[149] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6738,7 +6910,7 @@ func (x *ResolveCategoryCategoryStatusRequest) String() string { func (*ResolveCategoryCategoryStatusRequest) ProtoMessage() {} func (x *ResolveCategoryCategoryStatusRequest) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[145] + mi := &file_product_proto_msgTypes[149] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6751,7 +6923,7 @@ func (x *ResolveCategoryCategoryStatusRequest) ProtoReflect() protoreflect.Messa // Deprecated: Use ResolveCategoryCategoryStatusRequest.ProtoReflect.Descriptor instead. func (*ResolveCategoryCategoryStatusRequest) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{145} + return file_product_proto_rawDescGZIP(), []int{149} } func (x *ResolveCategoryCategoryStatusRequest) GetContext() []*ResolveCategoryCategoryStatusContext { @@ -6777,7 +6949,7 @@ type ResolveCategoryCategoryStatusResult struct { func (x *ResolveCategoryCategoryStatusResult) Reset() { *x = ResolveCategoryCategoryStatusResult{} - mi := &file_product_proto_msgTypes[146] + mi := &file_product_proto_msgTypes[150] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6789,7 +6961,7 @@ func (x *ResolveCategoryCategoryStatusResult) String() string { func (*ResolveCategoryCategoryStatusResult) ProtoMessage() {} func (x *ResolveCategoryCategoryStatusResult) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[146] + mi := &file_product_proto_msgTypes[150] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6802,7 +6974,7 @@ func (x *ResolveCategoryCategoryStatusResult) ProtoReflect() protoreflect.Messag // Deprecated: Use ResolveCategoryCategoryStatusResult.ProtoReflect.Descriptor instead. func (*ResolveCategoryCategoryStatusResult) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{146} + return file_product_proto_rawDescGZIP(), []int{150} } func (x *ResolveCategoryCategoryStatusResult) GetCategoryStatus() *ActionResult { @@ -6821,7 +6993,7 @@ type ResolveCategoryCategoryStatusResponse struct { func (x *ResolveCategoryCategoryStatusResponse) Reset() { *x = ResolveCategoryCategoryStatusResponse{} - mi := &file_product_proto_msgTypes[147] + mi := &file_product_proto_msgTypes[151] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6833,7 +7005,7 @@ func (x *ResolveCategoryCategoryStatusResponse) String() string { func (*ResolveCategoryCategoryStatusResponse) ProtoMessage() {} func (x *ResolveCategoryCategoryStatusResponse) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[147] + mi := &file_product_proto_msgTypes[151] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6846,7 +7018,7 @@ func (x *ResolveCategoryCategoryStatusResponse) ProtoReflect() protoreflect.Mess // Deprecated: Use ResolveCategoryCategoryStatusResponse.ProtoReflect.Descriptor instead. func (*ResolveCategoryCategoryStatusResponse) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{147} + return file_product_proto_rawDescGZIP(), []int{151} } func (x *ResolveCategoryCategoryStatusResponse) GetResult() []*ResolveCategoryCategoryStatusResult { @@ -6865,7 +7037,7 @@ type ResolveSubcategoryItemCountArgs struct { func (x *ResolveSubcategoryItemCountArgs) Reset() { *x = ResolveSubcategoryItemCountArgs{} - mi := &file_product_proto_msgTypes[148] + mi := &file_product_proto_msgTypes[152] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6877,7 +7049,7 @@ func (x *ResolveSubcategoryItemCountArgs) String() string { func (*ResolveSubcategoryItemCountArgs) ProtoMessage() {} func (x *ResolveSubcategoryItemCountArgs) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[148] + mi := &file_product_proto_msgTypes[152] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6890,7 +7062,7 @@ func (x *ResolveSubcategoryItemCountArgs) ProtoReflect() protoreflect.Message { // Deprecated: Use ResolveSubcategoryItemCountArgs.ProtoReflect.Descriptor instead. func (*ResolveSubcategoryItemCountArgs) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{148} + return file_product_proto_rawDescGZIP(), []int{152} } func (x *ResolveSubcategoryItemCountArgs) GetFilters() *SubcategoryItemFilter { @@ -6909,7 +7081,7 @@ type ResolveSubcategoryItemCountContext struct { func (x *ResolveSubcategoryItemCountContext) Reset() { *x = ResolveSubcategoryItemCountContext{} - mi := &file_product_proto_msgTypes[149] + mi := &file_product_proto_msgTypes[153] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6921,7 +7093,7 @@ func (x *ResolveSubcategoryItemCountContext) String() string { func (*ResolveSubcategoryItemCountContext) ProtoMessage() {} func (x *ResolveSubcategoryItemCountContext) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[149] + mi := &file_product_proto_msgTypes[153] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6934,7 +7106,7 @@ func (x *ResolveSubcategoryItemCountContext) ProtoReflect() protoreflect.Message // Deprecated: Use ResolveSubcategoryItemCountContext.ProtoReflect.Descriptor instead. func (*ResolveSubcategoryItemCountContext) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{149} + return file_product_proto_rawDescGZIP(), []int{153} } func (x *ResolveSubcategoryItemCountContext) GetId() string { @@ -6956,7 +7128,7 @@ type ResolveSubcategoryItemCountRequest struct { func (x *ResolveSubcategoryItemCountRequest) Reset() { *x = ResolveSubcategoryItemCountRequest{} - mi := &file_product_proto_msgTypes[150] + mi := &file_product_proto_msgTypes[154] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6968,7 +7140,7 @@ func (x *ResolveSubcategoryItemCountRequest) String() string { func (*ResolveSubcategoryItemCountRequest) ProtoMessage() {} func (x *ResolveSubcategoryItemCountRequest) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[150] + mi := &file_product_proto_msgTypes[154] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6981,7 +7153,7 @@ func (x *ResolveSubcategoryItemCountRequest) ProtoReflect() protoreflect.Message // Deprecated: Use ResolveSubcategoryItemCountRequest.ProtoReflect.Descriptor instead. func (*ResolveSubcategoryItemCountRequest) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{150} + return file_product_proto_rawDescGZIP(), []int{154} } func (x *ResolveSubcategoryItemCountRequest) GetContext() []*ResolveSubcategoryItemCountContext { @@ -7007,7 +7179,7 @@ type ResolveSubcategoryItemCountResult struct { func (x *ResolveSubcategoryItemCountResult) Reset() { *x = ResolveSubcategoryItemCountResult{} - mi := &file_product_proto_msgTypes[151] + mi := &file_product_proto_msgTypes[155] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7019,7 +7191,7 @@ func (x *ResolveSubcategoryItemCountResult) String() string { func (*ResolveSubcategoryItemCountResult) ProtoMessage() {} func (x *ResolveSubcategoryItemCountResult) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[151] + mi := &file_product_proto_msgTypes[155] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7032,7 +7204,7 @@ func (x *ResolveSubcategoryItemCountResult) ProtoReflect() protoreflect.Message // Deprecated: Use ResolveSubcategoryItemCountResult.ProtoReflect.Descriptor instead. func (*ResolveSubcategoryItemCountResult) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{151} + return file_product_proto_rawDescGZIP(), []int{155} } func (x *ResolveSubcategoryItemCountResult) GetItemCount() int32 { @@ -7051,7 +7223,7 @@ type ResolveSubcategoryItemCountResponse struct { func (x *ResolveSubcategoryItemCountResponse) Reset() { *x = ResolveSubcategoryItemCountResponse{} - mi := &file_product_proto_msgTypes[152] + mi := &file_product_proto_msgTypes[156] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7063,7 +7235,7 @@ func (x *ResolveSubcategoryItemCountResponse) String() string { func (*ResolveSubcategoryItemCountResponse) ProtoMessage() {} func (x *ResolveSubcategoryItemCountResponse) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[152] + mi := &file_product_proto_msgTypes[156] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7076,7 +7248,7 @@ func (x *ResolveSubcategoryItemCountResponse) ProtoReflect() protoreflect.Messag // Deprecated: Use ResolveSubcategoryItemCountResponse.ProtoReflect.Descriptor instead. func (*ResolveSubcategoryItemCountResponse) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{152} + return file_product_proto_rawDescGZIP(), []int{156} } func (x *ResolveSubcategoryItemCountResponse) GetResult() []*ResolveSubcategoryItemCountResult { @@ -7095,7 +7267,7 @@ type ResolveCategoryMetricsNormalizedScoreArgs struct { func (x *ResolveCategoryMetricsNormalizedScoreArgs) Reset() { *x = ResolveCategoryMetricsNormalizedScoreArgs{} - mi := &file_product_proto_msgTypes[153] + mi := &file_product_proto_msgTypes[157] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7107,7 +7279,7 @@ func (x *ResolveCategoryMetricsNormalizedScoreArgs) String() string { func (*ResolveCategoryMetricsNormalizedScoreArgs) ProtoMessage() {} func (x *ResolveCategoryMetricsNormalizedScoreArgs) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[153] + mi := &file_product_proto_msgTypes[157] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7120,7 +7292,7 @@ func (x *ResolveCategoryMetricsNormalizedScoreArgs) ProtoReflect() protoreflect. // Deprecated: Use ResolveCategoryMetricsNormalizedScoreArgs.ProtoReflect.Descriptor instead. func (*ResolveCategoryMetricsNormalizedScoreArgs) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{153} + return file_product_proto_rawDescGZIP(), []int{157} } func (x *ResolveCategoryMetricsNormalizedScoreArgs) GetBaseline() float64 { @@ -7141,7 +7313,7 @@ type ResolveCategoryMetricsNormalizedScoreContext struct { func (x *ResolveCategoryMetricsNormalizedScoreContext) Reset() { *x = ResolveCategoryMetricsNormalizedScoreContext{} - mi := &file_product_proto_msgTypes[154] + mi := &file_product_proto_msgTypes[158] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7153,7 +7325,7 @@ func (x *ResolveCategoryMetricsNormalizedScoreContext) String() string { func (*ResolveCategoryMetricsNormalizedScoreContext) ProtoMessage() {} func (x *ResolveCategoryMetricsNormalizedScoreContext) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[154] + mi := &file_product_proto_msgTypes[158] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7166,7 +7338,7 @@ func (x *ResolveCategoryMetricsNormalizedScoreContext) ProtoReflect() protorefle // Deprecated: Use ResolveCategoryMetricsNormalizedScoreContext.ProtoReflect.Descriptor instead. func (*ResolveCategoryMetricsNormalizedScoreContext) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{154} + return file_product_proto_rawDescGZIP(), []int{158} } func (x *ResolveCategoryMetricsNormalizedScoreContext) GetId() string { @@ -7185,36 +7357,274 @@ func (x *ResolveCategoryMetricsNormalizedScoreContext) GetMetricType() string { func (x *ResolveCategoryMetricsNormalizedScoreContext) GetValue() float64 { if x != nil { - return x.Value + return x.Value + } + return 0 +} + +type ResolveCategoryMetricsNormalizedScoreRequest struct { + state protoimpl.MessageState `protogen:"open.v1"` + // context provides the resolver context for the field normalizedScore of type CategoryMetrics. + Context []*ResolveCategoryMetricsNormalizedScoreContext `protobuf:"bytes,1,rep,name=context,proto3" json:"context,omitempty"` + // field_args provides the arguments for the resolver field normalizedScore of type CategoryMetrics. + FieldArgs *ResolveCategoryMetricsNormalizedScoreArgs `protobuf:"bytes,2,opt,name=field_args,json=fieldArgs,proto3" json:"field_args,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ResolveCategoryMetricsNormalizedScoreRequest) Reset() { + *x = ResolveCategoryMetricsNormalizedScoreRequest{} + mi := &file_product_proto_msgTypes[159] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ResolveCategoryMetricsNormalizedScoreRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ResolveCategoryMetricsNormalizedScoreRequest) ProtoMessage() {} + +func (x *ResolveCategoryMetricsNormalizedScoreRequest) ProtoReflect() protoreflect.Message { + mi := &file_product_proto_msgTypes[159] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ResolveCategoryMetricsNormalizedScoreRequest.ProtoReflect.Descriptor instead. +func (*ResolveCategoryMetricsNormalizedScoreRequest) Descriptor() ([]byte, []int) { + return file_product_proto_rawDescGZIP(), []int{159} +} + +func (x *ResolveCategoryMetricsNormalizedScoreRequest) GetContext() []*ResolveCategoryMetricsNormalizedScoreContext { + if x != nil { + return x.Context + } + return nil +} + +func (x *ResolveCategoryMetricsNormalizedScoreRequest) GetFieldArgs() *ResolveCategoryMetricsNormalizedScoreArgs { + if x != nil { + return x.FieldArgs + } + return nil +} + +type ResolveCategoryMetricsNormalizedScoreResult struct { + state protoimpl.MessageState `protogen:"open.v1"` + NormalizedScore float64 `protobuf:"fixed64,1,opt,name=normalized_score,json=normalizedScore,proto3" json:"normalized_score,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ResolveCategoryMetricsNormalizedScoreResult) Reset() { + *x = ResolveCategoryMetricsNormalizedScoreResult{} + mi := &file_product_proto_msgTypes[160] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ResolveCategoryMetricsNormalizedScoreResult) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ResolveCategoryMetricsNormalizedScoreResult) ProtoMessage() {} + +func (x *ResolveCategoryMetricsNormalizedScoreResult) ProtoReflect() protoreflect.Message { + mi := &file_product_proto_msgTypes[160] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ResolveCategoryMetricsNormalizedScoreResult.ProtoReflect.Descriptor instead. +func (*ResolveCategoryMetricsNormalizedScoreResult) Descriptor() ([]byte, []int) { + return file_product_proto_rawDescGZIP(), []int{160} +} + +func (x *ResolveCategoryMetricsNormalizedScoreResult) GetNormalizedScore() float64 { + if x != nil { + return x.NormalizedScore + } + return 0 +} + +type ResolveCategoryMetricsNormalizedScoreResponse struct { + state protoimpl.MessageState `protogen:"open.v1"` + Result []*ResolveCategoryMetricsNormalizedScoreResult `protobuf:"bytes,1,rep,name=result,proto3" json:"result,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ResolveCategoryMetricsNormalizedScoreResponse) Reset() { + *x = ResolveCategoryMetricsNormalizedScoreResponse{} + mi := &file_product_proto_msgTypes[161] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ResolveCategoryMetricsNormalizedScoreResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ResolveCategoryMetricsNormalizedScoreResponse) ProtoMessage() {} + +func (x *ResolveCategoryMetricsNormalizedScoreResponse) ProtoReflect() protoreflect.Message { + mi := &file_product_proto_msgTypes[161] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ResolveCategoryMetricsNormalizedScoreResponse.ProtoReflect.Descriptor instead. +func (*ResolveCategoryMetricsNormalizedScoreResponse) Descriptor() ([]byte, []int) { + return file_product_proto_rawDescGZIP(), []int{161} +} + +func (x *ResolveCategoryMetricsNormalizedScoreResponse) GetResult() []*ResolveCategoryMetricsNormalizedScoreResult { + if x != nil { + return x.Result + } + return nil +} + +type ResolveTestContainerDetailsArgs struct { + state protoimpl.MessageState `protogen:"open.v1"` + IncludeExtended bool `protobuf:"varint,1,opt,name=include_extended,json=includeExtended,proto3" json:"include_extended,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ResolveTestContainerDetailsArgs) Reset() { + *x = ResolveTestContainerDetailsArgs{} + mi := &file_product_proto_msgTypes[162] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ResolveTestContainerDetailsArgs) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ResolveTestContainerDetailsArgs) ProtoMessage() {} + +func (x *ResolveTestContainerDetailsArgs) ProtoReflect() protoreflect.Message { + mi := &file_product_proto_msgTypes[162] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ResolveTestContainerDetailsArgs.ProtoReflect.Descriptor instead. +func (*ResolveTestContainerDetailsArgs) Descriptor() ([]byte, []int) { + return file_product_proto_rawDescGZIP(), []int{162} +} + +func (x *ResolveTestContainerDetailsArgs) GetIncludeExtended() bool { + if x != nil { + return x.IncludeExtended + } + return false +} + +type ResolveTestContainerDetailsContext struct { + state protoimpl.MessageState `protogen:"open.v1"` + Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` + Name string `protobuf:"bytes,2,opt,name=name,proto3" json:"name,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ResolveTestContainerDetailsContext) Reset() { + *x = ResolveTestContainerDetailsContext{} + mi := &file_product_proto_msgTypes[163] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ResolveTestContainerDetailsContext) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ResolveTestContainerDetailsContext) ProtoMessage() {} + +func (x *ResolveTestContainerDetailsContext) ProtoReflect() protoreflect.Message { + mi := &file_product_proto_msgTypes[163] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ResolveTestContainerDetailsContext.ProtoReflect.Descriptor instead. +func (*ResolveTestContainerDetailsContext) Descriptor() ([]byte, []int) { + return file_product_proto_rawDescGZIP(), []int{163} +} + +func (x *ResolveTestContainerDetailsContext) GetId() string { + if x != nil { + return x.Id + } + return "" +} + +func (x *ResolveTestContainerDetailsContext) GetName() string { + if x != nil { + return x.Name } - return 0 + return "" } -type ResolveCategoryMetricsNormalizedScoreRequest struct { +type ResolveTestContainerDetailsRequest struct { state protoimpl.MessageState `protogen:"open.v1"` - // context provides the resolver context for the field normalizedScore of type CategoryMetrics. - Context []*ResolveCategoryMetricsNormalizedScoreContext `protobuf:"bytes,1,rep,name=context,proto3" json:"context,omitempty"` - // field_args provides the arguments for the resolver field normalizedScore of type CategoryMetrics. - FieldArgs *ResolveCategoryMetricsNormalizedScoreArgs `protobuf:"bytes,2,opt,name=field_args,json=fieldArgs,proto3" json:"field_args,omitempty"` + // context provides the resolver context for the field details of type TestContainer. + Context []*ResolveTestContainerDetailsContext `protobuf:"bytes,1,rep,name=context,proto3" json:"context,omitempty"` + // field_args provides the arguments for the resolver field details of type TestContainer. + FieldArgs *ResolveTestContainerDetailsArgs `protobuf:"bytes,2,opt,name=field_args,json=fieldArgs,proto3" json:"field_args,omitempty"` unknownFields protoimpl.UnknownFields sizeCache protoimpl.SizeCache } -func (x *ResolveCategoryMetricsNormalizedScoreRequest) Reset() { - *x = ResolveCategoryMetricsNormalizedScoreRequest{} - mi := &file_product_proto_msgTypes[155] +func (x *ResolveTestContainerDetailsRequest) Reset() { + *x = ResolveTestContainerDetailsRequest{} + mi := &file_product_proto_msgTypes[164] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } -func (x *ResolveCategoryMetricsNormalizedScoreRequest) String() string { +func (x *ResolveTestContainerDetailsRequest) String() string { return protoimpl.X.MessageStringOf(x) } -func (*ResolveCategoryMetricsNormalizedScoreRequest) ProtoMessage() {} +func (*ResolveTestContainerDetailsRequest) ProtoMessage() {} -func (x *ResolveCategoryMetricsNormalizedScoreRequest) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[155] +func (x *ResolveTestContainerDetailsRequest) ProtoReflect() protoreflect.Message { + mi := &file_product_proto_msgTypes[164] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7225,47 +7635,47 @@ func (x *ResolveCategoryMetricsNormalizedScoreRequest) ProtoReflect() protorefle return mi.MessageOf(x) } -// Deprecated: Use ResolveCategoryMetricsNormalizedScoreRequest.ProtoReflect.Descriptor instead. -func (*ResolveCategoryMetricsNormalizedScoreRequest) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{155} +// Deprecated: Use ResolveTestContainerDetailsRequest.ProtoReflect.Descriptor instead. +func (*ResolveTestContainerDetailsRequest) Descriptor() ([]byte, []int) { + return file_product_proto_rawDescGZIP(), []int{164} } -func (x *ResolveCategoryMetricsNormalizedScoreRequest) GetContext() []*ResolveCategoryMetricsNormalizedScoreContext { +func (x *ResolveTestContainerDetailsRequest) GetContext() []*ResolveTestContainerDetailsContext { if x != nil { return x.Context } return nil } -func (x *ResolveCategoryMetricsNormalizedScoreRequest) GetFieldArgs() *ResolveCategoryMetricsNormalizedScoreArgs { +func (x *ResolveTestContainerDetailsRequest) GetFieldArgs() *ResolveTestContainerDetailsArgs { if x != nil { return x.FieldArgs } return nil } -type ResolveCategoryMetricsNormalizedScoreResult struct { - state protoimpl.MessageState `protogen:"open.v1"` - NormalizedScore float64 `protobuf:"fixed64,1,opt,name=normalized_score,json=normalizedScore,proto3" json:"normalized_score,omitempty"` - unknownFields protoimpl.UnknownFields - sizeCache protoimpl.SizeCache +type ResolveTestContainerDetailsResult struct { + state protoimpl.MessageState `protogen:"open.v1"` + Details *TestDetails `protobuf:"bytes,1,opt,name=details,proto3" json:"details,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } -func (x *ResolveCategoryMetricsNormalizedScoreResult) Reset() { - *x = ResolveCategoryMetricsNormalizedScoreResult{} - mi := &file_product_proto_msgTypes[156] +func (x *ResolveTestContainerDetailsResult) Reset() { + *x = ResolveTestContainerDetailsResult{} + mi := &file_product_proto_msgTypes[165] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } -func (x *ResolveCategoryMetricsNormalizedScoreResult) String() string { +func (x *ResolveTestContainerDetailsResult) String() string { return protoimpl.X.MessageStringOf(x) } -func (*ResolveCategoryMetricsNormalizedScoreResult) ProtoMessage() {} +func (*ResolveTestContainerDetailsResult) ProtoMessage() {} -func (x *ResolveCategoryMetricsNormalizedScoreResult) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[156] +func (x *ResolveTestContainerDetailsResult) ProtoReflect() protoreflect.Message { + mi := &file_product_proto_msgTypes[165] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7276,40 +7686,40 @@ func (x *ResolveCategoryMetricsNormalizedScoreResult) ProtoReflect() protoreflec return mi.MessageOf(x) } -// Deprecated: Use ResolveCategoryMetricsNormalizedScoreResult.ProtoReflect.Descriptor instead. -func (*ResolveCategoryMetricsNormalizedScoreResult) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{156} +// Deprecated: Use ResolveTestContainerDetailsResult.ProtoReflect.Descriptor instead. +func (*ResolveTestContainerDetailsResult) Descriptor() ([]byte, []int) { + return file_product_proto_rawDescGZIP(), []int{165} } -func (x *ResolveCategoryMetricsNormalizedScoreResult) GetNormalizedScore() float64 { +func (x *ResolveTestContainerDetailsResult) GetDetails() *TestDetails { if x != nil { - return x.NormalizedScore + return x.Details } - return 0 + return nil } -type ResolveCategoryMetricsNormalizedScoreResponse struct { - state protoimpl.MessageState `protogen:"open.v1"` - Result []*ResolveCategoryMetricsNormalizedScoreResult `protobuf:"bytes,1,rep,name=result,proto3" json:"result,omitempty"` +type ResolveTestContainerDetailsResponse struct { + state protoimpl.MessageState `protogen:"open.v1"` + Result []*ResolveTestContainerDetailsResult `protobuf:"bytes,1,rep,name=result,proto3" json:"result,omitempty"` unknownFields protoimpl.UnknownFields sizeCache protoimpl.SizeCache } -func (x *ResolveCategoryMetricsNormalizedScoreResponse) Reset() { - *x = ResolveCategoryMetricsNormalizedScoreResponse{} - mi := &file_product_proto_msgTypes[157] +func (x *ResolveTestContainerDetailsResponse) Reset() { + *x = ResolveTestContainerDetailsResponse{} + mi := &file_product_proto_msgTypes[166] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } -func (x *ResolveCategoryMetricsNormalizedScoreResponse) String() string { +func (x *ResolveTestContainerDetailsResponse) String() string { return protoimpl.X.MessageStringOf(x) } -func (*ResolveCategoryMetricsNormalizedScoreResponse) ProtoMessage() {} +func (*ResolveTestContainerDetailsResponse) ProtoMessage() {} -func (x *ResolveCategoryMetricsNormalizedScoreResponse) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[157] +func (x *ResolveTestContainerDetailsResponse) ProtoReflect() protoreflect.Message { + mi := &file_product_proto_msgTypes[166] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7320,12 +7730,12 @@ func (x *ResolveCategoryMetricsNormalizedScoreResponse) ProtoReflect() protorefl return mi.MessageOf(x) } -// Deprecated: Use ResolveCategoryMetricsNormalizedScoreResponse.ProtoReflect.Descriptor instead. -func (*ResolveCategoryMetricsNormalizedScoreResponse) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{157} +// Deprecated: Use ResolveTestContainerDetailsResponse.ProtoReflect.Descriptor instead. +func (*ResolveTestContainerDetailsResponse) Descriptor() ([]byte, []int) { + return file_product_proto_rawDescGZIP(), []int{166} } -func (x *ResolveCategoryMetricsNormalizedScoreResponse) GetResult() []*ResolveCategoryMetricsNormalizedScoreResult { +func (x *ResolveTestContainerDetailsResponse) GetResult() []*ResolveTestContainerDetailsResult { if x != nil { return x.Result } @@ -7343,7 +7753,7 @@ type Product struct { func (x *Product) Reset() { *x = Product{} - mi := &file_product_proto_msgTypes[158] + mi := &file_product_proto_msgTypes[167] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7355,7 +7765,7 @@ func (x *Product) String() string { func (*Product) ProtoMessage() {} func (x *Product) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[158] + mi := &file_product_proto_msgTypes[167] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7368,7 +7778,7 @@ func (x *Product) ProtoReflect() protoreflect.Message { // Deprecated: Use Product.ProtoReflect.Descriptor instead. func (*Product) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{158} + return file_product_proto_rawDescGZIP(), []int{167} } func (x *Product) GetId() string { @@ -7403,7 +7813,7 @@ type Storage struct { func (x *Storage) Reset() { *x = Storage{} - mi := &file_product_proto_msgTypes[159] + mi := &file_product_proto_msgTypes[168] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7415,7 +7825,7 @@ func (x *Storage) String() string { func (*Storage) ProtoMessage() {} func (x *Storage) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[159] + mi := &file_product_proto_msgTypes[168] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7428,7 +7838,7 @@ func (x *Storage) ProtoReflect() protoreflect.Message { // Deprecated: Use Storage.ProtoReflect.Descriptor instead. func (*Storage) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{159} + return file_product_proto_rawDescGZIP(), []int{168} } func (x *Storage) GetId() string { @@ -7463,7 +7873,7 @@ type Warehouse struct { func (x *Warehouse) Reset() { *x = Warehouse{} - mi := &file_product_proto_msgTypes[160] + mi := &file_product_proto_msgTypes[169] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7475,7 +7885,7 @@ func (x *Warehouse) String() string { func (*Warehouse) ProtoMessage() {} func (x *Warehouse) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[160] + mi := &file_product_proto_msgTypes[169] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7488,7 +7898,7 @@ func (x *Warehouse) ProtoReflect() protoreflect.Message { // Deprecated: Use Warehouse.ProtoReflect.Descriptor instead. func (*Warehouse) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{160} + return file_product_proto_rawDescGZIP(), []int{169} } func (x *Warehouse) GetId() string { @@ -7522,7 +7932,7 @@ type User struct { func (x *User) Reset() { *x = User{} - mi := &file_product_proto_msgTypes[161] + mi := &file_product_proto_msgTypes[170] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7534,7 +7944,7 @@ func (x *User) String() string { func (*User) ProtoMessage() {} func (x *User) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[161] + mi := &file_product_proto_msgTypes[170] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7547,7 +7957,7 @@ func (x *User) ProtoReflect() protoreflect.Message { // Deprecated: Use User.ProtoReflect.Descriptor instead. func (*User) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{161} + return file_product_proto_rawDescGZIP(), []int{170} } func (x *User) GetId() string { @@ -7575,7 +7985,7 @@ type NestedTypeA struct { func (x *NestedTypeA) Reset() { *x = NestedTypeA{} - mi := &file_product_proto_msgTypes[162] + mi := &file_product_proto_msgTypes[171] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7587,7 +7997,7 @@ func (x *NestedTypeA) String() string { func (*NestedTypeA) ProtoMessage() {} func (x *NestedTypeA) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[162] + mi := &file_product_proto_msgTypes[171] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7600,7 +8010,7 @@ func (x *NestedTypeA) ProtoReflect() protoreflect.Message { // Deprecated: Use NestedTypeA.ProtoReflect.Descriptor instead. func (*NestedTypeA) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{162} + return file_product_proto_rawDescGZIP(), []int{171} } func (x *NestedTypeA) GetId() string { @@ -7635,7 +8045,7 @@ type RecursiveType struct { func (x *RecursiveType) Reset() { *x = RecursiveType{} - mi := &file_product_proto_msgTypes[163] + mi := &file_product_proto_msgTypes[172] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7647,7 +8057,7 @@ func (x *RecursiveType) String() string { func (*RecursiveType) ProtoMessage() {} func (x *RecursiveType) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[163] + mi := &file_product_proto_msgTypes[172] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7660,7 +8070,7 @@ func (x *RecursiveType) ProtoReflect() protoreflect.Message { // Deprecated: Use RecursiveType.ProtoReflect.Descriptor instead. func (*RecursiveType) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{163} + return file_product_proto_rawDescGZIP(), []int{172} } func (x *RecursiveType) GetId() string { @@ -7696,7 +8106,7 @@ type TypeWithMultipleFilterFields struct { func (x *TypeWithMultipleFilterFields) Reset() { *x = TypeWithMultipleFilterFields{} - mi := &file_product_proto_msgTypes[164] + mi := &file_product_proto_msgTypes[173] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7708,7 +8118,7 @@ func (x *TypeWithMultipleFilterFields) String() string { func (*TypeWithMultipleFilterFields) ProtoMessage() {} func (x *TypeWithMultipleFilterFields) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[164] + mi := &file_product_proto_msgTypes[173] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7721,7 +8131,7 @@ func (x *TypeWithMultipleFilterFields) ProtoReflect() protoreflect.Message { // Deprecated: Use TypeWithMultipleFilterFields.ProtoReflect.Descriptor instead. func (*TypeWithMultipleFilterFields) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{164} + return file_product_proto_rawDescGZIP(), []int{173} } func (x *TypeWithMultipleFilterFields) GetId() string { @@ -7762,7 +8172,7 @@ type FilterTypeInput struct { func (x *FilterTypeInput) Reset() { *x = FilterTypeInput{} - mi := &file_product_proto_msgTypes[165] + mi := &file_product_proto_msgTypes[174] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7774,7 +8184,7 @@ func (x *FilterTypeInput) String() string { func (*FilterTypeInput) ProtoMessage() {} func (x *FilterTypeInput) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[165] + mi := &file_product_proto_msgTypes[174] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7787,7 +8197,7 @@ func (x *FilterTypeInput) ProtoReflect() protoreflect.Message { // Deprecated: Use FilterTypeInput.ProtoReflect.Descriptor instead. func (*FilterTypeInput) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{165} + return file_product_proto_rawDescGZIP(), []int{174} } func (x *FilterTypeInput) GetFilterField_1() string { @@ -7813,7 +8223,7 @@ type ComplexFilterTypeInput struct { func (x *ComplexFilterTypeInput) Reset() { *x = ComplexFilterTypeInput{} - mi := &file_product_proto_msgTypes[166] + mi := &file_product_proto_msgTypes[175] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7825,7 +8235,7 @@ func (x *ComplexFilterTypeInput) String() string { func (*ComplexFilterTypeInput) ProtoMessage() {} func (x *ComplexFilterTypeInput) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[166] + mi := &file_product_proto_msgTypes[175] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7838,7 +8248,7 @@ func (x *ComplexFilterTypeInput) ProtoReflect() protoreflect.Message { // Deprecated: Use ComplexFilterTypeInput.ProtoReflect.Descriptor instead. func (*ComplexFilterTypeInput) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{166} + return file_product_proto_rawDescGZIP(), []int{175} } func (x *ComplexFilterTypeInput) GetFilter() *FilterType { @@ -7858,7 +8268,7 @@ type TypeWithComplexFilterInput struct { func (x *TypeWithComplexFilterInput) Reset() { *x = TypeWithComplexFilterInput{} - mi := &file_product_proto_msgTypes[167] + mi := &file_product_proto_msgTypes[176] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7870,7 +8280,7 @@ func (x *TypeWithComplexFilterInput) String() string { func (*TypeWithComplexFilterInput) ProtoMessage() {} func (x *TypeWithComplexFilterInput) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[167] + mi := &file_product_proto_msgTypes[176] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7883,7 +8293,7 @@ func (x *TypeWithComplexFilterInput) ProtoReflect() protoreflect.Message { // Deprecated: Use TypeWithComplexFilterInput.ProtoReflect.Descriptor instead. func (*TypeWithComplexFilterInput) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{167} + return file_product_proto_rawDescGZIP(), []int{176} } func (x *TypeWithComplexFilterInput) GetId() string { @@ -7911,7 +8321,7 @@ type OrderInput struct { func (x *OrderInput) Reset() { *x = OrderInput{} - mi := &file_product_proto_msgTypes[168] + mi := &file_product_proto_msgTypes[177] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7923,7 +8333,7 @@ func (x *OrderInput) String() string { func (*OrderInput) ProtoMessage() {} func (x *OrderInput) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[168] + mi := &file_product_proto_msgTypes[177] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7936,7 +8346,7 @@ func (x *OrderInput) ProtoReflect() protoreflect.Message { // Deprecated: Use OrderInput.ProtoReflect.Descriptor instead. func (*OrderInput) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{168} + return file_product_proto_rawDescGZIP(), []int{177} } func (x *OrderInput) GetOrderId() string { @@ -7972,7 +8382,7 @@ type Order struct { func (x *Order) Reset() { *x = Order{} - mi := &file_product_proto_msgTypes[169] + mi := &file_product_proto_msgTypes[178] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7984,7 +8394,7 @@ func (x *Order) String() string { func (*Order) ProtoMessage() {} func (x *Order) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[169] + mi := &file_product_proto_msgTypes[178] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7997,7 +8407,7 @@ func (x *Order) ProtoReflect() protoreflect.Message { // Deprecated: Use Order.ProtoReflect.Descriptor instead. func (*Order) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{169} + return file_product_proto_rawDescGZIP(), []int{178} } func (x *Order) GetOrderId() string { @@ -8040,7 +8450,7 @@ type Category struct { func (x *Category) Reset() { *x = Category{} - mi := &file_product_proto_msgTypes[170] + mi := &file_product_proto_msgTypes[179] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8052,7 +8462,7 @@ func (x *Category) String() string { func (*Category) ProtoMessage() {} func (x *Category) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[170] + mi := &file_product_proto_msgTypes[179] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8065,7 +8475,7 @@ func (x *Category) ProtoReflect() protoreflect.Message { // Deprecated: Use Category.ProtoReflect.Descriptor instead. func (*Category) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{170} + return file_product_proto_rawDescGZIP(), []int{179} } func (x *Category) GetId() string { @@ -8106,7 +8516,7 @@ type CategoryFilter struct { func (x *CategoryFilter) Reset() { *x = CategoryFilter{} - mi := &file_product_proto_msgTypes[171] + mi := &file_product_proto_msgTypes[180] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8118,7 +8528,7 @@ func (x *CategoryFilter) String() string { func (*CategoryFilter) ProtoMessage() {} func (x *CategoryFilter) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[171] + mi := &file_product_proto_msgTypes[180] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8131,7 +8541,7 @@ func (x *CategoryFilter) ProtoReflect() protoreflect.Message { // Deprecated: Use CategoryFilter.ProtoReflect.Descriptor instead. func (*CategoryFilter) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{171} + return file_product_proto_rawDescGZIP(), []int{180} } func (x *CategoryFilter) GetCategory() CategoryKind { @@ -8161,7 +8571,7 @@ type Animal struct { func (x *Animal) Reset() { *x = Animal{} - mi := &file_product_proto_msgTypes[172] + mi := &file_product_proto_msgTypes[181] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8173,7 +8583,7 @@ func (x *Animal) String() string { func (*Animal) ProtoMessage() {} func (x *Animal) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[172] + mi := &file_product_proto_msgTypes[181] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8186,7 +8596,7 @@ func (x *Animal) ProtoReflect() protoreflect.Message { // Deprecated: Use Animal.ProtoReflect.Descriptor instead. func (*Animal) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{172} + return file_product_proto_rawDescGZIP(), []int{181} } func (x *Animal) GetInstance() isAnimal_Instance { @@ -8240,7 +8650,7 @@ type SearchInput struct { func (x *SearchInput) Reset() { *x = SearchInput{} - mi := &file_product_proto_msgTypes[173] + mi := &file_product_proto_msgTypes[182] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8252,7 +8662,7 @@ func (x *SearchInput) String() string { func (*SearchInput) ProtoMessage() {} func (x *SearchInput) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[173] + mi := &file_product_proto_msgTypes[182] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8265,7 +8675,7 @@ func (x *SearchInput) ProtoReflect() protoreflect.Message { // Deprecated: Use SearchInput.ProtoReflect.Descriptor instead. func (*SearchInput) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{173} + return file_product_proto_rawDescGZIP(), []int{182} } func (x *SearchInput) GetQuery() string { @@ -8296,7 +8706,7 @@ type SearchResult struct { func (x *SearchResult) Reset() { *x = SearchResult{} - mi := &file_product_proto_msgTypes[174] + mi := &file_product_proto_msgTypes[183] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8308,7 +8718,7 @@ func (x *SearchResult) String() string { func (*SearchResult) ProtoMessage() {} func (x *SearchResult) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[174] + mi := &file_product_proto_msgTypes[183] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8321,7 +8731,7 @@ func (x *SearchResult) ProtoReflect() protoreflect.Message { // Deprecated: Use SearchResult.ProtoReflect.Descriptor instead. func (*SearchResult) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{174} + return file_product_proto_rawDescGZIP(), []int{183} } func (x *SearchResult) GetValue() isSearchResult_Value { @@ -8396,7 +8806,7 @@ type NullableFieldsType struct { func (x *NullableFieldsType) Reset() { *x = NullableFieldsType{} - mi := &file_product_proto_msgTypes[175] + mi := &file_product_proto_msgTypes[184] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8408,7 +8818,7 @@ func (x *NullableFieldsType) String() string { func (*NullableFieldsType) ProtoMessage() {} func (x *NullableFieldsType) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[175] + mi := &file_product_proto_msgTypes[184] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8421,7 +8831,7 @@ func (x *NullableFieldsType) ProtoReflect() protoreflect.Message { // Deprecated: Use NullableFieldsType.ProtoReflect.Descriptor instead. func (*NullableFieldsType) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{175} + return file_product_proto_rawDescGZIP(), []int{184} } func (x *NullableFieldsType) GetId() string { @@ -8491,7 +8901,7 @@ type NullableFieldsFilter struct { func (x *NullableFieldsFilter) Reset() { *x = NullableFieldsFilter{} - mi := &file_product_proto_msgTypes[176] + mi := &file_product_proto_msgTypes[185] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8503,7 +8913,7 @@ func (x *NullableFieldsFilter) String() string { func (*NullableFieldsFilter) ProtoMessage() {} func (x *NullableFieldsFilter) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[176] + mi := &file_product_proto_msgTypes[185] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8516,7 +8926,7 @@ func (x *NullableFieldsFilter) ProtoReflect() protoreflect.Message { // Deprecated: Use NullableFieldsFilter.ProtoReflect.Descriptor instead. func (*NullableFieldsFilter) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{176} + return file_product_proto_rawDescGZIP(), []int{185} } func (x *NullableFieldsFilter) GetName() *wrapperspb.StringValue { @@ -8568,7 +8978,7 @@ type BlogPost struct { func (x *BlogPost) Reset() { *x = BlogPost{} - mi := &file_product_proto_msgTypes[177] + mi := &file_product_proto_msgTypes[186] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8580,7 +8990,7 @@ func (x *BlogPost) String() string { func (*BlogPost) ProtoMessage() {} func (x *BlogPost) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[177] + mi := &file_product_proto_msgTypes[186] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8593,7 +9003,7 @@ func (x *BlogPost) ProtoReflect() protoreflect.Message { // Deprecated: Use BlogPost.ProtoReflect.Descriptor instead. func (*BlogPost) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{177} + return file_product_proto_rawDescGZIP(), []int{186} } func (x *BlogPost) GetId() string { @@ -8747,7 +9157,7 @@ type BlogPostFilter struct { func (x *BlogPostFilter) Reset() { *x = BlogPostFilter{} - mi := &file_product_proto_msgTypes[178] + mi := &file_product_proto_msgTypes[187] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8759,7 +9169,7 @@ func (x *BlogPostFilter) String() string { func (*BlogPostFilter) ProtoMessage() {} func (x *BlogPostFilter) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[178] + mi := &file_product_proto_msgTypes[187] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8772,7 +9182,7 @@ func (x *BlogPostFilter) ProtoReflect() protoreflect.Message { // Deprecated: Use BlogPostFilter.ProtoReflect.Descriptor instead. func (*BlogPostFilter) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{178} + return file_product_proto_rawDescGZIP(), []int{187} } func (x *BlogPostFilter) GetTitle() *wrapperspb.StringValue { @@ -8819,7 +9229,7 @@ type Author struct { func (x *Author) Reset() { *x = Author{} - mi := &file_product_proto_msgTypes[179] + mi := &file_product_proto_msgTypes[188] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8831,7 +9241,7 @@ func (x *Author) String() string { func (*Author) ProtoMessage() {} func (x *Author) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[179] + mi := &file_product_proto_msgTypes[188] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8844,7 +9254,7 @@ func (x *Author) ProtoReflect() protoreflect.Message { // Deprecated: Use Author.ProtoReflect.Descriptor instead. func (*Author) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{179} + return file_product_proto_rawDescGZIP(), []int{188} } func (x *Author) GetId() string { @@ -8963,7 +9373,7 @@ type AuthorFilter struct { func (x *AuthorFilter) Reset() { *x = AuthorFilter{} - mi := &file_product_proto_msgTypes[180] + mi := &file_product_proto_msgTypes[189] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8974,8 +9384,68 @@ func (x *AuthorFilter) String() string { func (*AuthorFilter) ProtoMessage() {} -func (x *AuthorFilter) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[180] +func (x *AuthorFilter) ProtoReflect() protoreflect.Message { + mi := &file_product_proto_msgTypes[189] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use AuthorFilter.ProtoReflect.Descriptor instead. +func (*AuthorFilter) Descriptor() ([]byte, []int) { + return file_product_proto_rawDescGZIP(), []int{189} +} + +func (x *AuthorFilter) GetName() *wrapperspb.StringValue { + if x != nil { + return x.Name + } + return nil +} + +func (x *AuthorFilter) GetHasTeams() *wrapperspb.BoolValue { + if x != nil { + return x.HasTeams + } + return nil +} + +func (x *AuthorFilter) GetSkillCount() *wrapperspb.Int32Value { + if x != nil { + return x.SkillCount + } + return nil +} + +type TestContainer struct { + state protoimpl.MessageState `protogen:"open.v1"` + Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` + Name string `protobuf:"bytes,2,opt,name=name,proto3" json:"name,omitempty"` + Description *wrapperspb.StringValue `protobuf:"bytes,3,opt,name=description,proto3" json:"description,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *TestContainer) Reset() { + *x = TestContainer{} + mi := &file_product_proto_msgTypes[190] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *TestContainer) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*TestContainer) ProtoMessage() {} + +func (x *TestContainer) ProtoReflect() protoreflect.Message { + mi := &file_product_proto_msgTypes[190] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8986,28 +9456,28 @@ func (x *AuthorFilter) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use AuthorFilter.ProtoReflect.Descriptor instead. -func (*AuthorFilter) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{180} +// Deprecated: Use TestContainer.ProtoReflect.Descriptor instead. +func (*TestContainer) Descriptor() ([]byte, []int) { + return file_product_proto_rawDescGZIP(), []int{190} } -func (x *AuthorFilter) GetName() *wrapperspb.StringValue { +func (x *TestContainer) GetId() string { if x != nil { - return x.Name + return x.Id } - return nil + return "" } -func (x *AuthorFilter) GetHasTeams() *wrapperspb.BoolValue { +func (x *TestContainer) GetName() string { if x != nil { - return x.HasTeams + return x.Name } - return nil + return "" } -func (x *AuthorFilter) GetSkillCount() *wrapperspb.Int32Value { +func (x *TestContainer) GetDescription() *wrapperspb.StringValue { if x != nil { - return x.SkillCount + return x.Description } return nil } @@ -9021,7 +9491,7 @@ type UserInput struct { func (x *UserInput) Reset() { *x = UserInput{} - mi := &file_product_proto_msgTypes[181] + mi := &file_product_proto_msgTypes[191] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9033,7 +9503,7 @@ func (x *UserInput) String() string { func (*UserInput) ProtoMessage() {} func (x *UserInput) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[181] + mi := &file_product_proto_msgTypes[191] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9046,7 +9516,7 @@ func (x *UserInput) ProtoReflect() protoreflect.Message { // Deprecated: Use UserInput.ProtoReflect.Descriptor instead. func (*UserInput) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{181} + return file_product_proto_rawDescGZIP(), []int{191} } func (x *UserInput) GetName() string { @@ -9066,7 +9536,7 @@ type ActionInput struct { func (x *ActionInput) Reset() { *x = ActionInput{} - mi := &file_product_proto_msgTypes[182] + mi := &file_product_proto_msgTypes[192] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9078,7 +9548,7 @@ func (x *ActionInput) String() string { func (*ActionInput) ProtoMessage() {} func (x *ActionInput) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[182] + mi := &file_product_proto_msgTypes[192] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9091,7 +9561,7 @@ func (x *ActionInput) ProtoReflect() protoreflect.Message { // Deprecated: Use ActionInput.ProtoReflect.Descriptor instead. func (*ActionInput) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{182} + return file_product_proto_rawDescGZIP(), []int{192} } func (x *ActionInput) GetType() string { @@ -9121,7 +9591,7 @@ type ActionResult struct { func (x *ActionResult) Reset() { *x = ActionResult{} - mi := &file_product_proto_msgTypes[183] + mi := &file_product_proto_msgTypes[193] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9133,7 +9603,7 @@ func (x *ActionResult) String() string { func (*ActionResult) ProtoMessage() {} func (x *ActionResult) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[183] + mi := &file_product_proto_msgTypes[193] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9146,7 +9616,7 @@ func (x *ActionResult) ProtoReflect() protoreflect.Message { // Deprecated: Use ActionResult.ProtoReflect.Descriptor instead. func (*ActionResult) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{183} + return file_product_proto_rawDescGZIP(), []int{193} } func (x *ActionResult) GetValue() isActionResult_Value { @@ -9205,7 +9675,7 @@ type NullableFieldsInput struct { func (x *NullableFieldsInput) Reset() { *x = NullableFieldsInput{} - mi := &file_product_proto_msgTypes[184] + mi := &file_product_proto_msgTypes[194] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9217,7 +9687,7 @@ func (x *NullableFieldsInput) String() string { func (*NullableFieldsInput) ProtoMessage() {} func (x *NullableFieldsInput) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[184] + mi := &file_product_proto_msgTypes[194] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9230,7 +9700,7 @@ func (x *NullableFieldsInput) ProtoReflect() protoreflect.Message { // Deprecated: Use NullableFieldsInput.ProtoReflect.Descriptor instead. func (*NullableFieldsInput) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{184} + return file_product_proto_rawDescGZIP(), []int{194} } func (x *NullableFieldsInput) GetName() string { @@ -9306,7 +9776,7 @@ type BlogPostInput struct { func (x *BlogPostInput) Reset() { *x = BlogPostInput{} - mi := &file_product_proto_msgTypes[185] + mi := &file_product_proto_msgTypes[195] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9318,7 +9788,7 @@ func (x *BlogPostInput) String() string { func (*BlogPostInput) ProtoMessage() {} func (x *BlogPostInput) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[185] + mi := &file_product_proto_msgTypes[195] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9331,7 +9801,7 @@ func (x *BlogPostInput) ProtoReflect() protoreflect.Message { // Deprecated: Use BlogPostInput.ProtoReflect.Descriptor instead. func (*BlogPostInput) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{185} + return file_product_proto_rawDescGZIP(), []int{195} } func (x *BlogPostInput) GetTitle() string { @@ -9464,7 +9934,7 @@ type AuthorInput struct { func (x *AuthorInput) Reset() { *x = AuthorInput{} - mi := &file_product_proto_msgTypes[186] + mi := &file_product_proto_msgTypes[196] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9476,7 +9946,7 @@ func (x *AuthorInput) String() string { func (*AuthorInput) ProtoMessage() {} func (x *AuthorInput) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[186] + mi := &file_product_proto_msgTypes[196] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9489,7 +9959,7 @@ func (x *AuthorInput) ProtoReflect() protoreflect.Message { // Deprecated: Use AuthorInput.ProtoReflect.Descriptor instead. func (*AuthorInput) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{186} + return file_product_proto_rawDescGZIP(), []int{196} } func (x *AuthorInput) GetName() string { @@ -9573,7 +10043,7 @@ type NestedTypeB struct { func (x *NestedTypeB) Reset() { *x = NestedTypeB{} - mi := &file_product_proto_msgTypes[187] + mi := &file_product_proto_msgTypes[197] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9585,7 +10055,7 @@ func (x *NestedTypeB) String() string { func (*NestedTypeB) ProtoMessage() {} func (x *NestedTypeB) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[187] + mi := &file_product_proto_msgTypes[197] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9598,7 +10068,7 @@ func (x *NestedTypeB) ProtoReflect() protoreflect.Message { // Deprecated: Use NestedTypeB.ProtoReflect.Descriptor instead. func (*NestedTypeB) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{187} + return file_product_proto_rawDescGZIP(), []int{197} } func (x *NestedTypeB) GetId() string { @@ -9632,7 +10102,7 @@ type NestedTypeC struct { func (x *NestedTypeC) Reset() { *x = NestedTypeC{} - mi := &file_product_proto_msgTypes[188] + mi := &file_product_proto_msgTypes[198] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9644,7 +10114,7 @@ func (x *NestedTypeC) String() string { func (*NestedTypeC) ProtoMessage() {} func (x *NestedTypeC) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[188] + mi := &file_product_proto_msgTypes[198] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9657,7 +10127,7 @@ func (x *NestedTypeC) ProtoReflect() protoreflect.Message { // Deprecated: Use NestedTypeC.ProtoReflect.Descriptor instead. func (*NestedTypeC) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{188} + return file_product_proto_rawDescGZIP(), []int{198} } func (x *NestedTypeC) GetId() string { @@ -9686,7 +10156,7 @@ type FilterType struct { func (x *FilterType) Reset() { *x = FilterType{} - mi := &file_product_proto_msgTypes[189] + mi := &file_product_proto_msgTypes[199] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9698,7 +10168,7 @@ func (x *FilterType) String() string { func (*FilterType) ProtoMessage() {} func (x *FilterType) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[189] + mi := &file_product_proto_msgTypes[199] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9711,7 +10181,7 @@ func (x *FilterType) ProtoReflect() protoreflect.Message { // Deprecated: Use FilterType.ProtoReflect.Descriptor instead. func (*FilterType) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{189} + return file_product_proto_rawDescGZIP(), []int{199} } func (x *FilterType) GetName() string { @@ -9752,7 +10222,7 @@ type Pagination struct { func (x *Pagination) Reset() { *x = Pagination{} - mi := &file_product_proto_msgTypes[190] + mi := &file_product_proto_msgTypes[200] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9764,7 +10234,7 @@ func (x *Pagination) String() string { func (*Pagination) ProtoMessage() {} func (x *Pagination) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[190] + mi := &file_product_proto_msgTypes[200] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9777,7 +10247,7 @@ func (x *Pagination) ProtoReflect() protoreflect.Message { // Deprecated: Use Pagination.ProtoReflect.Descriptor instead. func (*Pagination) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{190} + return file_product_proto_rawDescGZIP(), []int{200} } func (x *Pagination) GetPage() int32 { @@ -9805,7 +10275,7 @@ type OrderLineInput struct { func (x *OrderLineInput) Reset() { *x = OrderLineInput{} - mi := &file_product_proto_msgTypes[191] + mi := &file_product_proto_msgTypes[201] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9817,7 +10287,7 @@ func (x *OrderLineInput) String() string { func (*OrderLineInput) ProtoMessage() {} func (x *OrderLineInput) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[191] + mi := &file_product_proto_msgTypes[201] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9830,7 +10300,7 @@ func (x *OrderLineInput) ProtoReflect() protoreflect.Message { // Deprecated: Use OrderLineInput.ProtoReflect.Descriptor instead. func (*OrderLineInput) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{191} + return file_product_proto_rawDescGZIP(), []int{201} } func (x *OrderLineInput) GetProductId() string { @@ -9865,7 +10335,7 @@ type OrderLine struct { func (x *OrderLine) Reset() { *x = OrderLine{} - mi := &file_product_proto_msgTypes[192] + mi := &file_product_proto_msgTypes[202] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9877,7 +10347,7 @@ func (x *OrderLine) String() string { func (*OrderLine) ProtoMessage() {} func (x *OrderLine) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[192] + mi := &file_product_proto_msgTypes[202] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9890,7 +10360,7 @@ func (x *OrderLine) ProtoReflect() protoreflect.Message { // Deprecated: Use OrderLine.ProtoReflect.Descriptor instead. func (*OrderLine) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{192} + return file_product_proto_rawDescGZIP(), []int{202} } func (x *OrderLine) GetProductId() string { @@ -9926,7 +10396,7 @@ type Subcategory struct { func (x *Subcategory) Reset() { *x = Subcategory{} - mi := &file_product_proto_msgTypes[193] + mi := &file_product_proto_msgTypes[203] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9938,7 +10408,7 @@ func (x *Subcategory) String() string { func (*Subcategory) ProtoMessage() {} func (x *Subcategory) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[193] + mi := &file_product_proto_msgTypes[203] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9951,7 +10421,7 @@ func (x *Subcategory) ProtoReflect() protoreflect.Message { // Deprecated: Use Subcategory.ProtoReflect.Descriptor instead. func (*Subcategory) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{193} + return file_product_proto_rawDescGZIP(), []int{203} } func (x *Subcategory) GetId() string { @@ -9996,7 +10466,7 @@ type CategoryMetrics struct { func (x *CategoryMetrics) Reset() { *x = CategoryMetrics{} - mi := &file_product_proto_msgTypes[194] + mi := &file_product_proto_msgTypes[204] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -10008,7 +10478,7 @@ func (x *CategoryMetrics) String() string { func (*CategoryMetrics) ProtoMessage() {} func (x *CategoryMetrics) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[194] + mi := &file_product_proto_msgTypes[204] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -10021,7 +10491,7 @@ func (x *CategoryMetrics) ProtoReflect() protoreflect.Message { // Deprecated: Use CategoryMetrics.ProtoReflect.Descriptor instead. func (*CategoryMetrics) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{194} + return file_product_proto_rawDescGZIP(), []int{204} } func (x *CategoryMetrics) GetId() string { @@ -10078,7 +10548,7 @@ type Cat struct { func (x *Cat) Reset() { *x = Cat{} - mi := &file_product_proto_msgTypes[195] + mi := &file_product_proto_msgTypes[205] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -10090,7 +10560,7 @@ func (x *Cat) String() string { func (*Cat) ProtoMessage() {} func (x *Cat) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[195] + mi := &file_product_proto_msgTypes[205] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -10103,7 +10573,7 @@ func (x *Cat) ProtoReflect() protoreflect.Message { // Deprecated: Use Cat.ProtoReflect.Descriptor instead. func (*Cat) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{195} + return file_product_proto_rawDescGZIP(), []int{205} } func (x *Cat) GetId() string { @@ -10146,7 +10616,7 @@ type Dog struct { func (x *Dog) Reset() { *x = Dog{} - mi := &file_product_proto_msgTypes[196] + mi := &file_product_proto_msgTypes[206] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -10158,7 +10628,7 @@ func (x *Dog) String() string { func (*Dog) ProtoMessage() {} func (x *Dog) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[196] + mi := &file_product_proto_msgTypes[206] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -10171,7 +10641,7 @@ func (x *Dog) ProtoReflect() protoreflect.Message { // Deprecated: Use Dog.ProtoReflect.Descriptor instead. func (*Dog) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{196} + return file_product_proto_rawDescGZIP(), []int{206} } func (x *Dog) GetId() string { @@ -10212,7 +10682,7 @@ type ActionSuccess struct { func (x *ActionSuccess) Reset() { *x = ActionSuccess{} - mi := &file_product_proto_msgTypes[197] + mi := &file_product_proto_msgTypes[207] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -10224,7 +10694,7 @@ func (x *ActionSuccess) String() string { func (*ActionSuccess) ProtoMessage() {} func (x *ActionSuccess) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[197] + mi := &file_product_proto_msgTypes[207] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -10237,7 +10707,7 @@ func (x *ActionSuccess) ProtoReflect() protoreflect.Message { // Deprecated: Use ActionSuccess.ProtoReflect.Descriptor instead. func (*ActionSuccess) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{197} + return file_product_proto_rawDescGZIP(), []int{207} } func (x *ActionSuccess) GetMessage() string { @@ -10264,7 +10734,7 @@ type ActionError struct { func (x *ActionError) Reset() { *x = ActionError{} - mi := &file_product_proto_msgTypes[198] + mi := &file_product_proto_msgTypes[208] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -10276,7 +10746,7 @@ func (x *ActionError) String() string { func (*ActionError) ProtoMessage() {} func (x *ActionError) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[198] + mi := &file_product_proto_msgTypes[208] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -10289,7 +10759,7 @@ func (x *ActionError) ProtoReflect() protoreflect.Message { // Deprecated: Use ActionError.ProtoReflect.Descriptor instead. func (*ActionError) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{198} + return file_product_proto_rawDescGZIP(), []int{208} } func (x *ActionError) GetMessage() string { @@ -10306,6 +10776,74 @@ func (x *ActionError) GetCode() string { return "" } +type TestDetails struct { + state protoimpl.MessageState `protogen:"open.v1"` + Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` + Summary string `protobuf:"bytes,2,opt,name=summary,proto3" json:"summary,omitempty"` + Pet *Animal `protobuf:"bytes,3,opt,name=pet,proto3" json:"pet,omitempty"` + Status *ActionResult `protobuf:"bytes,4,opt,name=status,proto3" json:"status,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *TestDetails) Reset() { + *x = TestDetails{} + mi := &file_product_proto_msgTypes[209] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *TestDetails) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*TestDetails) ProtoMessage() {} + +func (x *TestDetails) ProtoReflect() protoreflect.Message { + mi := &file_product_proto_msgTypes[209] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use TestDetails.ProtoReflect.Descriptor instead. +func (*TestDetails) Descriptor() ([]byte, []int) { + return file_product_proto_rawDescGZIP(), []int{209} +} + +func (x *TestDetails) GetId() string { + if x != nil { + return x.Id + } + return "" +} + +func (x *TestDetails) GetSummary() string { + if x != nil { + return x.Summary + } + return "" +} + +func (x *TestDetails) GetPet() *Animal { + if x != nil { + return x.Pet + } + return nil +} + +func (x *TestDetails) GetStatus() *ActionResult { + if x != nil { + return x.Status + } + return nil +} + type CategoryInput struct { state protoimpl.MessageState `protogen:"open.v1"` Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` @@ -10316,7 +10854,7 @@ type CategoryInput struct { func (x *CategoryInput) Reset() { *x = CategoryInput{} - mi := &file_product_proto_msgTypes[199] + mi := &file_product_proto_msgTypes[210] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -10328,7 +10866,7 @@ func (x *CategoryInput) String() string { func (*CategoryInput) ProtoMessage() {} func (x *CategoryInput) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[199] + mi := &file_product_proto_msgTypes[210] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -10341,7 +10879,7 @@ func (x *CategoryInput) ProtoReflect() protoreflect.Message { // Deprecated: Use CategoryInput.ProtoReflect.Descriptor instead. func (*CategoryInput) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{199} + return file_product_proto_rawDescGZIP(), []int{210} } func (x *CategoryInput) GetName() string { @@ -10370,7 +10908,7 @@ type ProductCountFilter struct { func (x *ProductCountFilter) Reset() { *x = ProductCountFilter{} - mi := &file_product_proto_msgTypes[200] + mi := &file_product_proto_msgTypes[211] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -10382,7 +10920,7 @@ func (x *ProductCountFilter) String() string { func (*ProductCountFilter) ProtoMessage() {} func (x *ProductCountFilter) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[200] + mi := &file_product_proto_msgTypes[211] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -10395,7 +10933,7 @@ func (x *ProductCountFilter) ProtoReflect() protoreflect.Message { // Deprecated: Use ProductCountFilter.ProtoReflect.Descriptor instead. func (*ProductCountFilter) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{200} + return file_product_proto_rawDescGZIP(), []int{211} } func (x *ProductCountFilter) GetMinPrice() *wrapperspb.DoubleValue { @@ -10439,7 +10977,7 @@ type SubcategoryItemFilter struct { func (x *SubcategoryItemFilter) Reset() { *x = SubcategoryItemFilter{} - mi := &file_product_proto_msgTypes[201] + mi := &file_product_proto_msgTypes[212] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -10451,7 +10989,7 @@ func (x *SubcategoryItemFilter) String() string { func (*SubcategoryItemFilter) ProtoMessage() {} func (x *SubcategoryItemFilter) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[201] + mi := &file_product_proto_msgTypes[212] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -10464,7 +11002,7 @@ func (x *SubcategoryItemFilter) ProtoReflect() protoreflect.Message { // Deprecated: Use SubcategoryItemFilter.ProtoReflect.Descriptor instead. func (*SubcategoryItemFilter) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{201} + return file_product_proto_rawDescGZIP(), []int{212} } func (x *SubcategoryItemFilter) GetMinPrice() *wrapperspb.DoubleValue { @@ -10513,7 +11051,7 @@ type ShippingEstimateInput struct { func (x *ShippingEstimateInput) Reset() { *x = ShippingEstimateInput{} - mi := &file_product_proto_msgTypes[202] + mi := &file_product_proto_msgTypes[213] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -10525,7 +11063,7 @@ func (x *ShippingEstimateInput) String() string { func (*ShippingEstimateInput) ProtoMessage() {} func (x *ShippingEstimateInput) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[202] + mi := &file_product_proto_msgTypes[213] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -10538,7 +11076,7 @@ func (x *ShippingEstimateInput) ProtoReflect() protoreflect.Message { // Deprecated: Use ShippingEstimateInput.ProtoReflect.Descriptor instead. func (*ShippingEstimateInput) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{202} + return file_product_proto_rawDescGZIP(), []int{213} } func (x *ShippingEstimateInput) GetDestination() ShippingDestination { @@ -10571,7 +11109,7 @@ type ListOfAuthorFilter_List struct { func (x *ListOfAuthorFilter_List) Reset() { *x = ListOfAuthorFilter_List{} - mi := &file_product_proto_msgTypes[203] + mi := &file_product_proto_msgTypes[214] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -10583,7 +11121,7 @@ func (x *ListOfAuthorFilter_List) String() string { func (*ListOfAuthorFilter_List) ProtoMessage() {} func (x *ListOfAuthorFilter_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[203] + mi := &file_product_proto_msgTypes[214] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -10615,7 +11153,7 @@ type ListOfAuthorInput_List struct { func (x *ListOfAuthorInput_List) Reset() { *x = ListOfAuthorInput_List{} - mi := &file_product_proto_msgTypes[204] + mi := &file_product_proto_msgTypes[215] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -10627,7 +11165,7 @@ func (x *ListOfAuthorInput_List) String() string { func (*ListOfAuthorInput_List) ProtoMessage() {} func (x *ListOfAuthorInput_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[204] + mi := &file_product_proto_msgTypes[215] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -10659,7 +11197,7 @@ type ListOfBlogPost_List struct { func (x *ListOfBlogPost_List) Reset() { *x = ListOfBlogPost_List{} - mi := &file_product_proto_msgTypes[205] + mi := &file_product_proto_msgTypes[216] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -10671,7 +11209,7 @@ func (x *ListOfBlogPost_List) String() string { func (*ListOfBlogPost_List) ProtoMessage() {} func (x *ListOfBlogPost_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[205] + mi := &file_product_proto_msgTypes[216] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -10703,7 +11241,7 @@ type ListOfBlogPostFilter_List struct { func (x *ListOfBlogPostFilter_List) Reset() { *x = ListOfBlogPostFilter_List{} - mi := &file_product_proto_msgTypes[206] + mi := &file_product_proto_msgTypes[217] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -10715,7 +11253,7 @@ func (x *ListOfBlogPostFilter_List) String() string { func (*ListOfBlogPostFilter_List) ProtoMessage() {} func (x *ListOfBlogPostFilter_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[206] + mi := &file_product_proto_msgTypes[217] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -10747,7 +11285,7 @@ type ListOfBlogPostInput_List struct { func (x *ListOfBlogPostInput_List) Reset() { *x = ListOfBlogPostInput_List{} - mi := &file_product_proto_msgTypes[207] + mi := &file_product_proto_msgTypes[218] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -10759,7 +11297,7 @@ func (x *ListOfBlogPostInput_List) String() string { func (*ListOfBlogPostInput_List) ProtoMessage() {} func (x *ListOfBlogPostInput_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[207] + mi := &file_product_proto_msgTypes[218] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -10791,7 +11329,7 @@ type ListOfBoolean_List struct { func (x *ListOfBoolean_List) Reset() { *x = ListOfBoolean_List{} - mi := &file_product_proto_msgTypes[208] + mi := &file_product_proto_msgTypes[219] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -10803,7 +11341,7 @@ func (x *ListOfBoolean_List) String() string { func (*ListOfBoolean_List) ProtoMessage() {} func (x *ListOfBoolean_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[208] + mi := &file_product_proto_msgTypes[219] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -10835,7 +11373,7 @@ type ListOfCategory_List struct { func (x *ListOfCategory_List) Reset() { *x = ListOfCategory_List{} - mi := &file_product_proto_msgTypes[209] + mi := &file_product_proto_msgTypes[220] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -10847,7 +11385,7 @@ func (x *ListOfCategory_List) String() string { func (*ListOfCategory_List) ProtoMessage() {} func (x *ListOfCategory_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[209] + mi := &file_product_proto_msgTypes[220] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -10879,7 +11417,7 @@ type ListOfCategoryInput_List struct { func (x *ListOfCategoryInput_List) Reset() { *x = ListOfCategoryInput_List{} - mi := &file_product_proto_msgTypes[210] + mi := &file_product_proto_msgTypes[221] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -10891,7 +11429,7 @@ func (x *ListOfCategoryInput_List) String() string { func (*ListOfCategoryInput_List) ProtoMessage() {} func (x *ListOfCategoryInput_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[210] + mi := &file_product_proto_msgTypes[221] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -10923,7 +11461,7 @@ type ListOfFloat_List struct { func (x *ListOfFloat_List) Reset() { *x = ListOfFloat_List{} - mi := &file_product_proto_msgTypes[211] + mi := &file_product_proto_msgTypes[222] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -10935,7 +11473,7 @@ func (x *ListOfFloat_List) String() string { func (*ListOfFloat_List) ProtoMessage() {} func (x *ListOfFloat_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[211] + mi := &file_product_proto_msgTypes[222] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -10967,7 +11505,7 @@ type ListOfListOfCategory_List struct { func (x *ListOfListOfCategory_List) Reset() { *x = ListOfListOfCategory_List{} - mi := &file_product_proto_msgTypes[212] + mi := &file_product_proto_msgTypes[223] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -10979,7 +11517,7 @@ func (x *ListOfListOfCategory_List) String() string { func (*ListOfListOfCategory_List) ProtoMessage() {} func (x *ListOfListOfCategory_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[212] + mi := &file_product_proto_msgTypes[223] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -11011,7 +11549,7 @@ type ListOfListOfCategoryInput_List struct { func (x *ListOfListOfCategoryInput_List) Reset() { *x = ListOfListOfCategoryInput_List{} - mi := &file_product_proto_msgTypes[213] + mi := &file_product_proto_msgTypes[224] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -11023,7 +11561,7 @@ func (x *ListOfListOfCategoryInput_List) String() string { func (*ListOfListOfCategoryInput_List) ProtoMessage() {} func (x *ListOfListOfCategoryInput_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[213] + mi := &file_product_proto_msgTypes[224] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -11055,7 +11593,7 @@ type ListOfListOfString_List struct { func (x *ListOfListOfString_List) Reset() { *x = ListOfListOfString_List{} - mi := &file_product_proto_msgTypes[214] + mi := &file_product_proto_msgTypes[225] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -11067,7 +11605,7 @@ func (x *ListOfListOfString_List) String() string { func (*ListOfListOfString_List) ProtoMessage() {} func (x *ListOfListOfString_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[214] + mi := &file_product_proto_msgTypes[225] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -11099,7 +11637,7 @@ type ListOfListOfUser_List struct { func (x *ListOfListOfUser_List) Reset() { *x = ListOfListOfUser_List{} - mi := &file_product_proto_msgTypes[215] + mi := &file_product_proto_msgTypes[226] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -11111,7 +11649,7 @@ func (x *ListOfListOfUser_List) String() string { func (*ListOfListOfUser_List) ProtoMessage() {} func (x *ListOfListOfUser_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[215] + mi := &file_product_proto_msgTypes[226] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -11143,7 +11681,7 @@ type ListOfListOfUserInput_List struct { func (x *ListOfListOfUserInput_List) Reset() { *x = ListOfListOfUserInput_List{} - mi := &file_product_proto_msgTypes[216] + mi := &file_product_proto_msgTypes[227] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -11155,7 +11693,7 @@ func (x *ListOfListOfUserInput_List) String() string { func (*ListOfListOfUserInput_List) ProtoMessage() {} func (x *ListOfListOfUserInput_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[216] + mi := &file_product_proto_msgTypes[227] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -11187,7 +11725,7 @@ type ListOfOrderLine_List struct { func (x *ListOfOrderLine_List) Reset() { *x = ListOfOrderLine_List{} - mi := &file_product_proto_msgTypes[217] + mi := &file_product_proto_msgTypes[228] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -11199,7 +11737,7 @@ func (x *ListOfOrderLine_List) String() string { func (*ListOfOrderLine_List) ProtoMessage() {} func (x *ListOfOrderLine_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[217] + mi := &file_product_proto_msgTypes[228] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -11231,7 +11769,7 @@ type ListOfProduct_List struct { func (x *ListOfProduct_List) Reset() { *x = ListOfProduct_List{} - mi := &file_product_proto_msgTypes[218] + mi := &file_product_proto_msgTypes[229] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -11243,7 +11781,7 @@ func (x *ListOfProduct_List) String() string { func (*ListOfProduct_List) ProtoMessage() {} func (x *ListOfProduct_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[218] + mi := &file_product_proto_msgTypes[229] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -11275,7 +11813,7 @@ type ListOfString_List struct { func (x *ListOfString_List) Reset() { *x = ListOfString_List{} - mi := &file_product_proto_msgTypes[219] + mi := &file_product_proto_msgTypes[230] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -11287,7 +11825,7 @@ func (x *ListOfString_List) String() string { func (*ListOfString_List) ProtoMessage() {} func (x *ListOfString_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[219] + mi := &file_product_proto_msgTypes[230] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -11319,7 +11857,7 @@ type ListOfSubcategory_List struct { func (x *ListOfSubcategory_List) Reset() { *x = ListOfSubcategory_List{} - mi := &file_product_proto_msgTypes[220] + mi := &file_product_proto_msgTypes[231] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -11331,7 +11869,7 @@ func (x *ListOfSubcategory_List) String() string { func (*ListOfSubcategory_List) ProtoMessage() {} func (x *ListOfSubcategory_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[220] + mi := &file_product_proto_msgTypes[231] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -11363,7 +11901,7 @@ type ListOfUser_List struct { func (x *ListOfUser_List) Reset() { *x = ListOfUser_List{} - mi := &file_product_proto_msgTypes[221] + mi := &file_product_proto_msgTypes[232] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -11375,7 +11913,7 @@ func (x *ListOfUser_List) String() string { func (*ListOfUser_List) ProtoMessage() {} func (x *ListOfUser_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[221] + mi := &file_product_proto_msgTypes[232] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -11407,7 +11945,7 @@ type ListOfUserInput_List struct { func (x *ListOfUserInput_List) Reset() { *x = ListOfUserInput_List{} - mi := &file_product_proto_msgTypes[222] + mi := &file_product_proto_msgTypes[233] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -11419,7 +11957,7 @@ func (x *ListOfUserInput_List) String() string { func (*ListOfUserInput_List) ProtoMessage() {} func (x *ListOfUserInput_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[222] + mi := &file_product_proto_msgTypes[233] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -11659,7 +12197,14 @@ const file_product_proto_rawDesc = "" + "\x1fQueryBulkSearchBlogPostsRequest\x129\n" + "\afilters\x18\x01 \x01(\v2\x1f.productv1.ListOfBlogPostFilterR\afilters\"l\n" + " QueryBulkSearchBlogPostsResponse\x12H\n" + - "\x16bulk_search_blog_posts\x18\x01 \x03(\v2\x13.productv1.BlogPostR\x13bulkSearchBlogPosts\"G\n" + + "\x16bulk_search_blog_posts\x18\x01 \x03(\v2\x13.productv1.BlogPostR\x13bulkSearchBlogPosts\"+\n" + + "\x19QueryTestContainerRequest\x12\x0e\n" + + "\x02id\x18\x01 \x01(\tR\x02id\"]\n" + + "\x1aQueryTestContainerResponse\x12?\n" + + "\x0etest_container\x18\x01 \x01(\v2\x18.productv1.TestContainerR\rtestContainer\"\x1c\n" + + "\x1aQueryTestContainersRequest\"`\n" + + "\x1bQueryTestContainersResponse\x12A\n" + + "\x0ftest_containers\x18\x01 \x03(\v2\x18.productv1.TestContainerR\x0etestContainers\"G\n" + "\x19MutationCreateUserRequest\x12*\n" + "\x05input\x18\x01 \x01(\v2\x14.productv1.UserInputR\x05input\"N\n" + "\x1aMutationCreateUserResponse\x120\n" + @@ -11834,7 +12379,20 @@ const file_product_proto_rawDesc = "" + "+ResolveCategoryMetricsNormalizedScoreResult\x12)\n" + "\x10normalized_score\x18\x01 \x01(\x01R\x0fnormalizedScore\"\x7f\n" + "-ResolveCategoryMetricsNormalizedScoreResponse\x12N\n" + - "\x06result\x18\x01 \x03(\v26.productv1.ResolveCategoryMetricsNormalizedScoreResultR\x06result\"C\n" + + "\x06result\x18\x01 \x03(\v26.productv1.ResolveCategoryMetricsNormalizedScoreResultR\x06result\"L\n" + + "\x1fResolveTestContainerDetailsArgs\x12)\n" + + "\x10include_extended\x18\x01 \x01(\bR\x0fincludeExtended\"H\n" + + "\"ResolveTestContainerDetailsContext\x12\x0e\n" + + "\x02id\x18\x01 \x01(\tR\x02id\x12\x12\n" + + "\x04name\x18\x02 \x01(\tR\x04name\"\xb8\x01\n" + + "\"ResolveTestContainerDetailsRequest\x12G\n" + + "\acontext\x18\x01 \x03(\v2-.productv1.ResolveTestContainerDetailsContextR\acontext\x12I\n" + + "\n" + + "field_args\x18\x02 \x01(\v2*.productv1.ResolveTestContainerDetailsArgsR\tfieldArgs\"U\n" + + "!ResolveTestContainerDetailsResult\x120\n" + + "\adetails\x18\x01 \x01(\v2\x16.productv1.TestDetailsR\adetails\"k\n" + + "#ResolveTestContainerDetailsResponse\x12D\n" + + "\x06result\x18\x01 \x03(\v2,.productv1.ResolveTestContainerDetailsResultR\x06result\"C\n" + "\aProduct\x12\x0e\n" + "\x02id\x18\x01 \x01(\tR\x02id\x12\x12\n" + "\x04name\x18\x02 \x01(\tR\x04name\x12\x14\n" + @@ -11970,7 +12528,11 @@ const file_product_proto_rawDesc = "" + "\x04name\x18\x01 \x01(\v2\x1c.google.protobuf.StringValueR\x04name\x127\n" + "\thas_teams\x18\x02 \x01(\v2\x1a.google.protobuf.BoolValueR\bhasTeams\x12<\n" + "\vskill_count\x18\x03 \x01(\v2\x1b.google.protobuf.Int32ValueR\n" + - "skillCount\"\x1f\n" + + "skillCount\"s\n" + + "\rTestContainer\x12\x0e\n" + + "\x02id\x18\x01 \x01(\tR\x02id\x12\x12\n" + + "\x04name\x18\x02 \x01(\tR\x04name\x12>\n" + + "\vdescription\x18\x03 \x01(\v2\x1c.google.protobuf.StringValueR\vdescription\"\x1f\n" + "\tUserInput\x12\x12\n" + "\x04name\x18\x01 \x01(\tR\x04name\";\n" + "\vActionInput\x12\x12\n" + @@ -12082,7 +12644,12 @@ const file_product_proto_rawDesc = "" + "\ttimestamp\x18\x02 \x01(\tR\ttimestamp\";\n" + "\vActionError\x12\x18\n" + "\amessage\x18\x01 \x01(\tR\amessage\x12\x12\n" + - "\x04code\x18\x02 \x01(\tR\x04code\"P\n" + + "\x04code\x18\x02 \x01(\tR\x04code\"\x8d\x01\n" + + "\vTestDetails\x12\x0e\n" + + "\x02id\x18\x01 \x01(\tR\x02id\x12\x18\n" + + "\asummary\x18\x02 \x01(\tR\asummary\x12#\n" + + "\x03pet\x18\x03 \x01(\v2\x11.productv1.AnimalR\x03pet\x12/\n" + + "\x06status\x18\x04 \x01(\v2\x17.productv1.ActionResultR\x06status\"P\n" + "\rCategoryInput\x12\x12\n" + "\x04name\x18\x01 \x01(\tR\x04name\x12+\n" + "\x04kind\x18\x02 \x01(\x0e2\x17.productv1.CategoryKindR\x04kind\"\x80\x02\n" + @@ -12113,7 +12680,7 @@ const file_product_proto_rawDesc = "" + " SHIPPING_DESTINATION_UNSPECIFIED\x10\x00\x12!\n" + "\x1dSHIPPING_DESTINATION_DOMESTIC\x10\x01\x12 \n" + "\x1cSHIPPING_DESTINATION_EXPRESS\x10\x02\x12&\n" + - "\"SHIPPING_DESTINATION_INTERNATIONAL\x10\x032\xfd/\n" + + "\"SHIPPING_DESTINATION_INTERNATIONAL\x10\x032\xca2\n" + "\x0eProductService\x12`\n" + "\x11LookupProductById\x12#.productv1.LookupProductByIdRequest\x1a$.productv1.LookupProductByIdResponse\"\x00\x12`\n" + "\x11LookupStorageById\x12#.productv1.LookupStorageByIdRequest\x1a$.productv1.LookupStorageByIdResponse\"\x00\x12f\n" + @@ -12155,7 +12722,9 @@ const file_product_proto_rawDesc = "" + "\x0eQueryRandomPet\x12 .productv1.QueryRandomPetRequest\x1a!.productv1.QueryRandomPetResponse\"\x00\x12r\n" + "\x17QueryRandomSearchResult\x12).productv1.QueryRandomSearchResultRequest\x1a*.productv1.QueryRandomSearchResultResponse\"\x00\x12c\n" + "\x12QueryRecursiveType\x12$.productv1.QueryRecursiveTypeRequest\x1a%.productv1.QueryRecursiveTypeResponse\"\x00\x12N\n" + - "\vQuerySearch\x12\x1d.productv1.QuerySearchRequest\x1a\x1e.productv1.QuerySearchResponse\"\x00\x12\x81\x01\n" + + "\vQuerySearch\x12\x1d.productv1.QuerySearchRequest\x1a\x1e.productv1.QuerySearchResponse\"\x00\x12c\n" + + "\x12QueryTestContainer\x12$.productv1.QueryTestContainerRequest\x1a%.productv1.QueryTestContainerResponse\"\x00\x12f\n" + + "\x13QueryTestContainers\x12%.productv1.QueryTestContainersRequest\x1a&.productv1.QueryTestContainersResponse\"\x00\x12\x81\x01\n" + "\x1cQueryTypeFilterWithArguments\x12..productv1.QueryTypeFilterWithArgumentsRequest\x1a/.productv1.QueryTypeFilterWithArgumentsResponse\"\x00\x12\x90\x01\n" + "!QueryTypeWithMultipleFilterFields\x123.productv1.QueryTypeWithMultipleFilterFieldsRequest\x1a4.productv1.QueryTypeWithMultipleFilterFieldsResponse\"\x00\x12H\n" + "\tQueryUser\x12\x1b.productv1.QueryUserRequest\x1a\x1c.productv1.QueryUserResponse\"\x00\x12K\n" + @@ -12169,7 +12738,8 @@ const file_product_proto_rawDesc = "" + "\x1bResolveCategoryProductCount\x12-.productv1.ResolveCategoryProductCountRequest\x1a..productv1.ResolveCategoryProductCountResponse\"\x00\x12\x90\x01\n" + "!ResolveProductRecommendedCategory\x123.productv1.ResolveProductRecommendedCategoryRequest\x1a4.productv1.ResolveProductRecommendedCategoryResponse\"\x00\x12\x87\x01\n" + "\x1eResolveProductShippingEstimate\x120.productv1.ResolveProductShippingEstimateRequest\x1a1.productv1.ResolveProductShippingEstimateResponse\"\x00\x12~\n" + - "\x1bResolveSubcategoryItemCount\x12-.productv1.ResolveSubcategoryItemCountRequest\x1a..productv1.ResolveSubcategoryItemCountResponse\"\x00B%Z#cosmo/pkg/proto/productv1;productv1b\x06proto3" + "\x1bResolveSubcategoryItemCount\x12-.productv1.ResolveSubcategoryItemCountRequest\x1a..productv1.ResolveSubcategoryItemCountResponse\"\x00\x12~\n" + + "\x1bResolveTestContainerDetails\x12-.productv1.ResolveTestContainerDetailsRequest\x1a..productv1.ResolveTestContainerDetailsResponse\"\x00B%Z#cosmo/pkg/proto/productv1;productv1b\x06proto3" var ( file_product_proto_rawDescOnce sync.Once @@ -12184,7 +12754,7 @@ func file_product_proto_rawDescGZIP() []byte { } var file_product_proto_enumTypes = make([]protoimpl.EnumInfo, 2) -var file_product_proto_msgTypes = make([]protoimpl.MessageInfo, 223) +var file_product_proto_msgTypes = make([]protoimpl.MessageInfo, 234) var file_product_proto_goTypes = []any{ (CategoryKind)(0), // 0: productv1.CategoryKind (ShippingDestination)(0), // 1: productv1.ShippingDestination @@ -12277,500 +12847,526 @@ var file_product_proto_goTypes = []any{ (*QueryBulkSearchAuthorsResponse)(nil), // 88: productv1.QueryBulkSearchAuthorsResponse (*QueryBulkSearchBlogPostsRequest)(nil), // 89: productv1.QueryBulkSearchBlogPostsRequest (*QueryBulkSearchBlogPostsResponse)(nil), // 90: productv1.QueryBulkSearchBlogPostsResponse - (*MutationCreateUserRequest)(nil), // 91: productv1.MutationCreateUserRequest - (*MutationCreateUserResponse)(nil), // 92: productv1.MutationCreateUserResponse - (*MutationPerformActionRequest)(nil), // 93: productv1.MutationPerformActionRequest - (*MutationPerformActionResponse)(nil), // 94: productv1.MutationPerformActionResponse - (*MutationCreateNullableFieldsTypeRequest)(nil), // 95: productv1.MutationCreateNullableFieldsTypeRequest - (*MutationCreateNullableFieldsTypeResponse)(nil), // 96: productv1.MutationCreateNullableFieldsTypeResponse - (*MutationUpdateNullableFieldsTypeRequest)(nil), // 97: productv1.MutationUpdateNullableFieldsTypeRequest - (*MutationUpdateNullableFieldsTypeResponse)(nil), // 98: productv1.MutationUpdateNullableFieldsTypeResponse - (*MutationCreateBlogPostRequest)(nil), // 99: productv1.MutationCreateBlogPostRequest - (*MutationCreateBlogPostResponse)(nil), // 100: productv1.MutationCreateBlogPostResponse - (*MutationUpdateBlogPostRequest)(nil), // 101: productv1.MutationUpdateBlogPostRequest - (*MutationUpdateBlogPostResponse)(nil), // 102: productv1.MutationUpdateBlogPostResponse - (*MutationCreateAuthorRequest)(nil), // 103: productv1.MutationCreateAuthorRequest - (*MutationCreateAuthorResponse)(nil), // 104: productv1.MutationCreateAuthorResponse - (*MutationUpdateAuthorRequest)(nil), // 105: productv1.MutationUpdateAuthorRequest - (*MutationUpdateAuthorResponse)(nil), // 106: productv1.MutationUpdateAuthorResponse - (*MutationBulkCreateAuthorsRequest)(nil), // 107: productv1.MutationBulkCreateAuthorsRequest - (*MutationBulkCreateAuthorsResponse)(nil), // 108: productv1.MutationBulkCreateAuthorsResponse - (*MutationBulkUpdateAuthorsRequest)(nil), // 109: productv1.MutationBulkUpdateAuthorsRequest - (*MutationBulkUpdateAuthorsResponse)(nil), // 110: productv1.MutationBulkUpdateAuthorsResponse - (*MutationBulkCreateBlogPostsRequest)(nil), // 111: productv1.MutationBulkCreateBlogPostsRequest - (*MutationBulkCreateBlogPostsResponse)(nil), // 112: productv1.MutationBulkCreateBlogPostsResponse - (*MutationBulkUpdateBlogPostsRequest)(nil), // 113: productv1.MutationBulkUpdateBlogPostsRequest - (*MutationBulkUpdateBlogPostsResponse)(nil), // 114: productv1.MutationBulkUpdateBlogPostsResponse - (*ResolveProductShippingEstimateArgs)(nil), // 115: productv1.ResolveProductShippingEstimateArgs - (*ResolveProductShippingEstimateContext)(nil), // 116: productv1.ResolveProductShippingEstimateContext - (*ResolveProductShippingEstimateRequest)(nil), // 117: productv1.ResolveProductShippingEstimateRequest - (*ResolveProductShippingEstimateResult)(nil), // 118: productv1.ResolveProductShippingEstimateResult - (*ResolveProductShippingEstimateResponse)(nil), // 119: productv1.ResolveProductShippingEstimateResponse - (*ResolveProductRecommendedCategoryArgs)(nil), // 120: productv1.ResolveProductRecommendedCategoryArgs - (*ResolveProductRecommendedCategoryContext)(nil), // 121: productv1.ResolveProductRecommendedCategoryContext - (*ResolveProductRecommendedCategoryRequest)(nil), // 122: productv1.ResolveProductRecommendedCategoryRequest - (*ResolveProductRecommendedCategoryResult)(nil), // 123: productv1.ResolveProductRecommendedCategoryResult - (*ResolveProductRecommendedCategoryResponse)(nil), // 124: productv1.ResolveProductRecommendedCategoryResponse - (*ResolveCategoryProductCountArgs)(nil), // 125: productv1.ResolveCategoryProductCountArgs - (*ResolveCategoryProductCountContext)(nil), // 126: productv1.ResolveCategoryProductCountContext - (*ResolveCategoryProductCountRequest)(nil), // 127: productv1.ResolveCategoryProductCountRequest - (*ResolveCategoryProductCountResult)(nil), // 128: productv1.ResolveCategoryProductCountResult - (*ResolveCategoryProductCountResponse)(nil), // 129: productv1.ResolveCategoryProductCountResponse - (*ResolveCategoryPopularityScoreArgs)(nil), // 130: productv1.ResolveCategoryPopularityScoreArgs - (*ResolveCategoryPopularityScoreContext)(nil), // 131: productv1.ResolveCategoryPopularityScoreContext - (*ResolveCategoryPopularityScoreRequest)(nil), // 132: productv1.ResolveCategoryPopularityScoreRequest - (*ResolveCategoryPopularityScoreResult)(nil), // 133: productv1.ResolveCategoryPopularityScoreResult - (*ResolveCategoryPopularityScoreResponse)(nil), // 134: productv1.ResolveCategoryPopularityScoreResponse - (*ResolveCategoryCategoryMetricsArgs)(nil), // 135: productv1.ResolveCategoryCategoryMetricsArgs - (*ResolveCategoryCategoryMetricsContext)(nil), // 136: productv1.ResolveCategoryCategoryMetricsContext - (*ResolveCategoryCategoryMetricsRequest)(nil), // 137: productv1.ResolveCategoryCategoryMetricsRequest - (*ResolveCategoryCategoryMetricsResult)(nil), // 138: productv1.ResolveCategoryCategoryMetricsResult - (*ResolveCategoryCategoryMetricsResponse)(nil), // 139: productv1.ResolveCategoryCategoryMetricsResponse - (*ResolveCategoryMascotArgs)(nil), // 140: productv1.ResolveCategoryMascotArgs - (*ResolveCategoryMascotContext)(nil), // 141: productv1.ResolveCategoryMascotContext - (*ResolveCategoryMascotRequest)(nil), // 142: productv1.ResolveCategoryMascotRequest - (*ResolveCategoryMascotResult)(nil), // 143: productv1.ResolveCategoryMascotResult - (*ResolveCategoryMascotResponse)(nil), // 144: productv1.ResolveCategoryMascotResponse - (*ResolveCategoryCategoryStatusArgs)(nil), // 145: productv1.ResolveCategoryCategoryStatusArgs - (*ResolveCategoryCategoryStatusContext)(nil), // 146: productv1.ResolveCategoryCategoryStatusContext - (*ResolveCategoryCategoryStatusRequest)(nil), // 147: productv1.ResolveCategoryCategoryStatusRequest - (*ResolveCategoryCategoryStatusResult)(nil), // 148: productv1.ResolveCategoryCategoryStatusResult - (*ResolveCategoryCategoryStatusResponse)(nil), // 149: productv1.ResolveCategoryCategoryStatusResponse - (*ResolveSubcategoryItemCountArgs)(nil), // 150: productv1.ResolveSubcategoryItemCountArgs - (*ResolveSubcategoryItemCountContext)(nil), // 151: productv1.ResolveSubcategoryItemCountContext - (*ResolveSubcategoryItemCountRequest)(nil), // 152: productv1.ResolveSubcategoryItemCountRequest - (*ResolveSubcategoryItemCountResult)(nil), // 153: productv1.ResolveSubcategoryItemCountResult - (*ResolveSubcategoryItemCountResponse)(nil), // 154: productv1.ResolveSubcategoryItemCountResponse - (*ResolveCategoryMetricsNormalizedScoreArgs)(nil), // 155: productv1.ResolveCategoryMetricsNormalizedScoreArgs - (*ResolveCategoryMetricsNormalizedScoreContext)(nil), // 156: productv1.ResolveCategoryMetricsNormalizedScoreContext - (*ResolveCategoryMetricsNormalizedScoreRequest)(nil), // 157: productv1.ResolveCategoryMetricsNormalizedScoreRequest - (*ResolveCategoryMetricsNormalizedScoreResult)(nil), // 158: productv1.ResolveCategoryMetricsNormalizedScoreResult - (*ResolveCategoryMetricsNormalizedScoreResponse)(nil), // 159: productv1.ResolveCategoryMetricsNormalizedScoreResponse - (*Product)(nil), // 160: productv1.Product - (*Storage)(nil), // 161: productv1.Storage - (*Warehouse)(nil), // 162: productv1.Warehouse - (*User)(nil), // 163: productv1.User - (*NestedTypeA)(nil), // 164: productv1.NestedTypeA - (*RecursiveType)(nil), // 165: productv1.RecursiveType - (*TypeWithMultipleFilterFields)(nil), // 166: productv1.TypeWithMultipleFilterFields - (*FilterTypeInput)(nil), // 167: productv1.FilterTypeInput - (*ComplexFilterTypeInput)(nil), // 168: productv1.ComplexFilterTypeInput - (*TypeWithComplexFilterInput)(nil), // 169: productv1.TypeWithComplexFilterInput - (*OrderInput)(nil), // 170: productv1.OrderInput - (*Order)(nil), // 171: productv1.Order - (*Category)(nil), // 172: productv1.Category - (*CategoryFilter)(nil), // 173: productv1.CategoryFilter - (*Animal)(nil), // 174: productv1.Animal - (*SearchInput)(nil), // 175: productv1.SearchInput - (*SearchResult)(nil), // 176: productv1.SearchResult - (*NullableFieldsType)(nil), // 177: productv1.NullableFieldsType - (*NullableFieldsFilter)(nil), // 178: productv1.NullableFieldsFilter - (*BlogPost)(nil), // 179: productv1.BlogPost - (*BlogPostFilter)(nil), // 180: productv1.BlogPostFilter - (*Author)(nil), // 181: productv1.Author - (*AuthorFilter)(nil), // 182: productv1.AuthorFilter - (*UserInput)(nil), // 183: productv1.UserInput - (*ActionInput)(nil), // 184: productv1.ActionInput - (*ActionResult)(nil), // 185: productv1.ActionResult - (*NullableFieldsInput)(nil), // 186: productv1.NullableFieldsInput - (*BlogPostInput)(nil), // 187: productv1.BlogPostInput - (*AuthorInput)(nil), // 188: productv1.AuthorInput - (*NestedTypeB)(nil), // 189: productv1.NestedTypeB - (*NestedTypeC)(nil), // 190: productv1.NestedTypeC - (*FilterType)(nil), // 191: productv1.FilterType - (*Pagination)(nil), // 192: productv1.Pagination - (*OrderLineInput)(nil), // 193: productv1.OrderLineInput - (*OrderLine)(nil), // 194: productv1.OrderLine - (*Subcategory)(nil), // 195: productv1.Subcategory - (*CategoryMetrics)(nil), // 196: productv1.CategoryMetrics - (*Cat)(nil), // 197: productv1.Cat - (*Dog)(nil), // 198: productv1.Dog - (*ActionSuccess)(nil), // 199: productv1.ActionSuccess - (*ActionError)(nil), // 200: productv1.ActionError - (*CategoryInput)(nil), // 201: productv1.CategoryInput - (*ProductCountFilter)(nil), // 202: productv1.ProductCountFilter - (*SubcategoryItemFilter)(nil), // 203: productv1.SubcategoryItemFilter - (*ShippingEstimateInput)(nil), // 204: productv1.ShippingEstimateInput - (*ListOfAuthorFilter_List)(nil), // 205: productv1.ListOfAuthorFilter.List - (*ListOfAuthorInput_List)(nil), // 206: productv1.ListOfAuthorInput.List - (*ListOfBlogPost_List)(nil), // 207: productv1.ListOfBlogPost.List - (*ListOfBlogPostFilter_List)(nil), // 208: productv1.ListOfBlogPostFilter.List - (*ListOfBlogPostInput_List)(nil), // 209: productv1.ListOfBlogPostInput.List - (*ListOfBoolean_List)(nil), // 210: productv1.ListOfBoolean.List - (*ListOfCategory_List)(nil), // 211: productv1.ListOfCategory.List - (*ListOfCategoryInput_List)(nil), // 212: productv1.ListOfCategoryInput.List - (*ListOfFloat_List)(nil), // 213: productv1.ListOfFloat.List - (*ListOfListOfCategory_List)(nil), // 214: productv1.ListOfListOfCategory.List - (*ListOfListOfCategoryInput_List)(nil), // 215: productv1.ListOfListOfCategoryInput.List - (*ListOfListOfString_List)(nil), // 216: productv1.ListOfListOfString.List - (*ListOfListOfUser_List)(nil), // 217: productv1.ListOfListOfUser.List - (*ListOfListOfUserInput_List)(nil), // 218: productv1.ListOfListOfUserInput.List - (*ListOfOrderLine_List)(nil), // 219: productv1.ListOfOrderLine.List - (*ListOfProduct_List)(nil), // 220: productv1.ListOfProduct.List - (*ListOfString_List)(nil), // 221: productv1.ListOfString.List - (*ListOfSubcategory_List)(nil), // 222: productv1.ListOfSubcategory.List - (*ListOfUser_List)(nil), // 223: productv1.ListOfUser.List - (*ListOfUserInput_List)(nil), // 224: productv1.ListOfUserInput.List - (*wrapperspb.Int32Value)(nil), // 225: google.protobuf.Int32Value - (*wrapperspb.StringValue)(nil), // 226: google.protobuf.StringValue - (*wrapperspb.DoubleValue)(nil), // 227: google.protobuf.DoubleValue - (*wrapperspb.BoolValue)(nil), // 228: google.protobuf.BoolValue + (*QueryTestContainerRequest)(nil), // 91: productv1.QueryTestContainerRequest + (*QueryTestContainerResponse)(nil), // 92: productv1.QueryTestContainerResponse + (*QueryTestContainersRequest)(nil), // 93: productv1.QueryTestContainersRequest + (*QueryTestContainersResponse)(nil), // 94: productv1.QueryTestContainersResponse + (*MutationCreateUserRequest)(nil), // 95: productv1.MutationCreateUserRequest + (*MutationCreateUserResponse)(nil), // 96: productv1.MutationCreateUserResponse + (*MutationPerformActionRequest)(nil), // 97: productv1.MutationPerformActionRequest + (*MutationPerformActionResponse)(nil), // 98: productv1.MutationPerformActionResponse + (*MutationCreateNullableFieldsTypeRequest)(nil), // 99: productv1.MutationCreateNullableFieldsTypeRequest + (*MutationCreateNullableFieldsTypeResponse)(nil), // 100: productv1.MutationCreateNullableFieldsTypeResponse + (*MutationUpdateNullableFieldsTypeRequest)(nil), // 101: productv1.MutationUpdateNullableFieldsTypeRequest + (*MutationUpdateNullableFieldsTypeResponse)(nil), // 102: productv1.MutationUpdateNullableFieldsTypeResponse + (*MutationCreateBlogPostRequest)(nil), // 103: productv1.MutationCreateBlogPostRequest + (*MutationCreateBlogPostResponse)(nil), // 104: productv1.MutationCreateBlogPostResponse + (*MutationUpdateBlogPostRequest)(nil), // 105: productv1.MutationUpdateBlogPostRequest + (*MutationUpdateBlogPostResponse)(nil), // 106: productv1.MutationUpdateBlogPostResponse + (*MutationCreateAuthorRequest)(nil), // 107: productv1.MutationCreateAuthorRequest + (*MutationCreateAuthorResponse)(nil), // 108: productv1.MutationCreateAuthorResponse + (*MutationUpdateAuthorRequest)(nil), // 109: productv1.MutationUpdateAuthorRequest + (*MutationUpdateAuthorResponse)(nil), // 110: productv1.MutationUpdateAuthorResponse + (*MutationBulkCreateAuthorsRequest)(nil), // 111: productv1.MutationBulkCreateAuthorsRequest + (*MutationBulkCreateAuthorsResponse)(nil), // 112: productv1.MutationBulkCreateAuthorsResponse + (*MutationBulkUpdateAuthorsRequest)(nil), // 113: productv1.MutationBulkUpdateAuthorsRequest + (*MutationBulkUpdateAuthorsResponse)(nil), // 114: productv1.MutationBulkUpdateAuthorsResponse + (*MutationBulkCreateBlogPostsRequest)(nil), // 115: productv1.MutationBulkCreateBlogPostsRequest + (*MutationBulkCreateBlogPostsResponse)(nil), // 116: productv1.MutationBulkCreateBlogPostsResponse + (*MutationBulkUpdateBlogPostsRequest)(nil), // 117: productv1.MutationBulkUpdateBlogPostsRequest + (*MutationBulkUpdateBlogPostsResponse)(nil), // 118: productv1.MutationBulkUpdateBlogPostsResponse + (*ResolveProductShippingEstimateArgs)(nil), // 119: productv1.ResolveProductShippingEstimateArgs + (*ResolveProductShippingEstimateContext)(nil), // 120: productv1.ResolveProductShippingEstimateContext + (*ResolveProductShippingEstimateRequest)(nil), // 121: productv1.ResolveProductShippingEstimateRequest + (*ResolveProductShippingEstimateResult)(nil), // 122: productv1.ResolveProductShippingEstimateResult + (*ResolveProductShippingEstimateResponse)(nil), // 123: productv1.ResolveProductShippingEstimateResponse + (*ResolveProductRecommendedCategoryArgs)(nil), // 124: productv1.ResolveProductRecommendedCategoryArgs + (*ResolveProductRecommendedCategoryContext)(nil), // 125: productv1.ResolveProductRecommendedCategoryContext + (*ResolveProductRecommendedCategoryRequest)(nil), // 126: productv1.ResolveProductRecommendedCategoryRequest + (*ResolveProductRecommendedCategoryResult)(nil), // 127: productv1.ResolveProductRecommendedCategoryResult + (*ResolveProductRecommendedCategoryResponse)(nil), // 128: productv1.ResolveProductRecommendedCategoryResponse + (*ResolveCategoryProductCountArgs)(nil), // 129: productv1.ResolveCategoryProductCountArgs + (*ResolveCategoryProductCountContext)(nil), // 130: productv1.ResolveCategoryProductCountContext + (*ResolveCategoryProductCountRequest)(nil), // 131: productv1.ResolveCategoryProductCountRequest + (*ResolveCategoryProductCountResult)(nil), // 132: productv1.ResolveCategoryProductCountResult + (*ResolveCategoryProductCountResponse)(nil), // 133: productv1.ResolveCategoryProductCountResponse + (*ResolveCategoryPopularityScoreArgs)(nil), // 134: productv1.ResolveCategoryPopularityScoreArgs + (*ResolveCategoryPopularityScoreContext)(nil), // 135: productv1.ResolveCategoryPopularityScoreContext + (*ResolveCategoryPopularityScoreRequest)(nil), // 136: productv1.ResolveCategoryPopularityScoreRequest + (*ResolveCategoryPopularityScoreResult)(nil), // 137: productv1.ResolveCategoryPopularityScoreResult + (*ResolveCategoryPopularityScoreResponse)(nil), // 138: productv1.ResolveCategoryPopularityScoreResponse + (*ResolveCategoryCategoryMetricsArgs)(nil), // 139: productv1.ResolveCategoryCategoryMetricsArgs + (*ResolveCategoryCategoryMetricsContext)(nil), // 140: productv1.ResolveCategoryCategoryMetricsContext + (*ResolveCategoryCategoryMetricsRequest)(nil), // 141: productv1.ResolveCategoryCategoryMetricsRequest + (*ResolveCategoryCategoryMetricsResult)(nil), // 142: productv1.ResolveCategoryCategoryMetricsResult + (*ResolveCategoryCategoryMetricsResponse)(nil), // 143: productv1.ResolveCategoryCategoryMetricsResponse + (*ResolveCategoryMascotArgs)(nil), // 144: productv1.ResolveCategoryMascotArgs + (*ResolveCategoryMascotContext)(nil), // 145: productv1.ResolveCategoryMascotContext + (*ResolveCategoryMascotRequest)(nil), // 146: productv1.ResolveCategoryMascotRequest + (*ResolveCategoryMascotResult)(nil), // 147: productv1.ResolveCategoryMascotResult + (*ResolveCategoryMascotResponse)(nil), // 148: productv1.ResolveCategoryMascotResponse + (*ResolveCategoryCategoryStatusArgs)(nil), // 149: productv1.ResolveCategoryCategoryStatusArgs + (*ResolveCategoryCategoryStatusContext)(nil), // 150: productv1.ResolveCategoryCategoryStatusContext + (*ResolveCategoryCategoryStatusRequest)(nil), // 151: productv1.ResolveCategoryCategoryStatusRequest + (*ResolveCategoryCategoryStatusResult)(nil), // 152: productv1.ResolveCategoryCategoryStatusResult + (*ResolveCategoryCategoryStatusResponse)(nil), // 153: productv1.ResolveCategoryCategoryStatusResponse + (*ResolveSubcategoryItemCountArgs)(nil), // 154: productv1.ResolveSubcategoryItemCountArgs + (*ResolveSubcategoryItemCountContext)(nil), // 155: productv1.ResolveSubcategoryItemCountContext + (*ResolveSubcategoryItemCountRequest)(nil), // 156: productv1.ResolveSubcategoryItemCountRequest + (*ResolveSubcategoryItemCountResult)(nil), // 157: productv1.ResolveSubcategoryItemCountResult + (*ResolveSubcategoryItemCountResponse)(nil), // 158: productv1.ResolveSubcategoryItemCountResponse + (*ResolveCategoryMetricsNormalizedScoreArgs)(nil), // 159: productv1.ResolveCategoryMetricsNormalizedScoreArgs + (*ResolveCategoryMetricsNormalizedScoreContext)(nil), // 160: productv1.ResolveCategoryMetricsNormalizedScoreContext + (*ResolveCategoryMetricsNormalizedScoreRequest)(nil), // 161: productv1.ResolveCategoryMetricsNormalizedScoreRequest + (*ResolveCategoryMetricsNormalizedScoreResult)(nil), // 162: productv1.ResolveCategoryMetricsNormalizedScoreResult + (*ResolveCategoryMetricsNormalizedScoreResponse)(nil), // 163: productv1.ResolveCategoryMetricsNormalizedScoreResponse + (*ResolveTestContainerDetailsArgs)(nil), // 164: productv1.ResolveTestContainerDetailsArgs + (*ResolveTestContainerDetailsContext)(nil), // 165: productv1.ResolveTestContainerDetailsContext + (*ResolveTestContainerDetailsRequest)(nil), // 166: productv1.ResolveTestContainerDetailsRequest + (*ResolveTestContainerDetailsResult)(nil), // 167: productv1.ResolveTestContainerDetailsResult + (*ResolveTestContainerDetailsResponse)(nil), // 168: productv1.ResolveTestContainerDetailsResponse + (*Product)(nil), // 169: productv1.Product + (*Storage)(nil), // 170: productv1.Storage + (*Warehouse)(nil), // 171: productv1.Warehouse + (*User)(nil), // 172: productv1.User + (*NestedTypeA)(nil), // 173: productv1.NestedTypeA + (*RecursiveType)(nil), // 174: productv1.RecursiveType + (*TypeWithMultipleFilterFields)(nil), // 175: productv1.TypeWithMultipleFilterFields + (*FilterTypeInput)(nil), // 176: productv1.FilterTypeInput + (*ComplexFilterTypeInput)(nil), // 177: productv1.ComplexFilterTypeInput + (*TypeWithComplexFilterInput)(nil), // 178: productv1.TypeWithComplexFilterInput + (*OrderInput)(nil), // 179: productv1.OrderInput + (*Order)(nil), // 180: productv1.Order + (*Category)(nil), // 181: productv1.Category + (*CategoryFilter)(nil), // 182: productv1.CategoryFilter + (*Animal)(nil), // 183: productv1.Animal + (*SearchInput)(nil), // 184: productv1.SearchInput + (*SearchResult)(nil), // 185: productv1.SearchResult + (*NullableFieldsType)(nil), // 186: productv1.NullableFieldsType + (*NullableFieldsFilter)(nil), // 187: productv1.NullableFieldsFilter + (*BlogPost)(nil), // 188: productv1.BlogPost + (*BlogPostFilter)(nil), // 189: productv1.BlogPostFilter + (*Author)(nil), // 190: productv1.Author + (*AuthorFilter)(nil), // 191: productv1.AuthorFilter + (*TestContainer)(nil), // 192: productv1.TestContainer + (*UserInput)(nil), // 193: productv1.UserInput + (*ActionInput)(nil), // 194: productv1.ActionInput + (*ActionResult)(nil), // 195: productv1.ActionResult + (*NullableFieldsInput)(nil), // 196: productv1.NullableFieldsInput + (*BlogPostInput)(nil), // 197: productv1.BlogPostInput + (*AuthorInput)(nil), // 198: productv1.AuthorInput + (*NestedTypeB)(nil), // 199: productv1.NestedTypeB + (*NestedTypeC)(nil), // 200: productv1.NestedTypeC + (*FilterType)(nil), // 201: productv1.FilterType + (*Pagination)(nil), // 202: productv1.Pagination + (*OrderLineInput)(nil), // 203: productv1.OrderLineInput + (*OrderLine)(nil), // 204: productv1.OrderLine + (*Subcategory)(nil), // 205: productv1.Subcategory + (*CategoryMetrics)(nil), // 206: productv1.CategoryMetrics + (*Cat)(nil), // 207: productv1.Cat + (*Dog)(nil), // 208: productv1.Dog + (*ActionSuccess)(nil), // 209: productv1.ActionSuccess + (*ActionError)(nil), // 210: productv1.ActionError + (*TestDetails)(nil), // 211: productv1.TestDetails + (*CategoryInput)(nil), // 212: productv1.CategoryInput + (*ProductCountFilter)(nil), // 213: productv1.ProductCountFilter + (*SubcategoryItemFilter)(nil), // 214: productv1.SubcategoryItemFilter + (*ShippingEstimateInput)(nil), // 215: productv1.ShippingEstimateInput + (*ListOfAuthorFilter_List)(nil), // 216: productv1.ListOfAuthorFilter.List + (*ListOfAuthorInput_List)(nil), // 217: productv1.ListOfAuthorInput.List + (*ListOfBlogPost_List)(nil), // 218: productv1.ListOfBlogPost.List + (*ListOfBlogPostFilter_List)(nil), // 219: productv1.ListOfBlogPostFilter.List + (*ListOfBlogPostInput_List)(nil), // 220: productv1.ListOfBlogPostInput.List + (*ListOfBoolean_List)(nil), // 221: productv1.ListOfBoolean.List + (*ListOfCategory_List)(nil), // 222: productv1.ListOfCategory.List + (*ListOfCategoryInput_List)(nil), // 223: productv1.ListOfCategoryInput.List + (*ListOfFloat_List)(nil), // 224: productv1.ListOfFloat.List + (*ListOfListOfCategory_List)(nil), // 225: productv1.ListOfListOfCategory.List + (*ListOfListOfCategoryInput_List)(nil), // 226: productv1.ListOfListOfCategoryInput.List + (*ListOfListOfString_List)(nil), // 227: productv1.ListOfListOfString.List + (*ListOfListOfUser_List)(nil), // 228: productv1.ListOfListOfUser.List + (*ListOfListOfUserInput_List)(nil), // 229: productv1.ListOfListOfUserInput.List + (*ListOfOrderLine_List)(nil), // 230: productv1.ListOfOrderLine.List + (*ListOfProduct_List)(nil), // 231: productv1.ListOfProduct.List + (*ListOfString_List)(nil), // 232: productv1.ListOfString.List + (*ListOfSubcategory_List)(nil), // 233: productv1.ListOfSubcategory.List + (*ListOfUser_List)(nil), // 234: productv1.ListOfUser.List + (*ListOfUserInput_List)(nil), // 235: productv1.ListOfUserInput.List + (*wrapperspb.Int32Value)(nil), // 236: google.protobuf.Int32Value + (*wrapperspb.StringValue)(nil), // 237: google.protobuf.StringValue + (*wrapperspb.DoubleValue)(nil), // 238: google.protobuf.DoubleValue + (*wrapperspb.BoolValue)(nil), // 239: google.protobuf.BoolValue } var file_product_proto_depIdxs = []int32{ - 205, // 0: productv1.ListOfAuthorFilter.list:type_name -> productv1.ListOfAuthorFilter.List - 206, // 1: productv1.ListOfAuthorInput.list:type_name -> productv1.ListOfAuthorInput.List - 207, // 2: productv1.ListOfBlogPost.list:type_name -> productv1.ListOfBlogPost.List - 208, // 3: productv1.ListOfBlogPostFilter.list:type_name -> productv1.ListOfBlogPostFilter.List - 209, // 4: productv1.ListOfBlogPostInput.list:type_name -> productv1.ListOfBlogPostInput.List - 210, // 5: productv1.ListOfBoolean.list:type_name -> productv1.ListOfBoolean.List - 211, // 6: productv1.ListOfCategory.list:type_name -> productv1.ListOfCategory.List - 212, // 7: productv1.ListOfCategoryInput.list:type_name -> productv1.ListOfCategoryInput.List - 213, // 8: productv1.ListOfFloat.list:type_name -> productv1.ListOfFloat.List - 214, // 9: productv1.ListOfListOfCategory.list:type_name -> productv1.ListOfListOfCategory.List - 215, // 10: productv1.ListOfListOfCategoryInput.list:type_name -> productv1.ListOfListOfCategoryInput.List - 216, // 11: productv1.ListOfListOfString.list:type_name -> productv1.ListOfListOfString.List - 217, // 12: productv1.ListOfListOfUser.list:type_name -> productv1.ListOfListOfUser.List - 218, // 13: productv1.ListOfListOfUserInput.list:type_name -> productv1.ListOfListOfUserInput.List - 219, // 14: productv1.ListOfOrderLine.list:type_name -> productv1.ListOfOrderLine.List - 220, // 15: productv1.ListOfProduct.list:type_name -> productv1.ListOfProduct.List - 221, // 16: productv1.ListOfString.list:type_name -> productv1.ListOfString.List - 222, // 17: productv1.ListOfSubcategory.list:type_name -> productv1.ListOfSubcategory.List - 223, // 18: productv1.ListOfUser.list:type_name -> productv1.ListOfUser.List - 224, // 19: productv1.ListOfUserInput.list:type_name -> productv1.ListOfUserInput.List + 216, // 0: productv1.ListOfAuthorFilter.list:type_name -> productv1.ListOfAuthorFilter.List + 217, // 1: productv1.ListOfAuthorInput.list:type_name -> productv1.ListOfAuthorInput.List + 218, // 2: productv1.ListOfBlogPost.list:type_name -> productv1.ListOfBlogPost.List + 219, // 3: productv1.ListOfBlogPostFilter.list:type_name -> productv1.ListOfBlogPostFilter.List + 220, // 4: productv1.ListOfBlogPostInput.list:type_name -> productv1.ListOfBlogPostInput.List + 221, // 5: productv1.ListOfBoolean.list:type_name -> productv1.ListOfBoolean.List + 222, // 6: productv1.ListOfCategory.list:type_name -> productv1.ListOfCategory.List + 223, // 7: productv1.ListOfCategoryInput.list:type_name -> productv1.ListOfCategoryInput.List + 224, // 8: productv1.ListOfFloat.list:type_name -> productv1.ListOfFloat.List + 225, // 9: productv1.ListOfListOfCategory.list:type_name -> productv1.ListOfListOfCategory.List + 226, // 10: productv1.ListOfListOfCategoryInput.list:type_name -> productv1.ListOfListOfCategoryInput.List + 227, // 11: productv1.ListOfListOfString.list:type_name -> productv1.ListOfListOfString.List + 228, // 12: productv1.ListOfListOfUser.list:type_name -> productv1.ListOfListOfUser.List + 229, // 13: productv1.ListOfListOfUserInput.list:type_name -> productv1.ListOfListOfUserInput.List + 230, // 14: productv1.ListOfOrderLine.list:type_name -> productv1.ListOfOrderLine.List + 231, // 15: productv1.ListOfProduct.list:type_name -> productv1.ListOfProduct.List + 232, // 16: productv1.ListOfString.list:type_name -> productv1.ListOfString.List + 233, // 17: productv1.ListOfSubcategory.list:type_name -> productv1.ListOfSubcategory.List + 234, // 18: productv1.ListOfUser.list:type_name -> productv1.ListOfUser.List + 235, // 19: productv1.ListOfUserInput.list:type_name -> productv1.ListOfUserInput.List 22, // 20: productv1.LookupProductByIdRequest.keys:type_name -> productv1.LookupProductByIdRequestKey - 160, // 21: productv1.LookupProductByIdResponse.result:type_name -> productv1.Product + 169, // 21: productv1.LookupProductByIdResponse.result:type_name -> productv1.Product 25, // 22: productv1.LookupStorageByIdRequest.keys:type_name -> productv1.LookupStorageByIdRequestKey - 161, // 23: productv1.LookupStorageByIdResponse.result:type_name -> productv1.Storage + 170, // 23: productv1.LookupStorageByIdResponse.result:type_name -> productv1.Storage 28, // 24: productv1.LookupWarehouseByIdRequest.keys:type_name -> productv1.LookupWarehouseByIdRequestKey - 162, // 25: productv1.LookupWarehouseByIdResponse.result:type_name -> productv1.Warehouse - 163, // 26: productv1.QueryUsersResponse.users:type_name -> productv1.User - 163, // 27: productv1.QueryUserResponse.user:type_name -> productv1.User - 164, // 28: productv1.QueryNestedTypeResponse.nested_type:type_name -> productv1.NestedTypeA - 165, // 29: productv1.QueryRecursiveTypeResponse.recursive_type:type_name -> productv1.RecursiveType - 166, // 30: productv1.QueryTypeFilterWithArgumentsResponse.type_filter_with_arguments:type_name -> productv1.TypeWithMultipleFilterFields - 167, // 31: productv1.QueryTypeWithMultipleFilterFieldsRequest.filter:type_name -> productv1.FilterTypeInput - 166, // 32: productv1.QueryTypeWithMultipleFilterFieldsResponse.type_with_multiple_filter_fields:type_name -> productv1.TypeWithMultipleFilterFields - 168, // 33: productv1.QueryComplexFilterTypeRequest.filter:type_name -> productv1.ComplexFilterTypeInput - 169, // 34: productv1.QueryComplexFilterTypeResponse.complex_filter_type:type_name -> productv1.TypeWithComplexFilterInput - 170, // 35: productv1.QueryCalculateTotalsRequest.orders:type_name -> productv1.OrderInput - 171, // 36: productv1.QueryCalculateTotalsResponse.calculate_totals:type_name -> productv1.Order - 172, // 37: productv1.QueryCategoriesResponse.categories:type_name -> productv1.Category + 171, // 25: productv1.LookupWarehouseByIdResponse.result:type_name -> productv1.Warehouse + 172, // 26: productv1.QueryUsersResponse.users:type_name -> productv1.User + 172, // 27: productv1.QueryUserResponse.user:type_name -> productv1.User + 173, // 28: productv1.QueryNestedTypeResponse.nested_type:type_name -> productv1.NestedTypeA + 174, // 29: productv1.QueryRecursiveTypeResponse.recursive_type:type_name -> productv1.RecursiveType + 175, // 30: productv1.QueryTypeFilterWithArgumentsResponse.type_filter_with_arguments:type_name -> productv1.TypeWithMultipleFilterFields + 176, // 31: productv1.QueryTypeWithMultipleFilterFieldsRequest.filter:type_name -> productv1.FilterTypeInput + 175, // 32: productv1.QueryTypeWithMultipleFilterFieldsResponse.type_with_multiple_filter_fields:type_name -> productv1.TypeWithMultipleFilterFields + 177, // 33: productv1.QueryComplexFilterTypeRequest.filter:type_name -> productv1.ComplexFilterTypeInput + 178, // 34: productv1.QueryComplexFilterTypeResponse.complex_filter_type:type_name -> productv1.TypeWithComplexFilterInput + 179, // 35: productv1.QueryCalculateTotalsRequest.orders:type_name -> productv1.OrderInput + 180, // 36: productv1.QueryCalculateTotalsResponse.calculate_totals:type_name -> productv1.Order + 181, // 37: productv1.QueryCategoriesResponse.categories:type_name -> productv1.Category 0, // 38: productv1.QueryCategoriesByKindRequest.kind:type_name -> productv1.CategoryKind - 172, // 39: productv1.QueryCategoriesByKindResponse.categories_by_kind:type_name -> productv1.Category + 181, // 39: productv1.QueryCategoriesByKindResponse.categories_by_kind:type_name -> productv1.Category 0, // 40: productv1.QueryCategoriesByKindsRequest.kinds:type_name -> productv1.CategoryKind - 172, // 41: productv1.QueryCategoriesByKindsResponse.categories_by_kinds:type_name -> productv1.Category - 173, // 42: productv1.QueryFilterCategoriesRequest.filter:type_name -> productv1.CategoryFilter - 172, // 43: productv1.QueryFilterCategoriesResponse.filter_categories:type_name -> productv1.Category - 174, // 44: productv1.QueryRandomPetResponse.random_pet:type_name -> productv1.Animal - 174, // 45: productv1.QueryAllPetsResponse.all_pets:type_name -> productv1.Animal - 175, // 46: productv1.QuerySearchRequest.input:type_name -> productv1.SearchInput - 176, // 47: productv1.QuerySearchResponse.search:type_name -> productv1.SearchResult - 176, // 48: productv1.QueryRandomSearchResultResponse.random_search_result:type_name -> productv1.SearchResult - 177, // 49: productv1.QueryNullableFieldsTypeResponse.nullable_fields_type:type_name -> productv1.NullableFieldsType - 177, // 50: productv1.QueryNullableFieldsTypeByIdResponse.nullable_fields_type_by_id:type_name -> productv1.NullableFieldsType - 178, // 51: productv1.QueryNullableFieldsTypeWithFilterRequest.filter:type_name -> productv1.NullableFieldsFilter - 177, // 52: productv1.QueryNullableFieldsTypeWithFilterResponse.nullable_fields_type_with_filter:type_name -> productv1.NullableFieldsType - 177, // 53: productv1.QueryAllNullableFieldsTypesResponse.all_nullable_fields_types:type_name -> productv1.NullableFieldsType - 179, // 54: productv1.QueryBlogPostResponse.blog_post:type_name -> productv1.BlogPost - 179, // 55: productv1.QueryBlogPostByIdResponse.blog_post_by_id:type_name -> productv1.BlogPost - 180, // 56: productv1.QueryBlogPostsWithFilterRequest.filter:type_name -> productv1.BlogPostFilter - 179, // 57: productv1.QueryBlogPostsWithFilterResponse.blog_posts_with_filter:type_name -> productv1.BlogPost - 179, // 58: productv1.QueryAllBlogPostsResponse.all_blog_posts:type_name -> productv1.BlogPost - 181, // 59: productv1.QueryAuthorResponse.author:type_name -> productv1.Author - 181, // 60: productv1.QueryAuthorByIdResponse.author_by_id:type_name -> productv1.Author - 182, // 61: productv1.QueryAuthorsWithFilterRequest.filter:type_name -> productv1.AuthorFilter - 181, // 62: productv1.QueryAuthorsWithFilterResponse.authors_with_filter:type_name -> productv1.Author - 181, // 63: productv1.QueryAllAuthorsResponse.all_authors:type_name -> productv1.Author + 181, // 41: productv1.QueryCategoriesByKindsResponse.categories_by_kinds:type_name -> productv1.Category + 182, // 42: productv1.QueryFilterCategoriesRequest.filter:type_name -> productv1.CategoryFilter + 181, // 43: productv1.QueryFilterCategoriesResponse.filter_categories:type_name -> productv1.Category + 183, // 44: productv1.QueryRandomPetResponse.random_pet:type_name -> productv1.Animal + 183, // 45: productv1.QueryAllPetsResponse.all_pets:type_name -> productv1.Animal + 184, // 46: productv1.QuerySearchRequest.input:type_name -> productv1.SearchInput + 185, // 47: productv1.QuerySearchResponse.search:type_name -> productv1.SearchResult + 185, // 48: productv1.QueryRandomSearchResultResponse.random_search_result:type_name -> productv1.SearchResult + 186, // 49: productv1.QueryNullableFieldsTypeResponse.nullable_fields_type:type_name -> productv1.NullableFieldsType + 186, // 50: productv1.QueryNullableFieldsTypeByIdResponse.nullable_fields_type_by_id:type_name -> productv1.NullableFieldsType + 187, // 51: productv1.QueryNullableFieldsTypeWithFilterRequest.filter:type_name -> productv1.NullableFieldsFilter + 186, // 52: productv1.QueryNullableFieldsTypeWithFilterResponse.nullable_fields_type_with_filter:type_name -> productv1.NullableFieldsType + 186, // 53: productv1.QueryAllNullableFieldsTypesResponse.all_nullable_fields_types:type_name -> productv1.NullableFieldsType + 188, // 54: productv1.QueryBlogPostResponse.blog_post:type_name -> productv1.BlogPost + 188, // 55: productv1.QueryBlogPostByIdResponse.blog_post_by_id:type_name -> productv1.BlogPost + 189, // 56: productv1.QueryBlogPostsWithFilterRequest.filter:type_name -> productv1.BlogPostFilter + 188, // 57: productv1.QueryBlogPostsWithFilterResponse.blog_posts_with_filter:type_name -> productv1.BlogPost + 188, // 58: productv1.QueryAllBlogPostsResponse.all_blog_posts:type_name -> productv1.BlogPost + 190, // 59: productv1.QueryAuthorResponse.author:type_name -> productv1.Author + 190, // 60: productv1.QueryAuthorByIdResponse.author_by_id:type_name -> productv1.Author + 191, // 61: productv1.QueryAuthorsWithFilterRequest.filter:type_name -> productv1.AuthorFilter + 190, // 62: productv1.QueryAuthorsWithFilterResponse.authors_with_filter:type_name -> productv1.Author + 190, // 63: productv1.QueryAllAuthorsResponse.all_authors:type_name -> productv1.Author 2, // 64: productv1.QueryBulkSearchAuthorsRequest.filters:type_name -> productv1.ListOfAuthorFilter - 181, // 65: productv1.QueryBulkSearchAuthorsResponse.bulk_search_authors:type_name -> productv1.Author + 190, // 65: productv1.QueryBulkSearchAuthorsResponse.bulk_search_authors:type_name -> productv1.Author 5, // 66: productv1.QueryBulkSearchBlogPostsRequest.filters:type_name -> productv1.ListOfBlogPostFilter - 179, // 67: productv1.QueryBulkSearchBlogPostsResponse.bulk_search_blog_posts:type_name -> productv1.BlogPost - 183, // 68: productv1.MutationCreateUserRequest.input:type_name -> productv1.UserInput - 163, // 69: productv1.MutationCreateUserResponse.create_user:type_name -> productv1.User - 184, // 70: productv1.MutationPerformActionRequest.input:type_name -> productv1.ActionInput - 185, // 71: productv1.MutationPerformActionResponse.perform_action:type_name -> productv1.ActionResult - 186, // 72: productv1.MutationCreateNullableFieldsTypeRequest.input:type_name -> productv1.NullableFieldsInput - 177, // 73: productv1.MutationCreateNullableFieldsTypeResponse.create_nullable_fields_type:type_name -> productv1.NullableFieldsType - 186, // 74: productv1.MutationUpdateNullableFieldsTypeRequest.input:type_name -> productv1.NullableFieldsInput - 177, // 75: productv1.MutationUpdateNullableFieldsTypeResponse.update_nullable_fields_type:type_name -> productv1.NullableFieldsType - 187, // 76: productv1.MutationCreateBlogPostRequest.input:type_name -> productv1.BlogPostInput - 179, // 77: productv1.MutationCreateBlogPostResponse.create_blog_post:type_name -> productv1.BlogPost - 187, // 78: productv1.MutationUpdateBlogPostRequest.input:type_name -> productv1.BlogPostInput - 179, // 79: productv1.MutationUpdateBlogPostResponse.update_blog_post:type_name -> productv1.BlogPost - 188, // 80: productv1.MutationCreateAuthorRequest.input:type_name -> productv1.AuthorInput - 181, // 81: productv1.MutationCreateAuthorResponse.create_author:type_name -> productv1.Author - 188, // 82: productv1.MutationUpdateAuthorRequest.input:type_name -> productv1.AuthorInput - 181, // 83: productv1.MutationUpdateAuthorResponse.update_author:type_name -> productv1.Author - 3, // 84: productv1.MutationBulkCreateAuthorsRequest.authors:type_name -> productv1.ListOfAuthorInput - 181, // 85: productv1.MutationBulkCreateAuthorsResponse.bulk_create_authors:type_name -> productv1.Author - 3, // 86: productv1.MutationBulkUpdateAuthorsRequest.authors:type_name -> productv1.ListOfAuthorInput - 181, // 87: productv1.MutationBulkUpdateAuthorsResponse.bulk_update_authors:type_name -> productv1.Author - 6, // 88: productv1.MutationBulkCreateBlogPostsRequest.blog_posts:type_name -> productv1.ListOfBlogPostInput - 179, // 89: productv1.MutationBulkCreateBlogPostsResponse.bulk_create_blog_posts:type_name -> productv1.BlogPost - 6, // 90: productv1.MutationBulkUpdateBlogPostsRequest.blog_posts:type_name -> productv1.ListOfBlogPostInput - 179, // 91: productv1.MutationBulkUpdateBlogPostsResponse.bulk_update_blog_posts:type_name -> productv1.BlogPost - 204, // 92: productv1.ResolveProductShippingEstimateArgs.input:type_name -> productv1.ShippingEstimateInput - 116, // 93: productv1.ResolveProductShippingEstimateRequest.context:type_name -> productv1.ResolveProductShippingEstimateContext - 115, // 94: productv1.ResolveProductShippingEstimateRequest.field_args:type_name -> productv1.ResolveProductShippingEstimateArgs - 118, // 95: productv1.ResolveProductShippingEstimateResponse.result:type_name -> productv1.ResolveProductShippingEstimateResult - 121, // 96: productv1.ResolveProductRecommendedCategoryRequest.context:type_name -> productv1.ResolveProductRecommendedCategoryContext - 120, // 97: productv1.ResolveProductRecommendedCategoryRequest.field_args:type_name -> productv1.ResolveProductRecommendedCategoryArgs - 172, // 98: productv1.ResolveProductRecommendedCategoryResult.recommended_category:type_name -> productv1.Category - 123, // 99: productv1.ResolveProductRecommendedCategoryResponse.result:type_name -> productv1.ResolveProductRecommendedCategoryResult - 202, // 100: productv1.ResolveCategoryProductCountArgs.filters:type_name -> productv1.ProductCountFilter - 126, // 101: productv1.ResolveCategoryProductCountRequest.context:type_name -> productv1.ResolveCategoryProductCountContext - 125, // 102: productv1.ResolveCategoryProductCountRequest.field_args:type_name -> productv1.ResolveCategoryProductCountArgs - 128, // 103: productv1.ResolveCategoryProductCountResponse.result:type_name -> productv1.ResolveCategoryProductCountResult - 225, // 104: productv1.ResolveCategoryPopularityScoreArgs.threshold:type_name -> google.protobuf.Int32Value - 131, // 105: productv1.ResolveCategoryPopularityScoreRequest.context:type_name -> productv1.ResolveCategoryPopularityScoreContext - 130, // 106: productv1.ResolveCategoryPopularityScoreRequest.field_args:type_name -> productv1.ResolveCategoryPopularityScoreArgs - 225, // 107: productv1.ResolveCategoryPopularityScoreResult.popularity_score:type_name -> google.protobuf.Int32Value - 133, // 108: productv1.ResolveCategoryPopularityScoreResponse.result:type_name -> productv1.ResolveCategoryPopularityScoreResult - 136, // 109: productv1.ResolveCategoryCategoryMetricsRequest.context:type_name -> productv1.ResolveCategoryCategoryMetricsContext - 135, // 110: productv1.ResolveCategoryCategoryMetricsRequest.field_args:type_name -> productv1.ResolveCategoryCategoryMetricsArgs - 196, // 111: productv1.ResolveCategoryCategoryMetricsResult.category_metrics:type_name -> productv1.CategoryMetrics - 138, // 112: productv1.ResolveCategoryCategoryMetricsResponse.result:type_name -> productv1.ResolveCategoryCategoryMetricsResult - 0, // 113: productv1.ResolveCategoryMascotContext.kind:type_name -> productv1.CategoryKind - 141, // 114: productv1.ResolveCategoryMascotRequest.context:type_name -> productv1.ResolveCategoryMascotContext - 140, // 115: productv1.ResolveCategoryMascotRequest.field_args:type_name -> productv1.ResolveCategoryMascotArgs - 174, // 116: productv1.ResolveCategoryMascotResult.mascot:type_name -> productv1.Animal - 143, // 117: productv1.ResolveCategoryMascotResponse.result:type_name -> productv1.ResolveCategoryMascotResult - 146, // 118: productv1.ResolveCategoryCategoryStatusRequest.context:type_name -> productv1.ResolveCategoryCategoryStatusContext - 145, // 119: productv1.ResolveCategoryCategoryStatusRequest.field_args:type_name -> productv1.ResolveCategoryCategoryStatusArgs - 185, // 120: productv1.ResolveCategoryCategoryStatusResult.category_status:type_name -> productv1.ActionResult - 148, // 121: productv1.ResolveCategoryCategoryStatusResponse.result:type_name -> productv1.ResolveCategoryCategoryStatusResult - 203, // 122: productv1.ResolveSubcategoryItemCountArgs.filters:type_name -> productv1.SubcategoryItemFilter - 151, // 123: productv1.ResolveSubcategoryItemCountRequest.context:type_name -> productv1.ResolveSubcategoryItemCountContext - 150, // 124: productv1.ResolveSubcategoryItemCountRequest.field_args:type_name -> productv1.ResolveSubcategoryItemCountArgs - 153, // 125: productv1.ResolveSubcategoryItemCountResponse.result:type_name -> productv1.ResolveSubcategoryItemCountResult - 156, // 126: productv1.ResolveCategoryMetricsNormalizedScoreRequest.context:type_name -> productv1.ResolveCategoryMetricsNormalizedScoreContext - 155, // 127: productv1.ResolveCategoryMetricsNormalizedScoreRequest.field_args:type_name -> productv1.ResolveCategoryMetricsNormalizedScoreArgs - 158, // 128: productv1.ResolveCategoryMetricsNormalizedScoreResponse.result:type_name -> productv1.ResolveCategoryMetricsNormalizedScoreResult - 189, // 129: productv1.NestedTypeA.b:type_name -> productv1.NestedTypeB - 165, // 130: productv1.RecursiveType.recursive_type:type_name -> productv1.RecursiveType - 191, // 131: productv1.ComplexFilterTypeInput.filter:type_name -> productv1.FilterType - 193, // 132: productv1.OrderInput.lines:type_name -> productv1.OrderLineInput - 16, // 133: productv1.Order.order_lines:type_name -> productv1.ListOfOrderLine - 0, // 134: productv1.Category.kind:type_name -> productv1.CategoryKind - 19, // 135: productv1.Category.subcategories:type_name -> productv1.ListOfSubcategory - 0, // 136: productv1.CategoryFilter.category:type_name -> productv1.CategoryKind - 192, // 137: productv1.CategoryFilter.pagination:type_name -> productv1.Pagination - 197, // 138: productv1.Animal.cat:type_name -> productv1.Cat - 198, // 139: productv1.Animal.dog:type_name -> productv1.Dog - 225, // 140: productv1.SearchInput.limit:type_name -> google.protobuf.Int32Value - 160, // 141: productv1.SearchResult.product:type_name -> productv1.Product - 163, // 142: productv1.SearchResult.user:type_name -> productv1.User - 172, // 143: productv1.SearchResult.category:type_name -> productv1.Category - 226, // 144: productv1.NullableFieldsType.optional_string:type_name -> google.protobuf.StringValue - 225, // 145: productv1.NullableFieldsType.optional_int:type_name -> google.protobuf.Int32Value - 227, // 146: productv1.NullableFieldsType.optional_float:type_name -> google.protobuf.DoubleValue - 228, // 147: productv1.NullableFieldsType.optional_boolean:type_name -> google.protobuf.BoolValue - 226, // 148: productv1.NullableFieldsFilter.name:type_name -> google.protobuf.StringValue - 226, // 149: productv1.NullableFieldsFilter.optional_string:type_name -> google.protobuf.StringValue - 228, // 150: productv1.NullableFieldsFilter.include_nulls:type_name -> google.protobuf.BoolValue - 18, // 151: productv1.BlogPost.optional_tags:type_name -> productv1.ListOfString - 18, // 152: productv1.BlogPost.keywords:type_name -> productv1.ListOfString - 10, // 153: productv1.BlogPost.ratings:type_name -> productv1.ListOfFloat - 7, // 154: productv1.BlogPost.is_published:type_name -> productv1.ListOfBoolean - 13, // 155: productv1.BlogPost.tag_groups:type_name -> productv1.ListOfListOfString - 13, // 156: productv1.BlogPost.related_topics:type_name -> productv1.ListOfListOfString - 13, // 157: productv1.BlogPost.comment_threads:type_name -> productv1.ListOfListOfString - 13, // 158: productv1.BlogPost.suggestions:type_name -> productv1.ListOfListOfString - 172, // 159: productv1.BlogPost.related_categories:type_name -> productv1.Category - 163, // 160: productv1.BlogPost.contributors:type_name -> productv1.User - 17, // 161: productv1.BlogPost.mentioned_products:type_name -> productv1.ListOfProduct - 20, // 162: productv1.BlogPost.mentioned_users:type_name -> productv1.ListOfUser - 11, // 163: productv1.BlogPost.category_groups:type_name -> productv1.ListOfListOfCategory - 14, // 164: productv1.BlogPost.contributor_teams:type_name -> productv1.ListOfListOfUser - 226, // 165: productv1.BlogPostFilter.title:type_name -> google.protobuf.StringValue - 228, // 166: productv1.BlogPostFilter.has_categories:type_name -> google.protobuf.BoolValue - 225, // 167: productv1.BlogPostFilter.min_tags:type_name -> google.protobuf.Int32Value - 226, // 168: productv1.Author.email:type_name -> google.protobuf.StringValue - 18, // 169: productv1.Author.social_links:type_name -> productv1.ListOfString - 13, // 170: productv1.Author.teams_by_project:type_name -> productv1.ListOfListOfString - 13, // 171: productv1.Author.collaborations:type_name -> productv1.ListOfListOfString - 4, // 172: productv1.Author.written_posts:type_name -> productv1.ListOfBlogPost - 172, // 173: productv1.Author.favorite_categories:type_name -> productv1.Category - 20, // 174: productv1.Author.related_authors:type_name -> productv1.ListOfUser - 17, // 175: productv1.Author.product_reviews:type_name -> productv1.ListOfProduct - 14, // 176: productv1.Author.author_groups:type_name -> productv1.ListOfListOfUser - 11, // 177: productv1.Author.category_preferences:type_name -> productv1.ListOfListOfCategory - 14, // 178: productv1.Author.project_teams:type_name -> productv1.ListOfListOfUser - 226, // 179: productv1.AuthorFilter.name:type_name -> google.protobuf.StringValue - 228, // 180: productv1.AuthorFilter.has_teams:type_name -> google.protobuf.BoolValue - 225, // 181: productv1.AuthorFilter.skill_count:type_name -> google.protobuf.Int32Value - 199, // 182: productv1.ActionResult.action_success:type_name -> productv1.ActionSuccess - 200, // 183: productv1.ActionResult.action_error:type_name -> productv1.ActionError - 226, // 184: productv1.NullableFieldsInput.optional_string:type_name -> google.protobuf.StringValue - 225, // 185: productv1.NullableFieldsInput.optional_int:type_name -> google.protobuf.Int32Value - 227, // 186: productv1.NullableFieldsInput.optional_float:type_name -> google.protobuf.DoubleValue - 228, // 187: productv1.NullableFieldsInput.optional_boolean:type_name -> google.protobuf.BoolValue - 18, // 188: productv1.BlogPostInput.optional_tags:type_name -> productv1.ListOfString - 18, // 189: productv1.BlogPostInput.keywords:type_name -> productv1.ListOfString - 10, // 190: productv1.BlogPostInput.ratings:type_name -> productv1.ListOfFloat - 7, // 191: productv1.BlogPostInput.is_published:type_name -> productv1.ListOfBoolean - 13, // 192: productv1.BlogPostInput.tag_groups:type_name -> productv1.ListOfListOfString - 13, // 193: productv1.BlogPostInput.related_topics:type_name -> productv1.ListOfListOfString - 13, // 194: productv1.BlogPostInput.comment_threads:type_name -> productv1.ListOfListOfString - 13, // 195: productv1.BlogPostInput.suggestions:type_name -> productv1.ListOfListOfString - 9, // 196: productv1.BlogPostInput.related_categories:type_name -> productv1.ListOfCategoryInput - 21, // 197: productv1.BlogPostInput.contributors:type_name -> productv1.ListOfUserInput - 12, // 198: productv1.BlogPostInput.category_groups:type_name -> productv1.ListOfListOfCategoryInput - 226, // 199: productv1.AuthorInput.email:type_name -> google.protobuf.StringValue - 18, // 200: productv1.AuthorInput.social_links:type_name -> productv1.ListOfString - 13, // 201: productv1.AuthorInput.teams_by_project:type_name -> productv1.ListOfListOfString - 13, // 202: productv1.AuthorInput.collaborations:type_name -> productv1.ListOfListOfString - 201, // 203: productv1.AuthorInput.favorite_categories:type_name -> productv1.CategoryInput - 15, // 204: productv1.AuthorInput.author_groups:type_name -> productv1.ListOfListOfUserInput - 15, // 205: productv1.AuthorInput.project_teams:type_name -> productv1.ListOfListOfUserInput - 190, // 206: productv1.NestedTypeB.c:type_name -> productv1.NestedTypeC - 192, // 207: productv1.FilterType.pagination:type_name -> productv1.Pagination - 18, // 208: productv1.OrderLineInput.modifiers:type_name -> productv1.ListOfString - 18, // 209: productv1.OrderLine.modifiers:type_name -> productv1.ListOfString - 226, // 210: productv1.Subcategory.description:type_name -> google.protobuf.StringValue - 172, // 211: productv1.CategoryMetrics.related_category:type_name -> productv1.Category - 0, // 212: productv1.CategoryInput.kind:type_name -> productv1.CategoryKind - 227, // 213: productv1.ProductCountFilter.min_price:type_name -> google.protobuf.DoubleValue - 227, // 214: productv1.ProductCountFilter.max_price:type_name -> google.protobuf.DoubleValue - 228, // 215: productv1.ProductCountFilter.in_stock:type_name -> google.protobuf.BoolValue - 226, // 216: productv1.ProductCountFilter.search_term:type_name -> google.protobuf.StringValue - 227, // 217: productv1.SubcategoryItemFilter.min_price:type_name -> google.protobuf.DoubleValue - 227, // 218: productv1.SubcategoryItemFilter.max_price:type_name -> google.protobuf.DoubleValue - 228, // 219: productv1.SubcategoryItemFilter.in_stock:type_name -> google.protobuf.BoolValue - 228, // 220: productv1.SubcategoryItemFilter.is_active:type_name -> google.protobuf.BoolValue - 226, // 221: productv1.SubcategoryItemFilter.search_term:type_name -> google.protobuf.StringValue - 1, // 222: productv1.ShippingEstimateInput.destination:type_name -> productv1.ShippingDestination - 228, // 223: productv1.ShippingEstimateInput.expedited:type_name -> google.protobuf.BoolValue - 182, // 224: productv1.ListOfAuthorFilter.List.items:type_name -> productv1.AuthorFilter - 188, // 225: productv1.ListOfAuthorInput.List.items:type_name -> productv1.AuthorInput - 179, // 226: productv1.ListOfBlogPost.List.items:type_name -> productv1.BlogPost - 180, // 227: productv1.ListOfBlogPostFilter.List.items:type_name -> productv1.BlogPostFilter - 187, // 228: productv1.ListOfBlogPostInput.List.items:type_name -> productv1.BlogPostInput - 172, // 229: productv1.ListOfCategory.List.items:type_name -> productv1.Category - 201, // 230: productv1.ListOfCategoryInput.List.items:type_name -> productv1.CategoryInput - 8, // 231: productv1.ListOfListOfCategory.List.items:type_name -> productv1.ListOfCategory - 9, // 232: productv1.ListOfListOfCategoryInput.List.items:type_name -> productv1.ListOfCategoryInput - 18, // 233: productv1.ListOfListOfString.List.items:type_name -> productv1.ListOfString - 20, // 234: productv1.ListOfListOfUser.List.items:type_name -> productv1.ListOfUser - 21, // 235: productv1.ListOfListOfUserInput.List.items:type_name -> productv1.ListOfUserInput - 194, // 236: productv1.ListOfOrderLine.List.items:type_name -> productv1.OrderLine - 160, // 237: productv1.ListOfProduct.List.items:type_name -> productv1.Product - 195, // 238: productv1.ListOfSubcategory.List.items:type_name -> productv1.Subcategory - 163, // 239: productv1.ListOfUser.List.items:type_name -> productv1.User - 183, // 240: productv1.ListOfUserInput.List.items:type_name -> productv1.UserInput - 23, // 241: productv1.ProductService.LookupProductById:input_type -> productv1.LookupProductByIdRequest - 26, // 242: productv1.ProductService.LookupStorageById:input_type -> productv1.LookupStorageByIdRequest - 29, // 243: productv1.ProductService.LookupWarehouseById:input_type -> productv1.LookupWarehouseByIdRequest - 107, // 244: productv1.ProductService.MutationBulkCreateAuthors:input_type -> productv1.MutationBulkCreateAuthorsRequest - 111, // 245: productv1.ProductService.MutationBulkCreateBlogPosts:input_type -> productv1.MutationBulkCreateBlogPostsRequest - 109, // 246: productv1.ProductService.MutationBulkUpdateAuthors:input_type -> productv1.MutationBulkUpdateAuthorsRequest - 113, // 247: productv1.ProductService.MutationBulkUpdateBlogPosts:input_type -> productv1.MutationBulkUpdateBlogPostsRequest - 103, // 248: productv1.ProductService.MutationCreateAuthor:input_type -> productv1.MutationCreateAuthorRequest - 99, // 249: productv1.ProductService.MutationCreateBlogPost:input_type -> productv1.MutationCreateBlogPostRequest - 95, // 250: productv1.ProductService.MutationCreateNullableFieldsType:input_type -> productv1.MutationCreateNullableFieldsTypeRequest - 91, // 251: productv1.ProductService.MutationCreateUser:input_type -> productv1.MutationCreateUserRequest - 93, // 252: productv1.ProductService.MutationPerformAction:input_type -> productv1.MutationPerformActionRequest - 105, // 253: productv1.ProductService.MutationUpdateAuthor:input_type -> productv1.MutationUpdateAuthorRequest - 101, // 254: productv1.ProductService.MutationUpdateBlogPost:input_type -> productv1.MutationUpdateBlogPostRequest - 97, // 255: productv1.ProductService.MutationUpdateNullableFieldsType:input_type -> productv1.MutationUpdateNullableFieldsTypeRequest - 85, // 256: productv1.ProductService.QueryAllAuthors:input_type -> productv1.QueryAllAuthorsRequest - 77, // 257: productv1.ProductService.QueryAllBlogPosts:input_type -> productv1.QueryAllBlogPostsRequest - 69, // 258: productv1.ProductService.QueryAllNullableFieldsTypes:input_type -> productv1.QueryAllNullableFieldsTypesRequest - 57, // 259: productv1.ProductService.QueryAllPets:input_type -> productv1.QueryAllPetsRequest - 79, // 260: productv1.ProductService.QueryAuthor:input_type -> productv1.QueryAuthorRequest - 81, // 261: productv1.ProductService.QueryAuthorById:input_type -> productv1.QueryAuthorByIdRequest - 83, // 262: productv1.ProductService.QueryAuthorsWithFilter:input_type -> productv1.QueryAuthorsWithFilterRequest - 71, // 263: productv1.ProductService.QueryBlogPost:input_type -> productv1.QueryBlogPostRequest - 73, // 264: productv1.ProductService.QueryBlogPostById:input_type -> productv1.QueryBlogPostByIdRequest - 75, // 265: productv1.ProductService.QueryBlogPostsWithFilter:input_type -> productv1.QueryBlogPostsWithFilterRequest - 87, // 266: productv1.ProductService.QueryBulkSearchAuthors:input_type -> productv1.QueryBulkSearchAuthorsRequest - 89, // 267: productv1.ProductService.QueryBulkSearchBlogPosts:input_type -> productv1.QueryBulkSearchBlogPostsRequest - 45, // 268: productv1.ProductService.QueryCalculateTotals:input_type -> productv1.QueryCalculateTotalsRequest - 47, // 269: productv1.ProductService.QueryCategories:input_type -> productv1.QueryCategoriesRequest - 49, // 270: productv1.ProductService.QueryCategoriesByKind:input_type -> productv1.QueryCategoriesByKindRequest - 51, // 271: productv1.ProductService.QueryCategoriesByKinds:input_type -> productv1.QueryCategoriesByKindsRequest - 43, // 272: productv1.ProductService.QueryComplexFilterType:input_type -> productv1.QueryComplexFilterTypeRequest - 53, // 273: productv1.ProductService.QueryFilterCategories:input_type -> productv1.QueryFilterCategoriesRequest - 35, // 274: productv1.ProductService.QueryNestedType:input_type -> productv1.QueryNestedTypeRequest - 63, // 275: productv1.ProductService.QueryNullableFieldsType:input_type -> productv1.QueryNullableFieldsTypeRequest - 65, // 276: productv1.ProductService.QueryNullableFieldsTypeById:input_type -> productv1.QueryNullableFieldsTypeByIdRequest - 67, // 277: productv1.ProductService.QueryNullableFieldsTypeWithFilter:input_type -> productv1.QueryNullableFieldsTypeWithFilterRequest - 55, // 278: productv1.ProductService.QueryRandomPet:input_type -> productv1.QueryRandomPetRequest - 61, // 279: productv1.ProductService.QueryRandomSearchResult:input_type -> productv1.QueryRandomSearchResultRequest - 37, // 280: productv1.ProductService.QueryRecursiveType:input_type -> productv1.QueryRecursiveTypeRequest - 59, // 281: productv1.ProductService.QuerySearch:input_type -> productv1.QuerySearchRequest - 39, // 282: productv1.ProductService.QueryTypeFilterWithArguments:input_type -> productv1.QueryTypeFilterWithArgumentsRequest - 41, // 283: productv1.ProductService.QueryTypeWithMultipleFilterFields:input_type -> productv1.QueryTypeWithMultipleFilterFieldsRequest - 33, // 284: productv1.ProductService.QueryUser:input_type -> productv1.QueryUserRequest - 31, // 285: productv1.ProductService.QueryUsers:input_type -> productv1.QueryUsersRequest - 137, // 286: productv1.ProductService.ResolveCategoryCategoryMetrics:input_type -> productv1.ResolveCategoryCategoryMetricsRequest - 147, // 287: productv1.ProductService.ResolveCategoryCategoryStatus:input_type -> productv1.ResolveCategoryCategoryStatusRequest - 142, // 288: productv1.ProductService.ResolveCategoryMascot:input_type -> productv1.ResolveCategoryMascotRequest - 157, // 289: productv1.ProductService.ResolveCategoryMetricsNormalizedScore:input_type -> productv1.ResolveCategoryMetricsNormalizedScoreRequest - 132, // 290: productv1.ProductService.ResolveCategoryPopularityScore:input_type -> productv1.ResolveCategoryPopularityScoreRequest - 127, // 291: productv1.ProductService.ResolveCategoryProductCount:input_type -> productv1.ResolveCategoryProductCountRequest - 122, // 292: productv1.ProductService.ResolveProductRecommendedCategory:input_type -> productv1.ResolveProductRecommendedCategoryRequest - 117, // 293: productv1.ProductService.ResolveProductShippingEstimate:input_type -> productv1.ResolveProductShippingEstimateRequest - 152, // 294: productv1.ProductService.ResolveSubcategoryItemCount:input_type -> productv1.ResolveSubcategoryItemCountRequest - 24, // 295: productv1.ProductService.LookupProductById:output_type -> productv1.LookupProductByIdResponse - 27, // 296: productv1.ProductService.LookupStorageById:output_type -> productv1.LookupStorageByIdResponse - 30, // 297: productv1.ProductService.LookupWarehouseById:output_type -> productv1.LookupWarehouseByIdResponse - 108, // 298: productv1.ProductService.MutationBulkCreateAuthors:output_type -> productv1.MutationBulkCreateAuthorsResponse - 112, // 299: productv1.ProductService.MutationBulkCreateBlogPosts:output_type -> productv1.MutationBulkCreateBlogPostsResponse - 110, // 300: productv1.ProductService.MutationBulkUpdateAuthors:output_type -> productv1.MutationBulkUpdateAuthorsResponse - 114, // 301: productv1.ProductService.MutationBulkUpdateBlogPosts:output_type -> productv1.MutationBulkUpdateBlogPostsResponse - 104, // 302: productv1.ProductService.MutationCreateAuthor:output_type -> productv1.MutationCreateAuthorResponse - 100, // 303: productv1.ProductService.MutationCreateBlogPost:output_type -> productv1.MutationCreateBlogPostResponse - 96, // 304: productv1.ProductService.MutationCreateNullableFieldsType:output_type -> productv1.MutationCreateNullableFieldsTypeResponse - 92, // 305: productv1.ProductService.MutationCreateUser:output_type -> productv1.MutationCreateUserResponse - 94, // 306: productv1.ProductService.MutationPerformAction:output_type -> productv1.MutationPerformActionResponse - 106, // 307: productv1.ProductService.MutationUpdateAuthor:output_type -> productv1.MutationUpdateAuthorResponse - 102, // 308: productv1.ProductService.MutationUpdateBlogPost:output_type -> productv1.MutationUpdateBlogPostResponse - 98, // 309: productv1.ProductService.MutationUpdateNullableFieldsType:output_type -> productv1.MutationUpdateNullableFieldsTypeResponse - 86, // 310: productv1.ProductService.QueryAllAuthors:output_type -> productv1.QueryAllAuthorsResponse - 78, // 311: productv1.ProductService.QueryAllBlogPosts:output_type -> productv1.QueryAllBlogPostsResponse - 70, // 312: productv1.ProductService.QueryAllNullableFieldsTypes:output_type -> productv1.QueryAllNullableFieldsTypesResponse - 58, // 313: productv1.ProductService.QueryAllPets:output_type -> productv1.QueryAllPetsResponse - 80, // 314: productv1.ProductService.QueryAuthor:output_type -> productv1.QueryAuthorResponse - 82, // 315: productv1.ProductService.QueryAuthorById:output_type -> productv1.QueryAuthorByIdResponse - 84, // 316: productv1.ProductService.QueryAuthorsWithFilter:output_type -> productv1.QueryAuthorsWithFilterResponse - 72, // 317: productv1.ProductService.QueryBlogPost:output_type -> productv1.QueryBlogPostResponse - 74, // 318: productv1.ProductService.QueryBlogPostById:output_type -> productv1.QueryBlogPostByIdResponse - 76, // 319: productv1.ProductService.QueryBlogPostsWithFilter:output_type -> productv1.QueryBlogPostsWithFilterResponse - 88, // 320: productv1.ProductService.QueryBulkSearchAuthors:output_type -> productv1.QueryBulkSearchAuthorsResponse - 90, // 321: productv1.ProductService.QueryBulkSearchBlogPosts:output_type -> productv1.QueryBulkSearchBlogPostsResponse - 46, // 322: productv1.ProductService.QueryCalculateTotals:output_type -> productv1.QueryCalculateTotalsResponse - 48, // 323: productv1.ProductService.QueryCategories:output_type -> productv1.QueryCategoriesResponse - 50, // 324: productv1.ProductService.QueryCategoriesByKind:output_type -> productv1.QueryCategoriesByKindResponse - 52, // 325: productv1.ProductService.QueryCategoriesByKinds:output_type -> productv1.QueryCategoriesByKindsResponse - 44, // 326: productv1.ProductService.QueryComplexFilterType:output_type -> productv1.QueryComplexFilterTypeResponse - 54, // 327: productv1.ProductService.QueryFilterCategories:output_type -> productv1.QueryFilterCategoriesResponse - 36, // 328: productv1.ProductService.QueryNestedType:output_type -> productv1.QueryNestedTypeResponse - 64, // 329: productv1.ProductService.QueryNullableFieldsType:output_type -> productv1.QueryNullableFieldsTypeResponse - 66, // 330: productv1.ProductService.QueryNullableFieldsTypeById:output_type -> productv1.QueryNullableFieldsTypeByIdResponse - 68, // 331: productv1.ProductService.QueryNullableFieldsTypeWithFilter:output_type -> productv1.QueryNullableFieldsTypeWithFilterResponse - 56, // 332: productv1.ProductService.QueryRandomPet:output_type -> productv1.QueryRandomPetResponse - 62, // 333: productv1.ProductService.QueryRandomSearchResult:output_type -> productv1.QueryRandomSearchResultResponse - 38, // 334: productv1.ProductService.QueryRecursiveType:output_type -> productv1.QueryRecursiveTypeResponse - 60, // 335: productv1.ProductService.QuerySearch:output_type -> productv1.QuerySearchResponse - 40, // 336: productv1.ProductService.QueryTypeFilterWithArguments:output_type -> productv1.QueryTypeFilterWithArgumentsResponse - 42, // 337: productv1.ProductService.QueryTypeWithMultipleFilterFields:output_type -> productv1.QueryTypeWithMultipleFilterFieldsResponse - 34, // 338: productv1.ProductService.QueryUser:output_type -> productv1.QueryUserResponse - 32, // 339: productv1.ProductService.QueryUsers:output_type -> productv1.QueryUsersResponse - 139, // 340: productv1.ProductService.ResolveCategoryCategoryMetrics:output_type -> productv1.ResolveCategoryCategoryMetricsResponse - 149, // 341: productv1.ProductService.ResolveCategoryCategoryStatus:output_type -> productv1.ResolveCategoryCategoryStatusResponse - 144, // 342: productv1.ProductService.ResolveCategoryMascot:output_type -> productv1.ResolveCategoryMascotResponse - 159, // 343: productv1.ProductService.ResolveCategoryMetricsNormalizedScore:output_type -> productv1.ResolveCategoryMetricsNormalizedScoreResponse - 134, // 344: productv1.ProductService.ResolveCategoryPopularityScore:output_type -> productv1.ResolveCategoryPopularityScoreResponse - 129, // 345: productv1.ProductService.ResolveCategoryProductCount:output_type -> productv1.ResolveCategoryProductCountResponse - 124, // 346: productv1.ProductService.ResolveProductRecommendedCategory:output_type -> productv1.ResolveProductRecommendedCategoryResponse - 119, // 347: productv1.ProductService.ResolveProductShippingEstimate:output_type -> productv1.ResolveProductShippingEstimateResponse - 154, // 348: productv1.ProductService.ResolveSubcategoryItemCount:output_type -> productv1.ResolveSubcategoryItemCountResponse - 295, // [295:349] is the sub-list for method output_type - 241, // [241:295] is the sub-list for method input_type - 241, // [241:241] is the sub-list for extension type_name - 241, // [241:241] is the sub-list for extension extendee - 0, // [0:241] is the sub-list for field type_name + 188, // 67: productv1.QueryBulkSearchBlogPostsResponse.bulk_search_blog_posts:type_name -> productv1.BlogPost + 192, // 68: productv1.QueryTestContainerResponse.test_container:type_name -> productv1.TestContainer + 192, // 69: productv1.QueryTestContainersResponse.test_containers:type_name -> productv1.TestContainer + 193, // 70: productv1.MutationCreateUserRequest.input:type_name -> productv1.UserInput + 172, // 71: productv1.MutationCreateUserResponse.create_user:type_name -> productv1.User + 194, // 72: productv1.MutationPerformActionRequest.input:type_name -> productv1.ActionInput + 195, // 73: productv1.MutationPerformActionResponse.perform_action:type_name -> productv1.ActionResult + 196, // 74: productv1.MutationCreateNullableFieldsTypeRequest.input:type_name -> productv1.NullableFieldsInput + 186, // 75: productv1.MutationCreateNullableFieldsTypeResponse.create_nullable_fields_type:type_name -> productv1.NullableFieldsType + 196, // 76: productv1.MutationUpdateNullableFieldsTypeRequest.input:type_name -> productv1.NullableFieldsInput + 186, // 77: productv1.MutationUpdateNullableFieldsTypeResponse.update_nullable_fields_type:type_name -> productv1.NullableFieldsType + 197, // 78: productv1.MutationCreateBlogPostRequest.input:type_name -> productv1.BlogPostInput + 188, // 79: productv1.MutationCreateBlogPostResponse.create_blog_post:type_name -> productv1.BlogPost + 197, // 80: productv1.MutationUpdateBlogPostRequest.input:type_name -> productv1.BlogPostInput + 188, // 81: productv1.MutationUpdateBlogPostResponse.update_blog_post:type_name -> productv1.BlogPost + 198, // 82: productv1.MutationCreateAuthorRequest.input:type_name -> productv1.AuthorInput + 190, // 83: productv1.MutationCreateAuthorResponse.create_author:type_name -> productv1.Author + 198, // 84: productv1.MutationUpdateAuthorRequest.input:type_name -> productv1.AuthorInput + 190, // 85: productv1.MutationUpdateAuthorResponse.update_author:type_name -> productv1.Author + 3, // 86: productv1.MutationBulkCreateAuthorsRequest.authors:type_name -> productv1.ListOfAuthorInput + 190, // 87: productv1.MutationBulkCreateAuthorsResponse.bulk_create_authors:type_name -> productv1.Author + 3, // 88: productv1.MutationBulkUpdateAuthorsRequest.authors:type_name -> productv1.ListOfAuthorInput + 190, // 89: productv1.MutationBulkUpdateAuthorsResponse.bulk_update_authors:type_name -> productv1.Author + 6, // 90: productv1.MutationBulkCreateBlogPostsRequest.blog_posts:type_name -> productv1.ListOfBlogPostInput + 188, // 91: productv1.MutationBulkCreateBlogPostsResponse.bulk_create_blog_posts:type_name -> productv1.BlogPost + 6, // 92: productv1.MutationBulkUpdateBlogPostsRequest.blog_posts:type_name -> productv1.ListOfBlogPostInput + 188, // 93: productv1.MutationBulkUpdateBlogPostsResponse.bulk_update_blog_posts:type_name -> productv1.BlogPost + 215, // 94: productv1.ResolveProductShippingEstimateArgs.input:type_name -> productv1.ShippingEstimateInput + 120, // 95: productv1.ResolveProductShippingEstimateRequest.context:type_name -> productv1.ResolveProductShippingEstimateContext + 119, // 96: productv1.ResolveProductShippingEstimateRequest.field_args:type_name -> productv1.ResolveProductShippingEstimateArgs + 122, // 97: productv1.ResolveProductShippingEstimateResponse.result:type_name -> productv1.ResolveProductShippingEstimateResult + 125, // 98: productv1.ResolveProductRecommendedCategoryRequest.context:type_name -> productv1.ResolveProductRecommendedCategoryContext + 124, // 99: productv1.ResolveProductRecommendedCategoryRequest.field_args:type_name -> productv1.ResolveProductRecommendedCategoryArgs + 181, // 100: productv1.ResolveProductRecommendedCategoryResult.recommended_category:type_name -> productv1.Category + 127, // 101: productv1.ResolveProductRecommendedCategoryResponse.result:type_name -> productv1.ResolveProductRecommendedCategoryResult + 213, // 102: productv1.ResolveCategoryProductCountArgs.filters:type_name -> productv1.ProductCountFilter + 130, // 103: productv1.ResolveCategoryProductCountRequest.context:type_name -> productv1.ResolveCategoryProductCountContext + 129, // 104: productv1.ResolveCategoryProductCountRequest.field_args:type_name -> productv1.ResolveCategoryProductCountArgs + 132, // 105: productv1.ResolveCategoryProductCountResponse.result:type_name -> productv1.ResolveCategoryProductCountResult + 236, // 106: productv1.ResolveCategoryPopularityScoreArgs.threshold:type_name -> google.protobuf.Int32Value + 135, // 107: productv1.ResolveCategoryPopularityScoreRequest.context:type_name -> productv1.ResolveCategoryPopularityScoreContext + 134, // 108: productv1.ResolveCategoryPopularityScoreRequest.field_args:type_name -> productv1.ResolveCategoryPopularityScoreArgs + 236, // 109: productv1.ResolveCategoryPopularityScoreResult.popularity_score:type_name -> google.protobuf.Int32Value + 137, // 110: productv1.ResolveCategoryPopularityScoreResponse.result:type_name -> productv1.ResolveCategoryPopularityScoreResult + 140, // 111: productv1.ResolveCategoryCategoryMetricsRequest.context:type_name -> productv1.ResolveCategoryCategoryMetricsContext + 139, // 112: productv1.ResolveCategoryCategoryMetricsRequest.field_args:type_name -> productv1.ResolveCategoryCategoryMetricsArgs + 206, // 113: productv1.ResolveCategoryCategoryMetricsResult.category_metrics:type_name -> productv1.CategoryMetrics + 142, // 114: productv1.ResolveCategoryCategoryMetricsResponse.result:type_name -> productv1.ResolveCategoryCategoryMetricsResult + 0, // 115: productv1.ResolveCategoryMascotContext.kind:type_name -> productv1.CategoryKind + 145, // 116: productv1.ResolveCategoryMascotRequest.context:type_name -> productv1.ResolveCategoryMascotContext + 144, // 117: productv1.ResolveCategoryMascotRequest.field_args:type_name -> productv1.ResolveCategoryMascotArgs + 183, // 118: productv1.ResolveCategoryMascotResult.mascot:type_name -> productv1.Animal + 147, // 119: productv1.ResolveCategoryMascotResponse.result:type_name -> productv1.ResolveCategoryMascotResult + 150, // 120: productv1.ResolveCategoryCategoryStatusRequest.context:type_name -> productv1.ResolveCategoryCategoryStatusContext + 149, // 121: productv1.ResolveCategoryCategoryStatusRequest.field_args:type_name -> productv1.ResolveCategoryCategoryStatusArgs + 195, // 122: productv1.ResolveCategoryCategoryStatusResult.category_status:type_name -> productv1.ActionResult + 152, // 123: productv1.ResolveCategoryCategoryStatusResponse.result:type_name -> productv1.ResolveCategoryCategoryStatusResult + 214, // 124: productv1.ResolveSubcategoryItemCountArgs.filters:type_name -> productv1.SubcategoryItemFilter + 155, // 125: productv1.ResolveSubcategoryItemCountRequest.context:type_name -> productv1.ResolveSubcategoryItemCountContext + 154, // 126: productv1.ResolveSubcategoryItemCountRequest.field_args:type_name -> productv1.ResolveSubcategoryItemCountArgs + 157, // 127: productv1.ResolveSubcategoryItemCountResponse.result:type_name -> productv1.ResolveSubcategoryItemCountResult + 160, // 128: productv1.ResolveCategoryMetricsNormalizedScoreRequest.context:type_name -> productv1.ResolveCategoryMetricsNormalizedScoreContext + 159, // 129: productv1.ResolveCategoryMetricsNormalizedScoreRequest.field_args:type_name -> productv1.ResolveCategoryMetricsNormalizedScoreArgs + 162, // 130: productv1.ResolveCategoryMetricsNormalizedScoreResponse.result:type_name -> productv1.ResolveCategoryMetricsNormalizedScoreResult + 165, // 131: productv1.ResolveTestContainerDetailsRequest.context:type_name -> productv1.ResolveTestContainerDetailsContext + 164, // 132: productv1.ResolveTestContainerDetailsRequest.field_args:type_name -> productv1.ResolveTestContainerDetailsArgs + 211, // 133: productv1.ResolveTestContainerDetailsResult.details:type_name -> productv1.TestDetails + 167, // 134: productv1.ResolveTestContainerDetailsResponse.result:type_name -> productv1.ResolveTestContainerDetailsResult + 199, // 135: productv1.NestedTypeA.b:type_name -> productv1.NestedTypeB + 174, // 136: productv1.RecursiveType.recursive_type:type_name -> productv1.RecursiveType + 201, // 137: productv1.ComplexFilterTypeInput.filter:type_name -> productv1.FilterType + 203, // 138: productv1.OrderInput.lines:type_name -> productv1.OrderLineInput + 16, // 139: productv1.Order.order_lines:type_name -> productv1.ListOfOrderLine + 0, // 140: productv1.Category.kind:type_name -> productv1.CategoryKind + 19, // 141: productv1.Category.subcategories:type_name -> productv1.ListOfSubcategory + 0, // 142: productv1.CategoryFilter.category:type_name -> productv1.CategoryKind + 202, // 143: productv1.CategoryFilter.pagination:type_name -> productv1.Pagination + 207, // 144: productv1.Animal.cat:type_name -> productv1.Cat + 208, // 145: productv1.Animal.dog:type_name -> productv1.Dog + 236, // 146: productv1.SearchInput.limit:type_name -> google.protobuf.Int32Value + 169, // 147: productv1.SearchResult.product:type_name -> productv1.Product + 172, // 148: productv1.SearchResult.user:type_name -> productv1.User + 181, // 149: productv1.SearchResult.category:type_name -> productv1.Category + 237, // 150: productv1.NullableFieldsType.optional_string:type_name -> google.protobuf.StringValue + 236, // 151: productv1.NullableFieldsType.optional_int:type_name -> google.protobuf.Int32Value + 238, // 152: productv1.NullableFieldsType.optional_float:type_name -> google.protobuf.DoubleValue + 239, // 153: productv1.NullableFieldsType.optional_boolean:type_name -> google.protobuf.BoolValue + 237, // 154: productv1.NullableFieldsFilter.name:type_name -> google.protobuf.StringValue + 237, // 155: productv1.NullableFieldsFilter.optional_string:type_name -> google.protobuf.StringValue + 239, // 156: productv1.NullableFieldsFilter.include_nulls:type_name -> google.protobuf.BoolValue + 18, // 157: productv1.BlogPost.optional_tags:type_name -> productv1.ListOfString + 18, // 158: productv1.BlogPost.keywords:type_name -> productv1.ListOfString + 10, // 159: productv1.BlogPost.ratings:type_name -> productv1.ListOfFloat + 7, // 160: productv1.BlogPost.is_published:type_name -> productv1.ListOfBoolean + 13, // 161: productv1.BlogPost.tag_groups:type_name -> productv1.ListOfListOfString + 13, // 162: productv1.BlogPost.related_topics:type_name -> productv1.ListOfListOfString + 13, // 163: productv1.BlogPost.comment_threads:type_name -> productv1.ListOfListOfString + 13, // 164: productv1.BlogPost.suggestions:type_name -> productv1.ListOfListOfString + 181, // 165: productv1.BlogPost.related_categories:type_name -> productv1.Category + 172, // 166: productv1.BlogPost.contributors:type_name -> productv1.User + 17, // 167: productv1.BlogPost.mentioned_products:type_name -> productv1.ListOfProduct + 20, // 168: productv1.BlogPost.mentioned_users:type_name -> productv1.ListOfUser + 11, // 169: productv1.BlogPost.category_groups:type_name -> productv1.ListOfListOfCategory + 14, // 170: productv1.BlogPost.contributor_teams:type_name -> productv1.ListOfListOfUser + 237, // 171: productv1.BlogPostFilter.title:type_name -> google.protobuf.StringValue + 239, // 172: productv1.BlogPostFilter.has_categories:type_name -> google.protobuf.BoolValue + 236, // 173: productv1.BlogPostFilter.min_tags:type_name -> google.protobuf.Int32Value + 237, // 174: productv1.Author.email:type_name -> google.protobuf.StringValue + 18, // 175: productv1.Author.social_links:type_name -> productv1.ListOfString + 13, // 176: productv1.Author.teams_by_project:type_name -> productv1.ListOfListOfString + 13, // 177: productv1.Author.collaborations:type_name -> productv1.ListOfListOfString + 4, // 178: productv1.Author.written_posts:type_name -> productv1.ListOfBlogPost + 181, // 179: productv1.Author.favorite_categories:type_name -> productv1.Category + 20, // 180: productv1.Author.related_authors:type_name -> productv1.ListOfUser + 17, // 181: productv1.Author.product_reviews:type_name -> productv1.ListOfProduct + 14, // 182: productv1.Author.author_groups:type_name -> productv1.ListOfListOfUser + 11, // 183: productv1.Author.category_preferences:type_name -> productv1.ListOfListOfCategory + 14, // 184: productv1.Author.project_teams:type_name -> productv1.ListOfListOfUser + 237, // 185: productv1.AuthorFilter.name:type_name -> google.protobuf.StringValue + 239, // 186: productv1.AuthorFilter.has_teams:type_name -> google.protobuf.BoolValue + 236, // 187: productv1.AuthorFilter.skill_count:type_name -> google.protobuf.Int32Value + 237, // 188: productv1.TestContainer.description:type_name -> google.protobuf.StringValue + 209, // 189: productv1.ActionResult.action_success:type_name -> productv1.ActionSuccess + 210, // 190: productv1.ActionResult.action_error:type_name -> productv1.ActionError + 237, // 191: productv1.NullableFieldsInput.optional_string:type_name -> google.protobuf.StringValue + 236, // 192: productv1.NullableFieldsInput.optional_int:type_name -> google.protobuf.Int32Value + 238, // 193: productv1.NullableFieldsInput.optional_float:type_name -> google.protobuf.DoubleValue + 239, // 194: productv1.NullableFieldsInput.optional_boolean:type_name -> google.protobuf.BoolValue + 18, // 195: productv1.BlogPostInput.optional_tags:type_name -> productv1.ListOfString + 18, // 196: productv1.BlogPostInput.keywords:type_name -> productv1.ListOfString + 10, // 197: productv1.BlogPostInput.ratings:type_name -> productv1.ListOfFloat + 7, // 198: productv1.BlogPostInput.is_published:type_name -> productv1.ListOfBoolean + 13, // 199: productv1.BlogPostInput.tag_groups:type_name -> productv1.ListOfListOfString + 13, // 200: productv1.BlogPostInput.related_topics:type_name -> productv1.ListOfListOfString + 13, // 201: productv1.BlogPostInput.comment_threads:type_name -> productv1.ListOfListOfString + 13, // 202: productv1.BlogPostInput.suggestions:type_name -> productv1.ListOfListOfString + 9, // 203: productv1.BlogPostInput.related_categories:type_name -> productv1.ListOfCategoryInput + 21, // 204: productv1.BlogPostInput.contributors:type_name -> productv1.ListOfUserInput + 12, // 205: productv1.BlogPostInput.category_groups:type_name -> productv1.ListOfListOfCategoryInput + 237, // 206: productv1.AuthorInput.email:type_name -> google.protobuf.StringValue + 18, // 207: productv1.AuthorInput.social_links:type_name -> productv1.ListOfString + 13, // 208: productv1.AuthorInput.teams_by_project:type_name -> productv1.ListOfListOfString + 13, // 209: productv1.AuthorInput.collaborations:type_name -> productv1.ListOfListOfString + 212, // 210: productv1.AuthorInput.favorite_categories:type_name -> productv1.CategoryInput + 15, // 211: productv1.AuthorInput.author_groups:type_name -> productv1.ListOfListOfUserInput + 15, // 212: productv1.AuthorInput.project_teams:type_name -> productv1.ListOfListOfUserInput + 200, // 213: productv1.NestedTypeB.c:type_name -> productv1.NestedTypeC + 202, // 214: productv1.FilterType.pagination:type_name -> productv1.Pagination + 18, // 215: productv1.OrderLineInput.modifiers:type_name -> productv1.ListOfString + 18, // 216: productv1.OrderLine.modifiers:type_name -> productv1.ListOfString + 237, // 217: productv1.Subcategory.description:type_name -> google.protobuf.StringValue + 181, // 218: productv1.CategoryMetrics.related_category:type_name -> productv1.Category + 183, // 219: productv1.TestDetails.pet:type_name -> productv1.Animal + 195, // 220: productv1.TestDetails.status:type_name -> productv1.ActionResult + 0, // 221: productv1.CategoryInput.kind:type_name -> productv1.CategoryKind + 238, // 222: productv1.ProductCountFilter.min_price:type_name -> google.protobuf.DoubleValue + 238, // 223: productv1.ProductCountFilter.max_price:type_name -> google.protobuf.DoubleValue + 239, // 224: productv1.ProductCountFilter.in_stock:type_name -> google.protobuf.BoolValue + 237, // 225: productv1.ProductCountFilter.search_term:type_name -> google.protobuf.StringValue + 238, // 226: productv1.SubcategoryItemFilter.min_price:type_name -> google.protobuf.DoubleValue + 238, // 227: productv1.SubcategoryItemFilter.max_price:type_name -> google.protobuf.DoubleValue + 239, // 228: productv1.SubcategoryItemFilter.in_stock:type_name -> google.protobuf.BoolValue + 239, // 229: productv1.SubcategoryItemFilter.is_active:type_name -> google.protobuf.BoolValue + 237, // 230: productv1.SubcategoryItemFilter.search_term:type_name -> google.protobuf.StringValue + 1, // 231: productv1.ShippingEstimateInput.destination:type_name -> productv1.ShippingDestination + 239, // 232: productv1.ShippingEstimateInput.expedited:type_name -> google.protobuf.BoolValue + 191, // 233: productv1.ListOfAuthorFilter.List.items:type_name -> productv1.AuthorFilter + 198, // 234: productv1.ListOfAuthorInput.List.items:type_name -> productv1.AuthorInput + 188, // 235: productv1.ListOfBlogPost.List.items:type_name -> productv1.BlogPost + 189, // 236: productv1.ListOfBlogPostFilter.List.items:type_name -> productv1.BlogPostFilter + 197, // 237: productv1.ListOfBlogPostInput.List.items:type_name -> productv1.BlogPostInput + 181, // 238: productv1.ListOfCategory.List.items:type_name -> productv1.Category + 212, // 239: productv1.ListOfCategoryInput.List.items:type_name -> productv1.CategoryInput + 8, // 240: productv1.ListOfListOfCategory.List.items:type_name -> productv1.ListOfCategory + 9, // 241: productv1.ListOfListOfCategoryInput.List.items:type_name -> productv1.ListOfCategoryInput + 18, // 242: productv1.ListOfListOfString.List.items:type_name -> productv1.ListOfString + 20, // 243: productv1.ListOfListOfUser.List.items:type_name -> productv1.ListOfUser + 21, // 244: productv1.ListOfListOfUserInput.List.items:type_name -> productv1.ListOfUserInput + 204, // 245: productv1.ListOfOrderLine.List.items:type_name -> productv1.OrderLine + 169, // 246: productv1.ListOfProduct.List.items:type_name -> productv1.Product + 205, // 247: productv1.ListOfSubcategory.List.items:type_name -> productv1.Subcategory + 172, // 248: productv1.ListOfUser.List.items:type_name -> productv1.User + 193, // 249: productv1.ListOfUserInput.List.items:type_name -> productv1.UserInput + 23, // 250: productv1.ProductService.LookupProductById:input_type -> productv1.LookupProductByIdRequest + 26, // 251: productv1.ProductService.LookupStorageById:input_type -> productv1.LookupStorageByIdRequest + 29, // 252: productv1.ProductService.LookupWarehouseById:input_type -> productv1.LookupWarehouseByIdRequest + 111, // 253: productv1.ProductService.MutationBulkCreateAuthors:input_type -> productv1.MutationBulkCreateAuthorsRequest + 115, // 254: productv1.ProductService.MutationBulkCreateBlogPosts:input_type -> productv1.MutationBulkCreateBlogPostsRequest + 113, // 255: productv1.ProductService.MutationBulkUpdateAuthors:input_type -> productv1.MutationBulkUpdateAuthorsRequest + 117, // 256: productv1.ProductService.MutationBulkUpdateBlogPosts:input_type -> productv1.MutationBulkUpdateBlogPostsRequest + 107, // 257: productv1.ProductService.MutationCreateAuthor:input_type -> productv1.MutationCreateAuthorRequest + 103, // 258: productv1.ProductService.MutationCreateBlogPost:input_type -> productv1.MutationCreateBlogPostRequest + 99, // 259: productv1.ProductService.MutationCreateNullableFieldsType:input_type -> productv1.MutationCreateNullableFieldsTypeRequest + 95, // 260: productv1.ProductService.MutationCreateUser:input_type -> productv1.MutationCreateUserRequest + 97, // 261: productv1.ProductService.MutationPerformAction:input_type -> productv1.MutationPerformActionRequest + 109, // 262: productv1.ProductService.MutationUpdateAuthor:input_type -> productv1.MutationUpdateAuthorRequest + 105, // 263: productv1.ProductService.MutationUpdateBlogPost:input_type -> productv1.MutationUpdateBlogPostRequest + 101, // 264: productv1.ProductService.MutationUpdateNullableFieldsType:input_type -> productv1.MutationUpdateNullableFieldsTypeRequest + 85, // 265: productv1.ProductService.QueryAllAuthors:input_type -> productv1.QueryAllAuthorsRequest + 77, // 266: productv1.ProductService.QueryAllBlogPosts:input_type -> productv1.QueryAllBlogPostsRequest + 69, // 267: productv1.ProductService.QueryAllNullableFieldsTypes:input_type -> productv1.QueryAllNullableFieldsTypesRequest + 57, // 268: productv1.ProductService.QueryAllPets:input_type -> productv1.QueryAllPetsRequest + 79, // 269: productv1.ProductService.QueryAuthor:input_type -> productv1.QueryAuthorRequest + 81, // 270: productv1.ProductService.QueryAuthorById:input_type -> productv1.QueryAuthorByIdRequest + 83, // 271: productv1.ProductService.QueryAuthorsWithFilter:input_type -> productv1.QueryAuthorsWithFilterRequest + 71, // 272: productv1.ProductService.QueryBlogPost:input_type -> productv1.QueryBlogPostRequest + 73, // 273: productv1.ProductService.QueryBlogPostById:input_type -> productv1.QueryBlogPostByIdRequest + 75, // 274: productv1.ProductService.QueryBlogPostsWithFilter:input_type -> productv1.QueryBlogPostsWithFilterRequest + 87, // 275: productv1.ProductService.QueryBulkSearchAuthors:input_type -> productv1.QueryBulkSearchAuthorsRequest + 89, // 276: productv1.ProductService.QueryBulkSearchBlogPosts:input_type -> productv1.QueryBulkSearchBlogPostsRequest + 45, // 277: productv1.ProductService.QueryCalculateTotals:input_type -> productv1.QueryCalculateTotalsRequest + 47, // 278: productv1.ProductService.QueryCategories:input_type -> productv1.QueryCategoriesRequest + 49, // 279: productv1.ProductService.QueryCategoriesByKind:input_type -> productv1.QueryCategoriesByKindRequest + 51, // 280: productv1.ProductService.QueryCategoriesByKinds:input_type -> productv1.QueryCategoriesByKindsRequest + 43, // 281: productv1.ProductService.QueryComplexFilterType:input_type -> productv1.QueryComplexFilterTypeRequest + 53, // 282: productv1.ProductService.QueryFilterCategories:input_type -> productv1.QueryFilterCategoriesRequest + 35, // 283: productv1.ProductService.QueryNestedType:input_type -> productv1.QueryNestedTypeRequest + 63, // 284: productv1.ProductService.QueryNullableFieldsType:input_type -> productv1.QueryNullableFieldsTypeRequest + 65, // 285: productv1.ProductService.QueryNullableFieldsTypeById:input_type -> productv1.QueryNullableFieldsTypeByIdRequest + 67, // 286: productv1.ProductService.QueryNullableFieldsTypeWithFilter:input_type -> productv1.QueryNullableFieldsTypeWithFilterRequest + 55, // 287: productv1.ProductService.QueryRandomPet:input_type -> productv1.QueryRandomPetRequest + 61, // 288: productv1.ProductService.QueryRandomSearchResult:input_type -> productv1.QueryRandomSearchResultRequest + 37, // 289: productv1.ProductService.QueryRecursiveType:input_type -> productv1.QueryRecursiveTypeRequest + 59, // 290: productv1.ProductService.QuerySearch:input_type -> productv1.QuerySearchRequest + 91, // 291: productv1.ProductService.QueryTestContainer:input_type -> productv1.QueryTestContainerRequest + 93, // 292: productv1.ProductService.QueryTestContainers:input_type -> productv1.QueryTestContainersRequest + 39, // 293: productv1.ProductService.QueryTypeFilterWithArguments:input_type -> productv1.QueryTypeFilterWithArgumentsRequest + 41, // 294: productv1.ProductService.QueryTypeWithMultipleFilterFields:input_type -> productv1.QueryTypeWithMultipleFilterFieldsRequest + 33, // 295: productv1.ProductService.QueryUser:input_type -> productv1.QueryUserRequest + 31, // 296: productv1.ProductService.QueryUsers:input_type -> productv1.QueryUsersRequest + 141, // 297: productv1.ProductService.ResolveCategoryCategoryMetrics:input_type -> productv1.ResolveCategoryCategoryMetricsRequest + 151, // 298: productv1.ProductService.ResolveCategoryCategoryStatus:input_type -> productv1.ResolveCategoryCategoryStatusRequest + 146, // 299: productv1.ProductService.ResolveCategoryMascot:input_type -> productv1.ResolveCategoryMascotRequest + 161, // 300: productv1.ProductService.ResolveCategoryMetricsNormalizedScore:input_type -> productv1.ResolveCategoryMetricsNormalizedScoreRequest + 136, // 301: productv1.ProductService.ResolveCategoryPopularityScore:input_type -> productv1.ResolveCategoryPopularityScoreRequest + 131, // 302: productv1.ProductService.ResolveCategoryProductCount:input_type -> productv1.ResolveCategoryProductCountRequest + 126, // 303: productv1.ProductService.ResolveProductRecommendedCategory:input_type -> productv1.ResolveProductRecommendedCategoryRequest + 121, // 304: productv1.ProductService.ResolveProductShippingEstimate:input_type -> productv1.ResolveProductShippingEstimateRequest + 156, // 305: productv1.ProductService.ResolveSubcategoryItemCount:input_type -> productv1.ResolveSubcategoryItemCountRequest + 166, // 306: productv1.ProductService.ResolveTestContainerDetails:input_type -> productv1.ResolveTestContainerDetailsRequest + 24, // 307: productv1.ProductService.LookupProductById:output_type -> productv1.LookupProductByIdResponse + 27, // 308: productv1.ProductService.LookupStorageById:output_type -> productv1.LookupStorageByIdResponse + 30, // 309: productv1.ProductService.LookupWarehouseById:output_type -> productv1.LookupWarehouseByIdResponse + 112, // 310: productv1.ProductService.MutationBulkCreateAuthors:output_type -> productv1.MutationBulkCreateAuthorsResponse + 116, // 311: productv1.ProductService.MutationBulkCreateBlogPosts:output_type -> productv1.MutationBulkCreateBlogPostsResponse + 114, // 312: productv1.ProductService.MutationBulkUpdateAuthors:output_type -> productv1.MutationBulkUpdateAuthorsResponse + 118, // 313: productv1.ProductService.MutationBulkUpdateBlogPosts:output_type -> productv1.MutationBulkUpdateBlogPostsResponse + 108, // 314: productv1.ProductService.MutationCreateAuthor:output_type -> productv1.MutationCreateAuthorResponse + 104, // 315: productv1.ProductService.MutationCreateBlogPost:output_type -> productv1.MutationCreateBlogPostResponse + 100, // 316: productv1.ProductService.MutationCreateNullableFieldsType:output_type -> productv1.MutationCreateNullableFieldsTypeResponse + 96, // 317: productv1.ProductService.MutationCreateUser:output_type -> productv1.MutationCreateUserResponse + 98, // 318: productv1.ProductService.MutationPerformAction:output_type -> productv1.MutationPerformActionResponse + 110, // 319: productv1.ProductService.MutationUpdateAuthor:output_type -> productv1.MutationUpdateAuthorResponse + 106, // 320: productv1.ProductService.MutationUpdateBlogPost:output_type -> productv1.MutationUpdateBlogPostResponse + 102, // 321: productv1.ProductService.MutationUpdateNullableFieldsType:output_type -> productv1.MutationUpdateNullableFieldsTypeResponse + 86, // 322: productv1.ProductService.QueryAllAuthors:output_type -> productv1.QueryAllAuthorsResponse + 78, // 323: productv1.ProductService.QueryAllBlogPosts:output_type -> productv1.QueryAllBlogPostsResponse + 70, // 324: productv1.ProductService.QueryAllNullableFieldsTypes:output_type -> productv1.QueryAllNullableFieldsTypesResponse + 58, // 325: productv1.ProductService.QueryAllPets:output_type -> productv1.QueryAllPetsResponse + 80, // 326: productv1.ProductService.QueryAuthor:output_type -> productv1.QueryAuthorResponse + 82, // 327: productv1.ProductService.QueryAuthorById:output_type -> productv1.QueryAuthorByIdResponse + 84, // 328: productv1.ProductService.QueryAuthorsWithFilter:output_type -> productv1.QueryAuthorsWithFilterResponse + 72, // 329: productv1.ProductService.QueryBlogPost:output_type -> productv1.QueryBlogPostResponse + 74, // 330: productv1.ProductService.QueryBlogPostById:output_type -> productv1.QueryBlogPostByIdResponse + 76, // 331: productv1.ProductService.QueryBlogPostsWithFilter:output_type -> productv1.QueryBlogPostsWithFilterResponse + 88, // 332: productv1.ProductService.QueryBulkSearchAuthors:output_type -> productv1.QueryBulkSearchAuthorsResponse + 90, // 333: productv1.ProductService.QueryBulkSearchBlogPosts:output_type -> productv1.QueryBulkSearchBlogPostsResponse + 46, // 334: productv1.ProductService.QueryCalculateTotals:output_type -> productv1.QueryCalculateTotalsResponse + 48, // 335: productv1.ProductService.QueryCategories:output_type -> productv1.QueryCategoriesResponse + 50, // 336: productv1.ProductService.QueryCategoriesByKind:output_type -> productv1.QueryCategoriesByKindResponse + 52, // 337: productv1.ProductService.QueryCategoriesByKinds:output_type -> productv1.QueryCategoriesByKindsResponse + 44, // 338: productv1.ProductService.QueryComplexFilterType:output_type -> productv1.QueryComplexFilterTypeResponse + 54, // 339: productv1.ProductService.QueryFilterCategories:output_type -> productv1.QueryFilterCategoriesResponse + 36, // 340: productv1.ProductService.QueryNestedType:output_type -> productv1.QueryNestedTypeResponse + 64, // 341: productv1.ProductService.QueryNullableFieldsType:output_type -> productv1.QueryNullableFieldsTypeResponse + 66, // 342: productv1.ProductService.QueryNullableFieldsTypeById:output_type -> productv1.QueryNullableFieldsTypeByIdResponse + 68, // 343: productv1.ProductService.QueryNullableFieldsTypeWithFilter:output_type -> productv1.QueryNullableFieldsTypeWithFilterResponse + 56, // 344: productv1.ProductService.QueryRandomPet:output_type -> productv1.QueryRandomPetResponse + 62, // 345: productv1.ProductService.QueryRandomSearchResult:output_type -> productv1.QueryRandomSearchResultResponse + 38, // 346: productv1.ProductService.QueryRecursiveType:output_type -> productv1.QueryRecursiveTypeResponse + 60, // 347: productv1.ProductService.QuerySearch:output_type -> productv1.QuerySearchResponse + 92, // 348: productv1.ProductService.QueryTestContainer:output_type -> productv1.QueryTestContainerResponse + 94, // 349: productv1.ProductService.QueryTestContainers:output_type -> productv1.QueryTestContainersResponse + 40, // 350: productv1.ProductService.QueryTypeFilterWithArguments:output_type -> productv1.QueryTypeFilterWithArgumentsResponse + 42, // 351: productv1.ProductService.QueryTypeWithMultipleFilterFields:output_type -> productv1.QueryTypeWithMultipleFilterFieldsResponse + 34, // 352: productv1.ProductService.QueryUser:output_type -> productv1.QueryUserResponse + 32, // 353: productv1.ProductService.QueryUsers:output_type -> productv1.QueryUsersResponse + 143, // 354: productv1.ProductService.ResolveCategoryCategoryMetrics:output_type -> productv1.ResolveCategoryCategoryMetricsResponse + 153, // 355: productv1.ProductService.ResolveCategoryCategoryStatus:output_type -> productv1.ResolveCategoryCategoryStatusResponse + 148, // 356: productv1.ProductService.ResolveCategoryMascot:output_type -> productv1.ResolveCategoryMascotResponse + 163, // 357: productv1.ProductService.ResolveCategoryMetricsNormalizedScore:output_type -> productv1.ResolveCategoryMetricsNormalizedScoreResponse + 138, // 358: productv1.ProductService.ResolveCategoryPopularityScore:output_type -> productv1.ResolveCategoryPopularityScoreResponse + 133, // 359: productv1.ProductService.ResolveCategoryProductCount:output_type -> productv1.ResolveCategoryProductCountResponse + 128, // 360: productv1.ProductService.ResolveProductRecommendedCategory:output_type -> productv1.ResolveProductRecommendedCategoryResponse + 123, // 361: productv1.ProductService.ResolveProductShippingEstimate:output_type -> productv1.ResolveProductShippingEstimateResponse + 158, // 362: productv1.ProductService.ResolveSubcategoryItemCount:output_type -> productv1.ResolveSubcategoryItemCountResponse + 168, // 363: productv1.ProductService.ResolveTestContainerDetails:output_type -> productv1.ResolveTestContainerDetailsResponse + 307, // [307:364] is the sub-list for method output_type + 250, // [250:307] is the sub-list for method input_type + 250, // [250:250] is the sub-list for extension type_name + 250, // [250:250] is the sub-list for extension extendee + 0, // [0:250] is the sub-list for field type_name } func init() { file_product_proto_init() } @@ -12778,16 +13374,16 @@ func file_product_proto_init() { if File_product_proto != nil { return } - file_product_proto_msgTypes[172].OneofWrappers = []any{ + file_product_proto_msgTypes[181].OneofWrappers = []any{ (*Animal_Cat)(nil), (*Animal_Dog)(nil), } - file_product_proto_msgTypes[174].OneofWrappers = []any{ + file_product_proto_msgTypes[183].OneofWrappers = []any{ (*SearchResult_Product)(nil), (*SearchResult_User)(nil), (*SearchResult_Category)(nil), } - file_product_proto_msgTypes[183].OneofWrappers = []any{ + file_product_proto_msgTypes[193].OneofWrappers = []any{ (*ActionResult_ActionSuccess)(nil), (*ActionResult_ActionError)(nil), } @@ -12797,7 +13393,7 @@ func file_product_proto_init() { GoPackagePath: reflect.TypeOf(x{}).PkgPath(), RawDescriptor: unsafe.Slice(unsafe.StringData(file_product_proto_rawDesc), len(file_product_proto_rawDesc)), NumEnums: 2, - NumMessages: 223, + NumMessages: 234, NumExtensions: 0, NumServices: 1, }, diff --git a/v2/pkg/grpctest/productv1/product_grpc.pb.go b/v2/pkg/grpctest/productv1/product_grpc.pb.go index 1d5a28249..d6aeee39e 100644 --- a/v2/pkg/grpctest/productv1/product_grpc.pb.go +++ b/v2/pkg/grpctest/productv1/product_grpc.pb.go @@ -60,6 +60,8 @@ const ( ProductService_QueryRandomSearchResult_FullMethodName = "/productv1.ProductService/QueryRandomSearchResult" ProductService_QueryRecursiveType_FullMethodName = "/productv1.ProductService/QueryRecursiveType" ProductService_QuerySearch_FullMethodName = "/productv1.ProductService/QuerySearch" + ProductService_QueryTestContainer_FullMethodName = "/productv1.ProductService/QueryTestContainer" + ProductService_QueryTestContainers_FullMethodName = "/productv1.ProductService/QueryTestContainers" ProductService_QueryTypeFilterWithArguments_FullMethodName = "/productv1.ProductService/QueryTypeFilterWithArguments" ProductService_QueryTypeWithMultipleFilterFields_FullMethodName = "/productv1.ProductService/QueryTypeWithMultipleFilterFields" ProductService_QueryUser_FullMethodName = "/productv1.ProductService/QueryUser" @@ -73,6 +75,7 @@ const ( ProductService_ResolveProductRecommendedCategory_FullMethodName = "/productv1.ProductService/ResolveProductRecommendedCategory" ProductService_ResolveProductShippingEstimate_FullMethodName = "/productv1.ProductService/ResolveProductShippingEstimate" ProductService_ResolveSubcategoryItemCount_FullMethodName = "/productv1.ProductService/ResolveSubcategoryItemCount" + ProductService_ResolveTestContainerDetails_FullMethodName = "/productv1.ProductService/ResolveTestContainerDetails" ) // ProductServiceClient is the client API for ProductService service. @@ -125,6 +128,8 @@ type ProductServiceClient interface { QueryRandomSearchResult(ctx context.Context, in *QueryRandomSearchResultRequest, opts ...grpc.CallOption) (*QueryRandomSearchResultResponse, error) QueryRecursiveType(ctx context.Context, in *QueryRecursiveTypeRequest, opts ...grpc.CallOption) (*QueryRecursiveTypeResponse, error) QuerySearch(ctx context.Context, in *QuerySearchRequest, opts ...grpc.CallOption) (*QuerySearchResponse, error) + QueryTestContainer(ctx context.Context, in *QueryTestContainerRequest, opts ...grpc.CallOption) (*QueryTestContainerResponse, error) + QueryTestContainers(ctx context.Context, in *QueryTestContainersRequest, opts ...grpc.CallOption) (*QueryTestContainersResponse, error) QueryTypeFilterWithArguments(ctx context.Context, in *QueryTypeFilterWithArgumentsRequest, opts ...grpc.CallOption) (*QueryTypeFilterWithArgumentsResponse, error) QueryTypeWithMultipleFilterFields(ctx context.Context, in *QueryTypeWithMultipleFilterFieldsRequest, opts ...grpc.CallOption) (*QueryTypeWithMultipleFilterFieldsResponse, error) QueryUser(ctx context.Context, in *QueryUserRequest, opts ...grpc.CallOption) (*QueryUserResponse, error) @@ -138,6 +143,7 @@ type ProductServiceClient interface { ResolveProductRecommendedCategory(ctx context.Context, in *ResolveProductRecommendedCategoryRequest, opts ...grpc.CallOption) (*ResolveProductRecommendedCategoryResponse, error) ResolveProductShippingEstimate(ctx context.Context, in *ResolveProductShippingEstimateRequest, opts ...grpc.CallOption) (*ResolveProductShippingEstimateResponse, error) ResolveSubcategoryItemCount(ctx context.Context, in *ResolveSubcategoryItemCountRequest, opts ...grpc.CallOption) (*ResolveSubcategoryItemCountResponse, error) + ResolveTestContainerDetails(ctx context.Context, in *ResolveTestContainerDetailsRequest, opts ...grpc.CallOption) (*ResolveTestContainerDetailsResponse, error) } type productServiceClient struct { @@ -558,6 +564,26 @@ func (c *productServiceClient) QuerySearch(ctx context.Context, in *QuerySearchR return out, nil } +func (c *productServiceClient) QueryTestContainer(ctx context.Context, in *QueryTestContainerRequest, opts ...grpc.CallOption) (*QueryTestContainerResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(QueryTestContainerResponse) + err := c.cc.Invoke(ctx, ProductService_QueryTestContainer_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *productServiceClient) QueryTestContainers(ctx context.Context, in *QueryTestContainersRequest, opts ...grpc.CallOption) (*QueryTestContainersResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(QueryTestContainersResponse) + err := c.cc.Invoke(ctx, ProductService_QueryTestContainers_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + func (c *productServiceClient) QueryTypeFilterWithArguments(ctx context.Context, in *QueryTypeFilterWithArgumentsRequest, opts ...grpc.CallOption) (*QueryTypeFilterWithArgumentsResponse, error) { cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) out := new(QueryTypeFilterWithArgumentsResponse) @@ -688,6 +714,16 @@ func (c *productServiceClient) ResolveSubcategoryItemCount(ctx context.Context, return out, nil } +func (c *productServiceClient) ResolveTestContainerDetails(ctx context.Context, in *ResolveTestContainerDetailsRequest, opts ...grpc.CallOption) (*ResolveTestContainerDetailsResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(ResolveTestContainerDetailsResponse) + err := c.cc.Invoke(ctx, ProductService_ResolveTestContainerDetails_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + // ProductServiceServer is the server API for ProductService service. // All implementations must embed UnimplementedProductServiceServer // for forward compatibility. @@ -738,6 +774,8 @@ type ProductServiceServer interface { QueryRandomSearchResult(context.Context, *QueryRandomSearchResultRequest) (*QueryRandomSearchResultResponse, error) QueryRecursiveType(context.Context, *QueryRecursiveTypeRequest) (*QueryRecursiveTypeResponse, error) QuerySearch(context.Context, *QuerySearchRequest) (*QuerySearchResponse, error) + QueryTestContainer(context.Context, *QueryTestContainerRequest) (*QueryTestContainerResponse, error) + QueryTestContainers(context.Context, *QueryTestContainersRequest) (*QueryTestContainersResponse, error) QueryTypeFilterWithArguments(context.Context, *QueryTypeFilterWithArgumentsRequest) (*QueryTypeFilterWithArgumentsResponse, error) QueryTypeWithMultipleFilterFields(context.Context, *QueryTypeWithMultipleFilterFieldsRequest) (*QueryTypeWithMultipleFilterFieldsResponse, error) QueryUser(context.Context, *QueryUserRequest) (*QueryUserResponse, error) @@ -751,6 +789,7 @@ type ProductServiceServer interface { ResolveProductRecommendedCategory(context.Context, *ResolveProductRecommendedCategoryRequest) (*ResolveProductRecommendedCategoryResponse, error) ResolveProductShippingEstimate(context.Context, *ResolveProductShippingEstimateRequest) (*ResolveProductShippingEstimateResponse, error) ResolveSubcategoryItemCount(context.Context, *ResolveSubcategoryItemCountRequest) (*ResolveSubcategoryItemCountResponse, error) + ResolveTestContainerDetails(context.Context, *ResolveTestContainerDetailsRequest) (*ResolveTestContainerDetailsResponse, error) mustEmbedUnimplementedProductServiceServer() } @@ -884,6 +923,12 @@ func (UnimplementedProductServiceServer) QueryRecursiveType(context.Context, *Qu func (UnimplementedProductServiceServer) QuerySearch(context.Context, *QuerySearchRequest) (*QuerySearchResponse, error) { return nil, status.Errorf(codes.Unimplemented, "method QuerySearch not implemented") } +func (UnimplementedProductServiceServer) QueryTestContainer(context.Context, *QueryTestContainerRequest) (*QueryTestContainerResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method QueryTestContainer not implemented") +} +func (UnimplementedProductServiceServer) QueryTestContainers(context.Context, *QueryTestContainersRequest) (*QueryTestContainersResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method QueryTestContainers not implemented") +} func (UnimplementedProductServiceServer) QueryTypeFilterWithArguments(context.Context, *QueryTypeFilterWithArgumentsRequest) (*QueryTypeFilterWithArgumentsResponse, error) { return nil, status.Errorf(codes.Unimplemented, "method QueryTypeFilterWithArguments not implemented") } @@ -923,6 +968,9 @@ func (UnimplementedProductServiceServer) ResolveProductShippingEstimate(context. func (UnimplementedProductServiceServer) ResolveSubcategoryItemCount(context.Context, *ResolveSubcategoryItemCountRequest) (*ResolveSubcategoryItemCountResponse, error) { return nil, status.Errorf(codes.Unimplemented, "method ResolveSubcategoryItemCount not implemented") } +func (UnimplementedProductServiceServer) ResolveTestContainerDetails(context.Context, *ResolveTestContainerDetailsRequest) (*ResolveTestContainerDetailsResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method ResolveTestContainerDetails not implemented") +} func (UnimplementedProductServiceServer) mustEmbedUnimplementedProductServiceServer() {} func (UnimplementedProductServiceServer) testEmbeddedByValue() {} @@ -1682,6 +1730,42 @@ func _ProductService_QuerySearch_Handler(srv interface{}, ctx context.Context, d return interceptor(ctx, in, info, handler) } +func _ProductService_QueryTestContainer_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(QueryTestContainerRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(ProductServiceServer).QueryTestContainer(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: ProductService_QueryTestContainer_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(ProductServiceServer).QueryTestContainer(ctx, req.(*QueryTestContainerRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _ProductService_QueryTestContainers_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(QueryTestContainersRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(ProductServiceServer).QueryTestContainers(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: ProductService_QueryTestContainers_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(ProductServiceServer).QueryTestContainers(ctx, req.(*QueryTestContainersRequest)) + } + return interceptor(ctx, in, info, handler) +} + func _ProductService_QueryTypeFilterWithArguments_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { in := new(QueryTypeFilterWithArgumentsRequest) if err := dec(in); err != nil { @@ -1916,6 +2000,24 @@ func _ProductService_ResolveSubcategoryItemCount_Handler(srv interface{}, ctx co return interceptor(ctx, in, info, handler) } +func _ProductService_ResolveTestContainerDetails_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(ResolveTestContainerDetailsRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(ProductServiceServer).ResolveTestContainerDetails(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: ProductService_ResolveTestContainerDetails_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(ProductServiceServer).ResolveTestContainerDetails(ctx, req.(*ResolveTestContainerDetailsRequest)) + } + return interceptor(ctx, in, info, handler) +} + // ProductService_ServiceDesc is the grpc.ServiceDesc for ProductService service. // It's only intended for direct use with grpc.RegisterService, // and not to be introspected or modified (even as a copy) @@ -2087,6 +2189,14 @@ var ProductService_ServiceDesc = grpc.ServiceDesc{ MethodName: "QuerySearch", Handler: _ProductService_QuerySearch_Handler, }, + { + MethodName: "QueryTestContainer", + Handler: _ProductService_QueryTestContainer_Handler, + }, + { + MethodName: "QueryTestContainers", + Handler: _ProductService_QueryTestContainers_Handler, + }, { MethodName: "QueryTypeFilterWithArguments", Handler: _ProductService_QueryTypeFilterWithArguments_Handler, @@ -2139,6 +2249,10 @@ var ProductService_ServiceDesc = grpc.ServiceDesc{ MethodName: "ResolveSubcategoryItemCount", Handler: _ProductService_ResolveSubcategoryItemCount_Handler, }, + { + MethodName: "ResolveTestContainerDetails", + Handler: _ProductService_ResolveTestContainerDetails_Handler, + }, }, Streams: []grpc.StreamDesc{}, Metadata: "product.proto", diff --git a/v2/pkg/grpctest/testdata/products.graphqls b/v2/pkg/grpctest/testdata/products.graphqls index b28ddbdf4..8e1008223 100644 --- a/v2/pkg/grpctest/testdata/products.graphqls +++ b/v2/pkg/grpctest/testdata/products.graphqls @@ -186,6 +186,21 @@ type ActionError { code: String! } +# Test container type for testing field resolvers with interface and union returns +type TestContainer { + id: ID! + name: String! + description: String + details(includeExtended: Boolean!): TestDetails @connect__fieldResolver(context: "id name") +} + +type TestDetails { + id: ID! + summary: String! + pet: Animal! + status: ActionResult! +} + # Input types for union operations input SearchInput { query: String! @@ -418,6 +433,10 @@ type Query { # Bulk search operations for blog posts bulkSearchBlogPosts(filters: [BlogPostFilter!]): [BlogPost!]! + + # Test container queries (testing field resolvers with interface and union returns) + testContainer(id: ID!): TestContainer! + testContainers: [TestContainer!]! } input UserInput { From acae81472869e3df5a8d859338587b8c56d88867 Mon Sep 17 00:00:00 2001 From: Ludwig Bedacht Date: Fri, 31 Oct 2025 16:04:16 +0100 Subject: [PATCH 46/72] feat: support composite types in the federation visitor --- .../grpc_datasource/execution_plan.go | 24 + .../execution_plan_federation_test.go | 643 +++++ .../execution_plan_field_resolvers_test.go | 2 +- .../grpc_datasource/execution_plan_visitor.go | 31 +- .../execution_plan_visitor_federation.go | 23 +- .../grpc_datasource/mapping_test_helper.go | 65 + v2/pkg/grpctest/mapping/mapping.go | 65 + v2/pkg/grpctest/mockservice.go | 175 ++ v2/pkg/grpctest/product.proto | 84 + v2/pkg/grpctest/productv1/product.pb.go | 2431 +++++++++++------ v2/pkg/grpctest/productv1/product_grpc.pb.go | 114 + v2/pkg/grpctest/testdata/products.graphqls | 10 + 12 files changed, 2864 insertions(+), 803 deletions(-) diff --git a/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go b/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go index d46ef6909..10c2acd08 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go +++ b/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go @@ -820,6 +820,30 @@ type fragmentSelection struct { selectionSetRef int } +// enterResolverCompositeSelectionSet handles logic when entering a composite selection set for a given field resolver. +// It appends the inline fragment selections to the resolved field and sets the fragment type. +func (r *rpcPlanningContext) enterResolverCompositeSelectionSet(oneOfType OneOfType, selectionSetRef int, resolvedField *resolvedField) { + inlineFragSelections := r.operation.SelectionSetInlineFragmentSelections(selectionSetRef) + if len(inlineFragSelections) == 0 { + return + } + + for _, inlineFragSelectionRef := range inlineFragSelections { + inlineFragRef := r.operation.Selections[inlineFragSelectionRef].Ref + inlinFragSelectionSetRef, ok := r.operation.InlineFragmentSelectionSet(inlineFragRef) + if !ok { + continue + } + + resolvedField.fragmentSelections = append(resolvedField.fragmentSelections, fragmentSelection{ + typeName: r.operation.InlineFragmentTypeConditionNameString(inlineFragRef), + selectionSetRef: inlinFragSelectionSetRef, + }) + } + + resolvedField.fragmentType = oneOfType +} + // isFieldResolver checks if a field is a field resolver. func (r *rpcPlanningContext) isFieldResolver(fieldRef int, isRootField bool) bool { if isRootField { diff --git a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_federation_test.go b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_federation_test.go index 21d0a8615..854f56155 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_federation_test.go +++ b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_federation_test.go @@ -1323,6 +1323,649 @@ func TestEntityLookupWithFieldResolvers(t *testing.T) { } } +func TestEntityLookupWithFieldResolvers_WithCompositeTypes(t *testing.T) { + t.Parallel() + tests := []struct { + name string + query string + expectedPlan *RPCExecutionPlan + mapping *GRPCMapping + federationConfigs plan.FederationFieldConfigurations + }{ + { + name: "Should create an execution plan for an entity lookup with a field resolver returning interface type", + query: `query EntityLookupWithInterface($representations: [_Any!]!, $includeDetails: Boolean!) { _entities(representations: $representations) { ... on Product { __typename id name mascotRecommendation(includeDetails: $includeDetails) { ... on Cat { name meowVolume } ... on Dog { name barkVolume } } } } }`, + mapping: testMapping(), + federationConfigs: plan.FederationFieldConfigurations{ + { + TypeName: "Product", + SelectionSet: "id", + }, + }, + expectedPlan: &RPCExecutionPlan{ + Calls: []RPCCall{ + { + ServiceName: "Products", + MethodName: "LookupProductById", + Kind: CallKindEntity, + Request: RPCMessage{ + Name: "LookupProductByIdRequest", + Fields: []RPCField{ + { + Name: "keys", + ProtoTypeName: DataTypeMessage, + Repeated: true, + JSONPath: "representations", + Message: &RPCMessage{ + Name: "LookupProductByIdKey", + MemberTypes: []string{"Product"}, + Fields: []RPCField{ + { + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", + }, + }, + }, + }, + }, + }, + Response: RPCMessage{ + Name: "LookupProductByIdResponse", + Fields: []RPCField{ + { + Name: "result", + ProtoTypeName: DataTypeMessage, + Repeated: true, + JSONPath: "_entities", + Message: &RPCMessage{ + Name: "Product", + Fields: []RPCField{ + { + Name: "__typename", + ProtoTypeName: DataTypeString, + JSONPath: "__typename", + StaticValue: "Product", + }, + { + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", + }, + { + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", + }, + }, + }, + }, + }, + }, + }, + { + ServiceName: "Products", + MethodName: "ResolveProductMascotRecommendation", + Kind: CallKindResolve, + DependentCalls: []int{0}, + ResponsePath: buildPath("_entities.mascotRecommendation"), + Request: RPCMessage{ + Name: "ResolveProductMascotRecommendationRequest", + Fields: []RPCField{ + { + Name: "context", + ProtoTypeName: DataTypeMessage, + JSONPath: "", + Repeated: true, + Message: &RPCMessage{ + Name: "ResolveProductMascotRecommendationContext", + Fields: []RPCField{ + { + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", + ResolvePath: buildPath("result.id"), + }, + { + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", + ResolvePath: buildPath("result.name"), + }, + }, + }, + }, + { + Name: "field_args", + ProtoTypeName: DataTypeMessage, + JSONPath: "", + Message: &RPCMessage{ + Name: "ResolveProductMascotRecommendationArgs", + Fields: []RPCField{ + { + Name: "include_details", + ProtoTypeName: DataTypeBool, + JSONPath: "includeDetails", + }, + }, + }, + }, + }, + }, + Response: RPCMessage{ + Name: "ResolveProductMascotRecommendationResponse", + Fields: []RPCField{ + { + Name: "result", + ProtoTypeName: DataTypeMessage, + JSONPath: "result", + Repeated: true, + Message: &RPCMessage{ + Name: "ResolveProductMascotRecommendationResult", + Fields: []RPCField{ + { + Name: "mascot_recommendation", + ProtoTypeName: DataTypeMessage, + JSONPath: "mascotRecommendation", + Optional: true, + Message: &RPCMessage{ + Name: "Animal", + OneOfType: OneOfTypeInterface, + MemberTypes: []string{"Cat", "Dog"}, + FieldSelectionSet: RPCFieldSelectionSet{ + "Cat": { + { + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", + }, + { + Name: "meowVolume", + ProtoTypeName: DataTypeInt32, + JSONPath: "meowVolume", + }, + }, + "Dog": { + { + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", + }, + { + Name: "barkVolume", + ProtoTypeName: DataTypeInt32, + JSONPath: "barkVolume", + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + { + name: "Should create an execution plan for an entity lookup with a field resolver returning union type", + query: `query EntityLookupWithUnion($representations: [_Any!]!, $checkAvailability: Boolean!) { _entities(representations: $representations) { ... on Product { __typename id name stockStatus(checkAvailability: $checkAvailability) { ... on ActionSuccess { message timestamp } ... on ActionError { message code } } } } }`, + mapping: testMapping(), + federationConfigs: plan.FederationFieldConfigurations{ + { + TypeName: "Product", + SelectionSet: "id", + }, + }, + expectedPlan: &RPCExecutionPlan{ + Calls: []RPCCall{ + { + ServiceName: "Products", + MethodName: "LookupProductById", + Kind: CallKindEntity, + Request: RPCMessage{ + Name: "LookupProductByIdRequest", + Fields: []RPCField{ + { + Name: "keys", + ProtoTypeName: DataTypeMessage, + Repeated: true, + JSONPath: "representations", + Message: &RPCMessage{ + Name: "LookupProductByIdKey", + MemberTypes: []string{"Product"}, + Fields: []RPCField{ + { + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", + }, + }, + }, + }, + }, + }, + Response: RPCMessage{ + Name: "LookupProductByIdResponse", + Fields: []RPCField{ + { + Name: "result", + ProtoTypeName: DataTypeMessage, + Repeated: true, + JSONPath: "_entities", + Message: &RPCMessage{ + Name: "Product", + Fields: []RPCField{ + { + Name: "__typename", + ProtoTypeName: DataTypeString, + JSONPath: "__typename", + StaticValue: "Product", + }, + { + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", + }, + { + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", + }, + }, + }, + }, + }, + }, + }, + { + ServiceName: "Products", + MethodName: "ResolveProductStockStatus", + Kind: CallKindResolve, + DependentCalls: []int{0}, + ResponsePath: buildPath("_entities.stockStatus"), + Request: RPCMessage{ + Name: "ResolveProductStockStatusRequest", + Fields: []RPCField{ + { + Name: "context", + ProtoTypeName: DataTypeMessage, + JSONPath: "", + Repeated: true, + Message: &RPCMessage{ + Name: "ResolveProductStockStatusContext", + Fields: []RPCField{ + { + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", + ResolvePath: buildPath("result.id"), + }, + { + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", + ResolvePath: buildPath("result.name"), + }, + { + Name: "price", + ProtoTypeName: DataTypeDouble, + JSONPath: "price", + ResolvePath: buildPath("result.price"), + }, + }, + }, + }, + { + Name: "field_args", + ProtoTypeName: DataTypeMessage, + JSONPath: "", + Message: &RPCMessage{ + Name: "ResolveProductStockStatusArgs", + Fields: []RPCField{ + { + Name: "check_availability", + ProtoTypeName: DataTypeBool, + JSONPath: "checkAvailability", + }, + }, + }, + }, + }, + }, + Response: RPCMessage{ + Name: "ResolveProductStockStatusResponse", + Fields: []RPCField{ + { + Name: "result", + ProtoTypeName: DataTypeMessage, + JSONPath: "result", + Repeated: true, + Message: &RPCMessage{ + Name: "ResolveProductStockStatusResult", + Fields: []RPCField{ + { + Name: "stock_status", + ProtoTypeName: DataTypeMessage, + JSONPath: "stockStatus", + Message: &RPCMessage{ + Name: "ActionResult", + OneOfType: OneOfTypeUnion, + MemberTypes: []string{"ActionSuccess", "ActionError"}, + FieldSelectionSet: RPCFieldSelectionSet{ + "ActionSuccess": { + { + Name: "message", + ProtoTypeName: DataTypeString, + JSONPath: "message", + }, + { + Name: "timestamp", + ProtoTypeName: DataTypeString, + JSONPath: "timestamp", + }, + }, + "ActionError": { + { + Name: "message", + ProtoTypeName: DataTypeString, + JSONPath: "message", + }, + { + Name: "code", + ProtoTypeName: DataTypeString, + JSONPath: "code", + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + { + name: "Should create an execution plan for an entity lookup with a field resolver returning nested composite types", + query: `query EntityLookupWithNested($representations: [_Any!]!, $includeExtended: Boolean!) { _entities(representations: $representations) { ... on Product { __typename id name price productDetails(includeExtended: $includeExtended) { id description recommendedPet { ... on Cat { name meowVolume } ... on Dog { name barkVolume } } reviewSummary { ... on ActionSuccess { message timestamp } ... on ActionError { message code } } } } } }`, + mapping: testMapping(), + federationConfigs: plan.FederationFieldConfigurations{ + { + TypeName: "Product", + SelectionSet: "id", + }, + }, + expectedPlan: &RPCExecutionPlan{ + Calls: []RPCCall{ + { + ServiceName: "Products", + MethodName: "LookupProductById", + Kind: CallKindEntity, + Request: RPCMessage{ + Name: "LookupProductByIdRequest", + Fields: []RPCField{ + { + Name: "keys", + ProtoTypeName: DataTypeMessage, + Repeated: true, + JSONPath: "representations", + Message: &RPCMessage{ + Name: "LookupProductByIdKey", + MemberTypes: []string{"Product"}, + Fields: []RPCField{ + { + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", + }, + }, + }, + }, + }, + }, + Response: RPCMessage{ + Name: "LookupProductByIdResponse", + Fields: []RPCField{ + { + Name: "result", + ProtoTypeName: DataTypeMessage, + Repeated: true, + JSONPath: "_entities", + Message: &RPCMessage{ + Name: "Product", + Fields: []RPCField{ + { + Name: "__typename", + ProtoTypeName: DataTypeString, + JSONPath: "__typename", + StaticValue: "Product", + }, + { + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", + }, + { + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", + }, + { + Name: "price", + ProtoTypeName: DataTypeDouble, + JSONPath: "price", + }, + }, + }, + }, + }, + }, + }, + { + ServiceName: "Products", + MethodName: "ResolveProductProductDetails", + Kind: CallKindResolve, + DependentCalls: []int{0}, + ResponsePath: buildPath("_entities.productDetails"), + Request: RPCMessage{ + Name: "ResolveProductProductDetailsRequest", + Fields: []RPCField{ + { + Name: "context", + ProtoTypeName: DataTypeMessage, + JSONPath: "", + Repeated: true, + Message: &RPCMessage{ + Name: "ResolveProductProductDetailsContext", + Fields: []RPCField{ + { + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", + ResolvePath: buildPath("result.id"), + }, + { + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", + ResolvePath: buildPath("result.name"), + }, + { + Name: "price", + ProtoTypeName: DataTypeDouble, + JSONPath: "price", + ResolvePath: buildPath("result.price"), + }, + }, + }, + }, + { + Name: "field_args", + ProtoTypeName: DataTypeMessage, + JSONPath: "", + Message: &RPCMessage{ + Name: "ResolveProductProductDetailsArgs", + Fields: []RPCField{ + { + Name: "include_extended", + ProtoTypeName: DataTypeBool, + JSONPath: "includeExtended", + }, + }, + }, + }, + }, + }, + Response: RPCMessage{ + Name: "ResolveProductProductDetailsResponse", + Fields: []RPCField{ + { + Name: "result", + ProtoTypeName: DataTypeMessage, + JSONPath: "result", + Repeated: true, + Message: &RPCMessage{ + Name: "ResolveProductProductDetailsResult", + Fields: []RPCField{ + { + Name: "product_details", + ProtoTypeName: DataTypeMessage, + JSONPath: "productDetails", + Optional: true, + Message: &RPCMessage{ + Name: "ProductDetails", + Fields: []RPCField{ + { + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", + }, + { + Name: "description", + ProtoTypeName: DataTypeString, + JSONPath: "description", + }, + { + Name: "recommended_pet", + ProtoTypeName: DataTypeMessage, + JSONPath: "recommendedPet", + Message: &RPCMessage{ + Name: "Animal", + OneOfType: OneOfTypeInterface, + MemberTypes: []string{"Cat", "Dog"}, + FieldSelectionSet: RPCFieldSelectionSet{ + "Cat": { + { + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", + }, + { + Name: "meow_volume", + ProtoTypeName: DataTypeInt32, + JSONPath: "meowVolume", + }, + }, + "Dog": { + { + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", + }, + { + Name: "bark_volume", + ProtoTypeName: DataTypeInt32, + JSONPath: "barkVolume", + }, + }, + }, + }, + }, + { + Name: "review_summary", + ProtoTypeName: DataTypeMessage, + JSONPath: "reviewSummary", + Message: &RPCMessage{ + Name: "ActionResult", + OneOfType: OneOfTypeUnion, + MemberTypes: []string{"ActionSuccess", "ActionError"}, + FieldSelectionSet: RPCFieldSelectionSet{ + "ActionSuccess": { + { + Name: "message", + ProtoTypeName: DataTypeString, + JSONPath: "message", + }, + { + Name: "timestamp", + ProtoTypeName: DataTypeString, + JSONPath: "timestamp", + }, + }, + "ActionError": { + { + Name: "message", + ProtoTypeName: DataTypeString, + JSONPath: "message", + }, + { + Name: "code", + ProtoTypeName: DataTypeString, + JSONPath: "code", + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + } + + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + t.Parallel() + // Parse the GraphQL schema + schemaDoc := grpctest.MustGraphQLSchema(t) + + // Parse the GraphQL query + queryDoc, report := astparser.ParseGraphqlDocumentString(tt.query) + if report.HasErrors() { + t.Fatalf("failed to parse query: %s", report.Error()) + } + + planner, err := NewPlanner("Products", tt.mapping, tt.federationConfigs) + if err != nil { + t.Fatalf("failed to create planner: %s", err) + } + plan, err := planner.PlanOperation(&queryDoc, &schemaDoc) + if err != nil { + t.Fatalf("failed to plan operation: %s", err) + } + + diff := cmp.Diff(tt.expectedPlan, plan) + if diff != "" { + t.Fatalf("execution plan mismatch: %s", diff) + } + }) + } +} + func runFederationTest(t *testing.T, tt struct { name string query string diff --git a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_field_resolvers_test.go b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_field_resolvers_test.go index d98060295..aa2b529f9 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_field_resolvers_test.go +++ b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_field_resolvers_test.go @@ -1110,7 +1110,7 @@ func TestExecutionPlanFieldResolvers_WithNestedResolvers(t *testing.T) { } } -func TestExecutionPlanFieldResolvers_WithOneOfTypes(t *testing.T) { +func TestExecutionPlanFieldResolvers_WithCompositeTypes(t *testing.T) { t.Parallel() tests := []struct { name string diff --git a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor.go b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor.go index d50f1c548..40ddfda92 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor.go +++ b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor.go @@ -189,30 +189,6 @@ func (r *rpcPlanVisitor) EnterArgument(ref int) { } -func (r *rpcPlanVisitor) enterResolverCompositeSelectionSet(oneOfType OneOfType, selectionSetRef int) { - inlineFragmentSelections := r.operation.SelectionSetInlineFragmentSelections(selectionSetRef) - if len(inlineFragmentSelections) == 0 { - return - } - - ancestor := r.fieldResolverAncestors.peek() - for _, inlineFragmentSelectionRef := range inlineFragmentSelections { - inlineFragmentRef := r.operation.Selections[inlineFragmentSelectionRef].Ref - ss, ok := r.operation.InlineFragmentSelectionSet(inlineFragmentRef) - if !ok { - continue - } - - r.resolvedFields[ancestor].fragmentSelections = append(r.resolvedFields[ancestor].fragmentSelections, fragmentSelection{ - typeName: r.operation.InlineFragmentTypeConditionNameString(inlineFragmentRef), - selectionSetRef: ss, - }) - } - - // r.resolvedFields[ancestor].fragmentSelections = append(r.resolvedFields[ancestor].fragmentSelections, *fragmentSelection) - r.resolvedFields[ancestor].fragmentType = oneOfType -} - // EnterSelectionSet implements astvisitor.EnterSelectionSetVisitor. // Checks if this is in the root level below the operation definition. func (r *rpcPlanVisitor) EnterSelectionSet(ref int) { @@ -233,10 +209,11 @@ func (r *rpcPlanVisitor) EnterSelectionSet(ref int) { r.walker.StopWithInternalErr(err) return } - r.resolvedFields[resolvedFieldAncestor].memberTypes = memberTypes - r.resolvedFields[resolvedFieldAncestor].fieldsSelectionSetRef = ast.InvalidRef + resolvedField := &r.resolvedFields[resolvedFieldAncestor] + resolvedField.memberTypes = memberTypes + resolvedField.fieldsSelectionSetRef = ast.InvalidRef - r.enterResolverCompositeSelectionSet(compositType, ref) + r.planCtx.enterResolverCompositeSelectionSet(compositType, ref, resolvedField) return } diff --git a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor_federation.go b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor_federation.go index 01e74e043..2b1a34157 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor_federation.go +++ b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor_federation.go @@ -188,7 +188,28 @@ func (r *rpcPlanVisitorFederation) EnterSelectionSet(ref int) { // If we are inside of a resolved field that selects multiple fields, we get all the fields from the input and pass them to the required fields visitor. if r.fieldResolverAncestors.len() > 0 { - r.resolvedFields[r.fieldResolverAncestors.peek()].fieldsSelectionSetRef = ref + if r.walker.Ancestor().Kind == ast.NodeKindInlineFragment { + return + } + + resolvedFieldAncestor := r.fieldResolverAncestors.peek() + if compositType := r.planCtx.getCompositeType(r.walker.EnclosingTypeDefinition); compositType != OneOfTypeNone { + memberTypes, err := r.planCtx.getMemberTypes(r.walker.EnclosingTypeDefinition) + if err != nil { + r.walker.StopWithInternalErr(err) + return + } + resolvedField := &r.resolvedFields[resolvedFieldAncestor] + + resolvedField.memberTypes = memberTypes + resolvedField.fieldsSelectionSetRef = ast.InvalidRef + + r.planCtx.enterResolverCompositeSelectionSet(compositType, ref, resolvedField) + return + } + + // TODO: handle nested resolved fields. + r.resolvedFields[resolvedFieldAncestor].fieldsSelectionSetRef = ref return } diff --git a/v2/pkg/engine/datasource/grpc_datasource/mapping_test_helper.go b/v2/pkg/engine/datasource/grpc_datasource/mapping_test_helper.go index e5387a130..aabc3e10e 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/mapping_test_helper.go +++ b/v2/pkg/engine/datasource/grpc_datasource/mapping_test_helper.go @@ -322,6 +322,39 @@ func testMapping() *GRPCMapping { Request: "ResolveProductRecommendedCategoryRequest", Response: "ResolveProductRecommendedCategoryResponse", }, + "mascotRecommendation": { + FieldMappingData: FieldMapData{ + TargetName: "mascot_recommendation", + ArgumentMappings: FieldArgumentMap{ + "includeDetails": "include_details", + }, + }, + RPC: "ResolveProductMascotRecommendation", + Request: "ResolveProductMascotRecommendationRequest", + Response: "ResolveProductMascotRecommendationResponse", + }, + "stockStatus": { + FieldMappingData: FieldMapData{ + TargetName: "stock_status", + ArgumentMappings: FieldArgumentMap{ + "checkAvailability": "check_availability", + }, + }, + RPC: "ResolveProductStockStatus", + Request: "ResolveProductStockStatusRequest", + Response: "ResolveProductStockStatusResponse", + }, + "productDetails": { + FieldMappingData: FieldMapData{ + TargetName: "product_details", + ArgumentMappings: FieldArgumentMap{ + "includeExtended": "include_extended", + }, + }, + RPC: "ResolveProductProductDetails", + Request: "ResolveProductProductDetailsRequest", + Response: "ResolveProductProductDetailsResponse", + }, }, "Subcategory": { "itemCount": { @@ -648,6 +681,38 @@ func testMapping() *GRPCMapping { "maxPrice": "max_price", }, }, + "mascotRecommendation": { + TargetName: "mascot_recommendation", + ArgumentMappings: FieldArgumentMap{ + "includeDetails": "include_details", + }, + }, + "stockStatus": { + TargetName: "stock_status", + ArgumentMappings: FieldArgumentMap{ + "checkAvailability": "check_availability", + }, + }, + "productDetails": { + TargetName: "product_details", + ArgumentMappings: FieldArgumentMap{ + "includeExtended": "include_extended", + }, + }, + }, + "ProductDetails": { + "id": { + TargetName: "id", + }, + "description": { + TargetName: "description", + }, + "reviewSummary": { + TargetName: "review_summary", + }, + "recommendedPet": { + TargetName: "recommended_pet", + }, }, "Storage": { "id": { diff --git a/v2/pkg/grpctest/mapping/mapping.go b/v2/pkg/grpctest/mapping/mapping.go index 26898ea22..7f7060237 100644 --- a/v2/pkg/grpctest/mapping/mapping.go +++ b/v2/pkg/grpctest/mapping/mapping.go @@ -329,6 +329,39 @@ func DefaultGRPCMapping() *grpcdatasource.GRPCMapping { Request: "ResolveProductRecommendedCategoryRequest", Response: "ResolveProductRecommendedCategoryResponse", }, + "mascotRecommendation": { + FieldMappingData: grpcdatasource.FieldMapData{ + TargetName: "mascot_recommendation", + ArgumentMappings: grpcdatasource.FieldArgumentMap{ + "includeDetails": "include_details", + }, + }, + RPC: "ResolveProductMascotRecommendation", + Request: "ResolveProductMascotRecommendationRequest", + Response: "ResolveProductMascotRecommendationResponse", + }, + "stockStatus": { + FieldMappingData: grpcdatasource.FieldMapData{ + TargetName: "stock_status", + ArgumentMappings: grpcdatasource.FieldArgumentMap{ + "checkAvailability": "check_availability", + }, + }, + RPC: "ResolveProductStockStatus", + Request: "ResolveProductStockStatusRequest", + Response: "ResolveProductStockStatusResponse", + }, + "productDetails": { + FieldMappingData: grpcdatasource.FieldMapData{ + TargetName: "product_details", + ArgumentMappings: grpcdatasource.FieldArgumentMap{ + "includeExtended": "include_extended", + }, + }, + RPC: "ResolveProductProductDetails", + Request: "ResolveProductProductDetailsRequest", + Response: "ResolveProductProductDetailsResponse", + }, }, "Subcategory": { "itemCount": { @@ -655,6 +688,38 @@ func DefaultGRPCMapping() *grpcdatasource.GRPCMapping { "maxPrice": "max_price", }, }, + "mascotRecommendation": { + TargetName: "mascot_recommendation", + ArgumentMappings: grpcdatasource.FieldArgumentMap{ + "includeDetails": "include_details", + }, + }, + "stockStatus": { + TargetName: "stock_status", + ArgumentMappings: grpcdatasource.FieldArgumentMap{ + "checkAvailability": "check_availability", + }, + }, + "productDetails": { + TargetName: "product_details", + ArgumentMappings: grpcdatasource.FieldArgumentMap{ + "includeExtended": "include_extended", + }, + }, + }, + "ProductDetails": { + "id": { + TargetName: "id", + }, + "description": { + TargetName: "description", + }, + "reviewSummary": { + TargetName: "review_summary", + }, + "recommendedPet": { + TargetName: "recommended_pet", + }, }, "Storage": { "id": { diff --git a/v2/pkg/grpctest/mockservice.go b/v2/pkg/grpctest/mockservice.go index 91dbaf434..99c19bad8 100644 --- a/v2/pkg/grpctest/mockservice.go +++ b/v2/pkg/grpctest/mockservice.go @@ -20,6 +20,181 @@ type MockService struct { productv1.UnimplementedProductServiceServer } +// ResolveProductMascotRecommendation implements productv1.ProductServiceServer. +func (s *MockService) ResolveProductMascotRecommendation(_ context.Context, req *productv1.ResolveProductMascotRecommendationRequest) (*productv1.ResolveProductMascotRecommendationResponse, error) { + results := make([]*productv1.ResolveProductMascotRecommendationResult, 0, len(req.GetContext())) + + includeDetails := false + if req.GetFieldArgs() != nil { + includeDetails = req.GetFieldArgs().GetIncludeDetails() + } + + for i, ctx := range req.GetContext() { + // Alternate between Cat and Dog based on index + var animal *productv1.Animal + if i%2 == 0 { + volume := int32(5) + if includeDetails { + volume = int32((i + 1) * 8) + } + animal = &productv1.Animal{ + Instance: &productv1.Animal_Cat{ + Cat: &productv1.Cat{ + Id: fmt.Sprintf("mascot-cat-%s", ctx.GetId()), + Name: fmt.Sprintf("MascotCat for %s", ctx.GetName()), + Kind: "Cat", + MeowVolume: volume, + }, + }, + } + } else { + volume := int32(7) + if includeDetails { + volume = int32((i + 1) * 10) + } + animal = &productv1.Animal{ + Instance: &productv1.Animal_Dog{ + Dog: &productv1.Dog{ + Id: fmt.Sprintf("mascot-dog-%s", ctx.GetId()), + Name: fmt.Sprintf("MascotDog for %s", ctx.GetName()), + Kind: "Dog", + BarkVolume: volume, + }, + }, + } + } + + results = append(results, &productv1.ResolveProductMascotRecommendationResult{ + MascotRecommendation: animal, + }) + } + + return &productv1.ResolveProductMascotRecommendationResponse{ + Result: results, + }, nil +} + +// ResolveProductProductDetails implements productv1.ProductServiceServer. +func (s *MockService) ResolveProductProductDetails(_ context.Context, req *productv1.ResolveProductProductDetailsRequest) (*productv1.ResolveProductProductDetailsResponse, error) { + results := make([]*productv1.ResolveProductProductDetailsResult, 0, len(req.GetContext())) + + includeExtended := false + if req.GetFieldArgs() != nil { + includeExtended = req.GetFieldArgs().GetIncludeExtended() + } + + for i, ctx := range req.GetContext() { + // Create recommended pet (alternate between Cat and Dog) + var pet *productv1.Animal + if i%2 == 0 { + pet = &productv1.Animal{ + Instance: &productv1.Animal_Cat{ + Cat: &productv1.Cat{ + Id: fmt.Sprintf("details-cat-%s", ctx.GetId()), + Name: fmt.Sprintf("RecommendedCat for %s", ctx.GetName()), + Kind: "Cat", + MeowVolume: int32((i + 1) * 6), + }, + }, + } + } else { + pet = &productv1.Animal{ + Instance: &productv1.Animal_Dog{ + Dog: &productv1.Dog{ + Id: fmt.Sprintf("details-dog-%s", ctx.GetId()), + Name: fmt.Sprintf("RecommendedDog for %s", ctx.GetName()), + Kind: "Dog", + BarkVolume: int32((i + 1) * 9), + }, + }, + } + } + + // Create review summary (alternate between success and error based on price and extended flag) + var reviewSummary *productv1.ActionResult + if includeExtended && ctx.GetPrice() > 500 { + reviewSummary = &productv1.ActionResult{ + Value: &productv1.ActionResult_ActionError{ + ActionError: &productv1.ActionError{ + Message: fmt.Sprintf("Product %s has negative reviews", ctx.GetName()), + Code: "NEGATIVE_REVIEWS", + }, + }, + } + } else { + reviewSummary = &productv1.ActionResult{ + Value: &productv1.ActionResult_ActionSuccess{ + ActionSuccess: &productv1.ActionSuccess{ + Message: fmt.Sprintf("Product %s has positive reviews", ctx.GetName()), + Timestamp: "2024-01-01T15:00:00Z", + }, + }, + } + } + + description := fmt.Sprintf("Standard details for %s", ctx.GetName()) + if includeExtended { + description = fmt.Sprintf("Extended details for %s with comprehensive information", ctx.GetName()) + } + + results = append(results, &productv1.ResolveProductProductDetailsResult{ + ProductDetails: &productv1.ProductDetails{ + Id: fmt.Sprintf("details-%s-%d", ctx.GetId(), i), + Description: description, + ReviewSummary: reviewSummary, + RecommendedPet: pet, + }, + }) + } + + return &productv1.ResolveProductProductDetailsResponse{ + Result: results, + }, nil +} + +// ResolveProductStockStatus implements productv1.ProductServiceServer. +func (s *MockService) ResolveProductStockStatus(_ context.Context, req *productv1.ResolveProductStockStatusRequest) (*productv1.ResolveProductStockStatusResponse, error) { + results := make([]*productv1.ResolveProductStockStatusResult, 0, len(req.GetContext())) + + checkAvailability := false + if req.GetFieldArgs() != nil { + checkAvailability = req.GetFieldArgs().GetCheckAvailability() + } + + for i, ctx := range req.GetContext() { + var stockStatus *productv1.ActionResult + + // If checking availability and price is high, return out of stock error + if checkAvailability && ctx.GetPrice() > 300 && i%2 == 0 { + stockStatus = &productv1.ActionResult{ + Value: &productv1.ActionResult_ActionError{ + ActionError: &productv1.ActionError{ + Message: fmt.Sprintf("Product %s is currently out of stock", ctx.GetName()), + Code: "OUT_OF_STOCK", + }, + }, + } + } else { + stockStatus = &productv1.ActionResult{ + Value: &productv1.ActionResult_ActionSuccess{ + ActionSuccess: &productv1.ActionSuccess{ + Message: fmt.Sprintf("Product %s is in stock and available", ctx.GetName()), + Timestamp: "2024-01-01T10:00:00Z", + }, + }, + } + } + + results = append(results, &productv1.ResolveProductStockStatusResult{ + StockStatus: stockStatus, + }) + } + + return &productv1.ResolveProductStockStatusResponse{ + Result: results, + }, nil +} + // QueryTestContainer implements productv1.ProductServiceServer. func (s *MockService) QueryTestContainer(_ context.Context, req *productv1.QueryTestContainerRequest) (*productv1.QueryTestContainerResponse, error) { id := req.GetId() diff --git a/v2/pkg/grpctest/product.proto b/v2/pkg/grpctest/product.proto index 7f1fdb9f8..480a1c3ea 100644 --- a/v2/pkg/grpctest/product.proto +++ b/v2/pkg/grpctest/product.proto @@ -63,8 +63,11 @@ service ProductService { rpc ResolveCategoryMetricsNormalizedScore(ResolveCategoryMetricsNormalizedScoreRequest) returns (ResolveCategoryMetricsNormalizedScoreResponse) {} rpc ResolveCategoryPopularityScore(ResolveCategoryPopularityScoreRequest) returns (ResolveCategoryPopularityScoreResponse) {} rpc ResolveCategoryProductCount(ResolveCategoryProductCountRequest) returns (ResolveCategoryProductCountResponse) {} + rpc ResolveProductMascotRecommendation(ResolveProductMascotRecommendationRequest) returns (ResolveProductMascotRecommendationResponse) {} + rpc ResolveProductProductDetails(ResolveProductProductDetailsRequest) returns (ResolveProductProductDetailsResponse) {} rpc ResolveProductRecommendedCategory(ResolveProductRecommendedCategoryRequest) returns (ResolveProductRecommendedCategoryResponse) {} rpc ResolveProductShippingEstimate(ResolveProductShippingEstimateRequest) returns (ResolveProductShippingEstimateResponse) {} + rpc ResolveProductStockStatus(ResolveProductStockStatusRequest) returns (ResolveProductStockStatusResponse) {} rpc ResolveSubcategoryItemCount(ResolveSubcategoryItemCountRequest) returns (ResolveSubcategoryItemCountResponse) {} rpc ResolveTestContainerDetails(ResolveTestContainerDetailsRequest) returns (ResolveTestContainerDetailsResponse) {} } @@ -702,6 +705,80 @@ message ResolveProductRecommendedCategoryResponse { repeated ResolveProductRecommendedCategoryResult result = 1; } +message ResolveProductMascotRecommendationArgs { + bool include_details = 1; +} + +message ResolveProductMascotRecommendationContext { + string id = 1; + string name = 2; +} + +message ResolveProductMascotRecommendationRequest { + // context provides the resolver context for the field mascotRecommendation of type Product. + repeated ResolveProductMascotRecommendationContext context = 1; + // field_args provides the arguments for the resolver field mascotRecommendation of type Product. + ResolveProductMascotRecommendationArgs field_args = 2; +} + +message ResolveProductMascotRecommendationResult { + Animal mascot_recommendation = 1; +} + +message ResolveProductMascotRecommendationResponse { + repeated ResolveProductMascotRecommendationResult result = 1; +} + +message ResolveProductStockStatusArgs { + bool check_availability = 1; +} + +message ResolveProductStockStatusContext { + string id = 1; + string name = 2; + double price = 3; +} + +message ResolveProductStockStatusRequest { + // context provides the resolver context for the field stockStatus of type Product. + repeated ResolveProductStockStatusContext context = 1; + // field_args provides the arguments for the resolver field stockStatus of type Product. + ResolveProductStockStatusArgs field_args = 2; +} + +message ResolveProductStockStatusResult { + ActionResult stock_status = 1; +} + +message ResolveProductStockStatusResponse { + repeated ResolveProductStockStatusResult result = 1; +} + +message ResolveProductProductDetailsArgs { + bool include_extended = 1; +} + +message ResolveProductProductDetailsContext { + string id = 1; + string name = 2; + double price = 3; +} + +message ResolveProductProductDetailsRequest { + // context provides the resolver context for the field productDetails of type Product. + repeated ResolveProductProductDetailsContext context = 1; + // field_args provides the arguments for the resolver field productDetails of type Product. + ResolveProductProductDetailsArgs field_args = 2; +} + +message ResolveProductProductDetailsResult { + ProductDetails product_details = 1; +} + +message ResolveProductProductDetailsResponse { + repeated ResolveProductProductDetailsResult result = 1; +} + message ResolveCategoryProductCountArgs { ProductCountFilter filters = 1; } @@ -1128,6 +1205,13 @@ message AuthorInput { ListOfListOfUserInput project_teams = 10; } +message ProductDetails { + string id = 1; + string description = 2; + ActionResult review_summary = 3; + Animal recommended_pet = 4; +} + message NestedTypeB { string id = 1; string name = 2; diff --git a/v2/pkg/grpctest/productv1/product.pb.go b/v2/pkg/grpctest/productv1/product.pb.go index 7b48740ba..57b62207e 100644 --- a/v2/pkg/grpctest/productv1/product.pb.go +++ b/v2/pkg/grpctest/productv1/product.pb.go @@ -5846,6 +5846,736 @@ func (x *ResolveProductRecommendedCategoryResponse) GetResult() []*ResolveProduc return nil } +type ResolveProductMascotRecommendationArgs struct { + state protoimpl.MessageState `protogen:"open.v1"` + IncludeDetails bool `protobuf:"varint,1,opt,name=include_details,json=includeDetails,proto3" json:"include_details,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ResolveProductMascotRecommendationArgs) Reset() { + *x = ResolveProductMascotRecommendationArgs{} + mi := &file_product_proto_msgTypes[127] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ResolveProductMascotRecommendationArgs) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ResolveProductMascotRecommendationArgs) ProtoMessage() {} + +func (x *ResolveProductMascotRecommendationArgs) ProtoReflect() protoreflect.Message { + mi := &file_product_proto_msgTypes[127] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ResolveProductMascotRecommendationArgs.ProtoReflect.Descriptor instead. +func (*ResolveProductMascotRecommendationArgs) Descriptor() ([]byte, []int) { + return file_product_proto_rawDescGZIP(), []int{127} +} + +func (x *ResolveProductMascotRecommendationArgs) GetIncludeDetails() bool { + if x != nil { + return x.IncludeDetails + } + return false +} + +type ResolveProductMascotRecommendationContext struct { + state protoimpl.MessageState `protogen:"open.v1"` + Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` + Name string `protobuf:"bytes,2,opt,name=name,proto3" json:"name,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ResolveProductMascotRecommendationContext) Reset() { + *x = ResolveProductMascotRecommendationContext{} + mi := &file_product_proto_msgTypes[128] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ResolveProductMascotRecommendationContext) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ResolveProductMascotRecommendationContext) ProtoMessage() {} + +func (x *ResolveProductMascotRecommendationContext) ProtoReflect() protoreflect.Message { + mi := &file_product_proto_msgTypes[128] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ResolveProductMascotRecommendationContext.ProtoReflect.Descriptor instead. +func (*ResolveProductMascotRecommendationContext) Descriptor() ([]byte, []int) { + return file_product_proto_rawDescGZIP(), []int{128} +} + +func (x *ResolveProductMascotRecommendationContext) GetId() string { + if x != nil { + return x.Id + } + return "" +} + +func (x *ResolveProductMascotRecommendationContext) GetName() string { + if x != nil { + return x.Name + } + return "" +} + +type ResolveProductMascotRecommendationRequest struct { + state protoimpl.MessageState `protogen:"open.v1"` + // context provides the resolver context for the field mascotRecommendation of type Product. + Context []*ResolveProductMascotRecommendationContext `protobuf:"bytes,1,rep,name=context,proto3" json:"context,omitempty"` + // field_args provides the arguments for the resolver field mascotRecommendation of type Product. + FieldArgs *ResolveProductMascotRecommendationArgs `protobuf:"bytes,2,opt,name=field_args,json=fieldArgs,proto3" json:"field_args,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ResolveProductMascotRecommendationRequest) Reset() { + *x = ResolveProductMascotRecommendationRequest{} + mi := &file_product_proto_msgTypes[129] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ResolveProductMascotRecommendationRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ResolveProductMascotRecommendationRequest) ProtoMessage() {} + +func (x *ResolveProductMascotRecommendationRequest) ProtoReflect() protoreflect.Message { + mi := &file_product_proto_msgTypes[129] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ResolveProductMascotRecommendationRequest.ProtoReflect.Descriptor instead. +func (*ResolveProductMascotRecommendationRequest) Descriptor() ([]byte, []int) { + return file_product_proto_rawDescGZIP(), []int{129} +} + +func (x *ResolveProductMascotRecommendationRequest) GetContext() []*ResolveProductMascotRecommendationContext { + if x != nil { + return x.Context + } + return nil +} + +func (x *ResolveProductMascotRecommendationRequest) GetFieldArgs() *ResolveProductMascotRecommendationArgs { + if x != nil { + return x.FieldArgs + } + return nil +} + +type ResolveProductMascotRecommendationResult struct { + state protoimpl.MessageState `protogen:"open.v1"` + MascotRecommendation *Animal `protobuf:"bytes,1,opt,name=mascot_recommendation,json=mascotRecommendation,proto3" json:"mascot_recommendation,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ResolveProductMascotRecommendationResult) Reset() { + *x = ResolveProductMascotRecommendationResult{} + mi := &file_product_proto_msgTypes[130] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ResolveProductMascotRecommendationResult) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ResolveProductMascotRecommendationResult) ProtoMessage() {} + +func (x *ResolveProductMascotRecommendationResult) ProtoReflect() protoreflect.Message { + mi := &file_product_proto_msgTypes[130] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ResolveProductMascotRecommendationResult.ProtoReflect.Descriptor instead. +func (*ResolveProductMascotRecommendationResult) Descriptor() ([]byte, []int) { + return file_product_proto_rawDescGZIP(), []int{130} +} + +func (x *ResolveProductMascotRecommendationResult) GetMascotRecommendation() *Animal { + if x != nil { + return x.MascotRecommendation + } + return nil +} + +type ResolveProductMascotRecommendationResponse struct { + state protoimpl.MessageState `protogen:"open.v1"` + Result []*ResolveProductMascotRecommendationResult `protobuf:"bytes,1,rep,name=result,proto3" json:"result,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ResolveProductMascotRecommendationResponse) Reset() { + *x = ResolveProductMascotRecommendationResponse{} + mi := &file_product_proto_msgTypes[131] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ResolveProductMascotRecommendationResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ResolveProductMascotRecommendationResponse) ProtoMessage() {} + +func (x *ResolveProductMascotRecommendationResponse) ProtoReflect() protoreflect.Message { + mi := &file_product_proto_msgTypes[131] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ResolveProductMascotRecommendationResponse.ProtoReflect.Descriptor instead. +func (*ResolveProductMascotRecommendationResponse) Descriptor() ([]byte, []int) { + return file_product_proto_rawDescGZIP(), []int{131} +} + +func (x *ResolveProductMascotRecommendationResponse) GetResult() []*ResolveProductMascotRecommendationResult { + if x != nil { + return x.Result + } + return nil +} + +type ResolveProductStockStatusArgs struct { + state protoimpl.MessageState `protogen:"open.v1"` + CheckAvailability bool `protobuf:"varint,1,opt,name=check_availability,json=checkAvailability,proto3" json:"check_availability,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ResolveProductStockStatusArgs) Reset() { + *x = ResolveProductStockStatusArgs{} + mi := &file_product_proto_msgTypes[132] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ResolveProductStockStatusArgs) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ResolveProductStockStatusArgs) ProtoMessage() {} + +func (x *ResolveProductStockStatusArgs) ProtoReflect() protoreflect.Message { + mi := &file_product_proto_msgTypes[132] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ResolveProductStockStatusArgs.ProtoReflect.Descriptor instead. +func (*ResolveProductStockStatusArgs) Descriptor() ([]byte, []int) { + return file_product_proto_rawDescGZIP(), []int{132} +} + +func (x *ResolveProductStockStatusArgs) GetCheckAvailability() bool { + if x != nil { + return x.CheckAvailability + } + return false +} + +type ResolveProductStockStatusContext struct { + state protoimpl.MessageState `protogen:"open.v1"` + Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` + Name string `protobuf:"bytes,2,opt,name=name,proto3" json:"name,omitempty"` + Price float64 `protobuf:"fixed64,3,opt,name=price,proto3" json:"price,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ResolveProductStockStatusContext) Reset() { + *x = ResolveProductStockStatusContext{} + mi := &file_product_proto_msgTypes[133] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ResolveProductStockStatusContext) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ResolveProductStockStatusContext) ProtoMessage() {} + +func (x *ResolveProductStockStatusContext) ProtoReflect() protoreflect.Message { + mi := &file_product_proto_msgTypes[133] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ResolveProductStockStatusContext.ProtoReflect.Descriptor instead. +func (*ResolveProductStockStatusContext) Descriptor() ([]byte, []int) { + return file_product_proto_rawDescGZIP(), []int{133} +} + +func (x *ResolveProductStockStatusContext) GetId() string { + if x != nil { + return x.Id + } + return "" +} + +func (x *ResolveProductStockStatusContext) GetName() string { + if x != nil { + return x.Name + } + return "" +} + +func (x *ResolveProductStockStatusContext) GetPrice() float64 { + if x != nil { + return x.Price + } + return 0 +} + +type ResolveProductStockStatusRequest struct { + state protoimpl.MessageState `protogen:"open.v1"` + // context provides the resolver context for the field stockStatus of type Product. + Context []*ResolveProductStockStatusContext `protobuf:"bytes,1,rep,name=context,proto3" json:"context,omitempty"` + // field_args provides the arguments for the resolver field stockStatus of type Product. + FieldArgs *ResolveProductStockStatusArgs `protobuf:"bytes,2,opt,name=field_args,json=fieldArgs,proto3" json:"field_args,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ResolveProductStockStatusRequest) Reset() { + *x = ResolveProductStockStatusRequest{} + mi := &file_product_proto_msgTypes[134] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ResolveProductStockStatusRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ResolveProductStockStatusRequest) ProtoMessage() {} + +func (x *ResolveProductStockStatusRequest) ProtoReflect() protoreflect.Message { + mi := &file_product_proto_msgTypes[134] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ResolveProductStockStatusRequest.ProtoReflect.Descriptor instead. +func (*ResolveProductStockStatusRequest) Descriptor() ([]byte, []int) { + return file_product_proto_rawDescGZIP(), []int{134} +} + +func (x *ResolveProductStockStatusRequest) GetContext() []*ResolveProductStockStatusContext { + if x != nil { + return x.Context + } + return nil +} + +func (x *ResolveProductStockStatusRequest) GetFieldArgs() *ResolveProductStockStatusArgs { + if x != nil { + return x.FieldArgs + } + return nil +} + +type ResolveProductStockStatusResult struct { + state protoimpl.MessageState `protogen:"open.v1"` + StockStatus *ActionResult `protobuf:"bytes,1,opt,name=stock_status,json=stockStatus,proto3" json:"stock_status,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ResolveProductStockStatusResult) Reset() { + *x = ResolveProductStockStatusResult{} + mi := &file_product_proto_msgTypes[135] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ResolveProductStockStatusResult) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ResolveProductStockStatusResult) ProtoMessage() {} + +func (x *ResolveProductStockStatusResult) ProtoReflect() protoreflect.Message { + mi := &file_product_proto_msgTypes[135] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ResolveProductStockStatusResult.ProtoReflect.Descriptor instead. +func (*ResolveProductStockStatusResult) Descriptor() ([]byte, []int) { + return file_product_proto_rawDescGZIP(), []int{135} +} + +func (x *ResolveProductStockStatusResult) GetStockStatus() *ActionResult { + if x != nil { + return x.StockStatus + } + return nil +} + +type ResolveProductStockStatusResponse struct { + state protoimpl.MessageState `protogen:"open.v1"` + Result []*ResolveProductStockStatusResult `protobuf:"bytes,1,rep,name=result,proto3" json:"result,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ResolveProductStockStatusResponse) Reset() { + *x = ResolveProductStockStatusResponse{} + mi := &file_product_proto_msgTypes[136] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ResolveProductStockStatusResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ResolveProductStockStatusResponse) ProtoMessage() {} + +func (x *ResolveProductStockStatusResponse) ProtoReflect() protoreflect.Message { + mi := &file_product_proto_msgTypes[136] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ResolveProductStockStatusResponse.ProtoReflect.Descriptor instead. +func (*ResolveProductStockStatusResponse) Descriptor() ([]byte, []int) { + return file_product_proto_rawDescGZIP(), []int{136} +} + +func (x *ResolveProductStockStatusResponse) GetResult() []*ResolveProductStockStatusResult { + if x != nil { + return x.Result + } + return nil +} + +type ResolveProductProductDetailsArgs struct { + state protoimpl.MessageState `protogen:"open.v1"` + IncludeExtended bool `protobuf:"varint,1,opt,name=include_extended,json=includeExtended,proto3" json:"include_extended,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ResolveProductProductDetailsArgs) Reset() { + *x = ResolveProductProductDetailsArgs{} + mi := &file_product_proto_msgTypes[137] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ResolveProductProductDetailsArgs) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ResolveProductProductDetailsArgs) ProtoMessage() {} + +func (x *ResolveProductProductDetailsArgs) ProtoReflect() protoreflect.Message { + mi := &file_product_proto_msgTypes[137] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ResolveProductProductDetailsArgs.ProtoReflect.Descriptor instead. +func (*ResolveProductProductDetailsArgs) Descriptor() ([]byte, []int) { + return file_product_proto_rawDescGZIP(), []int{137} +} + +func (x *ResolveProductProductDetailsArgs) GetIncludeExtended() bool { + if x != nil { + return x.IncludeExtended + } + return false +} + +type ResolveProductProductDetailsContext struct { + state protoimpl.MessageState `protogen:"open.v1"` + Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` + Name string `protobuf:"bytes,2,opt,name=name,proto3" json:"name,omitempty"` + Price float64 `protobuf:"fixed64,3,opt,name=price,proto3" json:"price,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ResolveProductProductDetailsContext) Reset() { + *x = ResolveProductProductDetailsContext{} + mi := &file_product_proto_msgTypes[138] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ResolveProductProductDetailsContext) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ResolveProductProductDetailsContext) ProtoMessage() {} + +func (x *ResolveProductProductDetailsContext) ProtoReflect() protoreflect.Message { + mi := &file_product_proto_msgTypes[138] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ResolveProductProductDetailsContext.ProtoReflect.Descriptor instead. +func (*ResolveProductProductDetailsContext) Descriptor() ([]byte, []int) { + return file_product_proto_rawDescGZIP(), []int{138} +} + +func (x *ResolveProductProductDetailsContext) GetId() string { + if x != nil { + return x.Id + } + return "" +} + +func (x *ResolveProductProductDetailsContext) GetName() string { + if x != nil { + return x.Name + } + return "" +} + +func (x *ResolveProductProductDetailsContext) GetPrice() float64 { + if x != nil { + return x.Price + } + return 0 +} + +type ResolveProductProductDetailsRequest struct { + state protoimpl.MessageState `protogen:"open.v1"` + // context provides the resolver context for the field productDetails of type Product. + Context []*ResolveProductProductDetailsContext `protobuf:"bytes,1,rep,name=context,proto3" json:"context,omitempty"` + // field_args provides the arguments for the resolver field productDetails of type Product. + FieldArgs *ResolveProductProductDetailsArgs `protobuf:"bytes,2,opt,name=field_args,json=fieldArgs,proto3" json:"field_args,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ResolveProductProductDetailsRequest) Reset() { + *x = ResolveProductProductDetailsRequest{} + mi := &file_product_proto_msgTypes[139] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ResolveProductProductDetailsRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ResolveProductProductDetailsRequest) ProtoMessage() {} + +func (x *ResolveProductProductDetailsRequest) ProtoReflect() protoreflect.Message { + mi := &file_product_proto_msgTypes[139] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ResolveProductProductDetailsRequest.ProtoReflect.Descriptor instead. +func (*ResolveProductProductDetailsRequest) Descriptor() ([]byte, []int) { + return file_product_proto_rawDescGZIP(), []int{139} +} + +func (x *ResolveProductProductDetailsRequest) GetContext() []*ResolveProductProductDetailsContext { + if x != nil { + return x.Context + } + return nil +} + +func (x *ResolveProductProductDetailsRequest) GetFieldArgs() *ResolveProductProductDetailsArgs { + if x != nil { + return x.FieldArgs + } + return nil +} + +type ResolveProductProductDetailsResult struct { + state protoimpl.MessageState `protogen:"open.v1"` + ProductDetails *ProductDetails `protobuf:"bytes,1,opt,name=product_details,json=productDetails,proto3" json:"product_details,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ResolveProductProductDetailsResult) Reset() { + *x = ResolveProductProductDetailsResult{} + mi := &file_product_proto_msgTypes[140] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ResolveProductProductDetailsResult) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ResolveProductProductDetailsResult) ProtoMessage() {} + +func (x *ResolveProductProductDetailsResult) ProtoReflect() protoreflect.Message { + mi := &file_product_proto_msgTypes[140] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ResolveProductProductDetailsResult.ProtoReflect.Descriptor instead. +func (*ResolveProductProductDetailsResult) Descriptor() ([]byte, []int) { + return file_product_proto_rawDescGZIP(), []int{140} +} + +func (x *ResolveProductProductDetailsResult) GetProductDetails() *ProductDetails { + if x != nil { + return x.ProductDetails + } + return nil +} + +type ResolveProductProductDetailsResponse struct { + state protoimpl.MessageState `protogen:"open.v1"` + Result []*ResolveProductProductDetailsResult `protobuf:"bytes,1,rep,name=result,proto3" json:"result,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ResolveProductProductDetailsResponse) Reset() { + *x = ResolveProductProductDetailsResponse{} + mi := &file_product_proto_msgTypes[141] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ResolveProductProductDetailsResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ResolveProductProductDetailsResponse) ProtoMessage() {} + +func (x *ResolveProductProductDetailsResponse) ProtoReflect() protoreflect.Message { + mi := &file_product_proto_msgTypes[141] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ResolveProductProductDetailsResponse.ProtoReflect.Descriptor instead. +func (*ResolveProductProductDetailsResponse) Descriptor() ([]byte, []int) { + return file_product_proto_rawDescGZIP(), []int{141} +} + +func (x *ResolveProductProductDetailsResponse) GetResult() []*ResolveProductProductDetailsResult { + if x != nil { + return x.Result + } + return nil +} + type ResolveCategoryProductCountArgs struct { state protoimpl.MessageState `protogen:"open.v1"` Filters *ProductCountFilter `protobuf:"bytes,1,opt,name=filters,proto3" json:"filters,omitempty"` @@ -5855,7 +6585,7 @@ type ResolveCategoryProductCountArgs struct { func (x *ResolveCategoryProductCountArgs) Reset() { *x = ResolveCategoryProductCountArgs{} - mi := &file_product_proto_msgTypes[127] + mi := &file_product_proto_msgTypes[142] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5867,7 +6597,7 @@ func (x *ResolveCategoryProductCountArgs) String() string { func (*ResolveCategoryProductCountArgs) ProtoMessage() {} func (x *ResolveCategoryProductCountArgs) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[127] + mi := &file_product_proto_msgTypes[142] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5880,7 +6610,7 @@ func (x *ResolveCategoryProductCountArgs) ProtoReflect() protoreflect.Message { // Deprecated: Use ResolveCategoryProductCountArgs.ProtoReflect.Descriptor instead. func (*ResolveCategoryProductCountArgs) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{127} + return file_product_proto_rawDescGZIP(), []int{142} } func (x *ResolveCategoryProductCountArgs) GetFilters() *ProductCountFilter { @@ -5900,7 +6630,7 @@ type ResolveCategoryProductCountContext struct { func (x *ResolveCategoryProductCountContext) Reset() { *x = ResolveCategoryProductCountContext{} - mi := &file_product_proto_msgTypes[128] + mi := &file_product_proto_msgTypes[143] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5912,7 +6642,7 @@ func (x *ResolveCategoryProductCountContext) String() string { func (*ResolveCategoryProductCountContext) ProtoMessage() {} func (x *ResolveCategoryProductCountContext) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[128] + mi := &file_product_proto_msgTypes[143] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5925,7 +6655,7 @@ func (x *ResolveCategoryProductCountContext) ProtoReflect() protoreflect.Message // Deprecated: Use ResolveCategoryProductCountContext.ProtoReflect.Descriptor instead. func (*ResolveCategoryProductCountContext) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{128} + return file_product_proto_rawDescGZIP(), []int{143} } func (x *ResolveCategoryProductCountContext) GetId() string { @@ -5954,7 +6684,7 @@ type ResolveCategoryProductCountRequest struct { func (x *ResolveCategoryProductCountRequest) Reset() { *x = ResolveCategoryProductCountRequest{} - mi := &file_product_proto_msgTypes[129] + mi := &file_product_proto_msgTypes[144] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5966,7 +6696,7 @@ func (x *ResolveCategoryProductCountRequest) String() string { func (*ResolveCategoryProductCountRequest) ProtoMessage() {} func (x *ResolveCategoryProductCountRequest) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[129] + mi := &file_product_proto_msgTypes[144] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5979,7 +6709,7 @@ func (x *ResolveCategoryProductCountRequest) ProtoReflect() protoreflect.Message // Deprecated: Use ResolveCategoryProductCountRequest.ProtoReflect.Descriptor instead. func (*ResolveCategoryProductCountRequest) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{129} + return file_product_proto_rawDescGZIP(), []int{144} } func (x *ResolveCategoryProductCountRequest) GetContext() []*ResolveCategoryProductCountContext { @@ -6005,7 +6735,7 @@ type ResolveCategoryProductCountResult struct { func (x *ResolveCategoryProductCountResult) Reset() { *x = ResolveCategoryProductCountResult{} - mi := &file_product_proto_msgTypes[130] + mi := &file_product_proto_msgTypes[145] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6017,7 +6747,7 @@ func (x *ResolveCategoryProductCountResult) String() string { func (*ResolveCategoryProductCountResult) ProtoMessage() {} func (x *ResolveCategoryProductCountResult) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[130] + mi := &file_product_proto_msgTypes[145] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6030,7 +6760,7 @@ func (x *ResolveCategoryProductCountResult) ProtoReflect() protoreflect.Message // Deprecated: Use ResolveCategoryProductCountResult.ProtoReflect.Descriptor instead. func (*ResolveCategoryProductCountResult) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{130} + return file_product_proto_rawDescGZIP(), []int{145} } func (x *ResolveCategoryProductCountResult) GetProductCount() int32 { @@ -6049,7 +6779,7 @@ type ResolveCategoryProductCountResponse struct { func (x *ResolveCategoryProductCountResponse) Reset() { *x = ResolveCategoryProductCountResponse{} - mi := &file_product_proto_msgTypes[131] + mi := &file_product_proto_msgTypes[146] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6061,7 +6791,7 @@ func (x *ResolveCategoryProductCountResponse) String() string { func (*ResolveCategoryProductCountResponse) ProtoMessage() {} func (x *ResolveCategoryProductCountResponse) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[131] + mi := &file_product_proto_msgTypes[146] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6074,7 +6804,7 @@ func (x *ResolveCategoryProductCountResponse) ProtoReflect() protoreflect.Messag // Deprecated: Use ResolveCategoryProductCountResponse.ProtoReflect.Descriptor instead. func (*ResolveCategoryProductCountResponse) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{131} + return file_product_proto_rawDescGZIP(), []int{146} } func (x *ResolveCategoryProductCountResponse) GetResult() []*ResolveCategoryProductCountResult { @@ -6093,7 +6823,7 @@ type ResolveCategoryPopularityScoreArgs struct { func (x *ResolveCategoryPopularityScoreArgs) Reset() { *x = ResolveCategoryPopularityScoreArgs{} - mi := &file_product_proto_msgTypes[132] + mi := &file_product_proto_msgTypes[147] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6105,7 +6835,7 @@ func (x *ResolveCategoryPopularityScoreArgs) String() string { func (*ResolveCategoryPopularityScoreArgs) ProtoMessage() {} func (x *ResolveCategoryPopularityScoreArgs) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[132] + mi := &file_product_proto_msgTypes[147] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6118,7 +6848,7 @@ func (x *ResolveCategoryPopularityScoreArgs) ProtoReflect() protoreflect.Message // Deprecated: Use ResolveCategoryPopularityScoreArgs.ProtoReflect.Descriptor instead. func (*ResolveCategoryPopularityScoreArgs) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{132} + return file_product_proto_rawDescGZIP(), []int{147} } func (x *ResolveCategoryPopularityScoreArgs) GetThreshold() *wrapperspb.Int32Value { @@ -6137,7 +6867,7 @@ type ResolveCategoryPopularityScoreContext struct { func (x *ResolveCategoryPopularityScoreContext) Reset() { *x = ResolveCategoryPopularityScoreContext{} - mi := &file_product_proto_msgTypes[133] + mi := &file_product_proto_msgTypes[148] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6149,7 +6879,7 @@ func (x *ResolveCategoryPopularityScoreContext) String() string { func (*ResolveCategoryPopularityScoreContext) ProtoMessage() {} func (x *ResolveCategoryPopularityScoreContext) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[133] + mi := &file_product_proto_msgTypes[148] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6162,7 +6892,7 @@ func (x *ResolveCategoryPopularityScoreContext) ProtoReflect() protoreflect.Mess // Deprecated: Use ResolveCategoryPopularityScoreContext.ProtoReflect.Descriptor instead. func (*ResolveCategoryPopularityScoreContext) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{133} + return file_product_proto_rawDescGZIP(), []int{148} } func (x *ResolveCategoryPopularityScoreContext) GetId() string { @@ -6184,7 +6914,7 @@ type ResolveCategoryPopularityScoreRequest struct { func (x *ResolveCategoryPopularityScoreRequest) Reset() { *x = ResolveCategoryPopularityScoreRequest{} - mi := &file_product_proto_msgTypes[134] + mi := &file_product_proto_msgTypes[149] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6196,7 +6926,7 @@ func (x *ResolveCategoryPopularityScoreRequest) String() string { func (*ResolveCategoryPopularityScoreRequest) ProtoMessage() {} func (x *ResolveCategoryPopularityScoreRequest) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[134] + mi := &file_product_proto_msgTypes[149] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6209,7 +6939,7 @@ func (x *ResolveCategoryPopularityScoreRequest) ProtoReflect() protoreflect.Mess // Deprecated: Use ResolveCategoryPopularityScoreRequest.ProtoReflect.Descriptor instead. func (*ResolveCategoryPopularityScoreRequest) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{134} + return file_product_proto_rawDescGZIP(), []int{149} } func (x *ResolveCategoryPopularityScoreRequest) GetContext() []*ResolveCategoryPopularityScoreContext { @@ -6235,7 +6965,7 @@ type ResolveCategoryPopularityScoreResult struct { func (x *ResolveCategoryPopularityScoreResult) Reset() { *x = ResolveCategoryPopularityScoreResult{} - mi := &file_product_proto_msgTypes[135] + mi := &file_product_proto_msgTypes[150] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6247,7 +6977,7 @@ func (x *ResolveCategoryPopularityScoreResult) String() string { func (*ResolveCategoryPopularityScoreResult) ProtoMessage() {} func (x *ResolveCategoryPopularityScoreResult) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[135] + mi := &file_product_proto_msgTypes[150] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6260,7 +6990,7 @@ func (x *ResolveCategoryPopularityScoreResult) ProtoReflect() protoreflect.Messa // Deprecated: Use ResolveCategoryPopularityScoreResult.ProtoReflect.Descriptor instead. func (*ResolveCategoryPopularityScoreResult) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{135} + return file_product_proto_rawDescGZIP(), []int{150} } func (x *ResolveCategoryPopularityScoreResult) GetPopularityScore() *wrapperspb.Int32Value { @@ -6279,7 +7009,7 @@ type ResolveCategoryPopularityScoreResponse struct { func (x *ResolveCategoryPopularityScoreResponse) Reset() { *x = ResolveCategoryPopularityScoreResponse{} - mi := &file_product_proto_msgTypes[136] + mi := &file_product_proto_msgTypes[151] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6291,7 +7021,7 @@ func (x *ResolveCategoryPopularityScoreResponse) String() string { func (*ResolveCategoryPopularityScoreResponse) ProtoMessage() {} func (x *ResolveCategoryPopularityScoreResponse) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[136] + mi := &file_product_proto_msgTypes[151] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6304,7 +7034,7 @@ func (x *ResolveCategoryPopularityScoreResponse) ProtoReflect() protoreflect.Mes // Deprecated: Use ResolveCategoryPopularityScoreResponse.ProtoReflect.Descriptor instead. func (*ResolveCategoryPopularityScoreResponse) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{136} + return file_product_proto_rawDescGZIP(), []int{151} } func (x *ResolveCategoryPopularityScoreResponse) GetResult() []*ResolveCategoryPopularityScoreResult { @@ -6323,7 +7053,7 @@ type ResolveCategoryCategoryMetricsArgs struct { func (x *ResolveCategoryCategoryMetricsArgs) Reset() { *x = ResolveCategoryCategoryMetricsArgs{} - mi := &file_product_proto_msgTypes[137] + mi := &file_product_proto_msgTypes[152] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6335,7 +7065,7 @@ func (x *ResolveCategoryCategoryMetricsArgs) String() string { func (*ResolveCategoryCategoryMetricsArgs) ProtoMessage() {} func (x *ResolveCategoryCategoryMetricsArgs) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[137] + mi := &file_product_proto_msgTypes[152] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6348,7 +7078,7 @@ func (x *ResolveCategoryCategoryMetricsArgs) ProtoReflect() protoreflect.Message // Deprecated: Use ResolveCategoryCategoryMetricsArgs.ProtoReflect.Descriptor instead. func (*ResolveCategoryCategoryMetricsArgs) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{137} + return file_product_proto_rawDescGZIP(), []int{152} } func (x *ResolveCategoryCategoryMetricsArgs) GetMetricType() string { @@ -6368,7 +7098,7 @@ type ResolveCategoryCategoryMetricsContext struct { func (x *ResolveCategoryCategoryMetricsContext) Reset() { *x = ResolveCategoryCategoryMetricsContext{} - mi := &file_product_proto_msgTypes[138] + mi := &file_product_proto_msgTypes[153] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6380,7 +7110,7 @@ func (x *ResolveCategoryCategoryMetricsContext) String() string { func (*ResolveCategoryCategoryMetricsContext) ProtoMessage() {} func (x *ResolveCategoryCategoryMetricsContext) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[138] + mi := &file_product_proto_msgTypes[153] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6393,7 +7123,7 @@ func (x *ResolveCategoryCategoryMetricsContext) ProtoReflect() protoreflect.Mess // Deprecated: Use ResolveCategoryCategoryMetricsContext.ProtoReflect.Descriptor instead. func (*ResolveCategoryCategoryMetricsContext) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{138} + return file_product_proto_rawDescGZIP(), []int{153} } func (x *ResolveCategoryCategoryMetricsContext) GetId() string { @@ -6422,7 +7152,7 @@ type ResolveCategoryCategoryMetricsRequest struct { func (x *ResolveCategoryCategoryMetricsRequest) Reset() { *x = ResolveCategoryCategoryMetricsRequest{} - mi := &file_product_proto_msgTypes[139] + mi := &file_product_proto_msgTypes[154] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6434,7 +7164,7 @@ func (x *ResolveCategoryCategoryMetricsRequest) String() string { func (*ResolveCategoryCategoryMetricsRequest) ProtoMessage() {} func (x *ResolveCategoryCategoryMetricsRequest) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[139] + mi := &file_product_proto_msgTypes[154] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6447,7 +7177,7 @@ func (x *ResolveCategoryCategoryMetricsRequest) ProtoReflect() protoreflect.Mess // Deprecated: Use ResolveCategoryCategoryMetricsRequest.ProtoReflect.Descriptor instead. func (*ResolveCategoryCategoryMetricsRequest) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{139} + return file_product_proto_rawDescGZIP(), []int{154} } func (x *ResolveCategoryCategoryMetricsRequest) GetContext() []*ResolveCategoryCategoryMetricsContext { @@ -6473,7 +7203,7 @@ type ResolveCategoryCategoryMetricsResult struct { func (x *ResolveCategoryCategoryMetricsResult) Reset() { *x = ResolveCategoryCategoryMetricsResult{} - mi := &file_product_proto_msgTypes[140] + mi := &file_product_proto_msgTypes[155] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6485,7 +7215,7 @@ func (x *ResolveCategoryCategoryMetricsResult) String() string { func (*ResolveCategoryCategoryMetricsResult) ProtoMessage() {} func (x *ResolveCategoryCategoryMetricsResult) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[140] + mi := &file_product_proto_msgTypes[155] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6498,7 +7228,7 @@ func (x *ResolveCategoryCategoryMetricsResult) ProtoReflect() protoreflect.Messa // Deprecated: Use ResolveCategoryCategoryMetricsResult.ProtoReflect.Descriptor instead. func (*ResolveCategoryCategoryMetricsResult) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{140} + return file_product_proto_rawDescGZIP(), []int{155} } func (x *ResolveCategoryCategoryMetricsResult) GetCategoryMetrics() *CategoryMetrics { @@ -6517,7 +7247,7 @@ type ResolveCategoryCategoryMetricsResponse struct { func (x *ResolveCategoryCategoryMetricsResponse) Reset() { *x = ResolveCategoryCategoryMetricsResponse{} - mi := &file_product_proto_msgTypes[141] + mi := &file_product_proto_msgTypes[156] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6529,7 +7259,7 @@ func (x *ResolveCategoryCategoryMetricsResponse) String() string { func (*ResolveCategoryCategoryMetricsResponse) ProtoMessage() {} func (x *ResolveCategoryCategoryMetricsResponse) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[141] + mi := &file_product_proto_msgTypes[156] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6542,7 +7272,7 @@ func (x *ResolveCategoryCategoryMetricsResponse) ProtoReflect() protoreflect.Mes // Deprecated: Use ResolveCategoryCategoryMetricsResponse.ProtoReflect.Descriptor instead. func (*ResolveCategoryCategoryMetricsResponse) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{141} + return file_product_proto_rawDescGZIP(), []int{156} } func (x *ResolveCategoryCategoryMetricsResponse) GetResult() []*ResolveCategoryCategoryMetricsResult { @@ -6561,7 +7291,7 @@ type ResolveCategoryMascotArgs struct { func (x *ResolveCategoryMascotArgs) Reset() { *x = ResolveCategoryMascotArgs{} - mi := &file_product_proto_msgTypes[142] + mi := &file_product_proto_msgTypes[157] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6573,7 +7303,7 @@ func (x *ResolveCategoryMascotArgs) String() string { func (*ResolveCategoryMascotArgs) ProtoMessage() {} func (x *ResolveCategoryMascotArgs) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[142] + mi := &file_product_proto_msgTypes[157] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6586,7 +7316,7 @@ func (x *ResolveCategoryMascotArgs) ProtoReflect() protoreflect.Message { // Deprecated: Use ResolveCategoryMascotArgs.ProtoReflect.Descriptor instead. func (*ResolveCategoryMascotArgs) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{142} + return file_product_proto_rawDescGZIP(), []int{157} } func (x *ResolveCategoryMascotArgs) GetIncludeVolume() bool { @@ -6606,7 +7336,7 @@ type ResolveCategoryMascotContext struct { func (x *ResolveCategoryMascotContext) Reset() { *x = ResolveCategoryMascotContext{} - mi := &file_product_proto_msgTypes[143] + mi := &file_product_proto_msgTypes[158] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6618,7 +7348,7 @@ func (x *ResolveCategoryMascotContext) String() string { func (*ResolveCategoryMascotContext) ProtoMessage() {} func (x *ResolveCategoryMascotContext) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[143] + mi := &file_product_proto_msgTypes[158] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6631,7 +7361,7 @@ func (x *ResolveCategoryMascotContext) ProtoReflect() protoreflect.Message { // Deprecated: Use ResolveCategoryMascotContext.ProtoReflect.Descriptor instead. func (*ResolveCategoryMascotContext) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{143} + return file_product_proto_rawDescGZIP(), []int{158} } func (x *ResolveCategoryMascotContext) GetId() string { @@ -6660,7 +7390,7 @@ type ResolveCategoryMascotRequest struct { func (x *ResolveCategoryMascotRequest) Reset() { *x = ResolveCategoryMascotRequest{} - mi := &file_product_proto_msgTypes[144] + mi := &file_product_proto_msgTypes[159] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6672,7 +7402,7 @@ func (x *ResolveCategoryMascotRequest) String() string { func (*ResolveCategoryMascotRequest) ProtoMessage() {} func (x *ResolveCategoryMascotRequest) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[144] + mi := &file_product_proto_msgTypes[159] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6685,7 +7415,7 @@ func (x *ResolveCategoryMascotRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use ResolveCategoryMascotRequest.ProtoReflect.Descriptor instead. func (*ResolveCategoryMascotRequest) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{144} + return file_product_proto_rawDescGZIP(), []int{159} } func (x *ResolveCategoryMascotRequest) GetContext() []*ResolveCategoryMascotContext { @@ -6711,7 +7441,7 @@ type ResolveCategoryMascotResult struct { func (x *ResolveCategoryMascotResult) Reset() { *x = ResolveCategoryMascotResult{} - mi := &file_product_proto_msgTypes[145] + mi := &file_product_proto_msgTypes[160] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6723,7 +7453,7 @@ func (x *ResolveCategoryMascotResult) String() string { func (*ResolveCategoryMascotResult) ProtoMessage() {} func (x *ResolveCategoryMascotResult) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[145] + mi := &file_product_proto_msgTypes[160] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6736,7 +7466,7 @@ func (x *ResolveCategoryMascotResult) ProtoReflect() protoreflect.Message { // Deprecated: Use ResolveCategoryMascotResult.ProtoReflect.Descriptor instead. func (*ResolveCategoryMascotResult) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{145} + return file_product_proto_rawDescGZIP(), []int{160} } func (x *ResolveCategoryMascotResult) GetMascot() *Animal { @@ -6755,7 +7485,7 @@ type ResolveCategoryMascotResponse struct { func (x *ResolveCategoryMascotResponse) Reset() { *x = ResolveCategoryMascotResponse{} - mi := &file_product_proto_msgTypes[146] + mi := &file_product_proto_msgTypes[161] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6767,7 +7497,7 @@ func (x *ResolveCategoryMascotResponse) String() string { func (*ResolveCategoryMascotResponse) ProtoMessage() {} func (x *ResolveCategoryMascotResponse) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[146] + mi := &file_product_proto_msgTypes[161] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6780,7 +7510,7 @@ func (x *ResolveCategoryMascotResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use ResolveCategoryMascotResponse.ProtoReflect.Descriptor instead. func (*ResolveCategoryMascotResponse) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{146} + return file_product_proto_rawDescGZIP(), []int{161} } func (x *ResolveCategoryMascotResponse) GetResult() []*ResolveCategoryMascotResult { @@ -6799,7 +7529,7 @@ type ResolveCategoryCategoryStatusArgs struct { func (x *ResolveCategoryCategoryStatusArgs) Reset() { *x = ResolveCategoryCategoryStatusArgs{} - mi := &file_product_proto_msgTypes[147] + mi := &file_product_proto_msgTypes[162] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6811,7 +7541,7 @@ func (x *ResolveCategoryCategoryStatusArgs) String() string { func (*ResolveCategoryCategoryStatusArgs) ProtoMessage() {} func (x *ResolveCategoryCategoryStatusArgs) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[147] + mi := &file_product_proto_msgTypes[162] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6824,7 +7554,7 @@ func (x *ResolveCategoryCategoryStatusArgs) ProtoReflect() protoreflect.Message // Deprecated: Use ResolveCategoryCategoryStatusArgs.ProtoReflect.Descriptor instead. func (*ResolveCategoryCategoryStatusArgs) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{147} + return file_product_proto_rawDescGZIP(), []int{162} } func (x *ResolveCategoryCategoryStatusArgs) GetCheckHealth() bool { @@ -6844,7 +7574,7 @@ type ResolveCategoryCategoryStatusContext struct { func (x *ResolveCategoryCategoryStatusContext) Reset() { *x = ResolveCategoryCategoryStatusContext{} - mi := &file_product_proto_msgTypes[148] + mi := &file_product_proto_msgTypes[163] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6856,7 +7586,7 @@ func (x *ResolveCategoryCategoryStatusContext) String() string { func (*ResolveCategoryCategoryStatusContext) ProtoMessage() {} func (x *ResolveCategoryCategoryStatusContext) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[148] + mi := &file_product_proto_msgTypes[163] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6869,7 +7599,7 @@ func (x *ResolveCategoryCategoryStatusContext) ProtoReflect() protoreflect.Messa // Deprecated: Use ResolveCategoryCategoryStatusContext.ProtoReflect.Descriptor instead. func (*ResolveCategoryCategoryStatusContext) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{148} + return file_product_proto_rawDescGZIP(), []int{163} } func (x *ResolveCategoryCategoryStatusContext) GetId() string { @@ -6898,7 +7628,7 @@ type ResolveCategoryCategoryStatusRequest struct { func (x *ResolveCategoryCategoryStatusRequest) Reset() { *x = ResolveCategoryCategoryStatusRequest{} - mi := &file_product_proto_msgTypes[149] + mi := &file_product_proto_msgTypes[164] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6910,7 +7640,7 @@ func (x *ResolveCategoryCategoryStatusRequest) String() string { func (*ResolveCategoryCategoryStatusRequest) ProtoMessage() {} func (x *ResolveCategoryCategoryStatusRequest) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[149] + mi := &file_product_proto_msgTypes[164] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6923,7 +7653,7 @@ func (x *ResolveCategoryCategoryStatusRequest) ProtoReflect() protoreflect.Messa // Deprecated: Use ResolveCategoryCategoryStatusRequest.ProtoReflect.Descriptor instead. func (*ResolveCategoryCategoryStatusRequest) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{149} + return file_product_proto_rawDescGZIP(), []int{164} } func (x *ResolveCategoryCategoryStatusRequest) GetContext() []*ResolveCategoryCategoryStatusContext { @@ -6949,7 +7679,7 @@ type ResolveCategoryCategoryStatusResult struct { func (x *ResolveCategoryCategoryStatusResult) Reset() { *x = ResolveCategoryCategoryStatusResult{} - mi := &file_product_proto_msgTypes[150] + mi := &file_product_proto_msgTypes[165] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6961,7 +7691,7 @@ func (x *ResolveCategoryCategoryStatusResult) String() string { func (*ResolveCategoryCategoryStatusResult) ProtoMessage() {} func (x *ResolveCategoryCategoryStatusResult) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[150] + mi := &file_product_proto_msgTypes[165] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6974,7 +7704,7 @@ func (x *ResolveCategoryCategoryStatusResult) ProtoReflect() protoreflect.Messag // Deprecated: Use ResolveCategoryCategoryStatusResult.ProtoReflect.Descriptor instead. func (*ResolveCategoryCategoryStatusResult) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{150} + return file_product_proto_rawDescGZIP(), []int{165} } func (x *ResolveCategoryCategoryStatusResult) GetCategoryStatus() *ActionResult { @@ -6993,7 +7723,7 @@ type ResolveCategoryCategoryStatusResponse struct { func (x *ResolveCategoryCategoryStatusResponse) Reset() { *x = ResolveCategoryCategoryStatusResponse{} - mi := &file_product_proto_msgTypes[151] + mi := &file_product_proto_msgTypes[166] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7005,7 +7735,7 @@ func (x *ResolveCategoryCategoryStatusResponse) String() string { func (*ResolveCategoryCategoryStatusResponse) ProtoMessage() {} func (x *ResolveCategoryCategoryStatusResponse) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[151] + mi := &file_product_proto_msgTypes[166] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7018,7 +7748,7 @@ func (x *ResolveCategoryCategoryStatusResponse) ProtoReflect() protoreflect.Mess // Deprecated: Use ResolveCategoryCategoryStatusResponse.ProtoReflect.Descriptor instead. func (*ResolveCategoryCategoryStatusResponse) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{151} + return file_product_proto_rawDescGZIP(), []int{166} } func (x *ResolveCategoryCategoryStatusResponse) GetResult() []*ResolveCategoryCategoryStatusResult { @@ -7037,7 +7767,7 @@ type ResolveSubcategoryItemCountArgs struct { func (x *ResolveSubcategoryItemCountArgs) Reset() { *x = ResolveSubcategoryItemCountArgs{} - mi := &file_product_proto_msgTypes[152] + mi := &file_product_proto_msgTypes[167] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7049,7 +7779,7 @@ func (x *ResolveSubcategoryItemCountArgs) String() string { func (*ResolveSubcategoryItemCountArgs) ProtoMessage() {} func (x *ResolveSubcategoryItemCountArgs) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[152] + mi := &file_product_proto_msgTypes[167] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7062,7 +7792,7 @@ func (x *ResolveSubcategoryItemCountArgs) ProtoReflect() protoreflect.Message { // Deprecated: Use ResolveSubcategoryItemCountArgs.ProtoReflect.Descriptor instead. func (*ResolveSubcategoryItemCountArgs) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{152} + return file_product_proto_rawDescGZIP(), []int{167} } func (x *ResolveSubcategoryItemCountArgs) GetFilters() *SubcategoryItemFilter { @@ -7081,7 +7811,7 @@ type ResolveSubcategoryItemCountContext struct { func (x *ResolveSubcategoryItemCountContext) Reset() { *x = ResolveSubcategoryItemCountContext{} - mi := &file_product_proto_msgTypes[153] + mi := &file_product_proto_msgTypes[168] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7093,7 +7823,7 @@ func (x *ResolveSubcategoryItemCountContext) String() string { func (*ResolveSubcategoryItemCountContext) ProtoMessage() {} func (x *ResolveSubcategoryItemCountContext) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[153] + mi := &file_product_proto_msgTypes[168] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7106,7 +7836,7 @@ func (x *ResolveSubcategoryItemCountContext) ProtoReflect() protoreflect.Message // Deprecated: Use ResolveSubcategoryItemCountContext.ProtoReflect.Descriptor instead. func (*ResolveSubcategoryItemCountContext) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{153} + return file_product_proto_rawDescGZIP(), []int{168} } func (x *ResolveSubcategoryItemCountContext) GetId() string { @@ -7128,7 +7858,7 @@ type ResolveSubcategoryItemCountRequest struct { func (x *ResolveSubcategoryItemCountRequest) Reset() { *x = ResolveSubcategoryItemCountRequest{} - mi := &file_product_proto_msgTypes[154] + mi := &file_product_proto_msgTypes[169] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7140,7 +7870,7 @@ func (x *ResolveSubcategoryItemCountRequest) String() string { func (*ResolveSubcategoryItemCountRequest) ProtoMessage() {} func (x *ResolveSubcategoryItemCountRequest) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[154] + mi := &file_product_proto_msgTypes[169] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7153,7 +7883,7 @@ func (x *ResolveSubcategoryItemCountRequest) ProtoReflect() protoreflect.Message // Deprecated: Use ResolveSubcategoryItemCountRequest.ProtoReflect.Descriptor instead. func (*ResolveSubcategoryItemCountRequest) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{154} + return file_product_proto_rawDescGZIP(), []int{169} } func (x *ResolveSubcategoryItemCountRequest) GetContext() []*ResolveSubcategoryItemCountContext { @@ -7179,7 +7909,7 @@ type ResolveSubcategoryItemCountResult struct { func (x *ResolveSubcategoryItemCountResult) Reset() { *x = ResolveSubcategoryItemCountResult{} - mi := &file_product_proto_msgTypes[155] + mi := &file_product_proto_msgTypes[170] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7191,7 +7921,7 @@ func (x *ResolveSubcategoryItemCountResult) String() string { func (*ResolveSubcategoryItemCountResult) ProtoMessage() {} func (x *ResolveSubcategoryItemCountResult) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[155] + mi := &file_product_proto_msgTypes[170] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7204,7 +7934,7 @@ func (x *ResolveSubcategoryItemCountResult) ProtoReflect() protoreflect.Message // Deprecated: Use ResolveSubcategoryItemCountResult.ProtoReflect.Descriptor instead. func (*ResolveSubcategoryItemCountResult) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{155} + return file_product_proto_rawDescGZIP(), []int{170} } func (x *ResolveSubcategoryItemCountResult) GetItemCount() int32 { @@ -7223,7 +7953,7 @@ type ResolveSubcategoryItemCountResponse struct { func (x *ResolveSubcategoryItemCountResponse) Reset() { *x = ResolveSubcategoryItemCountResponse{} - mi := &file_product_proto_msgTypes[156] + mi := &file_product_proto_msgTypes[171] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7235,7 +7965,7 @@ func (x *ResolveSubcategoryItemCountResponse) String() string { func (*ResolveSubcategoryItemCountResponse) ProtoMessage() {} func (x *ResolveSubcategoryItemCountResponse) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[156] + mi := &file_product_proto_msgTypes[171] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7248,7 +7978,7 @@ func (x *ResolveSubcategoryItemCountResponse) ProtoReflect() protoreflect.Messag // Deprecated: Use ResolveSubcategoryItemCountResponse.ProtoReflect.Descriptor instead. func (*ResolveSubcategoryItemCountResponse) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{156} + return file_product_proto_rawDescGZIP(), []int{171} } func (x *ResolveSubcategoryItemCountResponse) GetResult() []*ResolveSubcategoryItemCountResult { @@ -7267,7 +7997,7 @@ type ResolveCategoryMetricsNormalizedScoreArgs struct { func (x *ResolveCategoryMetricsNormalizedScoreArgs) Reset() { *x = ResolveCategoryMetricsNormalizedScoreArgs{} - mi := &file_product_proto_msgTypes[157] + mi := &file_product_proto_msgTypes[172] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7279,7 +8009,7 @@ func (x *ResolveCategoryMetricsNormalizedScoreArgs) String() string { func (*ResolveCategoryMetricsNormalizedScoreArgs) ProtoMessage() {} func (x *ResolveCategoryMetricsNormalizedScoreArgs) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[157] + mi := &file_product_proto_msgTypes[172] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7292,7 +8022,7 @@ func (x *ResolveCategoryMetricsNormalizedScoreArgs) ProtoReflect() protoreflect. // Deprecated: Use ResolveCategoryMetricsNormalizedScoreArgs.ProtoReflect.Descriptor instead. func (*ResolveCategoryMetricsNormalizedScoreArgs) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{157} + return file_product_proto_rawDescGZIP(), []int{172} } func (x *ResolveCategoryMetricsNormalizedScoreArgs) GetBaseline() float64 { @@ -7313,7 +8043,7 @@ type ResolveCategoryMetricsNormalizedScoreContext struct { func (x *ResolveCategoryMetricsNormalizedScoreContext) Reset() { *x = ResolveCategoryMetricsNormalizedScoreContext{} - mi := &file_product_proto_msgTypes[158] + mi := &file_product_proto_msgTypes[173] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7325,7 +8055,7 @@ func (x *ResolveCategoryMetricsNormalizedScoreContext) String() string { func (*ResolveCategoryMetricsNormalizedScoreContext) ProtoMessage() {} func (x *ResolveCategoryMetricsNormalizedScoreContext) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[158] + mi := &file_product_proto_msgTypes[173] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7338,7 +8068,7 @@ func (x *ResolveCategoryMetricsNormalizedScoreContext) ProtoReflect() protorefle // Deprecated: Use ResolveCategoryMetricsNormalizedScoreContext.ProtoReflect.Descriptor instead. func (*ResolveCategoryMetricsNormalizedScoreContext) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{158} + return file_product_proto_rawDescGZIP(), []int{173} } func (x *ResolveCategoryMetricsNormalizedScoreContext) GetId() string { @@ -7374,7 +8104,7 @@ type ResolveCategoryMetricsNormalizedScoreRequest struct { func (x *ResolveCategoryMetricsNormalizedScoreRequest) Reset() { *x = ResolveCategoryMetricsNormalizedScoreRequest{} - mi := &file_product_proto_msgTypes[159] + mi := &file_product_proto_msgTypes[174] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7386,7 +8116,7 @@ func (x *ResolveCategoryMetricsNormalizedScoreRequest) String() string { func (*ResolveCategoryMetricsNormalizedScoreRequest) ProtoMessage() {} func (x *ResolveCategoryMetricsNormalizedScoreRequest) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[159] + mi := &file_product_proto_msgTypes[174] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7399,7 +8129,7 @@ func (x *ResolveCategoryMetricsNormalizedScoreRequest) ProtoReflect() protorefle // Deprecated: Use ResolveCategoryMetricsNormalizedScoreRequest.ProtoReflect.Descriptor instead. func (*ResolveCategoryMetricsNormalizedScoreRequest) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{159} + return file_product_proto_rawDescGZIP(), []int{174} } func (x *ResolveCategoryMetricsNormalizedScoreRequest) GetContext() []*ResolveCategoryMetricsNormalizedScoreContext { @@ -7425,7 +8155,7 @@ type ResolveCategoryMetricsNormalizedScoreResult struct { func (x *ResolveCategoryMetricsNormalizedScoreResult) Reset() { *x = ResolveCategoryMetricsNormalizedScoreResult{} - mi := &file_product_proto_msgTypes[160] + mi := &file_product_proto_msgTypes[175] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7437,7 +8167,7 @@ func (x *ResolveCategoryMetricsNormalizedScoreResult) String() string { func (*ResolveCategoryMetricsNormalizedScoreResult) ProtoMessage() {} func (x *ResolveCategoryMetricsNormalizedScoreResult) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[160] + mi := &file_product_proto_msgTypes[175] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7450,7 +8180,7 @@ func (x *ResolveCategoryMetricsNormalizedScoreResult) ProtoReflect() protoreflec // Deprecated: Use ResolveCategoryMetricsNormalizedScoreResult.ProtoReflect.Descriptor instead. func (*ResolveCategoryMetricsNormalizedScoreResult) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{160} + return file_product_proto_rawDescGZIP(), []int{175} } func (x *ResolveCategoryMetricsNormalizedScoreResult) GetNormalizedScore() float64 { @@ -7469,7 +8199,7 @@ type ResolveCategoryMetricsNormalizedScoreResponse struct { func (x *ResolveCategoryMetricsNormalizedScoreResponse) Reset() { *x = ResolveCategoryMetricsNormalizedScoreResponse{} - mi := &file_product_proto_msgTypes[161] + mi := &file_product_proto_msgTypes[176] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7481,7 +8211,7 @@ func (x *ResolveCategoryMetricsNormalizedScoreResponse) String() string { func (*ResolveCategoryMetricsNormalizedScoreResponse) ProtoMessage() {} func (x *ResolveCategoryMetricsNormalizedScoreResponse) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[161] + mi := &file_product_proto_msgTypes[176] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7494,7 +8224,7 @@ func (x *ResolveCategoryMetricsNormalizedScoreResponse) ProtoReflect() protorefl // Deprecated: Use ResolveCategoryMetricsNormalizedScoreResponse.ProtoReflect.Descriptor instead. func (*ResolveCategoryMetricsNormalizedScoreResponse) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{161} + return file_product_proto_rawDescGZIP(), []int{176} } func (x *ResolveCategoryMetricsNormalizedScoreResponse) GetResult() []*ResolveCategoryMetricsNormalizedScoreResult { @@ -7513,7 +8243,7 @@ type ResolveTestContainerDetailsArgs struct { func (x *ResolveTestContainerDetailsArgs) Reset() { *x = ResolveTestContainerDetailsArgs{} - mi := &file_product_proto_msgTypes[162] + mi := &file_product_proto_msgTypes[177] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7525,7 +8255,7 @@ func (x *ResolveTestContainerDetailsArgs) String() string { func (*ResolveTestContainerDetailsArgs) ProtoMessage() {} func (x *ResolveTestContainerDetailsArgs) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[162] + mi := &file_product_proto_msgTypes[177] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7538,7 +8268,7 @@ func (x *ResolveTestContainerDetailsArgs) ProtoReflect() protoreflect.Message { // Deprecated: Use ResolveTestContainerDetailsArgs.ProtoReflect.Descriptor instead. func (*ResolveTestContainerDetailsArgs) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{162} + return file_product_proto_rawDescGZIP(), []int{177} } func (x *ResolveTestContainerDetailsArgs) GetIncludeExtended() bool { @@ -7558,7 +8288,7 @@ type ResolveTestContainerDetailsContext struct { func (x *ResolveTestContainerDetailsContext) Reset() { *x = ResolveTestContainerDetailsContext{} - mi := &file_product_proto_msgTypes[163] + mi := &file_product_proto_msgTypes[178] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7570,7 +8300,7 @@ func (x *ResolveTestContainerDetailsContext) String() string { func (*ResolveTestContainerDetailsContext) ProtoMessage() {} func (x *ResolveTestContainerDetailsContext) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[163] + mi := &file_product_proto_msgTypes[178] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7583,7 +8313,7 @@ func (x *ResolveTestContainerDetailsContext) ProtoReflect() protoreflect.Message // Deprecated: Use ResolveTestContainerDetailsContext.ProtoReflect.Descriptor instead. func (*ResolveTestContainerDetailsContext) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{163} + return file_product_proto_rawDescGZIP(), []int{178} } func (x *ResolveTestContainerDetailsContext) GetId() string { @@ -7612,7 +8342,7 @@ type ResolveTestContainerDetailsRequest struct { func (x *ResolveTestContainerDetailsRequest) Reset() { *x = ResolveTestContainerDetailsRequest{} - mi := &file_product_proto_msgTypes[164] + mi := &file_product_proto_msgTypes[179] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7624,7 +8354,7 @@ func (x *ResolveTestContainerDetailsRequest) String() string { func (*ResolveTestContainerDetailsRequest) ProtoMessage() {} func (x *ResolveTestContainerDetailsRequest) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[164] + mi := &file_product_proto_msgTypes[179] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7637,7 +8367,7 @@ func (x *ResolveTestContainerDetailsRequest) ProtoReflect() protoreflect.Message // Deprecated: Use ResolveTestContainerDetailsRequest.ProtoReflect.Descriptor instead. func (*ResolveTestContainerDetailsRequest) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{164} + return file_product_proto_rawDescGZIP(), []int{179} } func (x *ResolveTestContainerDetailsRequest) GetContext() []*ResolveTestContainerDetailsContext { @@ -7663,7 +8393,7 @@ type ResolveTestContainerDetailsResult struct { func (x *ResolveTestContainerDetailsResult) Reset() { *x = ResolveTestContainerDetailsResult{} - mi := &file_product_proto_msgTypes[165] + mi := &file_product_proto_msgTypes[180] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7675,7 +8405,7 @@ func (x *ResolveTestContainerDetailsResult) String() string { func (*ResolveTestContainerDetailsResult) ProtoMessage() {} func (x *ResolveTestContainerDetailsResult) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[165] + mi := &file_product_proto_msgTypes[180] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7688,7 +8418,7 @@ func (x *ResolveTestContainerDetailsResult) ProtoReflect() protoreflect.Message // Deprecated: Use ResolveTestContainerDetailsResult.ProtoReflect.Descriptor instead. func (*ResolveTestContainerDetailsResult) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{165} + return file_product_proto_rawDescGZIP(), []int{180} } func (x *ResolveTestContainerDetailsResult) GetDetails() *TestDetails { @@ -7707,7 +8437,7 @@ type ResolveTestContainerDetailsResponse struct { func (x *ResolveTestContainerDetailsResponse) Reset() { *x = ResolveTestContainerDetailsResponse{} - mi := &file_product_proto_msgTypes[166] + mi := &file_product_proto_msgTypes[181] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7719,7 +8449,7 @@ func (x *ResolveTestContainerDetailsResponse) String() string { func (*ResolveTestContainerDetailsResponse) ProtoMessage() {} func (x *ResolveTestContainerDetailsResponse) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[166] + mi := &file_product_proto_msgTypes[181] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7732,7 +8462,7 @@ func (x *ResolveTestContainerDetailsResponse) ProtoReflect() protoreflect.Messag // Deprecated: Use ResolveTestContainerDetailsResponse.ProtoReflect.Descriptor instead. func (*ResolveTestContainerDetailsResponse) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{166} + return file_product_proto_rawDescGZIP(), []int{181} } func (x *ResolveTestContainerDetailsResponse) GetResult() []*ResolveTestContainerDetailsResult { @@ -7753,7 +8483,7 @@ type Product struct { func (x *Product) Reset() { *x = Product{} - mi := &file_product_proto_msgTypes[167] + mi := &file_product_proto_msgTypes[182] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7765,7 +8495,7 @@ func (x *Product) String() string { func (*Product) ProtoMessage() {} func (x *Product) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[167] + mi := &file_product_proto_msgTypes[182] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7778,7 +8508,7 @@ func (x *Product) ProtoReflect() protoreflect.Message { // Deprecated: Use Product.ProtoReflect.Descriptor instead. func (*Product) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{167} + return file_product_proto_rawDescGZIP(), []int{182} } func (x *Product) GetId() string { @@ -7813,7 +8543,7 @@ type Storage struct { func (x *Storage) Reset() { *x = Storage{} - mi := &file_product_proto_msgTypes[168] + mi := &file_product_proto_msgTypes[183] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7825,7 +8555,7 @@ func (x *Storage) String() string { func (*Storage) ProtoMessage() {} func (x *Storage) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[168] + mi := &file_product_proto_msgTypes[183] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7838,7 +8568,7 @@ func (x *Storage) ProtoReflect() protoreflect.Message { // Deprecated: Use Storage.ProtoReflect.Descriptor instead. func (*Storage) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{168} + return file_product_proto_rawDescGZIP(), []int{183} } func (x *Storage) GetId() string { @@ -7873,7 +8603,7 @@ type Warehouse struct { func (x *Warehouse) Reset() { *x = Warehouse{} - mi := &file_product_proto_msgTypes[169] + mi := &file_product_proto_msgTypes[184] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7885,7 +8615,7 @@ func (x *Warehouse) String() string { func (*Warehouse) ProtoMessage() {} func (x *Warehouse) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[169] + mi := &file_product_proto_msgTypes[184] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7898,7 +8628,7 @@ func (x *Warehouse) ProtoReflect() protoreflect.Message { // Deprecated: Use Warehouse.ProtoReflect.Descriptor instead. func (*Warehouse) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{169} + return file_product_proto_rawDescGZIP(), []int{184} } func (x *Warehouse) GetId() string { @@ -7932,7 +8662,7 @@ type User struct { func (x *User) Reset() { *x = User{} - mi := &file_product_proto_msgTypes[170] + mi := &file_product_proto_msgTypes[185] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7944,7 +8674,7 @@ func (x *User) String() string { func (*User) ProtoMessage() {} func (x *User) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[170] + mi := &file_product_proto_msgTypes[185] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7957,7 +8687,7 @@ func (x *User) ProtoReflect() protoreflect.Message { // Deprecated: Use User.ProtoReflect.Descriptor instead. func (*User) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{170} + return file_product_proto_rawDescGZIP(), []int{185} } func (x *User) GetId() string { @@ -7985,7 +8715,7 @@ type NestedTypeA struct { func (x *NestedTypeA) Reset() { *x = NestedTypeA{} - mi := &file_product_proto_msgTypes[171] + mi := &file_product_proto_msgTypes[186] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7997,7 +8727,7 @@ func (x *NestedTypeA) String() string { func (*NestedTypeA) ProtoMessage() {} func (x *NestedTypeA) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[171] + mi := &file_product_proto_msgTypes[186] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8010,7 +8740,7 @@ func (x *NestedTypeA) ProtoReflect() protoreflect.Message { // Deprecated: Use NestedTypeA.ProtoReflect.Descriptor instead. func (*NestedTypeA) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{171} + return file_product_proto_rawDescGZIP(), []int{186} } func (x *NestedTypeA) GetId() string { @@ -8045,7 +8775,7 @@ type RecursiveType struct { func (x *RecursiveType) Reset() { *x = RecursiveType{} - mi := &file_product_proto_msgTypes[172] + mi := &file_product_proto_msgTypes[187] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8057,7 +8787,7 @@ func (x *RecursiveType) String() string { func (*RecursiveType) ProtoMessage() {} func (x *RecursiveType) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[172] + mi := &file_product_proto_msgTypes[187] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8070,7 +8800,7 @@ func (x *RecursiveType) ProtoReflect() protoreflect.Message { // Deprecated: Use RecursiveType.ProtoReflect.Descriptor instead. func (*RecursiveType) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{172} + return file_product_proto_rawDescGZIP(), []int{187} } func (x *RecursiveType) GetId() string { @@ -8106,7 +8836,7 @@ type TypeWithMultipleFilterFields struct { func (x *TypeWithMultipleFilterFields) Reset() { *x = TypeWithMultipleFilterFields{} - mi := &file_product_proto_msgTypes[173] + mi := &file_product_proto_msgTypes[188] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8118,7 +8848,7 @@ func (x *TypeWithMultipleFilterFields) String() string { func (*TypeWithMultipleFilterFields) ProtoMessage() {} func (x *TypeWithMultipleFilterFields) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[173] + mi := &file_product_proto_msgTypes[188] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8131,7 +8861,7 @@ func (x *TypeWithMultipleFilterFields) ProtoReflect() protoreflect.Message { // Deprecated: Use TypeWithMultipleFilterFields.ProtoReflect.Descriptor instead. func (*TypeWithMultipleFilterFields) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{173} + return file_product_proto_rawDescGZIP(), []int{188} } func (x *TypeWithMultipleFilterFields) GetId() string { @@ -8172,7 +8902,7 @@ type FilterTypeInput struct { func (x *FilterTypeInput) Reset() { *x = FilterTypeInput{} - mi := &file_product_proto_msgTypes[174] + mi := &file_product_proto_msgTypes[189] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8184,7 +8914,7 @@ func (x *FilterTypeInput) String() string { func (*FilterTypeInput) ProtoMessage() {} func (x *FilterTypeInput) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[174] + mi := &file_product_proto_msgTypes[189] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8197,7 +8927,7 @@ func (x *FilterTypeInput) ProtoReflect() protoreflect.Message { // Deprecated: Use FilterTypeInput.ProtoReflect.Descriptor instead. func (*FilterTypeInput) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{174} + return file_product_proto_rawDescGZIP(), []int{189} } func (x *FilterTypeInput) GetFilterField_1() string { @@ -8223,7 +8953,7 @@ type ComplexFilterTypeInput struct { func (x *ComplexFilterTypeInput) Reset() { *x = ComplexFilterTypeInput{} - mi := &file_product_proto_msgTypes[175] + mi := &file_product_proto_msgTypes[190] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8235,7 +8965,7 @@ func (x *ComplexFilterTypeInput) String() string { func (*ComplexFilterTypeInput) ProtoMessage() {} func (x *ComplexFilterTypeInput) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[175] + mi := &file_product_proto_msgTypes[190] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8248,7 +8978,7 @@ func (x *ComplexFilterTypeInput) ProtoReflect() protoreflect.Message { // Deprecated: Use ComplexFilterTypeInput.ProtoReflect.Descriptor instead. func (*ComplexFilterTypeInput) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{175} + return file_product_proto_rawDescGZIP(), []int{190} } func (x *ComplexFilterTypeInput) GetFilter() *FilterType { @@ -8268,7 +8998,7 @@ type TypeWithComplexFilterInput struct { func (x *TypeWithComplexFilterInput) Reset() { *x = TypeWithComplexFilterInput{} - mi := &file_product_proto_msgTypes[176] + mi := &file_product_proto_msgTypes[191] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8280,7 +9010,7 @@ func (x *TypeWithComplexFilterInput) String() string { func (*TypeWithComplexFilterInput) ProtoMessage() {} func (x *TypeWithComplexFilterInput) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[176] + mi := &file_product_proto_msgTypes[191] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8293,7 +9023,7 @@ func (x *TypeWithComplexFilterInput) ProtoReflect() protoreflect.Message { // Deprecated: Use TypeWithComplexFilterInput.ProtoReflect.Descriptor instead. func (*TypeWithComplexFilterInput) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{176} + return file_product_proto_rawDescGZIP(), []int{191} } func (x *TypeWithComplexFilterInput) GetId() string { @@ -8321,7 +9051,7 @@ type OrderInput struct { func (x *OrderInput) Reset() { *x = OrderInput{} - mi := &file_product_proto_msgTypes[177] + mi := &file_product_proto_msgTypes[192] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8333,7 +9063,7 @@ func (x *OrderInput) String() string { func (*OrderInput) ProtoMessage() {} func (x *OrderInput) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[177] + mi := &file_product_proto_msgTypes[192] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8346,7 +9076,7 @@ func (x *OrderInput) ProtoReflect() protoreflect.Message { // Deprecated: Use OrderInput.ProtoReflect.Descriptor instead. func (*OrderInput) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{177} + return file_product_proto_rawDescGZIP(), []int{192} } func (x *OrderInput) GetOrderId() string { @@ -8382,7 +9112,7 @@ type Order struct { func (x *Order) Reset() { *x = Order{} - mi := &file_product_proto_msgTypes[178] + mi := &file_product_proto_msgTypes[193] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8394,7 +9124,7 @@ func (x *Order) String() string { func (*Order) ProtoMessage() {} func (x *Order) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[178] + mi := &file_product_proto_msgTypes[193] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8407,7 +9137,7 @@ func (x *Order) ProtoReflect() protoreflect.Message { // Deprecated: Use Order.ProtoReflect.Descriptor instead. func (*Order) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{178} + return file_product_proto_rawDescGZIP(), []int{193} } func (x *Order) GetOrderId() string { @@ -8450,7 +9180,7 @@ type Category struct { func (x *Category) Reset() { *x = Category{} - mi := &file_product_proto_msgTypes[179] + mi := &file_product_proto_msgTypes[194] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8462,7 +9192,7 @@ func (x *Category) String() string { func (*Category) ProtoMessage() {} func (x *Category) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[179] + mi := &file_product_proto_msgTypes[194] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8475,7 +9205,7 @@ func (x *Category) ProtoReflect() protoreflect.Message { // Deprecated: Use Category.ProtoReflect.Descriptor instead. func (*Category) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{179} + return file_product_proto_rawDescGZIP(), []int{194} } func (x *Category) GetId() string { @@ -8516,7 +9246,7 @@ type CategoryFilter struct { func (x *CategoryFilter) Reset() { *x = CategoryFilter{} - mi := &file_product_proto_msgTypes[180] + mi := &file_product_proto_msgTypes[195] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8528,7 +9258,7 @@ func (x *CategoryFilter) String() string { func (*CategoryFilter) ProtoMessage() {} func (x *CategoryFilter) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[180] + mi := &file_product_proto_msgTypes[195] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8541,7 +9271,7 @@ func (x *CategoryFilter) ProtoReflect() protoreflect.Message { // Deprecated: Use CategoryFilter.ProtoReflect.Descriptor instead. func (*CategoryFilter) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{180} + return file_product_proto_rawDescGZIP(), []int{195} } func (x *CategoryFilter) GetCategory() CategoryKind { @@ -8571,7 +9301,7 @@ type Animal struct { func (x *Animal) Reset() { *x = Animal{} - mi := &file_product_proto_msgTypes[181] + mi := &file_product_proto_msgTypes[196] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8583,7 +9313,7 @@ func (x *Animal) String() string { func (*Animal) ProtoMessage() {} func (x *Animal) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[181] + mi := &file_product_proto_msgTypes[196] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8596,7 +9326,7 @@ func (x *Animal) ProtoReflect() protoreflect.Message { // Deprecated: Use Animal.ProtoReflect.Descriptor instead. func (*Animal) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{181} + return file_product_proto_rawDescGZIP(), []int{196} } func (x *Animal) GetInstance() isAnimal_Instance { @@ -8650,7 +9380,7 @@ type SearchInput struct { func (x *SearchInput) Reset() { *x = SearchInput{} - mi := &file_product_proto_msgTypes[182] + mi := &file_product_proto_msgTypes[197] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8662,7 +9392,7 @@ func (x *SearchInput) String() string { func (*SearchInput) ProtoMessage() {} func (x *SearchInput) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[182] + mi := &file_product_proto_msgTypes[197] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8675,7 +9405,7 @@ func (x *SearchInput) ProtoReflect() protoreflect.Message { // Deprecated: Use SearchInput.ProtoReflect.Descriptor instead. func (*SearchInput) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{182} + return file_product_proto_rawDescGZIP(), []int{197} } func (x *SearchInput) GetQuery() string { @@ -8706,7 +9436,7 @@ type SearchResult struct { func (x *SearchResult) Reset() { *x = SearchResult{} - mi := &file_product_proto_msgTypes[183] + mi := &file_product_proto_msgTypes[198] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8718,7 +9448,7 @@ func (x *SearchResult) String() string { func (*SearchResult) ProtoMessage() {} func (x *SearchResult) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[183] + mi := &file_product_proto_msgTypes[198] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8731,7 +9461,7 @@ func (x *SearchResult) ProtoReflect() protoreflect.Message { // Deprecated: Use SearchResult.ProtoReflect.Descriptor instead. func (*SearchResult) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{183} + return file_product_proto_rawDescGZIP(), []int{198} } func (x *SearchResult) GetValue() isSearchResult_Value { @@ -8806,7 +9536,7 @@ type NullableFieldsType struct { func (x *NullableFieldsType) Reset() { *x = NullableFieldsType{} - mi := &file_product_proto_msgTypes[184] + mi := &file_product_proto_msgTypes[199] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8818,7 +9548,7 @@ func (x *NullableFieldsType) String() string { func (*NullableFieldsType) ProtoMessage() {} func (x *NullableFieldsType) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[184] + mi := &file_product_proto_msgTypes[199] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8831,7 +9561,7 @@ func (x *NullableFieldsType) ProtoReflect() protoreflect.Message { // Deprecated: Use NullableFieldsType.ProtoReflect.Descriptor instead. func (*NullableFieldsType) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{184} + return file_product_proto_rawDescGZIP(), []int{199} } func (x *NullableFieldsType) GetId() string { @@ -8901,7 +9631,7 @@ type NullableFieldsFilter struct { func (x *NullableFieldsFilter) Reset() { *x = NullableFieldsFilter{} - mi := &file_product_proto_msgTypes[185] + mi := &file_product_proto_msgTypes[200] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8913,7 +9643,7 @@ func (x *NullableFieldsFilter) String() string { func (*NullableFieldsFilter) ProtoMessage() {} func (x *NullableFieldsFilter) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[185] + mi := &file_product_proto_msgTypes[200] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8926,7 +9656,7 @@ func (x *NullableFieldsFilter) ProtoReflect() protoreflect.Message { // Deprecated: Use NullableFieldsFilter.ProtoReflect.Descriptor instead. func (*NullableFieldsFilter) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{185} + return file_product_proto_rawDescGZIP(), []int{200} } func (x *NullableFieldsFilter) GetName() *wrapperspb.StringValue { @@ -8978,7 +9708,7 @@ type BlogPost struct { func (x *BlogPost) Reset() { *x = BlogPost{} - mi := &file_product_proto_msgTypes[186] + mi := &file_product_proto_msgTypes[201] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8990,7 +9720,7 @@ func (x *BlogPost) String() string { func (*BlogPost) ProtoMessage() {} func (x *BlogPost) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[186] + mi := &file_product_proto_msgTypes[201] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9003,7 +9733,7 @@ func (x *BlogPost) ProtoReflect() protoreflect.Message { // Deprecated: Use BlogPost.ProtoReflect.Descriptor instead. func (*BlogPost) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{186} + return file_product_proto_rawDescGZIP(), []int{201} } func (x *BlogPost) GetId() string { @@ -9157,7 +9887,7 @@ type BlogPostFilter struct { func (x *BlogPostFilter) Reset() { *x = BlogPostFilter{} - mi := &file_product_proto_msgTypes[187] + mi := &file_product_proto_msgTypes[202] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9169,7 +9899,7 @@ func (x *BlogPostFilter) String() string { func (*BlogPostFilter) ProtoMessage() {} func (x *BlogPostFilter) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[187] + mi := &file_product_proto_msgTypes[202] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9182,7 +9912,7 @@ func (x *BlogPostFilter) ProtoReflect() protoreflect.Message { // Deprecated: Use BlogPostFilter.ProtoReflect.Descriptor instead. func (*BlogPostFilter) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{187} + return file_product_proto_rawDescGZIP(), []int{202} } func (x *BlogPostFilter) GetTitle() *wrapperspb.StringValue { @@ -9229,7 +9959,7 @@ type Author struct { func (x *Author) Reset() { *x = Author{} - mi := &file_product_proto_msgTypes[188] + mi := &file_product_proto_msgTypes[203] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9241,7 +9971,7 @@ func (x *Author) String() string { func (*Author) ProtoMessage() {} func (x *Author) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[188] + mi := &file_product_proto_msgTypes[203] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9254,7 +9984,7 @@ func (x *Author) ProtoReflect() protoreflect.Message { // Deprecated: Use Author.ProtoReflect.Descriptor instead. func (*Author) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{188} + return file_product_proto_rawDescGZIP(), []int{203} } func (x *Author) GetId() string { @@ -9373,7 +10103,7 @@ type AuthorFilter struct { func (x *AuthorFilter) Reset() { *x = AuthorFilter{} - mi := &file_product_proto_msgTypes[189] + mi := &file_product_proto_msgTypes[204] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9385,7 +10115,7 @@ func (x *AuthorFilter) String() string { func (*AuthorFilter) ProtoMessage() {} func (x *AuthorFilter) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[189] + mi := &file_product_proto_msgTypes[204] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9398,7 +10128,7 @@ func (x *AuthorFilter) ProtoReflect() protoreflect.Message { // Deprecated: Use AuthorFilter.ProtoReflect.Descriptor instead. func (*AuthorFilter) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{189} + return file_product_proto_rawDescGZIP(), []int{204} } func (x *AuthorFilter) GetName() *wrapperspb.StringValue { @@ -9433,7 +10163,7 @@ type TestContainer struct { func (x *TestContainer) Reset() { *x = TestContainer{} - mi := &file_product_proto_msgTypes[190] + mi := &file_product_proto_msgTypes[205] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9445,7 +10175,7 @@ func (x *TestContainer) String() string { func (*TestContainer) ProtoMessage() {} func (x *TestContainer) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[190] + mi := &file_product_proto_msgTypes[205] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9458,7 +10188,7 @@ func (x *TestContainer) ProtoReflect() protoreflect.Message { // Deprecated: Use TestContainer.ProtoReflect.Descriptor instead. func (*TestContainer) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{190} + return file_product_proto_rawDescGZIP(), []int{205} } func (x *TestContainer) GetId() string { @@ -9491,7 +10221,7 @@ type UserInput struct { func (x *UserInput) Reset() { *x = UserInput{} - mi := &file_product_proto_msgTypes[191] + mi := &file_product_proto_msgTypes[206] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9503,7 +10233,7 @@ func (x *UserInput) String() string { func (*UserInput) ProtoMessage() {} func (x *UserInput) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[191] + mi := &file_product_proto_msgTypes[206] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9516,7 +10246,7 @@ func (x *UserInput) ProtoReflect() protoreflect.Message { // Deprecated: Use UserInput.ProtoReflect.Descriptor instead. func (*UserInput) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{191} + return file_product_proto_rawDescGZIP(), []int{206} } func (x *UserInput) GetName() string { @@ -9536,7 +10266,7 @@ type ActionInput struct { func (x *ActionInput) Reset() { *x = ActionInput{} - mi := &file_product_proto_msgTypes[192] + mi := &file_product_proto_msgTypes[207] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9548,7 +10278,7 @@ func (x *ActionInput) String() string { func (*ActionInput) ProtoMessage() {} func (x *ActionInput) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[192] + mi := &file_product_proto_msgTypes[207] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9561,7 +10291,7 @@ func (x *ActionInput) ProtoReflect() protoreflect.Message { // Deprecated: Use ActionInput.ProtoReflect.Descriptor instead. func (*ActionInput) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{192} + return file_product_proto_rawDescGZIP(), []int{207} } func (x *ActionInput) GetType() string { @@ -9591,7 +10321,7 @@ type ActionResult struct { func (x *ActionResult) Reset() { *x = ActionResult{} - mi := &file_product_proto_msgTypes[193] + mi := &file_product_proto_msgTypes[208] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9603,7 +10333,7 @@ func (x *ActionResult) String() string { func (*ActionResult) ProtoMessage() {} func (x *ActionResult) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[193] + mi := &file_product_proto_msgTypes[208] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9616,7 +10346,7 @@ func (x *ActionResult) ProtoReflect() protoreflect.Message { // Deprecated: Use ActionResult.ProtoReflect.Descriptor instead. func (*ActionResult) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{193} + return file_product_proto_rawDescGZIP(), []int{208} } func (x *ActionResult) GetValue() isActionResult_Value { @@ -9675,7 +10405,7 @@ type NullableFieldsInput struct { func (x *NullableFieldsInput) Reset() { *x = NullableFieldsInput{} - mi := &file_product_proto_msgTypes[194] + mi := &file_product_proto_msgTypes[209] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9687,7 +10417,7 @@ func (x *NullableFieldsInput) String() string { func (*NullableFieldsInput) ProtoMessage() {} func (x *NullableFieldsInput) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[194] + mi := &file_product_proto_msgTypes[209] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9700,7 +10430,7 @@ func (x *NullableFieldsInput) ProtoReflect() protoreflect.Message { // Deprecated: Use NullableFieldsInput.ProtoReflect.Descriptor instead. func (*NullableFieldsInput) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{194} + return file_product_proto_rawDescGZIP(), []int{209} } func (x *NullableFieldsInput) GetName() string { @@ -9776,7 +10506,7 @@ type BlogPostInput struct { func (x *BlogPostInput) Reset() { *x = BlogPostInput{} - mi := &file_product_proto_msgTypes[195] + mi := &file_product_proto_msgTypes[210] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9788,7 +10518,7 @@ func (x *BlogPostInput) String() string { func (*BlogPostInput) ProtoMessage() {} func (x *BlogPostInput) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[195] + mi := &file_product_proto_msgTypes[210] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9801,7 +10531,7 @@ func (x *BlogPostInput) ProtoReflect() protoreflect.Message { // Deprecated: Use BlogPostInput.ProtoReflect.Descriptor instead. func (*BlogPostInput) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{195} + return file_product_proto_rawDescGZIP(), []int{210} } func (x *BlogPostInput) GetTitle() string { @@ -9934,7 +10664,7 @@ type AuthorInput struct { func (x *AuthorInput) Reset() { *x = AuthorInput{} - mi := &file_product_proto_msgTypes[196] + mi := &file_product_proto_msgTypes[211] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9946,7 +10676,7 @@ func (x *AuthorInput) String() string { func (*AuthorInput) ProtoMessage() {} func (x *AuthorInput) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[196] + mi := &file_product_proto_msgTypes[211] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9959,7 +10689,7 @@ func (x *AuthorInput) ProtoReflect() protoreflect.Message { // Deprecated: Use AuthorInput.ProtoReflect.Descriptor instead. func (*AuthorInput) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{196} + return file_product_proto_rawDescGZIP(), []int{211} } func (x *AuthorInput) GetName() string { @@ -10032,6 +10762,74 @@ func (x *AuthorInput) GetProjectTeams() *ListOfListOfUserInput { return nil } +type ProductDetails struct { + state protoimpl.MessageState `protogen:"open.v1"` + Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` + Description string `protobuf:"bytes,2,opt,name=description,proto3" json:"description,omitempty"` + ReviewSummary *ActionResult `protobuf:"bytes,3,opt,name=review_summary,json=reviewSummary,proto3" json:"review_summary,omitempty"` + RecommendedPet *Animal `protobuf:"bytes,4,opt,name=recommended_pet,json=recommendedPet,proto3" json:"recommended_pet,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ProductDetails) Reset() { + *x = ProductDetails{} + mi := &file_product_proto_msgTypes[212] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ProductDetails) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ProductDetails) ProtoMessage() {} + +func (x *ProductDetails) ProtoReflect() protoreflect.Message { + mi := &file_product_proto_msgTypes[212] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ProductDetails.ProtoReflect.Descriptor instead. +func (*ProductDetails) Descriptor() ([]byte, []int) { + return file_product_proto_rawDescGZIP(), []int{212} +} + +func (x *ProductDetails) GetId() string { + if x != nil { + return x.Id + } + return "" +} + +func (x *ProductDetails) GetDescription() string { + if x != nil { + return x.Description + } + return "" +} + +func (x *ProductDetails) GetReviewSummary() *ActionResult { + if x != nil { + return x.ReviewSummary + } + return nil +} + +func (x *ProductDetails) GetRecommendedPet() *Animal { + if x != nil { + return x.RecommendedPet + } + return nil +} + type NestedTypeB struct { state protoimpl.MessageState `protogen:"open.v1"` Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` @@ -10043,7 +10841,7 @@ type NestedTypeB struct { func (x *NestedTypeB) Reset() { *x = NestedTypeB{} - mi := &file_product_proto_msgTypes[197] + mi := &file_product_proto_msgTypes[213] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -10055,7 +10853,7 @@ func (x *NestedTypeB) String() string { func (*NestedTypeB) ProtoMessage() {} func (x *NestedTypeB) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[197] + mi := &file_product_proto_msgTypes[213] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -10068,7 +10866,7 @@ func (x *NestedTypeB) ProtoReflect() protoreflect.Message { // Deprecated: Use NestedTypeB.ProtoReflect.Descriptor instead. func (*NestedTypeB) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{197} + return file_product_proto_rawDescGZIP(), []int{213} } func (x *NestedTypeB) GetId() string { @@ -10102,7 +10900,7 @@ type NestedTypeC struct { func (x *NestedTypeC) Reset() { *x = NestedTypeC{} - mi := &file_product_proto_msgTypes[198] + mi := &file_product_proto_msgTypes[214] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -10114,7 +10912,7 @@ func (x *NestedTypeC) String() string { func (*NestedTypeC) ProtoMessage() {} func (x *NestedTypeC) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[198] + mi := &file_product_proto_msgTypes[214] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -10127,7 +10925,7 @@ func (x *NestedTypeC) ProtoReflect() protoreflect.Message { // Deprecated: Use NestedTypeC.ProtoReflect.Descriptor instead. func (*NestedTypeC) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{198} + return file_product_proto_rawDescGZIP(), []int{214} } func (x *NestedTypeC) GetId() string { @@ -10156,7 +10954,7 @@ type FilterType struct { func (x *FilterType) Reset() { *x = FilterType{} - mi := &file_product_proto_msgTypes[199] + mi := &file_product_proto_msgTypes[215] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -10168,7 +10966,7 @@ func (x *FilterType) String() string { func (*FilterType) ProtoMessage() {} func (x *FilterType) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[199] + mi := &file_product_proto_msgTypes[215] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -10181,7 +10979,7 @@ func (x *FilterType) ProtoReflect() protoreflect.Message { // Deprecated: Use FilterType.ProtoReflect.Descriptor instead. func (*FilterType) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{199} + return file_product_proto_rawDescGZIP(), []int{215} } func (x *FilterType) GetName() string { @@ -10222,7 +11020,7 @@ type Pagination struct { func (x *Pagination) Reset() { *x = Pagination{} - mi := &file_product_proto_msgTypes[200] + mi := &file_product_proto_msgTypes[216] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -10234,7 +11032,7 @@ func (x *Pagination) String() string { func (*Pagination) ProtoMessage() {} func (x *Pagination) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[200] + mi := &file_product_proto_msgTypes[216] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -10247,7 +11045,7 @@ func (x *Pagination) ProtoReflect() protoreflect.Message { // Deprecated: Use Pagination.ProtoReflect.Descriptor instead. func (*Pagination) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{200} + return file_product_proto_rawDescGZIP(), []int{216} } func (x *Pagination) GetPage() int32 { @@ -10275,7 +11073,7 @@ type OrderLineInput struct { func (x *OrderLineInput) Reset() { *x = OrderLineInput{} - mi := &file_product_proto_msgTypes[201] + mi := &file_product_proto_msgTypes[217] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -10287,7 +11085,7 @@ func (x *OrderLineInput) String() string { func (*OrderLineInput) ProtoMessage() {} func (x *OrderLineInput) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[201] + mi := &file_product_proto_msgTypes[217] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -10300,7 +11098,7 @@ func (x *OrderLineInput) ProtoReflect() protoreflect.Message { // Deprecated: Use OrderLineInput.ProtoReflect.Descriptor instead. func (*OrderLineInput) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{201} + return file_product_proto_rawDescGZIP(), []int{217} } func (x *OrderLineInput) GetProductId() string { @@ -10335,7 +11133,7 @@ type OrderLine struct { func (x *OrderLine) Reset() { *x = OrderLine{} - mi := &file_product_proto_msgTypes[202] + mi := &file_product_proto_msgTypes[218] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -10347,7 +11145,7 @@ func (x *OrderLine) String() string { func (*OrderLine) ProtoMessage() {} func (x *OrderLine) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[202] + mi := &file_product_proto_msgTypes[218] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -10360,7 +11158,7 @@ func (x *OrderLine) ProtoReflect() protoreflect.Message { // Deprecated: Use OrderLine.ProtoReflect.Descriptor instead. func (*OrderLine) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{202} + return file_product_proto_rawDescGZIP(), []int{218} } func (x *OrderLine) GetProductId() string { @@ -10396,7 +11194,7 @@ type Subcategory struct { func (x *Subcategory) Reset() { *x = Subcategory{} - mi := &file_product_proto_msgTypes[203] + mi := &file_product_proto_msgTypes[219] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -10408,7 +11206,7 @@ func (x *Subcategory) String() string { func (*Subcategory) ProtoMessage() {} func (x *Subcategory) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[203] + mi := &file_product_proto_msgTypes[219] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -10421,7 +11219,7 @@ func (x *Subcategory) ProtoReflect() protoreflect.Message { // Deprecated: Use Subcategory.ProtoReflect.Descriptor instead. func (*Subcategory) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{203} + return file_product_proto_rawDescGZIP(), []int{219} } func (x *Subcategory) GetId() string { @@ -10466,7 +11264,7 @@ type CategoryMetrics struct { func (x *CategoryMetrics) Reset() { *x = CategoryMetrics{} - mi := &file_product_proto_msgTypes[204] + mi := &file_product_proto_msgTypes[220] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -10478,7 +11276,7 @@ func (x *CategoryMetrics) String() string { func (*CategoryMetrics) ProtoMessage() {} func (x *CategoryMetrics) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[204] + mi := &file_product_proto_msgTypes[220] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -10491,7 +11289,7 @@ func (x *CategoryMetrics) ProtoReflect() protoreflect.Message { // Deprecated: Use CategoryMetrics.ProtoReflect.Descriptor instead. func (*CategoryMetrics) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{204} + return file_product_proto_rawDescGZIP(), []int{220} } func (x *CategoryMetrics) GetId() string { @@ -10548,7 +11346,7 @@ type Cat struct { func (x *Cat) Reset() { *x = Cat{} - mi := &file_product_proto_msgTypes[205] + mi := &file_product_proto_msgTypes[221] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -10560,7 +11358,7 @@ func (x *Cat) String() string { func (*Cat) ProtoMessage() {} func (x *Cat) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[205] + mi := &file_product_proto_msgTypes[221] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -10573,7 +11371,7 @@ func (x *Cat) ProtoReflect() protoreflect.Message { // Deprecated: Use Cat.ProtoReflect.Descriptor instead. func (*Cat) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{205} + return file_product_proto_rawDescGZIP(), []int{221} } func (x *Cat) GetId() string { @@ -10616,7 +11414,7 @@ type Dog struct { func (x *Dog) Reset() { *x = Dog{} - mi := &file_product_proto_msgTypes[206] + mi := &file_product_proto_msgTypes[222] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -10628,7 +11426,7 @@ func (x *Dog) String() string { func (*Dog) ProtoMessage() {} func (x *Dog) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[206] + mi := &file_product_proto_msgTypes[222] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -10641,7 +11439,7 @@ func (x *Dog) ProtoReflect() protoreflect.Message { // Deprecated: Use Dog.ProtoReflect.Descriptor instead. func (*Dog) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{206} + return file_product_proto_rawDescGZIP(), []int{222} } func (x *Dog) GetId() string { @@ -10682,7 +11480,7 @@ type ActionSuccess struct { func (x *ActionSuccess) Reset() { *x = ActionSuccess{} - mi := &file_product_proto_msgTypes[207] + mi := &file_product_proto_msgTypes[223] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -10694,7 +11492,7 @@ func (x *ActionSuccess) String() string { func (*ActionSuccess) ProtoMessage() {} func (x *ActionSuccess) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[207] + mi := &file_product_proto_msgTypes[223] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -10707,7 +11505,7 @@ func (x *ActionSuccess) ProtoReflect() protoreflect.Message { // Deprecated: Use ActionSuccess.ProtoReflect.Descriptor instead. func (*ActionSuccess) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{207} + return file_product_proto_rawDescGZIP(), []int{223} } func (x *ActionSuccess) GetMessage() string { @@ -10734,7 +11532,7 @@ type ActionError struct { func (x *ActionError) Reset() { *x = ActionError{} - mi := &file_product_proto_msgTypes[208] + mi := &file_product_proto_msgTypes[224] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -10746,7 +11544,7 @@ func (x *ActionError) String() string { func (*ActionError) ProtoMessage() {} func (x *ActionError) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[208] + mi := &file_product_proto_msgTypes[224] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -10759,7 +11557,7 @@ func (x *ActionError) ProtoReflect() protoreflect.Message { // Deprecated: Use ActionError.ProtoReflect.Descriptor instead. func (*ActionError) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{208} + return file_product_proto_rawDescGZIP(), []int{224} } func (x *ActionError) GetMessage() string { @@ -10788,7 +11586,7 @@ type TestDetails struct { func (x *TestDetails) Reset() { *x = TestDetails{} - mi := &file_product_proto_msgTypes[209] + mi := &file_product_proto_msgTypes[225] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -10800,7 +11598,7 @@ func (x *TestDetails) String() string { func (*TestDetails) ProtoMessage() {} func (x *TestDetails) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[209] + mi := &file_product_proto_msgTypes[225] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -10813,7 +11611,7 @@ func (x *TestDetails) ProtoReflect() protoreflect.Message { // Deprecated: Use TestDetails.ProtoReflect.Descriptor instead. func (*TestDetails) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{209} + return file_product_proto_rawDescGZIP(), []int{225} } func (x *TestDetails) GetId() string { @@ -10854,7 +11652,7 @@ type CategoryInput struct { func (x *CategoryInput) Reset() { *x = CategoryInput{} - mi := &file_product_proto_msgTypes[210] + mi := &file_product_proto_msgTypes[226] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -10866,7 +11664,7 @@ func (x *CategoryInput) String() string { func (*CategoryInput) ProtoMessage() {} func (x *CategoryInput) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[210] + mi := &file_product_proto_msgTypes[226] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -10879,7 +11677,7 @@ func (x *CategoryInput) ProtoReflect() protoreflect.Message { // Deprecated: Use CategoryInput.ProtoReflect.Descriptor instead. func (*CategoryInput) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{210} + return file_product_proto_rawDescGZIP(), []int{226} } func (x *CategoryInput) GetName() string { @@ -10908,7 +11706,7 @@ type ProductCountFilter struct { func (x *ProductCountFilter) Reset() { *x = ProductCountFilter{} - mi := &file_product_proto_msgTypes[211] + mi := &file_product_proto_msgTypes[227] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -10920,7 +11718,7 @@ func (x *ProductCountFilter) String() string { func (*ProductCountFilter) ProtoMessage() {} func (x *ProductCountFilter) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[211] + mi := &file_product_proto_msgTypes[227] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -10933,7 +11731,7 @@ func (x *ProductCountFilter) ProtoReflect() protoreflect.Message { // Deprecated: Use ProductCountFilter.ProtoReflect.Descriptor instead. func (*ProductCountFilter) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{211} + return file_product_proto_rawDescGZIP(), []int{227} } func (x *ProductCountFilter) GetMinPrice() *wrapperspb.DoubleValue { @@ -10977,7 +11775,7 @@ type SubcategoryItemFilter struct { func (x *SubcategoryItemFilter) Reset() { *x = SubcategoryItemFilter{} - mi := &file_product_proto_msgTypes[212] + mi := &file_product_proto_msgTypes[228] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -10989,7 +11787,7 @@ func (x *SubcategoryItemFilter) String() string { func (*SubcategoryItemFilter) ProtoMessage() {} func (x *SubcategoryItemFilter) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[212] + mi := &file_product_proto_msgTypes[228] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -11002,7 +11800,7 @@ func (x *SubcategoryItemFilter) ProtoReflect() protoreflect.Message { // Deprecated: Use SubcategoryItemFilter.ProtoReflect.Descriptor instead. func (*SubcategoryItemFilter) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{212} + return file_product_proto_rawDescGZIP(), []int{228} } func (x *SubcategoryItemFilter) GetMinPrice() *wrapperspb.DoubleValue { @@ -11051,7 +11849,7 @@ type ShippingEstimateInput struct { func (x *ShippingEstimateInput) Reset() { *x = ShippingEstimateInput{} - mi := &file_product_proto_msgTypes[213] + mi := &file_product_proto_msgTypes[229] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -11063,7 +11861,7 @@ func (x *ShippingEstimateInput) String() string { func (*ShippingEstimateInput) ProtoMessage() {} func (x *ShippingEstimateInput) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[213] + mi := &file_product_proto_msgTypes[229] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -11076,7 +11874,7 @@ func (x *ShippingEstimateInput) ProtoReflect() protoreflect.Message { // Deprecated: Use ShippingEstimateInput.ProtoReflect.Descriptor instead. func (*ShippingEstimateInput) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{213} + return file_product_proto_rawDescGZIP(), []int{229} } func (x *ShippingEstimateInput) GetDestination() ShippingDestination { @@ -11109,7 +11907,7 @@ type ListOfAuthorFilter_List struct { func (x *ListOfAuthorFilter_List) Reset() { *x = ListOfAuthorFilter_List{} - mi := &file_product_proto_msgTypes[214] + mi := &file_product_proto_msgTypes[230] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -11121,7 +11919,7 @@ func (x *ListOfAuthorFilter_List) String() string { func (*ListOfAuthorFilter_List) ProtoMessage() {} func (x *ListOfAuthorFilter_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[214] + mi := &file_product_proto_msgTypes[230] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -11153,7 +11951,7 @@ type ListOfAuthorInput_List struct { func (x *ListOfAuthorInput_List) Reset() { *x = ListOfAuthorInput_List{} - mi := &file_product_proto_msgTypes[215] + mi := &file_product_proto_msgTypes[231] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -11165,7 +11963,7 @@ func (x *ListOfAuthorInput_List) String() string { func (*ListOfAuthorInput_List) ProtoMessage() {} func (x *ListOfAuthorInput_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[215] + mi := &file_product_proto_msgTypes[231] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -11197,7 +11995,7 @@ type ListOfBlogPost_List struct { func (x *ListOfBlogPost_List) Reset() { *x = ListOfBlogPost_List{} - mi := &file_product_proto_msgTypes[216] + mi := &file_product_proto_msgTypes[232] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -11209,7 +12007,7 @@ func (x *ListOfBlogPost_List) String() string { func (*ListOfBlogPost_List) ProtoMessage() {} func (x *ListOfBlogPost_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[216] + mi := &file_product_proto_msgTypes[232] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -11241,7 +12039,7 @@ type ListOfBlogPostFilter_List struct { func (x *ListOfBlogPostFilter_List) Reset() { *x = ListOfBlogPostFilter_List{} - mi := &file_product_proto_msgTypes[217] + mi := &file_product_proto_msgTypes[233] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -11253,7 +12051,7 @@ func (x *ListOfBlogPostFilter_List) String() string { func (*ListOfBlogPostFilter_List) ProtoMessage() {} func (x *ListOfBlogPostFilter_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[217] + mi := &file_product_proto_msgTypes[233] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -11285,7 +12083,7 @@ type ListOfBlogPostInput_List struct { func (x *ListOfBlogPostInput_List) Reset() { *x = ListOfBlogPostInput_List{} - mi := &file_product_proto_msgTypes[218] + mi := &file_product_proto_msgTypes[234] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -11297,7 +12095,7 @@ func (x *ListOfBlogPostInput_List) String() string { func (*ListOfBlogPostInput_List) ProtoMessage() {} func (x *ListOfBlogPostInput_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[218] + mi := &file_product_proto_msgTypes[234] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -11329,7 +12127,7 @@ type ListOfBoolean_List struct { func (x *ListOfBoolean_List) Reset() { *x = ListOfBoolean_List{} - mi := &file_product_proto_msgTypes[219] + mi := &file_product_proto_msgTypes[235] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -11341,7 +12139,7 @@ func (x *ListOfBoolean_List) String() string { func (*ListOfBoolean_List) ProtoMessage() {} func (x *ListOfBoolean_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[219] + mi := &file_product_proto_msgTypes[235] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -11373,7 +12171,7 @@ type ListOfCategory_List struct { func (x *ListOfCategory_List) Reset() { *x = ListOfCategory_List{} - mi := &file_product_proto_msgTypes[220] + mi := &file_product_proto_msgTypes[236] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -11385,7 +12183,7 @@ func (x *ListOfCategory_List) String() string { func (*ListOfCategory_List) ProtoMessage() {} func (x *ListOfCategory_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[220] + mi := &file_product_proto_msgTypes[236] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -11417,7 +12215,7 @@ type ListOfCategoryInput_List struct { func (x *ListOfCategoryInput_List) Reset() { *x = ListOfCategoryInput_List{} - mi := &file_product_proto_msgTypes[221] + mi := &file_product_proto_msgTypes[237] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -11429,7 +12227,7 @@ func (x *ListOfCategoryInput_List) String() string { func (*ListOfCategoryInput_List) ProtoMessage() {} func (x *ListOfCategoryInput_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[221] + mi := &file_product_proto_msgTypes[237] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -11461,7 +12259,7 @@ type ListOfFloat_List struct { func (x *ListOfFloat_List) Reset() { *x = ListOfFloat_List{} - mi := &file_product_proto_msgTypes[222] + mi := &file_product_proto_msgTypes[238] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -11473,7 +12271,7 @@ func (x *ListOfFloat_List) String() string { func (*ListOfFloat_List) ProtoMessage() {} func (x *ListOfFloat_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[222] + mi := &file_product_proto_msgTypes[238] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -11505,7 +12303,7 @@ type ListOfListOfCategory_List struct { func (x *ListOfListOfCategory_List) Reset() { *x = ListOfListOfCategory_List{} - mi := &file_product_proto_msgTypes[223] + mi := &file_product_proto_msgTypes[239] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -11517,7 +12315,7 @@ func (x *ListOfListOfCategory_List) String() string { func (*ListOfListOfCategory_List) ProtoMessage() {} func (x *ListOfListOfCategory_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[223] + mi := &file_product_proto_msgTypes[239] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -11549,7 +12347,7 @@ type ListOfListOfCategoryInput_List struct { func (x *ListOfListOfCategoryInput_List) Reset() { *x = ListOfListOfCategoryInput_List{} - mi := &file_product_proto_msgTypes[224] + mi := &file_product_proto_msgTypes[240] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -11561,7 +12359,7 @@ func (x *ListOfListOfCategoryInput_List) String() string { func (*ListOfListOfCategoryInput_List) ProtoMessage() {} func (x *ListOfListOfCategoryInput_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[224] + mi := &file_product_proto_msgTypes[240] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -11593,7 +12391,7 @@ type ListOfListOfString_List struct { func (x *ListOfListOfString_List) Reset() { *x = ListOfListOfString_List{} - mi := &file_product_proto_msgTypes[225] + mi := &file_product_proto_msgTypes[241] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -11605,7 +12403,7 @@ func (x *ListOfListOfString_List) String() string { func (*ListOfListOfString_List) ProtoMessage() {} func (x *ListOfListOfString_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[225] + mi := &file_product_proto_msgTypes[241] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -11637,7 +12435,7 @@ type ListOfListOfUser_List struct { func (x *ListOfListOfUser_List) Reset() { *x = ListOfListOfUser_List{} - mi := &file_product_proto_msgTypes[226] + mi := &file_product_proto_msgTypes[242] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -11649,7 +12447,7 @@ func (x *ListOfListOfUser_List) String() string { func (*ListOfListOfUser_List) ProtoMessage() {} func (x *ListOfListOfUser_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[226] + mi := &file_product_proto_msgTypes[242] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -11681,7 +12479,7 @@ type ListOfListOfUserInput_List struct { func (x *ListOfListOfUserInput_List) Reset() { *x = ListOfListOfUserInput_List{} - mi := &file_product_proto_msgTypes[227] + mi := &file_product_proto_msgTypes[243] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -11693,7 +12491,7 @@ func (x *ListOfListOfUserInput_List) String() string { func (*ListOfListOfUserInput_List) ProtoMessage() {} func (x *ListOfListOfUserInput_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[227] + mi := &file_product_proto_msgTypes[243] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -11725,7 +12523,7 @@ type ListOfOrderLine_List struct { func (x *ListOfOrderLine_List) Reset() { *x = ListOfOrderLine_List{} - mi := &file_product_proto_msgTypes[228] + mi := &file_product_proto_msgTypes[244] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -11737,7 +12535,7 @@ func (x *ListOfOrderLine_List) String() string { func (*ListOfOrderLine_List) ProtoMessage() {} func (x *ListOfOrderLine_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[228] + mi := &file_product_proto_msgTypes[244] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -11769,7 +12567,7 @@ type ListOfProduct_List struct { func (x *ListOfProduct_List) Reset() { *x = ListOfProduct_List{} - mi := &file_product_proto_msgTypes[229] + mi := &file_product_proto_msgTypes[245] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -11781,7 +12579,7 @@ func (x *ListOfProduct_List) String() string { func (*ListOfProduct_List) ProtoMessage() {} func (x *ListOfProduct_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[229] + mi := &file_product_proto_msgTypes[245] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -11813,7 +12611,7 @@ type ListOfString_List struct { func (x *ListOfString_List) Reset() { *x = ListOfString_List{} - mi := &file_product_proto_msgTypes[230] + mi := &file_product_proto_msgTypes[246] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -11825,7 +12623,7 @@ func (x *ListOfString_List) String() string { func (*ListOfString_List) ProtoMessage() {} func (x *ListOfString_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[230] + mi := &file_product_proto_msgTypes[246] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -11857,7 +12655,7 @@ type ListOfSubcategory_List struct { func (x *ListOfSubcategory_List) Reset() { *x = ListOfSubcategory_List{} - mi := &file_product_proto_msgTypes[231] + mi := &file_product_proto_msgTypes[247] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -11869,7 +12667,7 @@ func (x *ListOfSubcategory_List) String() string { func (*ListOfSubcategory_List) ProtoMessage() {} func (x *ListOfSubcategory_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[231] + mi := &file_product_proto_msgTypes[247] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -11901,7 +12699,7 @@ type ListOfUser_List struct { func (x *ListOfUser_List) Reset() { *x = ListOfUser_List{} - mi := &file_product_proto_msgTypes[232] + mi := &file_product_proto_msgTypes[248] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -11913,7 +12711,7 @@ func (x *ListOfUser_List) String() string { func (*ListOfUser_List) ProtoMessage() {} func (x *ListOfUser_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[232] + mi := &file_product_proto_msgTypes[248] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -11945,7 +12743,7 @@ type ListOfUserInput_List struct { func (x *ListOfUserInput_List) Reset() { *x = ListOfUserInput_List{} - mi := &file_product_proto_msgTypes[233] + mi := &file_product_proto_msgTypes[249] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -11957,7 +12755,7 @@ func (x *ListOfUserInput_List) String() string { func (*ListOfUserInput_List) ProtoMessage() {} func (x *ListOfUserInput_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[233] + mi := &file_product_proto_msgTypes[249] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -12285,7 +13083,48 @@ const file_product_proto_rawDesc = "" + "'ResolveProductRecommendedCategoryResult\x12F\n" + "\x14recommended_category\x18\x01 \x01(\v2\x13.productv1.CategoryR\x13recommendedCategory\"w\n" + ")ResolveProductRecommendedCategoryResponse\x12J\n" + - "\x06result\x18\x01 \x03(\v22.productv1.ResolveProductRecommendedCategoryResultR\x06result\"Z\n" + + "\x06result\x18\x01 \x03(\v22.productv1.ResolveProductRecommendedCategoryResultR\x06result\"Q\n" + + "&ResolveProductMascotRecommendationArgs\x12'\n" + + "\x0finclude_details\x18\x01 \x01(\bR\x0eincludeDetails\"O\n" + + ")ResolveProductMascotRecommendationContext\x12\x0e\n" + + "\x02id\x18\x01 \x01(\tR\x02id\x12\x12\n" + + "\x04name\x18\x02 \x01(\tR\x04name\"\xcd\x01\n" + + ")ResolveProductMascotRecommendationRequest\x12N\n" + + "\acontext\x18\x01 \x03(\v24.productv1.ResolveProductMascotRecommendationContextR\acontext\x12P\n" + + "\n" + + "field_args\x18\x02 \x01(\v21.productv1.ResolveProductMascotRecommendationArgsR\tfieldArgs\"r\n" + + "(ResolveProductMascotRecommendationResult\x12F\n" + + "\x15mascot_recommendation\x18\x01 \x01(\v2\x11.productv1.AnimalR\x14mascotRecommendation\"y\n" + + "*ResolveProductMascotRecommendationResponse\x12K\n" + + "\x06result\x18\x01 \x03(\v23.productv1.ResolveProductMascotRecommendationResultR\x06result\"N\n" + + "\x1dResolveProductStockStatusArgs\x12-\n" + + "\x12check_availability\x18\x01 \x01(\bR\x11checkAvailability\"\\\n" + + " ResolveProductStockStatusContext\x12\x0e\n" + + "\x02id\x18\x01 \x01(\tR\x02id\x12\x12\n" + + "\x04name\x18\x02 \x01(\tR\x04name\x12\x14\n" + + "\x05price\x18\x03 \x01(\x01R\x05price\"\xb2\x01\n" + + " ResolveProductStockStatusRequest\x12E\n" + + "\acontext\x18\x01 \x03(\v2+.productv1.ResolveProductStockStatusContextR\acontext\x12G\n" + + "\n" + + "field_args\x18\x02 \x01(\v2(.productv1.ResolveProductStockStatusArgsR\tfieldArgs\"]\n" + + "\x1fResolveProductStockStatusResult\x12:\n" + + "\fstock_status\x18\x01 \x01(\v2\x17.productv1.ActionResultR\vstockStatus\"g\n" + + "!ResolveProductStockStatusResponse\x12B\n" + + "\x06result\x18\x01 \x03(\v2*.productv1.ResolveProductStockStatusResultR\x06result\"M\n" + + " ResolveProductProductDetailsArgs\x12)\n" + + "\x10include_extended\x18\x01 \x01(\bR\x0fincludeExtended\"_\n" + + "#ResolveProductProductDetailsContext\x12\x0e\n" + + "\x02id\x18\x01 \x01(\tR\x02id\x12\x12\n" + + "\x04name\x18\x02 \x01(\tR\x04name\x12\x14\n" + + "\x05price\x18\x03 \x01(\x01R\x05price\"\xbb\x01\n" + + "#ResolveProductProductDetailsRequest\x12H\n" + + "\acontext\x18\x01 \x03(\v2..productv1.ResolveProductProductDetailsContextR\acontext\x12J\n" + + "\n" + + "field_args\x18\x02 \x01(\v2+.productv1.ResolveProductProductDetailsArgsR\tfieldArgs\"h\n" + + "\"ResolveProductProductDetailsResult\x12B\n" + + "\x0fproduct_details\x18\x01 \x01(\v2\x19.productv1.ProductDetailsR\x0eproductDetails\"m\n" + + "$ResolveProductProductDetailsResponse\x12E\n" + + "\x06result\x18\x01 \x03(\v2-.productv1.ResolveProductProductDetailsResultR\x06result\"Z\n" + "\x1fResolveCategoryProductCountArgs\x127\n" + "\afilters\x18\x01 \x01(\v2\x1d.productv1.ProductCountFilterR\afilters\"H\n" + "\"ResolveCategoryProductCountContext\x12\x0e\n" + @@ -12583,7 +13422,12 @@ const file_product_proto_rawDesc = "" + "\x13favorite_categories\x18\b \x03(\v2\x18.productv1.CategoryInputR\x12favoriteCategories\x12E\n" + "\rauthor_groups\x18\t \x01(\v2 .productv1.ListOfListOfUserInputR\fauthorGroups\x12E\n" + "\rproject_teams\x18\n" + - " \x01(\v2 .productv1.ListOfListOfUserInputR\fprojectTeams\"W\n" + + " \x01(\v2 .productv1.ListOfListOfUserInputR\fprojectTeams\"\xbe\x01\n" + + "\x0eProductDetails\x12\x0e\n" + + "\x02id\x18\x01 \x01(\tR\x02id\x12 \n" + + "\vdescription\x18\x02 \x01(\tR\vdescription\x12>\n" + + "\x0ereview_summary\x18\x03 \x01(\v2\x17.productv1.ActionResultR\rreviewSummary\x12:\n" + + "\x0frecommended_pet\x18\x04 \x01(\v2\x11.productv1.AnimalR\x0erecommendedPet\"W\n" + "\vNestedTypeB\x12\x0e\n" + "\x02id\x18\x01 \x01(\tR\x02id\x12\x12\n" + "\x04name\x18\x02 \x01(\tR\x04name\x12$\n" + @@ -12680,7 +13524,7 @@ const file_product_proto_rawDesc = "" + " SHIPPING_DESTINATION_UNSPECIFIED\x10\x00\x12!\n" + "\x1dSHIPPING_DESTINATION_DOMESTIC\x10\x01\x12 \n" + "\x1cSHIPPING_DESTINATION_EXPRESS\x10\x02\x12&\n" + - "\"SHIPPING_DESTINATION_INTERNATIONAL\x10\x032\xca2\n" + + "\"SHIPPING_DESTINATION_INTERNATIONAL\x10\x032\xde5\n" + "\x0eProductService\x12`\n" + "\x11LookupProductById\x12#.productv1.LookupProductByIdRequest\x1a$.productv1.LookupProductByIdResponse\"\x00\x12`\n" + "\x11LookupStorageById\x12#.productv1.LookupStorageByIdRequest\x1a$.productv1.LookupStorageByIdResponse\"\x00\x12f\n" + @@ -12735,9 +13579,12 @@ const file_product_proto_rawDesc = "" + "\x15ResolveCategoryMascot\x12'.productv1.ResolveCategoryMascotRequest\x1a(.productv1.ResolveCategoryMascotResponse\"\x00\x12\x9c\x01\n" + "%ResolveCategoryMetricsNormalizedScore\x127.productv1.ResolveCategoryMetricsNormalizedScoreRequest\x1a8.productv1.ResolveCategoryMetricsNormalizedScoreResponse\"\x00\x12\x87\x01\n" + "\x1eResolveCategoryPopularityScore\x120.productv1.ResolveCategoryPopularityScoreRequest\x1a1.productv1.ResolveCategoryPopularityScoreResponse\"\x00\x12~\n" + - "\x1bResolveCategoryProductCount\x12-.productv1.ResolveCategoryProductCountRequest\x1a..productv1.ResolveCategoryProductCountResponse\"\x00\x12\x90\x01\n" + + "\x1bResolveCategoryProductCount\x12-.productv1.ResolveCategoryProductCountRequest\x1a..productv1.ResolveCategoryProductCountResponse\"\x00\x12\x93\x01\n" + + "\"ResolveProductMascotRecommendation\x124.productv1.ResolveProductMascotRecommendationRequest\x1a5.productv1.ResolveProductMascotRecommendationResponse\"\x00\x12\x81\x01\n" + + "\x1cResolveProductProductDetails\x12..productv1.ResolveProductProductDetailsRequest\x1a/.productv1.ResolveProductProductDetailsResponse\"\x00\x12\x90\x01\n" + "!ResolveProductRecommendedCategory\x123.productv1.ResolveProductRecommendedCategoryRequest\x1a4.productv1.ResolveProductRecommendedCategoryResponse\"\x00\x12\x87\x01\n" + - "\x1eResolveProductShippingEstimate\x120.productv1.ResolveProductShippingEstimateRequest\x1a1.productv1.ResolveProductShippingEstimateResponse\"\x00\x12~\n" + + "\x1eResolveProductShippingEstimate\x120.productv1.ResolveProductShippingEstimateRequest\x1a1.productv1.ResolveProductShippingEstimateResponse\"\x00\x12x\n" + + "\x19ResolveProductStockStatus\x12+.productv1.ResolveProductStockStatusRequest\x1a,.productv1.ResolveProductStockStatusResponse\"\x00\x12~\n" + "\x1bResolveSubcategoryItemCount\x12-.productv1.ResolveSubcategoryItemCountRequest\x1a..productv1.ResolveSubcategoryItemCountResponse\"\x00\x12~\n" + "\x1bResolveTestContainerDetails\x12-.productv1.ResolveTestContainerDetailsRequest\x1a..productv1.ResolveTestContainerDetailsResponse\"\x00B%Z#cosmo/pkg/proto/productv1;productv1b\x06proto3" @@ -12754,7 +13601,7 @@ func file_product_proto_rawDescGZIP() []byte { } var file_product_proto_enumTypes = make([]protoimpl.EnumInfo, 2) -var file_product_proto_msgTypes = make([]protoimpl.MessageInfo, 234) +var file_product_proto_msgTypes = make([]protoimpl.MessageInfo, 250) var file_product_proto_goTypes = []any{ (CategoryKind)(0), // 0: productv1.CategoryKind (ShippingDestination)(0), // 1: productv1.ShippingDestination @@ -12885,488 +13732,524 @@ var file_product_proto_goTypes = []any{ (*ResolveProductRecommendedCategoryRequest)(nil), // 126: productv1.ResolveProductRecommendedCategoryRequest (*ResolveProductRecommendedCategoryResult)(nil), // 127: productv1.ResolveProductRecommendedCategoryResult (*ResolveProductRecommendedCategoryResponse)(nil), // 128: productv1.ResolveProductRecommendedCategoryResponse - (*ResolveCategoryProductCountArgs)(nil), // 129: productv1.ResolveCategoryProductCountArgs - (*ResolveCategoryProductCountContext)(nil), // 130: productv1.ResolveCategoryProductCountContext - (*ResolveCategoryProductCountRequest)(nil), // 131: productv1.ResolveCategoryProductCountRequest - (*ResolveCategoryProductCountResult)(nil), // 132: productv1.ResolveCategoryProductCountResult - (*ResolveCategoryProductCountResponse)(nil), // 133: productv1.ResolveCategoryProductCountResponse - (*ResolveCategoryPopularityScoreArgs)(nil), // 134: productv1.ResolveCategoryPopularityScoreArgs - (*ResolveCategoryPopularityScoreContext)(nil), // 135: productv1.ResolveCategoryPopularityScoreContext - (*ResolveCategoryPopularityScoreRequest)(nil), // 136: productv1.ResolveCategoryPopularityScoreRequest - (*ResolveCategoryPopularityScoreResult)(nil), // 137: productv1.ResolveCategoryPopularityScoreResult - (*ResolveCategoryPopularityScoreResponse)(nil), // 138: productv1.ResolveCategoryPopularityScoreResponse - (*ResolveCategoryCategoryMetricsArgs)(nil), // 139: productv1.ResolveCategoryCategoryMetricsArgs - (*ResolveCategoryCategoryMetricsContext)(nil), // 140: productv1.ResolveCategoryCategoryMetricsContext - (*ResolveCategoryCategoryMetricsRequest)(nil), // 141: productv1.ResolveCategoryCategoryMetricsRequest - (*ResolveCategoryCategoryMetricsResult)(nil), // 142: productv1.ResolveCategoryCategoryMetricsResult - (*ResolveCategoryCategoryMetricsResponse)(nil), // 143: productv1.ResolveCategoryCategoryMetricsResponse - (*ResolveCategoryMascotArgs)(nil), // 144: productv1.ResolveCategoryMascotArgs - (*ResolveCategoryMascotContext)(nil), // 145: productv1.ResolveCategoryMascotContext - (*ResolveCategoryMascotRequest)(nil), // 146: productv1.ResolveCategoryMascotRequest - (*ResolveCategoryMascotResult)(nil), // 147: productv1.ResolveCategoryMascotResult - (*ResolveCategoryMascotResponse)(nil), // 148: productv1.ResolveCategoryMascotResponse - (*ResolveCategoryCategoryStatusArgs)(nil), // 149: productv1.ResolveCategoryCategoryStatusArgs - (*ResolveCategoryCategoryStatusContext)(nil), // 150: productv1.ResolveCategoryCategoryStatusContext - (*ResolveCategoryCategoryStatusRequest)(nil), // 151: productv1.ResolveCategoryCategoryStatusRequest - (*ResolveCategoryCategoryStatusResult)(nil), // 152: productv1.ResolveCategoryCategoryStatusResult - (*ResolveCategoryCategoryStatusResponse)(nil), // 153: productv1.ResolveCategoryCategoryStatusResponse - (*ResolveSubcategoryItemCountArgs)(nil), // 154: productv1.ResolveSubcategoryItemCountArgs - (*ResolveSubcategoryItemCountContext)(nil), // 155: productv1.ResolveSubcategoryItemCountContext - (*ResolveSubcategoryItemCountRequest)(nil), // 156: productv1.ResolveSubcategoryItemCountRequest - (*ResolveSubcategoryItemCountResult)(nil), // 157: productv1.ResolveSubcategoryItemCountResult - (*ResolveSubcategoryItemCountResponse)(nil), // 158: productv1.ResolveSubcategoryItemCountResponse - (*ResolveCategoryMetricsNormalizedScoreArgs)(nil), // 159: productv1.ResolveCategoryMetricsNormalizedScoreArgs - (*ResolveCategoryMetricsNormalizedScoreContext)(nil), // 160: productv1.ResolveCategoryMetricsNormalizedScoreContext - (*ResolveCategoryMetricsNormalizedScoreRequest)(nil), // 161: productv1.ResolveCategoryMetricsNormalizedScoreRequest - (*ResolveCategoryMetricsNormalizedScoreResult)(nil), // 162: productv1.ResolveCategoryMetricsNormalizedScoreResult - (*ResolveCategoryMetricsNormalizedScoreResponse)(nil), // 163: productv1.ResolveCategoryMetricsNormalizedScoreResponse - (*ResolveTestContainerDetailsArgs)(nil), // 164: productv1.ResolveTestContainerDetailsArgs - (*ResolveTestContainerDetailsContext)(nil), // 165: productv1.ResolveTestContainerDetailsContext - (*ResolveTestContainerDetailsRequest)(nil), // 166: productv1.ResolveTestContainerDetailsRequest - (*ResolveTestContainerDetailsResult)(nil), // 167: productv1.ResolveTestContainerDetailsResult - (*ResolveTestContainerDetailsResponse)(nil), // 168: productv1.ResolveTestContainerDetailsResponse - (*Product)(nil), // 169: productv1.Product - (*Storage)(nil), // 170: productv1.Storage - (*Warehouse)(nil), // 171: productv1.Warehouse - (*User)(nil), // 172: productv1.User - (*NestedTypeA)(nil), // 173: productv1.NestedTypeA - (*RecursiveType)(nil), // 174: productv1.RecursiveType - (*TypeWithMultipleFilterFields)(nil), // 175: productv1.TypeWithMultipleFilterFields - (*FilterTypeInput)(nil), // 176: productv1.FilterTypeInput - (*ComplexFilterTypeInput)(nil), // 177: productv1.ComplexFilterTypeInput - (*TypeWithComplexFilterInput)(nil), // 178: productv1.TypeWithComplexFilterInput - (*OrderInput)(nil), // 179: productv1.OrderInput - (*Order)(nil), // 180: productv1.Order - (*Category)(nil), // 181: productv1.Category - (*CategoryFilter)(nil), // 182: productv1.CategoryFilter - (*Animal)(nil), // 183: productv1.Animal - (*SearchInput)(nil), // 184: productv1.SearchInput - (*SearchResult)(nil), // 185: productv1.SearchResult - (*NullableFieldsType)(nil), // 186: productv1.NullableFieldsType - (*NullableFieldsFilter)(nil), // 187: productv1.NullableFieldsFilter - (*BlogPost)(nil), // 188: productv1.BlogPost - (*BlogPostFilter)(nil), // 189: productv1.BlogPostFilter - (*Author)(nil), // 190: productv1.Author - (*AuthorFilter)(nil), // 191: productv1.AuthorFilter - (*TestContainer)(nil), // 192: productv1.TestContainer - (*UserInput)(nil), // 193: productv1.UserInput - (*ActionInput)(nil), // 194: productv1.ActionInput - (*ActionResult)(nil), // 195: productv1.ActionResult - (*NullableFieldsInput)(nil), // 196: productv1.NullableFieldsInput - (*BlogPostInput)(nil), // 197: productv1.BlogPostInput - (*AuthorInput)(nil), // 198: productv1.AuthorInput - (*NestedTypeB)(nil), // 199: productv1.NestedTypeB - (*NestedTypeC)(nil), // 200: productv1.NestedTypeC - (*FilterType)(nil), // 201: productv1.FilterType - (*Pagination)(nil), // 202: productv1.Pagination - (*OrderLineInput)(nil), // 203: productv1.OrderLineInput - (*OrderLine)(nil), // 204: productv1.OrderLine - (*Subcategory)(nil), // 205: productv1.Subcategory - (*CategoryMetrics)(nil), // 206: productv1.CategoryMetrics - (*Cat)(nil), // 207: productv1.Cat - (*Dog)(nil), // 208: productv1.Dog - (*ActionSuccess)(nil), // 209: productv1.ActionSuccess - (*ActionError)(nil), // 210: productv1.ActionError - (*TestDetails)(nil), // 211: productv1.TestDetails - (*CategoryInput)(nil), // 212: productv1.CategoryInput - (*ProductCountFilter)(nil), // 213: productv1.ProductCountFilter - (*SubcategoryItemFilter)(nil), // 214: productv1.SubcategoryItemFilter - (*ShippingEstimateInput)(nil), // 215: productv1.ShippingEstimateInput - (*ListOfAuthorFilter_List)(nil), // 216: productv1.ListOfAuthorFilter.List - (*ListOfAuthorInput_List)(nil), // 217: productv1.ListOfAuthorInput.List - (*ListOfBlogPost_List)(nil), // 218: productv1.ListOfBlogPost.List - (*ListOfBlogPostFilter_List)(nil), // 219: productv1.ListOfBlogPostFilter.List - (*ListOfBlogPostInput_List)(nil), // 220: productv1.ListOfBlogPostInput.List - (*ListOfBoolean_List)(nil), // 221: productv1.ListOfBoolean.List - (*ListOfCategory_List)(nil), // 222: productv1.ListOfCategory.List - (*ListOfCategoryInput_List)(nil), // 223: productv1.ListOfCategoryInput.List - (*ListOfFloat_List)(nil), // 224: productv1.ListOfFloat.List - (*ListOfListOfCategory_List)(nil), // 225: productv1.ListOfListOfCategory.List - (*ListOfListOfCategoryInput_List)(nil), // 226: productv1.ListOfListOfCategoryInput.List - (*ListOfListOfString_List)(nil), // 227: productv1.ListOfListOfString.List - (*ListOfListOfUser_List)(nil), // 228: productv1.ListOfListOfUser.List - (*ListOfListOfUserInput_List)(nil), // 229: productv1.ListOfListOfUserInput.List - (*ListOfOrderLine_List)(nil), // 230: productv1.ListOfOrderLine.List - (*ListOfProduct_List)(nil), // 231: productv1.ListOfProduct.List - (*ListOfString_List)(nil), // 232: productv1.ListOfString.List - (*ListOfSubcategory_List)(nil), // 233: productv1.ListOfSubcategory.List - (*ListOfUser_List)(nil), // 234: productv1.ListOfUser.List - (*ListOfUserInput_List)(nil), // 235: productv1.ListOfUserInput.List - (*wrapperspb.Int32Value)(nil), // 236: google.protobuf.Int32Value - (*wrapperspb.StringValue)(nil), // 237: google.protobuf.StringValue - (*wrapperspb.DoubleValue)(nil), // 238: google.protobuf.DoubleValue - (*wrapperspb.BoolValue)(nil), // 239: google.protobuf.BoolValue + (*ResolveProductMascotRecommendationArgs)(nil), // 129: productv1.ResolveProductMascotRecommendationArgs + (*ResolveProductMascotRecommendationContext)(nil), // 130: productv1.ResolveProductMascotRecommendationContext + (*ResolveProductMascotRecommendationRequest)(nil), // 131: productv1.ResolveProductMascotRecommendationRequest + (*ResolveProductMascotRecommendationResult)(nil), // 132: productv1.ResolveProductMascotRecommendationResult + (*ResolveProductMascotRecommendationResponse)(nil), // 133: productv1.ResolveProductMascotRecommendationResponse + (*ResolveProductStockStatusArgs)(nil), // 134: productv1.ResolveProductStockStatusArgs + (*ResolveProductStockStatusContext)(nil), // 135: productv1.ResolveProductStockStatusContext + (*ResolveProductStockStatusRequest)(nil), // 136: productv1.ResolveProductStockStatusRequest + (*ResolveProductStockStatusResult)(nil), // 137: productv1.ResolveProductStockStatusResult + (*ResolveProductStockStatusResponse)(nil), // 138: productv1.ResolveProductStockStatusResponse + (*ResolveProductProductDetailsArgs)(nil), // 139: productv1.ResolveProductProductDetailsArgs + (*ResolveProductProductDetailsContext)(nil), // 140: productv1.ResolveProductProductDetailsContext + (*ResolveProductProductDetailsRequest)(nil), // 141: productv1.ResolveProductProductDetailsRequest + (*ResolveProductProductDetailsResult)(nil), // 142: productv1.ResolveProductProductDetailsResult + (*ResolveProductProductDetailsResponse)(nil), // 143: productv1.ResolveProductProductDetailsResponse + (*ResolveCategoryProductCountArgs)(nil), // 144: productv1.ResolveCategoryProductCountArgs + (*ResolveCategoryProductCountContext)(nil), // 145: productv1.ResolveCategoryProductCountContext + (*ResolveCategoryProductCountRequest)(nil), // 146: productv1.ResolveCategoryProductCountRequest + (*ResolveCategoryProductCountResult)(nil), // 147: productv1.ResolveCategoryProductCountResult + (*ResolveCategoryProductCountResponse)(nil), // 148: productv1.ResolveCategoryProductCountResponse + (*ResolveCategoryPopularityScoreArgs)(nil), // 149: productv1.ResolveCategoryPopularityScoreArgs + (*ResolveCategoryPopularityScoreContext)(nil), // 150: productv1.ResolveCategoryPopularityScoreContext + (*ResolveCategoryPopularityScoreRequest)(nil), // 151: productv1.ResolveCategoryPopularityScoreRequest + (*ResolveCategoryPopularityScoreResult)(nil), // 152: productv1.ResolveCategoryPopularityScoreResult + (*ResolveCategoryPopularityScoreResponse)(nil), // 153: productv1.ResolveCategoryPopularityScoreResponse + (*ResolveCategoryCategoryMetricsArgs)(nil), // 154: productv1.ResolveCategoryCategoryMetricsArgs + (*ResolveCategoryCategoryMetricsContext)(nil), // 155: productv1.ResolveCategoryCategoryMetricsContext + (*ResolveCategoryCategoryMetricsRequest)(nil), // 156: productv1.ResolveCategoryCategoryMetricsRequest + (*ResolveCategoryCategoryMetricsResult)(nil), // 157: productv1.ResolveCategoryCategoryMetricsResult + (*ResolveCategoryCategoryMetricsResponse)(nil), // 158: productv1.ResolveCategoryCategoryMetricsResponse + (*ResolveCategoryMascotArgs)(nil), // 159: productv1.ResolveCategoryMascotArgs + (*ResolveCategoryMascotContext)(nil), // 160: productv1.ResolveCategoryMascotContext + (*ResolveCategoryMascotRequest)(nil), // 161: productv1.ResolveCategoryMascotRequest + (*ResolveCategoryMascotResult)(nil), // 162: productv1.ResolveCategoryMascotResult + (*ResolveCategoryMascotResponse)(nil), // 163: productv1.ResolveCategoryMascotResponse + (*ResolveCategoryCategoryStatusArgs)(nil), // 164: productv1.ResolveCategoryCategoryStatusArgs + (*ResolveCategoryCategoryStatusContext)(nil), // 165: productv1.ResolveCategoryCategoryStatusContext + (*ResolveCategoryCategoryStatusRequest)(nil), // 166: productv1.ResolveCategoryCategoryStatusRequest + (*ResolveCategoryCategoryStatusResult)(nil), // 167: productv1.ResolveCategoryCategoryStatusResult + (*ResolveCategoryCategoryStatusResponse)(nil), // 168: productv1.ResolveCategoryCategoryStatusResponse + (*ResolveSubcategoryItemCountArgs)(nil), // 169: productv1.ResolveSubcategoryItemCountArgs + (*ResolveSubcategoryItemCountContext)(nil), // 170: productv1.ResolveSubcategoryItemCountContext + (*ResolveSubcategoryItemCountRequest)(nil), // 171: productv1.ResolveSubcategoryItemCountRequest + (*ResolveSubcategoryItemCountResult)(nil), // 172: productv1.ResolveSubcategoryItemCountResult + (*ResolveSubcategoryItemCountResponse)(nil), // 173: productv1.ResolveSubcategoryItemCountResponse + (*ResolveCategoryMetricsNormalizedScoreArgs)(nil), // 174: productv1.ResolveCategoryMetricsNormalizedScoreArgs + (*ResolveCategoryMetricsNormalizedScoreContext)(nil), // 175: productv1.ResolveCategoryMetricsNormalizedScoreContext + (*ResolveCategoryMetricsNormalizedScoreRequest)(nil), // 176: productv1.ResolveCategoryMetricsNormalizedScoreRequest + (*ResolveCategoryMetricsNormalizedScoreResult)(nil), // 177: productv1.ResolveCategoryMetricsNormalizedScoreResult + (*ResolveCategoryMetricsNormalizedScoreResponse)(nil), // 178: productv1.ResolveCategoryMetricsNormalizedScoreResponse + (*ResolveTestContainerDetailsArgs)(nil), // 179: productv1.ResolveTestContainerDetailsArgs + (*ResolveTestContainerDetailsContext)(nil), // 180: productv1.ResolveTestContainerDetailsContext + (*ResolveTestContainerDetailsRequest)(nil), // 181: productv1.ResolveTestContainerDetailsRequest + (*ResolveTestContainerDetailsResult)(nil), // 182: productv1.ResolveTestContainerDetailsResult + (*ResolveTestContainerDetailsResponse)(nil), // 183: productv1.ResolveTestContainerDetailsResponse + (*Product)(nil), // 184: productv1.Product + (*Storage)(nil), // 185: productv1.Storage + (*Warehouse)(nil), // 186: productv1.Warehouse + (*User)(nil), // 187: productv1.User + (*NestedTypeA)(nil), // 188: productv1.NestedTypeA + (*RecursiveType)(nil), // 189: productv1.RecursiveType + (*TypeWithMultipleFilterFields)(nil), // 190: productv1.TypeWithMultipleFilterFields + (*FilterTypeInput)(nil), // 191: productv1.FilterTypeInput + (*ComplexFilterTypeInput)(nil), // 192: productv1.ComplexFilterTypeInput + (*TypeWithComplexFilterInput)(nil), // 193: productv1.TypeWithComplexFilterInput + (*OrderInput)(nil), // 194: productv1.OrderInput + (*Order)(nil), // 195: productv1.Order + (*Category)(nil), // 196: productv1.Category + (*CategoryFilter)(nil), // 197: productv1.CategoryFilter + (*Animal)(nil), // 198: productv1.Animal + (*SearchInput)(nil), // 199: productv1.SearchInput + (*SearchResult)(nil), // 200: productv1.SearchResult + (*NullableFieldsType)(nil), // 201: productv1.NullableFieldsType + (*NullableFieldsFilter)(nil), // 202: productv1.NullableFieldsFilter + (*BlogPost)(nil), // 203: productv1.BlogPost + (*BlogPostFilter)(nil), // 204: productv1.BlogPostFilter + (*Author)(nil), // 205: productv1.Author + (*AuthorFilter)(nil), // 206: productv1.AuthorFilter + (*TestContainer)(nil), // 207: productv1.TestContainer + (*UserInput)(nil), // 208: productv1.UserInput + (*ActionInput)(nil), // 209: productv1.ActionInput + (*ActionResult)(nil), // 210: productv1.ActionResult + (*NullableFieldsInput)(nil), // 211: productv1.NullableFieldsInput + (*BlogPostInput)(nil), // 212: productv1.BlogPostInput + (*AuthorInput)(nil), // 213: productv1.AuthorInput + (*ProductDetails)(nil), // 214: productv1.ProductDetails + (*NestedTypeB)(nil), // 215: productv1.NestedTypeB + (*NestedTypeC)(nil), // 216: productv1.NestedTypeC + (*FilterType)(nil), // 217: productv1.FilterType + (*Pagination)(nil), // 218: productv1.Pagination + (*OrderLineInput)(nil), // 219: productv1.OrderLineInput + (*OrderLine)(nil), // 220: productv1.OrderLine + (*Subcategory)(nil), // 221: productv1.Subcategory + (*CategoryMetrics)(nil), // 222: productv1.CategoryMetrics + (*Cat)(nil), // 223: productv1.Cat + (*Dog)(nil), // 224: productv1.Dog + (*ActionSuccess)(nil), // 225: productv1.ActionSuccess + (*ActionError)(nil), // 226: productv1.ActionError + (*TestDetails)(nil), // 227: productv1.TestDetails + (*CategoryInput)(nil), // 228: productv1.CategoryInput + (*ProductCountFilter)(nil), // 229: productv1.ProductCountFilter + (*SubcategoryItemFilter)(nil), // 230: productv1.SubcategoryItemFilter + (*ShippingEstimateInput)(nil), // 231: productv1.ShippingEstimateInput + (*ListOfAuthorFilter_List)(nil), // 232: productv1.ListOfAuthorFilter.List + (*ListOfAuthorInput_List)(nil), // 233: productv1.ListOfAuthorInput.List + (*ListOfBlogPost_List)(nil), // 234: productv1.ListOfBlogPost.List + (*ListOfBlogPostFilter_List)(nil), // 235: productv1.ListOfBlogPostFilter.List + (*ListOfBlogPostInput_List)(nil), // 236: productv1.ListOfBlogPostInput.List + (*ListOfBoolean_List)(nil), // 237: productv1.ListOfBoolean.List + (*ListOfCategory_List)(nil), // 238: productv1.ListOfCategory.List + (*ListOfCategoryInput_List)(nil), // 239: productv1.ListOfCategoryInput.List + (*ListOfFloat_List)(nil), // 240: productv1.ListOfFloat.List + (*ListOfListOfCategory_List)(nil), // 241: productv1.ListOfListOfCategory.List + (*ListOfListOfCategoryInput_List)(nil), // 242: productv1.ListOfListOfCategoryInput.List + (*ListOfListOfString_List)(nil), // 243: productv1.ListOfListOfString.List + (*ListOfListOfUser_List)(nil), // 244: productv1.ListOfListOfUser.List + (*ListOfListOfUserInput_List)(nil), // 245: productv1.ListOfListOfUserInput.List + (*ListOfOrderLine_List)(nil), // 246: productv1.ListOfOrderLine.List + (*ListOfProduct_List)(nil), // 247: productv1.ListOfProduct.List + (*ListOfString_List)(nil), // 248: productv1.ListOfString.List + (*ListOfSubcategory_List)(nil), // 249: productv1.ListOfSubcategory.List + (*ListOfUser_List)(nil), // 250: productv1.ListOfUser.List + (*ListOfUserInput_List)(nil), // 251: productv1.ListOfUserInput.List + (*wrapperspb.Int32Value)(nil), // 252: google.protobuf.Int32Value + (*wrapperspb.StringValue)(nil), // 253: google.protobuf.StringValue + (*wrapperspb.DoubleValue)(nil), // 254: google.protobuf.DoubleValue + (*wrapperspb.BoolValue)(nil), // 255: google.protobuf.BoolValue } var file_product_proto_depIdxs = []int32{ - 216, // 0: productv1.ListOfAuthorFilter.list:type_name -> productv1.ListOfAuthorFilter.List - 217, // 1: productv1.ListOfAuthorInput.list:type_name -> productv1.ListOfAuthorInput.List - 218, // 2: productv1.ListOfBlogPost.list:type_name -> productv1.ListOfBlogPost.List - 219, // 3: productv1.ListOfBlogPostFilter.list:type_name -> productv1.ListOfBlogPostFilter.List - 220, // 4: productv1.ListOfBlogPostInput.list:type_name -> productv1.ListOfBlogPostInput.List - 221, // 5: productv1.ListOfBoolean.list:type_name -> productv1.ListOfBoolean.List - 222, // 6: productv1.ListOfCategory.list:type_name -> productv1.ListOfCategory.List - 223, // 7: productv1.ListOfCategoryInput.list:type_name -> productv1.ListOfCategoryInput.List - 224, // 8: productv1.ListOfFloat.list:type_name -> productv1.ListOfFloat.List - 225, // 9: productv1.ListOfListOfCategory.list:type_name -> productv1.ListOfListOfCategory.List - 226, // 10: productv1.ListOfListOfCategoryInput.list:type_name -> productv1.ListOfListOfCategoryInput.List - 227, // 11: productv1.ListOfListOfString.list:type_name -> productv1.ListOfListOfString.List - 228, // 12: productv1.ListOfListOfUser.list:type_name -> productv1.ListOfListOfUser.List - 229, // 13: productv1.ListOfListOfUserInput.list:type_name -> productv1.ListOfListOfUserInput.List - 230, // 14: productv1.ListOfOrderLine.list:type_name -> productv1.ListOfOrderLine.List - 231, // 15: productv1.ListOfProduct.list:type_name -> productv1.ListOfProduct.List - 232, // 16: productv1.ListOfString.list:type_name -> productv1.ListOfString.List - 233, // 17: productv1.ListOfSubcategory.list:type_name -> productv1.ListOfSubcategory.List - 234, // 18: productv1.ListOfUser.list:type_name -> productv1.ListOfUser.List - 235, // 19: productv1.ListOfUserInput.list:type_name -> productv1.ListOfUserInput.List + 232, // 0: productv1.ListOfAuthorFilter.list:type_name -> productv1.ListOfAuthorFilter.List + 233, // 1: productv1.ListOfAuthorInput.list:type_name -> productv1.ListOfAuthorInput.List + 234, // 2: productv1.ListOfBlogPost.list:type_name -> productv1.ListOfBlogPost.List + 235, // 3: productv1.ListOfBlogPostFilter.list:type_name -> productv1.ListOfBlogPostFilter.List + 236, // 4: productv1.ListOfBlogPostInput.list:type_name -> productv1.ListOfBlogPostInput.List + 237, // 5: productv1.ListOfBoolean.list:type_name -> productv1.ListOfBoolean.List + 238, // 6: productv1.ListOfCategory.list:type_name -> productv1.ListOfCategory.List + 239, // 7: productv1.ListOfCategoryInput.list:type_name -> productv1.ListOfCategoryInput.List + 240, // 8: productv1.ListOfFloat.list:type_name -> productv1.ListOfFloat.List + 241, // 9: productv1.ListOfListOfCategory.list:type_name -> productv1.ListOfListOfCategory.List + 242, // 10: productv1.ListOfListOfCategoryInput.list:type_name -> productv1.ListOfListOfCategoryInput.List + 243, // 11: productv1.ListOfListOfString.list:type_name -> productv1.ListOfListOfString.List + 244, // 12: productv1.ListOfListOfUser.list:type_name -> productv1.ListOfListOfUser.List + 245, // 13: productv1.ListOfListOfUserInput.list:type_name -> productv1.ListOfListOfUserInput.List + 246, // 14: productv1.ListOfOrderLine.list:type_name -> productv1.ListOfOrderLine.List + 247, // 15: productv1.ListOfProduct.list:type_name -> productv1.ListOfProduct.List + 248, // 16: productv1.ListOfString.list:type_name -> productv1.ListOfString.List + 249, // 17: productv1.ListOfSubcategory.list:type_name -> productv1.ListOfSubcategory.List + 250, // 18: productv1.ListOfUser.list:type_name -> productv1.ListOfUser.List + 251, // 19: productv1.ListOfUserInput.list:type_name -> productv1.ListOfUserInput.List 22, // 20: productv1.LookupProductByIdRequest.keys:type_name -> productv1.LookupProductByIdRequestKey - 169, // 21: productv1.LookupProductByIdResponse.result:type_name -> productv1.Product + 184, // 21: productv1.LookupProductByIdResponse.result:type_name -> productv1.Product 25, // 22: productv1.LookupStorageByIdRequest.keys:type_name -> productv1.LookupStorageByIdRequestKey - 170, // 23: productv1.LookupStorageByIdResponse.result:type_name -> productv1.Storage + 185, // 23: productv1.LookupStorageByIdResponse.result:type_name -> productv1.Storage 28, // 24: productv1.LookupWarehouseByIdRequest.keys:type_name -> productv1.LookupWarehouseByIdRequestKey - 171, // 25: productv1.LookupWarehouseByIdResponse.result:type_name -> productv1.Warehouse - 172, // 26: productv1.QueryUsersResponse.users:type_name -> productv1.User - 172, // 27: productv1.QueryUserResponse.user:type_name -> productv1.User - 173, // 28: productv1.QueryNestedTypeResponse.nested_type:type_name -> productv1.NestedTypeA - 174, // 29: productv1.QueryRecursiveTypeResponse.recursive_type:type_name -> productv1.RecursiveType - 175, // 30: productv1.QueryTypeFilterWithArgumentsResponse.type_filter_with_arguments:type_name -> productv1.TypeWithMultipleFilterFields - 176, // 31: productv1.QueryTypeWithMultipleFilterFieldsRequest.filter:type_name -> productv1.FilterTypeInput - 175, // 32: productv1.QueryTypeWithMultipleFilterFieldsResponse.type_with_multiple_filter_fields:type_name -> productv1.TypeWithMultipleFilterFields - 177, // 33: productv1.QueryComplexFilterTypeRequest.filter:type_name -> productv1.ComplexFilterTypeInput - 178, // 34: productv1.QueryComplexFilterTypeResponse.complex_filter_type:type_name -> productv1.TypeWithComplexFilterInput - 179, // 35: productv1.QueryCalculateTotalsRequest.orders:type_name -> productv1.OrderInput - 180, // 36: productv1.QueryCalculateTotalsResponse.calculate_totals:type_name -> productv1.Order - 181, // 37: productv1.QueryCategoriesResponse.categories:type_name -> productv1.Category + 186, // 25: productv1.LookupWarehouseByIdResponse.result:type_name -> productv1.Warehouse + 187, // 26: productv1.QueryUsersResponse.users:type_name -> productv1.User + 187, // 27: productv1.QueryUserResponse.user:type_name -> productv1.User + 188, // 28: productv1.QueryNestedTypeResponse.nested_type:type_name -> productv1.NestedTypeA + 189, // 29: productv1.QueryRecursiveTypeResponse.recursive_type:type_name -> productv1.RecursiveType + 190, // 30: productv1.QueryTypeFilterWithArgumentsResponse.type_filter_with_arguments:type_name -> productv1.TypeWithMultipleFilterFields + 191, // 31: productv1.QueryTypeWithMultipleFilterFieldsRequest.filter:type_name -> productv1.FilterTypeInput + 190, // 32: productv1.QueryTypeWithMultipleFilterFieldsResponse.type_with_multiple_filter_fields:type_name -> productv1.TypeWithMultipleFilterFields + 192, // 33: productv1.QueryComplexFilterTypeRequest.filter:type_name -> productv1.ComplexFilterTypeInput + 193, // 34: productv1.QueryComplexFilterTypeResponse.complex_filter_type:type_name -> productv1.TypeWithComplexFilterInput + 194, // 35: productv1.QueryCalculateTotalsRequest.orders:type_name -> productv1.OrderInput + 195, // 36: productv1.QueryCalculateTotalsResponse.calculate_totals:type_name -> productv1.Order + 196, // 37: productv1.QueryCategoriesResponse.categories:type_name -> productv1.Category 0, // 38: productv1.QueryCategoriesByKindRequest.kind:type_name -> productv1.CategoryKind - 181, // 39: productv1.QueryCategoriesByKindResponse.categories_by_kind:type_name -> productv1.Category + 196, // 39: productv1.QueryCategoriesByKindResponse.categories_by_kind:type_name -> productv1.Category 0, // 40: productv1.QueryCategoriesByKindsRequest.kinds:type_name -> productv1.CategoryKind - 181, // 41: productv1.QueryCategoriesByKindsResponse.categories_by_kinds:type_name -> productv1.Category - 182, // 42: productv1.QueryFilterCategoriesRequest.filter:type_name -> productv1.CategoryFilter - 181, // 43: productv1.QueryFilterCategoriesResponse.filter_categories:type_name -> productv1.Category - 183, // 44: productv1.QueryRandomPetResponse.random_pet:type_name -> productv1.Animal - 183, // 45: productv1.QueryAllPetsResponse.all_pets:type_name -> productv1.Animal - 184, // 46: productv1.QuerySearchRequest.input:type_name -> productv1.SearchInput - 185, // 47: productv1.QuerySearchResponse.search:type_name -> productv1.SearchResult - 185, // 48: productv1.QueryRandomSearchResultResponse.random_search_result:type_name -> productv1.SearchResult - 186, // 49: productv1.QueryNullableFieldsTypeResponse.nullable_fields_type:type_name -> productv1.NullableFieldsType - 186, // 50: productv1.QueryNullableFieldsTypeByIdResponse.nullable_fields_type_by_id:type_name -> productv1.NullableFieldsType - 187, // 51: productv1.QueryNullableFieldsTypeWithFilterRequest.filter:type_name -> productv1.NullableFieldsFilter - 186, // 52: productv1.QueryNullableFieldsTypeWithFilterResponse.nullable_fields_type_with_filter:type_name -> productv1.NullableFieldsType - 186, // 53: productv1.QueryAllNullableFieldsTypesResponse.all_nullable_fields_types:type_name -> productv1.NullableFieldsType - 188, // 54: productv1.QueryBlogPostResponse.blog_post:type_name -> productv1.BlogPost - 188, // 55: productv1.QueryBlogPostByIdResponse.blog_post_by_id:type_name -> productv1.BlogPost - 189, // 56: productv1.QueryBlogPostsWithFilterRequest.filter:type_name -> productv1.BlogPostFilter - 188, // 57: productv1.QueryBlogPostsWithFilterResponse.blog_posts_with_filter:type_name -> productv1.BlogPost - 188, // 58: productv1.QueryAllBlogPostsResponse.all_blog_posts:type_name -> productv1.BlogPost - 190, // 59: productv1.QueryAuthorResponse.author:type_name -> productv1.Author - 190, // 60: productv1.QueryAuthorByIdResponse.author_by_id:type_name -> productv1.Author - 191, // 61: productv1.QueryAuthorsWithFilterRequest.filter:type_name -> productv1.AuthorFilter - 190, // 62: productv1.QueryAuthorsWithFilterResponse.authors_with_filter:type_name -> productv1.Author - 190, // 63: productv1.QueryAllAuthorsResponse.all_authors:type_name -> productv1.Author + 196, // 41: productv1.QueryCategoriesByKindsResponse.categories_by_kinds:type_name -> productv1.Category + 197, // 42: productv1.QueryFilterCategoriesRequest.filter:type_name -> productv1.CategoryFilter + 196, // 43: productv1.QueryFilterCategoriesResponse.filter_categories:type_name -> productv1.Category + 198, // 44: productv1.QueryRandomPetResponse.random_pet:type_name -> productv1.Animal + 198, // 45: productv1.QueryAllPetsResponse.all_pets:type_name -> productv1.Animal + 199, // 46: productv1.QuerySearchRequest.input:type_name -> productv1.SearchInput + 200, // 47: productv1.QuerySearchResponse.search:type_name -> productv1.SearchResult + 200, // 48: productv1.QueryRandomSearchResultResponse.random_search_result:type_name -> productv1.SearchResult + 201, // 49: productv1.QueryNullableFieldsTypeResponse.nullable_fields_type:type_name -> productv1.NullableFieldsType + 201, // 50: productv1.QueryNullableFieldsTypeByIdResponse.nullable_fields_type_by_id:type_name -> productv1.NullableFieldsType + 202, // 51: productv1.QueryNullableFieldsTypeWithFilterRequest.filter:type_name -> productv1.NullableFieldsFilter + 201, // 52: productv1.QueryNullableFieldsTypeWithFilterResponse.nullable_fields_type_with_filter:type_name -> productv1.NullableFieldsType + 201, // 53: productv1.QueryAllNullableFieldsTypesResponse.all_nullable_fields_types:type_name -> productv1.NullableFieldsType + 203, // 54: productv1.QueryBlogPostResponse.blog_post:type_name -> productv1.BlogPost + 203, // 55: productv1.QueryBlogPostByIdResponse.blog_post_by_id:type_name -> productv1.BlogPost + 204, // 56: productv1.QueryBlogPostsWithFilterRequest.filter:type_name -> productv1.BlogPostFilter + 203, // 57: productv1.QueryBlogPostsWithFilterResponse.blog_posts_with_filter:type_name -> productv1.BlogPost + 203, // 58: productv1.QueryAllBlogPostsResponse.all_blog_posts:type_name -> productv1.BlogPost + 205, // 59: productv1.QueryAuthorResponse.author:type_name -> productv1.Author + 205, // 60: productv1.QueryAuthorByIdResponse.author_by_id:type_name -> productv1.Author + 206, // 61: productv1.QueryAuthorsWithFilterRequest.filter:type_name -> productv1.AuthorFilter + 205, // 62: productv1.QueryAuthorsWithFilterResponse.authors_with_filter:type_name -> productv1.Author + 205, // 63: productv1.QueryAllAuthorsResponse.all_authors:type_name -> productv1.Author 2, // 64: productv1.QueryBulkSearchAuthorsRequest.filters:type_name -> productv1.ListOfAuthorFilter - 190, // 65: productv1.QueryBulkSearchAuthorsResponse.bulk_search_authors:type_name -> productv1.Author + 205, // 65: productv1.QueryBulkSearchAuthorsResponse.bulk_search_authors:type_name -> productv1.Author 5, // 66: productv1.QueryBulkSearchBlogPostsRequest.filters:type_name -> productv1.ListOfBlogPostFilter - 188, // 67: productv1.QueryBulkSearchBlogPostsResponse.bulk_search_blog_posts:type_name -> productv1.BlogPost - 192, // 68: productv1.QueryTestContainerResponse.test_container:type_name -> productv1.TestContainer - 192, // 69: productv1.QueryTestContainersResponse.test_containers:type_name -> productv1.TestContainer - 193, // 70: productv1.MutationCreateUserRequest.input:type_name -> productv1.UserInput - 172, // 71: productv1.MutationCreateUserResponse.create_user:type_name -> productv1.User - 194, // 72: productv1.MutationPerformActionRequest.input:type_name -> productv1.ActionInput - 195, // 73: productv1.MutationPerformActionResponse.perform_action:type_name -> productv1.ActionResult - 196, // 74: productv1.MutationCreateNullableFieldsTypeRequest.input:type_name -> productv1.NullableFieldsInput - 186, // 75: productv1.MutationCreateNullableFieldsTypeResponse.create_nullable_fields_type:type_name -> productv1.NullableFieldsType - 196, // 76: productv1.MutationUpdateNullableFieldsTypeRequest.input:type_name -> productv1.NullableFieldsInput - 186, // 77: productv1.MutationUpdateNullableFieldsTypeResponse.update_nullable_fields_type:type_name -> productv1.NullableFieldsType - 197, // 78: productv1.MutationCreateBlogPostRequest.input:type_name -> productv1.BlogPostInput - 188, // 79: productv1.MutationCreateBlogPostResponse.create_blog_post:type_name -> productv1.BlogPost - 197, // 80: productv1.MutationUpdateBlogPostRequest.input:type_name -> productv1.BlogPostInput - 188, // 81: productv1.MutationUpdateBlogPostResponse.update_blog_post:type_name -> productv1.BlogPost - 198, // 82: productv1.MutationCreateAuthorRequest.input:type_name -> productv1.AuthorInput - 190, // 83: productv1.MutationCreateAuthorResponse.create_author:type_name -> productv1.Author - 198, // 84: productv1.MutationUpdateAuthorRequest.input:type_name -> productv1.AuthorInput - 190, // 85: productv1.MutationUpdateAuthorResponse.update_author:type_name -> productv1.Author + 203, // 67: productv1.QueryBulkSearchBlogPostsResponse.bulk_search_blog_posts:type_name -> productv1.BlogPost + 207, // 68: productv1.QueryTestContainerResponse.test_container:type_name -> productv1.TestContainer + 207, // 69: productv1.QueryTestContainersResponse.test_containers:type_name -> productv1.TestContainer + 208, // 70: productv1.MutationCreateUserRequest.input:type_name -> productv1.UserInput + 187, // 71: productv1.MutationCreateUserResponse.create_user:type_name -> productv1.User + 209, // 72: productv1.MutationPerformActionRequest.input:type_name -> productv1.ActionInput + 210, // 73: productv1.MutationPerformActionResponse.perform_action:type_name -> productv1.ActionResult + 211, // 74: productv1.MutationCreateNullableFieldsTypeRequest.input:type_name -> productv1.NullableFieldsInput + 201, // 75: productv1.MutationCreateNullableFieldsTypeResponse.create_nullable_fields_type:type_name -> productv1.NullableFieldsType + 211, // 76: productv1.MutationUpdateNullableFieldsTypeRequest.input:type_name -> productv1.NullableFieldsInput + 201, // 77: productv1.MutationUpdateNullableFieldsTypeResponse.update_nullable_fields_type:type_name -> productv1.NullableFieldsType + 212, // 78: productv1.MutationCreateBlogPostRequest.input:type_name -> productv1.BlogPostInput + 203, // 79: productv1.MutationCreateBlogPostResponse.create_blog_post:type_name -> productv1.BlogPost + 212, // 80: productv1.MutationUpdateBlogPostRequest.input:type_name -> productv1.BlogPostInput + 203, // 81: productv1.MutationUpdateBlogPostResponse.update_blog_post:type_name -> productv1.BlogPost + 213, // 82: productv1.MutationCreateAuthorRequest.input:type_name -> productv1.AuthorInput + 205, // 83: productv1.MutationCreateAuthorResponse.create_author:type_name -> productv1.Author + 213, // 84: productv1.MutationUpdateAuthorRequest.input:type_name -> productv1.AuthorInput + 205, // 85: productv1.MutationUpdateAuthorResponse.update_author:type_name -> productv1.Author 3, // 86: productv1.MutationBulkCreateAuthorsRequest.authors:type_name -> productv1.ListOfAuthorInput - 190, // 87: productv1.MutationBulkCreateAuthorsResponse.bulk_create_authors:type_name -> productv1.Author + 205, // 87: productv1.MutationBulkCreateAuthorsResponse.bulk_create_authors:type_name -> productv1.Author 3, // 88: productv1.MutationBulkUpdateAuthorsRequest.authors:type_name -> productv1.ListOfAuthorInput - 190, // 89: productv1.MutationBulkUpdateAuthorsResponse.bulk_update_authors:type_name -> productv1.Author + 205, // 89: productv1.MutationBulkUpdateAuthorsResponse.bulk_update_authors:type_name -> productv1.Author 6, // 90: productv1.MutationBulkCreateBlogPostsRequest.blog_posts:type_name -> productv1.ListOfBlogPostInput - 188, // 91: productv1.MutationBulkCreateBlogPostsResponse.bulk_create_blog_posts:type_name -> productv1.BlogPost + 203, // 91: productv1.MutationBulkCreateBlogPostsResponse.bulk_create_blog_posts:type_name -> productv1.BlogPost 6, // 92: productv1.MutationBulkUpdateBlogPostsRequest.blog_posts:type_name -> productv1.ListOfBlogPostInput - 188, // 93: productv1.MutationBulkUpdateBlogPostsResponse.bulk_update_blog_posts:type_name -> productv1.BlogPost - 215, // 94: productv1.ResolveProductShippingEstimateArgs.input:type_name -> productv1.ShippingEstimateInput + 203, // 93: productv1.MutationBulkUpdateBlogPostsResponse.bulk_update_blog_posts:type_name -> productv1.BlogPost + 231, // 94: productv1.ResolveProductShippingEstimateArgs.input:type_name -> productv1.ShippingEstimateInput 120, // 95: productv1.ResolveProductShippingEstimateRequest.context:type_name -> productv1.ResolveProductShippingEstimateContext 119, // 96: productv1.ResolveProductShippingEstimateRequest.field_args:type_name -> productv1.ResolveProductShippingEstimateArgs 122, // 97: productv1.ResolveProductShippingEstimateResponse.result:type_name -> productv1.ResolveProductShippingEstimateResult 125, // 98: productv1.ResolveProductRecommendedCategoryRequest.context:type_name -> productv1.ResolveProductRecommendedCategoryContext 124, // 99: productv1.ResolveProductRecommendedCategoryRequest.field_args:type_name -> productv1.ResolveProductRecommendedCategoryArgs - 181, // 100: productv1.ResolveProductRecommendedCategoryResult.recommended_category:type_name -> productv1.Category + 196, // 100: productv1.ResolveProductRecommendedCategoryResult.recommended_category:type_name -> productv1.Category 127, // 101: productv1.ResolveProductRecommendedCategoryResponse.result:type_name -> productv1.ResolveProductRecommendedCategoryResult - 213, // 102: productv1.ResolveCategoryProductCountArgs.filters:type_name -> productv1.ProductCountFilter - 130, // 103: productv1.ResolveCategoryProductCountRequest.context:type_name -> productv1.ResolveCategoryProductCountContext - 129, // 104: productv1.ResolveCategoryProductCountRequest.field_args:type_name -> productv1.ResolveCategoryProductCountArgs - 132, // 105: productv1.ResolveCategoryProductCountResponse.result:type_name -> productv1.ResolveCategoryProductCountResult - 236, // 106: productv1.ResolveCategoryPopularityScoreArgs.threshold:type_name -> google.protobuf.Int32Value - 135, // 107: productv1.ResolveCategoryPopularityScoreRequest.context:type_name -> productv1.ResolveCategoryPopularityScoreContext - 134, // 108: productv1.ResolveCategoryPopularityScoreRequest.field_args:type_name -> productv1.ResolveCategoryPopularityScoreArgs - 236, // 109: productv1.ResolveCategoryPopularityScoreResult.popularity_score:type_name -> google.protobuf.Int32Value - 137, // 110: productv1.ResolveCategoryPopularityScoreResponse.result:type_name -> productv1.ResolveCategoryPopularityScoreResult - 140, // 111: productv1.ResolveCategoryCategoryMetricsRequest.context:type_name -> productv1.ResolveCategoryCategoryMetricsContext - 139, // 112: productv1.ResolveCategoryCategoryMetricsRequest.field_args:type_name -> productv1.ResolveCategoryCategoryMetricsArgs - 206, // 113: productv1.ResolveCategoryCategoryMetricsResult.category_metrics:type_name -> productv1.CategoryMetrics - 142, // 114: productv1.ResolveCategoryCategoryMetricsResponse.result:type_name -> productv1.ResolveCategoryCategoryMetricsResult - 0, // 115: productv1.ResolveCategoryMascotContext.kind:type_name -> productv1.CategoryKind - 145, // 116: productv1.ResolveCategoryMascotRequest.context:type_name -> productv1.ResolveCategoryMascotContext - 144, // 117: productv1.ResolveCategoryMascotRequest.field_args:type_name -> productv1.ResolveCategoryMascotArgs - 183, // 118: productv1.ResolveCategoryMascotResult.mascot:type_name -> productv1.Animal - 147, // 119: productv1.ResolveCategoryMascotResponse.result:type_name -> productv1.ResolveCategoryMascotResult - 150, // 120: productv1.ResolveCategoryCategoryStatusRequest.context:type_name -> productv1.ResolveCategoryCategoryStatusContext - 149, // 121: productv1.ResolveCategoryCategoryStatusRequest.field_args:type_name -> productv1.ResolveCategoryCategoryStatusArgs - 195, // 122: productv1.ResolveCategoryCategoryStatusResult.category_status:type_name -> productv1.ActionResult - 152, // 123: productv1.ResolveCategoryCategoryStatusResponse.result:type_name -> productv1.ResolveCategoryCategoryStatusResult - 214, // 124: productv1.ResolveSubcategoryItemCountArgs.filters:type_name -> productv1.SubcategoryItemFilter - 155, // 125: productv1.ResolveSubcategoryItemCountRequest.context:type_name -> productv1.ResolveSubcategoryItemCountContext - 154, // 126: productv1.ResolveSubcategoryItemCountRequest.field_args:type_name -> productv1.ResolveSubcategoryItemCountArgs - 157, // 127: productv1.ResolveSubcategoryItemCountResponse.result:type_name -> productv1.ResolveSubcategoryItemCountResult - 160, // 128: productv1.ResolveCategoryMetricsNormalizedScoreRequest.context:type_name -> productv1.ResolveCategoryMetricsNormalizedScoreContext - 159, // 129: productv1.ResolveCategoryMetricsNormalizedScoreRequest.field_args:type_name -> productv1.ResolveCategoryMetricsNormalizedScoreArgs - 162, // 130: productv1.ResolveCategoryMetricsNormalizedScoreResponse.result:type_name -> productv1.ResolveCategoryMetricsNormalizedScoreResult - 165, // 131: productv1.ResolveTestContainerDetailsRequest.context:type_name -> productv1.ResolveTestContainerDetailsContext - 164, // 132: productv1.ResolveTestContainerDetailsRequest.field_args:type_name -> productv1.ResolveTestContainerDetailsArgs - 211, // 133: productv1.ResolveTestContainerDetailsResult.details:type_name -> productv1.TestDetails - 167, // 134: productv1.ResolveTestContainerDetailsResponse.result:type_name -> productv1.ResolveTestContainerDetailsResult - 199, // 135: productv1.NestedTypeA.b:type_name -> productv1.NestedTypeB - 174, // 136: productv1.RecursiveType.recursive_type:type_name -> productv1.RecursiveType - 201, // 137: productv1.ComplexFilterTypeInput.filter:type_name -> productv1.FilterType - 203, // 138: productv1.OrderInput.lines:type_name -> productv1.OrderLineInput - 16, // 139: productv1.Order.order_lines:type_name -> productv1.ListOfOrderLine - 0, // 140: productv1.Category.kind:type_name -> productv1.CategoryKind - 19, // 141: productv1.Category.subcategories:type_name -> productv1.ListOfSubcategory - 0, // 142: productv1.CategoryFilter.category:type_name -> productv1.CategoryKind - 202, // 143: productv1.CategoryFilter.pagination:type_name -> productv1.Pagination - 207, // 144: productv1.Animal.cat:type_name -> productv1.Cat - 208, // 145: productv1.Animal.dog:type_name -> productv1.Dog - 236, // 146: productv1.SearchInput.limit:type_name -> google.protobuf.Int32Value - 169, // 147: productv1.SearchResult.product:type_name -> productv1.Product - 172, // 148: productv1.SearchResult.user:type_name -> productv1.User - 181, // 149: productv1.SearchResult.category:type_name -> productv1.Category - 237, // 150: productv1.NullableFieldsType.optional_string:type_name -> google.protobuf.StringValue - 236, // 151: productv1.NullableFieldsType.optional_int:type_name -> google.protobuf.Int32Value - 238, // 152: productv1.NullableFieldsType.optional_float:type_name -> google.protobuf.DoubleValue - 239, // 153: productv1.NullableFieldsType.optional_boolean:type_name -> google.protobuf.BoolValue - 237, // 154: productv1.NullableFieldsFilter.name:type_name -> google.protobuf.StringValue - 237, // 155: productv1.NullableFieldsFilter.optional_string:type_name -> google.protobuf.StringValue - 239, // 156: productv1.NullableFieldsFilter.include_nulls:type_name -> google.protobuf.BoolValue - 18, // 157: productv1.BlogPost.optional_tags:type_name -> productv1.ListOfString - 18, // 158: productv1.BlogPost.keywords:type_name -> productv1.ListOfString - 10, // 159: productv1.BlogPost.ratings:type_name -> productv1.ListOfFloat - 7, // 160: productv1.BlogPost.is_published:type_name -> productv1.ListOfBoolean - 13, // 161: productv1.BlogPost.tag_groups:type_name -> productv1.ListOfListOfString - 13, // 162: productv1.BlogPost.related_topics:type_name -> productv1.ListOfListOfString - 13, // 163: productv1.BlogPost.comment_threads:type_name -> productv1.ListOfListOfString - 13, // 164: productv1.BlogPost.suggestions:type_name -> productv1.ListOfListOfString - 181, // 165: productv1.BlogPost.related_categories:type_name -> productv1.Category - 172, // 166: productv1.BlogPost.contributors:type_name -> productv1.User - 17, // 167: productv1.BlogPost.mentioned_products:type_name -> productv1.ListOfProduct - 20, // 168: productv1.BlogPost.mentioned_users:type_name -> productv1.ListOfUser - 11, // 169: productv1.BlogPost.category_groups:type_name -> productv1.ListOfListOfCategory - 14, // 170: productv1.BlogPost.contributor_teams:type_name -> productv1.ListOfListOfUser - 237, // 171: productv1.BlogPostFilter.title:type_name -> google.protobuf.StringValue - 239, // 172: productv1.BlogPostFilter.has_categories:type_name -> google.protobuf.BoolValue - 236, // 173: productv1.BlogPostFilter.min_tags:type_name -> google.protobuf.Int32Value - 237, // 174: productv1.Author.email:type_name -> google.protobuf.StringValue - 18, // 175: productv1.Author.social_links:type_name -> productv1.ListOfString - 13, // 176: productv1.Author.teams_by_project:type_name -> productv1.ListOfListOfString - 13, // 177: productv1.Author.collaborations:type_name -> productv1.ListOfListOfString - 4, // 178: productv1.Author.written_posts:type_name -> productv1.ListOfBlogPost - 181, // 179: productv1.Author.favorite_categories:type_name -> productv1.Category - 20, // 180: productv1.Author.related_authors:type_name -> productv1.ListOfUser - 17, // 181: productv1.Author.product_reviews:type_name -> productv1.ListOfProduct - 14, // 182: productv1.Author.author_groups:type_name -> productv1.ListOfListOfUser - 11, // 183: productv1.Author.category_preferences:type_name -> productv1.ListOfListOfCategory - 14, // 184: productv1.Author.project_teams:type_name -> productv1.ListOfListOfUser - 237, // 185: productv1.AuthorFilter.name:type_name -> google.protobuf.StringValue - 239, // 186: productv1.AuthorFilter.has_teams:type_name -> google.protobuf.BoolValue - 236, // 187: productv1.AuthorFilter.skill_count:type_name -> google.protobuf.Int32Value - 237, // 188: productv1.TestContainer.description:type_name -> google.protobuf.StringValue - 209, // 189: productv1.ActionResult.action_success:type_name -> productv1.ActionSuccess - 210, // 190: productv1.ActionResult.action_error:type_name -> productv1.ActionError - 237, // 191: productv1.NullableFieldsInput.optional_string:type_name -> google.protobuf.StringValue - 236, // 192: productv1.NullableFieldsInput.optional_int:type_name -> google.protobuf.Int32Value - 238, // 193: productv1.NullableFieldsInput.optional_float:type_name -> google.protobuf.DoubleValue - 239, // 194: productv1.NullableFieldsInput.optional_boolean:type_name -> google.protobuf.BoolValue - 18, // 195: productv1.BlogPostInput.optional_tags:type_name -> productv1.ListOfString - 18, // 196: productv1.BlogPostInput.keywords:type_name -> productv1.ListOfString - 10, // 197: productv1.BlogPostInput.ratings:type_name -> productv1.ListOfFloat - 7, // 198: productv1.BlogPostInput.is_published:type_name -> productv1.ListOfBoolean - 13, // 199: productv1.BlogPostInput.tag_groups:type_name -> productv1.ListOfListOfString - 13, // 200: productv1.BlogPostInput.related_topics:type_name -> productv1.ListOfListOfString - 13, // 201: productv1.BlogPostInput.comment_threads:type_name -> productv1.ListOfListOfString - 13, // 202: productv1.BlogPostInput.suggestions:type_name -> productv1.ListOfListOfString - 9, // 203: productv1.BlogPostInput.related_categories:type_name -> productv1.ListOfCategoryInput - 21, // 204: productv1.BlogPostInput.contributors:type_name -> productv1.ListOfUserInput - 12, // 205: productv1.BlogPostInput.category_groups:type_name -> productv1.ListOfListOfCategoryInput - 237, // 206: productv1.AuthorInput.email:type_name -> google.protobuf.StringValue - 18, // 207: productv1.AuthorInput.social_links:type_name -> productv1.ListOfString - 13, // 208: productv1.AuthorInput.teams_by_project:type_name -> productv1.ListOfListOfString - 13, // 209: productv1.AuthorInput.collaborations:type_name -> productv1.ListOfListOfString - 212, // 210: productv1.AuthorInput.favorite_categories:type_name -> productv1.CategoryInput - 15, // 211: productv1.AuthorInput.author_groups:type_name -> productv1.ListOfListOfUserInput - 15, // 212: productv1.AuthorInput.project_teams:type_name -> productv1.ListOfListOfUserInput - 200, // 213: productv1.NestedTypeB.c:type_name -> productv1.NestedTypeC - 202, // 214: productv1.FilterType.pagination:type_name -> productv1.Pagination - 18, // 215: productv1.OrderLineInput.modifiers:type_name -> productv1.ListOfString - 18, // 216: productv1.OrderLine.modifiers:type_name -> productv1.ListOfString - 237, // 217: productv1.Subcategory.description:type_name -> google.protobuf.StringValue - 181, // 218: productv1.CategoryMetrics.related_category:type_name -> productv1.Category - 183, // 219: productv1.TestDetails.pet:type_name -> productv1.Animal - 195, // 220: productv1.TestDetails.status:type_name -> productv1.ActionResult - 0, // 221: productv1.CategoryInput.kind:type_name -> productv1.CategoryKind - 238, // 222: productv1.ProductCountFilter.min_price:type_name -> google.protobuf.DoubleValue - 238, // 223: productv1.ProductCountFilter.max_price:type_name -> google.protobuf.DoubleValue - 239, // 224: productv1.ProductCountFilter.in_stock:type_name -> google.protobuf.BoolValue - 237, // 225: productv1.ProductCountFilter.search_term:type_name -> google.protobuf.StringValue - 238, // 226: productv1.SubcategoryItemFilter.min_price:type_name -> google.protobuf.DoubleValue - 238, // 227: productv1.SubcategoryItemFilter.max_price:type_name -> google.protobuf.DoubleValue - 239, // 228: productv1.SubcategoryItemFilter.in_stock:type_name -> google.protobuf.BoolValue - 239, // 229: productv1.SubcategoryItemFilter.is_active:type_name -> google.protobuf.BoolValue - 237, // 230: productv1.SubcategoryItemFilter.search_term:type_name -> google.protobuf.StringValue - 1, // 231: productv1.ShippingEstimateInput.destination:type_name -> productv1.ShippingDestination - 239, // 232: productv1.ShippingEstimateInput.expedited:type_name -> google.protobuf.BoolValue - 191, // 233: productv1.ListOfAuthorFilter.List.items:type_name -> productv1.AuthorFilter - 198, // 234: productv1.ListOfAuthorInput.List.items:type_name -> productv1.AuthorInput - 188, // 235: productv1.ListOfBlogPost.List.items:type_name -> productv1.BlogPost - 189, // 236: productv1.ListOfBlogPostFilter.List.items:type_name -> productv1.BlogPostFilter - 197, // 237: productv1.ListOfBlogPostInput.List.items:type_name -> productv1.BlogPostInput - 181, // 238: productv1.ListOfCategory.List.items:type_name -> productv1.Category - 212, // 239: productv1.ListOfCategoryInput.List.items:type_name -> productv1.CategoryInput - 8, // 240: productv1.ListOfListOfCategory.List.items:type_name -> productv1.ListOfCategory - 9, // 241: productv1.ListOfListOfCategoryInput.List.items:type_name -> productv1.ListOfCategoryInput - 18, // 242: productv1.ListOfListOfString.List.items:type_name -> productv1.ListOfString - 20, // 243: productv1.ListOfListOfUser.List.items:type_name -> productv1.ListOfUser - 21, // 244: productv1.ListOfListOfUserInput.List.items:type_name -> productv1.ListOfUserInput - 204, // 245: productv1.ListOfOrderLine.List.items:type_name -> productv1.OrderLine - 169, // 246: productv1.ListOfProduct.List.items:type_name -> productv1.Product - 205, // 247: productv1.ListOfSubcategory.List.items:type_name -> productv1.Subcategory - 172, // 248: productv1.ListOfUser.List.items:type_name -> productv1.User - 193, // 249: productv1.ListOfUserInput.List.items:type_name -> productv1.UserInput - 23, // 250: productv1.ProductService.LookupProductById:input_type -> productv1.LookupProductByIdRequest - 26, // 251: productv1.ProductService.LookupStorageById:input_type -> productv1.LookupStorageByIdRequest - 29, // 252: productv1.ProductService.LookupWarehouseById:input_type -> productv1.LookupWarehouseByIdRequest - 111, // 253: productv1.ProductService.MutationBulkCreateAuthors:input_type -> productv1.MutationBulkCreateAuthorsRequest - 115, // 254: productv1.ProductService.MutationBulkCreateBlogPosts:input_type -> productv1.MutationBulkCreateBlogPostsRequest - 113, // 255: productv1.ProductService.MutationBulkUpdateAuthors:input_type -> productv1.MutationBulkUpdateAuthorsRequest - 117, // 256: productv1.ProductService.MutationBulkUpdateBlogPosts:input_type -> productv1.MutationBulkUpdateBlogPostsRequest - 107, // 257: productv1.ProductService.MutationCreateAuthor:input_type -> productv1.MutationCreateAuthorRequest - 103, // 258: productv1.ProductService.MutationCreateBlogPost:input_type -> productv1.MutationCreateBlogPostRequest - 99, // 259: productv1.ProductService.MutationCreateNullableFieldsType:input_type -> productv1.MutationCreateNullableFieldsTypeRequest - 95, // 260: productv1.ProductService.MutationCreateUser:input_type -> productv1.MutationCreateUserRequest - 97, // 261: productv1.ProductService.MutationPerformAction:input_type -> productv1.MutationPerformActionRequest - 109, // 262: productv1.ProductService.MutationUpdateAuthor:input_type -> productv1.MutationUpdateAuthorRequest - 105, // 263: productv1.ProductService.MutationUpdateBlogPost:input_type -> productv1.MutationUpdateBlogPostRequest - 101, // 264: productv1.ProductService.MutationUpdateNullableFieldsType:input_type -> productv1.MutationUpdateNullableFieldsTypeRequest - 85, // 265: productv1.ProductService.QueryAllAuthors:input_type -> productv1.QueryAllAuthorsRequest - 77, // 266: productv1.ProductService.QueryAllBlogPosts:input_type -> productv1.QueryAllBlogPostsRequest - 69, // 267: productv1.ProductService.QueryAllNullableFieldsTypes:input_type -> productv1.QueryAllNullableFieldsTypesRequest - 57, // 268: productv1.ProductService.QueryAllPets:input_type -> productv1.QueryAllPetsRequest - 79, // 269: productv1.ProductService.QueryAuthor:input_type -> productv1.QueryAuthorRequest - 81, // 270: productv1.ProductService.QueryAuthorById:input_type -> productv1.QueryAuthorByIdRequest - 83, // 271: productv1.ProductService.QueryAuthorsWithFilter:input_type -> productv1.QueryAuthorsWithFilterRequest - 71, // 272: productv1.ProductService.QueryBlogPost:input_type -> productv1.QueryBlogPostRequest - 73, // 273: productv1.ProductService.QueryBlogPostById:input_type -> productv1.QueryBlogPostByIdRequest - 75, // 274: productv1.ProductService.QueryBlogPostsWithFilter:input_type -> productv1.QueryBlogPostsWithFilterRequest - 87, // 275: productv1.ProductService.QueryBulkSearchAuthors:input_type -> productv1.QueryBulkSearchAuthorsRequest - 89, // 276: productv1.ProductService.QueryBulkSearchBlogPosts:input_type -> productv1.QueryBulkSearchBlogPostsRequest - 45, // 277: productv1.ProductService.QueryCalculateTotals:input_type -> productv1.QueryCalculateTotalsRequest - 47, // 278: productv1.ProductService.QueryCategories:input_type -> productv1.QueryCategoriesRequest - 49, // 279: productv1.ProductService.QueryCategoriesByKind:input_type -> productv1.QueryCategoriesByKindRequest - 51, // 280: productv1.ProductService.QueryCategoriesByKinds:input_type -> productv1.QueryCategoriesByKindsRequest - 43, // 281: productv1.ProductService.QueryComplexFilterType:input_type -> productv1.QueryComplexFilterTypeRequest - 53, // 282: productv1.ProductService.QueryFilterCategories:input_type -> productv1.QueryFilterCategoriesRequest - 35, // 283: productv1.ProductService.QueryNestedType:input_type -> productv1.QueryNestedTypeRequest - 63, // 284: productv1.ProductService.QueryNullableFieldsType:input_type -> productv1.QueryNullableFieldsTypeRequest - 65, // 285: productv1.ProductService.QueryNullableFieldsTypeById:input_type -> productv1.QueryNullableFieldsTypeByIdRequest - 67, // 286: productv1.ProductService.QueryNullableFieldsTypeWithFilter:input_type -> productv1.QueryNullableFieldsTypeWithFilterRequest - 55, // 287: productv1.ProductService.QueryRandomPet:input_type -> productv1.QueryRandomPetRequest - 61, // 288: productv1.ProductService.QueryRandomSearchResult:input_type -> productv1.QueryRandomSearchResultRequest - 37, // 289: productv1.ProductService.QueryRecursiveType:input_type -> productv1.QueryRecursiveTypeRequest - 59, // 290: productv1.ProductService.QuerySearch:input_type -> productv1.QuerySearchRequest - 91, // 291: productv1.ProductService.QueryTestContainer:input_type -> productv1.QueryTestContainerRequest - 93, // 292: productv1.ProductService.QueryTestContainers:input_type -> productv1.QueryTestContainersRequest - 39, // 293: productv1.ProductService.QueryTypeFilterWithArguments:input_type -> productv1.QueryTypeFilterWithArgumentsRequest - 41, // 294: productv1.ProductService.QueryTypeWithMultipleFilterFields:input_type -> productv1.QueryTypeWithMultipleFilterFieldsRequest - 33, // 295: productv1.ProductService.QueryUser:input_type -> productv1.QueryUserRequest - 31, // 296: productv1.ProductService.QueryUsers:input_type -> productv1.QueryUsersRequest - 141, // 297: productv1.ProductService.ResolveCategoryCategoryMetrics:input_type -> productv1.ResolveCategoryCategoryMetricsRequest - 151, // 298: productv1.ProductService.ResolveCategoryCategoryStatus:input_type -> productv1.ResolveCategoryCategoryStatusRequest - 146, // 299: productv1.ProductService.ResolveCategoryMascot:input_type -> productv1.ResolveCategoryMascotRequest - 161, // 300: productv1.ProductService.ResolveCategoryMetricsNormalizedScore:input_type -> productv1.ResolveCategoryMetricsNormalizedScoreRequest - 136, // 301: productv1.ProductService.ResolveCategoryPopularityScore:input_type -> productv1.ResolveCategoryPopularityScoreRequest - 131, // 302: productv1.ProductService.ResolveCategoryProductCount:input_type -> productv1.ResolveCategoryProductCountRequest - 126, // 303: productv1.ProductService.ResolveProductRecommendedCategory:input_type -> productv1.ResolveProductRecommendedCategoryRequest - 121, // 304: productv1.ProductService.ResolveProductShippingEstimate:input_type -> productv1.ResolveProductShippingEstimateRequest - 156, // 305: productv1.ProductService.ResolveSubcategoryItemCount:input_type -> productv1.ResolveSubcategoryItemCountRequest - 166, // 306: productv1.ProductService.ResolveTestContainerDetails:input_type -> productv1.ResolveTestContainerDetailsRequest - 24, // 307: productv1.ProductService.LookupProductById:output_type -> productv1.LookupProductByIdResponse - 27, // 308: productv1.ProductService.LookupStorageById:output_type -> productv1.LookupStorageByIdResponse - 30, // 309: productv1.ProductService.LookupWarehouseById:output_type -> productv1.LookupWarehouseByIdResponse - 112, // 310: productv1.ProductService.MutationBulkCreateAuthors:output_type -> productv1.MutationBulkCreateAuthorsResponse - 116, // 311: productv1.ProductService.MutationBulkCreateBlogPosts:output_type -> productv1.MutationBulkCreateBlogPostsResponse - 114, // 312: productv1.ProductService.MutationBulkUpdateAuthors:output_type -> productv1.MutationBulkUpdateAuthorsResponse - 118, // 313: productv1.ProductService.MutationBulkUpdateBlogPosts:output_type -> productv1.MutationBulkUpdateBlogPostsResponse - 108, // 314: productv1.ProductService.MutationCreateAuthor:output_type -> productv1.MutationCreateAuthorResponse - 104, // 315: productv1.ProductService.MutationCreateBlogPost:output_type -> productv1.MutationCreateBlogPostResponse - 100, // 316: productv1.ProductService.MutationCreateNullableFieldsType:output_type -> productv1.MutationCreateNullableFieldsTypeResponse - 96, // 317: productv1.ProductService.MutationCreateUser:output_type -> productv1.MutationCreateUserResponse - 98, // 318: productv1.ProductService.MutationPerformAction:output_type -> productv1.MutationPerformActionResponse - 110, // 319: productv1.ProductService.MutationUpdateAuthor:output_type -> productv1.MutationUpdateAuthorResponse - 106, // 320: productv1.ProductService.MutationUpdateBlogPost:output_type -> productv1.MutationUpdateBlogPostResponse - 102, // 321: productv1.ProductService.MutationUpdateNullableFieldsType:output_type -> productv1.MutationUpdateNullableFieldsTypeResponse - 86, // 322: productv1.ProductService.QueryAllAuthors:output_type -> productv1.QueryAllAuthorsResponse - 78, // 323: productv1.ProductService.QueryAllBlogPosts:output_type -> productv1.QueryAllBlogPostsResponse - 70, // 324: productv1.ProductService.QueryAllNullableFieldsTypes:output_type -> productv1.QueryAllNullableFieldsTypesResponse - 58, // 325: productv1.ProductService.QueryAllPets:output_type -> productv1.QueryAllPetsResponse - 80, // 326: productv1.ProductService.QueryAuthor:output_type -> productv1.QueryAuthorResponse - 82, // 327: productv1.ProductService.QueryAuthorById:output_type -> productv1.QueryAuthorByIdResponse - 84, // 328: productv1.ProductService.QueryAuthorsWithFilter:output_type -> productv1.QueryAuthorsWithFilterResponse - 72, // 329: productv1.ProductService.QueryBlogPost:output_type -> productv1.QueryBlogPostResponse - 74, // 330: productv1.ProductService.QueryBlogPostById:output_type -> productv1.QueryBlogPostByIdResponse - 76, // 331: productv1.ProductService.QueryBlogPostsWithFilter:output_type -> productv1.QueryBlogPostsWithFilterResponse - 88, // 332: productv1.ProductService.QueryBulkSearchAuthors:output_type -> productv1.QueryBulkSearchAuthorsResponse - 90, // 333: productv1.ProductService.QueryBulkSearchBlogPosts:output_type -> productv1.QueryBulkSearchBlogPostsResponse - 46, // 334: productv1.ProductService.QueryCalculateTotals:output_type -> productv1.QueryCalculateTotalsResponse - 48, // 335: productv1.ProductService.QueryCategories:output_type -> productv1.QueryCategoriesResponse - 50, // 336: productv1.ProductService.QueryCategoriesByKind:output_type -> productv1.QueryCategoriesByKindResponse - 52, // 337: productv1.ProductService.QueryCategoriesByKinds:output_type -> productv1.QueryCategoriesByKindsResponse - 44, // 338: productv1.ProductService.QueryComplexFilterType:output_type -> productv1.QueryComplexFilterTypeResponse - 54, // 339: productv1.ProductService.QueryFilterCategories:output_type -> productv1.QueryFilterCategoriesResponse - 36, // 340: productv1.ProductService.QueryNestedType:output_type -> productv1.QueryNestedTypeResponse - 64, // 341: productv1.ProductService.QueryNullableFieldsType:output_type -> productv1.QueryNullableFieldsTypeResponse - 66, // 342: productv1.ProductService.QueryNullableFieldsTypeById:output_type -> productv1.QueryNullableFieldsTypeByIdResponse - 68, // 343: productv1.ProductService.QueryNullableFieldsTypeWithFilter:output_type -> productv1.QueryNullableFieldsTypeWithFilterResponse - 56, // 344: productv1.ProductService.QueryRandomPet:output_type -> productv1.QueryRandomPetResponse - 62, // 345: productv1.ProductService.QueryRandomSearchResult:output_type -> productv1.QueryRandomSearchResultResponse - 38, // 346: productv1.ProductService.QueryRecursiveType:output_type -> productv1.QueryRecursiveTypeResponse - 60, // 347: productv1.ProductService.QuerySearch:output_type -> productv1.QuerySearchResponse - 92, // 348: productv1.ProductService.QueryTestContainer:output_type -> productv1.QueryTestContainerResponse - 94, // 349: productv1.ProductService.QueryTestContainers:output_type -> productv1.QueryTestContainersResponse - 40, // 350: productv1.ProductService.QueryTypeFilterWithArguments:output_type -> productv1.QueryTypeFilterWithArgumentsResponse - 42, // 351: productv1.ProductService.QueryTypeWithMultipleFilterFields:output_type -> productv1.QueryTypeWithMultipleFilterFieldsResponse - 34, // 352: productv1.ProductService.QueryUser:output_type -> productv1.QueryUserResponse - 32, // 353: productv1.ProductService.QueryUsers:output_type -> productv1.QueryUsersResponse - 143, // 354: productv1.ProductService.ResolveCategoryCategoryMetrics:output_type -> productv1.ResolveCategoryCategoryMetricsResponse - 153, // 355: productv1.ProductService.ResolveCategoryCategoryStatus:output_type -> productv1.ResolveCategoryCategoryStatusResponse - 148, // 356: productv1.ProductService.ResolveCategoryMascot:output_type -> productv1.ResolveCategoryMascotResponse - 163, // 357: productv1.ProductService.ResolveCategoryMetricsNormalizedScore:output_type -> productv1.ResolveCategoryMetricsNormalizedScoreResponse - 138, // 358: productv1.ProductService.ResolveCategoryPopularityScore:output_type -> productv1.ResolveCategoryPopularityScoreResponse - 133, // 359: productv1.ProductService.ResolveCategoryProductCount:output_type -> productv1.ResolveCategoryProductCountResponse - 128, // 360: productv1.ProductService.ResolveProductRecommendedCategory:output_type -> productv1.ResolveProductRecommendedCategoryResponse - 123, // 361: productv1.ProductService.ResolveProductShippingEstimate:output_type -> productv1.ResolveProductShippingEstimateResponse - 158, // 362: productv1.ProductService.ResolveSubcategoryItemCount:output_type -> productv1.ResolveSubcategoryItemCountResponse - 168, // 363: productv1.ProductService.ResolveTestContainerDetails:output_type -> productv1.ResolveTestContainerDetailsResponse - 307, // [307:364] is the sub-list for method output_type - 250, // [250:307] is the sub-list for method input_type - 250, // [250:250] is the sub-list for extension type_name - 250, // [250:250] is the sub-list for extension extendee - 0, // [0:250] is the sub-list for field type_name + 130, // 102: productv1.ResolveProductMascotRecommendationRequest.context:type_name -> productv1.ResolveProductMascotRecommendationContext + 129, // 103: productv1.ResolveProductMascotRecommendationRequest.field_args:type_name -> productv1.ResolveProductMascotRecommendationArgs + 198, // 104: productv1.ResolveProductMascotRecommendationResult.mascot_recommendation:type_name -> productv1.Animal + 132, // 105: productv1.ResolveProductMascotRecommendationResponse.result:type_name -> productv1.ResolveProductMascotRecommendationResult + 135, // 106: productv1.ResolveProductStockStatusRequest.context:type_name -> productv1.ResolveProductStockStatusContext + 134, // 107: productv1.ResolveProductStockStatusRequest.field_args:type_name -> productv1.ResolveProductStockStatusArgs + 210, // 108: productv1.ResolveProductStockStatusResult.stock_status:type_name -> productv1.ActionResult + 137, // 109: productv1.ResolveProductStockStatusResponse.result:type_name -> productv1.ResolveProductStockStatusResult + 140, // 110: productv1.ResolveProductProductDetailsRequest.context:type_name -> productv1.ResolveProductProductDetailsContext + 139, // 111: productv1.ResolveProductProductDetailsRequest.field_args:type_name -> productv1.ResolveProductProductDetailsArgs + 214, // 112: productv1.ResolveProductProductDetailsResult.product_details:type_name -> productv1.ProductDetails + 142, // 113: productv1.ResolveProductProductDetailsResponse.result:type_name -> productv1.ResolveProductProductDetailsResult + 229, // 114: productv1.ResolveCategoryProductCountArgs.filters:type_name -> productv1.ProductCountFilter + 145, // 115: productv1.ResolveCategoryProductCountRequest.context:type_name -> productv1.ResolveCategoryProductCountContext + 144, // 116: productv1.ResolveCategoryProductCountRequest.field_args:type_name -> productv1.ResolveCategoryProductCountArgs + 147, // 117: productv1.ResolveCategoryProductCountResponse.result:type_name -> productv1.ResolveCategoryProductCountResult + 252, // 118: productv1.ResolveCategoryPopularityScoreArgs.threshold:type_name -> google.protobuf.Int32Value + 150, // 119: productv1.ResolveCategoryPopularityScoreRequest.context:type_name -> productv1.ResolveCategoryPopularityScoreContext + 149, // 120: productv1.ResolveCategoryPopularityScoreRequest.field_args:type_name -> productv1.ResolveCategoryPopularityScoreArgs + 252, // 121: productv1.ResolveCategoryPopularityScoreResult.popularity_score:type_name -> google.protobuf.Int32Value + 152, // 122: productv1.ResolveCategoryPopularityScoreResponse.result:type_name -> productv1.ResolveCategoryPopularityScoreResult + 155, // 123: productv1.ResolveCategoryCategoryMetricsRequest.context:type_name -> productv1.ResolveCategoryCategoryMetricsContext + 154, // 124: productv1.ResolveCategoryCategoryMetricsRequest.field_args:type_name -> productv1.ResolveCategoryCategoryMetricsArgs + 222, // 125: productv1.ResolveCategoryCategoryMetricsResult.category_metrics:type_name -> productv1.CategoryMetrics + 157, // 126: productv1.ResolveCategoryCategoryMetricsResponse.result:type_name -> productv1.ResolveCategoryCategoryMetricsResult + 0, // 127: productv1.ResolveCategoryMascotContext.kind:type_name -> productv1.CategoryKind + 160, // 128: productv1.ResolveCategoryMascotRequest.context:type_name -> productv1.ResolveCategoryMascotContext + 159, // 129: productv1.ResolveCategoryMascotRequest.field_args:type_name -> productv1.ResolveCategoryMascotArgs + 198, // 130: productv1.ResolveCategoryMascotResult.mascot:type_name -> productv1.Animal + 162, // 131: productv1.ResolveCategoryMascotResponse.result:type_name -> productv1.ResolveCategoryMascotResult + 165, // 132: productv1.ResolveCategoryCategoryStatusRequest.context:type_name -> productv1.ResolveCategoryCategoryStatusContext + 164, // 133: productv1.ResolveCategoryCategoryStatusRequest.field_args:type_name -> productv1.ResolveCategoryCategoryStatusArgs + 210, // 134: productv1.ResolveCategoryCategoryStatusResult.category_status:type_name -> productv1.ActionResult + 167, // 135: productv1.ResolveCategoryCategoryStatusResponse.result:type_name -> productv1.ResolveCategoryCategoryStatusResult + 230, // 136: productv1.ResolveSubcategoryItemCountArgs.filters:type_name -> productv1.SubcategoryItemFilter + 170, // 137: productv1.ResolveSubcategoryItemCountRequest.context:type_name -> productv1.ResolveSubcategoryItemCountContext + 169, // 138: productv1.ResolveSubcategoryItemCountRequest.field_args:type_name -> productv1.ResolveSubcategoryItemCountArgs + 172, // 139: productv1.ResolveSubcategoryItemCountResponse.result:type_name -> productv1.ResolveSubcategoryItemCountResult + 175, // 140: productv1.ResolveCategoryMetricsNormalizedScoreRequest.context:type_name -> productv1.ResolveCategoryMetricsNormalizedScoreContext + 174, // 141: productv1.ResolveCategoryMetricsNormalizedScoreRequest.field_args:type_name -> productv1.ResolveCategoryMetricsNormalizedScoreArgs + 177, // 142: productv1.ResolveCategoryMetricsNormalizedScoreResponse.result:type_name -> productv1.ResolveCategoryMetricsNormalizedScoreResult + 180, // 143: productv1.ResolveTestContainerDetailsRequest.context:type_name -> productv1.ResolveTestContainerDetailsContext + 179, // 144: productv1.ResolveTestContainerDetailsRequest.field_args:type_name -> productv1.ResolveTestContainerDetailsArgs + 227, // 145: productv1.ResolveTestContainerDetailsResult.details:type_name -> productv1.TestDetails + 182, // 146: productv1.ResolveTestContainerDetailsResponse.result:type_name -> productv1.ResolveTestContainerDetailsResult + 215, // 147: productv1.NestedTypeA.b:type_name -> productv1.NestedTypeB + 189, // 148: productv1.RecursiveType.recursive_type:type_name -> productv1.RecursiveType + 217, // 149: productv1.ComplexFilterTypeInput.filter:type_name -> productv1.FilterType + 219, // 150: productv1.OrderInput.lines:type_name -> productv1.OrderLineInput + 16, // 151: productv1.Order.order_lines:type_name -> productv1.ListOfOrderLine + 0, // 152: productv1.Category.kind:type_name -> productv1.CategoryKind + 19, // 153: productv1.Category.subcategories:type_name -> productv1.ListOfSubcategory + 0, // 154: productv1.CategoryFilter.category:type_name -> productv1.CategoryKind + 218, // 155: productv1.CategoryFilter.pagination:type_name -> productv1.Pagination + 223, // 156: productv1.Animal.cat:type_name -> productv1.Cat + 224, // 157: productv1.Animal.dog:type_name -> productv1.Dog + 252, // 158: productv1.SearchInput.limit:type_name -> google.protobuf.Int32Value + 184, // 159: productv1.SearchResult.product:type_name -> productv1.Product + 187, // 160: productv1.SearchResult.user:type_name -> productv1.User + 196, // 161: productv1.SearchResult.category:type_name -> productv1.Category + 253, // 162: productv1.NullableFieldsType.optional_string:type_name -> google.protobuf.StringValue + 252, // 163: productv1.NullableFieldsType.optional_int:type_name -> google.protobuf.Int32Value + 254, // 164: productv1.NullableFieldsType.optional_float:type_name -> google.protobuf.DoubleValue + 255, // 165: productv1.NullableFieldsType.optional_boolean:type_name -> google.protobuf.BoolValue + 253, // 166: productv1.NullableFieldsFilter.name:type_name -> google.protobuf.StringValue + 253, // 167: productv1.NullableFieldsFilter.optional_string:type_name -> google.protobuf.StringValue + 255, // 168: productv1.NullableFieldsFilter.include_nulls:type_name -> google.protobuf.BoolValue + 18, // 169: productv1.BlogPost.optional_tags:type_name -> productv1.ListOfString + 18, // 170: productv1.BlogPost.keywords:type_name -> productv1.ListOfString + 10, // 171: productv1.BlogPost.ratings:type_name -> productv1.ListOfFloat + 7, // 172: productv1.BlogPost.is_published:type_name -> productv1.ListOfBoolean + 13, // 173: productv1.BlogPost.tag_groups:type_name -> productv1.ListOfListOfString + 13, // 174: productv1.BlogPost.related_topics:type_name -> productv1.ListOfListOfString + 13, // 175: productv1.BlogPost.comment_threads:type_name -> productv1.ListOfListOfString + 13, // 176: productv1.BlogPost.suggestions:type_name -> productv1.ListOfListOfString + 196, // 177: productv1.BlogPost.related_categories:type_name -> productv1.Category + 187, // 178: productv1.BlogPost.contributors:type_name -> productv1.User + 17, // 179: productv1.BlogPost.mentioned_products:type_name -> productv1.ListOfProduct + 20, // 180: productv1.BlogPost.mentioned_users:type_name -> productv1.ListOfUser + 11, // 181: productv1.BlogPost.category_groups:type_name -> productv1.ListOfListOfCategory + 14, // 182: productv1.BlogPost.contributor_teams:type_name -> productv1.ListOfListOfUser + 253, // 183: productv1.BlogPostFilter.title:type_name -> google.protobuf.StringValue + 255, // 184: productv1.BlogPostFilter.has_categories:type_name -> google.protobuf.BoolValue + 252, // 185: productv1.BlogPostFilter.min_tags:type_name -> google.protobuf.Int32Value + 253, // 186: productv1.Author.email:type_name -> google.protobuf.StringValue + 18, // 187: productv1.Author.social_links:type_name -> productv1.ListOfString + 13, // 188: productv1.Author.teams_by_project:type_name -> productv1.ListOfListOfString + 13, // 189: productv1.Author.collaborations:type_name -> productv1.ListOfListOfString + 4, // 190: productv1.Author.written_posts:type_name -> productv1.ListOfBlogPost + 196, // 191: productv1.Author.favorite_categories:type_name -> productv1.Category + 20, // 192: productv1.Author.related_authors:type_name -> productv1.ListOfUser + 17, // 193: productv1.Author.product_reviews:type_name -> productv1.ListOfProduct + 14, // 194: productv1.Author.author_groups:type_name -> productv1.ListOfListOfUser + 11, // 195: productv1.Author.category_preferences:type_name -> productv1.ListOfListOfCategory + 14, // 196: productv1.Author.project_teams:type_name -> productv1.ListOfListOfUser + 253, // 197: productv1.AuthorFilter.name:type_name -> google.protobuf.StringValue + 255, // 198: productv1.AuthorFilter.has_teams:type_name -> google.protobuf.BoolValue + 252, // 199: productv1.AuthorFilter.skill_count:type_name -> google.protobuf.Int32Value + 253, // 200: productv1.TestContainer.description:type_name -> google.protobuf.StringValue + 225, // 201: productv1.ActionResult.action_success:type_name -> productv1.ActionSuccess + 226, // 202: productv1.ActionResult.action_error:type_name -> productv1.ActionError + 253, // 203: productv1.NullableFieldsInput.optional_string:type_name -> google.protobuf.StringValue + 252, // 204: productv1.NullableFieldsInput.optional_int:type_name -> google.protobuf.Int32Value + 254, // 205: productv1.NullableFieldsInput.optional_float:type_name -> google.protobuf.DoubleValue + 255, // 206: productv1.NullableFieldsInput.optional_boolean:type_name -> google.protobuf.BoolValue + 18, // 207: productv1.BlogPostInput.optional_tags:type_name -> productv1.ListOfString + 18, // 208: productv1.BlogPostInput.keywords:type_name -> productv1.ListOfString + 10, // 209: productv1.BlogPostInput.ratings:type_name -> productv1.ListOfFloat + 7, // 210: productv1.BlogPostInput.is_published:type_name -> productv1.ListOfBoolean + 13, // 211: productv1.BlogPostInput.tag_groups:type_name -> productv1.ListOfListOfString + 13, // 212: productv1.BlogPostInput.related_topics:type_name -> productv1.ListOfListOfString + 13, // 213: productv1.BlogPostInput.comment_threads:type_name -> productv1.ListOfListOfString + 13, // 214: productv1.BlogPostInput.suggestions:type_name -> productv1.ListOfListOfString + 9, // 215: productv1.BlogPostInput.related_categories:type_name -> productv1.ListOfCategoryInput + 21, // 216: productv1.BlogPostInput.contributors:type_name -> productv1.ListOfUserInput + 12, // 217: productv1.BlogPostInput.category_groups:type_name -> productv1.ListOfListOfCategoryInput + 253, // 218: productv1.AuthorInput.email:type_name -> google.protobuf.StringValue + 18, // 219: productv1.AuthorInput.social_links:type_name -> productv1.ListOfString + 13, // 220: productv1.AuthorInput.teams_by_project:type_name -> productv1.ListOfListOfString + 13, // 221: productv1.AuthorInput.collaborations:type_name -> productv1.ListOfListOfString + 228, // 222: productv1.AuthorInput.favorite_categories:type_name -> productv1.CategoryInput + 15, // 223: productv1.AuthorInput.author_groups:type_name -> productv1.ListOfListOfUserInput + 15, // 224: productv1.AuthorInput.project_teams:type_name -> productv1.ListOfListOfUserInput + 210, // 225: productv1.ProductDetails.review_summary:type_name -> productv1.ActionResult + 198, // 226: productv1.ProductDetails.recommended_pet:type_name -> productv1.Animal + 216, // 227: productv1.NestedTypeB.c:type_name -> productv1.NestedTypeC + 218, // 228: productv1.FilterType.pagination:type_name -> productv1.Pagination + 18, // 229: productv1.OrderLineInput.modifiers:type_name -> productv1.ListOfString + 18, // 230: productv1.OrderLine.modifiers:type_name -> productv1.ListOfString + 253, // 231: productv1.Subcategory.description:type_name -> google.protobuf.StringValue + 196, // 232: productv1.CategoryMetrics.related_category:type_name -> productv1.Category + 198, // 233: productv1.TestDetails.pet:type_name -> productv1.Animal + 210, // 234: productv1.TestDetails.status:type_name -> productv1.ActionResult + 0, // 235: productv1.CategoryInput.kind:type_name -> productv1.CategoryKind + 254, // 236: productv1.ProductCountFilter.min_price:type_name -> google.protobuf.DoubleValue + 254, // 237: productv1.ProductCountFilter.max_price:type_name -> google.protobuf.DoubleValue + 255, // 238: productv1.ProductCountFilter.in_stock:type_name -> google.protobuf.BoolValue + 253, // 239: productv1.ProductCountFilter.search_term:type_name -> google.protobuf.StringValue + 254, // 240: productv1.SubcategoryItemFilter.min_price:type_name -> google.protobuf.DoubleValue + 254, // 241: productv1.SubcategoryItemFilter.max_price:type_name -> google.protobuf.DoubleValue + 255, // 242: productv1.SubcategoryItemFilter.in_stock:type_name -> google.protobuf.BoolValue + 255, // 243: productv1.SubcategoryItemFilter.is_active:type_name -> google.protobuf.BoolValue + 253, // 244: productv1.SubcategoryItemFilter.search_term:type_name -> google.protobuf.StringValue + 1, // 245: productv1.ShippingEstimateInput.destination:type_name -> productv1.ShippingDestination + 255, // 246: productv1.ShippingEstimateInput.expedited:type_name -> google.protobuf.BoolValue + 206, // 247: productv1.ListOfAuthorFilter.List.items:type_name -> productv1.AuthorFilter + 213, // 248: productv1.ListOfAuthorInput.List.items:type_name -> productv1.AuthorInput + 203, // 249: productv1.ListOfBlogPost.List.items:type_name -> productv1.BlogPost + 204, // 250: productv1.ListOfBlogPostFilter.List.items:type_name -> productv1.BlogPostFilter + 212, // 251: productv1.ListOfBlogPostInput.List.items:type_name -> productv1.BlogPostInput + 196, // 252: productv1.ListOfCategory.List.items:type_name -> productv1.Category + 228, // 253: productv1.ListOfCategoryInput.List.items:type_name -> productv1.CategoryInput + 8, // 254: productv1.ListOfListOfCategory.List.items:type_name -> productv1.ListOfCategory + 9, // 255: productv1.ListOfListOfCategoryInput.List.items:type_name -> productv1.ListOfCategoryInput + 18, // 256: productv1.ListOfListOfString.List.items:type_name -> productv1.ListOfString + 20, // 257: productv1.ListOfListOfUser.List.items:type_name -> productv1.ListOfUser + 21, // 258: productv1.ListOfListOfUserInput.List.items:type_name -> productv1.ListOfUserInput + 220, // 259: productv1.ListOfOrderLine.List.items:type_name -> productv1.OrderLine + 184, // 260: productv1.ListOfProduct.List.items:type_name -> productv1.Product + 221, // 261: productv1.ListOfSubcategory.List.items:type_name -> productv1.Subcategory + 187, // 262: productv1.ListOfUser.List.items:type_name -> productv1.User + 208, // 263: productv1.ListOfUserInput.List.items:type_name -> productv1.UserInput + 23, // 264: productv1.ProductService.LookupProductById:input_type -> productv1.LookupProductByIdRequest + 26, // 265: productv1.ProductService.LookupStorageById:input_type -> productv1.LookupStorageByIdRequest + 29, // 266: productv1.ProductService.LookupWarehouseById:input_type -> productv1.LookupWarehouseByIdRequest + 111, // 267: productv1.ProductService.MutationBulkCreateAuthors:input_type -> productv1.MutationBulkCreateAuthorsRequest + 115, // 268: productv1.ProductService.MutationBulkCreateBlogPosts:input_type -> productv1.MutationBulkCreateBlogPostsRequest + 113, // 269: productv1.ProductService.MutationBulkUpdateAuthors:input_type -> productv1.MutationBulkUpdateAuthorsRequest + 117, // 270: productv1.ProductService.MutationBulkUpdateBlogPosts:input_type -> productv1.MutationBulkUpdateBlogPostsRequest + 107, // 271: productv1.ProductService.MutationCreateAuthor:input_type -> productv1.MutationCreateAuthorRequest + 103, // 272: productv1.ProductService.MutationCreateBlogPost:input_type -> productv1.MutationCreateBlogPostRequest + 99, // 273: productv1.ProductService.MutationCreateNullableFieldsType:input_type -> productv1.MutationCreateNullableFieldsTypeRequest + 95, // 274: productv1.ProductService.MutationCreateUser:input_type -> productv1.MutationCreateUserRequest + 97, // 275: productv1.ProductService.MutationPerformAction:input_type -> productv1.MutationPerformActionRequest + 109, // 276: productv1.ProductService.MutationUpdateAuthor:input_type -> productv1.MutationUpdateAuthorRequest + 105, // 277: productv1.ProductService.MutationUpdateBlogPost:input_type -> productv1.MutationUpdateBlogPostRequest + 101, // 278: productv1.ProductService.MutationUpdateNullableFieldsType:input_type -> productv1.MutationUpdateNullableFieldsTypeRequest + 85, // 279: productv1.ProductService.QueryAllAuthors:input_type -> productv1.QueryAllAuthorsRequest + 77, // 280: productv1.ProductService.QueryAllBlogPosts:input_type -> productv1.QueryAllBlogPostsRequest + 69, // 281: productv1.ProductService.QueryAllNullableFieldsTypes:input_type -> productv1.QueryAllNullableFieldsTypesRequest + 57, // 282: productv1.ProductService.QueryAllPets:input_type -> productv1.QueryAllPetsRequest + 79, // 283: productv1.ProductService.QueryAuthor:input_type -> productv1.QueryAuthorRequest + 81, // 284: productv1.ProductService.QueryAuthorById:input_type -> productv1.QueryAuthorByIdRequest + 83, // 285: productv1.ProductService.QueryAuthorsWithFilter:input_type -> productv1.QueryAuthorsWithFilterRequest + 71, // 286: productv1.ProductService.QueryBlogPost:input_type -> productv1.QueryBlogPostRequest + 73, // 287: productv1.ProductService.QueryBlogPostById:input_type -> productv1.QueryBlogPostByIdRequest + 75, // 288: productv1.ProductService.QueryBlogPostsWithFilter:input_type -> productv1.QueryBlogPostsWithFilterRequest + 87, // 289: productv1.ProductService.QueryBulkSearchAuthors:input_type -> productv1.QueryBulkSearchAuthorsRequest + 89, // 290: productv1.ProductService.QueryBulkSearchBlogPosts:input_type -> productv1.QueryBulkSearchBlogPostsRequest + 45, // 291: productv1.ProductService.QueryCalculateTotals:input_type -> productv1.QueryCalculateTotalsRequest + 47, // 292: productv1.ProductService.QueryCategories:input_type -> productv1.QueryCategoriesRequest + 49, // 293: productv1.ProductService.QueryCategoriesByKind:input_type -> productv1.QueryCategoriesByKindRequest + 51, // 294: productv1.ProductService.QueryCategoriesByKinds:input_type -> productv1.QueryCategoriesByKindsRequest + 43, // 295: productv1.ProductService.QueryComplexFilterType:input_type -> productv1.QueryComplexFilterTypeRequest + 53, // 296: productv1.ProductService.QueryFilterCategories:input_type -> productv1.QueryFilterCategoriesRequest + 35, // 297: productv1.ProductService.QueryNestedType:input_type -> productv1.QueryNestedTypeRequest + 63, // 298: productv1.ProductService.QueryNullableFieldsType:input_type -> productv1.QueryNullableFieldsTypeRequest + 65, // 299: productv1.ProductService.QueryNullableFieldsTypeById:input_type -> productv1.QueryNullableFieldsTypeByIdRequest + 67, // 300: productv1.ProductService.QueryNullableFieldsTypeWithFilter:input_type -> productv1.QueryNullableFieldsTypeWithFilterRequest + 55, // 301: productv1.ProductService.QueryRandomPet:input_type -> productv1.QueryRandomPetRequest + 61, // 302: productv1.ProductService.QueryRandomSearchResult:input_type -> productv1.QueryRandomSearchResultRequest + 37, // 303: productv1.ProductService.QueryRecursiveType:input_type -> productv1.QueryRecursiveTypeRequest + 59, // 304: productv1.ProductService.QuerySearch:input_type -> productv1.QuerySearchRequest + 91, // 305: productv1.ProductService.QueryTestContainer:input_type -> productv1.QueryTestContainerRequest + 93, // 306: productv1.ProductService.QueryTestContainers:input_type -> productv1.QueryTestContainersRequest + 39, // 307: productv1.ProductService.QueryTypeFilterWithArguments:input_type -> productv1.QueryTypeFilterWithArgumentsRequest + 41, // 308: productv1.ProductService.QueryTypeWithMultipleFilterFields:input_type -> productv1.QueryTypeWithMultipleFilterFieldsRequest + 33, // 309: productv1.ProductService.QueryUser:input_type -> productv1.QueryUserRequest + 31, // 310: productv1.ProductService.QueryUsers:input_type -> productv1.QueryUsersRequest + 156, // 311: productv1.ProductService.ResolveCategoryCategoryMetrics:input_type -> productv1.ResolveCategoryCategoryMetricsRequest + 166, // 312: productv1.ProductService.ResolveCategoryCategoryStatus:input_type -> productv1.ResolveCategoryCategoryStatusRequest + 161, // 313: productv1.ProductService.ResolveCategoryMascot:input_type -> productv1.ResolveCategoryMascotRequest + 176, // 314: productv1.ProductService.ResolveCategoryMetricsNormalizedScore:input_type -> productv1.ResolveCategoryMetricsNormalizedScoreRequest + 151, // 315: productv1.ProductService.ResolveCategoryPopularityScore:input_type -> productv1.ResolveCategoryPopularityScoreRequest + 146, // 316: productv1.ProductService.ResolveCategoryProductCount:input_type -> productv1.ResolveCategoryProductCountRequest + 131, // 317: productv1.ProductService.ResolveProductMascotRecommendation:input_type -> productv1.ResolveProductMascotRecommendationRequest + 141, // 318: productv1.ProductService.ResolveProductProductDetails:input_type -> productv1.ResolveProductProductDetailsRequest + 126, // 319: productv1.ProductService.ResolveProductRecommendedCategory:input_type -> productv1.ResolveProductRecommendedCategoryRequest + 121, // 320: productv1.ProductService.ResolveProductShippingEstimate:input_type -> productv1.ResolveProductShippingEstimateRequest + 136, // 321: productv1.ProductService.ResolveProductStockStatus:input_type -> productv1.ResolveProductStockStatusRequest + 171, // 322: productv1.ProductService.ResolveSubcategoryItemCount:input_type -> productv1.ResolveSubcategoryItemCountRequest + 181, // 323: productv1.ProductService.ResolveTestContainerDetails:input_type -> productv1.ResolveTestContainerDetailsRequest + 24, // 324: productv1.ProductService.LookupProductById:output_type -> productv1.LookupProductByIdResponse + 27, // 325: productv1.ProductService.LookupStorageById:output_type -> productv1.LookupStorageByIdResponse + 30, // 326: productv1.ProductService.LookupWarehouseById:output_type -> productv1.LookupWarehouseByIdResponse + 112, // 327: productv1.ProductService.MutationBulkCreateAuthors:output_type -> productv1.MutationBulkCreateAuthorsResponse + 116, // 328: productv1.ProductService.MutationBulkCreateBlogPosts:output_type -> productv1.MutationBulkCreateBlogPostsResponse + 114, // 329: productv1.ProductService.MutationBulkUpdateAuthors:output_type -> productv1.MutationBulkUpdateAuthorsResponse + 118, // 330: productv1.ProductService.MutationBulkUpdateBlogPosts:output_type -> productv1.MutationBulkUpdateBlogPostsResponse + 108, // 331: productv1.ProductService.MutationCreateAuthor:output_type -> productv1.MutationCreateAuthorResponse + 104, // 332: productv1.ProductService.MutationCreateBlogPost:output_type -> productv1.MutationCreateBlogPostResponse + 100, // 333: productv1.ProductService.MutationCreateNullableFieldsType:output_type -> productv1.MutationCreateNullableFieldsTypeResponse + 96, // 334: productv1.ProductService.MutationCreateUser:output_type -> productv1.MutationCreateUserResponse + 98, // 335: productv1.ProductService.MutationPerformAction:output_type -> productv1.MutationPerformActionResponse + 110, // 336: productv1.ProductService.MutationUpdateAuthor:output_type -> productv1.MutationUpdateAuthorResponse + 106, // 337: productv1.ProductService.MutationUpdateBlogPost:output_type -> productv1.MutationUpdateBlogPostResponse + 102, // 338: productv1.ProductService.MutationUpdateNullableFieldsType:output_type -> productv1.MutationUpdateNullableFieldsTypeResponse + 86, // 339: productv1.ProductService.QueryAllAuthors:output_type -> productv1.QueryAllAuthorsResponse + 78, // 340: productv1.ProductService.QueryAllBlogPosts:output_type -> productv1.QueryAllBlogPostsResponse + 70, // 341: productv1.ProductService.QueryAllNullableFieldsTypes:output_type -> productv1.QueryAllNullableFieldsTypesResponse + 58, // 342: productv1.ProductService.QueryAllPets:output_type -> productv1.QueryAllPetsResponse + 80, // 343: productv1.ProductService.QueryAuthor:output_type -> productv1.QueryAuthorResponse + 82, // 344: productv1.ProductService.QueryAuthorById:output_type -> productv1.QueryAuthorByIdResponse + 84, // 345: productv1.ProductService.QueryAuthorsWithFilter:output_type -> productv1.QueryAuthorsWithFilterResponse + 72, // 346: productv1.ProductService.QueryBlogPost:output_type -> productv1.QueryBlogPostResponse + 74, // 347: productv1.ProductService.QueryBlogPostById:output_type -> productv1.QueryBlogPostByIdResponse + 76, // 348: productv1.ProductService.QueryBlogPostsWithFilter:output_type -> productv1.QueryBlogPostsWithFilterResponse + 88, // 349: productv1.ProductService.QueryBulkSearchAuthors:output_type -> productv1.QueryBulkSearchAuthorsResponse + 90, // 350: productv1.ProductService.QueryBulkSearchBlogPosts:output_type -> productv1.QueryBulkSearchBlogPostsResponse + 46, // 351: productv1.ProductService.QueryCalculateTotals:output_type -> productv1.QueryCalculateTotalsResponse + 48, // 352: productv1.ProductService.QueryCategories:output_type -> productv1.QueryCategoriesResponse + 50, // 353: productv1.ProductService.QueryCategoriesByKind:output_type -> productv1.QueryCategoriesByKindResponse + 52, // 354: productv1.ProductService.QueryCategoriesByKinds:output_type -> productv1.QueryCategoriesByKindsResponse + 44, // 355: productv1.ProductService.QueryComplexFilterType:output_type -> productv1.QueryComplexFilterTypeResponse + 54, // 356: productv1.ProductService.QueryFilterCategories:output_type -> productv1.QueryFilterCategoriesResponse + 36, // 357: productv1.ProductService.QueryNestedType:output_type -> productv1.QueryNestedTypeResponse + 64, // 358: productv1.ProductService.QueryNullableFieldsType:output_type -> productv1.QueryNullableFieldsTypeResponse + 66, // 359: productv1.ProductService.QueryNullableFieldsTypeById:output_type -> productv1.QueryNullableFieldsTypeByIdResponse + 68, // 360: productv1.ProductService.QueryNullableFieldsTypeWithFilter:output_type -> productv1.QueryNullableFieldsTypeWithFilterResponse + 56, // 361: productv1.ProductService.QueryRandomPet:output_type -> productv1.QueryRandomPetResponse + 62, // 362: productv1.ProductService.QueryRandomSearchResult:output_type -> productv1.QueryRandomSearchResultResponse + 38, // 363: productv1.ProductService.QueryRecursiveType:output_type -> productv1.QueryRecursiveTypeResponse + 60, // 364: productv1.ProductService.QuerySearch:output_type -> productv1.QuerySearchResponse + 92, // 365: productv1.ProductService.QueryTestContainer:output_type -> productv1.QueryTestContainerResponse + 94, // 366: productv1.ProductService.QueryTestContainers:output_type -> productv1.QueryTestContainersResponse + 40, // 367: productv1.ProductService.QueryTypeFilterWithArguments:output_type -> productv1.QueryTypeFilterWithArgumentsResponse + 42, // 368: productv1.ProductService.QueryTypeWithMultipleFilterFields:output_type -> productv1.QueryTypeWithMultipleFilterFieldsResponse + 34, // 369: productv1.ProductService.QueryUser:output_type -> productv1.QueryUserResponse + 32, // 370: productv1.ProductService.QueryUsers:output_type -> productv1.QueryUsersResponse + 158, // 371: productv1.ProductService.ResolveCategoryCategoryMetrics:output_type -> productv1.ResolveCategoryCategoryMetricsResponse + 168, // 372: productv1.ProductService.ResolveCategoryCategoryStatus:output_type -> productv1.ResolveCategoryCategoryStatusResponse + 163, // 373: productv1.ProductService.ResolveCategoryMascot:output_type -> productv1.ResolveCategoryMascotResponse + 178, // 374: productv1.ProductService.ResolveCategoryMetricsNormalizedScore:output_type -> productv1.ResolveCategoryMetricsNormalizedScoreResponse + 153, // 375: productv1.ProductService.ResolveCategoryPopularityScore:output_type -> productv1.ResolveCategoryPopularityScoreResponse + 148, // 376: productv1.ProductService.ResolveCategoryProductCount:output_type -> productv1.ResolveCategoryProductCountResponse + 133, // 377: productv1.ProductService.ResolveProductMascotRecommendation:output_type -> productv1.ResolveProductMascotRecommendationResponse + 143, // 378: productv1.ProductService.ResolveProductProductDetails:output_type -> productv1.ResolveProductProductDetailsResponse + 128, // 379: productv1.ProductService.ResolveProductRecommendedCategory:output_type -> productv1.ResolveProductRecommendedCategoryResponse + 123, // 380: productv1.ProductService.ResolveProductShippingEstimate:output_type -> productv1.ResolveProductShippingEstimateResponse + 138, // 381: productv1.ProductService.ResolveProductStockStatus:output_type -> productv1.ResolveProductStockStatusResponse + 173, // 382: productv1.ProductService.ResolveSubcategoryItemCount:output_type -> productv1.ResolveSubcategoryItemCountResponse + 183, // 383: productv1.ProductService.ResolveTestContainerDetails:output_type -> productv1.ResolveTestContainerDetailsResponse + 324, // [324:384] is the sub-list for method output_type + 264, // [264:324] is the sub-list for method input_type + 264, // [264:264] is the sub-list for extension type_name + 264, // [264:264] is the sub-list for extension extendee + 0, // [0:264] is the sub-list for field type_name } func init() { file_product_proto_init() } @@ -13374,16 +14257,16 @@ func file_product_proto_init() { if File_product_proto != nil { return } - file_product_proto_msgTypes[181].OneofWrappers = []any{ + file_product_proto_msgTypes[196].OneofWrappers = []any{ (*Animal_Cat)(nil), (*Animal_Dog)(nil), } - file_product_proto_msgTypes[183].OneofWrappers = []any{ + file_product_proto_msgTypes[198].OneofWrappers = []any{ (*SearchResult_Product)(nil), (*SearchResult_User)(nil), (*SearchResult_Category)(nil), } - file_product_proto_msgTypes[193].OneofWrappers = []any{ + file_product_proto_msgTypes[208].OneofWrappers = []any{ (*ActionResult_ActionSuccess)(nil), (*ActionResult_ActionError)(nil), } @@ -13393,7 +14276,7 @@ func file_product_proto_init() { GoPackagePath: reflect.TypeOf(x{}).PkgPath(), RawDescriptor: unsafe.Slice(unsafe.StringData(file_product_proto_rawDesc), len(file_product_proto_rawDesc)), NumEnums: 2, - NumMessages: 234, + NumMessages: 250, NumExtensions: 0, NumServices: 1, }, diff --git a/v2/pkg/grpctest/productv1/product_grpc.pb.go b/v2/pkg/grpctest/productv1/product_grpc.pb.go index d6aeee39e..3439376c1 100644 --- a/v2/pkg/grpctest/productv1/product_grpc.pb.go +++ b/v2/pkg/grpctest/productv1/product_grpc.pb.go @@ -72,8 +72,11 @@ const ( ProductService_ResolveCategoryMetricsNormalizedScore_FullMethodName = "/productv1.ProductService/ResolveCategoryMetricsNormalizedScore" ProductService_ResolveCategoryPopularityScore_FullMethodName = "/productv1.ProductService/ResolveCategoryPopularityScore" ProductService_ResolveCategoryProductCount_FullMethodName = "/productv1.ProductService/ResolveCategoryProductCount" + ProductService_ResolveProductMascotRecommendation_FullMethodName = "/productv1.ProductService/ResolveProductMascotRecommendation" + ProductService_ResolveProductProductDetails_FullMethodName = "/productv1.ProductService/ResolveProductProductDetails" ProductService_ResolveProductRecommendedCategory_FullMethodName = "/productv1.ProductService/ResolveProductRecommendedCategory" ProductService_ResolveProductShippingEstimate_FullMethodName = "/productv1.ProductService/ResolveProductShippingEstimate" + ProductService_ResolveProductStockStatus_FullMethodName = "/productv1.ProductService/ResolveProductStockStatus" ProductService_ResolveSubcategoryItemCount_FullMethodName = "/productv1.ProductService/ResolveSubcategoryItemCount" ProductService_ResolveTestContainerDetails_FullMethodName = "/productv1.ProductService/ResolveTestContainerDetails" ) @@ -140,8 +143,11 @@ type ProductServiceClient interface { ResolveCategoryMetricsNormalizedScore(ctx context.Context, in *ResolveCategoryMetricsNormalizedScoreRequest, opts ...grpc.CallOption) (*ResolveCategoryMetricsNormalizedScoreResponse, error) ResolveCategoryPopularityScore(ctx context.Context, in *ResolveCategoryPopularityScoreRequest, opts ...grpc.CallOption) (*ResolveCategoryPopularityScoreResponse, error) ResolveCategoryProductCount(ctx context.Context, in *ResolveCategoryProductCountRequest, opts ...grpc.CallOption) (*ResolveCategoryProductCountResponse, error) + ResolveProductMascotRecommendation(ctx context.Context, in *ResolveProductMascotRecommendationRequest, opts ...grpc.CallOption) (*ResolveProductMascotRecommendationResponse, error) + ResolveProductProductDetails(ctx context.Context, in *ResolveProductProductDetailsRequest, opts ...grpc.CallOption) (*ResolveProductProductDetailsResponse, error) ResolveProductRecommendedCategory(ctx context.Context, in *ResolveProductRecommendedCategoryRequest, opts ...grpc.CallOption) (*ResolveProductRecommendedCategoryResponse, error) ResolveProductShippingEstimate(ctx context.Context, in *ResolveProductShippingEstimateRequest, opts ...grpc.CallOption) (*ResolveProductShippingEstimateResponse, error) + ResolveProductStockStatus(ctx context.Context, in *ResolveProductStockStatusRequest, opts ...grpc.CallOption) (*ResolveProductStockStatusResponse, error) ResolveSubcategoryItemCount(ctx context.Context, in *ResolveSubcategoryItemCountRequest, opts ...grpc.CallOption) (*ResolveSubcategoryItemCountResponse, error) ResolveTestContainerDetails(ctx context.Context, in *ResolveTestContainerDetailsRequest, opts ...grpc.CallOption) (*ResolveTestContainerDetailsResponse, error) } @@ -684,6 +690,26 @@ func (c *productServiceClient) ResolveCategoryProductCount(ctx context.Context, return out, nil } +func (c *productServiceClient) ResolveProductMascotRecommendation(ctx context.Context, in *ResolveProductMascotRecommendationRequest, opts ...grpc.CallOption) (*ResolveProductMascotRecommendationResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(ResolveProductMascotRecommendationResponse) + err := c.cc.Invoke(ctx, ProductService_ResolveProductMascotRecommendation_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *productServiceClient) ResolveProductProductDetails(ctx context.Context, in *ResolveProductProductDetailsRequest, opts ...grpc.CallOption) (*ResolveProductProductDetailsResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(ResolveProductProductDetailsResponse) + err := c.cc.Invoke(ctx, ProductService_ResolveProductProductDetails_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + func (c *productServiceClient) ResolveProductRecommendedCategory(ctx context.Context, in *ResolveProductRecommendedCategoryRequest, opts ...grpc.CallOption) (*ResolveProductRecommendedCategoryResponse, error) { cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) out := new(ResolveProductRecommendedCategoryResponse) @@ -704,6 +730,16 @@ func (c *productServiceClient) ResolveProductShippingEstimate(ctx context.Contex return out, nil } +func (c *productServiceClient) ResolveProductStockStatus(ctx context.Context, in *ResolveProductStockStatusRequest, opts ...grpc.CallOption) (*ResolveProductStockStatusResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(ResolveProductStockStatusResponse) + err := c.cc.Invoke(ctx, ProductService_ResolveProductStockStatus_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + func (c *productServiceClient) ResolveSubcategoryItemCount(ctx context.Context, in *ResolveSubcategoryItemCountRequest, opts ...grpc.CallOption) (*ResolveSubcategoryItemCountResponse, error) { cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) out := new(ResolveSubcategoryItemCountResponse) @@ -786,8 +822,11 @@ type ProductServiceServer interface { ResolveCategoryMetricsNormalizedScore(context.Context, *ResolveCategoryMetricsNormalizedScoreRequest) (*ResolveCategoryMetricsNormalizedScoreResponse, error) ResolveCategoryPopularityScore(context.Context, *ResolveCategoryPopularityScoreRequest) (*ResolveCategoryPopularityScoreResponse, error) ResolveCategoryProductCount(context.Context, *ResolveCategoryProductCountRequest) (*ResolveCategoryProductCountResponse, error) + ResolveProductMascotRecommendation(context.Context, *ResolveProductMascotRecommendationRequest) (*ResolveProductMascotRecommendationResponse, error) + ResolveProductProductDetails(context.Context, *ResolveProductProductDetailsRequest) (*ResolveProductProductDetailsResponse, error) ResolveProductRecommendedCategory(context.Context, *ResolveProductRecommendedCategoryRequest) (*ResolveProductRecommendedCategoryResponse, error) ResolveProductShippingEstimate(context.Context, *ResolveProductShippingEstimateRequest) (*ResolveProductShippingEstimateResponse, error) + ResolveProductStockStatus(context.Context, *ResolveProductStockStatusRequest) (*ResolveProductStockStatusResponse, error) ResolveSubcategoryItemCount(context.Context, *ResolveSubcategoryItemCountRequest) (*ResolveSubcategoryItemCountResponse, error) ResolveTestContainerDetails(context.Context, *ResolveTestContainerDetailsRequest) (*ResolveTestContainerDetailsResponse, error) mustEmbedUnimplementedProductServiceServer() @@ -959,12 +998,21 @@ func (UnimplementedProductServiceServer) ResolveCategoryPopularityScore(context. func (UnimplementedProductServiceServer) ResolveCategoryProductCount(context.Context, *ResolveCategoryProductCountRequest) (*ResolveCategoryProductCountResponse, error) { return nil, status.Errorf(codes.Unimplemented, "method ResolveCategoryProductCount not implemented") } +func (UnimplementedProductServiceServer) ResolveProductMascotRecommendation(context.Context, *ResolveProductMascotRecommendationRequest) (*ResolveProductMascotRecommendationResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method ResolveProductMascotRecommendation not implemented") +} +func (UnimplementedProductServiceServer) ResolveProductProductDetails(context.Context, *ResolveProductProductDetailsRequest) (*ResolveProductProductDetailsResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method ResolveProductProductDetails not implemented") +} func (UnimplementedProductServiceServer) ResolveProductRecommendedCategory(context.Context, *ResolveProductRecommendedCategoryRequest) (*ResolveProductRecommendedCategoryResponse, error) { return nil, status.Errorf(codes.Unimplemented, "method ResolveProductRecommendedCategory not implemented") } func (UnimplementedProductServiceServer) ResolveProductShippingEstimate(context.Context, *ResolveProductShippingEstimateRequest) (*ResolveProductShippingEstimateResponse, error) { return nil, status.Errorf(codes.Unimplemented, "method ResolveProductShippingEstimate not implemented") } +func (UnimplementedProductServiceServer) ResolveProductStockStatus(context.Context, *ResolveProductStockStatusRequest) (*ResolveProductStockStatusResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method ResolveProductStockStatus not implemented") +} func (UnimplementedProductServiceServer) ResolveSubcategoryItemCount(context.Context, *ResolveSubcategoryItemCountRequest) (*ResolveSubcategoryItemCountResponse, error) { return nil, status.Errorf(codes.Unimplemented, "method ResolveSubcategoryItemCount not implemented") } @@ -1946,6 +1994,42 @@ func _ProductService_ResolveCategoryProductCount_Handler(srv interface{}, ctx co return interceptor(ctx, in, info, handler) } +func _ProductService_ResolveProductMascotRecommendation_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(ResolveProductMascotRecommendationRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(ProductServiceServer).ResolveProductMascotRecommendation(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: ProductService_ResolveProductMascotRecommendation_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(ProductServiceServer).ResolveProductMascotRecommendation(ctx, req.(*ResolveProductMascotRecommendationRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _ProductService_ResolveProductProductDetails_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(ResolveProductProductDetailsRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(ProductServiceServer).ResolveProductProductDetails(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: ProductService_ResolveProductProductDetails_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(ProductServiceServer).ResolveProductProductDetails(ctx, req.(*ResolveProductProductDetailsRequest)) + } + return interceptor(ctx, in, info, handler) +} + func _ProductService_ResolveProductRecommendedCategory_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { in := new(ResolveProductRecommendedCategoryRequest) if err := dec(in); err != nil { @@ -1982,6 +2066,24 @@ func _ProductService_ResolveProductShippingEstimate_Handler(srv interface{}, ctx return interceptor(ctx, in, info, handler) } +func _ProductService_ResolveProductStockStatus_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(ResolveProductStockStatusRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(ProductServiceServer).ResolveProductStockStatus(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: ProductService_ResolveProductStockStatus_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(ProductServiceServer).ResolveProductStockStatus(ctx, req.(*ResolveProductStockStatusRequest)) + } + return interceptor(ctx, in, info, handler) +} + func _ProductService_ResolveSubcategoryItemCount_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { in := new(ResolveSubcategoryItemCountRequest) if err := dec(in); err != nil { @@ -2237,6 +2339,14 @@ var ProductService_ServiceDesc = grpc.ServiceDesc{ MethodName: "ResolveCategoryProductCount", Handler: _ProductService_ResolveCategoryProductCount_Handler, }, + { + MethodName: "ResolveProductMascotRecommendation", + Handler: _ProductService_ResolveProductMascotRecommendation_Handler, + }, + { + MethodName: "ResolveProductProductDetails", + Handler: _ProductService_ResolveProductProductDetails_Handler, + }, { MethodName: "ResolveProductRecommendedCategory", Handler: _ProductService_ResolveProductRecommendedCategory_Handler, @@ -2245,6 +2355,10 @@ var ProductService_ServiceDesc = grpc.ServiceDesc{ MethodName: "ResolveProductShippingEstimate", Handler: _ProductService_ResolveProductShippingEstimate_Handler, }, + { + MethodName: "ResolveProductStockStatus", + Handler: _ProductService_ResolveProductStockStatus_Handler, + }, { MethodName: "ResolveSubcategoryItemCount", Handler: _ProductService_ResolveSubcategoryItemCount_Handler, diff --git a/v2/pkg/grpctest/testdata/products.graphqls b/v2/pkg/grpctest/testdata/products.graphqls index 8e1008223..f054de862 100644 --- a/v2/pkg/grpctest/testdata/products.graphqls +++ b/v2/pkg/grpctest/testdata/products.graphqls @@ -5,6 +5,16 @@ type Product @key(fields: "id") { price: Float! shippingEstimate(input: ShippingEstimateInput!): Float! @connect__fieldResolver(context: "id price") recommendedCategory(maxPrice: Int!): Category @connect__fieldResolver(context: "id name price") + mascotRecommendation(includeDetails: Boolean!): Animal @connect__fieldResolver(context: "id name") + stockStatus(checkAvailability: Boolean!): ActionResult! @connect__fieldResolver(context: "id name price") + productDetails(includeExtended: Boolean!): ProductDetails @connect__fieldResolver(context: "id name price") +} + +type ProductDetails { + id: ID! + description: String! + reviewSummary: ActionResult! + recommendedPet: Animal! } type Storage @key(fields: "id") { From 10490e60d6a7071c8f71a60eb6b65135bf723341 Mon Sep 17 00:00:00 2001 From: Ludwig Bedacht Date: Fri, 31 Oct 2025 16:25:29 +0100 Subject: [PATCH 47/72] chore: fixes incorrect parent mapping and adds tests for the datasource --- .../grpc_datasource/execution_plan.go | 21 +- .../execution_plan_federation_test.go | 4 +- .../grpc_datasource/grpc_datasource_test.go | 301 ++++++++++++++++++ 3 files changed, 316 insertions(+), 10 deletions(-) diff --git a/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go b/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go index 10c2acd08..93a2a78e1 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go +++ b/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go @@ -1045,11 +1045,6 @@ func (r *rpcPlanningContext) resolveRequiredFields(typeName string, resolvedFiel Name: typeName, } - parentTypeNode, found := r.definition.NodeByNameStr(typeName) - if !found { - return nil, fmt.Errorf("parent type node not found for type %s", typeName) - } - // TODO: handle composite types. if len(resolvedField.fragmentSelections) > 0 { message.FieldSelectionSet = make(RPCFieldSelectionSet, len(resolvedField.fragmentSelections)) @@ -1057,7 +1052,12 @@ func (r *rpcPlanningContext) resolveRequiredFields(typeName string, resolvedFiel message.MemberTypes = resolvedField.memberTypes for _, fragmentSelection := range resolvedField.fragmentSelections { - fields, err := r.buildCompositeField(parentTypeNode, fragmentSelection) + inlineFragmentTypeNode, found := r.definition.NodeByNameStr(fragmentSelection.typeName) + if !found { + return nil, fmt.Errorf("unable to build composite field: underlying fragment type node not found for type %s", fragmentSelection.typeName) + } + + fields, err := r.buildCompositeField(inlineFragmentTypeNode, fragmentSelection) if err != nil { return nil, err } @@ -1072,6 +1072,11 @@ func (r *rpcPlanningContext) resolveRequiredFields(typeName string, resolvedFiel return nil, errors.New("unable to resolve required fields: no fields selection set found") } + parentTypeNode, found := r.definition.NodeByNameStr(typeName) + if !found { + return nil, fmt.Errorf("parent type node not found for type %s", typeName) + } + fieldRefs := r.operation.SelectionSetFieldRefs(resolvedField.fieldsSelectionSetRef) message.Fields = make(RPCFields, 0, len(fieldRefs)) @@ -1126,7 +1131,7 @@ func (r *rpcPlanningContext) buildRequiredField(typeNode ast.Node, fieldRef int) return field, nil } -func (r *rpcPlanningContext) buildCompositeField(parentNode ast.Node, fragmentSelection fragmentSelection) ([]RPCField, error) { +func (r *rpcPlanningContext) buildCompositeField(inlineFragmentNode ast.Node, fragmentSelection fragmentSelection) ([]RPCField, error) { fieldRefs := r.operation.SelectionSetFieldRefs(fragmentSelection.selectionSetRef) result := make([]RPCField, 0, len(fieldRefs)) @@ -1136,7 +1141,7 @@ func (r *rpcPlanningContext) buildCompositeField(parentNode ast.Node, fragmentSe return nil, fmt.Errorf("unable to build composite field: field definition not found for field %s", r.operation.FieldNameString(fieldRef)) } - field, err := r.buildField(parentNode, fieldDef, r.operation.FieldNameString(fieldRef), "") + field, err := r.buildField(inlineFragmentNode, fieldDef, r.operation.FieldNameString(fieldRef), "") if err != nil { return nil, err } diff --git a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_federation_test.go b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_federation_test.go index 854f56155..2d8f76fce 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_federation_test.go +++ b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_federation_test.go @@ -1480,7 +1480,7 @@ func TestEntityLookupWithFieldResolvers_WithCompositeTypes(t *testing.T) { JSONPath: "name", }, { - Name: "meowVolume", + Name: "meow_volume", ProtoTypeName: DataTypeInt32, JSONPath: "meowVolume", }, @@ -1492,7 +1492,7 @@ func TestEntityLookupWithFieldResolvers_WithCompositeTypes(t *testing.T) { JSONPath: "name", }, { - Name: "barkVolume", + Name: "bark_volume", ProtoTypeName: DataTypeInt32, JSONPath: "barkVolume", }, diff --git a/v2/pkg/engine/datasource/grpc_datasource/grpc_datasource_test.go b/v2/pkg/engine/datasource/grpc_datasource/grpc_datasource_test.go index 97893033a..f6872b7e2 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/grpc_datasource_test.go +++ b/v2/pkg/engine/datasource/grpc_datasource/grpc_datasource_test.go @@ -3758,6 +3758,307 @@ func Test_DataSource_Load_WithEntity_Calls(t *testing.T) { } } +func Test_DataSource_Load_WithEntity_Calls_WithCompositeTypes(t *testing.T) { + conn, cleanup := setupTestGRPCServer(t) + t.Cleanup(cleanup) + + type graphqlError struct { + Message string `json:"message"` + } + type graphqlResponse struct { + Data map[string]interface{} `json:"data"` + Errors []graphqlError `json:"errors,omitempty"` + } + + testCases := []struct { + name string + query string + vars string + federationConfigs plan.FederationFieldConfigurations + validate func(t *testing.T, data map[string]interface{}) + validateError func(t *testing.T, errData []graphqlError) + }{ + { + name: "Query Product with field resolver returning interface type", + query: `query($representations: [_Any!]!, $includeDetails: Boolean!) { _entities(representations: $representations) { ...on Product { __typename id name mascotRecommendation(includeDetails: $includeDetails) { ... on Cat { __typename name meowVolume } ... on Dog { __typename name barkVolume } } } } }`, + vars: `{ + "variables": { + "representations": [ + {"__typename":"Product","id":"1"}, + {"__typename":"Product","id":"2"}, + {"__typename":"Product","id":"3"} + ], + "includeDetails": true + } + }`, + federationConfigs: plan.FederationFieldConfigurations{ + { + TypeName: "Product", + SelectionSet: "id", + }, + }, + validate: func(t *testing.T, data map[string]interface{}) { + require.NotEmpty(t, data) + + entities, ok := data["_entities"].([]interface{}) + require.True(t, ok, "_entities should be an array") + require.NotEmpty(t, entities, "_entities should not be empty") + require.Len(t, entities, 3, "Should return 3 entities") + + for index, entity := range entities { + entity, ok := entity.(map[string]interface{}) + require.True(t, ok, "entity should be an object") + productID := index + 1 + + require.Equal(t, fmt.Sprintf("%d", productID), entity["id"]) + require.Equal(t, fmt.Sprintf("Product %d", productID), entity["name"]) + + mascot, ok := entity["mascotRecommendation"].(map[string]interface{}) + require.True(t, ok, "mascotRecommendation should be an object") + + // Alternates between Cat and Dog based on index + if index%2 == 0 { + // Should be Cat + typename, ok := mascot["__typename"].(string) + require.True(t, ok, "__typename should be present") + require.Equal(t, "Cat", typename) + + require.Contains(t, mascot, "name") + require.Contains(t, mascot["name"], "MascotCat") + + // Validate meowVolume field + require.Contains(t, mascot, "meowVolume") + meowVolume, ok := mascot["meowVolume"].(float64) + require.True(t, ok, "meowVolume should be a number") + require.Greater(t, meowVolume, float64(0), "meowVolume should be greater than 0") + } else { + // Should be Dog + typename, ok := mascot["__typename"].(string) + require.True(t, ok, "__typename should be present") + require.Equal(t, "Dog", typename) + + require.Contains(t, mascot, "name") + require.Contains(t, mascot["name"], "MascotDog") + + // Validate barkVolume field + require.Contains(t, mascot, "barkVolume") + barkVolume, ok := mascot["barkVolume"].(float64) + require.True(t, ok, "barkVolume should be a number") + require.Greater(t, barkVolume, float64(0), "barkVolume should be greater than 0") + } + } + }, + validateError: func(t *testing.T, errorData []graphqlError) { + require.Empty(t, errorData) + }, + }, + { + name: "Query Product with field resolver returning union type", + query: `query($representations: [_Any!]!, $checkAvailability: Boolean!) { _entities(representations: $representations) { ...on Product { __typename id name stockStatus(checkAvailability: $checkAvailability) { ... on ActionSuccess { __typename message timestamp } ... on ActionError { __typename message code } } } } }`, + vars: `{ + "variables": { + "representations": [ + {"__typename":"Product","id":"1"}, + {"__typename":"Product","id":"2"}, + {"__typename":"Product","id":"3"} + ], + "checkAvailability": false + } + }`, + federationConfigs: plan.FederationFieldConfigurations{ + { + TypeName: "Product", + SelectionSet: "id", + }, + }, + validate: func(t *testing.T, data map[string]interface{}) { + require.NotEmpty(t, data) + + entities, ok := data["_entities"].([]interface{}) + require.True(t, ok, "_entities should be an array") + require.NotEmpty(t, entities, "_entities should not be empty") + require.Len(t, entities, 3, "Should return 3 entities") + + for index, entity := range entities { + entity, ok := entity.(map[string]interface{}) + require.True(t, ok, "entity should be an object") + productID := index + 1 + + require.Equal(t, fmt.Sprintf("%d", productID), entity["id"]) + require.Equal(t, fmt.Sprintf("Product %d", productID), entity["name"]) + + stockStatus, ok := entity["stockStatus"].(map[string]interface{}) + require.True(t, ok, "stockStatus should be an object") + + // With checkAvailability: false, all should be success + typename, ok := stockStatus["__typename"].(string) + require.True(t, ok, "__typename should be present") + require.Equal(t, "ActionSuccess", typename) + + require.Contains(t, stockStatus, "message") + require.Contains(t, stockStatus, "timestamp") + + message, ok := stockStatus["message"].(string) + require.True(t, ok, "message should be a string") + require.Contains(t, message, "in stock and available") + + timestamp, ok := stockStatus["timestamp"].(string) + require.True(t, ok, "timestamp should be a string") + require.NotEmpty(t, timestamp) + } + }, + validateError: func(t *testing.T, errorData []graphqlError) { + require.Empty(t, errorData) + }, + }, + { + name: "Query Product with field resolver returning nested composite types", + query: `query($representations: [_Any!]!, $includeExtended: Boolean!) { _entities(representations: $representations) { ...on Product { __typename id name price productDetails(includeExtended: $includeExtended) { id description recommendedPet { __typename ... on Cat { name meowVolume } ... on Dog { name barkVolume } } reviewSummary { __typename ... on ActionSuccess { message timestamp } ... on ActionError { message code } } } } } }`, + vars: `{ + "variables": { + "representations": [ + {"__typename":"Product","id":"1"}, + {"__typename":"Product","id":"2"} + ], + "includeExtended": false + } + }`, + federationConfigs: plan.FederationFieldConfigurations{ + { + TypeName: "Product", + SelectionSet: "id", + }, + }, + validate: func(t *testing.T, data map[string]interface{}) { + require.NotEmpty(t, data) + + entities, ok := data["_entities"].([]interface{}) + require.True(t, ok, "_entities should be an array") + require.NotEmpty(t, entities, "_entities should not be empty") + require.Len(t, entities, 2, "Should return 2 entities") + + for index, entity := range entities { + entity, ok := entity.(map[string]interface{}) + require.True(t, ok, "entity should be an object") + productID := index + 1 + + require.Equal(t, fmt.Sprintf("%d", productID), entity["id"]) + require.Equal(t, fmt.Sprintf("Product %d", productID), entity["name"]) + + details, ok := entity["productDetails"].(map[string]interface{}) + require.True(t, ok, "productDetails should be an object") + + require.Contains(t, details, "id") + require.Contains(t, details, "description") + require.Contains(t, details["description"], "Standard details") + + // Check recommendedPet (interface) + pet, ok := details["recommendedPet"].(map[string]interface{}) + require.True(t, ok, "recommendedPet should be an object") + + // Alternates between Cat and Dog + if index%2 == 0 { + // Should be Cat + petTypename, ok := pet["__typename"].(string) + require.True(t, ok, "pet __typename should be present") + require.Equal(t, "Cat", petTypename) + + require.Contains(t, pet, "name") + require.Contains(t, pet["name"], "RecommendedCat") + + // Validate meowVolume field + require.Contains(t, pet, "meowVolume") + meowVolume, ok := pet["meowVolume"].(float64) + require.True(t, ok, "meowVolume should be a number") + require.Greater(t, meowVolume, float64(0), "meowVolume should be greater than 0") + } else { + // Should be Dog + petTypename, ok := pet["__typename"].(string) + require.True(t, ok, "pet __typename should be present") + require.Equal(t, "Dog", petTypename) + + require.Contains(t, pet, "name") + require.Contains(t, pet["name"], "RecommendedDog") + + // Validate barkVolume field + require.Contains(t, pet, "barkVolume") + barkVolume, ok := pet["barkVolume"].(float64) + require.True(t, ok, "barkVolume should be a number") + require.Greater(t, barkVolume, float64(0), "barkVolume should be greater than 0") + } + + // Check reviewSummary (union) + reviewSummary, ok := details["reviewSummary"].(map[string]interface{}) + require.True(t, ok, "reviewSummary should be an object") + + // With includeExtended: false and low prices, should be success + reviewTypename, ok := reviewSummary["__typename"].(string) + require.True(t, ok, "reviewSummary __typename should be present") + require.Equal(t, "ActionSuccess", reviewTypename) + + require.Contains(t, reviewSummary, "message") + require.Contains(t, reviewSummary, "timestamp") + + message, ok := reviewSummary["message"].(string) + require.True(t, ok, "message should be a string") + require.Contains(t, message, "positive reviews") + + timestamp, ok := reviewSummary["timestamp"].(string) + require.True(t, ok, "timestamp should be a string") + require.NotEmpty(t, timestamp) + } + }, + validateError: func(t *testing.T, errorData []graphqlError) { + require.Empty(t, errorData) + }, + }, + } + + for _, tc := range testCases { + t.Run(tc.name, func(t *testing.T) { + // Parse the GraphQL schema + schemaDoc := grpctest.MustGraphQLSchema(t) + + // Parse the GraphQL query + queryDoc, report := astparser.ParseGraphqlDocumentString(tc.query) + if report.HasErrors() { + t.Fatalf("failed to parse query: %s", report.Error()) + } + + compiler, err := NewProtoCompiler(grpctest.MustProtoSchema(t), testMapping()) + if err != nil { + t.Fatalf("failed to compile proto: %v", err) + } + + // Create the datasource + ds, err := NewDataSource(conn, DataSourceConfig{ + Operation: &queryDoc, + Definition: &schemaDoc, + SubgraphName: "Products", + Mapping: testMapping(), + Compiler: compiler, + FederationConfigs: tc.federationConfigs, + }) + require.NoError(t, err) + + // Execute the query through our datasource + output := new(bytes.Buffer) + input := fmt.Sprintf(`{"query":%q,"body":%s}`, tc.query, tc.vars) + err = ds.Load(context.Background(), []byte(input), output) + require.NoError(t, err) + + // Parse the response + var resp graphqlResponse + + err = json.Unmarshal(output.Bytes(), &resp) + require.NoError(t, err, "Failed to unmarshal response") + + tc.validate(t, resp.Data) + tc.validateError(t, resp.Errors) + }) + } +} + func Test_Datasource_Load_WithFieldResolvers(t *testing.T) { conn, cleanup := setupTestGRPCServer(t) t.Cleanup(cleanup) From 993c92bc5e4f89798319bab83fed36d5918dbab1 Mon Sep 17 00:00:00 2001 From: Ludwig Bedacht Date: Fri, 31 Oct 2025 16:32:29 +0100 Subject: [PATCH 48/72] chore: add tests for stack --- .../datasource/grpc_datasource/util_test.go | 222 +++++++++++++++++- 1 file changed, 211 insertions(+), 11 deletions(-) diff --git a/v2/pkg/engine/datasource/grpc_datasource/util_test.go b/v2/pkg/engine/datasource/grpc_datasource/util_test.go index 44762e2be..6257b0dfe 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/util_test.go +++ b/v2/pkg/engine/datasource/grpc_datasource/util_test.go @@ -6,15 +6,215 @@ import ( "github.com/stretchr/testify/require" ) -func TestAncestor(t *testing.T) { - ancestor := newStack[int](10) - ancestor.push(1) - ancestor.push(2) - ancestor.push(3) - require.Equal(t, 10, ancestor.capacity()) - require.Equal(t, 3, ancestor.len()) - ancestor.pop() - require.Equal(t, 2, ancestor.len()) - require.Equal(t, 2, ancestor.peek()) - require.Equal(t, 2, ancestor.len()) +func TestStack(t *testing.T) { + t.Run("basic push pop and peek operations", func(t *testing.T) { + s := newStack[int](10) + s.push(1) + s.push(2) + s.push(3) + require.Equal(t, 10, s.capacity()) + require.Equal(t, 3, s.len()) + s.pop() + require.Equal(t, 2, s.len()) + require.Equal(t, 2, s.peek()) + require.Equal(t, 2, s.len()) + }) + + t.Run("empty stack", func(t *testing.T) { + s := newStack[int](5) + require.Equal(t, 0, s.len()) + require.Equal(t, 5, s.capacity()) + + // Pop on empty stack should not panic (documented as no-op) + s.pop() + require.Equal(t, 0, s.len()) + + // Peek on empty stack should panic (documented behavior) + require.Panics(t, func() { + s.peek() + }) + }) + + t.Run("push and pop to empty", func(t *testing.T) { + s := newStack[int](5) + s.push(10) + s.push(20) + s.push(30) + require.Equal(t, 3, s.len()) + require.Equal(t, 30, s.peek()) + + s.pop() + require.Equal(t, 2, s.len()) + require.Equal(t, 20, s.peek()) + + s.pop() + require.Equal(t, 1, s.len()) + require.Equal(t, 10, s.peek()) + + s.pop() + require.Equal(t, 0, s.len()) + + // After popping all elements, peek should panic + require.Panics(t, func() { + s.peek() + }) + }) + + t.Run("push after pop", func(t *testing.T) { + s := newStack[int](10) + s.push(1) + s.push(2) + s.push(3) + require.Equal(t, 3, s.len()) + + s.pop() + require.Equal(t, 2, s.len()) + + s.push(4) + require.Equal(t, 3, s.len()) + require.Equal(t, 4, s.peek()) + }) + + t.Run("fill to capacity and beyond", func(t *testing.T) { + s := newStack[int](3) + s.push(1) + s.push(2) + s.push(3) + require.Equal(t, 3, s.len()) + require.Equal(t, 3, s.capacity()) + + // Push beyond initial capacity (should grow) + s.push(4) + require.Equal(t, 4, s.len()) + require.Greater(t, s.capacity(), 3) + require.Equal(t, 4, s.peek()) + }) + + t.Run("LIFO order verification", func(t *testing.T) { + s := newStack[int](10) + for i := 1; i <= 5; i++ { + s.push(i) + } + + // Verify LIFO order + require.Equal(t, 5, s.peek()) + s.pop() + require.Equal(t, 4, s.peek()) + s.pop() + require.Equal(t, 3, s.peek()) + s.pop() + require.Equal(t, 2, s.peek()) + s.pop() + require.Equal(t, 1, s.peek()) + s.pop() + require.Equal(t, 0, s.len()) + }) + + t.Run("peek does not modify stack", func(t *testing.T) { + s := newStack[int](5) + s.push(100) + s.push(200) + + // Multiple peeks should return same value and not change length + for i := 0; i < 5; i++ { + require.Equal(t, 200, s.peek()) + require.Equal(t, 2, s.len()) + } + }) + + t.Run("stack with string type", func(t *testing.T) { + s := newStack[string](5) + s.push("hello") + s.push("world") + s.push("test") + + require.Equal(t, 3, s.len()) + require.Equal(t, "test", s.peek()) + + s.pop() + require.Equal(t, "world", s.peek()) + + s.pop() + require.Equal(t, "hello", s.peek()) + }) + + t.Run("stack with struct type", func(t *testing.T) { + type testStruct struct { + id int + name string + } + + s := newStack[testStruct](5) + s.push(testStruct{id: 1, name: "first"}) + s.push(testStruct{id: 2, name: "second"}) + + require.Equal(t, 2, s.len()) + top := s.peek() + require.Equal(t, 2, top.id) + require.Equal(t, "second", top.name) + + s.pop() + top = s.peek() + require.Equal(t, 1, top.id) + require.Equal(t, "first", top.name) + }) + + t.Run("large number of operations", func(t *testing.T) { + s := newStack[int](10) + + // Push 100 items + for i := 0; i < 100; i++ { + s.push(i) + } + require.Equal(t, 100, s.len()) + require.Equal(t, 99, s.peek()) + + // Pop 50 items + for i := 0; i < 50; i++ { + s.pop() + } + require.Equal(t, 50, s.len()) + require.Equal(t, 49, s.peek()) + + // Push 25 more items + for i := 100; i < 125; i++ { + s.push(i) + } + require.Equal(t, 75, s.len()) + require.Equal(t, 124, s.peek()) + }) + + t.Run("alternating push and pop", func(t *testing.T) { + s := newStack[int](5) + + s.push(1) + require.Equal(t, 1, s.len()) + + s.pop() + require.Equal(t, 0, s.len()) + + s.push(2) + s.push(3) + require.Equal(t, 2, s.len()) + + s.pop() + require.Equal(t, 1, s.len()) + require.Equal(t, 2, s.peek()) + + s.push(4) + s.push(5) + require.Equal(t, 3, s.len()) + require.Equal(t, 5, s.peek()) + }) + + t.Run("zero capacity stack", func(t *testing.T) { + s := newStack[int](0) + require.Equal(t, 0, s.len()) + require.Equal(t, 0, s.capacity()) + + // Should still be able to push (will grow) + s.push(42) + require.Equal(t, 1, s.len()) + require.Equal(t, 42, s.peek()) + }) } From 34f2b795066a275115033e5adff3bed96fe8c5d6 Mon Sep 17 00:00:00 2001 From: Ludwig Bedacht Date: Fri, 31 Oct 2025 16:38:16 +0100 Subject: [PATCH 49/72] chore: remove println --- v2/pkg/engine/datasource/grpc_datasource/execution_plan.go | 4 ---- 1 file changed, 4 deletions(-) diff --git a/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go b/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go index 93a2a78e1..52cc4d519 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go +++ b/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go @@ -735,10 +735,6 @@ func (r *rpcPlanningContext) buildFieldMessage(fieldTypeNode ast.Node, fieldRef } } - fmt.Println("fieldName", r.operation.FieldNameString(fieldRef)) - - fmt.Println("fieldRefs", fieldRefs, "inlineFragmentRefs", inlineFragmentRefs) - message := &RPCMessage{ Name: fieldTypeNode.NameString(r.definition), } From a165f7b88b2cdb8ccb497673b750a47654682b9a Mon Sep 17 00:00:00 2001 From: Ludwig Bedacht Date: Mon, 17 Nov 2025 10:27:13 +0100 Subject: [PATCH 50/72] chore: fix incorrect variable name --- .../grpc_datasource/execution_plan_field_resolvers_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_field_resolvers_test.go b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_field_resolvers_test.go index aa2b529f9..ab9d28d49 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_field_resolvers_test.go +++ b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_field_resolvers_test.go @@ -1120,7 +1120,7 @@ func TestExecutionPlanFieldResolvers_WithCompositeTypes(t *testing.T) { }{ { name: "Should create an execution plan for a query with interface type", - query: "query CategoriesWithNestedResolvers($includeValue: Boolean!) { categories { mascot(includeVolume: $includeVolume) { ... on Cat { name } ... on Dog { name } } } }", + query: "query CategoriesWithNestedResolvers($includeVolume: Boolean!) { categories { mascot(includeVolume: $includeVolume) { ... on Cat { name } ... on Dog { name } } } }", expectedPlan: &RPCExecutionPlan{ Calls: []RPCCall{ { From 0d1466b2fb44eee00681b1c18408903565ffe738 Mon Sep 17 00:00:00 2001 From: Ludwig Bedacht Date: Mon, 17 Nov 2025 13:26:30 +0100 Subject: [PATCH 51/72] feat: handle nested types when using composite types in field resolvers --- .../grpc_datasource/execution_plan.go | 15 + .../execution_plan_field_resolvers_test.go | 284 +- .../grpc_datasource/grpc_datasource_test.go | 93 + .../grpc_datasource/mapping_test_helper.go | 87 + v2/pkg/grpctest/mapping/mapping.go | 87 + v2/pkg/grpctest/mockservice.go | 3617 ++--------------- v2/pkg/grpctest/mockservice_enums.go | 143 + v2/pkg/grpctest/mockservice_lists.go | 1751 ++++++++ v2/pkg/grpctest/mockservice_lookup.go | 76 + .../grpctest/mockservice_nullable_fields.go | 292 ++ v2/pkg/grpctest/mockservice_resolve.go | 772 ++++ v2/pkg/grpctest/product.proto | 43 + v2/pkg/grpctest/productv1/product.pb.go | 1069 +++-- v2/pkg/grpctest/testdata/products.graphqls | 43 + v2/pkg/grpctest/util.go | 148 + 15 files changed, 4987 insertions(+), 3533 deletions(-) create mode 100644 v2/pkg/grpctest/mockservice_enums.go create mode 100644 v2/pkg/grpctest/mockservice_lists.go create mode 100644 v2/pkg/grpctest/mockservice_lookup.go create mode 100644 v2/pkg/grpctest/mockservice_nullable_fields.go create mode 100644 v2/pkg/grpctest/mockservice_resolve.go create mode 100644 v2/pkg/grpctest/util.go diff --git a/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go b/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go index 52cc4d519..7f89996b2 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go +++ b/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go @@ -1141,6 +1141,21 @@ func (r *rpcPlanningContext) buildCompositeField(inlineFragmentNode ast.Node, fr if err != nil { return nil, err } + + if field.ProtoTypeName == DataTypeMessage && r.operation.FieldHasSelections(fieldRef) { + fieldTypeNode, found := r.definition.ResolveNodeFromTypeRef(r.definition.FieldDefinitionType(fieldDef)) + if !found { + return nil, fmt.Errorf("unable to build required field: unable to resolve field type node for field %s", r.operation.FieldNameString(fieldRef)) + } + + message, err := r.buildFieldMessage(fieldTypeNode, fieldRef) + if err != nil { + return nil, err + } + + field.Message = message + } + result = append(result, field) } return result, nil diff --git a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_field_resolvers_test.go b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_field_resolvers_test.go index ab9d28d49..17cd47dd5 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_field_resolvers_test.go +++ b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_field_resolvers_test.go @@ -1120,7 +1120,7 @@ func TestExecutionPlanFieldResolvers_WithCompositeTypes(t *testing.T) { }{ { name: "Should create an execution plan for a query with interface type", - query: "query CategoriesWithNestedResolvers($includeVolume: Boolean!) { categories { mascot(includeVolume: $includeVolume) { ... on Cat { name } ... on Dog { name } } } }", + query: "query CategoriesWithNestedResolvers($includeVolume: Boolean!) { categories { mascot(includeVolume: $includeVolume) { ... on Cat { name } ... on Dog { name } } } }", expectedPlan: &RPCExecutionPlan{ Calls: []RPCCall{ { @@ -1892,6 +1892,288 @@ func TestExecutionPlanFieldResolvers_WithCompositeTypes(t *testing.T) { }, }, }, + { + name: "Should create an execution plan for a query with nested selection in an interface with inline fragments", + query: "query TestContainersWithInterface($includeExtended: Boolean!) { testContainers { id name details(includeExtended: $includeExtended) { id summary pet { ... on Cat { id name owner { name contact { email } } breed { name characteristics { temperament } } } ... on Dog { id name owner { name contact { phone } } breed { origin characteristics { size } } } } } } }", + expectedPlan: &RPCExecutionPlan{ + Calls: []RPCCall{ + { + ServiceName: "Products", + MethodName: "QueryTestContainers", + Request: RPCMessage{ + Name: "QueryTestContainersRequest", + }, + Response: RPCMessage{ + Name: "QueryTestContainersResponse", + Fields: []RPCField{ + { + Name: "test_containers", + ProtoTypeName: DataTypeMessage, + JSONPath: "testContainers", + Repeated: true, + Message: &RPCMessage{ + Name: "TestContainer", + Fields: []RPCField{ + { + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", + }, + { + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", + }, + }, + }, + }, + }, + }, + }, + { + DependentCalls: []int{0}, + ServiceName: "Products", + MethodName: "ResolveTestContainerDetails", + Kind: CallKindResolve, + ResponsePath: buildPath("testContainers.details"), + Request: RPCMessage{ + Name: "ResolveTestContainerDetailsRequest", + Fields: []RPCField{ + { + Name: "context", + ProtoTypeName: DataTypeMessage, + Repeated: true, + Message: &RPCMessage{ + Name: "ResolveTestContainerDetailsContext", + Fields: []RPCField{ + { + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", + ResolvePath: buildPath("test_containers.id"), + }, + { + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", + ResolvePath: buildPath("test_containers.name"), + }, + }, + }, + }, + { + Name: "field_args", + ProtoTypeName: DataTypeMessage, + Message: &RPCMessage{ + Name: "ResolveTestContainerDetailsArgs", + Fields: []RPCField{ + { + Name: "include_extended", + ProtoTypeName: DataTypeBool, + JSONPath: "includeExtended", + }, + }, + }, + }, + }, + }, + Response: RPCMessage{ + Name: "ResolveTestContainerDetailsResponse", + Fields: []RPCField{ + { + Name: "result", + ProtoTypeName: DataTypeMessage, + JSONPath: "result", + Repeated: true, + Message: &RPCMessage{ + Name: "ResolveTestContainerDetailsResult", + Fields: []RPCField{ + { + Name: "details", + ProtoTypeName: DataTypeMessage, + JSONPath: "details", + Optional: true, + Message: &RPCMessage{ + Name: "TestDetails", + Fields: []RPCField{ + { + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", + }, + { + Name: "summary", + ProtoTypeName: DataTypeString, + JSONPath: "summary", + }, + { + Name: "pet", + ProtoTypeName: DataTypeMessage, + JSONPath: "pet", + Message: &RPCMessage{ + Name: "Animal", + OneOfType: OneOfTypeInterface, + MemberTypes: []string{"Cat", "Dog"}, + FieldSelectionSet: RPCFieldSelectionSet{ + "Cat": { + { + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", + }, + { + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", + }, + { + Name: "owner", + ProtoTypeName: DataTypeMessage, + JSONPath: "owner", + Message: &RPCMessage{ + Name: "Owner", + Fields: []RPCField{ + { + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", + }, + { + Name: "contact", + ProtoTypeName: DataTypeMessage, + JSONPath: "contact", + Message: &RPCMessage{ + Name: "ContactInfo", + Fields: []RPCField{ + { + Name: "email", + ProtoTypeName: DataTypeString, + JSONPath: "email", + }, + }, + }, + }, + }, + }, + }, + { + Name: "breed", + ProtoTypeName: DataTypeMessage, + JSONPath: "breed", + Message: &RPCMessage{ + Name: "CatBreed", + Fields: []RPCField{ + { + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", + }, + { + Name: "characteristics", + ProtoTypeName: DataTypeMessage, + JSONPath: "characteristics", + Message: &RPCMessage{ + Name: "BreedCharacteristics", + Fields: []RPCField{ + { + Name: "temperament", + ProtoTypeName: DataTypeString, + JSONPath: "temperament", + }, + }, + }, + }, + }, + }, + }, + }, + "Dog": { + { + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", + }, + { + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", + }, + { + Name: "owner", + ProtoTypeName: DataTypeMessage, + JSONPath: "owner", + Message: &RPCMessage{ + Name: "Owner", + Fields: []RPCField{ + { + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", + }, + { + Name: "contact", + ProtoTypeName: DataTypeMessage, + JSONPath: "contact", + Message: &RPCMessage{ + Name: "ContactInfo", + Fields: []RPCField{ + { + Name: "phone", + ProtoTypeName: DataTypeString, + JSONPath: "phone", + }, + }, + }, + }, + }, + }, + }, + { + Name: "breed", + ProtoTypeName: DataTypeMessage, + JSONPath: "breed", + Message: &RPCMessage{ + Name: "DogBreed", + Fields: []RPCField{ + { + Name: "origin", + ProtoTypeName: DataTypeString, + JSONPath: "origin", + }, + { + Name: "characteristics", + ProtoTypeName: DataTypeMessage, + JSONPath: "characteristics", + Message: &RPCMessage{ + Name: "BreedCharacteristics", + Fields: []RPCField{ + { + Name: "size", + ProtoTypeName: DataTypeString, + JSONPath: "size", + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, } for _, tt := range tests { diff --git a/v2/pkg/engine/datasource/grpc_datasource/grpc_datasource_test.go b/v2/pkg/engine/datasource/grpc_datasource/grpc_datasource_test.go index f6872b7e2..76da7be5d 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/grpc_datasource_test.go +++ b/v2/pkg/engine/datasource/grpc_datasource/grpc_datasource_test.go @@ -4446,6 +4446,99 @@ func Test_Datasource_Load_WithFieldResolvers(t *testing.T) { require.Empty(t, errData) }, }, + { + name: "Query with nested field resolver returning interface with deeply nested fields", + query: "query TestContainersWithInterface($includeExtended: Boolean!) { testContainers { id name details(includeExtended: $includeExtended) { id summary pet { ... on Cat { id name owner { name contact { email } } breed { name characteristics { temperament } } } ... on Dog { id name owner { name contact { phone } } breed { origin characteristics { size } } } } } } }", + vars: `{"variables":{"includeExtended":false}}`, + validate: func(t *testing.T, data map[string]interface{}) { + require.NotEmpty(t, data) + + containers, ok := data["testContainers"].([]interface{}) + require.True(t, ok, "testContainers should be an array") + require.NotEmpty(t, containers, "testContainers should not be empty") + require.Len(t, containers, 3, "Should return 3 test containers") + + // Based on mockservice_resolve.go implementation: + // - Even indices (0, 2) return Cat with owner and breed details + // - Odd indices (1) return Dog with owner and breed details + for i, container := range containers { + container, ok := container.(map[string]interface{}) + require.True(t, ok, "container should be an object") + require.NotEmpty(t, container["id"]) + require.NotEmpty(t, container["name"]) + require.NotEmpty(t, container["details"]) + + details, ok := container["details"].(map[string]interface{}) + require.True(t, ok, "details should be an object") + require.NotEmpty(t, details["id"]) + require.NotEmpty(t, details["summary"]) + require.NotEmpty(t, details["pet"]) + + pet, ok := details["pet"].(map[string]interface{}) + require.True(t, ok, "pet should be an object") + require.NotEmpty(t, pet["id"]) + require.NotEmpty(t, pet["name"]) + + // Validate owner exists + owner, ok := pet["owner"].(map[string]interface{}) + require.True(t, ok, "owner should be an object") + require.NotEmpty(t, owner["name"]) + + // Validate contact exists + contact, ok := owner["contact"].(map[string]interface{}) + require.True(t, ok, "contact should be an object") + + // Validate breed exists + breed, ok := pet["breed"].(map[string]interface{}) + require.True(t, ok, "breed should be an object") + + // Validate characteristics exists + characteristics, ok := breed["characteristics"].(map[string]interface{}) + require.True(t, ok, "characteristics should be an object") + + if i%2 == 0 { + // Should be Cat + require.Contains(t, pet["name"], "TestCat", "Cat name should contain 'TestCat'") + require.Contains(t, owner["name"], "OwnerTestCat", "Cat owner name should contain 'OwnerTestCat'") + + // Cat should have email in contact + require.NotEmpty(t, contact["email"], "Cat owner should have email") + require.Equal(t, "owner-test-cat@example.com", contact["email"], "Cat owner email should match") + require.Empty(t, contact["phone"], "Cat query should not have phone") + + // Cat breed should have name and temperament in characteristics + require.NotEmpty(t, breed["name"], "Cat breed should have name") + require.Contains(t, breed["name"], "BreedTestCat", "Cat breed name should contain 'BreedTestCat'") + require.Empty(t, breed["origin"], "Cat query should not have breed origin") + + require.NotEmpty(t, characteristics["temperament"], "Cat breed should have temperament") + require.Equal(t, "Curious", characteristics["temperament"], "Cat breed temperament should be 'Curious'") + require.Empty(t, characteristics["size"], "Cat query should not have breed size") + } else { + // Should be Dog + require.Contains(t, pet["name"], "TestDog", "Dog name should contain 'TestDog'") + require.Contains(t, owner["name"], "OwnerTestDog", "Dog owner name should contain 'OwnerTestDog'") + + // Dog should have phone in contact + require.NotEmpty(t, contact["phone"], "Dog owner should have phone") + require.Equal(t, "555-666-7777", contact["phone"], "Dog owner phone should match") + require.Empty(t, contact["email"], "Dog query should not have email") + + // Dog breed should have origin and size in characteristics + require.NotEmpty(t, breed["origin"], "Dog breed should have origin") + require.Equal(t, "England", breed["origin"], "Dog breed origin should be 'England'") + require.Empty(t, breed["name"], "Dog query should not have breed name") + + require.NotEmpty(t, characteristics["size"], "Dog breed should have size") + require.Equal(t, "Medium", characteristics["size"], "Dog breed size should be 'Medium'") + require.Empty(t, characteristics["temperament"], "Dog query should not have breed temperament") + } + } + }, + validateError: func(t *testing.T, errData []graphqlError) { + require.Empty(t, errData) + }, + }, } for _, tc := range testCases { diff --git a/v2/pkg/engine/datasource/grpc_datasource/mapping_test_helper.go b/v2/pkg/engine/datasource/grpc_datasource/mapping_test_helper.go index aabc3e10e..c8b63d09c 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/mapping_test_helper.go +++ b/v2/pkg/engine/datasource/grpc_datasource/mapping_test_helper.go @@ -1000,6 +1000,12 @@ func testMapping() *GRPCMapping { "meowVolume": { TargetName: "meow_volume", }, + "owner": { + TargetName: "owner", + }, + "breed": { + TargetName: "breed", + }, }, "Dog": { "id": { @@ -1014,6 +1020,87 @@ func testMapping() *GRPCMapping { "barkVolume": { TargetName: "bark_volume", }, + "owner": { + TargetName: "owner", + }, + "breed": { + TargetName: "breed", + }, + }, + "Owner": { + "id": { + TargetName: "id", + }, + "name": { + TargetName: "name", + }, + "contact": { + TargetName: "contact", + }, + }, + "ContactInfo": { + "email": { + TargetName: "email", + }, + "phone": { + TargetName: "phone", + }, + "address": { + TargetName: "address", + }, + }, + "Address": { + "street": { + TargetName: "street", + }, + "city": { + TargetName: "city", + }, + "country": { + TargetName: "country", + }, + "zipCode": { + TargetName: "zip_code", + }, + }, + "CatBreed": { + "id": { + TargetName: "id", + }, + "name": { + TargetName: "name", + }, + "origin": { + TargetName: "origin", + }, + "characteristics": { + TargetName: "characteristics", + }, + }, + "DogBreed": { + "id": { + TargetName: "id", + }, + "name": { + TargetName: "name", + }, + "origin": { + TargetName: "origin", + }, + "characteristics": { + TargetName: "characteristics", + }, + }, + "BreedCharacteristics": { + "size": { + TargetName: "size", + }, + "temperament": { + TargetName: "temperament", + }, + "lifespan": { + TargetName: "lifespan", + }, }, "ActionSuccess": { "message": { diff --git a/v2/pkg/grpctest/mapping/mapping.go b/v2/pkg/grpctest/mapping/mapping.go index 7f7060237..5acef534d 100644 --- a/v2/pkg/grpctest/mapping/mapping.go +++ b/v2/pkg/grpctest/mapping/mapping.go @@ -1007,6 +1007,12 @@ func DefaultGRPCMapping() *grpcdatasource.GRPCMapping { "meowVolume": { TargetName: "meow_volume", }, + "owner": { + TargetName: "owner", + }, + "breed": { + TargetName: "breed", + }, }, "Dog": { "id": { @@ -1021,6 +1027,87 @@ func DefaultGRPCMapping() *grpcdatasource.GRPCMapping { "barkVolume": { TargetName: "bark_volume", }, + "owner": { + TargetName: "owner", + }, + "breed": { + TargetName: "breed", + }, + }, + "Owner": { + "id": { + TargetName: "id", + }, + "name": { + TargetName: "name", + }, + "contact": { + TargetName: "contact", + }, + }, + "ContactInfo": { + "email": { + TargetName: "email", + }, + "phone": { + TargetName: "phone", + }, + "address": { + TargetName: "address", + }, + }, + "Address": { + "street": { + TargetName: "street", + }, + "city": { + TargetName: "city", + }, + "country": { + TargetName: "country", + }, + "zipCode": { + TargetName: "zip_code", + }, + }, + "CatBreed": { + "id": { + TargetName: "id", + }, + "name": { + TargetName: "name", + }, + "origin": { + TargetName: "origin", + }, + "characteristics": { + TargetName: "characteristics", + }, + }, + "DogBreed": { + "id": { + TargetName: "id", + }, + "name": { + TargetName: "name", + }, + "origin": { + TargetName: "origin", + }, + "characteristics": { + TargetName: "characteristics", + }, + }, + "BreedCharacteristics": { + "size": { + TargetName: "size", + }, + "temperament": { + TargetName: "temperament", + }, + "lifespan": { + TargetName: "lifespan", + }, }, "ActionSuccess": { "message": { diff --git a/v2/pkg/grpctest/mockservice.go b/v2/pkg/grpctest/mockservice.go index 99c19bad8..d98c6ba69 100644 --- a/v2/pkg/grpctest/mockservice.go +++ b/v2/pkg/grpctest/mockservice.go @@ -3,7 +3,6 @@ package grpctest import ( context "context" "fmt" - "math" "math/rand" "strconv" @@ -20,181 +19,6 @@ type MockService struct { productv1.UnimplementedProductServiceServer } -// ResolveProductMascotRecommendation implements productv1.ProductServiceServer. -func (s *MockService) ResolveProductMascotRecommendation(_ context.Context, req *productv1.ResolveProductMascotRecommendationRequest) (*productv1.ResolveProductMascotRecommendationResponse, error) { - results := make([]*productv1.ResolveProductMascotRecommendationResult, 0, len(req.GetContext())) - - includeDetails := false - if req.GetFieldArgs() != nil { - includeDetails = req.GetFieldArgs().GetIncludeDetails() - } - - for i, ctx := range req.GetContext() { - // Alternate between Cat and Dog based on index - var animal *productv1.Animal - if i%2 == 0 { - volume := int32(5) - if includeDetails { - volume = int32((i + 1) * 8) - } - animal = &productv1.Animal{ - Instance: &productv1.Animal_Cat{ - Cat: &productv1.Cat{ - Id: fmt.Sprintf("mascot-cat-%s", ctx.GetId()), - Name: fmt.Sprintf("MascotCat for %s", ctx.GetName()), - Kind: "Cat", - MeowVolume: volume, - }, - }, - } - } else { - volume := int32(7) - if includeDetails { - volume = int32((i + 1) * 10) - } - animal = &productv1.Animal{ - Instance: &productv1.Animal_Dog{ - Dog: &productv1.Dog{ - Id: fmt.Sprintf("mascot-dog-%s", ctx.GetId()), - Name: fmt.Sprintf("MascotDog for %s", ctx.GetName()), - Kind: "Dog", - BarkVolume: volume, - }, - }, - } - } - - results = append(results, &productv1.ResolveProductMascotRecommendationResult{ - MascotRecommendation: animal, - }) - } - - return &productv1.ResolveProductMascotRecommendationResponse{ - Result: results, - }, nil -} - -// ResolveProductProductDetails implements productv1.ProductServiceServer. -func (s *MockService) ResolveProductProductDetails(_ context.Context, req *productv1.ResolveProductProductDetailsRequest) (*productv1.ResolveProductProductDetailsResponse, error) { - results := make([]*productv1.ResolveProductProductDetailsResult, 0, len(req.GetContext())) - - includeExtended := false - if req.GetFieldArgs() != nil { - includeExtended = req.GetFieldArgs().GetIncludeExtended() - } - - for i, ctx := range req.GetContext() { - // Create recommended pet (alternate between Cat and Dog) - var pet *productv1.Animal - if i%2 == 0 { - pet = &productv1.Animal{ - Instance: &productv1.Animal_Cat{ - Cat: &productv1.Cat{ - Id: fmt.Sprintf("details-cat-%s", ctx.GetId()), - Name: fmt.Sprintf("RecommendedCat for %s", ctx.GetName()), - Kind: "Cat", - MeowVolume: int32((i + 1) * 6), - }, - }, - } - } else { - pet = &productv1.Animal{ - Instance: &productv1.Animal_Dog{ - Dog: &productv1.Dog{ - Id: fmt.Sprintf("details-dog-%s", ctx.GetId()), - Name: fmt.Sprintf("RecommendedDog for %s", ctx.GetName()), - Kind: "Dog", - BarkVolume: int32((i + 1) * 9), - }, - }, - } - } - - // Create review summary (alternate between success and error based on price and extended flag) - var reviewSummary *productv1.ActionResult - if includeExtended && ctx.GetPrice() > 500 { - reviewSummary = &productv1.ActionResult{ - Value: &productv1.ActionResult_ActionError{ - ActionError: &productv1.ActionError{ - Message: fmt.Sprintf("Product %s has negative reviews", ctx.GetName()), - Code: "NEGATIVE_REVIEWS", - }, - }, - } - } else { - reviewSummary = &productv1.ActionResult{ - Value: &productv1.ActionResult_ActionSuccess{ - ActionSuccess: &productv1.ActionSuccess{ - Message: fmt.Sprintf("Product %s has positive reviews", ctx.GetName()), - Timestamp: "2024-01-01T15:00:00Z", - }, - }, - } - } - - description := fmt.Sprintf("Standard details for %s", ctx.GetName()) - if includeExtended { - description = fmt.Sprintf("Extended details for %s with comprehensive information", ctx.GetName()) - } - - results = append(results, &productv1.ResolveProductProductDetailsResult{ - ProductDetails: &productv1.ProductDetails{ - Id: fmt.Sprintf("details-%s-%d", ctx.GetId(), i), - Description: description, - ReviewSummary: reviewSummary, - RecommendedPet: pet, - }, - }) - } - - return &productv1.ResolveProductProductDetailsResponse{ - Result: results, - }, nil -} - -// ResolveProductStockStatus implements productv1.ProductServiceServer. -func (s *MockService) ResolveProductStockStatus(_ context.Context, req *productv1.ResolveProductStockStatusRequest) (*productv1.ResolveProductStockStatusResponse, error) { - results := make([]*productv1.ResolveProductStockStatusResult, 0, len(req.GetContext())) - - checkAvailability := false - if req.GetFieldArgs() != nil { - checkAvailability = req.GetFieldArgs().GetCheckAvailability() - } - - for i, ctx := range req.GetContext() { - var stockStatus *productv1.ActionResult - - // If checking availability and price is high, return out of stock error - if checkAvailability && ctx.GetPrice() > 300 && i%2 == 0 { - stockStatus = &productv1.ActionResult{ - Value: &productv1.ActionResult_ActionError{ - ActionError: &productv1.ActionError{ - Message: fmt.Sprintf("Product %s is currently out of stock", ctx.GetName()), - Code: "OUT_OF_STOCK", - }, - }, - } - } else { - stockStatus = &productv1.ActionResult{ - Value: &productv1.ActionResult_ActionSuccess{ - ActionSuccess: &productv1.ActionSuccess{ - Message: fmt.Sprintf("Product %s is in stock and available", ctx.GetName()), - Timestamp: "2024-01-01T10:00:00Z", - }, - }, - } - } - - results = append(results, &productv1.ResolveProductStockStatusResult{ - StockStatus: stockStatus, - }) - } - - return &productv1.ResolveProductStockStatusResponse{ - Result: results, - }, nil -} - // QueryTestContainer implements productv1.ProductServiceServer. func (s *MockService) QueryTestContainer(_ context.Context, req *productv1.QueryTestContainerRequest) (*productv1.QueryTestContainerResponse, error) { id := req.GetId() @@ -226,3180 +50,517 @@ func (s *MockService) QueryTestContainers(_ context.Context, _ *productv1.QueryT }, nil } -// ResolveTestContainerDetails implements productv1.ProductServiceServer. -func (s *MockService) ResolveTestContainerDetails(_ context.Context, req *productv1.ResolveTestContainerDetailsRequest) (*productv1.ResolveTestContainerDetailsResponse, error) { - results := make([]*productv1.ResolveTestContainerDetailsResult, 0, len(req.GetContext())) +// MutationPerformAction implements productv1.ProductServiceServer. +func (s *MockService) MutationPerformAction(ctx context.Context, in *productv1.MutationPerformActionRequest) (*productv1.MutationPerformActionResponse, error) { + input := in.GetInput() + actionType := input.GetType() - includeExtended := false - if req.GetFieldArgs() != nil { - includeExtended = req.GetFieldArgs().GetIncludeExtended() - } + // Simulate different action results based on the action type + var result *productv1.ActionResult - for i, ctx := range req.GetContext() { - // Alternate between Cat and Dog for the pet field (Animal interface) - var pet *productv1.Animal - if i%2 == 0 { - pet = &productv1.Animal{ - Instance: &productv1.Animal_Cat{ - Cat: &productv1.Cat{ - Id: fmt.Sprintf("test-cat-%s", ctx.GetId()), - Name: fmt.Sprintf("TestCat-%s", ctx.GetName()), - Kind: "Cat", - MeowVolume: int32((i + 1) * 5), - }, - }, - } - } else { - pet = &productv1.Animal{ - Instance: &productv1.Animal_Dog{ - Dog: &productv1.Dog{ - Id: fmt.Sprintf("test-dog-%s", ctx.GetId()), - Name: fmt.Sprintf("TestDog-%s", ctx.GetName()), - Kind: "Dog", - BarkVolume: int32((i + 1) * 7), - }, + switch actionType { + case "error_action": + // Return an error result + result = &productv1.ActionResult{ + Value: &productv1.ActionResult_ActionError{ + ActionError: &productv1.ActionError{ + Message: "Action failed due to validation error", + Code: "VALIDATION_ERROR", }, - } + }, } - - // Alternate between ActionSuccess and ActionError for the status field (ActionResult union) - var status *productv1.ActionResult - if includeExtended && i%3 == 0 { - // Return error status for extended mode on certain items - status = &productv1.ActionResult{ - Value: &productv1.ActionResult_ActionError{ - ActionError: &productv1.ActionError{ - Message: fmt.Sprintf("Extended check failed for %s", ctx.GetName()), - Code: "EXTENDED_CHECK_FAILED", - }, - }, - } - } else { - // Return success status - status = &productv1.ActionResult{ - Value: &productv1.ActionResult_ActionSuccess{ - ActionSuccess: &productv1.ActionSuccess{ - Message: fmt.Sprintf("TestContainer %s details loaded successfully", ctx.GetName()), - Timestamp: "2024-01-01T12:00:00Z", - }, + case "invalid_action": + // Return a different error result + result = &productv1.ActionResult{ + Value: &productv1.ActionResult_ActionError{ + ActionError: &productv1.ActionError{ + Message: "Invalid action type provided", + Code: "INVALID_ACTION", }, - } - } - - summary := fmt.Sprintf("Summary for %s", ctx.GetName()) - if includeExtended { - summary = fmt.Sprintf("Extended summary for %s with additional details", ctx.GetName()) + }, } - - results = append(results, &productv1.ResolveTestContainerDetailsResult{ - Details: &productv1.TestDetails{ - Id: fmt.Sprintf("details-%s-%d", ctx.GetId(), i), - Summary: summary, - Pet: pet, - Status: status, + default: + // Return a success result + result = &productv1.ActionResult{ + Value: &productv1.ActionResult_ActionSuccess{ + ActionSuccess: &productv1.ActionSuccess{ + Message: fmt.Sprintf("Action '%s' completed successfully", actionType), + Timestamp: "2024-01-01T00:00:00Z", + }, }, - }) + } } - return &productv1.ResolveTestContainerDetailsResponse{ - Result: results, + return &productv1.MutationPerformActionResponse{ + PerformAction: result, }, nil } -// ResolveCategoryMetricsNormalizedScore implements productv1.ProductServiceServer. -func (s *MockService) ResolveCategoryMetricsNormalizedScore(_ context.Context, req *productv1.ResolveCategoryMetricsNormalizedScoreRequest) (*productv1.ResolveCategoryMetricsNormalizedScoreResponse, error) { - results := make([]*productv1.ResolveCategoryMetricsNormalizedScoreResult, 0, len(req.GetContext())) - - baseline := req.GetFieldArgs().GetBaseline() - if baseline == 0 { - baseline = 1.0 // Avoid division by zero - } - - for _, ctx := range req.GetContext() { - // Calculate normalized score: (value / baseline) * 100 - // This gives a percentage relative to the baseline - normalizedScore := (ctx.GetValue() / baseline) * 100.0 - - results = append(results, &productv1.ResolveCategoryMetricsNormalizedScoreResult{ - NormalizedScore: normalizedScore, - }) - } - - resp := &productv1.ResolveCategoryMetricsNormalizedScoreResponse{ - Result: results, - } - - return resp, nil -} - -// ResolveCategoryMascot implements productv1.ProductServiceServer. -func (s *MockService) ResolveCategoryMascot(_ context.Context, req *productv1.ResolveCategoryMascotRequest) (*productv1.ResolveCategoryMascotResponse, error) { - results := make([]*productv1.ResolveCategoryMascotResult, 0, len(req.GetContext())) - - includeVolume := false - if req.GetFieldArgs() != nil { - includeVolume = req.GetFieldArgs().GetIncludeVolume() - } +// QueryRandomSearchResult implements productv1.ProductServiceServer. +func (s *MockService) QueryRandomSearchResult(ctx context.Context, in *productv1.QueryRandomSearchResultRequest) (*productv1.QueryRandomSearchResultResponse, error) { + // Randomly return one of the three union types + var result *productv1.SearchResult - for i, ctx := range req.GetContext() { - // Return nil for certain categories to test optional return - if ctx.GetKind() == productv1.CategoryKind_CATEGORY_KIND_OTHER { - results = append(results, &productv1.ResolveCategoryMascotResult{ - Mascot: nil, - }) - } else { - // Alternate between Cat and Dog based on category kind - var animal *productv1.Animal - if ctx.GetKind() == productv1.CategoryKind_CATEGORY_KIND_BOOK || ctx.GetKind() == productv1.CategoryKind_CATEGORY_KIND_ELECTRONICS { - volume := int32(0) - if includeVolume { - volume = int32(i*10 + 5) - } - animal = &productv1.Animal{ - Instance: &productv1.Animal_Cat{ - Cat: &productv1.Cat{ - Id: fmt.Sprintf("cat-mascot-%s", ctx.GetId()), - Name: fmt.Sprintf("Whiskers-%s", ctx.GetId()), - Kind: "Cat", - MeowVolume: volume, - }, - }, - } - } else { - volume := int32(0) - if includeVolume { - volume = int32(i*10 + 10) - } - animal = &productv1.Animal{ - Instance: &productv1.Animal_Dog{ - Dog: &productv1.Dog{ - Id: fmt.Sprintf("dog-mascot-%s", ctx.GetId()), - Name: fmt.Sprintf("Buddy-%s", ctx.GetId()), - Kind: "Dog", - BarkVolume: volume, - }, - }, - } - } - results = append(results, &productv1.ResolveCategoryMascotResult{ - Mascot: animal, - }) + switch rand.Intn(3) { + case 0: + // Return a Product + result = &productv1.SearchResult{ + Value: &productv1.SearchResult_Product{ + Product: &productv1.Product{ + Id: "product-random-1", + Name: "Random Product", + Price: 29.99, + }, + }, + } + case 1: + // Return a User + result = &productv1.SearchResult{ + Value: &productv1.SearchResult_User{ + User: &productv1.User{ + Id: "user-random-1", + Name: "Random User", + }, + }, + } + default: + // Return a Category + kind := productv1.CategoryKind_CATEGORY_KIND_ELECTRONICS + result = &productv1.SearchResult{ + Value: &productv1.SearchResult_Category{ + Category: &productv1.Category{ + Id: "category-random-1", + Name: "Random Category", + Kind: kind, + Subcategories: createSubcategories("category-random-1", kind, 2), + }, + }, } } - resp := &productv1.ResolveCategoryMascotResponse{ - Result: results, - } - - return resp, nil + return &productv1.QueryRandomSearchResultResponse{ + RandomSearchResult: result, + }, nil } -// ResolveCategoryCategoryStatus implements productv1.ProductServiceServer. -func (s *MockService) ResolveCategoryCategoryStatus(_ context.Context, req *productv1.ResolveCategoryCategoryStatusRequest) (*productv1.ResolveCategoryCategoryStatusResponse, error) { - results := make([]*productv1.ResolveCategoryCategoryStatusResult, 0, len(req.GetContext())) +// QuerySearch implements productv1.ProductServiceServer. +func (s *MockService) QuerySearch(ctx context.Context, in *productv1.QuerySearchRequest) (*productv1.QuerySearchResponse, error) { + input := in.GetInput() + query := input.GetQuery() + limit := input.GetLimit() - checkHealth := false - if req.GetFieldArgs() != nil { - checkHealth = req.GetFieldArgs().GetCheckHealth() + // Default limit if not specified + if limit.GetValue() <= 0 { + limit = &wrapperspb.Int32Value{Value: 10} } - for i, ctx := range req.GetContext() { - var actionResult *productv1.ActionResult + var results []*productv1.SearchResult - if checkHealth && i%3 == 0 { - // Return error status for health check failures - actionResult = &productv1.ActionResult{ - Value: &productv1.ActionResult_ActionError{ - ActionError: &productv1.ActionError{ - Message: fmt.Sprintf("Health check failed for category %s", ctx.GetName()), - Code: "HEALTH_CHECK_FAILED", + // Generate a mix of different union types based on the query + for i := int32(0); i < limit.GetValue() && i < 6; i++ { // Cap at 6 results for testing + switch i % 3 { + case 0: + // Add a Product + results = append(results, &productv1.SearchResult{ + Value: &productv1.SearchResult_Product{ + Product: &productv1.Product{ + Id: fmt.Sprintf("product-search-%d", i+1), + Name: fmt.Sprintf("Product matching '%s' #%d", query, i+1), + Price: float64(10 + i*5), }, }, - } - } else { - // Return success status - actionResult = &productv1.ActionResult{ - Value: &productv1.ActionResult_ActionSuccess{ - ActionSuccess: &productv1.ActionSuccess{ - Message: fmt.Sprintf("Category %s is healthy", ctx.GetName()), - Timestamp: "2024-01-01T00:00:00Z", + }) + case 1: + // Add a User + results = append(results, &productv1.SearchResult{ + Value: &productv1.SearchResult_User{ + User: &productv1.User{ + Id: fmt.Sprintf("user-search-%d", i+1), + Name: fmt.Sprintf("User matching '%s' #%d", query, i+1), }, }, - } - } - - results = append(results, &productv1.ResolveCategoryCategoryStatusResult{ - CategoryStatus: actionResult, - }) - } - - resp := &productv1.ResolveCategoryCategoryStatusResponse{ - Result: results, - } - - return resp, nil -} - -// ResolveProductRecommendedCategory implements productv1.ProductServiceServer. -func (s *MockService) ResolveProductRecommendedCategory(_ context.Context, req *productv1.ResolveProductRecommendedCategoryRequest) (*productv1.ResolveProductRecommendedCategoryResponse, error) { - results := make([]*productv1.ResolveProductRecommendedCategoryResult, 0, len(req.GetContext())) - - maxPrice := int32(0) - if req.GetFieldArgs() != nil { - maxPrice = req.GetFieldArgs().GetMaxPrice() - } - - for _, ctx := range req.GetContext() { - // Return nil for products with high price to test optional return - if maxPrice > 0 && ctx.GetPrice() > float64(maxPrice) { - results = append(results, &productv1.ResolveProductRecommendedCategoryResult{ - RecommendedCategory: nil, }) - } else { - // Create a recommended category based on product context - var categoryKind productv1.CategoryKind - if ctx.GetPrice() < 50 { - categoryKind = productv1.CategoryKind_CATEGORY_KIND_BOOK - } else if ctx.GetPrice() < 200 { - categoryKind = productv1.CategoryKind_CATEGORY_KIND_ELECTRONICS - } else { - categoryKind = productv1.CategoryKind_CATEGORY_KIND_FURNITURE + case 2: + // Add a Category + kinds := []productv1.CategoryKind{ + productv1.CategoryKind_CATEGORY_KIND_BOOK, + productv1.CategoryKind_CATEGORY_KIND_ELECTRONICS, + productv1.CategoryKind_CATEGORY_KIND_FURNITURE, } - - results = append(results, &productv1.ResolveProductRecommendedCategoryResult{ - RecommendedCategory: &productv1.Category{ - Id: fmt.Sprintf("recommended-cat-%s", ctx.GetId()), - Name: fmt.Sprintf("Recommended for %s", ctx.GetName()), - Kind: categoryKind, - Subcategories: createSubcategories(fmt.Sprintf("recommended-cat-%s", ctx.GetId()), categoryKind, 2), + kind := kinds[i%int32(len(kinds))] + results = append(results, &productv1.SearchResult{ + Value: &productv1.SearchResult_Category{ + Category: &productv1.Category{ + Id: fmt.Sprintf("category-search-%d", i+1), + Name: fmt.Sprintf("Category matching '%s' #%d", query, i+1), + Kind: kind, + Subcategories: createSubcategories(fmt.Sprintf("category-search-%d", i+1), kind, int(i%3)+1), + }, }, }) } } - resp := &productv1.ResolveProductRecommendedCategoryResponse{ - Result: results, - } - - return resp, nil + return &productv1.QuerySearchResponse{ + Search: results, + }, nil } -// ResolveProductShippingEstimate implements productv1.ProductServiceServer. -func (s *MockService) ResolveProductShippingEstimate(_ context.Context, req *productv1.ResolveProductShippingEstimateRequest) (*productv1.ResolveProductShippingEstimateResponse, error) { - results := make([]*productv1.ResolveProductShippingEstimateResult, 0, len(req.GetContext())) - - for _, ctx := range req.GetContext() { - // Base shipping cost calculation - baseCost := ctx.GetPrice() * 0.1 // 10% of product price - - // Add weight-based cost if input provided - if req.GetFieldArgs() != nil && req.GetFieldArgs().GetInput() != nil { - input := req.GetFieldArgs().GetInput() - - // Add weight cost - weightCost := float64(input.GetWeight()) * 2.5 - baseCost += weightCost - - // Add expedited shipping cost - if input.GetExpedited() != nil && input.GetExpedited().GetValue() { - baseCost *= 1.5 // 50% surcharge for expedited - } - - // Add destination-based cost - destination := input.GetDestination() - switch destination { - case productv1.ShippingDestination_SHIPPING_DESTINATION_INTERNATIONAL: - baseCost += 25.0 - case productv1.ShippingDestination_SHIPPING_DESTINATION_EXPRESS: - baseCost += 10.0 - case productv1.ShippingDestination_SHIPPING_DESTINATION_DOMESTIC: - // No additional cost for domestic shipping - } - } +func (s *MockService) QueryUsers(ctx context.Context, in *productv1.QueryUsersRequest) (*productv1.QueryUsersResponse, error) { + var results []*productv1.User - results = append(results, &productv1.ResolveProductShippingEstimateResult{ - ShippingEstimate: baseCost, + // Generate 3 random users + for i := 1; i <= 3; i++ { + results = append(results, &productv1.User{ + Id: fmt.Sprintf("user-%d", i), + Name: fmt.Sprintf("User %d", i), }) } - resp := &productv1.ResolveProductShippingEstimateResponse{ - Result: results, - } - - return resp, nil -} - -// ResolveCategoryCategoryMetrics implements productv1.ProductServiceServer. -func (s *MockService) ResolveCategoryCategoryMetrics(_ context.Context, req *productv1.ResolveCategoryCategoryMetricsRequest) (*productv1.ResolveCategoryCategoryMetricsResponse, error) { - results := make([]*productv1.ResolveCategoryCategoryMetricsResult, 0, len(req.GetContext())) - - metricType := "" - if req.GetFieldArgs() != nil { - metricType = req.GetFieldArgs().GetMetricType() - } - - for i, ctx := range req.GetContext() { - // Return nil for certain metric types to test optional return - if metricType == "unavailable" { - results = append(results, &productv1.ResolveCategoryCategoryMetricsResult{ - CategoryMetrics: nil, - }) - } else { - results = append(results, &productv1.ResolveCategoryCategoryMetricsResult{ - CategoryMetrics: &productv1.CategoryMetrics{ - Id: fmt.Sprintf("metrics-%s-%d", ctx.GetId(), i), - MetricType: metricType, - Value: float64(i*25 + 100), // Different values based on index - Timestamp: "2024-01-01T00:00:00Z", - CategoryId: ctx.GetId(), - }, - }) - } - } - - resp := &productv1.ResolveCategoryCategoryMetricsResponse{ - Result: results, - } - - return resp, nil -} - -// ResolveCategoryPopularityScore implements productv1.ProductServiceServer. -func (s *MockService) ResolveCategoryPopularityScore(_ context.Context, req *productv1.ResolveCategoryPopularityScoreRequest) (*productv1.ResolveCategoryPopularityScoreResponse, error) { - results := make([]*productv1.ResolveCategoryPopularityScoreResult, 0, len(req.GetContext())) - - threshold := req.GetFieldArgs().GetThreshold() - - baseScore := 50 - for range req.GetContext() { - if int(threshold.GetValue()) > baseScore { - results = append(results, &productv1.ResolveCategoryPopularityScoreResult{ - PopularityScore: nil, - }) - } else { - results = append(results, &productv1.ResolveCategoryPopularityScoreResult{ - PopularityScore: &wrapperspb.Int32Value{Value: int32(baseScore)}, - }) - } - } - - resp := &productv1.ResolveCategoryPopularityScoreResponse{ - Result: results, - } - - return resp, nil -} - -// ResolveSubcategoryItemCount implements productv1.ProductServiceServer. -func (s *MockService) ResolveSubcategoryItemCount(_ context.Context, req *productv1.ResolveSubcategoryItemCountRequest) (*productv1.ResolveSubcategoryItemCountResponse, error) { - results := make([]*productv1.ResolveSubcategoryItemCountResult, 0, len(req.GetContext())) - for i := range req.GetContext() { - results = append(results, &productv1.ResolveSubcategoryItemCountResult{ - ItemCount: int32(i * 10), // Different multiplier to distinguish from productCount - }) - } - - resp := &productv1.ResolveSubcategoryItemCountResponse{ - Result: results, - } - - return resp, nil -} - -// ResolveCategoryProductCount implements productv1.ProductServiceServer. -func (s *MockService) ResolveCategoryProductCount(_ context.Context, req *productv1.ResolveCategoryProductCountRequest) (*productv1.ResolveCategoryProductCountResponse, error) { - results := make([]*productv1.ResolveCategoryProductCountResult, 0, len(req.GetContext())) - for i := range req.GetContext() { - results = append(results, &productv1.ResolveCategoryProductCountResult{ - ProductCount: int32(i), - }) - } - - resp := &productv1.ResolveCategoryProductCountResponse{ - Result: results, - } - - return resp, nil -} - -// LookupWarehouseById implements productv1.ProductServiceServer. -func (s *MockService) LookupWarehouseById(ctx context.Context, in *productv1.LookupWarehouseByIdRequest) (*productv1.LookupWarehouseByIdResponse, error) { - var results []*productv1.Warehouse - - // Special requirement: return one less item than requested to test error handling - // This deliberately breaks the normal pattern of returning the same number of items as keys - keys := in.GetKeys() - if len(keys) == 0 { - return &productv1.LookupWarehouseByIdResponse{ - Result: results, - }, nil - } - - // Return all items except the last one to test error scenarios - for i, input := range keys { - // Skip the last item to create an intentional mismatch - if i == len(keys)-1 { - break - } - - warehouseId := input.GetId() - results = append(results, &productv1.Warehouse{ - Id: warehouseId, - Name: fmt.Sprintf("Warehouse %s", warehouseId), - Location: fmt.Sprintf("Location %d", rand.Intn(100)), - }) - } - - return &productv1.LookupWarehouseByIdResponse{ - Result: results, - }, nil -} - -// Helper function to create subcategories for a category -func createSubcategories(categoryId string, kind productv1.CategoryKind, count int) *productv1.ListOfSubcategory { - if count <= 0 { - return &productv1.ListOfSubcategory{ - List: &productv1.ListOfSubcategory_List{ - Items: []*productv1.Subcategory{}, - }, - } - } - - subcategories := make([]*productv1.Subcategory, 0, count) - for j := 1; j <= count; j++ { - subcategories = append(subcategories, &productv1.Subcategory{ - Id: fmt.Sprintf("%s-subcategory-%d", categoryId, j), - Name: fmt.Sprintf("%s Subcategory %d", kind.String(), j), - Description: &wrapperspb.StringValue{Value: fmt.Sprintf("Subcategory %d for %s", j, categoryId)}, - IsActive: true, - }) - } - - return &productv1.ListOfSubcategory{ - List: &productv1.ListOfSubcategory_List{ - Items: subcategories, - }, - } -} - -// Helper functions to convert input types to output types -func convertCategoryInputsToCategories(inputs []*productv1.CategoryInput) []*productv1.Category { - if inputs == nil { - return nil - } - results := make([]*productv1.Category, len(inputs)) - for i, input := range inputs { - results[i] = &productv1.Category{ - Id: fmt.Sprintf("cat-input-%d", i), - Name: input.GetName(), - Kind: input.GetKind(), - Subcategories: createSubcategories(fmt.Sprintf("cat-input-%d", i), input.GetKind(), i+1), - } - } - return results -} - -func convertCategoryInputListToCategories(inputs *productv1.ListOfCategoryInput) []*productv1.Category { - if inputs == nil || inputs.List == nil || inputs.List.Items == nil { - return nil - } - results := make([]*productv1.Category, len(inputs.List.Items)) - for i, input := range inputs.List.Items { - results[i] = &productv1.Category{ - Id: fmt.Sprintf("cat-list-input-%d", i), - Name: input.GetName(), - Kind: input.GetKind(), - Subcategories: createSubcategories(fmt.Sprintf("cat-list-input-%d", i), input.GetKind(), i+1), - } - } - return results -} - -func convertUserInputsToUsers(inputs *productv1.ListOfUserInput) []*productv1.User { - if inputs == nil || inputs.List == nil || inputs.List.Items == nil { - return nil - } - results := make([]*productv1.User, len(inputs.List.Items)) - for i, input := range inputs.List.Items { - results[i] = &productv1.User{ - Id: fmt.Sprintf("user-input-%d", i), - Name: input.GetName(), - } - } - return results -} - -func convertNestedUserInputsToUsers(nestedInputs *productv1.ListOfListOfUserInput) *productv1.ListOfListOfUser { - if nestedInputs == nil || nestedInputs.List == nil { - return &productv1.ListOfListOfUser{ - List: &productv1.ListOfListOfUser_List{ - Items: []*productv1.ListOfUser{}, - }, - } - } - - results := make([]*productv1.ListOfUser, len(nestedInputs.List.Items)) - for i, userList := range nestedInputs.List.Items { - users := make([]*productv1.User, len(userList.List.Items)) - for j, userInput := range userList.List.Items { - users[j] = &productv1.User{ - Id: fmt.Sprintf("nested-user-%d-%d", i, j), - Name: userInput.GetName(), - } - } - results[i] = &productv1.ListOfUser{ - List: &productv1.ListOfUser_List{ - Items: users, - }, - } - } - - return &productv1.ListOfListOfUser{ - List: &productv1.ListOfListOfUser_List{ - Items: results, - }, - } -} - -func convertNestedCategoryInputsToCategories(nestedInputs *productv1.ListOfListOfCategoryInput) *productv1.ListOfListOfCategory { - if nestedInputs == nil || nestedInputs.List == nil { - return &productv1.ListOfListOfCategory{ - List: &productv1.ListOfListOfCategory_List{ - Items: []*productv1.ListOfCategory{}, - }, - } - } - - results := make([]*productv1.ListOfCategory, len(nestedInputs.List.Items)) - for i, categoryList := range nestedInputs.List.Items { - categories := make([]*productv1.Category, len(categoryList.List.Items)) - for j, categoryInput := range categoryList.List.Items { - categories[j] = &productv1.Category{ - Id: fmt.Sprintf("nested-cat-%d-%d", i, j), - Name: categoryInput.GetName(), - Kind: categoryInput.GetKind(), - Subcategories: createSubcategories(fmt.Sprintf("nested-cat-%d-%d", i, j), categoryInput.GetKind(), j+1), - } - } - results[i] = &productv1.ListOfCategory{ - List: &productv1.ListOfCategory_List{ - Items: categories, - }, - } - } - - return &productv1.ListOfListOfCategory{ - List: &productv1.ListOfListOfCategory_List{ - Items: results, - }, - } -} - -// MutationCreateNullableFieldsType implements productv1.ProductServiceServer. -func (s *MockService) MutationCreateNullableFieldsType(ctx context.Context, in *productv1.MutationCreateNullableFieldsTypeRequest) (*productv1.MutationCreateNullableFieldsTypeResponse, error) { - input := in.GetInput() - - // Create a new NullableFieldsType from the input - result := &productv1.NullableFieldsType{ - Id: fmt.Sprintf("nullable-%d", rand.Intn(1000)), - Name: input.GetName(), - RequiredString: input.GetRequiredString(), - RequiredInt: input.GetRequiredInt(), - } - - // Handle optional fields - copy from input if they exist - if input.OptionalString != nil { - result.OptionalString = &wrapperspb.StringValue{Value: input.OptionalString.GetValue()} - } - if input.OptionalInt != nil { - result.OptionalInt = &wrapperspb.Int32Value{Value: input.OptionalInt.GetValue()} - } - if input.OptionalFloat != nil { - result.OptionalFloat = &wrapperspb.DoubleValue{Value: input.OptionalFloat.GetValue()} - } - if input.OptionalBoolean != nil { - result.OptionalBoolean = &wrapperspb.BoolValue{Value: input.OptionalBoolean.GetValue()} - } - - return &productv1.MutationCreateNullableFieldsTypeResponse{ - CreateNullableFieldsType: result, - }, nil -} - -// MutationUpdateNullableFieldsType implements productv1.ProductServiceServer. -func (s *MockService) MutationUpdateNullableFieldsType(ctx context.Context, in *productv1.MutationUpdateNullableFieldsTypeRequest) (*productv1.MutationUpdateNullableFieldsTypeResponse, error) { - id := in.GetId() - input := in.GetInput() - - // Return nil if trying to update a non-existent ID - if id == "non-existent" { - return &productv1.MutationUpdateNullableFieldsTypeResponse{ - UpdateNullableFieldsType: nil, - }, nil - } - - // Create updated NullableFieldsType - result := &productv1.NullableFieldsType{ - Id: id, - Name: input.GetName(), - RequiredString: input.GetRequiredString(), - RequiredInt: input.GetRequiredInt(), - } - - // Handle optional fields - copy from input if they exist - if input.OptionalString != nil { - result.OptionalString = &wrapperspb.StringValue{Value: input.OptionalString.GetValue()} - } - if input.OptionalInt != nil { - result.OptionalInt = &wrapperspb.Int32Value{Value: input.OptionalInt.GetValue()} - } - if input.OptionalFloat != nil { - result.OptionalFloat = &wrapperspb.DoubleValue{Value: input.OptionalFloat.GetValue()} - } - if input.OptionalBoolean != nil { - result.OptionalBoolean = &wrapperspb.BoolValue{Value: input.OptionalBoolean.GetValue()} - } - - return &productv1.MutationUpdateNullableFieldsTypeResponse{ - UpdateNullableFieldsType: result, - }, nil -} - -// QueryAllNullableFieldsTypes implements productv1.ProductServiceServer. -func (s *MockService) QueryAllNullableFieldsTypes(ctx context.Context, in *productv1.QueryAllNullableFieldsTypesRequest) (*productv1.QueryAllNullableFieldsTypesResponse, error) { - var results []*productv1.NullableFieldsType - - // Create a variety of test data with different nullable field combinations - - // Entry 1: All fields populated - results = append(results, &productv1.NullableFieldsType{ - Id: "nullable-1", - Name: "Full Data Entry", - OptionalString: &wrapperspb.StringValue{Value: "Optional String Value"}, - OptionalInt: &wrapperspb.Int32Value{Value: 42}, - OptionalFloat: &wrapperspb.DoubleValue{Value: math.MaxFloat64}, - OptionalBoolean: &wrapperspb.BoolValue{Value: true}, - RequiredString: "Required String 1", - RequiredInt: 100, - }) - - // Entry 2: Some nullable fields are null - results = append(results, &productv1.NullableFieldsType{ - Id: "nullable-2", - Name: "Partial Data Entry", - OptionalString: &wrapperspb.StringValue{Value: "Only string is set"}, - OptionalInt: nil, // null - OptionalFloat: nil, // null - OptionalBoolean: &wrapperspb.BoolValue{Value: false}, - RequiredString: "Required String 2", - RequiredInt: 200, - }) - - // Entry 3: All nullable fields are null - results = append(results, &productv1.NullableFieldsType{ - Id: "nullable-3", - Name: "Minimal Data Entry", - OptionalString: nil, // null - OptionalInt: nil, // null - OptionalFloat: nil, // null - OptionalBoolean: nil, // null - RequiredString: "Required String 3", - RequiredInt: 300, - }) - - return &productv1.QueryAllNullableFieldsTypesResponse{ - AllNullableFieldsTypes: results, - }, nil -} - -// QueryNullableFieldsType implements productv1.ProductServiceServer. -func (s *MockService) QueryNullableFieldsType(ctx context.Context, in *productv1.QueryNullableFieldsTypeRequest) (*productv1.QueryNullableFieldsTypeResponse, error) { - // Return a single NullableFieldsType with mixed null/non-null values - result := &productv1.NullableFieldsType{ - Id: "nullable-default", - Name: "Default Nullable Fields Type", - OptionalString: &wrapperspb.StringValue{Value: "Default optional string"}, - OptionalInt: &wrapperspb.Int32Value{Value: 777}, - OptionalFloat: nil, // null - OptionalBoolean: &wrapperspb.BoolValue{Value: true}, - RequiredString: "Default required string", - RequiredInt: 999, - } - - return &productv1.QueryNullableFieldsTypeResponse{ - NullableFieldsType: result, - }, nil -} - -// QueryNullableFieldsTypeById implements productv1.ProductServiceServer. -func (s *MockService) QueryNullableFieldsTypeById(ctx context.Context, in *productv1.QueryNullableFieldsTypeByIdRequest) (*productv1.QueryNullableFieldsTypeByIdResponse, error) { - id := in.GetId() - - // Return null for specific test IDs - if id == "not-found" || id == "null-test" { - return &productv1.QueryNullableFieldsTypeByIdResponse{ - NullableFieldsTypeById: nil, - }, nil - } - - // Create different test data based on ID - var result *productv1.NullableFieldsType - - switch id { - case "full-data": - result = &productv1.NullableFieldsType{ - Id: id, - Name: "Full Data by ID", - OptionalString: &wrapperspb.StringValue{Value: "All fields populated"}, - OptionalInt: &wrapperspb.Int32Value{Value: 123}, - OptionalFloat: &wrapperspb.DoubleValue{Value: 12.34}, - OptionalBoolean: &wrapperspb.BoolValue{Value: false}, - RequiredString: "Required by ID", - RequiredInt: 456, - } - case "partial-data": - result = &productv1.NullableFieldsType{ - Id: id, - Name: "Partial Data by ID", - OptionalString: nil, // null - OptionalInt: &wrapperspb.Int32Value{Value: 789}, - OptionalFloat: nil, // null - OptionalBoolean: &wrapperspb.BoolValue{Value: true}, - RequiredString: "Partial required by ID", - RequiredInt: 321, - } - case "minimal-data": - result = &productv1.NullableFieldsType{ - Id: id, - Name: "Minimal Data by ID", - OptionalString: nil, // null - OptionalInt: nil, // null - OptionalFloat: nil, // null - OptionalBoolean: nil, // null - RequiredString: "Only required fields", - RequiredInt: 111, - } - default: - // Generic response for any other ID - result = &productv1.NullableFieldsType{ - Id: id, - Name: fmt.Sprintf("Nullable Type %s", id), - OptionalString: &wrapperspb.StringValue{Value: fmt.Sprintf("Optional for %s", id)}, - OptionalInt: &wrapperspb.Int32Value{Value: int32(len(id) * 10)}, - OptionalFloat: &wrapperspb.DoubleValue{Value: float64(len(id)) * 1.5}, - OptionalBoolean: &wrapperspb.BoolValue{Value: len(id)%2 == 0}, - RequiredString: fmt.Sprintf("Required for %s", id), - RequiredInt: int32(len(id) * 100), - } - } - - return &productv1.QueryNullableFieldsTypeByIdResponse{ - NullableFieldsTypeById: result, - }, nil -} - -// QueryNullableFieldsTypeWithFilter implements productv1.ProductServiceServer. -func (s *MockService) QueryNullableFieldsTypeWithFilter(ctx context.Context, in *productv1.QueryNullableFieldsTypeWithFilterRequest) (*productv1.QueryNullableFieldsTypeWithFilterResponse, error) { - filter := in.GetFilter() - var results []*productv1.NullableFieldsType - - // If no filter provided, return empty results - if filter == nil { - return &productv1.QueryNullableFieldsTypeWithFilterResponse{ - NullableFieldsTypeWithFilter: results, - }, nil - } - - // Create test data based on filter criteria - nameFilter := "" - if filter.Name != nil { - nameFilter = filter.Name.GetValue() - } - - optionalStringFilter := "" - if filter.OptionalString != nil { - optionalStringFilter = filter.OptionalString.GetValue() - } - - includeNulls := false - if filter.IncludeNulls != nil { - includeNulls = filter.IncludeNulls.GetValue() - } - - // Generate filtered results - for i := 1; i <= 3; i++ { - var optionalString *wrapperspb.StringValue - var optionalInt *wrapperspb.Int32Value - var optionalFloat *wrapperspb.DoubleValue - var optionalBoolean *wrapperspb.BoolValue - - // Vary the nullable fields based on includeNulls and index - if includeNulls || i%2 == 1 { - if optionalStringFilter != "" { - optionalString = &wrapperspb.StringValue{Value: optionalStringFilter} - } else { - optionalString = &wrapperspb.StringValue{Value: fmt.Sprintf("Filtered string %d", i)} - } - } - - if includeNulls || i%3 != 0 { - optionalInt = &wrapperspb.Int32Value{Value: int32(i * 100)} - } - - if includeNulls || i%2 == 0 { - optionalFloat = &wrapperspb.DoubleValue{Value: float64(i) * 10.5} - } - - if includeNulls || i%4 != 0 { - optionalBoolean = &wrapperspb.BoolValue{Value: i%2 == 0} - } - - name := fmt.Sprintf("Filtered Item %d", i) - if nameFilter != "" { - name = fmt.Sprintf("%s - %d", nameFilter, i) - } - - results = append(results, &productv1.NullableFieldsType{ - Id: fmt.Sprintf("filtered-%d", i), - Name: name, - OptionalString: optionalString, - OptionalInt: optionalInt, - OptionalFloat: optionalFloat, - OptionalBoolean: optionalBoolean, - RequiredString: fmt.Sprintf("Required filtered %d", i), - RequiredInt: int32(i * 1000), - }) - } - - return &productv1.QueryNullableFieldsTypeWithFilterResponse{ - NullableFieldsTypeWithFilter: results, - }, nil -} - -// MutationPerformAction implements productv1.ProductServiceServer. -func (s *MockService) MutationPerformAction(ctx context.Context, in *productv1.MutationPerformActionRequest) (*productv1.MutationPerformActionResponse, error) { - input := in.GetInput() - actionType := input.GetType() - - // Simulate different action results based on the action type - var result *productv1.ActionResult - - switch actionType { - case "error_action": - // Return an error result - result = &productv1.ActionResult{ - Value: &productv1.ActionResult_ActionError{ - ActionError: &productv1.ActionError{ - Message: "Action failed due to validation error", - Code: "VALIDATION_ERROR", - }, - }, - } - case "invalid_action": - // Return a different error result - result = &productv1.ActionResult{ - Value: &productv1.ActionResult_ActionError{ - ActionError: &productv1.ActionError{ - Message: "Invalid action type provided", - Code: "INVALID_ACTION", - }, - }, - } - default: - // Return a success result - result = &productv1.ActionResult{ - Value: &productv1.ActionResult_ActionSuccess{ - ActionSuccess: &productv1.ActionSuccess{ - Message: fmt.Sprintf("Action '%s' completed successfully", actionType), - Timestamp: "2024-01-01T00:00:00Z", - }, - }, - } - } - - return &productv1.MutationPerformActionResponse{ - PerformAction: result, - }, nil -} - -// QueryRandomSearchResult implements productv1.ProductServiceServer. -func (s *MockService) QueryRandomSearchResult(ctx context.Context, in *productv1.QueryRandomSearchResultRequest) (*productv1.QueryRandomSearchResultResponse, error) { - // Randomly return one of the three union types - var result *productv1.SearchResult - - switch rand.Intn(3) { - case 0: - // Return a Product - result = &productv1.SearchResult{ - Value: &productv1.SearchResult_Product{ - Product: &productv1.Product{ - Id: "product-random-1", - Name: "Random Product", - Price: 29.99, - }, - }, - } - case 1: - // Return a User - result = &productv1.SearchResult{ - Value: &productv1.SearchResult_User{ - User: &productv1.User{ - Id: "user-random-1", - Name: "Random User", - }, - }, - } - default: - // Return a Category - kind := productv1.CategoryKind_CATEGORY_KIND_ELECTRONICS - result = &productv1.SearchResult{ - Value: &productv1.SearchResult_Category{ - Category: &productv1.Category{ - Id: "category-random-1", - Name: "Random Category", - Kind: kind, - Subcategories: createSubcategories("category-random-1", kind, 2), - }, - }, - } - } - - return &productv1.QueryRandomSearchResultResponse{ - RandomSearchResult: result, - }, nil -} - -// QuerySearch implements productv1.ProductServiceServer. -func (s *MockService) QuerySearch(ctx context.Context, in *productv1.QuerySearchRequest) (*productv1.QuerySearchResponse, error) { - input := in.GetInput() - query := input.GetQuery() - limit := input.GetLimit() - - // Default limit if not specified - if limit.GetValue() <= 0 { - limit = &wrapperspb.Int32Value{Value: 10} - } - - var results []*productv1.SearchResult - - // Generate a mix of different union types based on the query - for i := int32(0); i < limit.GetValue() && i < 6; i++ { // Cap at 6 results for testing - switch i % 3 { - case 0: - // Add a Product - results = append(results, &productv1.SearchResult{ - Value: &productv1.SearchResult_Product{ - Product: &productv1.Product{ - Id: fmt.Sprintf("product-search-%d", i+1), - Name: fmt.Sprintf("Product matching '%s' #%d", query, i+1), - Price: float64(10 + i*5), - }, - }, - }) - case 1: - // Add a User - results = append(results, &productv1.SearchResult{ - Value: &productv1.SearchResult_User{ - User: &productv1.User{ - Id: fmt.Sprintf("user-search-%d", i+1), - Name: fmt.Sprintf("User matching '%s' #%d", query, i+1), - }, - }, - }) - case 2: - // Add a Category - kinds := []productv1.CategoryKind{ - productv1.CategoryKind_CATEGORY_KIND_BOOK, - productv1.CategoryKind_CATEGORY_KIND_ELECTRONICS, - productv1.CategoryKind_CATEGORY_KIND_FURNITURE, - } - kind := kinds[i%int32(len(kinds))] - results = append(results, &productv1.SearchResult{ - Value: &productv1.SearchResult_Category{ - Category: &productv1.Category{ - Id: fmt.Sprintf("category-search-%d", i+1), - Name: fmt.Sprintf("Category matching '%s' #%d", query, i+1), - Kind: kind, - Subcategories: createSubcategories(fmt.Sprintf("category-search-%d", i+1), kind, int(i%3)+1), - }, - }, - }) - } - } - - return &productv1.QuerySearchResponse{ - Search: results, - }, nil -} - -func (s *MockService) LookupProductById(ctx context.Context, in *productv1.LookupProductByIdRequest) (*productv1.LookupProductByIdResponse, error) { - var results []*productv1.Product - - for _, input := range in.GetKeys() { - productId := input.GetId() - results = append(results, &productv1.Product{ - Id: productId, - Name: fmt.Sprintf("Product %s", productId), - Price: 99.99, - }) - } - - return &productv1.LookupProductByIdResponse{ - Result: results, - }, nil -} - -func (s *MockService) LookupStorageById(ctx context.Context, in *productv1.LookupStorageByIdRequest) (*productv1.LookupStorageByIdResponse, error) { - var results []*productv1.Storage - - for _, input := range in.GetKeys() { - storageId := input.GetId() - results = append(results, &productv1.Storage{ - Id: storageId, - Name: fmt.Sprintf("Storage %s", storageId), - Location: fmt.Sprintf("Location %d", rand.Intn(100)), - }) - } - - return &productv1.LookupStorageByIdResponse{ - Result: results, - }, nil -} - -func (s *MockService) QueryUsers(ctx context.Context, in *productv1.QueryUsersRequest) (*productv1.QueryUsersResponse, error) { - var results []*productv1.User - - // Generate 3 random users - for i := 1; i <= 3; i++ { - results = append(results, &productv1.User{ - Id: fmt.Sprintf("user-%d", i), - Name: fmt.Sprintf("User %d", i), - }) - } - - return &productv1.QueryUsersResponse{ - Users: results, - }, nil -} - -func (s *MockService) QueryUser(ctx context.Context, in *productv1.QueryUserRequest) (*productv1.QueryUserResponse, error) { - userId := in.GetId() - - // Return a gRPC status error for a specific test case - if userId == "error-user" { - return nil, status.Errorf(codes.NotFound, "user not found: %s", userId) - } - - return &productv1.QueryUserResponse{ - User: &productv1.User{ - Id: userId, - Name: fmt.Sprintf("User %s", userId), - }, - }, nil -} - -func (s *MockService) QueryNestedType(ctx context.Context, in *productv1.QueryNestedTypeRequest) (*productv1.QueryNestedTypeResponse, error) { - var nestedTypes []*productv1.NestedTypeA - - // Generate 2 nested types - for i := 1; i <= 2; i++ { - nestedTypes = append(nestedTypes, &productv1.NestedTypeA{ - Id: fmt.Sprintf("nested-a-%d", i), - Name: fmt.Sprintf("Nested A %d", i), - B: &productv1.NestedTypeB{ - Id: fmt.Sprintf("nested-b-%d", i), - Name: fmt.Sprintf("Nested B %d", i), - C: &productv1.NestedTypeC{ - Id: fmt.Sprintf("nested-c-%d", i), - Name: fmt.Sprintf("Nested C %d", i), - }, - }, - }) - } - - return &productv1.QueryNestedTypeResponse{ - NestedType: nestedTypes, - }, nil -} - -func (s *MockService) QueryRecursiveType(ctx context.Context, in *productv1.QueryRecursiveTypeRequest) (*productv1.QueryRecursiveTypeResponse, error) { - // Create a recursive structure 3 levels deep - recursiveType := &productv1.RecursiveType{ - Id: "recursive-1", - Name: "Level 1", - RecursiveType: &productv1.RecursiveType{ - Id: "recursive-2", - Name: "Level 2", - RecursiveType: &productv1.RecursiveType{ - Id: "recursive-3", - Name: "Level 3", - }, - }, - } - - return &productv1.QueryRecursiveTypeResponse{ - RecursiveType: recursiveType, - }, nil -} - -func (s *MockService) QueryTypeFilterWithArguments(ctx context.Context, in *productv1.QueryTypeFilterWithArgumentsRequest) (*productv1.QueryTypeFilterWithArgumentsResponse, error) { - filterField1 := in.GetFilterField_1() - filterField2 := in.GetFilterField_2() - - var fields []*productv1.TypeWithMultipleFilterFields - - // Create results that echo the filter values - for i := 1; i <= 2; i++ { - fields = append(fields, &productv1.TypeWithMultipleFilterFields{ - Id: fmt.Sprintf("multi-filter-%d", i), - Name: fmt.Sprintf("MultiFilter %d", i), - FilterField_1: filterField1, - FilterField_2: filterField2, - }) - } - - return &productv1.QueryTypeFilterWithArgumentsResponse{ - TypeFilterWithArguments: fields, - }, nil -} - -func (s *MockService) QueryTypeWithMultipleFilterFields(ctx context.Context, in *productv1.QueryTypeWithMultipleFilterFieldsRequest) (*productv1.QueryTypeWithMultipleFilterFieldsResponse, error) { - filter := in.GetFilter() - - var fields []*productv1.TypeWithMultipleFilterFields - - // Echo the filter values in the results - for i := 1; i <= 2; i++ { - fields = append(fields, &productv1.TypeWithMultipleFilterFields{ - Id: fmt.Sprintf("filtered-%d", i), - Name: "Filter: " + strconv.Itoa(i), - FilterField_1: filter.FilterField_1, - FilterField_2: filter.FilterField_2, - }) - } - - return &productv1.QueryTypeWithMultipleFilterFieldsResponse{ - TypeWithMultipleFilterFields: fields, - }, nil -} - -func (s *MockService) QueryComplexFilterType(ctx context.Context, in *productv1.QueryComplexFilterTypeRequest) (*productv1.QueryComplexFilterTypeResponse, error) { - filter := in.GetFilter() - - var name string - if filter != nil && filter.GetFilter() != nil { - name = filter.GetFilter().GetName() - } else { - name = "Default Product" - } - - return &productv1.QueryComplexFilterTypeResponse{ - ComplexFilterType: []*productv1.TypeWithComplexFilterInput{ - { - Id: "test-id-123", - Name: name, - }, - }, - }, nil -} - -func (s *MockService) QueryRandomPet(ctx context.Context, in *productv1.QueryRandomPetRequest) (*productv1.QueryRandomPetResponse, error) { - // Create either a cat or dog randomly - var pet *productv1.Animal - - // Random choice between cat and dog - if rand.Intn(2) == 0 { - // Create a cat - pet = &productv1.Animal{ - Instance: &productv1.Animal_Cat{ - Cat: &productv1.Cat{ - Id: "cat-1", - Name: "Whiskers", - Kind: "Siamese", - MeowVolume: int32(rand.Intn(10) + 1), // Random volume between 1-10 - }, - }, - } - } else { - // Create a dog - pet = &productv1.Animal{ - Instance: &productv1.Animal_Dog{ - Dog: &productv1.Dog{ - Id: "dog-1", - Name: "Spot", - Kind: "Dalmatian", - BarkVolume: int32(rand.Intn(10) + 1), // Random volume between 1-10 - }, - }, - } - } - - return &productv1.QueryRandomPetResponse{ - RandomPet: pet, - }, nil -} - -func (s *MockService) QueryAllPets(ctx context.Context, in *productv1.QueryAllPetsRequest) (*productv1.QueryAllPetsResponse, error) { - // Create a mix of cats and dogs - var pets []*productv1.Animal - - // Add 2 cats - for i := 1; i <= 2; i++ { - pets = append(pets, &productv1.Animal{ - Instance: &productv1.Animal_Cat{ - Cat: &productv1.Cat{ - Id: fmt.Sprintf("cat-%d", i), - Name: fmt.Sprintf("Cat %d", i), - Kind: fmt.Sprintf("Breed %d", i), - MeowVolume: int32(i + 3), // Different volumes - }, - }, - }) - } - - // Add 2 dogs - for i := 1; i <= 2; i++ { - pets = append(pets, &productv1.Animal{ - Instance: &productv1.Animal_Dog{ - Dog: &productv1.Dog{ - Id: fmt.Sprintf("dog-%d", i), - Name: fmt.Sprintf("Dog %d", i), - Kind: fmt.Sprintf("Breed %d", i), - BarkVolume: int32(i + 5), // Different volumes - }, - }, - }) - } - - return &productv1.QueryAllPetsResponse{ - AllPets: pets, - }, nil -} - -// Implementation for QueryCategories -func (s *MockService) QueryCategories(ctx context.Context, in *productv1.QueryCategoriesRequest) (*productv1.QueryCategoriesResponse, error) { - // Generate a list of categories - var categories []*productv1.Category - - // Create sample categories for each CategoryKind - categoryKinds := []productv1.CategoryKind{ - productv1.CategoryKind_CATEGORY_KIND_BOOK, - productv1.CategoryKind_CATEGORY_KIND_ELECTRONICS, - productv1.CategoryKind_CATEGORY_KIND_FURNITURE, - productv1.CategoryKind_CATEGORY_KIND_OTHER, - } - - for i, kind := range categoryKinds { - categories = append(categories, &productv1.Category{ - Id: fmt.Sprintf("category-%d", i+1), - Name: fmt.Sprintf("%s Category", kind.String()), - Kind: kind, - Subcategories: createSubcategories(fmt.Sprintf("category-%d", i+1), kind, i+1), - }) - } - - return &productv1.QueryCategoriesResponse{ - Categories: categories, - }, nil -} - -// Implementation for QueryCategoriesByKind -func (s *MockService) QueryCategoriesByKind(ctx context.Context, in *productv1.QueryCategoriesByKindRequest) (*productv1.QueryCategoriesByKindResponse, error) { - kind := in.GetKind() - - // Generate categories for the specified kind - var categories []*productv1.Category - - // Create 3 categories of the requested kind - for i := 1; i <= 3; i++ { - - subcategoties := make([]*productv1.Subcategory, 0, i) - for j := 1; j <= i; j++ { - subcategoties = append(subcategoties, &productv1.Subcategory{ - Id: fmt.Sprintf("%s-subcategory-%d", kind.String(), j), - Name: fmt.Sprintf("%s Subcategory %d", kind.String(), j), - Description: &wrapperspb.StringValue{Value: fmt.Sprintf("%s Subcategory %d", kind.String(), j)}, - IsActive: true, - }) - } - - categories = append(categories, &productv1.Category{ - Id: fmt.Sprintf("%s-category-%d", kind.String(), i), - Name: fmt.Sprintf("%s Category %d", kind.String(), i), - Kind: kind, - Subcategories: &productv1.ListOfSubcategory{ - List: &productv1.ListOfSubcategory_List{ - Items: subcategoties, - }, - }, - }) - } - - return &productv1.QueryCategoriesByKindResponse{ - CategoriesByKind: categories, - }, nil -} - -func (s *MockService) QueryCategoriesByKinds(ctx context.Context, in *productv1.QueryCategoriesByKindsRequest) (*productv1.QueryCategoriesByKindsResponse, error) { - kinds := in.GetKinds() - - var categories []*productv1.Category - - for i, kind := range kinds { - categories = append(categories, &productv1.Category{ - Id: fmt.Sprintf("%s-category-%d", kind.String(), i), - Name: fmt.Sprintf("%s Category %d", kind.String(), i), - Kind: kind, - Subcategories: createSubcategories(fmt.Sprintf("%s-category-%d", kind.String(), i), kind, i+1), - }) - } - - return &productv1.QueryCategoriesByKindsResponse{ - CategoriesByKinds: categories, - }, nil -} - -// Implementation for QueryFilterCategories -func (s *MockService) QueryFilterCategories(ctx context.Context, in *productv1.QueryFilterCategoriesRequest) (*productv1.QueryFilterCategoriesResponse, error) { - filter := in.GetFilter() - - if filter == nil { - return &productv1.QueryFilterCategoriesResponse{ - FilterCategories: []*productv1.Category{}, - }, nil - } - - kind := filter.GetCategory() - - // Generate filtered categories - var categories []*productv1.Category - - // Create categories that match the filter - for i := 1; i <= 5; i++ { - categories = append(categories, &productv1.Category{ - Id: fmt.Sprintf("filtered-%s-category-%d", kind.String(), i), - Name: fmt.Sprintf("Filtered %s Category %d", kind.String(), i), - Kind: kind, - Subcategories: createSubcategories(fmt.Sprintf("filtered-%s-category-%d", kind.String(), i), kind, i), - }) - } - - // Apply pagination if provided - pagination := filter.GetPagination() - if pagination != nil { - page := int(pagination.GetPage()) - perPage := int(pagination.GetPerPage()) - - if page > 0 && perPage > 0 && len(categories) > perPage { - startIdx := (page - 1) * perPage - endIdx := startIdx + perPage - - if startIdx < len(categories) { - if endIdx > len(categories) { - endIdx = len(categories) - } - categories = categories[startIdx:endIdx] - } else { - categories = []*productv1.Category{} - } - } - } - - return &productv1.QueryFilterCategoriesResponse{ - FilterCategories: categories, - }, nil -} - -// Implementation for CreateUser mutation -func (s *MockService) MutationCreateUser(ctx context.Context, in *productv1.MutationCreateUserRequest) (*productv1.MutationCreateUserResponse, error) { - input := in.GetInput() - - // Create a new user with the input name and a random ID - user := &productv1.User{ - Id: fmt.Sprintf("user-%d", rand.Intn(1000)), - Name: input.GetName(), - } - - return &productv1.MutationCreateUserResponse{ - CreateUser: user, - }, nil -} - -// Implementation for QueryCalculateTotals -func (s *MockService) QueryCalculateTotals(ctx context.Context, in *productv1.QueryCalculateTotalsRequest) (*productv1.QueryCalculateTotalsResponse, error) { - orders := in.GetOrders() - var calculatedOrders []*productv1.Order - - for _, orderInput := range orders { - // Calculate total items by summing up quantities from all order lines - var totalItems int32 - for _, line := range orderInput.GetLines() { - totalItems += line.GetQuantity() - } - - orderLines := []*productv1.OrderLine{} - for _, line := range orderInput.GetLines() { - orderLines = append(orderLines, &productv1.OrderLine{ - ProductId: line.GetProductId(), - Quantity: line.GetQuantity(), - Modifiers: line.GetModifiers(), - }) - } - - calculatedOrders = append(calculatedOrders, &productv1.Order{ - OrderId: orderInput.GetOrderId(), - CustomerName: orderInput.GetCustomerName(), - TotalItems: totalItems, - OrderLines: &productv1.ListOfOrderLine{ - List: &productv1.ListOfOrderLine_List{ - Items: orderLines, - }, - }, - }) - } - - return &productv1.QueryCalculateTotalsResponse{ - CalculateTotals: calculatedOrders, - }, nil -} - -// BlogPost query implementations -func (s *MockService) QueryBlogPost(ctx context.Context, in *productv1.QueryBlogPostRequest) (*productv1.QueryBlogPostResponse, error) { - // Return a default blog post with comprehensive list examples - result := &productv1.BlogPost{ - Id: "blog-default", - Title: "Default Blog Post", - Content: "This is a sample blog post content for testing nested lists.", - Tags: []string{"tech", "programming", "go"}, - OptionalTags: &productv1.ListOfString{ - List: &productv1.ListOfString_List{ - Items: []string{"optional1", "optional2"}, - }, - }, - Categories: []string{"Technology", "", "Programming"}, // includes null/empty - Keywords: &productv1.ListOfString{ - List: &productv1.ListOfString_List{ - Items: []string{"keyword1", "keyword2"}, - }, - }, - ViewCounts: []int32{100, 150, 200, 250}, - Ratings: &productv1.ListOfFloat{ - List: &productv1.ListOfFloat_List{ - Items: []float64{4.5, 3.8, 5.0}, - }, - }, - IsPublished: &productv1.ListOfBoolean{ - List: &productv1.ListOfBoolean_List{ - Items: []bool{false, true, true}, - }, - }, - TagGroups: &productv1.ListOfListOfString{ - List: &productv1.ListOfListOfString_List{ - Items: []*productv1.ListOfString{ - {List: &productv1.ListOfString_List{ - Items: []string{"tech", "programming"}, - }}, - {List: &productv1.ListOfString_List{ - Items: []string{"golang", "backend"}, - }}, - }, - }, - }, - RelatedTopics: &productv1.ListOfListOfString{ - List: &productv1.ListOfListOfString_List{ - Items: []*productv1.ListOfString{ - {List: &productv1.ListOfString_List{Items: []string{"microservices", "api"}}}, - {List: &productv1.ListOfString_List{Items: []string{"databases", "performance"}}}, - }, - }, - }, - CommentThreads: &productv1.ListOfListOfString{ - List: &productv1.ListOfListOfString_List{ - Items: []*productv1.ListOfString{ - {List: &productv1.ListOfString_List{Items: []string{"Great post!", "Very helpful"}}}, - {List: &productv1.ListOfString_List{Items: []string{"Could use more examples", "Thanks for sharing"}}}, - }, - }, - }, - Suggestions: &productv1.ListOfListOfString{ - List: &productv1.ListOfListOfString_List{ - Items: []*productv1.ListOfString{ - {List: &productv1.ListOfString_List{Items: []string{"Add code examples", "Include diagrams"}}}, - }, - }, - }, - RelatedCategories: []*productv1.Category{ - {Id: "cat-1", Name: "Technology", Kind: productv1.CategoryKind_CATEGORY_KIND_ELECTRONICS}, - {Id: "cat-2", Name: "Programming", Kind: productv1.CategoryKind_CATEGORY_KIND_BOOK}, - }, - Contributors: []*productv1.User{ - {Id: "user-1", Name: "John Doe"}, - {Id: "user-2", Name: "Jane Smith"}, - }, - MentionedProducts: &productv1.ListOfProduct{ - List: &productv1.ListOfProduct_List{ - Items: []*productv1.Product{ - {Id: "prod-1", Name: "Sample Product", Price: 99.99}, - }, - }, - }, - MentionedUsers: &productv1.ListOfUser{ - List: &productv1.ListOfUser_List{ - Items: []*productv1.User{ - {Id: "user-3", Name: "Bob Johnson"}, - }, - }, - }, - CategoryGroups: &productv1.ListOfListOfCategory{ - List: &productv1.ListOfListOfCategory_List{ - Items: []*productv1.ListOfCategory{ - {List: &productv1.ListOfCategory_List{ - Items: []*productv1.Category{ - {Id: "cat-3", Name: "Web Development", Kind: productv1.CategoryKind_CATEGORY_KIND_ELECTRONICS}, - {Id: "cat-4", Name: "Backend", Kind: productv1.CategoryKind_CATEGORY_KIND_ELECTRONICS}, - }, - }}, - }, - }, - }, - ContributorTeams: &productv1.ListOfListOfUser{ - List: &productv1.ListOfListOfUser_List{ - Items: []*productv1.ListOfUser{ - {List: &productv1.ListOfUser_List{ - Items: []*productv1.User{ - {Id: "user-4", Name: "Alice Brown"}, - {Id: "user-5", Name: "Charlie Wilson"}, - }, - }}, - }, - }, - }, - } - - return &productv1.QueryBlogPostResponse{ - BlogPost: result, - }, nil -} - -func (s *MockService) QueryBlogPostById(ctx context.Context, in *productv1.QueryBlogPostByIdRequest) (*productv1.QueryBlogPostByIdResponse, error) { - id := in.GetId() - - // Return null for specific test IDs - if id == "not-found" { - return &productv1.QueryBlogPostByIdResponse{ - BlogPostById: nil, - }, nil - } - - // Create different test data based on ID - var result *productv1.BlogPost - - switch id { - case "simple": - result = &productv1.BlogPost{ - Id: id, - Title: "Simple Post", - Content: "Simple content", - Tags: []string{"simple"}, - Categories: []string{"Basic"}, - ViewCounts: []int32{10}, - // Required nested lists must have data - TagGroups: &productv1.ListOfListOfString{ - List: &productv1.ListOfListOfString_List{ - Items: []*productv1.ListOfString{ - {List: &productv1.ListOfString_List{Items: []string{"simple"}}}, - }, - }, - }, - RelatedTopics: &productv1.ListOfListOfString{ - List: &productv1.ListOfListOfString_List{ - Items: []*productv1.ListOfString{ - {List: &productv1.ListOfString_List{Items: []string{"basic"}}}, - }, - }, - }, - CommentThreads: &productv1.ListOfListOfString{ - List: &productv1.ListOfListOfString_List{ - Items: []*productv1.ListOfString{ - {List: &productv1.ListOfString_List{Items: []string{"Nice post"}}}, - }, - }, - }, - // Required complex lists must have data - RelatedCategories: []*productv1.Category{ - {Id: "cat-simple", Name: "Basic", Kind: productv1.CategoryKind_CATEGORY_KIND_OTHER}, - }, - Contributors: []*productv1.User{ - {Id: "user-simple", Name: "Simple Author"}, - }, - CategoryGroups: &productv1.ListOfListOfCategory{ - List: &productv1.ListOfListOfCategory_List{ - Items: []*productv1.ListOfCategory{ - {List: &productv1.ListOfCategory_List{ - Items: []*productv1.Category{ - {Id: "cat-group-simple", Name: "Simple Category", Kind: productv1.CategoryKind_CATEGORY_KIND_OTHER}, - }, - }}, - }, - }, - }, - } - case "complex": - result = &productv1.BlogPost{ - Id: id, - Title: "Complex Blog Post", - Content: "Complex content with comprehensive lists", - Tags: []string{"complex", "advanced", "detailed"}, - OptionalTags: &productv1.ListOfString{ - List: &productv1.ListOfString_List{ - Items: []string{"deep-dive", "tutorial"}, - }, - }, - Categories: []string{"Advanced", "Tutorial", "Guide"}, - Keywords: &productv1.ListOfString{ - List: &productv1.ListOfString_List{ - Items: []string{"advanced", "complex", "comprehensive"}, - }, - }, - ViewCounts: []int32{500, 600, 750, 800, 950}, - Ratings: &productv1.ListOfFloat{ - List: &productv1.ListOfFloat_List{ - Items: []float64{4.8, 4.9, 4.7, 5.0}, - }, - }, - IsPublished: &productv1.ListOfBoolean{ - List: &productv1.ListOfBoolean_List{ - Items: []bool{false, false, true, true}, - }, - }, - TagGroups: &productv1.ListOfListOfString{ - List: &productv1.ListOfListOfString_List{ - Items: []*productv1.ListOfString{ - {List: &productv1.ListOfString_List{Items: []string{"advanced", "expert"}}}, - {List: &productv1.ListOfString_List{Items: []string{"tutorial", "guide", "comprehensive"}}}, - {List: &productv1.ListOfString_List{Items: []string{"deep-dive", "detailed"}}}, - }, - }, - }, - RelatedTopics: &productv1.ListOfListOfString{ - List: &productv1.ListOfListOfString_List{ - Items: []*productv1.ListOfString{ - {List: &productv1.ListOfString_List{Items: []string{"architecture", "patterns", "design"}}}, - {List: &productv1.ListOfString_List{Items: []string{"optimization", "performance", "scaling"}}}, - }, - }, - }, - CommentThreads: &productv1.ListOfListOfString{ - List: &productv1.ListOfListOfString_List{ - Items: []*productv1.ListOfString{ - {List: &productv1.ListOfString_List{Items: []string{"Excellent deep dive!", "Very thorough"}}}, - {List: &productv1.ListOfString_List{Items: []string{"Could be longer", "More examples please"}}}, - {List: &productv1.ListOfString_List{Items: []string{"Best tutorial I've read", "Thank you!"}}}, - }, - }, - }, - Suggestions: &productv1.ListOfListOfString{ - List: &productv1.ListOfListOfString_List{ - Items: []*productv1.ListOfString{ - {List: &productv1.ListOfString_List{Items: []string{"Add video content", "Include interactive examples"}}}, - {List: &productv1.ListOfString_List{Items: []string{"Create follow-up posts", "Add Q&A section"}}}, - }, - }, - }, - // Complex example includes all new complex list fields - RelatedCategories: []*productv1.Category{ - {Id: "cat-complex-1", Name: "Advanced Programming", Kind: productv1.CategoryKind_CATEGORY_KIND_ELECTRONICS}, - {Id: "cat-complex-2", Name: "Software Architecture", Kind: productv1.CategoryKind_CATEGORY_KIND_BOOK}, - }, - Contributors: []*productv1.User{ - {Id: "user-complex-1", Name: "Expert Author"}, - {Id: "user-complex-2", Name: "Technical Reviewer"}, - }, - MentionedProducts: &productv1.ListOfProduct{ - List: &productv1.ListOfProduct_List{ - Items: []*productv1.Product{ - {Id: "prod-complex-1", Name: "Advanced IDE", Price: 299.99}, - {Id: "prod-complex-2", Name: "Profiling Tool", Price: 149.99}, - }, - }, - }, - MentionedUsers: &productv1.ListOfUser{ - List: &productv1.ListOfUser_List{ - Items: []*productv1.User{ - {Id: "user-complex-3", Name: "Referenced Expert"}, - }, - }, - }, - CategoryGroups: &productv1.ListOfListOfCategory{ - List: &productv1.ListOfListOfCategory_List{ - Items: []*productv1.ListOfCategory{ - {List: &productv1.ListOfCategory_List{ - Items: []*productv1.Category{ - {Id: "cat-group-1", Name: "System Design", Kind: productv1.CategoryKind_CATEGORY_KIND_ELECTRONICS}, - {Id: "cat-group-2", Name: "Architecture Patterns", Kind: productv1.CategoryKind_CATEGORY_KIND_BOOK}, - }, - }}, - {List: &productv1.ListOfCategory_List{ - Items: []*productv1.Category{ - {Id: "cat-group-3", Name: "Performance", Kind: productv1.CategoryKind_CATEGORY_KIND_ELECTRONICS}, - }, - }}, - }, - }, - }, - ContributorTeams: &productv1.ListOfListOfUser{ - List: &productv1.ListOfListOfUser_List{ - Items: []*productv1.ListOfUser{ - {List: &productv1.ListOfUser_List{ - Items: []*productv1.User{ - {Id: "team-complex-1", Name: "Senior Engineer A"}, - {Id: "team-complex-2", Name: "Senior Engineer B"}, - }, - }}, - {List: &productv1.ListOfUser_List{ - Items: []*productv1.User{ - {Id: "team-complex-3", Name: "QA Lead"}, - }, - }}, - }, - }, - }, - } - default: - // Generic response for any other ID - result = &productv1.BlogPost{ - Id: id, - Title: fmt.Sprintf("Blog Post %s", id), - Content: fmt.Sprintf("Content for blog post %s", id), - Tags: []string{fmt.Sprintf("tag-%s", id), "general"}, - Categories: []string{"General", fmt.Sprintf("Category-%s", id)}, - ViewCounts: []int32{int32(len(id) * 10), int32(len(id) * 20)}, - // Required nested lists must have data - TagGroups: &productv1.ListOfListOfString{ - List: &productv1.ListOfListOfString_List{ - Items: []*productv1.ListOfString{ - {List: &productv1.ListOfString_List{ - Items: []string{fmt.Sprintf("tag-%s", id), "group"}, - }}, - }, - }, - }, - RelatedTopics: &productv1.ListOfListOfString{ - List: &productv1.ListOfListOfString_List{ - Items: []*productv1.ListOfString{ - {List: &productv1.ListOfString_List{ - Items: []string{fmt.Sprintf("topic-%s", id)}, - }}, - }, - }, - }, - CommentThreads: &productv1.ListOfListOfString{ - List: &productv1.ListOfListOfString_List{ - Items: []*productv1.ListOfString{ - {List: &productv1.ListOfString_List{ - Items: []string{fmt.Sprintf("Comment on %s", id)}, - }}, - }, - }, - }, - // Required complex lists must have data - RelatedCategories: []*productv1.Category{ - {Id: fmt.Sprintf("cat-%s", id), Name: fmt.Sprintf("Category %s", id), Kind: productv1.CategoryKind_CATEGORY_KIND_OTHER}, - }, - Contributors: []*productv1.User{ - {Id: fmt.Sprintf("user-%s", id), Name: fmt.Sprintf("Author %s", id)}, - }, - CategoryGroups: &productv1.ListOfListOfCategory{ - List: &productv1.ListOfListOfCategory_List{ - Items: []*productv1.ListOfCategory{ - {List: &productv1.ListOfCategory_List{ - Items: []*productv1.Category{ - {Id: fmt.Sprintf("cat-group-%s", id), Name: fmt.Sprintf("Group Category %s", id), Kind: productv1.CategoryKind_CATEGORY_KIND_OTHER}, - }, - }}, - }, - }, - }, - } - } - - return &productv1.QueryBlogPostByIdResponse{ - BlogPostById: result, - }, nil -} - -func (s *MockService) QueryBlogPostsWithFilter(ctx context.Context, in *productv1.QueryBlogPostsWithFilterRequest) (*productv1.QueryBlogPostsWithFilterResponse, error) { - filter := in.GetFilter() - var results []*productv1.BlogPost - - // If no filter provided, return empty results - if filter == nil { - return &productv1.QueryBlogPostsWithFilterResponse{ - BlogPostsWithFilter: results, - }, nil - } - - titleFilter := "" - if filter.Title != nil { - titleFilter = filter.Title.GetValue() - } - - hasCategories := false - if filter.HasCategories != nil { - hasCategories = filter.HasCategories.GetValue() - } - - minTags := int32(0) - if filter.MinTags != nil { - minTags = filter.MinTags.GetValue() - } - - // Generate filtered results - for i := 1; i <= 3; i++ { - title := fmt.Sprintf("Filtered Post %d", i) - if titleFilter != "" { - title = fmt.Sprintf("%s - Post %d", titleFilter, i) - } - - var tags []string - tagsCount := minTags + int32(i) - for j := int32(0); j < tagsCount; j++ { - tags = append(tags, fmt.Sprintf("tag%d", j+1)) - } - - var categories []string - if hasCategories { - categories = []string{fmt.Sprintf("Category%d", i), "Filtered"} - } - - results = append(results, &productv1.BlogPost{ - Id: fmt.Sprintf("filtered-blog-%d", i), - Title: title, - Content: fmt.Sprintf("Filtered content %d", i), - Tags: tags, - Categories: categories, - ViewCounts: []int32{int32(i * 100)}, - // Required nested lists must have data - TagGroups: &productv1.ListOfListOfString{ - List: &productv1.ListOfListOfString_List{ - Items: []*productv1.ListOfString{ - {List: &productv1.ListOfString_List{ - Items: []string{fmt.Sprintf("filtered-tag-%d", i)}, - }}, - }, - }, - }, - RelatedTopics: &productv1.ListOfListOfString{ - List: &productv1.ListOfListOfString_List{ - Items: []*productv1.ListOfString{ - {List: &productv1.ListOfString_List{ - Items: []string{fmt.Sprintf("filtered-topic-%d", i)}, - }}, - }, - }, - }, - CommentThreads: &productv1.ListOfListOfString{ - List: &productv1.ListOfListOfString_List{ - Items: []*productv1.ListOfString{ - {List: &productv1.ListOfString_List{ - Items: []string{fmt.Sprintf("Filtered comment %d", i)}, - }}, - }, - }, - }, - // Required complex lists must have data - RelatedCategories: []*productv1.Category{ - {Id: fmt.Sprintf("cat-filtered-%d", i), Name: fmt.Sprintf("Filtered Category %d", i), Kind: productv1.CategoryKind_CATEGORY_KIND_OTHER}, - }, - Contributors: []*productv1.User{ - {Id: fmt.Sprintf("user-filtered-%d", i), Name: fmt.Sprintf("Filtered Author %d", i)}, - }, - CategoryGroups: &productv1.ListOfListOfCategory{ - List: &productv1.ListOfListOfCategory_List{ - Items: []*productv1.ListOfCategory{ - {List: &productv1.ListOfCategory_List{ - Items: []*productv1.Category{ - {Id: fmt.Sprintf("cat-group-filtered-%d", i), Name: fmt.Sprintf("Filtered Group %d", i), Kind: productv1.CategoryKind_CATEGORY_KIND_OTHER}, - }, - }}, - }, - }, - }, - }) - } - - return &productv1.QueryBlogPostsWithFilterResponse{ - BlogPostsWithFilter: results, - }, nil -} - -func (s *MockService) QueryAllBlogPosts(ctx context.Context, in *productv1.QueryAllBlogPostsRequest) (*productv1.QueryAllBlogPostsResponse, error) { - var results []*productv1.BlogPost - - // Create a variety of blog posts - for i := 1; i <= 4; i++ { - var optionalTags *productv1.ListOfString - var keywords *productv1.ListOfString - var ratings *productv1.ListOfFloat - - // Vary the optional fields - if i%2 == 1 { - optionalTags = &productv1.ListOfString{ - List: &productv1.ListOfString_List{ - Items: []string{fmt.Sprintf("optional%d", i), "common"}, - }, - } - } - - if i%3 == 0 { - keywords = &productv1.ListOfString{ - List: &productv1.ListOfString_List{ - Items: []string{fmt.Sprintf("keyword%d", i)}, - }, - } - } - - if i%2 == 0 { - ratings = &productv1.ListOfFloat{ - List: &productv1.ListOfFloat_List{ - Items: []float64{float64(i) + 0.5, float64(i) + 1.0}, - }, - } - } - - results = append(results, &productv1.BlogPost{ - Id: fmt.Sprintf("blog-%d", i), - Title: fmt.Sprintf("Blog Post %d", i), - Content: fmt.Sprintf("Content for blog post %d", i), - Tags: []string{fmt.Sprintf("tag%d", i), "common"}, - OptionalTags: optionalTags, - Categories: []string{fmt.Sprintf("Category%d", i)}, - Keywords: keywords, - ViewCounts: []int32{int32(i * 100), int32(i * 150)}, - Ratings: ratings, - IsPublished: &productv1.ListOfBoolean{ - List: &productv1.ListOfBoolean_List{ - Items: []bool{i%2 == 0, true}, - }, - }, - TagGroups: &productv1.ListOfListOfString{ - List: &productv1.ListOfListOfString_List{ - Items: []*productv1.ListOfString{ - {List: &productv1.ListOfString_List{ - Items: []string{fmt.Sprintf("group%d", i), "shared"}, - }}, - }, - }, - }, - RelatedTopics: &productv1.ListOfListOfString{ - List: &productv1.ListOfListOfString_List{ - Items: []*productv1.ListOfString{ - {List: &productv1.ListOfString_List{ - Items: []string{fmt.Sprintf("topic%d", i)}, - }}, - }, - }, - }, - CommentThreads: &productv1.ListOfListOfString{ - List: &productv1.ListOfListOfString_List{ - Items: []*productv1.ListOfString{ - {List: &productv1.ListOfString_List{ - Items: []string{fmt.Sprintf("Comment for post %d", i)}, - }}, - }, - }, - }, - // Required complex lists must have data - RelatedCategories: []*productv1.Category{ - {Id: fmt.Sprintf("cat-all-%d", i), Name: fmt.Sprintf("Category %d", i), Kind: productv1.CategoryKind_CATEGORY_KIND_OTHER}, - }, - Contributors: []*productv1.User{ - {Id: fmt.Sprintf("user-all-%d", i), Name: fmt.Sprintf("Author %d", i)}, - }, - CategoryGroups: &productv1.ListOfListOfCategory{ - List: &productv1.ListOfListOfCategory_List{ - Items: []*productv1.ListOfCategory{ - {List: &productv1.ListOfCategory_List{ - Items: []*productv1.Category{ - {Id: fmt.Sprintf("cat-group-all-%d", i), Name: fmt.Sprintf("Group Category %d", i), Kind: productv1.CategoryKind_CATEGORY_KIND_OTHER}, - }, - }}, - }, - }, - }, - // Optional list - can be empty - Suggestions: &productv1.ListOfListOfString{}, - }) - } - - return &productv1.QueryAllBlogPostsResponse{ - AllBlogPosts: results, - }, nil -} - -// Author query implementations -func (s *MockService) QueryAuthor(ctx context.Context, in *productv1.QueryAuthorRequest) (*productv1.QueryAuthorResponse, error) { - result := &productv1.Author{ - Id: "author-default", - Name: "Default Author", - Email: &wrapperspb.StringValue{ - Value: "author@example.com", - }, - Skills: []string{"Go", "GraphQL", "Protocol Buffers"}, - Languages: []string{"English", "Spanish", ""}, - SocialLinks: &productv1.ListOfString{ - List: &productv1.ListOfString_List{ - Items: []string{"https://twitter.com/author", "https://linkedin.com/in/author"}, - }, - }, - TeamsByProject: &productv1.ListOfListOfString{ - List: &productv1.ListOfListOfString_List{ - Items: []*productv1.ListOfString{ - {List: &productv1.ListOfString_List{ - Items: []string{"Alice", "Bob", "Charlie"}, - }}, - {List: &productv1.ListOfString_List{ - Items: []string{"David", "Eve"}, - }}, - }, - }, - }, - Collaborations: &productv1.ListOfListOfString{ - List: &productv1.ListOfListOfString_List{ - Items: []*productv1.ListOfString{ - {List: &productv1.ListOfString_List{ - Items: []string{"Open Source Project A", "Research Paper B"}, - }}, - {List: &productv1.ListOfString_List{ - Items: []string{"Conference Talk C"}, - }}, - }, - }, - }, - WrittenPosts: &productv1.ListOfBlogPost{ - List: &productv1.ListOfBlogPost_List{ - Items: []*productv1.BlogPost{ - {Id: "blog-1", Title: "GraphQL Best Practices", Content: "Content here..."}, - {Id: "blog-2", Title: "gRPC vs REST", Content: "Comparison content..."}, - }, - }, - }, - FavoriteCategories: []*productv1.Category{ - {Id: "cat-fav-1", Name: "Software Engineering", Kind: productv1.CategoryKind_CATEGORY_KIND_ELECTRONICS}, - {Id: "cat-fav-2", Name: "Technical Writing", Kind: productv1.CategoryKind_CATEGORY_KIND_BOOK}, - }, - RelatedAuthors: &productv1.ListOfUser{ - List: &productv1.ListOfUser_List{ - Items: []*productv1.User{ - {Id: "author-rel-1", Name: "Related Author One"}, - {Id: "author-rel-2", Name: "Related Author Two"}, - }, - }, - }, - ProductReviews: &productv1.ListOfProduct{ - List: &productv1.ListOfProduct_List{ - Items: []*productv1.Product{ - {Id: "prod-rev-1", Name: "Code Editor Pro", Price: 199.99}, - }, - }, - }, - AuthorGroups: &productv1.ListOfListOfUser{ - List: &productv1.ListOfListOfUser_List{ - Items: []*productv1.ListOfUser{ - {List: &productv1.ListOfUser_List{ - Items: []*productv1.User{ - {Id: "group-auth-1", Name: "Team Lead Alpha"}, - {Id: "group-auth-2", Name: "Senior Dev Beta"}, - }, - }}, - {List: &productv1.ListOfUser_List{ - Items: []*productv1.User{ - {Id: "group-auth-3", Name: "Junior Dev Gamma"}, - }, - }}, - // empty list - {List: &productv1.ListOfUser_List{}}, - // null item - nil, - }, - }, - }, - CategoryPreferences: &productv1.ListOfListOfCategory{ - List: &productv1.ListOfListOfCategory_List{ - Items: []*productv1.ListOfCategory{ - {List: &productv1.ListOfCategory_List{ - Items: []*productv1.Category{ - {Id: "pref-cat-1", Name: "Microservices", Kind: productv1.CategoryKind_CATEGORY_KIND_ELECTRONICS}, - {Id: "pref-cat-2", Name: "Cloud Computing", Kind: productv1.CategoryKind_CATEGORY_KIND_ELECTRONICS}, - }, - }}, - }, - }, - }, - } - - return &productv1.QueryAuthorResponse{ - Author: result, - }, nil -} - -func (s *MockService) QueryAuthorById(ctx context.Context, in *productv1.QueryAuthorByIdRequest) (*productv1.QueryAuthorByIdResponse, error) { - id := in.GetId() - - // Return null for specific test IDs - if id == "not-found" { - return &productv1.QueryAuthorByIdResponse{ - AuthorById: nil, - }, nil - } - - var result *productv1.Author - - switch id { - case "minimal": - result = &productv1.Author{ - Id: id, - Name: "Minimal Author", - Skills: []string{"Basic"}, - Languages: []string{"English"}, - TeamsByProject: &productv1.ListOfListOfString{ - List: &productv1.ListOfListOfString_List{ - Items: []*productv1.ListOfString{ - {List: &productv1.ListOfString_List{ - Items: []string{"Solo"}, - }}, - }, - }, - }, - // Required complex lists must have data - FavoriteCategories: []*productv1.Category{ - {Id: "cat-minimal", Name: "Basic Category", Kind: productv1.CategoryKind_CATEGORY_KIND_OTHER}, - }, - CategoryPreferences: &productv1.ListOfListOfCategory{ - List: &productv1.ListOfListOfCategory_List{ - Items: []*productv1.ListOfCategory{ - {List: &productv1.ListOfCategory_List{ - Items: []*productv1.Category{ - {Id: "cat-pref-minimal", Name: "Minimal Preference", Kind: productv1.CategoryKind_CATEGORY_KIND_OTHER}, - }, - }}, - }, - }, - }, - // Optional list - can be empty - Collaborations: &productv1.ListOfListOfString{}, - } - case "experienced": - result = &productv1.Author{ - Id: id, - Name: "Experienced Author", - Email: &wrapperspb.StringValue{ - Value: "experienced@example.com", - }, - Skills: []string{"Go", "GraphQL", "gRPC", "Microservices", "Kubernetes"}, - Languages: []string{"English", "French", "German"}, - SocialLinks: &productv1.ListOfString{ - List: &productv1.ListOfString_List{ - Items: []string{ - "https://github.com/experienced", - "https://twitter.com/experienced", - "https://medium.com/@experienced", - }, - }, - }, - TeamsByProject: &productv1.ListOfListOfString{ - List: &productv1.ListOfListOfString_List{ - Items: []*productv1.ListOfString{ - {List: &productv1.ListOfString_List{ - Items: []string{"Senior Dev 1", "Senior Dev 2", "Tech Lead"}, - }}, - {List: &productv1.ListOfString_List{ - Items: []string{"Architect", "Principal Engineer"}, - }}, - {List: &productv1.ListOfString_List{ - Items: []string{"PM", "Designer", "QA Lead"}, - }}, - }, - }, - }, - Collaborations: &productv1.ListOfListOfString{ - List: &productv1.ListOfListOfString_List{ - Items: []*productv1.ListOfString{ - {List: &productv1.ListOfString_List{ - Items: []string{"Major OSS Project", "Industry Standard", "Research Initiative"}, - }}, - {List: &productv1.ListOfString_List{ - Items: []string{"Conference Keynote", "Workshop Series"}, - }}, - }, - }, - }, - // Required complex lists must have data - FavoriteCategories: []*productv1.Category{ - {Id: "cat-experienced-1", Name: "Advanced Programming", Kind: productv1.CategoryKind_CATEGORY_KIND_ELECTRONICS}, - {Id: "cat-experienced-2", Name: "Technical Leadership", Kind: productv1.CategoryKind_CATEGORY_KIND_BOOK}, - }, - CategoryPreferences: &productv1.ListOfListOfCategory{ - List: &productv1.ListOfListOfCategory_List{ - Items: []*productv1.ListOfCategory{ - {List: &productv1.ListOfCategory_List{ - Items: []*productv1.Category{ - {Id: "cat-pref-experienced-1", Name: "System Architecture", Kind: productv1.CategoryKind_CATEGORY_KIND_ELECTRONICS}, - {Id: "cat-pref-experienced-2", Name: "Team Management", Kind: productv1.CategoryKind_CATEGORY_KIND_BOOK}, - }, - }}, - }, - }, - }, - } - default: - result = &productv1.Author{ - Id: id, - Name: fmt.Sprintf("Author %s", id), - Email: &wrapperspb.StringValue{ - Value: fmt.Sprintf("%s@example.com", id), - }, - Skills: []string{fmt.Sprintf("Skill-%s", id), "General"}, - Languages: []string{"English", fmt.Sprintf("Language-%s", id)}, - TeamsByProject: &productv1.ListOfListOfString{ - List: &productv1.ListOfListOfString_List{ - Items: []*productv1.ListOfString{ - {List: &productv1.ListOfString_List{ - Items: []string{fmt.Sprintf("Team-%s", id)}, - }}, - }, - }, - }, - // Required complex lists must have data - FavoriteCategories: []*productv1.Category{ - {Id: fmt.Sprintf("cat-%s", id), Name: fmt.Sprintf("Favorite Category %s", id), Kind: productv1.CategoryKind_CATEGORY_KIND_OTHER}, - }, - CategoryPreferences: &productv1.ListOfListOfCategory{ - List: &productv1.ListOfListOfCategory_List{ - Items: []*productv1.ListOfCategory{ - {List: &productv1.ListOfCategory_List{ - Items: []*productv1.Category{ - {Id: fmt.Sprintf("cat-pref-%s", id), Name: fmt.Sprintf("Preference %s", id), Kind: productv1.CategoryKind_CATEGORY_KIND_OTHER}, - }, - }}, - }, - }, - }, - // Optional list - can be empty - Collaborations: &productv1.ListOfListOfString{}, - } - } - - return &productv1.QueryAuthorByIdResponse{ - AuthorById: result, - }, nil -} - -func (s *MockService) QueryAuthorsWithFilter(ctx context.Context, in *productv1.QueryAuthorsWithFilterRequest) (*productv1.QueryAuthorsWithFilterResponse, error) { - filter := in.GetFilter() - var results []*productv1.Author - - if filter == nil { - return &productv1.QueryAuthorsWithFilterResponse{ - AuthorsWithFilter: results, - }, nil - } - - nameFilter := "" - if filter.Name != nil { - nameFilter = filter.Name.GetValue() - } - - hasTeams := false - if filter.HasTeams != nil { - hasTeams = filter.HasTeams.GetValue() - } - - skillCount := int32(0) - if filter.SkillCount != nil { - skillCount = filter.SkillCount.GetValue() - } - - // Generate filtered results - for i := 1; i <= 3; i++ { - name := fmt.Sprintf("Filtered Author %d", i) - if nameFilter != "" { - name = fmt.Sprintf("%s - Author %d", nameFilter, i) - } - - var skills []string - skillsNeeded := skillCount + int32(i) - for j := int32(0); j < skillsNeeded; j++ { - skills = append(skills, fmt.Sprintf("Skill%d", j+1)) - } - - var teamsByProject *productv1.ListOfListOfString - if hasTeams { - teamsByProject = &productv1.ListOfListOfString{ - List: &productv1.ListOfListOfString_List{ - Items: []*productv1.ListOfString{ - {List: &productv1.ListOfString_List{ - Items: []string{fmt.Sprintf("Team%d", i), "SharedTeam"}, - }}, - }, - }, - } - } else { - teamsByProject = &productv1.ListOfListOfString{List: &productv1.ListOfListOfString_List{}} - } - - results = append(results, &productv1.Author{ - Id: fmt.Sprintf("filtered-author-%d", i), - Name: name, - Skills: skills, - Languages: []string{"English", fmt.Sprintf("Lang%d", i)}, - TeamsByProject: teamsByProject, - // Required complex lists must have data - FavoriteCategories: []*productv1.Category{ - {Id: fmt.Sprintf("cat-filtered-%d", i), Name: fmt.Sprintf("Filtered Category %d", i), Kind: productv1.CategoryKind_CATEGORY_KIND_OTHER}, - }, - CategoryPreferences: &productv1.ListOfListOfCategory{ - List: &productv1.ListOfListOfCategory_List{ - Items: []*productv1.ListOfCategory{ - {List: &productv1.ListOfCategory_List{ - Items: []*productv1.Category{ - {Id: fmt.Sprintf("cat-pref-filtered-%d", i), Name: fmt.Sprintf("Filtered Preference %d", i), Kind: productv1.CategoryKind_CATEGORY_KIND_OTHER}, - }, - }}, - }, - }, - }, - // Optional list - can be empty - Collaborations: &productv1.ListOfListOfString{}, - }) - } - - return &productv1.QueryAuthorsWithFilterResponse{ - AuthorsWithFilter: results, - }, nil -} - -func (s *MockService) QueryAllAuthors(ctx context.Context, in *productv1.QueryAllAuthorsRequest) (*productv1.QueryAllAuthorsResponse, error) { - var results []*productv1.Author - - for i := 1; i <= 3; i++ { - var email *wrapperspb.StringValue - var socialLinks *productv1.ListOfString - var collaborations *productv1.ListOfListOfString - - if i%2 == 1 { - email = &wrapperspb.StringValue{ - Value: fmt.Sprintf("author%d@example.com", i), - } - } - - if i%3 == 0 { - socialLinks = &productv1.ListOfString{ - List: &productv1.ListOfString_List{ - Items: []string{fmt.Sprintf("https://github.com/author%d", i)}, - }, - } - } - - if i == 2 { - collaborations = &productv1.ListOfListOfString{ - List: &productv1.ListOfListOfString_List{ - Items: []*productv1.ListOfString{ - {List: &productv1.ListOfString_List{ - Items: []string{"Collaboration A", "Collaboration B"}, - }}, - }, - }, - } - } else { - collaborations = &productv1.ListOfListOfString{} - } - - results = append(results, &productv1.Author{ - Id: fmt.Sprintf("author-%d", i), - Name: fmt.Sprintf("Author %d", i), - Email: email, - Skills: []string{fmt.Sprintf("Skill%d", i), "Common"}, - Languages: []string{"English", fmt.Sprintf("Language%d", i)}, - SocialLinks: socialLinks, - TeamsByProject: &productv1.ListOfListOfString{ - List: &productv1.ListOfListOfString_List{ - Items: []*productv1.ListOfString{ - {List: &productv1.ListOfString_List{ - Items: []string{fmt.Sprintf("Team%d", i)}, - }}, - }, - }, - }, - // Required complex lists must have data - FavoriteCategories: []*productv1.Category{ - {Id: fmt.Sprintf("cat-all-%d", i), Name: fmt.Sprintf("All Category %d", i), Kind: productv1.CategoryKind_CATEGORY_KIND_OTHER}, - }, - CategoryPreferences: &productv1.ListOfListOfCategory{ - List: &productv1.ListOfListOfCategory_List{ - Items: []*productv1.ListOfCategory{ - {List: &productv1.ListOfCategory_List{ - Items: []*productv1.Category{ - {Id: fmt.Sprintf("cat-pref-all-%d", i), Name: fmt.Sprintf("All Preference %d", i), Kind: productv1.CategoryKind_CATEGORY_KIND_OTHER}, - }, - }}, - }, - }, - }, - // Optional list - can be empty/variable - Collaborations: collaborations, - }) - } - - return &productv1.QueryAllAuthorsResponse{ - AllAuthors: results, - }, nil -} - -// BlogPost mutation implementations -func (s *MockService) MutationCreateBlogPost(ctx context.Context, in *productv1.MutationCreateBlogPostRequest) (*productv1.MutationCreateBlogPostResponse, error) { - input := in.GetInput() - - result := &productv1.BlogPost{ - Id: fmt.Sprintf("blog-%d", rand.Intn(1000)), - Title: input.GetTitle(), - Content: input.GetContent(), - Tags: input.GetTags(), - OptionalTags: input.GetOptionalTags(), - Categories: input.GetCategories(), - Keywords: input.GetKeywords(), - ViewCounts: input.GetViewCounts(), - Ratings: input.GetRatings(), - IsPublished: input.GetIsPublished(), - TagGroups: input.GetTagGroups(), - RelatedTopics: input.GetRelatedTopics(), - CommentThreads: input.GetCommentThreads(), - Suggestions: input.GetSuggestions(), - // Convert input types to output types - RelatedCategories: convertCategoryInputListToCategories(input.GetRelatedCategories()), - Contributors: convertUserInputsToUsers(input.GetContributors()), - CategoryGroups: convertNestedCategoryInputsToCategories(input.GetCategoryGroups()), - MentionedProducts: &productv1.ListOfProduct{ - List: &productv1.ListOfProduct_List{ - Items: []*productv1.Product{ - {Id: "prod-1", Name: "Sample Product", Price: 99.99}, - }, - }, - }, - MentionedUsers: &productv1.ListOfUser{ - List: &productv1.ListOfUser_List{ - Items: []*productv1.User{ - {Id: "user-3", Name: "Bob Johnson"}, - }, - }, - }, - ContributorTeams: &productv1.ListOfListOfUser{ - List: &productv1.ListOfListOfUser_List{ - Items: []*productv1.ListOfUser{ - {List: &productv1.ListOfUser_List{ - Items: []*productv1.User{ - {Id: "user-4", Name: "Alice Brown"}, - }, - }}, - }, - }, - }, - } - - return &productv1.MutationCreateBlogPostResponse{ - CreateBlogPost: result, + return &productv1.QueryUsersResponse{ + Users: results, }, nil } -func (s *MockService) MutationUpdateBlogPost(ctx context.Context, in *productv1.MutationUpdateBlogPostRequest) (*productv1.MutationUpdateBlogPostResponse, error) { - id := in.GetId() - input := in.GetInput() - - if id == "non-existent" { - return &productv1.MutationUpdateBlogPostResponse{ - UpdateBlogPost: nil, - }, nil - } +func (s *MockService) QueryUser(ctx context.Context, in *productv1.QueryUserRequest) (*productv1.QueryUserResponse, error) { + userId := in.GetId() - result := &productv1.BlogPost{ - Id: id, - Title: input.GetTitle(), - Content: input.GetContent(), - Tags: input.GetTags(), - OptionalTags: input.GetOptionalTags(), - Categories: input.GetCategories(), - Keywords: input.GetKeywords(), - ViewCounts: input.GetViewCounts(), - Ratings: input.GetRatings(), - IsPublished: input.GetIsPublished(), - TagGroups: input.GetTagGroups(), - RelatedTopics: input.GetRelatedTopics(), - CommentThreads: input.GetCommentThreads(), - Suggestions: input.GetSuggestions(), - // Convert input types to output types - RelatedCategories: convertCategoryInputListToCategories(input.GetRelatedCategories()), - Contributors: convertUserInputsToUsers(input.GetContributors()), - CategoryGroups: convertNestedCategoryInputsToCategories(input.GetCategoryGroups()), - MentionedProducts: &productv1.ListOfProduct{ - List: &productv1.ListOfProduct_List{ - Items: []*productv1.Product{ - {Id: "prod-updated", Name: "Updated Product", Price: 149.99}, - }, - }, - }, - MentionedUsers: &productv1.ListOfUser{ - List: &productv1.ListOfUser_List{ - Items: []*productv1.User{ - {Id: "user-updated", Name: "Updated User"}, - }, - }, - }, - ContributorTeams: &productv1.ListOfListOfUser{ - List: &productv1.ListOfListOfUser_List{ - Items: []*productv1.ListOfUser{ - {List: &productv1.ListOfUser_List{ - Items: []*productv1.User{ - {Id: "user-team-updated", Name: "Updated Team Member"}, - }, - }}, - }, - }, - }, + // Return a gRPC status error for a specific test case + if userId == "error-user" { + return nil, status.Errorf(codes.NotFound, "user not found: %s", userId) } - return &productv1.MutationUpdateBlogPostResponse{ - UpdateBlogPost: result, - }, nil -} - -// Author mutation implementations -func (s *MockService) MutationCreateAuthor(ctx context.Context, in *productv1.MutationCreateAuthorRequest) (*productv1.MutationCreateAuthorResponse, error) { - input := in.GetInput() - - result := &productv1.Author{ - Id: fmt.Sprintf("author-%d", rand.Intn(1000)), - Name: input.GetName(), - Email: input.GetEmail(), - Skills: input.GetSkills(), - Languages: input.GetLanguages(), - SocialLinks: input.GetSocialLinks(), - TeamsByProject: input.GetTeamsByProject(), - Collaborations: input.GetCollaborations(), - // Convert input types to output types for complex fields - FavoriteCategories: convertCategoryInputsToCategories(input.GetFavoriteCategories()), - AuthorGroups: convertNestedUserInputsToUsers(input.GetAuthorGroups()), - ProjectTeams: convertNestedUserInputsToUsers(input.GetProjectTeams()), - // Keep other complex fields with mock data since they're not in the simplified input - WrittenPosts: &productv1.ListOfBlogPost{ - List: &productv1.ListOfBlogPost_List{ - Items: []*productv1.BlogPost{ - {Id: "blog-created", Title: "Created Post", Content: "Content..."}, - }, - }, - }, - RelatedAuthors: &productv1.ListOfUser{ - List: &productv1.ListOfUser_List{ - Items: []*productv1.User{ - {Id: "related-author", Name: "Related Author"}, - }, - }, - }, - ProductReviews: &productv1.ListOfProduct{ - List: &productv1.ListOfProduct_List{ - Items: []*productv1.Product{ - {Id: "reviewed-product", Name: "Code Editor", Price: 199.99}, - }, - }, - }, - CategoryPreferences: &productv1.ListOfListOfCategory{ - List: &productv1.ListOfListOfCategory_List{ - Items: []*productv1.ListOfCategory{ - {List: &productv1.ListOfCategory_List{ - Items: []*productv1.Category{ - {Id: "pref-cat", Name: "Backend Development", Kind: productv1.CategoryKind_CATEGORY_KIND_ELECTRONICS}, - }, - }}, - }, - }, + return &productv1.QueryUserResponse{ + User: &productv1.User{ + Id: userId, + Name: fmt.Sprintf("User %s", userId), }, - } - - return &productv1.MutationCreateAuthorResponse{ - CreateAuthor: result, }, nil } -func (s *MockService) MutationUpdateAuthor(ctx context.Context, in *productv1.MutationUpdateAuthorRequest) (*productv1.MutationUpdateAuthorResponse, error) { - id := in.GetId() - input := in.GetInput() - - if id == "non-existent" { - return &productv1.MutationUpdateAuthorResponse{ - UpdateAuthor: nil, - }, nil - } +func (s *MockService) QueryNestedType(ctx context.Context, in *productv1.QueryNestedTypeRequest) (*productv1.QueryNestedTypeResponse, error) { + var nestedTypes []*productv1.NestedTypeA - result := &productv1.Author{ - Id: id, - Name: input.GetName(), - Email: input.GetEmail(), - Skills: input.GetSkills(), - Languages: input.GetLanguages(), - SocialLinks: input.GetSocialLinks(), - TeamsByProject: input.GetTeamsByProject(), - Collaborations: input.GetCollaborations(), - // Convert input types to output types for complex fields - FavoriteCategories: convertCategoryInputsToCategories(input.GetFavoriteCategories()), - AuthorGroups: convertNestedUserInputsToUsers(input.GetAuthorGroups()), - ProjectTeams: convertNestedUserInputsToUsers(input.GetProjectTeams()), - // Keep other complex fields with mock data since they're not in the simplified input - WrittenPosts: &productv1.ListOfBlogPost{ - List: &productv1.ListOfBlogPost_List{ - Items: []*productv1.BlogPost{ - {Id: "blog-updated", Title: "Updated Post", Content: "Updated content..."}, - }, - }, - }, - RelatedAuthors: &productv1.ListOfUser{ - List: &productv1.ListOfUser_List{ - Items: []*productv1.User{ - {Id: "related-author-updated", Name: "Updated Related Author"}, - }, - }, - }, - ProductReviews: &productv1.ListOfProduct{ - List: &productv1.ListOfProduct_List{ - Items: []*productv1.Product{ - {Id: "reviewed-product-updated", Name: "Updated Code Editor", Price: 249.99}, - }, - }, - }, - CategoryPreferences: &productv1.ListOfListOfCategory{ - List: &productv1.ListOfListOfCategory_List{ - Items: []*productv1.ListOfCategory{ - {List: &productv1.ListOfCategory_List{ - Items: []*productv1.Category{ - {Id: "pref-cat-updated", Name: "Updated Backend Development", Kind: productv1.CategoryKind_CATEGORY_KIND_ELECTRONICS}, - }, - }}, + // Generate 2 nested types + for i := 1; i <= 2; i++ { + nestedTypes = append(nestedTypes, &productv1.NestedTypeA{ + Id: fmt.Sprintf("nested-a-%d", i), + Name: fmt.Sprintf("Nested A %d", i), + B: &productv1.NestedTypeB{ + Id: fmt.Sprintf("nested-b-%d", i), + Name: fmt.Sprintf("Nested B %d", i), + C: &productv1.NestedTypeC{ + Id: fmt.Sprintf("nested-c-%d", i), + Name: fmt.Sprintf("Nested C %d", i), }, }, - }, + }) } - return &productv1.MutationUpdateAuthorResponse{ - UpdateAuthor: result, + return &productv1.QueryNestedTypeResponse{ + NestedType: nestedTypes, }, nil } -// Bulk operation implementations -func (s *MockService) QueryBulkSearchAuthors(ctx context.Context, in *productv1.QueryBulkSearchAuthorsRequest) (*productv1.QueryBulkSearchAuthorsResponse, error) { - var allResults []*productv1.Author - - // Handle nullable list - if filters is nil, return empty results - if in.Filters == nil { - return &productv1.QueryBulkSearchAuthorsResponse{ - BulkSearchAuthors: allResults, - }, nil +func (s *MockService) QueryRecursiveType(ctx context.Context, in *productv1.QueryRecursiveTypeRequest) (*productv1.QueryRecursiveTypeResponse, error) { + // Create a recursive structure 3 levels deep + recursiveType := &productv1.RecursiveType{ + Id: "recursive-1", + Name: "Level 1", + RecursiveType: &productv1.RecursiveType{ + Id: "recursive-2", + Name: "Level 2", + RecursiveType: &productv1.RecursiveType{ + Id: "recursive-3", + Name: "Level 3", + }, + }, } - // Process each filter in the list - if in.Filters.List != nil { - for i, filter := range in.Filters.List.Items { - // Create mock results for each filter - for j := 1; j <= 2; j++ { - name := fmt.Sprintf("Bulk Author %d-%d", i+1, j) - if filter.Name != nil { - name = fmt.Sprintf("%s - Bulk %d-%d", filter.Name.GetValue(), i+1, j) - } + return &productv1.QueryRecursiveTypeResponse{ + RecursiveType: recursiveType, + }, nil +} - var skills []string - skillCount := int32(3) - if filter.SkillCount != nil { - skillCount = filter.SkillCount.GetValue() - } - for k := int32(0); k < skillCount; k++ { - skills = append(skills, fmt.Sprintf("BulkSkill%d", k+1)) - } +func (s *MockService) QueryTypeFilterWithArguments(ctx context.Context, in *productv1.QueryTypeFilterWithArgumentsRequest) (*productv1.QueryTypeFilterWithArgumentsResponse, error) { + filterField1 := in.GetFilterField_1() + filterField2 := in.GetFilterField_2() - var teamsByProject *productv1.ListOfListOfString - if filter.HasTeams != nil && filter.HasTeams.GetValue() { - teamsByProject = &productv1.ListOfListOfString{ - List: &productv1.ListOfListOfString_List{ - Items: []*productv1.ListOfString{ - {List: &productv1.ListOfString_List{ - Items: []string{fmt.Sprintf("BulkTeam%d", j), "SharedBulkTeam"}, - }}, - }, - }, - } - } else { - teamsByProject = &productv1.ListOfListOfString{List: &productv1.ListOfListOfString_List{}} - } + var fields []*productv1.TypeWithMultipleFilterFields - allResults = append(allResults, &productv1.Author{ - Id: fmt.Sprintf("bulk-author-%d-%d", i+1, j), - Name: name, - Skills: skills, - Languages: []string{"English", fmt.Sprintf("BulkLang%d", j)}, - TeamsByProject: teamsByProject, - FavoriteCategories: []*productv1.Category{ - {Id: fmt.Sprintf("bulk-cat-%d-%d", i+1, j), Name: fmt.Sprintf("Bulk Category %d-%d", i+1, j), Kind: productv1.CategoryKind_CATEGORY_KIND_OTHER}, - }, - CategoryPreferences: &productv1.ListOfListOfCategory{ - List: &productv1.ListOfListOfCategory_List{ - Items: []*productv1.ListOfCategory{ - {List: &productv1.ListOfCategory_List{ - Items: []*productv1.Category{ - {Id: fmt.Sprintf("bulk-pref-%d-%d", i+1, j), Name: fmt.Sprintf("Bulk Preference %d-%d", i+1, j), Kind: productv1.CategoryKind_CATEGORY_KIND_BOOK}, - }, - }}, - }, - }, - }, - }) - } - } + // Create results that echo the filter values + for i := 1; i <= 2; i++ { + fields = append(fields, &productv1.TypeWithMultipleFilterFields{ + Id: fmt.Sprintf("multi-filter-%d", i), + Name: fmt.Sprintf("MultiFilter %d", i), + FilterField_1: filterField1, + FilterField_2: filterField2, + }) } - return &productv1.QueryBulkSearchAuthorsResponse{ - BulkSearchAuthors: allResults, + return &productv1.QueryTypeFilterWithArgumentsResponse{ + TypeFilterWithArguments: fields, }, nil } -func (s *MockService) QueryBulkSearchBlogPosts(ctx context.Context, in *productv1.QueryBulkSearchBlogPostsRequest) (*productv1.QueryBulkSearchBlogPostsResponse, error) { - var allResults []*productv1.BlogPost - - // Handle nullable list - if filters is nil, return empty results - if in.Filters == nil { - return &productv1.QueryBulkSearchBlogPostsResponse{ - BulkSearchBlogPosts: allResults, - }, nil - } - - // Process each filter in the list - if in.Filters.List != nil { - for i, filter := range in.Filters.List.Items { - // Create mock results for each filter - for j := 1; j <= 2; j++ { - title := fmt.Sprintf("Bulk Blog Post %d-%d", i+1, j) - if filter.Title != nil { - title = fmt.Sprintf("%s - Bulk %d-%d", filter.Title.GetValue(), i+1, j) - } - - var categories []string - if filter.HasCategories != nil && filter.HasCategories.GetValue() { - categories = []string{fmt.Sprintf("BulkCategory%d", j), "SharedBulkCategory"} - } else { - categories = []string{} - } +func (s *MockService) QueryTypeWithMultipleFilterFields(ctx context.Context, in *productv1.QueryTypeWithMultipleFilterFieldsRequest) (*productv1.QueryTypeWithMultipleFilterFieldsResponse, error) { + filter := in.GetFilter() - minTags := int32(2) - if filter.MinTags != nil { - minTags = filter.MinTags.GetValue() - } - var tags []string - for k := int32(0); k < minTags; k++ { - tags = append(tags, fmt.Sprintf("BulkTag%d", k+1)) - } + var fields []*productv1.TypeWithMultipleFilterFields - allResults = append(allResults, &productv1.BlogPost{ - Id: fmt.Sprintf("bulk-post-%d-%d", i+1, j), - Title: title, - Content: fmt.Sprintf("Bulk content for post %d-%d", i+1, j), - Tags: tags, - Categories: categories, - ViewCounts: []int32{100, 150, 200}, - TagGroups: &productv1.ListOfListOfString{ - List: &productv1.ListOfListOfString_List{ - Items: []*productv1.ListOfString{ - {List: &productv1.ListOfString_List{ - Items: []string{fmt.Sprintf("BulkGroup%d", j)}, - }}, - }, - }, - }, - RelatedTopics: &productv1.ListOfListOfString{ - List: &productv1.ListOfListOfString_List{ - Items: []*productv1.ListOfString{ - {List: &productv1.ListOfString_List{ - Items: []string{fmt.Sprintf("BulkTopic%d", j)}, - }}, - }, - }, - }, - CommentThreads: &productv1.ListOfListOfString{ - List: &productv1.ListOfListOfString_List{ - Items: []*productv1.ListOfString{ - {List: &productv1.ListOfString_List{ - Items: []string{fmt.Sprintf("BulkComment%d", j)}, - }}, - }, - }, - }, - RelatedCategories: []*productv1.Category{ - {Id: fmt.Sprintf("bulk-rel-cat-%d-%d", i+1, j), Name: fmt.Sprintf("Bulk Related %d-%d", i+1, j), Kind: productv1.CategoryKind_CATEGORY_KIND_OTHER}, - }, - Contributors: []*productv1.User{ - {Id: fmt.Sprintf("bulk-contrib-%d-%d", i+1, j), Name: fmt.Sprintf("Bulk Contributor %d-%d", i+1, j)}, - }, - CategoryGroups: &productv1.ListOfListOfCategory{ - List: &productv1.ListOfListOfCategory_List{ - Items: []*productv1.ListOfCategory{ - {List: &productv1.ListOfCategory_List{ - Items: []*productv1.Category{ - {Id: fmt.Sprintf("bulk-grp-cat-%d-%d", i+1, j), Name: fmt.Sprintf("Bulk Group Cat %d-%d", i+1, j), Kind: productv1.CategoryKind_CATEGORY_KIND_BOOK}, - }, - }}, - }, - }, - }, - }) - } - } + // Echo the filter values in the results + for i := 1; i <= 2; i++ { + fields = append(fields, &productv1.TypeWithMultipleFilterFields{ + Id: fmt.Sprintf("filtered-%d", i), + Name: "Filter: " + strconv.Itoa(i), + FilterField_1: filter.FilterField_1, + FilterField_2: filter.FilterField_2, + }) } - return &productv1.QueryBulkSearchBlogPostsResponse{ - BulkSearchBlogPosts: allResults, + return &productv1.QueryTypeWithMultipleFilterFieldsResponse{ + TypeWithMultipleFilterFields: fields, }, nil } -func (s *MockService) MutationBulkCreateAuthors(ctx context.Context, in *productv1.MutationBulkCreateAuthorsRequest) (*productv1.MutationBulkCreateAuthorsResponse, error) { - var results []*productv1.Author +func (s *MockService) QueryComplexFilterType(ctx context.Context, in *productv1.QueryComplexFilterTypeRequest) (*productv1.QueryComplexFilterTypeResponse, error) { + filter := in.GetFilter() - // Handle nullable list - if authors is nil, return empty results - if in.Authors == nil { - return &productv1.MutationBulkCreateAuthorsResponse{ - BulkCreateAuthors: results, - }, nil + var name string + if filter != nil && filter.GetFilter() != nil { + name = filter.GetFilter().GetName() + } else { + name = "Default Product" } - // Process each author input in the list - if in.Authors.List != nil { - for i, authorInput := range in.Authors.List.Items { - // Convert nested UserInput lists to Users for complex fields - var authorGroups *productv1.ListOfListOfUser - if authorInput.AuthorGroups != nil { - authorGroups = convertNestedUserInputsToUsers(authorInput.AuthorGroups) - } - - var projectTeams *productv1.ListOfListOfUser - if authorInput.ProjectTeams != nil { - projectTeams = convertNestedUserInputsToUsers(authorInput.ProjectTeams) - } + return &productv1.QueryComplexFilterTypeResponse{ + ComplexFilterType: []*productv1.TypeWithComplexFilterInput{ + { + Id: "test-id-123", + Name: name, + }, + }, + }, nil +} - // Convert CategoryInput list to Categories - var favoriteCategories []*productv1.Category - if authorInput.FavoriteCategories != nil { - favoriteCategories = convertCategoryInputsToCategories(authorInput.FavoriteCategories) - } +func (s *MockService) QueryRandomPet(ctx context.Context, in *productv1.QueryRandomPetRequest) (*productv1.QueryRandomPetResponse, error) { + // Create either a cat or dog randomly + var pet *productv1.Animal - author := &productv1.Author{ - Id: fmt.Sprintf("bulk-created-author-%d", i+1), - Name: authorInput.Name, - Email: authorInput.Email, - Skills: authorInput.Skills, - Languages: authorInput.Languages, - SocialLinks: authorInput.SocialLinks, - TeamsByProject: authorInput.TeamsByProject, - Collaborations: authorInput.Collaborations, - FavoriteCategories: favoriteCategories, - AuthorGroups: authorGroups, - ProjectTeams: projectTeams, - // Add required complex fields with mock data - WrittenPosts: &productv1.ListOfBlogPost{ - List: &productv1.ListOfBlogPost_List{ - Items: []*productv1.BlogPost{ - {Id: fmt.Sprintf("bulk-blog-%d", i+1), Title: fmt.Sprintf("Bulk Created Post %d", i+1), Content: "Bulk created content..."}, + // Random choice between cat and dog + if rand.Intn(2) == 0 { + // Create a cat + pet = &productv1.Animal{ + Instance: &productv1.Animal_Cat{ + Cat: &productv1.Cat{ + Id: "cat-1", + Name: "Whiskers", + Kind: "Siamese", + MeowVolume: int32(rand.Intn(10) + 1), // Random volume between 1-10 + Owner: &productv1.Owner{ + Id: "owner-cat-1", + Name: "Alice Johnson", + Contact: &productv1.ContactInfo{ + Email: "alice@example.com", + Phone: "555-100-2000", + Address: &productv1.Address{ + Street: "10 Cat Street", + City: "Catville", + Country: "USA", + ZipCode: "10101", + }, }, }, - }, - RelatedAuthors: &productv1.ListOfUser{ - List: &productv1.ListOfUser_List{ - Items: []*productv1.User{ - {Id: fmt.Sprintf("bulk-rel-author-%d", i+1), Name: fmt.Sprintf("Bulk Related Author %d", i+1)}, + Breed: &productv1.CatBreed{ + Id: "breed-cat-1", + Name: "Siamese", + Origin: "Thailand", + Characteristics: &productv1.BreedCharacteristics{ + Size: "Medium", + Temperament: "Vocal and Active", + Lifespan: "15-20 years", }, }, }, - ProductReviews: &productv1.ListOfProduct{ - List: &productv1.ListOfProduct_List{ - Items: []*productv1.Product{ - {Id: fmt.Sprintf("bulk-prod-%d", i+1), Name: fmt.Sprintf("Bulk Product %d", i+1), Price: 99.99}, + }, + } + } else { + // Create a dog + pet = &productv1.Animal{ + Instance: &productv1.Animal_Dog{ + Dog: &productv1.Dog{ + Id: "dog-1", + Name: "Spot", + Kind: "Dalmatian", + BarkVolume: int32(rand.Intn(10) + 1), // Random volume between 1-10 + Owner: &productv1.Owner{ + Id: "owner-dog-1", + Name: "Bob Smith", + Contact: &productv1.ContactInfo{ + Email: "bob@example.com", + Phone: "555-200-3000", + Address: &productv1.Address{ + Street: "20 Dog Lane", + City: "Dogtown", + Country: "USA", + ZipCode: "20202", + }, }, }, - }, - CategoryPreferences: &productv1.ListOfListOfCategory{ - List: &productv1.ListOfListOfCategory_List{ - Items: []*productv1.ListOfCategory{ - {List: &productv1.ListOfCategory_List{ - Items: []*productv1.Category{ - {Id: fmt.Sprintf("bulk-cat-pref-%d", i+1), Name: fmt.Sprintf("Bulk Category Preference %d", i+1), Kind: productv1.CategoryKind_CATEGORY_KIND_ELECTRONICS}, - }, - }}, + Breed: &productv1.DogBreed{ + Id: "breed-dog-1", + Name: "Dalmatian", + Origin: "Croatia", + Characteristics: &productv1.BreedCharacteristics{ + Size: "Large", + Temperament: "Outgoing and Friendly", + Lifespan: "10-13 years", }, }, }, - } - - results = append(results, author) + }, } } - return &productv1.MutationBulkCreateAuthorsResponse{ - BulkCreateAuthors: results, + return &productv1.QueryRandomPetResponse{ + RandomPet: pet, }, nil } -func (s *MockService) MutationBulkUpdateAuthors(ctx context.Context, in *productv1.MutationBulkUpdateAuthorsRequest) (*productv1.MutationBulkUpdateAuthorsResponse, error) { - var results []*productv1.Author - - // Handle nullable list - if authors is nil, return empty results - if in.Authors == nil { - return &productv1.MutationBulkUpdateAuthorsResponse{ - BulkUpdateAuthors: results, - }, nil - } - - // Process each author input in the list - if in.Authors.List != nil { - for i, authorInput := range in.Authors.List.Items { - // Convert nested UserInput lists to Users for complex fields - var authorGroups *productv1.ListOfListOfUser - if authorInput.AuthorGroups != nil { - authorGroups = convertNestedUserInputsToUsers(authorInput.AuthorGroups) - } - - var projectTeams *productv1.ListOfListOfUser - if authorInput.ProjectTeams != nil { - projectTeams = convertNestedUserInputsToUsers(authorInput.ProjectTeams) - } - - // Convert CategoryInput list to Categories - var favoriteCategories []*productv1.Category - if authorInput.FavoriteCategories != nil { - favoriteCategories = convertCategoryInputsToCategories(authorInput.FavoriteCategories) - } +func (s *MockService) QueryAllPets(ctx context.Context, in *productv1.QueryAllPetsRequest) (*productv1.QueryAllPetsResponse, error) { + // Create a mix of cats and dogs + var pets []*productv1.Animal - author := &productv1.Author{ - Id: fmt.Sprintf("bulk-updated-author-%d", i+1), - Name: authorInput.Name, - Email: authorInput.Email, - Skills: authorInput.Skills, - Languages: authorInput.Languages, - SocialLinks: authorInput.SocialLinks, - TeamsByProject: authorInput.TeamsByProject, - Collaborations: authorInput.Collaborations, - FavoriteCategories: favoriteCategories, - AuthorGroups: authorGroups, - ProjectTeams: projectTeams, - // Add required complex fields with mock data - WrittenPosts: &productv1.ListOfBlogPost{ - List: &productv1.ListOfBlogPost_List{ - Items: []*productv1.BlogPost{ - {Id: fmt.Sprintf("bulk-updated-blog-%d", i+1), Title: fmt.Sprintf("Bulk Updated Post %d", i+1), Content: "Bulk updated content..."}, + // Add 2 cats + for i := 1; i <= 2; i++ { + pets = append(pets, &productv1.Animal{ + Instance: &productv1.Animal_Cat{ + Cat: &productv1.Cat{ + Id: fmt.Sprintf("cat-%d", i), + Name: fmt.Sprintf("Cat %d", i), + Kind: fmt.Sprintf("Breed %d", i), + MeowVolume: int32(i + 3), // Different volumes + Owner: &productv1.Owner{ + Id: fmt.Sprintf("owner-cat-%d", i), + Name: fmt.Sprintf("Cat Owner %d", i), + Contact: &productv1.ContactInfo{ + Email: fmt.Sprintf("cat-owner-%d@example.com", i), + Phone: fmt.Sprintf("555-%03d-0000", i*100), + Address: &productv1.Address{ + Street: fmt.Sprintf("%d Cat Street", i*100), + City: "Feline City", + Country: "USA", + ZipCode: fmt.Sprintf("%05d", i*10000), + }, }, }, - }, - RelatedAuthors: &productv1.ListOfUser{ - List: &productv1.ListOfUser_List{ - Items: []*productv1.User{ - {Id: fmt.Sprintf("bulk-updated-rel-author-%d", i+1), Name: fmt.Sprintf("Bulk Updated Related Author %d", i+1)}, + Breed: &productv1.CatBreed{ + Id: fmt.Sprintf("breed-cat-%d", i), + Name: fmt.Sprintf("Cat Breed %d", i), + Origin: "Various", + Characteristics: &productv1.BreedCharacteristics{ + Size: "Medium", + Temperament: "Friendly", + Lifespan: "12-18 years", }, }, }, - ProductReviews: &productv1.ListOfProduct{ - List: &productv1.ListOfProduct_List{ - Items: []*productv1.Product{ - {Id: fmt.Sprintf("bulk-updated-prod-%d", i+1), Name: fmt.Sprintf("Bulk Updated Product %d", i+1), Price: 149.99}, + }, + }) + } + + // Add 2 dogs + for i := 1; i <= 2; i++ { + pets = append(pets, &productv1.Animal{ + Instance: &productv1.Animal_Dog{ + Dog: &productv1.Dog{ + Id: fmt.Sprintf("dog-%d", i), + Name: fmt.Sprintf("Dog %d", i), + Kind: fmt.Sprintf("Breed %d", i), + BarkVolume: int32(i + 5), // Different volumes + Owner: &productv1.Owner{ + Id: fmt.Sprintf("owner-dog-%d", i), + Name: fmt.Sprintf("Dog Owner %d", i), + Contact: &productv1.ContactInfo{ + Email: fmt.Sprintf("dog-owner-%d@example.com", i), + Phone: fmt.Sprintf("555-%03d-1111", i*100), + Address: &productv1.Address{ + Street: fmt.Sprintf("%d Dog Avenue", i*200), + City: "Canine City", + Country: "USA", + ZipCode: fmt.Sprintf("%05d", i*20000), + }, }, }, - }, - CategoryPreferences: &productv1.ListOfListOfCategory{ - List: &productv1.ListOfListOfCategory_List{ - Items: []*productv1.ListOfCategory{ - {List: &productv1.ListOfCategory_List{ - Items: []*productv1.Category{ - {Id: fmt.Sprintf("bulk-updated-cat-pref-%d", i+1), Name: fmt.Sprintf("Bulk Updated Category Preference %d", i+1), Kind: productv1.CategoryKind_CATEGORY_KIND_ELECTRONICS}, - }, - }}, + Breed: &productv1.DogBreed{ + Id: fmt.Sprintf("breed-dog-%d", i), + Name: fmt.Sprintf("Dog Breed %d", i), + Origin: "Various", + Characteristics: &productv1.BreedCharacteristics{ + Size: "Large", + Temperament: "Loyal", + Lifespan: "10-14 years", }, }, }, - } - - results = append(results, author) - } + }, + }) } - return &productv1.MutationBulkUpdateAuthorsResponse{ - BulkUpdateAuthors: results, + return &productv1.QueryAllPetsResponse{ + AllPets: pets, }, nil } -func (s *MockService) MutationBulkCreateBlogPosts(ctx context.Context, in *productv1.MutationBulkCreateBlogPostsRequest) (*productv1.MutationBulkCreateBlogPostsResponse, error) { - var results []*productv1.BlogPost - - // Handle nullable list - if blogPosts is nil, return empty results - if in.BlogPosts == nil { - return &productv1.MutationBulkCreateBlogPostsResponse{ - BulkCreateBlogPosts: results, - }, nil - } - - // Process each blog post input in the list - if in.BlogPosts.List != nil { - for i, blogPostInput := range in.BlogPosts.List.Items { - // Convert CategoryInput lists to Categories - var relatedCategories []*productv1.Category - if blogPostInput.RelatedCategories != nil { - relatedCategories = convertCategoryInputListToCategories(blogPostInput.RelatedCategories) - } - - var contributors []*productv1.User - if blogPostInput.Contributors != nil { - contributors = convertUserInputsToUsers(blogPostInput.Contributors) - } - - var categoryGroups *productv1.ListOfListOfCategory - if blogPostInput.CategoryGroups != nil { - categoryGroups = convertNestedCategoryInputsToCategories(blogPostInput.CategoryGroups) - } - - blogPost := &productv1.BlogPost{ - Id: fmt.Sprintf("bulk-created-post-%d", i+1), - Title: blogPostInput.Title, - Content: blogPostInput.Content, - Tags: blogPostInput.Tags, - OptionalTags: blogPostInput.OptionalTags, - Categories: blogPostInput.Categories, - Keywords: blogPostInput.Keywords, - ViewCounts: blogPostInput.ViewCounts, - Ratings: blogPostInput.Ratings, - IsPublished: blogPostInput.IsPublished, - TagGroups: blogPostInput.TagGroups, - RelatedTopics: blogPostInput.RelatedTopics, - CommentThreads: blogPostInput.CommentThreads, - Suggestions: blogPostInput.Suggestions, - RelatedCategories: relatedCategories, - Contributors: contributors, - CategoryGroups: categoryGroups, - // Add required fields with mock data - MentionedProducts: &productv1.ListOfProduct{ - List: &productv1.ListOfProduct_List{ - Items: []*productv1.Product{ - {Id: fmt.Sprintf("bulk-prod-%d", i+1), Name: fmt.Sprintf("Bulk Created Product %d", i+1), Price: 99.99}, - }, - }, - }, - MentionedUsers: &productv1.ListOfUser{ - List: &productv1.ListOfUser_List{ - Items: []*productv1.User{ - {Id: fmt.Sprintf("bulk-user-%d", i+1), Name: fmt.Sprintf("Bulk Created User %d", i+1)}, - }, - }, - }, - ContributorTeams: &productv1.ListOfListOfUser{ - List: &productv1.ListOfListOfUser_List{ - Items: []*productv1.ListOfUser{ - {List: &productv1.ListOfUser_List{ - Items: []*productv1.User{ - {Id: fmt.Sprintf("bulk-team-%d", i+1), Name: fmt.Sprintf("Bulk Created Team Member %d", i+1)}, - }, - }}, - }, - }, - }, - } +// Implementation for CreateUser mutation +func (s *MockService) MutationCreateUser(ctx context.Context, in *productv1.MutationCreateUserRequest) (*productv1.MutationCreateUserResponse, error) { + input := in.GetInput() - results = append(results, blogPost) - } + // Create a new user with the input name and a random ID + user := &productv1.User{ + Id: fmt.Sprintf("user-%d", rand.Intn(1000)), + Name: input.GetName(), } - return &productv1.MutationBulkCreateBlogPostsResponse{ - BulkCreateBlogPosts: results, + return &productv1.MutationCreateUserResponse{ + CreateUser: user, }, nil } -func (s *MockService) MutationBulkUpdateBlogPosts(ctx context.Context, in *productv1.MutationBulkUpdateBlogPostsRequest) (*productv1.MutationBulkUpdateBlogPostsResponse, error) { - var results []*productv1.BlogPost - - // Handle nullable list - if blogPosts is nil, return empty results - if in.BlogPosts == nil { - return &productv1.MutationBulkUpdateBlogPostsResponse{ - BulkUpdateBlogPosts: results, - }, nil - } - - // Process each blog post input in the list - if in.BlogPosts.List != nil { - for i, blogPostInput := range in.BlogPosts.List.Items { - // Convert CategoryInput lists to Categories - var relatedCategories []*productv1.Category - if blogPostInput.RelatedCategories != nil { - relatedCategories = convertCategoryInputListToCategories(blogPostInput.RelatedCategories) - } +// Implementation for QueryCalculateTotals +func (s *MockService) QueryCalculateTotals(ctx context.Context, in *productv1.QueryCalculateTotalsRequest) (*productv1.QueryCalculateTotalsResponse, error) { + orders := in.GetOrders() + var calculatedOrders []*productv1.Order - var contributors []*productv1.User - if blogPostInput.Contributors != nil { - contributors = convertUserInputsToUsers(blogPostInput.Contributors) - } + for _, orderInput := range orders { + // Calculate total items by summing up quantities from all order lines + var totalItems int32 + for _, line := range orderInput.GetLines() { + totalItems += line.GetQuantity() + } - var categoryGroups *productv1.ListOfListOfCategory - if blogPostInput.CategoryGroups != nil { - categoryGroups = convertNestedCategoryInputsToCategories(blogPostInput.CategoryGroups) - } + orderLines := []*productv1.OrderLine{} + for _, line := range orderInput.GetLines() { + orderLines = append(orderLines, &productv1.OrderLine{ + ProductId: line.GetProductId(), + Quantity: line.GetQuantity(), + Modifiers: line.GetModifiers(), + }) + } - blogPost := &productv1.BlogPost{ - Id: fmt.Sprintf("bulk-updated-post-%d", i+1), - Title: blogPostInput.Title, - Content: blogPostInput.Content, - Tags: blogPostInput.Tags, - OptionalTags: blogPostInput.OptionalTags, - Categories: blogPostInput.Categories, - Keywords: blogPostInput.Keywords, - ViewCounts: blogPostInput.ViewCounts, - Ratings: blogPostInput.Ratings, - IsPublished: blogPostInput.IsPublished, - TagGroups: blogPostInput.TagGroups, - RelatedTopics: blogPostInput.RelatedTopics, - CommentThreads: blogPostInput.CommentThreads, - Suggestions: blogPostInput.Suggestions, - RelatedCategories: relatedCategories, - Contributors: contributors, - CategoryGroups: categoryGroups, - // Add required fields with mock data - MentionedProducts: &productv1.ListOfProduct{ - List: &productv1.ListOfProduct_List{ - Items: []*productv1.Product{ - {Id: fmt.Sprintf("bulk-updated-prod-%d", i+1), Name: fmt.Sprintf("Bulk Updated Product %d", i+1), Price: 149.99}, - }, - }, - }, - MentionedUsers: &productv1.ListOfUser{ - List: &productv1.ListOfUser_List{ - Items: []*productv1.User{ - {Id: fmt.Sprintf("bulk-updated-user-%d", i+1), Name: fmt.Sprintf("Bulk Updated User %d", i+1)}, - }, - }, - }, - ContributorTeams: &productv1.ListOfListOfUser{ - List: &productv1.ListOfListOfUser_List{ - Items: []*productv1.ListOfUser{ - {List: &productv1.ListOfUser_List{ - Items: []*productv1.User{ - {Id: fmt.Sprintf("bulk-updated-team-%d", i+1), Name: fmt.Sprintf("Bulk Updated Team Member %d", i+1)}, - }, - }}, - }, - }, + calculatedOrders = append(calculatedOrders, &productv1.Order{ + OrderId: orderInput.GetOrderId(), + CustomerName: orderInput.GetCustomerName(), + TotalItems: totalItems, + OrderLines: &productv1.ListOfOrderLine{ + List: &productv1.ListOfOrderLine_List{ + Items: orderLines, }, - } - - results = append(results, blogPost) - } + }, + }) } - return &productv1.MutationBulkUpdateBlogPostsResponse{ - BulkUpdateBlogPosts: results, + return &productv1.QueryCalculateTotalsResponse{ + CalculateTotals: calculatedOrders, }, nil } diff --git a/v2/pkg/grpctest/mockservice_enums.go b/v2/pkg/grpctest/mockservice_enums.go new file mode 100644 index 000000000..9e50197a7 --- /dev/null +++ b/v2/pkg/grpctest/mockservice_enums.go @@ -0,0 +1,143 @@ +package grpctest + +import ( + context "context" + "fmt" + + "github.com/wundergraph/graphql-go-tools/v2/pkg/grpctest/productv1" + "google.golang.org/protobuf/types/known/wrapperspb" +) + +// Implementation for QueryCategories +func (s *MockService) QueryCategories(ctx context.Context, in *productv1.QueryCategoriesRequest) (*productv1.QueryCategoriesResponse, error) { + // Generate a list of categories + var categories []*productv1.Category + + // Create sample categories for each CategoryKind + categoryKinds := []productv1.CategoryKind{ + productv1.CategoryKind_CATEGORY_KIND_BOOK, + productv1.CategoryKind_CATEGORY_KIND_ELECTRONICS, + productv1.CategoryKind_CATEGORY_KIND_FURNITURE, + productv1.CategoryKind_CATEGORY_KIND_OTHER, + } + + for i, kind := range categoryKinds { + categories = append(categories, &productv1.Category{ + Id: fmt.Sprintf("category-%d", i+1), + Name: fmt.Sprintf("%s Category", kind.String()), + Kind: kind, + Subcategories: createSubcategories(fmt.Sprintf("category-%d", i+1), kind, i+1), + }) + } + + return &productv1.QueryCategoriesResponse{ + Categories: categories, + }, nil +} + +// Implementation for QueryCategoriesByKind +func (s *MockService) QueryCategoriesByKind(ctx context.Context, in *productv1.QueryCategoriesByKindRequest) (*productv1.QueryCategoriesByKindResponse, error) { + kind := in.GetKind() + + // Generate categories for the specified kind + var categories []*productv1.Category + + // Create 3 categories of the requested kind + for i := 1; i <= 3; i++ { + + subcategoties := make([]*productv1.Subcategory, 0, i) + for j := 1; j <= i; j++ { + subcategoties = append(subcategoties, &productv1.Subcategory{ + Id: fmt.Sprintf("%s-subcategory-%d", kind.String(), j), + Name: fmt.Sprintf("%s Subcategory %d", kind.String(), j), + Description: &wrapperspb.StringValue{Value: fmt.Sprintf("%s Subcategory %d", kind.String(), j)}, + IsActive: true, + }) + } + + categories = append(categories, &productv1.Category{ + Id: fmt.Sprintf("%s-category-%d", kind.String(), i), + Name: fmt.Sprintf("%s Category %d", kind.String(), i), + Kind: kind, + Subcategories: &productv1.ListOfSubcategory{ + List: &productv1.ListOfSubcategory_List{ + Items: subcategoties, + }, + }, + }) + } + + return &productv1.QueryCategoriesByKindResponse{ + CategoriesByKind: categories, + }, nil +} + +func (s *MockService) QueryCategoriesByKinds(ctx context.Context, in *productv1.QueryCategoriesByKindsRequest) (*productv1.QueryCategoriesByKindsResponse, error) { + kinds := in.GetKinds() + + var categories []*productv1.Category + + for i, kind := range kinds { + categories = append(categories, &productv1.Category{ + Id: fmt.Sprintf("%s-category-%d", kind.String(), i), + Name: fmt.Sprintf("%s Category %d", kind.String(), i), + Kind: kind, + Subcategories: createSubcategories(fmt.Sprintf("%s-category-%d", kind.String(), i), kind, i+1), + }) + } + + return &productv1.QueryCategoriesByKindsResponse{ + CategoriesByKinds: categories, + }, nil +} + +// Implementation for QueryFilterCategories +func (s *MockService) QueryFilterCategories(ctx context.Context, in *productv1.QueryFilterCategoriesRequest) (*productv1.QueryFilterCategoriesResponse, error) { + filter := in.GetFilter() + + if filter == nil { + return &productv1.QueryFilterCategoriesResponse{ + FilterCategories: []*productv1.Category{}, + }, nil + } + + kind := filter.GetCategory() + + // Generate filtered categories + var categories []*productv1.Category + + // Create categories that match the filter + for i := 1; i <= 5; i++ { + categories = append(categories, &productv1.Category{ + Id: fmt.Sprintf("filtered-%s-category-%d", kind.String(), i), + Name: fmt.Sprintf("Filtered %s Category %d", kind.String(), i), + Kind: kind, + Subcategories: createSubcategories(fmt.Sprintf("filtered-%s-category-%d", kind.String(), i), kind, i), + }) + } + + // Apply pagination if provided + pagination := filter.GetPagination() + if pagination != nil { + page := int(pagination.GetPage()) + perPage := int(pagination.GetPerPage()) + + if page > 0 && perPage > 0 && len(categories) > perPage { + startIdx := (page - 1) * perPage + endIdx := startIdx + perPage + + if startIdx < len(categories) { + if endIdx > len(categories) { + endIdx = len(categories) + } + categories = categories[startIdx:endIdx] + } else { + categories = []*productv1.Category{} + } + } + } + + return &productv1.QueryFilterCategoriesResponse{ + FilterCategories: categories, + }, nil +} diff --git a/v2/pkg/grpctest/mockservice_lists.go b/v2/pkg/grpctest/mockservice_lists.go new file mode 100644 index 000000000..09eec065f --- /dev/null +++ b/v2/pkg/grpctest/mockservice_lists.go @@ -0,0 +1,1751 @@ +package grpctest + +import ( + context "context" + "fmt" + "math/rand" + + "github.com/wundergraph/graphql-go-tools/v2/pkg/grpctest/productv1" + "google.golang.org/protobuf/types/known/wrapperspb" +) + +// BlogPost query implementations +func (s *MockService) QueryBlogPost(ctx context.Context, in *productv1.QueryBlogPostRequest) (*productv1.QueryBlogPostResponse, error) { + // Return a default blog post with comprehensive list examples + result := &productv1.BlogPost{ + Id: "blog-default", + Title: "Default Blog Post", + Content: "This is a sample blog post content for testing nested lists.", + Tags: []string{"tech", "programming", "go"}, + OptionalTags: &productv1.ListOfString{ + List: &productv1.ListOfString_List{ + Items: []string{"optional1", "optional2"}, + }, + }, + Categories: []string{"Technology", "", "Programming"}, // includes null/empty + Keywords: &productv1.ListOfString{ + List: &productv1.ListOfString_List{ + Items: []string{"keyword1", "keyword2"}, + }, + }, + ViewCounts: []int32{100, 150, 200, 250}, + Ratings: &productv1.ListOfFloat{ + List: &productv1.ListOfFloat_List{ + Items: []float64{4.5, 3.8, 5.0}, + }, + }, + IsPublished: &productv1.ListOfBoolean{ + List: &productv1.ListOfBoolean_List{ + Items: []bool{false, true, true}, + }, + }, + TagGroups: &productv1.ListOfListOfString{ + List: &productv1.ListOfListOfString_List{ + Items: []*productv1.ListOfString{ + {List: &productv1.ListOfString_List{ + Items: []string{"tech", "programming"}, + }}, + {List: &productv1.ListOfString_List{ + Items: []string{"golang", "backend"}, + }}, + }, + }, + }, + RelatedTopics: &productv1.ListOfListOfString{ + List: &productv1.ListOfListOfString_List{ + Items: []*productv1.ListOfString{ + {List: &productv1.ListOfString_List{Items: []string{"microservices", "api"}}}, + {List: &productv1.ListOfString_List{Items: []string{"databases", "performance"}}}, + }, + }, + }, + CommentThreads: &productv1.ListOfListOfString{ + List: &productv1.ListOfListOfString_List{ + Items: []*productv1.ListOfString{ + {List: &productv1.ListOfString_List{Items: []string{"Great post!", "Very helpful"}}}, + {List: &productv1.ListOfString_List{Items: []string{"Could use more examples", "Thanks for sharing"}}}, + }, + }, + }, + Suggestions: &productv1.ListOfListOfString{ + List: &productv1.ListOfListOfString_List{ + Items: []*productv1.ListOfString{ + {List: &productv1.ListOfString_List{Items: []string{"Add code examples", "Include diagrams"}}}, + }, + }, + }, + RelatedCategories: []*productv1.Category{ + {Id: "cat-1", Name: "Technology", Kind: productv1.CategoryKind_CATEGORY_KIND_ELECTRONICS}, + {Id: "cat-2", Name: "Programming", Kind: productv1.CategoryKind_CATEGORY_KIND_BOOK}, + }, + Contributors: []*productv1.User{ + {Id: "user-1", Name: "John Doe"}, + {Id: "user-2", Name: "Jane Smith"}, + }, + MentionedProducts: &productv1.ListOfProduct{ + List: &productv1.ListOfProduct_List{ + Items: []*productv1.Product{ + {Id: "prod-1", Name: "Sample Product", Price: 99.99}, + }, + }, + }, + MentionedUsers: &productv1.ListOfUser{ + List: &productv1.ListOfUser_List{ + Items: []*productv1.User{ + {Id: "user-3", Name: "Bob Johnson"}, + }, + }, + }, + CategoryGroups: &productv1.ListOfListOfCategory{ + List: &productv1.ListOfListOfCategory_List{ + Items: []*productv1.ListOfCategory{ + {List: &productv1.ListOfCategory_List{ + Items: []*productv1.Category{ + {Id: "cat-3", Name: "Web Development", Kind: productv1.CategoryKind_CATEGORY_KIND_ELECTRONICS}, + {Id: "cat-4", Name: "Backend", Kind: productv1.CategoryKind_CATEGORY_KIND_ELECTRONICS}, + }, + }}, + }, + }, + }, + ContributorTeams: &productv1.ListOfListOfUser{ + List: &productv1.ListOfListOfUser_List{ + Items: []*productv1.ListOfUser{ + {List: &productv1.ListOfUser_List{ + Items: []*productv1.User{ + {Id: "user-4", Name: "Alice Brown"}, + {Id: "user-5", Name: "Charlie Wilson"}, + }, + }}, + }, + }, + }, + } + + return &productv1.QueryBlogPostResponse{ + BlogPost: result, + }, nil +} + +func (s *MockService) QueryBlogPostById(ctx context.Context, in *productv1.QueryBlogPostByIdRequest) (*productv1.QueryBlogPostByIdResponse, error) { + id := in.GetId() + + // Return null for specific test IDs + if id == "not-found" { + return &productv1.QueryBlogPostByIdResponse{ + BlogPostById: nil, + }, nil + } + + // Create different test data based on ID + var result *productv1.BlogPost + + switch id { + case "simple": + result = &productv1.BlogPost{ + Id: id, + Title: "Simple Post", + Content: "Simple content", + Tags: []string{"simple"}, + Categories: []string{"Basic"}, + ViewCounts: []int32{10}, + // Required nested lists must have data + TagGroups: &productv1.ListOfListOfString{ + List: &productv1.ListOfListOfString_List{ + Items: []*productv1.ListOfString{ + {List: &productv1.ListOfString_List{Items: []string{"simple"}}}, + }, + }, + }, + RelatedTopics: &productv1.ListOfListOfString{ + List: &productv1.ListOfListOfString_List{ + Items: []*productv1.ListOfString{ + {List: &productv1.ListOfString_List{Items: []string{"basic"}}}, + }, + }, + }, + CommentThreads: &productv1.ListOfListOfString{ + List: &productv1.ListOfListOfString_List{ + Items: []*productv1.ListOfString{ + {List: &productv1.ListOfString_List{Items: []string{"Nice post"}}}, + }, + }, + }, + // Required complex lists must have data + RelatedCategories: []*productv1.Category{ + {Id: "cat-simple", Name: "Basic", Kind: productv1.CategoryKind_CATEGORY_KIND_OTHER}, + }, + Contributors: []*productv1.User{ + {Id: "user-simple", Name: "Simple Author"}, + }, + CategoryGroups: &productv1.ListOfListOfCategory{ + List: &productv1.ListOfListOfCategory_List{ + Items: []*productv1.ListOfCategory{ + {List: &productv1.ListOfCategory_List{ + Items: []*productv1.Category{ + {Id: "cat-group-simple", Name: "Simple Category", Kind: productv1.CategoryKind_CATEGORY_KIND_OTHER}, + }, + }}, + }, + }, + }, + } + case "complex": + result = &productv1.BlogPost{ + Id: id, + Title: "Complex Blog Post", + Content: "Complex content with comprehensive lists", + Tags: []string{"complex", "advanced", "detailed"}, + OptionalTags: &productv1.ListOfString{ + List: &productv1.ListOfString_List{ + Items: []string{"deep-dive", "tutorial"}, + }, + }, + Categories: []string{"Advanced", "Tutorial", "Guide"}, + Keywords: &productv1.ListOfString{ + List: &productv1.ListOfString_List{ + Items: []string{"advanced", "complex", "comprehensive"}, + }, + }, + ViewCounts: []int32{500, 600, 750, 800, 950}, + Ratings: &productv1.ListOfFloat{ + List: &productv1.ListOfFloat_List{ + Items: []float64{4.8, 4.9, 4.7, 5.0}, + }, + }, + IsPublished: &productv1.ListOfBoolean{ + List: &productv1.ListOfBoolean_List{ + Items: []bool{false, false, true, true}, + }, + }, + TagGroups: &productv1.ListOfListOfString{ + List: &productv1.ListOfListOfString_List{ + Items: []*productv1.ListOfString{ + {List: &productv1.ListOfString_List{Items: []string{"advanced", "expert"}}}, + {List: &productv1.ListOfString_List{Items: []string{"tutorial", "guide", "comprehensive"}}}, + {List: &productv1.ListOfString_List{Items: []string{"deep-dive", "detailed"}}}, + }, + }, + }, + RelatedTopics: &productv1.ListOfListOfString{ + List: &productv1.ListOfListOfString_List{ + Items: []*productv1.ListOfString{ + {List: &productv1.ListOfString_List{Items: []string{"architecture", "patterns", "design"}}}, + {List: &productv1.ListOfString_List{Items: []string{"optimization", "performance", "scaling"}}}, + }, + }, + }, + CommentThreads: &productv1.ListOfListOfString{ + List: &productv1.ListOfListOfString_List{ + Items: []*productv1.ListOfString{ + {List: &productv1.ListOfString_List{Items: []string{"Excellent deep dive!", "Very thorough"}}}, + {List: &productv1.ListOfString_List{Items: []string{"Could be longer", "More examples please"}}}, + {List: &productv1.ListOfString_List{Items: []string{"Best tutorial I've read", "Thank you!"}}}, + }, + }, + }, + Suggestions: &productv1.ListOfListOfString{ + List: &productv1.ListOfListOfString_List{ + Items: []*productv1.ListOfString{ + {List: &productv1.ListOfString_List{Items: []string{"Add video content", "Include interactive examples"}}}, + {List: &productv1.ListOfString_List{Items: []string{"Create follow-up posts", "Add Q&A section"}}}, + }, + }, + }, + // Complex example includes all new complex list fields + RelatedCategories: []*productv1.Category{ + {Id: "cat-complex-1", Name: "Advanced Programming", Kind: productv1.CategoryKind_CATEGORY_KIND_ELECTRONICS}, + {Id: "cat-complex-2", Name: "Software Architecture", Kind: productv1.CategoryKind_CATEGORY_KIND_BOOK}, + }, + Contributors: []*productv1.User{ + {Id: "user-complex-1", Name: "Expert Author"}, + {Id: "user-complex-2", Name: "Technical Reviewer"}, + }, + MentionedProducts: &productv1.ListOfProduct{ + List: &productv1.ListOfProduct_List{ + Items: []*productv1.Product{ + {Id: "prod-complex-1", Name: "Advanced IDE", Price: 299.99}, + {Id: "prod-complex-2", Name: "Profiling Tool", Price: 149.99}, + }, + }, + }, + MentionedUsers: &productv1.ListOfUser{ + List: &productv1.ListOfUser_List{ + Items: []*productv1.User{ + {Id: "user-complex-3", Name: "Referenced Expert"}, + }, + }, + }, + CategoryGroups: &productv1.ListOfListOfCategory{ + List: &productv1.ListOfListOfCategory_List{ + Items: []*productv1.ListOfCategory{ + {List: &productv1.ListOfCategory_List{ + Items: []*productv1.Category{ + {Id: "cat-group-1", Name: "System Design", Kind: productv1.CategoryKind_CATEGORY_KIND_ELECTRONICS}, + {Id: "cat-group-2", Name: "Architecture Patterns", Kind: productv1.CategoryKind_CATEGORY_KIND_BOOK}, + }, + }}, + {List: &productv1.ListOfCategory_List{ + Items: []*productv1.Category{ + {Id: "cat-group-3", Name: "Performance", Kind: productv1.CategoryKind_CATEGORY_KIND_ELECTRONICS}, + }, + }}, + }, + }, + }, + ContributorTeams: &productv1.ListOfListOfUser{ + List: &productv1.ListOfListOfUser_List{ + Items: []*productv1.ListOfUser{ + {List: &productv1.ListOfUser_List{ + Items: []*productv1.User{ + {Id: "team-complex-1", Name: "Senior Engineer A"}, + {Id: "team-complex-2", Name: "Senior Engineer B"}, + }, + }}, + {List: &productv1.ListOfUser_List{ + Items: []*productv1.User{ + {Id: "team-complex-3", Name: "QA Lead"}, + }, + }}, + }, + }, + }, + } + default: + // Generic response for any other ID + result = &productv1.BlogPost{ + Id: id, + Title: fmt.Sprintf("Blog Post %s", id), + Content: fmt.Sprintf("Content for blog post %s", id), + Tags: []string{fmt.Sprintf("tag-%s", id), "general"}, + Categories: []string{"General", fmt.Sprintf("Category-%s", id)}, + ViewCounts: []int32{int32(len(id) * 10), int32(len(id) * 20)}, + // Required nested lists must have data + TagGroups: &productv1.ListOfListOfString{ + List: &productv1.ListOfListOfString_List{ + Items: []*productv1.ListOfString{ + {List: &productv1.ListOfString_List{ + Items: []string{fmt.Sprintf("tag-%s", id), "group"}, + }}, + }, + }, + }, + RelatedTopics: &productv1.ListOfListOfString{ + List: &productv1.ListOfListOfString_List{ + Items: []*productv1.ListOfString{ + {List: &productv1.ListOfString_List{ + Items: []string{fmt.Sprintf("topic-%s", id)}, + }}, + }, + }, + }, + CommentThreads: &productv1.ListOfListOfString{ + List: &productv1.ListOfListOfString_List{ + Items: []*productv1.ListOfString{ + {List: &productv1.ListOfString_List{ + Items: []string{fmt.Sprintf("Comment on %s", id)}, + }}, + }, + }, + }, + // Required complex lists must have data + RelatedCategories: []*productv1.Category{ + {Id: fmt.Sprintf("cat-%s", id), Name: fmt.Sprintf("Category %s", id), Kind: productv1.CategoryKind_CATEGORY_KIND_OTHER}, + }, + Contributors: []*productv1.User{ + {Id: fmt.Sprintf("user-%s", id), Name: fmt.Sprintf("Author %s", id)}, + }, + CategoryGroups: &productv1.ListOfListOfCategory{ + List: &productv1.ListOfListOfCategory_List{ + Items: []*productv1.ListOfCategory{ + {List: &productv1.ListOfCategory_List{ + Items: []*productv1.Category{ + {Id: fmt.Sprintf("cat-group-%s", id), Name: fmt.Sprintf("Group Category %s", id), Kind: productv1.CategoryKind_CATEGORY_KIND_OTHER}, + }, + }}, + }, + }, + }, + } + } + + return &productv1.QueryBlogPostByIdResponse{ + BlogPostById: result, + }, nil +} + +func (s *MockService) QueryBlogPostsWithFilter(ctx context.Context, in *productv1.QueryBlogPostsWithFilterRequest) (*productv1.QueryBlogPostsWithFilterResponse, error) { + filter := in.GetFilter() + var results []*productv1.BlogPost + + // If no filter provided, return empty results + if filter == nil { + return &productv1.QueryBlogPostsWithFilterResponse{ + BlogPostsWithFilter: results, + }, nil + } + + titleFilter := "" + if filter.Title != nil { + titleFilter = filter.Title.GetValue() + } + + hasCategories := false + if filter.HasCategories != nil { + hasCategories = filter.HasCategories.GetValue() + } + + minTags := int32(0) + if filter.MinTags != nil { + minTags = filter.MinTags.GetValue() + } + + // Generate filtered results + for i := 1; i <= 3; i++ { + title := fmt.Sprintf("Filtered Post %d", i) + if titleFilter != "" { + title = fmt.Sprintf("%s - Post %d", titleFilter, i) + } + + var tags []string + tagsCount := minTags + int32(i) + for j := int32(0); j < tagsCount; j++ { + tags = append(tags, fmt.Sprintf("tag%d", j+1)) + } + + var categories []string + if hasCategories { + categories = []string{fmt.Sprintf("Category%d", i), "Filtered"} + } + + results = append(results, &productv1.BlogPost{ + Id: fmt.Sprintf("filtered-blog-%d", i), + Title: title, + Content: fmt.Sprintf("Filtered content %d", i), + Tags: tags, + Categories: categories, + ViewCounts: []int32{int32(i * 100)}, + // Required nested lists must have data + TagGroups: &productv1.ListOfListOfString{ + List: &productv1.ListOfListOfString_List{ + Items: []*productv1.ListOfString{ + {List: &productv1.ListOfString_List{ + Items: []string{fmt.Sprintf("filtered-tag-%d", i)}, + }}, + }, + }, + }, + RelatedTopics: &productv1.ListOfListOfString{ + List: &productv1.ListOfListOfString_List{ + Items: []*productv1.ListOfString{ + {List: &productv1.ListOfString_List{ + Items: []string{fmt.Sprintf("filtered-topic-%d", i)}, + }}, + }, + }, + }, + CommentThreads: &productv1.ListOfListOfString{ + List: &productv1.ListOfListOfString_List{ + Items: []*productv1.ListOfString{ + {List: &productv1.ListOfString_List{ + Items: []string{fmt.Sprintf("Filtered comment %d", i)}, + }}, + }, + }, + }, + // Required complex lists must have data + RelatedCategories: []*productv1.Category{ + {Id: fmt.Sprintf("cat-filtered-%d", i), Name: fmt.Sprintf("Filtered Category %d", i), Kind: productv1.CategoryKind_CATEGORY_KIND_OTHER}, + }, + Contributors: []*productv1.User{ + {Id: fmt.Sprintf("user-filtered-%d", i), Name: fmt.Sprintf("Filtered Author %d", i)}, + }, + CategoryGroups: &productv1.ListOfListOfCategory{ + List: &productv1.ListOfListOfCategory_List{ + Items: []*productv1.ListOfCategory{ + {List: &productv1.ListOfCategory_List{ + Items: []*productv1.Category{ + {Id: fmt.Sprintf("cat-group-filtered-%d", i), Name: fmt.Sprintf("Filtered Group %d", i), Kind: productv1.CategoryKind_CATEGORY_KIND_OTHER}, + }, + }}, + }, + }, + }, + }) + } + + return &productv1.QueryBlogPostsWithFilterResponse{ + BlogPostsWithFilter: results, + }, nil +} + +func (s *MockService) QueryAllBlogPosts(ctx context.Context, in *productv1.QueryAllBlogPostsRequest) (*productv1.QueryAllBlogPostsResponse, error) { + var results []*productv1.BlogPost + + // Create a variety of blog posts + for i := 1; i <= 4; i++ { + var optionalTags *productv1.ListOfString + var keywords *productv1.ListOfString + var ratings *productv1.ListOfFloat + + // Vary the optional fields + if i%2 == 1 { + optionalTags = &productv1.ListOfString{ + List: &productv1.ListOfString_List{ + Items: []string{fmt.Sprintf("optional%d", i), "common"}, + }, + } + } + + if i%3 == 0 { + keywords = &productv1.ListOfString{ + List: &productv1.ListOfString_List{ + Items: []string{fmt.Sprintf("keyword%d", i)}, + }, + } + } + + if i%2 == 0 { + ratings = &productv1.ListOfFloat{ + List: &productv1.ListOfFloat_List{ + Items: []float64{float64(i) + 0.5, float64(i) + 1.0}, + }, + } + } + + results = append(results, &productv1.BlogPost{ + Id: fmt.Sprintf("blog-%d", i), + Title: fmt.Sprintf("Blog Post %d", i), + Content: fmt.Sprintf("Content for blog post %d", i), + Tags: []string{fmt.Sprintf("tag%d", i), "common"}, + OptionalTags: optionalTags, + Categories: []string{fmt.Sprintf("Category%d", i)}, + Keywords: keywords, + ViewCounts: []int32{int32(i * 100), int32(i * 150)}, + Ratings: ratings, + IsPublished: &productv1.ListOfBoolean{ + List: &productv1.ListOfBoolean_List{ + Items: []bool{i%2 == 0, true}, + }, + }, + TagGroups: &productv1.ListOfListOfString{ + List: &productv1.ListOfListOfString_List{ + Items: []*productv1.ListOfString{ + {List: &productv1.ListOfString_List{ + Items: []string{fmt.Sprintf("group%d", i), "shared"}, + }}, + }, + }, + }, + RelatedTopics: &productv1.ListOfListOfString{ + List: &productv1.ListOfListOfString_List{ + Items: []*productv1.ListOfString{ + {List: &productv1.ListOfString_List{ + Items: []string{fmt.Sprintf("topic%d", i)}, + }}, + }, + }, + }, + CommentThreads: &productv1.ListOfListOfString{ + List: &productv1.ListOfListOfString_List{ + Items: []*productv1.ListOfString{ + {List: &productv1.ListOfString_List{ + Items: []string{fmt.Sprintf("Comment for post %d", i)}, + }}, + }, + }, + }, + // Required complex lists must have data + RelatedCategories: []*productv1.Category{ + {Id: fmt.Sprintf("cat-all-%d", i), Name: fmt.Sprintf("Category %d", i), Kind: productv1.CategoryKind_CATEGORY_KIND_OTHER}, + }, + Contributors: []*productv1.User{ + {Id: fmt.Sprintf("user-all-%d", i), Name: fmt.Sprintf("Author %d", i)}, + }, + CategoryGroups: &productv1.ListOfListOfCategory{ + List: &productv1.ListOfListOfCategory_List{ + Items: []*productv1.ListOfCategory{ + {List: &productv1.ListOfCategory_List{ + Items: []*productv1.Category{ + {Id: fmt.Sprintf("cat-group-all-%d", i), Name: fmt.Sprintf("Group Category %d", i), Kind: productv1.CategoryKind_CATEGORY_KIND_OTHER}, + }, + }}, + }, + }, + }, + // Optional list - can be empty + Suggestions: &productv1.ListOfListOfString{}, + }) + } + + return &productv1.QueryAllBlogPostsResponse{ + AllBlogPosts: results, + }, nil +} + +// Author query implementations +func (s *MockService) QueryAuthor(ctx context.Context, in *productv1.QueryAuthorRequest) (*productv1.QueryAuthorResponse, error) { + result := &productv1.Author{ + Id: "author-default", + Name: "Default Author", + Email: &wrapperspb.StringValue{ + Value: "author@example.com", + }, + Skills: []string{"Go", "GraphQL", "Protocol Buffers"}, + Languages: []string{"English", "Spanish", ""}, + SocialLinks: &productv1.ListOfString{ + List: &productv1.ListOfString_List{ + Items: []string{"https://twitter.com/author", "https://linkedin.com/in/author"}, + }, + }, + TeamsByProject: &productv1.ListOfListOfString{ + List: &productv1.ListOfListOfString_List{ + Items: []*productv1.ListOfString{ + {List: &productv1.ListOfString_List{ + Items: []string{"Alice", "Bob", "Charlie"}, + }}, + {List: &productv1.ListOfString_List{ + Items: []string{"David", "Eve"}, + }}, + }, + }, + }, + Collaborations: &productv1.ListOfListOfString{ + List: &productv1.ListOfListOfString_List{ + Items: []*productv1.ListOfString{ + {List: &productv1.ListOfString_List{ + Items: []string{"Open Source Project A", "Research Paper B"}, + }}, + {List: &productv1.ListOfString_List{ + Items: []string{"Conference Talk C"}, + }}, + }, + }, + }, + WrittenPosts: &productv1.ListOfBlogPost{ + List: &productv1.ListOfBlogPost_List{ + Items: []*productv1.BlogPost{ + {Id: "blog-1", Title: "GraphQL Best Practices", Content: "Content here..."}, + {Id: "blog-2", Title: "gRPC vs REST", Content: "Comparison content..."}, + }, + }, + }, + FavoriteCategories: []*productv1.Category{ + {Id: "cat-fav-1", Name: "Software Engineering", Kind: productv1.CategoryKind_CATEGORY_KIND_ELECTRONICS}, + {Id: "cat-fav-2", Name: "Technical Writing", Kind: productv1.CategoryKind_CATEGORY_KIND_BOOK}, + }, + RelatedAuthors: &productv1.ListOfUser{ + List: &productv1.ListOfUser_List{ + Items: []*productv1.User{ + {Id: "author-rel-1", Name: "Related Author One"}, + {Id: "author-rel-2", Name: "Related Author Two"}, + }, + }, + }, + ProductReviews: &productv1.ListOfProduct{ + List: &productv1.ListOfProduct_List{ + Items: []*productv1.Product{ + {Id: "prod-rev-1", Name: "Code Editor Pro", Price: 199.99}, + }, + }, + }, + AuthorGroups: &productv1.ListOfListOfUser{ + List: &productv1.ListOfListOfUser_List{ + Items: []*productv1.ListOfUser{ + {List: &productv1.ListOfUser_List{ + Items: []*productv1.User{ + {Id: "group-auth-1", Name: "Team Lead Alpha"}, + {Id: "group-auth-2", Name: "Senior Dev Beta"}, + }, + }}, + {List: &productv1.ListOfUser_List{ + Items: []*productv1.User{ + {Id: "group-auth-3", Name: "Junior Dev Gamma"}, + }, + }}, + // empty list + {List: &productv1.ListOfUser_List{}}, + // null item + nil, + }, + }, + }, + CategoryPreferences: &productv1.ListOfListOfCategory{ + List: &productv1.ListOfListOfCategory_List{ + Items: []*productv1.ListOfCategory{ + {List: &productv1.ListOfCategory_List{ + Items: []*productv1.Category{ + {Id: "pref-cat-1", Name: "Microservices", Kind: productv1.CategoryKind_CATEGORY_KIND_ELECTRONICS}, + {Id: "pref-cat-2", Name: "Cloud Computing", Kind: productv1.CategoryKind_CATEGORY_KIND_ELECTRONICS}, + }, + }}, + }, + }, + }, + } + + return &productv1.QueryAuthorResponse{ + Author: result, + }, nil +} + +func (s *MockService) QueryAuthorById(ctx context.Context, in *productv1.QueryAuthorByIdRequest) (*productv1.QueryAuthorByIdResponse, error) { + id := in.GetId() + + // Return null for specific test IDs + if id == "not-found" { + return &productv1.QueryAuthorByIdResponse{ + AuthorById: nil, + }, nil + } + + var result *productv1.Author + + switch id { + case "minimal": + result = &productv1.Author{ + Id: id, + Name: "Minimal Author", + Skills: []string{"Basic"}, + Languages: []string{"English"}, + TeamsByProject: &productv1.ListOfListOfString{ + List: &productv1.ListOfListOfString_List{ + Items: []*productv1.ListOfString{ + {List: &productv1.ListOfString_List{ + Items: []string{"Solo"}, + }}, + }, + }, + }, + // Required complex lists must have data + FavoriteCategories: []*productv1.Category{ + {Id: "cat-minimal", Name: "Basic Category", Kind: productv1.CategoryKind_CATEGORY_KIND_OTHER}, + }, + CategoryPreferences: &productv1.ListOfListOfCategory{ + List: &productv1.ListOfListOfCategory_List{ + Items: []*productv1.ListOfCategory{ + {List: &productv1.ListOfCategory_List{ + Items: []*productv1.Category{ + {Id: "cat-pref-minimal", Name: "Minimal Preference", Kind: productv1.CategoryKind_CATEGORY_KIND_OTHER}, + }, + }}, + }, + }, + }, + // Optional list - can be empty + Collaborations: &productv1.ListOfListOfString{}, + } + case "experienced": + result = &productv1.Author{ + Id: id, + Name: "Experienced Author", + Email: &wrapperspb.StringValue{ + Value: "experienced@example.com", + }, + Skills: []string{"Go", "GraphQL", "gRPC", "Microservices", "Kubernetes"}, + Languages: []string{"English", "French", "German"}, + SocialLinks: &productv1.ListOfString{ + List: &productv1.ListOfString_List{ + Items: []string{ + "https://github.com/experienced", + "https://twitter.com/experienced", + "https://medium.com/@experienced", + }, + }, + }, + TeamsByProject: &productv1.ListOfListOfString{ + List: &productv1.ListOfListOfString_List{ + Items: []*productv1.ListOfString{ + {List: &productv1.ListOfString_List{ + Items: []string{"Senior Dev 1", "Senior Dev 2", "Tech Lead"}, + }}, + {List: &productv1.ListOfString_List{ + Items: []string{"Architect", "Principal Engineer"}, + }}, + {List: &productv1.ListOfString_List{ + Items: []string{"PM", "Designer", "QA Lead"}, + }}, + }, + }, + }, + Collaborations: &productv1.ListOfListOfString{ + List: &productv1.ListOfListOfString_List{ + Items: []*productv1.ListOfString{ + {List: &productv1.ListOfString_List{ + Items: []string{"Major OSS Project", "Industry Standard", "Research Initiative"}, + }}, + {List: &productv1.ListOfString_List{ + Items: []string{"Conference Keynote", "Workshop Series"}, + }}, + }, + }, + }, + // Required complex lists must have data + FavoriteCategories: []*productv1.Category{ + {Id: "cat-experienced-1", Name: "Advanced Programming", Kind: productv1.CategoryKind_CATEGORY_KIND_ELECTRONICS}, + {Id: "cat-experienced-2", Name: "Technical Leadership", Kind: productv1.CategoryKind_CATEGORY_KIND_BOOK}, + }, + CategoryPreferences: &productv1.ListOfListOfCategory{ + List: &productv1.ListOfListOfCategory_List{ + Items: []*productv1.ListOfCategory{ + {List: &productv1.ListOfCategory_List{ + Items: []*productv1.Category{ + {Id: "cat-pref-experienced-1", Name: "System Architecture", Kind: productv1.CategoryKind_CATEGORY_KIND_ELECTRONICS}, + {Id: "cat-pref-experienced-2", Name: "Team Management", Kind: productv1.CategoryKind_CATEGORY_KIND_BOOK}, + }, + }}, + }, + }, + }, + } + default: + result = &productv1.Author{ + Id: id, + Name: fmt.Sprintf("Author %s", id), + Email: &wrapperspb.StringValue{ + Value: fmt.Sprintf("%s@example.com", id), + }, + Skills: []string{fmt.Sprintf("Skill-%s", id), "General"}, + Languages: []string{"English", fmt.Sprintf("Language-%s", id)}, + TeamsByProject: &productv1.ListOfListOfString{ + List: &productv1.ListOfListOfString_List{ + Items: []*productv1.ListOfString{ + {List: &productv1.ListOfString_List{ + Items: []string{fmt.Sprintf("Team-%s", id)}, + }}, + }, + }, + }, + // Required complex lists must have data + FavoriteCategories: []*productv1.Category{ + {Id: fmt.Sprintf("cat-%s", id), Name: fmt.Sprintf("Favorite Category %s", id), Kind: productv1.CategoryKind_CATEGORY_KIND_OTHER}, + }, + CategoryPreferences: &productv1.ListOfListOfCategory{ + List: &productv1.ListOfListOfCategory_List{ + Items: []*productv1.ListOfCategory{ + {List: &productv1.ListOfCategory_List{ + Items: []*productv1.Category{ + {Id: fmt.Sprintf("cat-pref-%s", id), Name: fmt.Sprintf("Preference %s", id), Kind: productv1.CategoryKind_CATEGORY_KIND_OTHER}, + }, + }}, + }, + }, + }, + // Optional list - can be empty + Collaborations: &productv1.ListOfListOfString{}, + } + } + + return &productv1.QueryAuthorByIdResponse{ + AuthorById: result, + }, nil +} + +func (s *MockService) QueryAuthorsWithFilter(ctx context.Context, in *productv1.QueryAuthorsWithFilterRequest) (*productv1.QueryAuthorsWithFilterResponse, error) { + filter := in.GetFilter() + var results []*productv1.Author + + if filter == nil { + return &productv1.QueryAuthorsWithFilterResponse{ + AuthorsWithFilter: results, + }, nil + } + + nameFilter := "" + if filter.Name != nil { + nameFilter = filter.Name.GetValue() + } + + hasTeams := false + if filter.HasTeams != nil { + hasTeams = filter.HasTeams.GetValue() + } + + skillCount := int32(0) + if filter.SkillCount != nil { + skillCount = filter.SkillCount.GetValue() + } + + // Generate filtered results + for i := 1; i <= 3; i++ { + name := fmt.Sprintf("Filtered Author %d", i) + if nameFilter != "" { + name = fmt.Sprintf("%s - Author %d", nameFilter, i) + } + + var skills []string + skillsNeeded := skillCount + int32(i) + for j := int32(0); j < skillsNeeded; j++ { + skills = append(skills, fmt.Sprintf("Skill%d", j+1)) + } + + var teamsByProject *productv1.ListOfListOfString + if hasTeams { + teamsByProject = &productv1.ListOfListOfString{ + List: &productv1.ListOfListOfString_List{ + Items: []*productv1.ListOfString{ + {List: &productv1.ListOfString_List{ + Items: []string{fmt.Sprintf("Team%d", i), "SharedTeam"}, + }}, + }, + }, + } + } else { + teamsByProject = &productv1.ListOfListOfString{List: &productv1.ListOfListOfString_List{}} + } + + results = append(results, &productv1.Author{ + Id: fmt.Sprintf("filtered-author-%d", i), + Name: name, + Skills: skills, + Languages: []string{"English", fmt.Sprintf("Lang%d", i)}, + TeamsByProject: teamsByProject, + // Required complex lists must have data + FavoriteCategories: []*productv1.Category{ + {Id: fmt.Sprintf("cat-filtered-%d", i), Name: fmt.Sprintf("Filtered Category %d", i), Kind: productv1.CategoryKind_CATEGORY_KIND_OTHER}, + }, + CategoryPreferences: &productv1.ListOfListOfCategory{ + List: &productv1.ListOfListOfCategory_List{ + Items: []*productv1.ListOfCategory{ + {List: &productv1.ListOfCategory_List{ + Items: []*productv1.Category{ + {Id: fmt.Sprintf("cat-pref-filtered-%d", i), Name: fmt.Sprintf("Filtered Preference %d", i), Kind: productv1.CategoryKind_CATEGORY_KIND_OTHER}, + }, + }}, + }, + }, + }, + // Optional list - can be empty + Collaborations: &productv1.ListOfListOfString{}, + }) + } + + return &productv1.QueryAuthorsWithFilterResponse{ + AuthorsWithFilter: results, + }, nil +} + +func (s *MockService) QueryAllAuthors(ctx context.Context, in *productv1.QueryAllAuthorsRequest) (*productv1.QueryAllAuthorsResponse, error) { + var results []*productv1.Author + + for i := 1; i <= 3; i++ { + var email *wrapperspb.StringValue + var socialLinks *productv1.ListOfString + var collaborations *productv1.ListOfListOfString + + if i%2 == 1 { + email = &wrapperspb.StringValue{ + Value: fmt.Sprintf("author%d@example.com", i), + } + } + + if i%3 == 0 { + socialLinks = &productv1.ListOfString{ + List: &productv1.ListOfString_List{ + Items: []string{fmt.Sprintf("https://github.com/author%d", i)}, + }, + } + } + + if i == 2 { + collaborations = &productv1.ListOfListOfString{ + List: &productv1.ListOfListOfString_List{ + Items: []*productv1.ListOfString{ + {List: &productv1.ListOfString_List{ + Items: []string{"Collaboration A", "Collaboration B"}, + }}, + }, + }, + } + } else { + collaborations = &productv1.ListOfListOfString{} + } + + results = append(results, &productv1.Author{ + Id: fmt.Sprintf("author-%d", i), + Name: fmt.Sprintf("Author %d", i), + Email: email, + Skills: []string{fmt.Sprintf("Skill%d", i), "Common"}, + Languages: []string{"English", fmt.Sprintf("Language%d", i)}, + SocialLinks: socialLinks, + TeamsByProject: &productv1.ListOfListOfString{ + List: &productv1.ListOfListOfString_List{ + Items: []*productv1.ListOfString{ + {List: &productv1.ListOfString_List{ + Items: []string{fmt.Sprintf("Team%d", i)}, + }}, + }, + }, + }, + // Required complex lists must have data + FavoriteCategories: []*productv1.Category{ + {Id: fmt.Sprintf("cat-all-%d", i), Name: fmt.Sprintf("All Category %d", i), Kind: productv1.CategoryKind_CATEGORY_KIND_OTHER}, + }, + CategoryPreferences: &productv1.ListOfListOfCategory{ + List: &productv1.ListOfListOfCategory_List{ + Items: []*productv1.ListOfCategory{ + {List: &productv1.ListOfCategory_List{ + Items: []*productv1.Category{ + {Id: fmt.Sprintf("cat-pref-all-%d", i), Name: fmt.Sprintf("All Preference %d", i), Kind: productv1.CategoryKind_CATEGORY_KIND_OTHER}, + }, + }}, + }, + }, + }, + // Optional list - can be empty/variable + Collaborations: collaborations, + }) + } + + return &productv1.QueryAllAuthorsResponse{ + AllAuthors: results, + }, nil +} + +// BlogPost mutation implementations +func (s *MockService) MutationCreateBlogPost(ctx context.Context, in *productv1.MutationCreateBlogPostRequest) (*productv1.MutationCreateBlogPostResponse, error) { + input := in.GetInput() + + result := &productv1.BlogPost{ + Id: fmt.Sprintf("blog-%d", rand.Intn(1000)), + Title: input.GetTitle(), + Content: input.GetContent(), + Tags: input.GetTags(), + OptionalTags: input.GetOptionalTags(), + Categories: input.GetCategories(), + Keywords: input.GetKeywords(), + ViewCounts: input.GetViewCounts(), + Ratings: input.GetRatings(), + IsPublished: input.GetIsPublished(), + TagGroups: input.GetTagGroups(), + RelatedTopics: input.GetRelatedTopics(), + CommentThreads: input.GetCommentThreads(), + Suggestions: input.GetSuggestions(), + // Convert input types to output types + RelatedCategories: convertCategoryInputListToCategories(input.GetRelatedCategories()), + Contributors: convertUserInputsToUsers(input.GetContributors()), + CategoryGroups: convertNestedCategoryInputsToCategories(input.GetCategoryGroups()), + MentionedProducts: &productv1.ListOfProduct{ + List: &productv1.ListOfProduct_List{ + Items: []*productv1.Product{ + {Id: "prod-1", Name: "Sample Product", Price: 99.99}, + }, + }, + }, + MentionedUsers: &productv1.ListOfUser{ + List: &productv1.ListOfUser_List{ + Items: []*productv1.User{ + {Id: "user-3", Name: "Bob Johnson"}, + }, + }, + }, + ContributorTeams: &productv1.ListOfListOfUser{ + List: &productv1.ListOfListOfUser_List{ + Items: []*productv1.ListOfUser{ + {List: &productv1.ListOfUser_List{ + Items: []*productv1.User{ + {Id: "user-4", Name: "Alice Brown"}, + }, + }}, + }, + }, + }, + } + + return &productv1.MutationCreateBlogPostResponse{ + CreateBlogPost: result, + }, nil +} + +func (s *MockService) MutationUpdateBlogPost(ctx context.Context, in *productv1.MutationUpdateBlogPostRequest) (*productv1.MutationUpdateBlogPostResponse, error) { + id := in.GetId() + input := in.GetInput() + + if id == "non-existent" { + return &productv1.MutationUpdateBlogPostResponse{ + UpdateBlogPost: nil, + }, nil + } + + result := &productv1.BlogPost{ + Id: id, + Title: input.GetTitle(), + Content: input.GetContent(), + Tags: input.GetTags(), + OptionalTags: input.GetOptionalTags(), + Categories: input.GetCategories(), + Keywords: input.GetKeywords(), + ViewCounts: input.GetViewCounts(), + Ratings: input.GetRatings(), + IsPublished: input.GetIsPublished(), + TagGroups: input.GetTagGroups(), + RelatedTopics: input.GetRelatedTopics(), + CommentThreads: input.GetCommentThreads(), + Suggestions: input.GetSuggestions(), + // Convert input types to output types + RelatedCategories: convertCategoryInputListToCategories(input.GetRelatedCategories()), + Contributors: convertUserInputsToUsers(input.GetContributors()), + CategoryGroups: convertNestedCategoryInputsToCategories(input.GetCategoryGroups()), + MentionedProducts: &productv1.ListOfProduct{ + List: &productv1.ListOfProduct_List{ + Items: []*productv1.Product{ + {Id: "prod-updated", Name: "Updated Product", Price: 149.99}, + }, + }, + }, + MentionedUsers: &productv1.ListOfUser{ + List: &productv1.ListOfUser_List{ + Items: []*productv1.User{ + {Id: "user-updated", Name: "Updated User"}, + }, + }, + }, + ContributorTeams: &productv1.ListOfListOfUser{ + List: &productv1.ListOfListOfUser_List{ + Items: []*productv1.ListOfUser{ + {List: &productv1.ListOfUser_List{ + Items: []*productv1.User{ + {Id: "user-team-updated", Name: "Updated Team Member"}, + }, + }}, + }, + }, + }, + } + + return &productv1.MutationUpdateBlogPostResponse{ + UpdateBlogPost: result, + }, nil +} + +// Author mutation implementations +func (s *MockService) MutationCreateAuthor(ctx context.Context, in *productv1.MutationCreateAuthorRequest) (*productv1.MutationCreateAuthorResponse, error) { + input := in.GetInput() + + result := &productv1.Author{ + Id: fmt.Sprintf("author-%d", rand.Intn(1000)), + Name: input.GetName(), + Email: input.GetEmail(), + Skills: input.GetSkills(), + Languages: input.GetLanguages(), + SocialLinks: input.GetSocialLinks(), + TeamsByProject: input.GetTeamsByProject(), + Collaborations: input.GetCollaborations(), + // Convert input types to output types for complex fields + FavoriteCategories: convertCategoryInputsToCategories(input.GetFavoriteCategories()), + AuthorGroups: convertNestedUserInputsToUsers(input.GetAuthorGroups()), + ProjectTeams: convertNestedUserInputsToUsers(input.GetProjectTeams()), + // Keep other complex fields with mock data since they're not in the simplified input + WrittenPosts: &productv1.ListOfBlogPost{ + List: &productv1.ListOfBlogPost_List{ + Items: []*productv1.BlogPost{ + {Id: "blog-created", Title: "Created Post", Content: "Content..."}, + }, + }, + }, + RelatedAuthors: &productv1.ListOfUser{ + List: &productv1.ListOfUser_List{ + Items: []*productv1.User{ + {Id: "related-author", Name: "Related Author"}, + }, + }, + }, + ProductReviews: &productv1.ListOfProduct{ + List: &productv1.ListOfProduct_List{ + Items: []*productv1.Product{ + {Id: "reviewed-product", Name: "Code Editor", Price: 199.99}, + }, + }, + }, + CategoryPreferences: &productv1.ListOfListOfCategory{ + List: &productv1.ListOfListOfCategory_List{ + Items: []*productv1.ListOfCategory{ + {List: &productv1.ListOfCategory_List{ + Items: []*productv1.Category{ + {Id: "pref-cat", Name: "Backend Development", Kind: productv1.CategoryKind_CATEGORY_KIND_ELECTRONICS}, + }, + }}, + }, + }, + }, + } + + return &productv1.MutationCreateAuthorResponse{ + CreateAuthor: result, + }, nil +} + +func (s *MockService) MutationUpdateAuthor(ctx context.Context, in *productv1.MutationUpdateAuthorRequest) (*productv1.MutationUpdateAuthorResponse, error) { + id := in.GetId() + input := in.GetInput() + + if id == "non-existent" { + return &productv1.MutationUpdateAuthorResponse{ + UpdateAuthor: nil, + }, nil + } + + result := &productv1.Author{ + Id: id, + Name: input.GetName(), + Email: input.GetEmail(), + Skills: input.GetSkills(), + Languages: input.GetLanguages(), + SocialLinks: input.GetSocialLinks(), + TeamsByProject: input.GetTeamsByProject(), + Collaborations: input.GetCollaborations(), + // Convert input types to output types for complex fields + FavoriteCategories: convertCategoryInputsToCategories(input.GetFavoriteCategories()), + AuthorGroups: convertNestedUserInputsToUsers(input.GetAuthorGroups()), + ProjectTeams: convertNestedUserInputsToUsers(input.GetProjectTeams()), + // Keep other complex fields with mock data since they're not in the simplified input + WrittenPosts: &productv1.ListOfBlogPost{ + List: &productv1.ListOfBlogPost_List{ + Items: []*productv1.BlogPost{ + {Id: "blog-updated", Title: "Updated Post", Content: "Updated content..."}, + }, + }, + }, + RelatedAuthors: &productv1.ListOfUser{ + List: &productv1.ListOfUser_List{ + Items: []*productv1.User{ + {Id: "related-author-updated", Name: "Updated Related Author"}, + }, + }, + }, + ProductReviews: &productv1.ListOfProduct{ + List: &productv1.ListOfProduct_List{ + Items: []*productv1.Product{ + {Id: "reviewed-product-updated", Name: "Updated Code Editor", Price: 249.99}, + }, + }, + }, + CategoryPreferences: &productv1.ListOfListOfCategory{ + List: &productv1.ListOfListOfCategory_List{ + Items: []*productv1.ListOfCategory{ + {List: &productv1.ListOfCategory_List{ + Items: []*productv1.Category{ + {Id: "pref-cat-updated", Name: "Updated Backend Development", Kind: productv1.CategoryKind_CATEGORY_KIND_ELECTRONICS}, + }, + }}, + }, + }, + }, + } + + return &productv1.MutationUpdateAuthorResponse{ + UpdateAuthor: result, + }, nil +} + +// Bulk operation implementations +func (s *MockService) QueryBulkSearchAuthors(ctx context.Context, in *productv1.QueryBulkSearchAuthorsRequest) (*productv1.QueryBulkSearchAuthorsResponse, error) { + var allResults []*productv1.Author + + // Handle nullable list - if filters is nil, return empty results + if in.Filters == nil { + return &productv1.QueryBulkSearchAuthorsResponse{ + BulkSearchAuthors: allResults, + }, nil + } + + // Process each filter in the list + if in.Filters.List != nil { + for i, filter := range in.Filters.List.Items { + // Create mock results for each filter + for j := 1; j <= 2; j++ { + name := fmt.Sprintf("Bulk Author %d-%d", i+1, j) + if filter.Name != nil { + name = fmt.Sprintf("%s - Bulk %d-%d", filter.Name.GetValue(), i+1, j) + } + + var skills []string + skillCount := int32(3) + if filter.SkillCount != nil { + skillCount = filter.SkillCount.GetValue() + } + for k := int32(0); k < skillCount; k++ { + skills = append(skills, fmt.Sprintf("BulkSkill%d", k+1)) + } + + var teamsByProject *productv1.ListOfListOfString + if filter.HasTeams != nil && filter.HasTeams.GetValue() { + teamsByProject = &productv1.ListOfListOfString{ + List: &productv1.ListOfListOfString_List{ + Items: []*productv1.ListOfString{ + {List: &productv1.ListOfString_List{ + Items: []string{fmt.Sprintf("BulkTeam%d", j), "SharedBulkTeam"}, + }}, + }, + }, + } + } else { + teamsByProject = &productv1.ListOfListOfString{List: &productv1.ListOfListOfString_List{}} + } + + allResults = append(allResults, &productv1.Author{ + Id: fmt.Sprintf("bulk-author-%d-%d", i+1, j), + Name: name, + Skills: skills, + Languages: []string{"English", fmt.Sprintf("BulkLang%d", j)}, + TeamsByProject: teamsByProject, + FavoriteCategories: []*productv1.Category{ + {Id: fmt.Sprintf("bulk-cat-%d-%d", i+1, j), Name: fmt.Sprintf("Bulk Category %d-%d", i+1, j), Kind: productv1.CategoryKind_CATEGORY_KIND_OTHER}, + }, + CategoryPreferences: &productv1.ListOfListOfCategory{ + List: &productv1.ListOfListOfCategory_List{ + Items: []*productv1.ListOfCategory{ + {List: &productv1.ListOfCategory_List{ + Items: []*productv1.Category{ + {Id: fmt.Sprintf("bulk-pref-%d-%d", i+1, j), Name: fmt.Sprintf("Bulk Preference %d-%d", i+1, j), Kind: productv1.CategoryKind_CATEGORY_KIND_BOOK}, + }, + }}, + }, + }, + }, + }) + } + } + } + + return &productv1.QueryBulkSearchAuthorsResponse{ + BulkSearchAuthors: allResults, + }, nil +} + +func (s *MockService) QueryBulkSearchBlogPosts(ctx context.Context, in *productv1.QueryBulkSearchBlogPostsRequest) (*productv1.QueryBulkSearchBlogPostsResponse, error) { + var allResults []*productv1.BlogPost + + // Handle nullable list - if filters is nil, return empty results + if in.Filters == nil { + return &productv1.QueryBulkSearchBlogPostsResponse{ + BulkSearchBlogPosts: allResults, + }, nil + } + + // Process each filter in the list + if in.Filters.List != nil { + for i, filter := range in.Filters.List.Items { + // Create mock results for each filter + for j := 1; j <= 2; j++ { + title := fmt.Sprintf("Bulk Blog Post %d-%d", i+1, j) + if filter.Title != nil { + title = fmt.Sprintf("%s - Bulk %d-%d", filter.Title.GetValue(), i+1, j) + } + + var categories []string + if filter.HasCategories != nil && filter.HasCategories.GetValue() { + categories = []string{fmt.Sprintf("BulkCategory%d", j), "SharedBulkCategory"} + } else { + categories = []string{} + } + + minTags := int32(2) + if filter.MinTags != nil { + minTags = filter.MinTags.GetValue() + } + var tags []string + for k := int32(0); k < minTags; k++ { + tags = append(tags, fmt.Sprintf("BulkTag%d", k+1)) + } + + allResults = append(allResults, &productv1.BlogPost{ + Id: fmt.Sprintf("bulk-post-%d-%d", i+1, j), + Title: title, + Content: fmt.Sprintf("Bulk content for post %d-%d", i+1, j), + Tags: tags, + Categories: categories, + ViewCounts: []int32{100, 150, 200}, + TagGroups: &productv1.ListOfListOfString{ + List: &productv1.ListOfListOfString_List{ + Items: []*productv1.ListOfString{ + {List: &productv1.ListOfString_List{ + Items: []string{fmt.Sprintf("BulkGroup%d", j)}, + }}, + }, + }, + }, + RelatedTopics: &productv1.ListOfListOfString{ + List: &productv1.ListOfListOfString_List{ + Items: []*productv1.ListOfString{ + {List: &productv1.ListOfString_List{ + Items: []string{fmt.Sprintf("BulkTopic%d", j)}, + }}, + }, + }, + }, + CommentThreads: &productv1.ListOfListOfString{ + List: &productv1.ListOfListOfString_List{ + Items: []*productv1.ListOfString{ + {List: &productv1.ListOfString_List{ + Items: []string{fmt.Sprintf("BulkComment%d", j)}, + }}, + }, + }, + }, + RelatedCategories: []*productv1.Category{ + {Id: fmt.Sprintf("bulk-rel-cat-%d-%d", i+1, j), Name: fmt.Sprintf("Bulk Related %d-%d", i+1, j), Kind: productv1.CategoryKind_CATEGORY_KIND_OTHER}, + }, + Contributors: []*productv1.User{ + {Id: fmt.Sprintf("bulk-contrib-%d-%d", i+1, j), Name: fmt.Sprintf("Bulk Contributor %d-%d", i+1, j)}, + }, + CategoryGroups: &productv1.ListOfListOfCategory{ + List: &productv1.ListOfListOfCategory_List{ + Items: []*productv1.ListOfCategory{ + {List: &productv1.ListOfCategory_List{ + Items: []*productv1.Category{ + {Id: fmt.Sprintf("bulk-grp-cat-%d-%d", i+1, j), Name: fmt.Sprintf("Bulk Group Cat %d-%d", i+1, j), Kind: productv1.CategoryKind_CATEGORY_KIND_BOOK}, + }, + }}, + }, + }, + }, + }) + } + } + } + + return &productv1.QueryBulkSearchBlogPostsResponse{ + BulkSearchBlogPosts: allResults, + }, nil +} + +func (s *MockService) MutationBulkCreateAuthors(ctx context.Context, in *productv1.MutationBulkCreateAuthorsRequest) (*productv1.MutationBulkCreateAuthorsResponse, error) { + var results []*productv1.Author + + // Handle nullable list - if authors is nil, return empty results + if in.Authors == nil { + return &productv1.MutationBulkCreateAuthorsResponse{ + BulkCreateAuthors: results, + }, nil + } + + // Process each author input in the list + if in.Authors.List != nil { + for i, authorInput := range in.Authors.List.Items { + // Convert nested UserInput lists to Users for complex fields + var authorGroups *productv1.ListOfListOfUser + if authorInput.AuthorGroups != nil { + authorGroups = convertNestedUserInputsToUsers(authorInput.AuthorGroups) + } + + var projectTeams *productv1.ListOfListOfUser + if authorInput.ProjectTeams != nil { + projectTeams = convertNestedUserInputsToUsers(authorInput.ProjectTeams) + } + + // Convert CategoryInput list to Categories + var favoriteCategories []*productv1.Category + if authorInput.FavoriteCategories != nil { + favoriteCategories = convertCategoryInputsToCategories(authorInput.FavoriteCategories) + } + + author := &productv1.Author{ + Id: fmt.Sprintf("bulk-created-author-%d", i+1), + Name: authorInput.Name, + Email: authorInput.Email, + Skills: authorInput.Skills, + Languages: authorInput.Languages, + SocialLinks: authorInput.SocialLinks, + TeamsByProject: authorInput.TeamsByProject, + Collaborations: authorInput.Collaborations, + FavoriteCategories: favoriteCategories, + AuthorGroups: authorGroups, + ProjectTeams: projectTeams, + // Add required complex fields with mock data + WrittenPosts: &productv1.ListOfBlogPost{ + List: &productv1.ListOfBlogPost_List{ + Items: []*productv1.BlogPost{ + {Id: fmt.Sprintf("bulk-blog-%d", i+1), Title: fmt.Sprintf("Bulk Created Post %d", i+1), Content: "Bulk created content..."}, + }, + }, + }, + RelatedAuthors: &productv1.ListOfUser{ + List: &productv1.ListOfUser_List{ + Items: []*productv1.User{ + {Id: fmt.Sprintf("bulk-rel-author-%d", i+1), Name: fmt.Sprintf("Bulk Related Author %d", i+1)}, + }, + }, + }, + ProductReviews: &productv1.ListOfProduct{ + List: &productv1.ListOfProduct_List{ + Items: []*productv1.Product{ + {Id: fmt.Sprintf("bulk-prod-%d", i+1), Name: fmt.Sprintf("Bulk Product %d", i+1), Price: 99.99}, + }, + }, + }, + CategoryPreferences: &productv1.ListOfListOfCategory{ + List: &productv1.ListOfListOfCategory_List{ + Items: []*productv1.ListOfCategory{ + {List: &productv1.ListOfCategory_List{ + Items: []*productv1.Category{ + {Id: fmt.Sprintf("bulk-cat-pref-%d", i+1), Name: fmt.Sprintf("Bulk Category Preference %d", i+1), Kind: productv1.CategoryKind_CATEGORY_KIND_ELECTRONICS}, + }, + }}, + }, + }, + }, + } + + results = append(results, author) + } + } + + return &productv1.MutationBulkCreateAuthorsResponse{ + BulkCreateAuthors: results, + }, nil +} + +func (s *MockService) MutationBulkUpdateAuthors(ctx context.Context, in *productv1.MutationBulkUpdateAuthorsRequest) (*productv1.MutationBulkUpdateAuthorsResponse, error) { + var results []*productv1.Author + + // Handle nullable list - if authors is nil, return empty results + if in.Authors == nil { + return &productv1.MutationBulkUpdateAuthorsResponse{ + BulkUpdateAuthors: results, + }, nil + } + + // Process each author input in the list + if in.Authors.List != nil { + for i, authorInput := range in.Authors.List.Items { + // Convert nested UserInput lists to Users for complex fields + var authorGroups *productv1.ListOfListOfUser + if authorInput.AuthorGroups != nil { + authorGroups = convertNestedUserInputsToUsers(authorInput.AuthorGroups) + } + + var projectTeams *productv1.ListOfListOfUser + if authorInput.ProjectTeams != nil { + projectTeams = convertNestedUserInputsToUsers(authorInput.ProjectTeams) + } + + // Convert CategoryInput list to Categories + var favoriteCategories []*productv1.Category + if authorInput.FavoriteCategories != nil { + favoriteCategories = convertCategoryInputsToCategories(authorInput.FavoriteCategories) + } + + author := &productv1.Author{ + Id: fmt.Sprintf("bulk-updated-author-%d", i+1), + Name: authorInput.Name, + Email: authorInput.Email, + Skills: authorInput.Skills, + Languages: authorInput.Languages, + SocialLinks: authorInput.SocialLinks, + TeamsByProject: authorInput.TeamsByProject, + Collaborations: authorInput.Collaborations, + FavoriteCategories: favoriteCategories, + AuthorGroups: authorGroups, + ProjectTeams: projectTeams, + // Add required complex fields with mock data + WrittenPosts: &productv1.ListOfBlogPost{ + List: &productv1.ListOfBlogPost_List{ + Items: []*productv1.BlogPost{ + {Id: fmt.Sprintf("bulk-updated-blog-%d", i+1), Title: fmt.Sprintf("Bulk Updated Post %d", i+1), Content: "Bulk updated content..."}, + }, + }, + }, + RelatedAuthors: &productv1.ListOfUser{ + List: &productv1.ListOfUser_List{ + Items: []*productv1.User{ + {Id: fmt.Sprintf("bulk-updated-rel-author-%d", i+1), Name: fmt.Sprintf("Bulk Updated Related Author %d", i+1)}, + }, + }, + }, + ProductReviews: &productv1.ListOfProduct{ + List: &productv1.ListOfProduct_List{ + Items: []*productv1.Product{ + {Id: fmt.Sprintf("bulk-updated-prod-%d", i+1), Name: fmt.Sprintf("Bulk Updated Product %d", i+1), Price: 149.99}, + }, + }, + }, + CategoryPreferences: &productv1.ListOfListOfCategory{ + List: &productv1.ListOfListOfCategory_List{ + Items: []*productv1.ListOfCategory{ + {List: &productv1.ListOfCategory_List{ + Items: []*productv1.Category{ + {Id: fmt.Sprintf("bulk-updated-cat-pref-%d", i+1), Name: fmt.Sprintf("Bulk Updated Category Preference %d", i+1), Kind: productv1.CategoryKind_CATEGORY_KIND_ELECTRONICS}, + }, + }}, + }, + }, + }, + } + + results = append(results, author) + } + } + + return &productv1.MutationBulkUpdateAuthorsResponse{ + BulkUpdateAuthors: results, + }, nil +} + +func (s *MockService) MutationBulkCreateBlogPosts(ctx context.Context, in *productv1.MutationBulkCreateBlogPostsRequest) (*productv1.MutationBulkCreateBlogPostsResponse, error) { + var results []*productv1.BlogPost + + // Handle nullable list - if blogPosts is nil, return empty results + if in.BlogPosts == nil { + return &productv1.MutationBulkCreateBlogPostsResponse{ + BulkCreateBlogPosts: results, + }, nil + } + + // Process each blog post input in the list + if in.BlogPosts.List != nil { + for i, blogPostInput := range in.BlogPosts.List.Items { + // Convert CategoryInput lists to Categories + var relatedCategories []*productv1.Category + if blogPostInput.RelatedCategories != nil { + relatedCategories = convertCategoryInputListToCategories(blogPostInput.RelatedCategories) + } + + var contributors []*productv1.User + if blogPostInput.Contributors != nil { + contributors = convertUserInputsToUsers(blogPostInput.Contributors) + } + + var categoryGroups *productv1.ListOfListOfCategory + if blogPostInput.CategoryGroups != nil { + categoryGroups = convertNestedCategoryInputsToCategories(blogPostInput.CategoryGroups) + } + + blogPost := &productv1.BlogPost{ + Id: fmt.Sprintf("bulk-created-post-%d", i+1), + Title: blogPostInput.Title, + Content: blogPostInput.Content, + Tags: blogPostInput.Tags, + OptionalTags: blogPostInput.OptionalTags, + Categories: blogPostInput.Categories, + Keywords: blogPostInput.Keywords, + ViewCounts: blogPostInput.ViewCounts, + Ratings: blogPostInput.Ratings, + IsPublished: blogPostInput.IsPublished, + TagGroups: blogPostInput.TagGroups, + RelatedTopics: blogPostInput.RelatedTopics, + CommentThreads: blogPostInput.CommentThreads, + Suggestions: blogPostInput.Suggestions, + RelatedCategories: relatedCategories, + Contributors: contributors, + CategoryGroups: categoryGroups, + // Add required fields with mock data + MentionedProducts: &productv1.ListOfProduct{ + List: &productv1.ListOfProduct_List{ + Items: []*productv1.Product{ + {Id: fmt.Sprintf("bulk-prod-%d", i+1), Name: fmt.Sprintf("Bulk Created Product %d", i+1), Price: 99.99}, + }, + }, + }, + MentionedUsers: &productv1.ListOfUser{ + List: &productv1.ListOfUser_List{ + Items: []*productv1.User{ + {Id: fmt.Sprintf("bulk-user-%d", i+1), Name: fmt.Sprintf("Bulk Created User %d", i+1)}, + }, + }, + }, + ContributorTeams: &productv1.ListOfListOfUser{ + List: &productv1.ListOfListOfUser_List{ + Items: []*productv1.ListOfUser{ + {List: &productv1.ListOfUser_List{ + Items: []*productv1.User{ + {Id: fmt.Sprintf("bulk-team-%d", i+1), Name: fmt.Sprintf("Bulk Created Team Member %d", i+1)}, + }, + }}, + }, + }, + }, + } + + results = append(results, blogPost) + } + } + + return &productv1.MutationBulkCreateBlogPostsResponse{ + BulkCreateBlogPosts: results, + }, nil +} + +func (s *MockService) MutationBulkUpdateBlogPosts(ctx context.Context, in *productv1.MutationBulkUpdateBlogPostsRequest) (*productv1.MutationBulkUpdateBlogPostsResponse, error) { + var results []*productv1.BlogPost + + // Handle nullable list - if blogPosts is nil, return empty results + if in.BlogPosts == nil { + return &productv1.MutationBulkUpdateBlogPostsResponse{ + BulkUpdateBlogPosts: results, + }, nil + } + + // Process each blog post input in the list + if in.BlogPosts.List != nil { + for i, blogPostInput := range in.BlogPosts.List.Items { + // Convert CategoryInput lists to Categories + var relatedCategories []*productv1.Category + if blogPostInput.RelatedCategories != nil { + relatedCategories = convertCategoryInputListToCategories(blogPostInput.RelatedCategories) + } + + var contributors []*productv1.User + if blogPostInput.Contributors != nil { + contributors = convertUserInputsToUsers(blogPostInput.Contributors) + } + + var categoryGroups *productv1.ListOfListOfCategory + if blogPostInput.CategoryGroups != nil { + categoryGroups = convertNestedCategoryInputsToCategories(blogPostInput.CategoryGroups) + } + + blogPost := &productv1.BlogPost{ + Id: fmt.Sprintf("bulk-updated-post-%d", i+1), + Title: blogPostInput.Title, + Content: blogPostInput.Content, + Tags: blogPostInput.Tags, + OptionalTags: blogPostInput.OptionalTags, + Categories: blogPostInput.Categories, + Keywords: blogPostInput.Keywords, + ViewCounts: blogPostInput.ViewCounts, + Ratings: blogPostInput.Ratings, + IsPublished: blogPostInput.IsPublished, + TagGroups: blogPostInput.TagGroups, + RelatedTopics: blogPostInput.RelatedTopics, + CommentThreads: blogPostInput.CommentThreads, + Suggestions: blogPostInput.Suggestions, + RelatedCategories: relatedCategories, + Contributors: contributors, + CategoryGroups: categoryGroups, + // Add required fields with mock data + MentionedProducts: &productv1.ListOfProduct{ + List: &productv1.ListOfProduct_List{ + Items: []*productv1.Product{ + {Id: fmt.Sprintf("bulk-updated-prod-%d", i+1), Name: fmt.Sprintf("Bulk Updated Product %d", i+1), Price: 149.99}, + }, + }, + }, + MentionedUsers: &productv1.ListOfUser{ + List: &productv1.ListOfUser_List{ + Items: []*productv1.User{ + {Id: fmt.Sprintf("bulk-updated-user-%d", i+1), Name: fmt.Sprintf("Bulk Updated User %d", i+1)}, + }, + }, + }, + ContributorTeams: &productv1.ListOfListOfUser{ + List: &productv1.ListOfListOfUser_List{ + Items: []*productv1.ListOfUser{ + {List: &productv1.ListOfUser_List{ + Items: []*productv1.User{ + {Id: fmt.Sprintf("bulk-updated-team-%d", i+1), Name: fmt.Sprintf("Bulk Updated Team Member %d", i+1)}, + }, + }}, + }, + }, + }, + } + + results = append(results, blogPost) + } + } + + return &productv1.MutationBulkUpdateBlogPostsResponse{ + BulkUpdateBlogPosts: results, + }, nil +} diff --git a/v2/pkg/grpctest/mockservice_lookup.go b/v2/pkg/grpctest/mockservice_lookup.go new file mode 100644 index 000000000..e431c137c --- /dev/null +++ b/v2/pkg/grpctest/mockservice_lookup.go @@ -0,0 +1,76 @@ +package grpctest + +import ( + context "context" + "fmt" + "math/rand" + + "github.com/wundergraph/graphql-go-tools/v2/pkg/grpctest/productv1" +) + +// LookupWarehouseById implements productv1.ProductServiceServer. +func (s *MockService) LookupWarehouseById(ctx context.Context, in *productv1.LookupWarehouseByIdRequest) (*productv1.LookupWarehouseByIdResponse, error) { + var results []*productv1.Warehouse + + // Special requirement: return one less item than requested to test error handling + // This deliberately breaks the normal pattern of returning the same number of items as keys + keys := in.GetKeys() + if len(keys) == 0 { + return &productv1.LookupWarehouseByIdResponse{ + Result: results, + }, nil + } + + // Return all items except the last one to test error scenarios + for i, input := range keys { + // Skip the last item to create an intentional mismatch + if i == len(keys)-1 { + break + } + + warehouseId := input.GetId() + results = append(results, &productv1.Warehouse{ + Id: warehouseId, + Name: fmt.Sprintf("Warehouse %s", warehouseId), + Location: fmt.Sprintf("Location %d", rand.Intn(100)), + }) + } + + return &productv1.LookupWarehouseByIdResponse{ + Result: results, + }, nil +} + +func (s *MockService) LookupProductById(ctx context.Context, in *productv1.LookupProductByIdRequest) (*productv1.LookupProductByIdResponse, error) { + var results []*productv1.Product + + for _, input := range in.GetKeys() { + productId := input.GetId() + results = append(results, &productv1.Product{ + Id: productId, + Name: fmt.Sprintf("Product %s", productId), + Price: 99.99, + }) + } + + return &productv1.LookupProductByIdResponse{ + Result: results, + }, nil +} + +func (s *MockService) LookupStorageById(ctx context.Context, in *productv1.LookupStorageByIdRequest) (*productv1.LookupStorageByIdResponse, error) { + var results []*productv1.Storage + + for _, input := range in.GetKeys() { + storageId := input.GetId() + results = append(results, &productv1.Storage{ + Id: storageId, + Name: fmt.Sprintf("Storage %s", storageId), + Location: fmt.Sprintf("Location %d", rand.Intn(100)), + }) + } + + return &productv1.LookupStorageByIdResponse{ + Result: results, + }, nil +} diff --git a/v2/pkg/grpctest/mockservice_nullable_fields.go b/v2/pkg/grpctest/mockservice_nullable_fields.go new file mode 100644 index 000000000..bac7255da --- /dev/null +++ b/v2/pkg/grpctest/mockservice_nullable_fields.go @@ -0,0 +1,292 @@ +package grpctest + +import ( + context "context" + "fmt" + "math" + "math/rand" + + "github.com/wundergraph/graphql-go-tools/v2/pkg/grpctest/productv1" + "google.golang.org/protobuf/types/known/wrapperspb" +) + +// MutationCreateNullableFieldsType implements productv1.ProductServiceServer. +func (s *MockService) MutationCreateNullableFieldsType(ctx context.Context, in *productv1.MutationCreateNullableFieldsTypeRequest) (*productv1.MutationCreateNullableFieldsTypeResponse, error) { + input := in.GetInput() + + // Create a new NullableFieldsType from the input + result := &productv1.NullableFieldsType{ + Id: fmt.Sprintf("nullable-%d", rand.Intn(1000)), + Name: input.GetName(), + RequiredString: input.GetRequiredString(), + RequiredInt: input.GetRequiredInt(), + } + + // Handle optional fields - copy from input if they exist + if input.OptionalString != nil { + result.OptionalString = &wrapperspb.StringValue{Value: input.OptionalString.GetValue()} + } + if input.OptionalInt != nil { + result.OptionalInt = &wrapperspb.Int32Value{Value: input.OptionalInt.GetValue()} + } + if input.OptionalFloat != nil { + result.OptionalFloat = &wrapperspb.DoubleValue{Value: input.OptionalFloat.GetValue()} + } + if input.OptionalBoolean != nil { + result.OptionalBoolean = &wrapperspb.BoolValue{Value: input.OptionalBoolean.GetValue()} + } + + return &productv1.MutationCreateNullableFieldsTypeResponse{ + CreateNullableFieldsType: result, + }, nil +} + +// MutationUpdateNullableFieldsType implements productv1.ProductServiceServer. +func (s *MockService) MutationUpdateNullableFieldsType(ctx context.Context, in *productv1.MutationUpdateNullableFieldsTypeRequest) (*productv1.MutationUpdateNullableFieldsTypeResponse, error) { + id := in.GetId() + input := in.GetInput() + + // Return nil if trying to update a non-existent ID + if id == "non-existent" { + return &productv1.MutationUpdateNullableFieldsTypeResponse{ + UpdateNullableFieldsType: nil, + }, nil + } + + // Create updated NullableFieldsType + result := &productv1.NullableFieldsType{ + Id: id, + Name: input.GetName(), + RequiredString: input.GetRequiredString(), + RequiredInt: input.GetRequiredInt(), + } + + // Handle optional fields - copy from input if they exist + if input.OptionalString != nil { + result.OptionalString = &wrapperspb.StringValue{Value: input.OptionalString.GetValue()} + } + if input.OptionalInt != nil { + result.OptionalInt = &wrapperspb.Int32Value{Value: input.OptionalInt.GetValue()} + } + if input.OptionalFloat != nil { + result.OptionalFloat = &wrapperspb.DoubleValue{Value: input.OptionalFloat.GetValue()} + } + if input.OptionalBoolean != nil { + result.OptionalBoolean = &wrapperspb.BoolValue{Value: input.OptionalBoolean.GetValue()} + } + + return &productv1.MutationUpdateNullableFieldsTypeResponse{ + UpdateNullableFieldsType: result, + }, nil +} + +// QueryAllNullableFieldsTypes implements productv1.ProductServiceServer. +func (s *MockService) QueryAllNullableFieldsTypes(ctx context.Context, in *productv1.QueryAllNullableFieldsTypesRequest) (*productv1.QueryAllNullableFieldsTypesResponse, error) { + var results []*productv1.NullableFieldsType + + // Create a variety of test data with different nullable field combinations + + // Entry 1: All fields populated + results = append(results, &productv1.NullableFieldsType{ + Id: "nullable-1", + Name: "Full Data Entry", + OptionalString: &wrapperspb.StringValue{Value: "Optional String Value"}, + OptionalInt: &wrapperspb.Int32Value{Value: 42}, + OptionalFloat: &wrapperspb.DoubleValue{Value: math.MaxFloat64}, + OptionalBoolean: &wrapperspb.BoolValue{Value: true}, + RequiredString: "Required String 1", + RequiredInt: 100, + }) + + // Entry 2: Some nullable fields are null + results = append(results, &productv1.NullableFieldsType{ + Id: "nullable-2", + Name: "Partial Data Entry", + OptionalString: &wrapperspb.StringValue{Value: "Only string is set"}, + OptionalInt: nil, // null + OptionalFloat: nil, // null + OptionalBoolean: &wrapperspb.BoolValue{Value: false}, + RequiredString: "Required String 2", + RequiredInt: 200, + }) + + // Entry 3: All nullable fields are null + results = append(results, &productv1.NullableFieldsType{ + Id: "nullable-3", + Name: "Minimal Data Entry", + OptionalString: nil, // null + OptionalInt: nil, // null + OptionalFloat: nil, // null + OptionalBoolean: nil, // null + RequiredString: "Required String 3", + RequiredInt: 300, + }) + + return &productv1.QueryAllNullableFieldsTypesResponse{ + AllNullableFieldsTypes: results, + }, nil +} + +// QueryNullableFieldsType implements productv1.ProductServiceServer. +func (s *MockService) QueryNullableFieldsType(ctx context.Context, in *productv1.QueryNullableFieldsTypeRequest) (*productv1.QueryNullableFieldsTypeResponse, error) { + // Return a single NullableFieldsType with mixed null/non-null values + result := &productv1.NullableFieldsType{ + Id: "nullable-default", + Name: "Default Nullable Fields Type", + OptionalString: &wrapperspb.StringValue{Value: "Default optional string"}, + OptionalInt: &wrapperspb.Int32Value{Value: 777}, + OptionalFloat: nil, // null + OptionalBoolean: &wrapperspb.BoolValue{Value: true}, + RequiredString: "Default required string", + RequiredInt: 999, + } + + return &productv1.QueryNullableFieldsTypeResponse{ + NullableFieldsType: result, + }, nil +} + +// QueryNullableFieldsTypeById implements productv1.ProductServiceServer. +func (s *MockService) QueryNullableFieldsTypeById(ctx context.Context, in *productv1.QueryNullableFieldsTypeByIdRequest) (*productv1.QueryNullableFieldsTypeByIdResponse, error) { + id := in.GetId() + + // Return null for specific test IDs + if id == "not-found" || id == "null-test" { + return &productv1.QueryNullableFieldsTypeByIdResponse{ + NullableFieldsTypeById: nil, + }, nil + } + + // Create different test data based on ID + var result *productv1.NullableFieldsType + + switch id { + case "full-data": + result = &productv1.NullableFieldsType{ + Id: id, + Name: "Full Data by ID", + OptionalString: &wrapperspb.StringValue{Value: "All fields populated"}, + OptionalInt: &wrapperspb.Int32Value{Value: 123}, + OptionalFloat: &wrapperspb.DoubleValue{Value: 12.34}, + OptionalBoolean: &wrapperspb.BoolValue{Value: false}, + RequiredString: "Required by ID", + RequiredInt: 456, + } + case "partial-data": + result = &productv1.NullableFieldsType{ + Id: id, + Name: "Partial Data by ID", + OptionalString: nil, // null + OptionalInt: &wrapperspb.Int32Value{Value: 789}, + OptionalFloat: nil, // null + OptionalBoolean: &wrapperspb.BoolValue{Value: true}, + RequiredString: "Partial required by ID", + RequiredInt: 321, + } + case "minimal-data": + result = &productv1.NullableFieldsType{ + Id: id, + Name: "Minimal Data by ID", + OptionalString: nil, // null + OptionalInt: nil, // null + OptionalFloat: nil, // null + OptionalBoolean: nil, // null + RequiredString: "Only required fields", + RequiredInt: 111, + } + default: + // Generic response for any other ID + result = &productv1.NullableFieldsType{ + Id: id, + Name: fmt.Sprintf("Nullable Type %s", id), + OptionalString: &wrapperspb.StringValue{Value: fmt.Sprintf("Optional for %s", id)}, + OptionalInt: &wrapperspb.Int32Value{Value: int32(len(id) * 10)}, + OptionalFloat: &wrapperspb.DoubleValue{Value: float64(len(id)) * 1.5}, + OptionalBoolean: &wrapperspb.BoolValue{Value: len(id)%2 == 0}, + RequiredString: fmt.Sprintf("Required for %s", id), + RequiredInt: int32(len(id) * 100), + } + } + + return &productv1.QueryNullableFieldsTypeByIdResponse{ + NullableFieldsTypeById: result, + }, nil +} + +// QueryNullableFieldsTypeWithFilter implements productv1.ProductServiceServer. +func (s *MockService) QueryNullableFieldsTypeWithFilter(ctx context.Context, in *productv1.QueryNullableFieldsTypeWithFilterRequest) (*productv1.QueryNullableFieldsTypeWithFilterResponse, error) { + filter := in.GetFilter() + var results []*productv1.NullableFieldsType + + // If no filter provided, return empty results + if filter == nil { + return &productv1.QueryNullableFieldsTypeWithFilterResponse{ + NullableFieldsTypeWithFilter: results, + }, nil + } + + // Create test data based on filter criteria + nameFilter := "" + if filter.Name != nil { + nameFilter = filter.Name.GetValue() + } + + optionalStringFilter := "" + if filter.OptionalString != nil { + optionalStringFilter = filter.OptionalString.GetValue() + } + + includeNulls := false + if filter.IncludeNulls != nil { + includeNulls = filter.IncludeNulls.GetValue() + } + + // Generate filtered results + for i := 1; i <= 3; i++ { + var optionalString *wrapperspb.StringValue + var optionalInt *wrapperspb.Int32Value + var optionalFloat *wrapperspb.DoubleValue + var optionalBoolean *wrapperspb.BoolValue + + // Vary the nullable fields based on includeNulls and index + if includeNulls || i%2 == 1 { + if optionalStringFilter != "" { + optionalString = &wrapperspb.StringValue{Value: optionalStringFilter} + } else { + optionalString = &wrapperspb.StringValue{Value: fmt.Sprintf("Filtered string %d", i)} + } + } + + if includeNulls || i%3 != 0 { + optionalInt = &wrapperspb.Int32Value{Value: int32(i * 100)} + } + + if includeNulls || i%2 == 0 { + optionalFloat = &wrapperspb.DoubleValue{Value: float64(i) * 10.5} + } + + if includeNulls || i%4 != 0 { + optionalBoolean = &wrapperspb.BoolValue{Value: i%2 == 0} + } + + name := fmt.Sprintf("Filtered Item %d", i) + if nameFilter != "" { + name = fmt.Sprintf("%s - %d", nameFilter, i) + } + + results = append(results, &productv1.NullableFieldsType{ + Id: fmt.Sprintf("filtered-%d", i), + Name: name, + OptionalString: optionalString, + OptionalInt: optionalInt, + OptionalFloat: optionalFloat, + OptionalBoolean: optionalBoolean, + RequiredString: fmt.Sprintf("Required filtered %d", i), + RequiredInt: int32(i * 1000), + }) + } + + return &productv1.QueryNullableFieldsTypeWithFilterResponse{ + NullableFieldsTypeWithFilter: results, + }, nil +} diff --git a/v2/pkg/grpctest/mockservice_resolve.go b/v2/pkg/grpctest/mockservice_resolve.go new file mode 100644 index 000000000..6c0bd18cb --- /dev/null +++ b/v2/pkg/grpctest/mockservice_resolve.go @@ -0,0 +1,772 @@ +package grpctest + +import ( + context "context" + "fmt" + + "github.com/wundergraph/graphql-go-tools/v2/pkg/grpctest/productv1" + "google.golang.org/protobuf/types/known/wrapperspb" +) + +// ResolveProductMascotRecommendation implements productv1.ProductServiceServer. +func (s *MockService) ResolveProductMascotRecommendation(_ context.Context, req *productv1.ResolveProductMascotRecommendationRequest) (*productv1.ResolveProductMascotRecommendationResponse, error) { + results := make([]*productv1.ResolveProductMascotRecommendationResult, 0, len(req.GetContext())) + + includeDetails := false + if req.GetFieldArgs() != nil { + includeDetails = req.GetFieldArgs().GetIncludeDetails() + } + + for i, ctx := range req.GetContext() { + // Alternate between Cat and Dog based on index + var animal *productv1.Animal + if i%2 == 0 { + volume := int32(5) + if includeDetails { + volume = int32((i + 1) * 8) + } + animal = &productv1.Animal{ + Instance: &productv1.Animal_Cat{ + Cat: &productv1.Cat{ + Id: fmt.Sprintf("mascot-cat-%s", ctx.GetId()), + Name: fmt.Sprintf("MascotCat for %s", ctx.GetName()), + Kind: "Cat", + MeowVolume: volume, + Owner: &productv1.Owner{ + Id: fmt.Sprintf("owner-cat-%s", ctx.GetId()), + Name: fmt.Sprintf("OwnerCat for %s", ctx.GetName()), + Contact: &productv1.ContactInfo{ + Email: "owner-cat@example.com", + Phone: "123-456-7890", + Address: &productv1.Address{ + Street: "123 Main St", + City: "Anytown", + Country: "USA", + ZipCode: "12345", + }, + }, + }, + Breed: &productv1.CatBreed{ + Id: fmt.Sprintf("breed-cat-%s", ctx.GetId()), + Name: fmt.Sprintf("BreedCat for %s", ctx.GetName()), + Origin: "USA", + Characteristics: &productv1.BreedCharacteristics{ + Size: "Small", + Temperament: "Curious", + Lifespan: "14-16 years", + }, + }, + }, + }, + } + } else { + volume := int32(7) + if includeDetails { + volume = int32((i + 1) * 10) + } + animal = &productv1.Animal{ + Instance: &productv1.Animal_Dog{ + Dog: &productv1.Dog{ + Id: fmt.Sprintf("mascot-dog-%s", ctx.GetId()), + Name: fmt.Sprintf("MascotDog for %s", ctx.GetName()), + Kind: "Dog", + BarkVolume: volume, + Owner: &productv1.Owner{ + Id: fmt.Sprintf("owner-dog-%s", ctx.GetId()), + Name: fmt.Sprintf("OwnerDog for %s", ctx.GetName()), + Contact: &productv1.ContactInfo{ + Email: "owner-dog@example.com", + Phone: "123-456-7890", + Address: &productv1.Address{ + Street: "123 Main St", + City: "Anytown", + Country: "USA", + ZipCode: "12345", + }, + }, + }, + Breed: &productv1.DogBreed{ + Id: fmt.Sprintf("breed-dog-%s", ctx.GetId()), + Name: fmt.Sprintf("BreedDog for %s", ctx.GetName()), + Origin: "USA", + Characteristics: &productv1.BreedCharacteristics{ + Size: "Medium", + Temperament: "Loyal", + Lifespan: "10-12 years", + }, + }, + }, + }, + } + } + + results = append(results, &productv1.ResolveProductMascotRecommendationResult{ + MascotRecommendation: animal, + }) + } + + return &productv1.ResolveProductMascotRecommendationResponse{ + Result: results, + }, nil +} + +// ResolveProductProductDetails implements productv1.ProductServiceServer. +func (s *MockService) ResolveProductProductDetails(_ context.Context, req *productv1.ResolveProductProductDetailsRequest) (*productv1.ResolveProductProductDetailsResponse, error) { + results := make([]*productv1.ResolveProductProductDetailsResult, 0, len(req.GetContext())) + + includeExtended := false + if req.GetFieldArgs() != nil { + includeExtended = req.GetFieldArgs().GetIncludeExtended() + } + + for i, ctx := range req.GetContext() { + // Create recommended pet (alternate between Cat and Dog) + var pet *productv1.Animal + if i%2 == 0 { + pet = &productv1.Animal{ + Instance: &productv1.Animal_Cat{ + Cat: &productv1.Cat{ + Id: fmt.Sprintf("details-cat-%s", ctx.GetId()), + Name: fmt.Sprintf("RecommendedCat for %s", ctx.GetName()), + Kind: "Cat", + MeowVolume: int32((i + 1) * 6), + Owner: &productv1.Owner{ + Id: fmt.Sprintf("owner-details-cat-%s", ctx.GetId()), + Name: fmt.Sprintf("OwnerDetailsCat for %s", ctx.GetName()), + Contact: &productv1.ContactInfo{ + Email: "owner-details-cat@example.com", + Phone: "555-111-2222", + Address: &productv1.Address{ + Street: "456 Oak Ave", + City: "Springfield", + Country: "USA", + ZipCode: "54321", + }, + }, + }, + Breed: &productv1.CatBreed{ + Id: fmt.Sprintf("breed-details-cat-%s", ctx.GetId()), + Name: fmt.Sprintf("BreedDetailsCat for %s", ctx.GetName()), + Origin: "France", + Characteristics: &productv1.BreedCharacteristics{ + Size: "Medium", + Temperament: "Friendly", + Lifespan: "12-15 years", + }, + }, + }, + }, + } + } else { + pet = &productv1.Animal{ + Instance: &productv1.Animal_Dog{ + Dog: &productv1.Dog{ + Id: fmt.Sprintf("details-dog-%s", ctx.GetId()), + Name: fmt.Sprintf("RecommendedDog for %s", ctx.GetName()), + Kind: "Dog", + BarkVolume: int32((i + 1) * 9), + Owner: &productv1.Owner{ + Id: fmt.Sprintf("owner-details-dog-%s", ctx.GetId()), + Name: fmt.Sprintf("OwnerDetailsDog for %s", ctx.GetName()), + Contact: &productv1.ContactInfo{ + Email: "owner-details-dog@example.com", + Phone: "555-333-4444", + Address: &productv1.Address{ + Street: "789 Elm St", + City: "Riverside", + Country: "USA", + ZipCode: "67890", + }, + }, + }, + Breed: &productv1.DogBreed{ + Id: fmt.Sprintf("breed-details-dog-%s", ctx.GetId()), + Name: fmt.Sprintf("BreedDetailsDog for %s", ctx.GetName()), + Origin: "Germany", + Characteristics: &productv1.BreedCharacteristics{ + Size: "Large", + Temperament: "Loyal", + Lifespan: "10-12 years", + }, + }, + }, + }, + } + } + + // Create review summary (alternate between success and error based on price and extended flag) + var reviewSummary *productv1.ActionResult + if includeExtended && ctx.GetPrice() > 500 { + reviewSummary = &productv1.ActionResult{ + Value: &productv1.ActionResult_ActionError{ + ActionError: &productv1.ActionError{ + Message: fmt.Sprintf("Product %s has negative reviews", ctx.GetName()), + Code: "NEGATIVE_REVIEWS", + }, + }, + } + } else { + reviewSummary = &productv1.ActionResult{ + Value: &productv1.ActionResult_ActionSuccess{ + ActionSuccess: &productv1.ActionSuccess{ + Message: fmt.Sprintf("Product %s has positive reviews", ctx.GetName()), + Timestamp: "2024-01-01T15:00:00Z", + }, + }, + } + } + + description := fmt.Sprintf("Standard details for %s", ctx.GetName()) + if includeExtended { + description = fmt.Sprintf("Extended details for %s with comprehensive information", ctx.GetName()) + } + + results = append(results, &productv1.ResolveProductProductDetailsResult{ + ProductDetails: &productv1.ProductDetails{ + Id: fmt.Sprintf("details-%s-%d", ctx.GetId(), i), + Description: description, + ReviewSummary: reviewSummary, + RecommendedPet: pet, + }, + }) + } + + return &productv1.ResolveProductProductDetailsResponse{ + Result: results, + }, nil +} + +// ResolveProductStockStatus implements productv1.ProductServiceServer. +func (s *MockService) ResolveProductStockStatus(_ context.Context, req *productv1.ResolveProductStockStatusRequest) (*productv1.ResolveProductStockStatusResponse, error) { + results := make([]*productv1.ResolveProductStockStatusResult, 0, len(req.GetContext())) + + checkAvailability := false + if req.GetFieldArgs() != nil { + checkAvailability = req.GetFieldArgs().GetCheckAvailability() + } + + for i, ctx := range req.GetContext() { + var stockStatus *productv1.ActionResult + + // If checking availability and price is high, return out of stock error + if checkAvailability && ctx.GetPrice() > 300 && i%2 == 0 { + stockStatus = &productv1.ActionResult{ + Value: &productv1.ActionResult_ActionError{ + ActionError: &productv1.ActionError{ + Message: fmt.Sprintf("Product %s is currently out of stock", ctx.GetName()), + Code: "OUT_OF_STOCK", + }, + }, + } + } else { + stockStatus = &productv1.ActionResult{ + Value: &productv1.ActionResult_ActionSuccess{ + ActionSuccess: &productv1.ActionSuccess{ + Message: fmt.Sprintf("Product %s is in stock and available", ctx.GetName()), + Timestamp: "2024-01-01T10:00:00Z", + }, + }, + } + } + + results = append(results, &productv1.ResolveProductStockStatusResult{ + StockStatus: stockStatus, + }) + } + + return &productv1.ResolveProductStockStatusResponse{ + Result: results, + }, nil +} + +// ResolveTestContainerDetails implements productv1.ProductServiceServer. +func (s *MockService) ResolveTestContainerDetails(_ context.Context, req *productv1.ResolveTestContainerDetailsRequest) (*productv1.ResolveTestContainerDetailsResponse, error) { + results := make([]*productv1.ResolveTestContainerDetailsResult, 0, len(req.GetContext())) + + includeExtended := false + if req.GetFieldArgs() != nil { + includeExtended = req.GetFieldArgs().GetIncludeExtended() + } + + for i, ctx := range req.GetContext() { + // Alternate between Cat and Dog for the pet field (Animal interface) + var pet *productv1.Animal + if i%2 == 0 { + pet = &productv1.Animal{ + Instance: &productv1.Animal_Cat{ + Cat: &productv1.Cat{ + Id: fmt.Sprintf("test-cat-%s", ctx.GetId()), + Name: fmt.Sprintf("TestCat-%s", ctx.GetName()), + Kind: "Cat", + MeowVolume: int32((i + 1) * 5), + Owner: &productv1.Owner{ + Id: fmt.Sprintf("owner-test-cat-%s", ctx.GetId()), + Name: fmt.Sprintf("OwnerTestCat for %s", ctx.GetName()), + Contact: &productv1.ContactInfo{ + Email: "owner-test-cat@example.com", + Phone: "555-555-5555", + Address: &productv1.Address{ + Street: "321 Pine Rd", + City: "Lakeside", + Country: "Canada", + ZipCode: "A1B2C3", + }, + }, + }, + Breed: &productv1.CatBreed{ + Id: fmt.Sprintf("breed-test-cat-%s", ctx.GetId()), + Name: fmt.Sprintf("BreedTestCat for %s", ctx.GetName()), + Origin: "Egypt", + Characteristics: &productv1.BreedCharacteristics{ + Size: "Small", + Temperament: "Curious", + Lifespan: "14-16 years", + }, + }, + }, + }, + } + } else { + pet = &productv1.Animal{ + Instance: &productv1.Animal_Dog{ + Dog: &productv1.Dog{ + Id: fmt.Sprintf("test-dog-%s", ctx.GetId()), + Name: fmt.Sprintf("TestDog-%s", ctx.GetName()), + Kind: "Dog", + BarkVolume: int32((i + 1) * 7), + Owner: &productv1.Owner{ + Id: fmt.Sprintf("owner-test-dog-%s", ctx.GetId()), + Name: fmt.Sprintf("OwnerTestDog for %s", ctx.GetName()), + Contact: &productv1.ContactInfo{ + Email: "owner-test-dog@example.com", + Phone: "555-666-7777", + Address: &productv1.Address{ + Street: "654 Birch Ln", + City: "Mountain View", + Country: "Canada", + ZipCode: "X9Y8Z7", + }, + }, + }, + Breed: &productv1.DogBreed{ + Id: fmt.Sprintf("breed-test-dog-%s", ctx.GetId()), + Name: fmt.Sprintf("BreedTestDog for %s", ctx.GetName()), + Origin: "England", + Characteristics: &productv1.BreedCharacteristics{ + Size: "Medium", + Temperament: "Energetic", + Lifespan: "11-13 years", + }, + }, + }, + }, + } + } + + // Alternate between ActionSuccess and ActionError for the status field (ActionResult union) + var status *productv1.ActionResult + if includeExtended && i%3 == 0 { + // Return error status for extended mode on certain items + status = &productv1.ActionResult{ + Value: &productv1.ActionResult_ActionError{ + ActionError: &productv1.ActionError{ + Message: fmt.Sprintf("Extended check failed for %s", ctx.GetName()), + Code: "EXTENDED_CHECK_FAILED", + }, + }, + } + } else { + // Return success status + status = &productv1.ActionResult{ + Value: &productv1.ActionResult_ActionSuccess{ + ActionSuccess: &productv1.ActionSuccess{ + Message: fmt.Sprintf("TestContainer %s details loaded successfully", ctx.GetName()), + Timestamp: "2024-01-01T12:00:00Z", + }, + }, + } + } + + summary := fmt.Sprintf("Summary for %s", ctx.GetName()) + if includeExtended { + summary = fmt.Sprintf("Extended summary for %s with additional details", ctx.GetName()) + } + + results = append(results, &productv1.ResolveTestContainerDetailsResult{ + Details: &productv1.TestDetails{ + Id: fmt.Sprintf("details-%s-%d", ctx.GetId(), i), + Summary: summary, + Pet: pet, + Status: status, + }, + }) + } + + return &productv1.ResolveTestContainerDetailsResponse{ + Result: results, + }, nil +} + +// ResolveCategoryMetricsNormalizedScore implements productv1.ProductServiceServer. +func (s *MockService) ResolveCategoryMetricsNormalizedScore(_ context.Context, req *productv1.ResolveCategoryMetricsNormalizedScoreRequest) (*productv1.ResolveCategoryMetricsNormalizedScoreResponse, error) { + results := make([]*productv1.ResolveCategoryMetricsNormalizedScoreResult, 0, len(req.GetContext())) + + baseline := req.GetFieldArgs().GetBaseline() + if baseline == 0 { + baseline = 1.0 // Avoid division by zero + } + + for _, ctx := range req.GetContext() { + // Calculate normalized score: (value / baseline) * 100 + // This gives a percentage relative to the baseline + normalizedScore := (ctx.GetValue() / baseline) * 100.0 + + results = append(results, &productv1.ResolveCategoryMetricsNormalizedScoreResult{ + NormalizedScore: normalizedScore, + }) + } + + resp := &productv1.ResolveCategoryMetricsNormalizedScoreResponse{ + Result: results, + } + + return resp, nil +} + +// ResolveCategoryMascot implements productv1.ProductServiceServer. +func (s *MockService) ResolveCategoryMascot(_ context.Context, req *productv1.ResolveCategoryMascotRequest) (*productv1.ResolveCategoryMascotResponse, error) { + results := make([]*productv1.ResolveCategoryMascotResult, 0, len(req.GetContext())) + + includeVolume := false + if req.GetFieldArgs() != nil { + includeVolume = req.GetFieldArgs().GetIncludeVolume() + } + + for i, ctx := range req.GetContext() { + // Return nil for certain categories to test optional return + if ctx.GetKind() == productv1.CategoryKind_CATEGORY_KIND_OTHER { + results = append(results, &productv1.ResolveCategoryMascotResult{ + Mascot: nil, + }) + } else { + // Alternate between Cat and Dog based on category kind + var animal *productv1.Animal + if ctx.GetKind() == productv1.CategoryKind_CATEGORY_KIND_BOOK || ctx.GetKind() == productv1.CategoryKind_CATEGORY_KIND_ELECTRONICS { + volume := int32(0) + if includeVolume { + volume = int32(i*10 + 5) + } + animal = &productv1.Animal{ + Instance: &productv1.Animal_Cat{ + Cat: &productv1.Cat{ + Id: fmt.Sprintf("cat-mascot-%s", ctx.GetId()), + Name: fmt.Sprintf("Whiskers-%s", ctx.GetId()), + Kind: "Cat", + MeowVolume: volume, + Owner: &productv1.Owner{ + Id: fmt.Sprintf("owner-cat-mascot-%s", ctx.GetId()), + Name: fmt.Sprintf("OwnerCatMascot for %s", ctx.GetId()), + Contact: &productv1.ContactInfo{ + Email: "owner-cat-mascot@example.com", + Phone: "555-777-8888", + Address: &productv1.Address{ + Street: "111 Maple Dr", + City: "Booktown", + Country: "USA", + ZipCode: "11111", + }, + }, + }, + Breed: &productv1.CatBreed{ + Id: fmt.Sprintf("breed-cat-mascot-%s", ctx.GetId()), + Name: fmt.Sprintf("BreedCatMascot for %s", ctx.GetId()), + Origin: "Scotland", + Characteristics: &productv1.BreedCharacteristics{ + Size: "Large", + Temperament: "Gentle", + Lifespan: "13-17 years", + }, + }, + }, + }, + } + } else { + volume := int32(0) + if includeVolume { + volume = int32(i*10 + 10) + } + animal = &productv1.Animal{ + Instance: &productv1.Animal_Dog{ + Dog: &productv1.Dog{ + Id: fmt.Sprintf("dog-mascot-%s", ctx.GetId()), + Name: fmt.Sprintf("Buddy-%s", ctx.GetId()), + Kind: "Dog", + BarkVolume: volume, + Owner: &productv1.Owner{ + Id: fmt.Sprintf("owner-dog-mascot-%s", ctx.GetId()), + Name: fmt.Sprintf("OwnerDogMascot for %s", ctx.GetId()), + Contact: &productv1.ContactInfo{ + Email: "owner-dog-mascot@example.com", + Phone: "555-888-9999", + Address: &productv1.Address{ + Street: "222 Cedar Ct", + City: "Mascotville", + Country: "USA", + ZipCode: "22222", + }, + }, + }, + Breed: &productv1.DogBreed{ + Id: fmt.Sprintf("breed-dog-mascot-%s", ctx.GetId()), + Name: fmt.Sprintf("BreedDogMascot for %s", ctx.GetId()), + Origin: "Australia", + Characteristics: &productv1.BreedCharacteristics{ + Size: "Medium", + Temperament: "Playful", + Lifespan: "10-14 years", + }, + }, + }, + }, + } + } + results = append(results, &productv1.ResolveCategoryMascotResult{ + Mascot: animal, + }) + } + } + + resp := &productv1.ResolveCategoryMascotResponse{ + Result: results, + } + + return resp, nil +} + +// ResolveCategoryCategoryStatus implements productv1.ProductServiceServer. +func (s *MockService) ResolveCategoryCategoryStatus(_ context.Context, req *productv1.ResolveCategoryCategoryStatusRequest) (*productv1.ResolveCategoryCategoryStatusResponse, error) { + results := make([]*productv1.ResolveCategoryCategoryStatusResult, 0, len(req.GetContext())) + + checkHealth := false + if req.GetFieldArgs() != nil { + checkHealth = req.GetFieldArgs().GetCheckHealth() + } + + for i, ctx := range req.GetContext() { + var actionResult *productv1.ActionResult + + if checkHealth && i%3 == 0 { + // Return error status for health check failures + actionResult = &productv1.ActionResult{ + Value: &productv1.ActionResult_ActionError{ + ActionError: &productv1.ActionError{ + Message: fmt.Sprintf("Health check failed for category %s", ctx.GetName()), + Code: "HEALTH_CHECK_FAILED", + }, + }, + } + } else { + // Return success status + actionResult = &productv1.ActionResult{ + Value: &productv1.ActionResult_ActionSuccess{ + ActionSuccess: &productv1.ActionSuccess{ + Message: fmt.Sprintf("Category %s is healthy", ctx.GetName()), + Timestamp: "2024-01-01T00:00:00Z", + }, + }, + } + } + + results = append(results, &productv1.ResolveCategoryCategoryStatusResult{ + CategoryStatus: actionResult, + }) + } + + resp := &productv1.ResolveCategoryCategoryStatusResponse{ + Result: results, + } + + return resp, nil +} + +// ResolveProductRecommendedCategory implements productv1.ProductServiceServer. +func (s *MockService) ResolveProductRecommendedCategory(_ context.Context, req *productv1.ResolveProductRecommendedCategoryRequest) (*productv1.ResolveProductRecommendedCategoryResponse, error) { + results := make([]*productv1.ResolveProductRecommendedCategoryResult, 0, len(req.GetContext())) + + maxPrice := int32(0) + if req.GetFieldArgs() != nil { + maxPrice = req.GetFieldArgs().GetMaxPrice() + } + + for _, ctx := range req.GetContext() { + // Return nil for products with high price to test optional return + if maxPrice > 0 && ctx.GetPrice() > float64(maxPrice) { + results = append(results, &productv1.ResolveProductRecommendedCategoryResult{ + RecommendedCategory: nil, + }) + } else { + // Create a recommended category based on product context + var categoryKind productv1.CategoryKind + if ctx.GetPrice() < 50 { + categoryKind = productv1.CategoryKind_CATEGORY_KIND_BOOK + } else if ctx.GetPrice() < 200 { + categoryKind = productv1.CategoryKind_CATEGORY_KIND_ELECTRONICS + } else { + categoryKind = productv1.CategoryKind_CATEGORY_KIND_FURNITURE + } + + results = append(results, &productv1.ResolveProductRecommendedCategoryResult{ + RecommendedCategory: &productv1.Category{ + Id: fmt.Sprintf("recommended-cat-%s", ctx.GetId()), + Name: fmt.Sprintf("Recommended for %s", ctx.GetName()), + Kind: categoryKind, + Subcategories: createSubcategories(fmt.Sprintf("recommended-cat-%s", ctx.GetId()), categoryKind, 2), + }, + }) + } + } + + resp := &productv1.ResolveProductRecommendedCategoryResponse{ + Result: results, + } + + return resp, nil +} + +// ResolveProductShippingEstimate implements productv1.ProductServiceServer. +func (s *MockService) ResolveProductShippingEstimate(_ context.Context, req *productv1.ResolveProductShippingEstimateRequest) (*productv1.ResolveProductShippingEstimateResponse, error) { + results := make([]*productv1.ResolveProductShippingEstimateResult, 0, len(req.GetContext())) + + for _, ctx := range req.GetContext() { + // Base shipping cost calculation + baseCost := ctx.GetPrice() * 0.1 // 10% of product price + + // Add weight-based cost if input provided + if req.GetFieldArgs() != nil && req.GetFieldArgs().GetInput() != nil { + input := req.GetFieldArgs().GetInput() + + // Add weight cost + weightCost := float64(input.GetWeight()) * 2.5 + baseCost += weightCost + + // Add expedited shipping cost + if input.GetExpedited() != nil && input.GetExpedited().GetValue() { + baseCost *= 1.5 // 50% surcharge for expedited + } + + // Add destination-based cost + destination := input.GetDestination() + switch destination { + case productv1.ShippingDestination_SHIPPING_DESTINATION_INTERNATIONAL: + baseCost += 25.0 + case productv1.ShippingDestination_SHIPPING_DESTINATION_EXPRESS: + baseCost += 10.0 + case productv1.ShippingDestination_SHIPPING_DESTINATION_DOMESTIC: + // No additional cost for domestic shipping + } + } + + results = append(results, &productv1.ResolveProductShippingEstimateResult{ + ShippingEstimate: baseCost, + }) + } + + resp := &productv1.ResolveProductShippingEstimateResponse{ + Result: results, + } + + return resp, nil +} + +// ResolveCategoryCategoryMetrics implements productv1.ProductServiceServer. +func (s *MockService) ResolveCategoryCategoryMetrics(_ context.Context, req *productv1.ResolveCategoryCategoryMetricsRequest) (*productv1.ResolveCategoryCategoryMetricsResponse, error) { + results := make([]*productv1.ResolveCategoryCategoryMetricsResult, 0, len(req.GetContext())) + + metricType := "" + if req.GetFieldArgs() != nil { + metricType = req.GetFieldArgs().GetMetricType() + } + + for i, ctx := range req.GetContext() { + // Return nil for certain metric types to test optional return + if metricType == "unavailable" { + results = append(results, &productv1.ResolveCategoryCategoryMetricsResult{ + CategoryMetrics: nil, + }) + } else { + results = append(results, &productv1.ResolveCategoryCategoryMetricsResult{ + CategoryMetrics: &productv1.CategoryMetrics{ + Id: fmt.Sprintf("metrics-%s-%d", ctx.GetId(), i), + MetricType: metricType, + Value: float64(i*25 + 100), // Different values based on index + Timestamp: "2024-01-01T00:00:00Z", + CategoryId: ctx.GetId(), + }, + }) + } + } + + resp := &productv1.ResolveCategoryCategoryMetricsResponse{ + Result: results, + } + + return resp, nil +} + +// ResolveCategoryPopularityScore implements productv1.ProductServiceServer. +func (s *MockService) ResolveCategoryPopularityScore(_ context.Context, req *productv1.ResolveCategoryPopularityScoreRequest) (*productv1.ResolveCategoryPopularityScoreResponse, error) { + results := make([]*productv1.ResolveCategoryPopularityScoreResult, 0, len(req.GetContext())) + + threshold := req.GetFieldArgs().GetThreshold() + + baseScore := 50 + for range req.GetContext() { + if int(threshold.GetValue()) > baseScore { + results = append(results, &productv1.ResolveCategoryPopularityScoreResult{ + PopularityScore: nil, + }) + } else { + results = append(results, &productv1.ResolveCategoryPopularityScoreResult{ + PopularityScore: &wrapperspb.Int32Value{Value: int32(baseScore)}, + }) + } + } + + resp := &productv1.ResolveCategoryPopularityScoreResponse{ + Result: results, + } + + return resp, nil +} + +// ResolveSubcategoryItemCount implements productv1.ProductServiceServer. +func (s *MockService) ResolveSubcategoryItemCount(_ context.Context, req *productv1.ResolveSubcategoryItemCountRequest) (*productv1.ResolveSubcategoryItemCountResponse, error) { + results := make([]*productv1.ResolveSubcategoryItemCountResult, 0, len(req.GetContext())) + for i := range req.GetContext() { + results = append(results, &productv1.ResolveSubcategoryItemCountResult{ + ItemCount: int32(i * 10), // Different multiplier to distinguish from productCount + }) + } + + resp := &productv1.ResolveSubcategoryItemCountResponse{ + Result: results, + } + + return resp, nil +} + +// ResolveCategoryProductCount implements productv1.ProductServiceServer. +func (s *MockService) ResolveCategoryProductCount(_ context.Context, req *productv1.ResolveCategoryProductCountRequest) (*productv1.ResolveCategoryProductCountResponse, error) { + results := make([]*productv1.ResolveCategoryProductCountResult, 0, len(req.GetContext())) + for i := range req.GetContext() { + results = append(results, &productv1.ResolveCategoryProductCountResult{ + ProductCount: int32(i), + }) + } + + resp := &productv1.ResolveCategoryProductCountResponse{ + Result: results, + } + + return resp, nil +} diff --git a/v2/pkg/grpctest/product.proto b/v2/pkg/grpctest/product.proto index 480a1c3ea..855c1cdde 100644 --- a/v2/pkg/grpctest/product.proto +++ b/v2/pkg/grpctest/product.proto @@ -1276,6 +1276,8 @@ message Cat { string name = 2; string kind = 3; int32 meow_volume = 4; + Owner owner = 5; + CatBreed breed = 6; } message Dog { @@ -1283,6 +1285,47 @@ message Dog { string name = 2; string kind = 3; int32 bark_volume = 4; + Owner owner = 5; + DogBreed breed = 6; +} + +message Owner { + string id = 1; + string name = 2; + ContactInfo contact = 3; +} + +message ContactInfo { + string email = 1; + string phone = 2; + Address address = 3; +} + +message Address { + string street = 1; + string city = 2; + string country = 3; + string zip_code = 4; +} + +message CatBreed { + string id = 1; + string name = 2; + string origin = 3; + BreedCharacteristics characteristics = 4; +} + +message DogBreed { + string id = 1; + string name = 2; + string origin = 3; + BreedCharacteristics characteristics = 4; +} + +message BreedCharacteristics { + string size = 1; + string temperament = 2; + string lifespan = 3; } message ActionSuccess { diff --git a/v2/pkg/grpctest/productv1/product.pb.go b/v2/pkg/grpctest/productv1/product.pb.go index 57b62207e..92c0c8187 100644 --- a/v2/pkg/grpctest/productv1/product.pb.go +++ b/v2/pkg/grpctest/productv1/product.pb.go @@ -11340,6 +11340,8 @@ type Cat struct { Name string `protobuf:"bytes,2,opt,name=name,proto3" json:"name,omitempty"` Kind string `protobuf:"bytes,3,opt,name=kind,proto3" json:"kind,omitempty"` MeowVolume int32 `protobuf:"varint,4,opt,name=meow_volume,json=meowVolume,proto3" json:"meow_volume,omitempty"` + Owner *Owner `protobuf:"bytes,5,opt,name=owner,proto3" json:"owner,omitempty"` + Breed *CatBreed `protobuf:"bytes,6,opt,name=breed,proto3" json:"breed,omitempty"` unknownFields protoimpl.UnknownFields sizeCache protoimpl.SizeCache } @@ -11402,12 +11404,28 @@ func (x *Cat) GetMeowVolume() int32 { return 0 } +func (x *Cat) GetOwner() *Owner { + if x != nil { + return x.Owner + } + return nil +} + +func (x *Cat) GetBreed() *CatBreed { + if x != nil { + return x.Breed + } + return nil +} + type Dog struct { state protoimpl.MessageState `protogen:"open.v1"` Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` Name string `protobuf:"bytes,2,opt,name=name,proto3" json:"name,omitempty"` Kind string `protobuf:"bytes,3,opt,name=kind,proto3" json:"kind,omitempty"` BarkVolume int32 `protobuf:"varint,4,opt,name=bark_volume,json=barkVolume,proto3" json:"bark_volume,omitempty"` + Owner *Owner `protobuf:"bytes,5,opt,name=owner,proto3" json:"owner,omitempty"` + Breed *DogBreed `protobuf:"bytes,6,opt,name=breed,proto3" json:"breed,omitempty"` unknownFields protoimpl.UnknownFields sizeCache protoimpl.SizeCache } @@ -11470,6 +11488,404 @@ func (x *Dog) GetBarkVolume() int32 { return 0 } +func (x *Dog) GetOwner() *Owner { + if x != nil { + return x.Owner + } + return nil +} + +func (x *Dog) GetBreed() *DogBreed { + if x != nil { + return x.Breed + } + return nil +} + +type Owner struct { + state protoimpl.MessageState `protogen:"open.v1"` + Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` + Name string `protobuf:"bytes,2,opt,name=name,proto3" json:"name,omitempty"` + Contact *ContactInfo `protobuf:"bytes,3,opt,name=contact,proto3" json:"contact,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *Owner) Reset() { + *x = Owner{} + mi := &file_product_proto_msgTypes[223] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *Owner) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Owner) ProtoMessage() {} + +func (x *Owner) ProtoReflect() protoreflect.Message { + mi := &file_product_proto_msgTypes[223] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use Owner.ProtoReflect.Descriptor instead. +func (*Owner) Descriptor() ([]byte, []int) { + return file_product_proto_rawDescGZIP(), []int{223} +} + +func (x *Owner) GetId() string { + if x != nil { + return x.Id + } + return "" +} + +func (x *Owner) GetName() string { + if x != nil { + return x.Name + } + return "" +} + +func (x *Owner) GetContact() *ContactInfo { + if x != nil { + return x.Contact + } + return nil +} + +type ContactInfo struct { + state protoimpl.MessageState `protogen:"open.v1"` + Email string `protobuf:"bytes,1,opt,name=email,proto3" json:"email,omitempty"` + Phone string `protobuf:"bytes,2,opt,name=phone,proto3" json:"phone,omitempty"` + Address *Address `protobuf:"bytes,3,opt,name=address,proto3" json:"address,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ContactInfo) Reset() { + *x = ContactInfo{} + mi := &file_product_proto_msgTypes[224] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ContactInfo) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ContactInfo) ProtoMessage() {} + +func (x *ContactInfo) ProtoReflect() protoreflect.Message { + mi := &file_product_proto_msgTypes[224] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ContactInfo.ProtoReflect.Descriptor instead. +func (*ContactInfo) Descriptor() ([]byte, []int) { + return file_product_proto_rawDescGZIP(), []int{224} +} + +func (x *ContactInfo) GetEmail() string { + if x != nil { + return x.Email + } + return "" +} + +func (x *ContactInfo) GetPhone() string { + if x != nil { + return x.Phone + } + return "" +} + +func (x *ContactInfo) GetAddress() *Address { + if x != nil { + return x.Address + } + return nil +} + +type Address struct { + state protoimpl.MessageState `protogen:"open.v1"` + Street string `protobuf:"bytes,1,opt,name=street,proto3" json:"street,omitempty"` + City string `protobuf:"bytes,2,opt,name=city,proto3" json:"city,omitempty"` + Country string `protobuf:"bytes,3,opt,name=country,proto3" json:"country,omitempty"` + ZipCode string `protobuf:"bytes,4,opt,name=zip_code,json=zipCode,proto3" json:"zip_code,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *Address) Reset() { + *x = Address{} + mi := &file_product_proto_msgTypes[225] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *Address) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Address) ProtoMessage() {} + +func (x *Address) ProtoReflect() protoreflect.Message { + mi := &file_product_proto_msgTypes[225] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use Address.ProtoReflect.Descriptor instead. +func (*Address) Descriptor() ([]byte, []int) { + return file_product_proto_rawDescGZIP(), []int{225} +} + +func (x *Address) GetStreet() string { + if x != nil { + return x.Street + } + return "" +} + +func (x *Address) GetCity() string { + if x != nil { + return x.City + } + return "" +} + +func (x *Address) GetCountry() string { + if x != nil { + return x.Country + } + return "" +} + +func (x *Address) GetZipCode() string { + if x != nil { + return x.ZipCode + } + return "" +} + +type CatBreed struct { + state protoimpl.MessageState `protogen:"open.v1"` + Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` + Name string `protobuf:"bytes,2,opt,name=name,proto3" json:"name,omitempty"` + Origin string `protobuf:"bytes,3,opt,name=origin,proto3" json:"origin,omitempty"` + Characteristics *BreedCharacteristics `protobuf:"bytes,4,opt,name=characteristics,proto3" json:"characteristics,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *CatBreed) Reset() { + *x = CatBreed{} + mi := &file_product_proto_msgTypes[226] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *CatBreed) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*CatBreed) ProtoMessage() {} + +func (x *CatBreed) ProtoReflect() protoreflect.Message { + mi := &file_product_proto_msgTypes[226] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use CatBreed.ProtoReflect.Descriptor instead. +func (*CatBreed) Descriptor() ([]byte, []int) { + return file_product_proto_rawDescGZIP(), []int{226} +} + +func (x *CatBreed) GetId() string { + if x != nil { + return x.Id + } + return "" +} + +func (x *CatBreed) GetName() string { + if x != nil { + return x.Name + } + return "" +} + +func (x *CatBreed) GetOrigin() string { + if x != nil { + return x.Origin + } + return "" +} + +func (x *CatBreed) GetCharacteristics() *BreedCharacteristics { + if x != nil { + return x.Characteristics + } + return nil +} + +type DogBreed struct { + state protoimpl.MessageState `protogen:"open.v1"` + Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` + Name string `protobuf:"bytes,2,opt,name=name,proto3" json:"name,omitempty"` + Origin string `protobuf:"bytes,3,opt,name=origin,proto3" json:"origin,omitempty"` + Characteristics *BreedCharacteristics `protobuf:"bytes,4,opt,name=characteristics,proto3" json:"characteristics,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *DogBreed) Reset() { + *x = DogBreed{} + mi := &file_product_proto_msgTypes[227] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *DogBreed) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*DogBreed) ProtoMessage() {} + +func (x *DogBreed) ProtoReflect() protoreflect.Message { + mi := &file_product_proto_msgTypes[227] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use DogBreed.ProtoReflect.Descriptor instead. +func (*DogBreed) Descriptor() ([]byte, []int) { + return file_product_proto_rawDescGZIP(), []int{227} +} + +func (x *DogBreed) GetId() string { + if x != nil { + return x.Id + } + return "" +} + +func (x *DogBreed) GetName() string { + if x != nil { + return x.Name + } + return "" +} + +func (x *DogBreed) GetOrigin() string { + if x != nil { + return x.Origin + } + return "" +} + +func (x *DogBreed) GetCharacteristics() *BreedCharacteristics { + if x != nil { + return x.Characteristics + } + return nil +} + +type BreedCharacteristics struct { + state protoimpl.MessageState `protogen:"open.v1"` + Size string `protobuf:"bytes,1,opt,name=size,proto3" json:"size,omitempty"` + Temperament string `protobuf:"bytes,2,opt,name=temperament,proto3" json:"temperament,omitempty"` + Lifespan string `protobuf:"bytes,3,opt,name=lifespan,proto3" json:"lifespan,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *BreedCharacteristics) Reset() { + *x = BreedCharacteristics{} + mi := &file_product_proto_msgTypes[228] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *BreedCharacteristics) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*BreedCharacteristics) ProtoMessage() {} + +func (x *BreedCharacteristics) ProtoReflect() protoreflect.Message { + mi := &file_product_proto_msgTypes[228] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use BreedCharacteristics.ProtoReflect.Descriptor instead. +func (*BreedCharacteristics) Descriptor() ([]byte, []int) { + return file_product_proto_rawDescGZIP(), []int{228} +} + +func (x *BreedCharacteristics) GetSize() string { + if x != nil { + return x.Size + } + return "" +} + +func (x *BreedCharacteristics) GetTemperament() string { + if x != nil { + return x.Temperament + } + return "" +} + +func (x *BreedCharacteristics) GetLifespan() string { + if x != nil { + return x.Lifespan + } + return "" +} + type ActionSuccess struct { state protoimpl.MessageState `protogen:"open.v1"` Message string `protobuf:"bytes,1,opt,name=message,proto3" json:"message,omitempty"` @@ -11480,7 +11896,7 @@ type ActionSuccess struct { func (x *ActionSuccess) Reset() { *x = ActionSuccess{} - mi := &file_product_proto_msgTypes[223] + mi := &file_product_proto_msgTypes[229] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -11492,7 +11908,7 @@ func (x *ActionSuccess) String() string { func (*ActionSuccess) ProtoMessage() {} func (x *ActionSuccess) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[223] + mi := &file_product_proto_msgTypes[229] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -11505,7 +11921,7 @@ func (x *ActionSuccess) ProtoReflect() protoreflect.Message { // Deprecated: Use ActionSuccess.ProtoReflect.Descriptor instead. func (*ActionSuccess) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{223} + return file_product_proto_rawDescGZIP(), []int{229} } func (x *ActionSuccess) GetMessage() string { @@ -11532,7 +11948,7 @@ type ActionError struct { func (x *ActionError) Reset() { *x = ActionError{} - mi := &file_product_proto_msgTypes[224] + mi := &file_product_proto_msgTypes[230] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -11544,7 +11960,7 @@ func (x *ActionError) String() string { func (*ActionError) ProtoMessage() {} func (x *ActionError) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[224] + mi := &file_product_proto_msgTypes[230] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -11557,7 +11973,7 @@ func (x *ActionError) ProtoReflect() protoreflect.Message { // Deprecated: Use ActionError.ProtoReflect.Descriptor instead. func (*ActionError) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{224} + return file_product_proto_rawDescGZIP(), []int{230} } func (x *ActionError) GetMessage() string { @@ -11586,7 +12002,7 @@ type TestDetails struct { func (x *TestDetails) Reset() { *x = TestDetails{} - mi := &file_product_proto_msgTypes[225] + mi := &file_product_proto_msgTypes[231] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -11598,7 +12014,7 @@ func (x *TestDetails) String() string { func (*TestDetails) ProtoMessage() {} func (x *TestDetails) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[225] + mi := &file_product_proto_msgTypes[231] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -11611,7 +12027,7 @@ func (x *TestDetails) ProtoReflect() protoreflect.Message { // Deprecated: Use TestDetails.ProtoReflect.Descriptor instead. func (*TestDetails) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{225} + return file_product_proto_rawDescGZIP(), []int{231} } func (x *TestDetails) GetId() string { @@ -11652,7 +12068,7 @@ type CategoryInput struct { func (x *CategoryInput) Reset() { *x = CategoryInput{} - mi := &file_product_proto_msgTypes[226] + mi := &file_product_proto_msgTypes[232] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -11664,7 +12080,7 @@ func (x *CategoryInput) String() string { func (*CategoryInput) ProtoMessage() {} func (x *CategoryInput) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[226] + mi := &file_product_proto_msgTypes[232] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -11677,7 +12093,7 @@ func (x *CategoryInput) ProtoReflect() protoreflect.Message { // Deprecated: Use CategoryInput.ProtoReflect.Descriptor instead. func (*CategoryInput) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{226} + return file_product_proto_rawDescGZIP(), []int{232} } func (x *CategoryInput) GetName() string { @@ -11706,7 +12122,7 @@ type ProductCountFilter struct { func (x *ProductCountFilter) Reset() { *x = ProductCountFilter{} - mi := &file_product_proto_msgTypes[227] + mi := &file_product_proto_msgTypes[233] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -11718,7 +12134,7 @@ func (x *ProductCountFilter) String() string { func (*ProductCountFilter) ProtoMessage() {} func (x *ProductCountFilter) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[227] + mi := &file_product_proto_msgTypes[233] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -11731,7 +12147,7 @@ func (x *ProductCountFilter) ProtoReflect() protoreflect.Message { // Deprecated: Use ProductCountFilter.ProtoReflect.Descriptor instead. func (*ProductCountFilter) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{227} + return file_product_proto_rawDescGZIP(), []int{233} } func (x *ProductCountFilter) GetMinPrice() *wrapperspb.DoubleValue { @@ -11775,7 +12191,7 @@ type SubcategoryItemFilter struct { func (x *SubcategoryItemFilter) Reset() { *x = SubcategoryItemFilter{} - mi := &file_product_proto_msgTypes[228] + mi := &file_product_proto_msgTypes[234] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -11787,7 +12203,7 @@ func (x *SubcategoryItemFilter) String() string { func (*SubcategoryItemFilter) ProtoMessage() {} func (x *SubcategoryItemFilter) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[228] + mi := &file_product_proto_msgTypes[234] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -11800,7 +12216,7 @@ func (x *SubcategoryItemFilter) ProtoReflect() protoreflect.Message { // Deprecated: Use SubcategoryItemFilter.ProtoReflect.Descriptor instead. func (*SubcategoryItemFilter) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{228} + return file_product_proto_rawDescGZIP(), []int{234} } func (x *SubcategoryItemFilter) GetMinPrice() *wrapperspb.DoubleValue { @@ -11849,7 +12265,7 @@ type ShippingEstimateInput struct { func (x *ShippingEstimateInput) Reset() { *x = ShippingEstimateInput{} - mi := &file_product_proto_msgTypes[229] + mi := &file_product_proto_msgTypes[235] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -11861,7 +12277,7 @@ func (x *ShippingEstimateInput) String() string { func (*ShippingEstimateInput) ProtoMessage() {} func (x *ShippingEstimateInput) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[229] + mi := &file_product_proto_msgTypes[235] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -11874,7 +12290,7 @@ func (x *ShippingEstimateInput) ProtoReflect() protoreflect.Message { // Deprecated: Use ShippingEstimateInput.ProtoReflect.Descriptor instead. func (*ShippingEstimateInput) Descriptor() ([]byte, []int) { - return file_product_proto_rawDescGZIP(), []int{229} + return file_product_proto_rawDescGZIP(), []int{235} } func (x *ShippingEstimateInput) GetDestination() ShippingDestination { @@ -11907,7 +12323,7 @@ type ListOfAuthorFilter_List struct { func (x *ListOfAuthorFilter_List) Reset() { *x = ListOfAuthorFilter_List{} - mi := &file_product_proto_msgTypes[230] + mi := &file_product_proto_msgTypes[236] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -11919,7 +12335,7 @@ func (x *ListOfAuthorFilter_List) String() string { func (*ListOfAuthorFilter_List) ProtoMessage() {} func (x *ListOfAuthorFilter_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[230] + mi := &file_product_proto_msgTypes[236] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -11951,7 +12367,7 @@ type ListOfAuthorInput_List struct { func (x *ListOfAuthorInput_List) Reset() { *x = ListOfAuthorInput_List{} - mi := &file_product_proto_msgTypes[231] + mi := &file_product_proto_msgTypes[237] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -11963,7 +12379,7 @@ func (x *ListOfAuthorInput_List) String() string { func (*ListOfAuthorInput_List) ProtoMessage() {} func (x *ListOfAuthorInput_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[231] + mi := &file_product_proto_msgTypes[237] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -11995,7 +12411,7 @@ type ListOfBlogPost_List struct { func (x *ListOfBlogPost_List) Reset() { *x = ListOfBlogPost_List{} - mi := &file_product_proto_msgTypes[232] + mi := &file_product_proto_msgTypes[238] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -12007,7 +12423,7 @@ func (x *ListOfBlogPost_List) String() string { func (*ListOfBlogPost_List) ProtoMessage() {} func (x *ListOfBlogPost_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[232] + mi := &file_product_proto_msgTypes[238] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -12039,7 +12455,7 @@ type ListOfBlogPostFilter_List struct { func (x *ListOfBlogPostFilter_List) Reset() { *x = ListOfBlogPostFilter_List{} - mi := &file_product_proto_msgTypes[233] + mi := &file_product_proto_msgTypes[239] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -12051,7 +12467,7 @@ func (x *ListOfBlogPostFilter_List) String() string { func (*ListOfBlogPostFilter_List) ProtoMessage() {} func (x *ListOfBlogPostFilter_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[233] + mi := &file_product_proto_msgTypes[239] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -12083,7 +12499,7 @@ type ListOfBlogPostInput_List struct { func (x *ListOfBlogPostInput_List) Reset() { *x = ListOfBlogPostInput_List{} - mi := &file_product_proto_msgTypes[234] + mi := &file_product_proto_msgTypes[240] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -12095,7 +12511,7 @@ func (x *ListOfBlogPostInput_List) String() string { func (*ListOfBlogPostInput_List) ProtoMessage() {} func (x *ListOfBlogPostInput_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[234] + mi := &file_product_proto_msgTypes[240] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -12127,7 +12543,7 @@ type ListOfBoolean_List struct { func (x *ListOfBoolean_List) Reset() { *x = ListOfBoolean_List{} - mi := &file_product_proto_msgTypes[235] + mi := &file_product_proto_msgTypes[241] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -12139,7 +12555,7 @@ func (x *ListOfBoolean_List) String() string { func (*ListOfBoolean_List) ProtoMessage() {} func (x *ListOfBoolean_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[235] + mi := &file_product_proto_msgTypes[241] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -12171,7 +12587,7 @@ type ListOfCategory_List struct { func (x *ListOfCategory_List) Reset() { *x = ListOfCategory_List{} - mi := &file_product_proto_msgTypes[236] + mi := &file_product_proto_msgTypes[242] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -12183,7 +12599,7 @@ func (x *ListOfCategory_List) String() string { func (*ListOfCategory_List) ProtoMessage() {} func (x *ListOfCategory_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[236] + mi := &file_product_proto_msgTypes[242] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -12215,7 +12631,7 @@ type ListOfCategoryInput_List struct { func (x *ListOfCategoryInput_List) Reset() { *x = ListOfCategoryInput_List{} - mi := &file_product_proto_msgTypes[237] + mi := &file_product_proto_msgTypes[243] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -12227,7 +12643,7 @@ func (x *ListOfCategoryInput_List) String() string { func (*ListOfCategoryInput_List) ProtoMessage() {} func (x *ListOfCategoryInput_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[237] + mi := &file_product_proto_msgTypes[243] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -12259,7 +12675,7 @@ type ListOfFloat_List struct { func (x *ListOfFloat_List) Reset() { *x = ListOfFloat_List{} - mi := &file_product_proto_msgTypes[238] + mi := &file_product_proto_msgTypes[244] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -12271,7 +12687,7 @@ func (x *ListOfFloat_List) String() string { func (*ListOfFloat_List) ProtoMessage() {} func (x *ListOfFloat_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[238] + mi := &file_product_proto_msgTypes[244] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -12303,7 +12719,7 @@ type ListOfListOfCategory_List struct { func (x *ListOfListOfCategory_List) Reset() { *x = ListOfListOfCategory_List{} - mi := &file_product_proto_msgTypes[239] + mi := &file_product_proto_msgTypes[245] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -12315,7 +12731,7 @@ func (x *ListOfListOfCategory_List) String() string { func (*ListOfListOfCategory_List) ProtoMessage() {} func (x *ListOfListOfCategory_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[239] + mi := &file_product_proto_msgTypes[245] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -12347,7 +12763,7 @@ type ListOfListOfCategoryInput_List struct { func (x *ListOfListOfCategoryInput_List) Reset() { *x = ListOfListOfCategoryInput_List{} - mi := &file_product_proto_msgTypes[240] + mi := &file_product_proto_msgTypes[246] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -12359,7 +12775,7 @@ func (x *ListOfListOfCategoryInput_List) String() string { func (*ListOfListOfCategoryInput_List) ProtoMessage() {} func (x *ListOfListOfCategoryInput_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[240] + mi := &file_product_proto_msgTypes[246] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -12391,7 +12807,7 @@ type ListOfListOfString_List struct { func (x *ListOfListOfString_List) Reset() { *x = ListOfListOfString_List{} - mi := &file_product_proto_msgTypes[241] + mi := &file_product_proto_msgTypes[247] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -12403,7 +12819,7 @@ func (x *ListOfListOfString_List) String() string { func (*ListOfListOfString_List) ProtoMessage() {} func (x *ListOfListOfString_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[241] + mi := &file_product_proto_msgTypes[247] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -12435,7 +12851,7 @@ type ListOfListOfUser_List struct { func (x *ListOfListOfUser_List) Reset() { *x = ListOfListOfUser_List{} - mi := &file_product_proto_msgTypes[242] + mi := &file_product_proto_msgTypes[248] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -12447,7 +12863,7 @@ func (x *ListOfListOfUser_List) String() string { func (*ListOfListOfUser_List) ProtoMessage() {} func (x *ListOfListOfUser_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[242] + mi := &file_product_proto_msgTypes[248] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -12479,7 +12895,7 @@ type ListOfListOfUserInput_List struct { func (x *ListOfListOfUserInput_List) Reset() { *x = ListOfListOfUserInput_List{} - mi := &file_product_proto_msgTypes[243] + mi := &file_product_proto_msgTypes[249] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -12491,7 +12907,7 @@ func (x *ListOfListOfUserInput_List) String() string { func (*ListOfListOfUserInput_List) ProtoMessage() {} func (x *ListOfListOfUserInput_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[243] + mi := &file_product_proto_msgTypes[249] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -12523,7 +12939,7 @@ type ListOfOrderLine_List struct { func (x *ListOfOrderLine_List) Reset() { *x = ListOfOrderLine_List{} - mi := &file_product_proto_msgTypes[244] + mi := &file_product_proto_msgTypes[250] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -12535,7 +12951,7 @@ func (x *ListOfOrderLine_List) String() string { func (*ListOfOrderLine_List) ProtoMessage() {} func (x *ListOfOrderLine_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[244] + mi := &file_product_proto_msgTypes[250] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -12567,7 +12983,7 @@ type ListOfProduct_List struct { func (x *ListOfProduct_List) Reset() { *x = ListOfProduct_List{} - mi := &file_product_proto_msgTypes[245] + mi := &file_product_proto_msgTypes[251] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -12579,7 +12995,7 @@ func (x *ListOfProduct_List) String() string { func (*ListOfProduct_List) ProtoMessage() {} func (x *ListOfProduct_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[245] + mi := &file_product_proto_msgTypes[251] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -12611,7 +13027,7 @@ type ListOfString_List struct { func (x *ListOfString_List) Reset() { *x = ListOfString_List{} - mi := &file_product_proto_msgTypes[246] + mi := &file_product_proto_msgTypes[252] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -12623,7 +13039,7 @@ func (x *ListOfString_List) String() string { func (*ListOfString_List) ProtoMessage() {} func (x *ListOfString_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[246] + mi := &file_product_proto_msgTypes[252] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -12655,7 +13071,7 @@ type ListOfSubcategory_List struct { func (x *ListOfSubcategory_List) Reset() { *x = ListOfSubcategory_List{} - mi := &file_product_proto_msgTypes[247] + mi := &file_product_proto_msgTypes[253] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -12667,7 +13083,7 @@ func (x *ListOfSubcategory_List) String() string { func (*ListOfSubcategory_List) ProtoMessage() {} func (x *ListOfSubcategory_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[247] + mi := &file_product_proto_msgTypes[253] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -12699,7 +13115,7 @@ type ListOfUser_List struct { func (x *ListOfUser_List) Reset() { *x = ListOfUser_List{} - mi := &file_product_proto_msgTypes[248] + mi := &file_product_proto_msgTypes[254] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -12711,7 +13127,7 @@ func (x *ListOfUser_List) String() string { func (*ListOfUser_List) ProtoMessage() {} func (x *ListOfUser_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[248] + mi := &file_product_proto_msgTypes[254] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -12743,7 +13159,7 @@ type ListOfUserInput_List struct { func (x *ListOfUserInput_List) Reset() { *x = ListOfUserInput_List{} - mi := &file_product_proto_msgTypes[249] + mi := &file_product_proto_msgTypes[255] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -12755,7 +13171,7 @@ func (x *ListOfUserInput_List) String() string { func (*ListOfUserInput_List) ProtoMessage() {} func (x *ListOfUserInput_List) ProtoReflect() protoreflect.Message { - mi := &file_product_proto_msgTypes[249] + mi := &file_product_proto_msgTypes[255] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -13470,19 +13886,50 @@ const file_product_proto_rawDesc = "" + "\ttimestamp\x18\x04 \x01(\tR\ttimestamp\x12\x1f\n" + "\vcategory_id\x18\x05 \x01(\tR\n" + "categoryId\x12>\n" + - "\x10related_category\x18\x06 \x01(\v2\x13.productv1.CategoryR\x0frelatedCategory\"^\n" + + "\x10related_category\x18\x06 \x01(\v2\x13.productv1.CategoryR\x0frelatedCategory\"\xb1\x01\n" + "\x03Cat\x12\x0e\n" + "\x02id\x18\x01 \x01(\tR\x02id\x12\x12\n" + "\x04name\x18\x02 \x01(\tR\x04name\x12\x12\n" + "\x04kind\x18\x03 \x01(\tR\x04kind\x12\x1f\n" + "\vmeow_volume\x18\x04 \x01(\x05R\n" + - "meowVolume\"^\n" + + "meowVolume\x12&\n" + + "\x05owner\x18\x05 \x01(\v2\x10.productv1.OwnerR\x05owner\x12)\n" + + "\x05breed\x18\x06 \x01(\v2\x13.productv1.CatBreedR\x05breed\"\xb1\x01\n" + "\x03Dog\x12\x0e\n" + "\x02id\x18\x01 \x01(\tR\x02id\x12\x12\n" + "\x04name\x18\x02 \x01(\tR\x04name\x12\x12\n" + "\x04kind\x18\x03 \x01(\tR\x04kind\x12\x1f\n" + "\vbark_volume\x18\x04 \x01(\x05R\n" + - "barkVolume\"G\n" + + "barkVolume\x12&\n" + + "\x05owner\x18\x05 \x01(\v2\x10.productv1.OwnerR\x05owner\x12)\n" + + "\x05breed\x18\x06 \x01(\v2\x13.productv1.DogBreedR\x05breed\"]\n" + + "\x05Owner\x12\x0e\n" + + "\x02id\x18\x01 \x01(\tR\x02id\x12\x12\n" + + "\x04name\x18\x02 \x01(\tR\x04name\x120\n" + + "\acontact\x18\x03 \x01(\v2\x16.productv1.ContactInfoR\acontact\"g\n" + + "\vContactInfo\x12\x14\n" + + "\x05email\x18\x01 \x01(\tR\x05email\x12\x14\n" + + "\x05phone\x18\x02 \x01(\tR\x05phone\x12,\n" + + "\aaddress\x18\x03 \x01(\v2\x12.productv1.AddressR\aaddress\"j\n" + + "\aAddress\x12\x16\n" + + "\x06street\x18\x01 \x01(\tR\x06street\x12\x12\n" + + "\x04city\x18\x02 \x01(\tR\x04city\x12\x18\n" + + "\acountry\x18\x03 \x01(\tR\acountry\x12\x19\n" + + "\bzip_code\x18\x04 \x01(\tR\azipCode\"\x91\x01\n" + + "\bCatBreed\x12\x0e\n" + + "\x02id\x18\x01 \x01(\tR\x02id\x12\x12\n" + + "\x04name\x18\x02 \x01(\tR\x04name\x12\x16\n" + + "\x06origin\x18\x03 \x01(\tR\x06origin\x12I\n" + + "\x0fcharacteristics\x18\x04 \x01(\v2\x1f.productv1.BreedCharacteristicsR\x0fcharacteristics\"\x91\x01\n" + + "\bDogBreed\x12\x0e\n" + + "\x02id\x18\x01 \x01(\tR\x02id\x12\x12\n" + + "\x04name\x18\x02 \x01(\tR\x04name\x12\x16\n" + + "\x06origin\x18\x03 \x01(\tR\x06origin\x12I\n" + + "\x0fcharacteristics\x18\x04 \x01(\v2\x1f.productv1.BreedCharacteristicsR\x0fcharacteristics\"h\n" + + "\x14BreedCharacteristics\x12\x12\n" + + "\x04size\x18\x01 \x01(\tR\x04size\x12 \n" + + "\vtemperament\x18\x02 \x01(\tR\vtemperament\x12\x1a\n" + + "\blifespan\x18\x03 \x01(\tR\blifespan\"G\n" + "\rActionSuccess\x12\x18\n" + "\amessage\x18\x01 \x01(\tR\amessage\x12\x1c\n" + "\ttimestamp\x18\x02 \x01(\tR\ttimestamp\";\n" + @@ -13601,7 +14048,7 @@ func file_product_proto_rawDescGZIP() []byte { } var file_product_proto_enumTypes = make([]protoimpl.EnumInfo, 2) -var file_product_proto_msgTypes = make([]protoimpl.MessageInfo, 250) +var file_product_proto_msgTypes = make([]protoimpl.MessageInfo, 256) var file_product_proto_goTypes = []any{ (CategoryKind)(0), // 0: productv1.CategoryKind (ShippingDestination)(0), // 1: productv1.ShippingDestination @@ -13828,59 +14275,65 @@ var file_product_proto_goTypes = []any{ (*CategoryMetrics)(nil), // 222: productv1.CategoryMetrics (*Cat)(nil), // 223: productv1.Cat (*Dog)(nil), // 224: productv1.Dog - (*ActionSuccess)(nil), // 225: productv1.ActionSuccess - (*ActionError)(nil), // 226: productv1.ActionError - (*TestDetails)(nil), // 227: productv1.TestDetails - (*CategoryInput)(nil), // 228: productv1.CategoryInput - (*ProductCountFilter)(nil), // 229: productv1.ProductCountFilter - (*SubcategoryItemFilter)(nil), // 230: productv1.SubcategoryItemFilter - (*ShippingEstimateInput)(nil), // 231: productv1.ShippingEstimateInput - (*ListOfAuthorFilter_List)(nil), // 232: productv1.ListOfAuthorFilter.List - (*ListOfAuthorInput_List)(nil), // 233: productv1.ListOfAuthorInput.List - (*ListOfBlogPost_List)(nil), // 234: productv1.ListOfBlogPost.List - (*ListOfBlogPostFilter_List)(nil), // 235: productv1.ListOfBlogPostFilter.List - (*ListOfBlogPostInput_List)(nil), // 236: productv1.ListOfBlogPostInput.List - (*ListOfBoolean_List)(nil), // 237: productv1.ListOfBoolean.List - (*ListOfCategory_List)(nil), // 238: productv1.ListOfCategory.List - (*ListOfCategoryInput_List)(nil), // 239: productv1.ListOfCategoryInput.List - (*ListOfFloat_List)(nil), // 240: productv1.ListOfFloat.List - (*ListOfListOfCategory_List)(nil), // 241: productv1.ListOfListOfCategory.List - (*ListOfListOfCategoryInput_List)(nil), // 242: productv1.ListOfListOfCategoryInput.List - (*ListOfListOfString_List)(nil), // 243: productv1.ListOfListOfString.List - (*ListOfListOfUser_List)(nil), // 244: productv1.ListOfListOfUser.List - (*ListOfListOfUserInput_List)(nil), // 245: productv1.ListOfListOfUserInput.List - (*ListOfOrderLine_List)(nil), // 246: productv1.ListOfOrderLine.List - (*ListOfProduct_List)(nil), // 247: productv1.ListOfProduct.List - (*ListOfString_List)(nil), // 248: productv1.ListOfString.List - (*ListOfSubcategory_List)(nil), // 249: productv1.ListOfSubcategory.List - (*ListOfUser_List)(nil), // 250: productv1.ListOfUser.List - (*ListOfUserInput_List)(nil), // 251: productv1.ListOfUserInput.List - (*wrapperspb.Int32Value)(nil), // 252: google.protobuf.Int32Value - (*wrapperspb.StringValue)(nil), // 253: google.protobuf.StringValue - (*wrapperspb.DoubleValue)(nil), // 254: google.protobuf.DoubleValue - (*wrapperspb.BoolValue)(nil), // 255: google.protobuf.BoolValue + (*Owner)(nil), // 225: productv1.Owner + (*ContactInfo)(nil), // 226: productv1.ContactInfo + (*Address)(nil), // 227: productv1.Address + (*CatBreed)(nil), // 228: productv1.CatBreed + (*DogBreed)(nil), // 229: productv1.DogBreed + (*BreedCharacteristics)(nil), // 230: productv1.BreedCharacteristics + (*ActionSuccess)(nil), // 231: productv1.ActionSuccess + (*ActionError)(nil), // 232: productv1.ActionError + (*TestDetails)(nil), // 233: productv1.TestDetails + (*CategoryInput)(nil), // 234: productv1.CategoryInput + (*ProductCountFilter)(nil), // 235: productv1.ProductCountFilter + (*SubcategoryItemFilter)(nil), // 236: productv1.SubcategoryItemFilter + (*ShippingEstimateInput)(nil), // 237: productv1.ShippingEstimateInput + (*ListOfAuthorFilter_List)(nil), // 238: productv1.ListOfAuthorFilter.List + (*ListOfAuthorInput_List)(nil), // 239: productv1.ListOfAuthorInput.List + (*ListOfBlogPost_List)(nil), // 240: productv1.ListOfBlogPost.List + (*ListOfBlogPostFilter_List)(nil), // 241: productv1.ListOfBlogPostFilter.List + (*ListOfBlogPostInput_List)(nil), // 242: productv1.ListOfBlogPostInput.List + (*ListOfBoolean_List)(nil), // 243: productv1.ListOfBoolean.List + (*ListOfCategory_List)(nil), // 244: productv1.ListOfCategory.List + (*ListOfCategoryInput_List)(nil), // 245: productv1.ListOfCategoryInput.List + (*ListOfFloat_List)(nil), // 246: productv1.ListOfFloat.List + (*ListOfListOfCategory_List)(nil), // 247: productv1.ListOfListOfCategory.List + (*ListOfListOfCategoryInput_List)(nil), // 248: productv1.ListOfListOfCategoryInput.List + (*ListOfListOfString_List)(nil), // 249: productv1.ListOfListOfString.List + (*ListOfListOfUser_List)(nil), // 250: productv1.ListOfListOfUser.List + (*ListOfListOfUserInput_List)(nil), // 251: productv1.ListOfListOfUserInput.List + (*ListOfOrderLine_List)(nil), // 252: productv1.ListOfOrderLine.List + (*ListOfProduct_List)(nil), // 253: productv1.ListOfProduct.List + (*ListOfString_List)(nil), // 254: productv1.ListOfString.List + (*ListOfSubcategory_List)(nil), // 255: productv1.ListOfSubcategory.List + (*ListOfUser_List)(nil), // 256: productv1.ListOfUser.List + (*ListOfUserInput_List)(nil), // 257: productv1.ListOfUserInput.List + (*wrapperspb.Int32Value)(nil), // 258: google.protobuf.Int32Value + (*wrapperspb.StringValue)(nil), // 259: google.protobuf.StringValue + (*wrapperspb.DoubleValue)(nil), // 260: google.protobuf.DoubleValue + (*wrapperspb.BoolValue)(nil), // 261: google.protobuf.BoolValue } var file_product_proto_depIdxs = []int32{ - 232, // 0: productv1.ListOfAuthorFilter.list:type_name -> productv1.ListOfAuthorFilter.List - 233, // 1: productv1.ListOfAuthorInput.list:type_name -> productv1.ListOfAuthorInput.List - 234, // 2: productv1.ListOfBlogPost.list:type_name -> productv1.ListOfBlogPost.List - 235, // 3: productv1.ListOfBlogPostFilter.list:type_name -> productv1.ListOfBlogPostFilter.List - 236, // 4: productv1.ListOfBlogPostInput.list:type_name -> productv1.ListOfBlogPostInput.List - 237, // 5: productv1.ListOfBoolean.list:type_name -> productv1.ListOfBoolean.List - 238, // 6: productv1.ListOfCategory.list:type_name -> productv1.ListOfCategory.List - 239, // 7: productv1.ListOfCategoryInput.list:type_name -> productv1.ListOfCategoryInput.List - 240, // 8: productv1.ListOfFloat.list:type_name -> productv1.ListOfFloat.List - 241, // 9: productv1.ListOfListOfCategory.list:type_name -> productv1.ListOfListOfCategory.List - 242, // 10: productv1.ListOfListOfCategoryInput.list:type_name -> productv1.ListOfListOfCategoryInput.List - 243, // 11: productv1.ListOfListOfString.list:type_name -> productv1.ListOfListOfString.List - 244, // 12: productv1.ListOfListOfUser.list:type_name -> productv1.ListOfListOfUser.List - 245, // 13: productv1.ListOfListOfUserInput.list:type_name -> productv1.ListOfListOfUserInput.List - 246, // 14: productv1.ListOfOrderLine.list:type_name -> productv1.ListOfOrderLine.List - 247, // 15: productv1.ListOfProduct.list:type_name -> productv1.ListOfProduct.List - 248, // 16: productv1.ListOfString.list:type_name -> productv1.ListOfString.List - 249, // 17: productv1.ListOfSubcategory.list:type_name -> productv1.ListOfSubcategory.List - 250, // 18: productv1.ListOfUser.list:type_name -> productv1.ListOfUser.List - 251, // 19: productv1.ListOfUserInput.list:type_name -> productv1.ListOfUserInput.List + 238, // 0: productv1.ListOfAuthorFilter.list:type_name -> productv1.ListOfAuthorFilter.List + 239, // 1: productv1.ListOfAuthorInput.list:type_name -> productv1.ListOfAuthorInput.List + 240, // 2: productv1.ListOfBlogPost.list:type_name -> productv1.ListOfBlogPost.List + 241, // 3: productv1.ListOfBlogPostFilter.list:type_name -> productv1.ListOfBlogPostFilter.List + 242, // 4: productv1.ListOfBlogPostInput.list:type_name -> productv1.ListOfBlogPostInput.List + 243, // 5: productv1.ListOfBoolean.list:type_name -> productv1.ListOfBoolean.List + 244, // 6: productv1.ListOfCategory.list:type_name -> productv1.ListOfCategory.List + 245, // 7: productv1.ListOfCategoryInput.list:type_name -> productv1.ListOfCategoryInput.List + 246, // 8: productv1.ListOfFloat.list:type_name -> productv1.ListOfFloat.List + 247, // 9: productv1.ListOfListOfCategory.list:type_name -> productv1.ListOfListOfCategory.List + 248, // 10: productv1.ListOfListOfCategoryInput.list:type_name -> productv1.ListOfListOfCategoryInput.List + 249, // 11: productv1.ListOfListOfString.list:type_name -> productv1.ListOfListOfString.List + 250, // 12: productv1.ListOfListOfUser.list:type_name -> productv1.ListOfListOfUser.List + 251, // 13: productv1.ListOfListOfUserInput.list:type_name -> productv1.ListOfListOfUserInput.List + 252, // 14: productv1.ListOfOrderLine.list:type_name -> productv1.ListOfOrderLine.List + 253, // 15: productv1.ListOfProduct.list:type_name -> productv1.ListOfProduct.List + 254, // 16: productv1.ListOfString.list:type_name -> productv1.ListOfString.List + 255, // 17: productv1.ListOfSubcategory.list:type_name -> productv1.ListOfSubcategory.List + 256, // 18: productv1.ListOfUser.list:type_name -> productv1.ListOfUser.List + 257, // 19: productv1.ListOfUserInput.list:type_name -> productv1.ListOfUserInput.List 22, // 20: productv1.LookupProductByIdRequest.keys:type_name -> productv1.LookupProductByIdRequestKey 184, // 21: productv1.LookupProductByIdResponse.result:type_name -> productv1.Product 25, // 22: productv1.LookupStorageByIdRequest.keys:type_name -> productv1.LookupStorageByIdRequestKey @@ -13955,7 +14408,7 @@ var file_product_proto_depIdxs = []int32{ 203, // 91: productv1.MutationBulkCreateBlogPostsResponse.bulk_create_blog_posts:type_name -> productv1.BlogPost 6, // 92: productv1.MutationBulkUpdateBlogPostsRequest.blog_posts:type_name -> productv1.ListOfBlogPostInput 203, // 93: productv1.MutationBulkUpdateBlogPostsResponse.bulk_update_blog_posts:type_name -> productv1.BlogPost - 231, // 94: productv1.ResolveProductShippingEstimateArgs.input:type_name -> productv1.ShippingEstimateInput + 237, // 94: productv1.ResolveProductShippingEstimateArgs.input:type_name -> productv1.ShippingEstimateInput 120, // 95: productv1.ResolveProductShippingEstimateRequest.context:type_name -> productv1.ResolveProductShippingEstimateContext 119, // 96: productv1.ResolveProductShippingEstimateRequest.field_args:type_name -> productv1.ResolveProductShippingEstimateArgs 122, // 97: productv1.ResolveProductShippingEstimateResponse.result:type_name -> productv1.ResolveProductShippingEstimateResult @@ -13975,14 +14428,14 @@ var file_product_proto_depIdxs = []int32{ 139, // 111: productv1.ResolveProductProductDetailsRequest.field_args:type_name -> productv1.ResolveProductProductDetailsArgs 214, // 112: productv1.ResolveProductProductDetailsResult.product_details:type_name -> productv1.ProductDetails 142, // 113: productv1.ResolveProductProductDetailsResponse.result:type_name -> productv1.ResolveProductProductDetailsResult - 229, // 114: productv1.ResolveCategoryProductCountArgs.filters:type_name -> productv1.ProductCountFilter + 235, // 114: productv1.ResolveCategoryProductCountArgs.filters:type_name -> productv1.ProductCountFilter 145, // 115: productv1.ResolveCategoryProductCountRequest.context:type_name -> productv1.ResolveCategoryProductCountContext 144, // 116: productv1.ResolveCategoryProductCountRequest.field_args:type_name -> productv1.ResolveCategoryProductCountArgs 147, // 117: productv1.ResolveCategoryProductCountResponse.result:type_name -> productv1.ResolveCategoryProductCountResult - 252, // 118: productv1.ResolveCategoryPopularityScoreArgs.threshold:type_name -> google.protobuf.Int32Value + 258, // 118: productv1.ResolveCategoryPopularityScoreArgs.threshold:type_name -> google.protobuf.Int32Value 150, // 119: productv1.ResolveCategoryPopularityScoreRequest.context:type_name -> productv1.ResolveCategoryPopularityScoreContext 149, // 120: productv1.ResolveCategoryPopularityScoreRequest.field_args:type_name -> productv1.ResolveCategoryPopularityScoreArgs - 252, // 121: productv1.ResolveCategoryPopularityScoreResult.popularity_score:type_name -> google.protobuf.Int32Value + 258, // 121: productv1.ResolveCategoryPopularityScoreResult.popularity_score:type_name -> google.protobuf.Int32Value 152, // 122: productv1.ResolveCategoryPopularityScoreResponse.result:type_name -> productv1.ResolveCategoryPopularityScoreResult 155, // 123: productv1.ResolveCategoryCategoryMetricsRequest.context:type_name -> productv1.ResolveCategoryCategoryMetricsContext 154, // 124: productv1.ResolveCategoryCategoryMetricsRequest.field_args:type_name -> productv1.ResolveCategoryCategoryMetricsArgs @@ -13997,7 +14450,7 @@ var file_product_proto_depIdxs = []int32{ 164, // 133: productv1.ResolveCategoryCategoryStatusRequest.field_args:type_name -> productv1.ResolveCategoryCategoryStatusArgs 210, // 134: productv1.ResolveCategoryCategoryStatusResult.category_status:type_name -> productv1.ActionResult 167, // 135: productv1.ResolveCategoryCategoryStatusResponse.result:type_name -> productv1.ResolveCategoryCategoryStatusResult - 230, // 136: productv1.ResolveSubcategoryItemCountArgs.filters:type_name -> productv1.SubcategoryItemFilter + 236, // 136: productv1.ResolveSubcategoryItemCountArgs.filters:type_name -> productv1.SubcategoryItemFilter 170, // 137: productv1.ResolveSubcategoryItemCountRequest.context:type_name -> productv1.ResolveSubcategoryItemCountContext 169, // 138: productv1.ResolveSubcategoryItemCountRequest.field_args:type_name -> productv1.ResolveSubcategoryItemCountArgs 172, // 139: productv1.ResolveSubcategoryItemCountResponse.result:type_name -> productv1.ResolveSubcategoryItemCountResult @@ -14006,7 +14459,7 @@ var file_product_proto_depIdxs = []int32{ 177, // 142: productv1.ResolveCategoryMetricsNormalizedScoreResponse.result:type_name -> productv1.ResolveCategoryMetricsNormalizedScoreResult 180, // 143: productv1.ResolveTestContainerDetailsRequest.context:type_name -> productv1.ResolveTestContainerDetailsContext 179, // 144: productv1.ResolveTestContainerDetailsRequest.field_args:type_name -> productv1.ResolveTestContainerDetailsArgs - 227, // 145: productv1.ResolveTestContainerDetailsResult.details:type_name -> productv1.TestDetails + 233, // 145: productv1.ResolveTestContainerDetailsResult.details:type_name -> productv1.TestDetails 182, // 146: productv1.ResolveTestContainerDetailsResponse.result:type_name -> productv1.ResolveTestContainerDetailsResult 215, // 147: productv1.NestedTypeA.b:type_name -> productv1.NestedTypeB 189, // 148: productv1.RecursiveType.recursive_type:type_name -> productv1.RecursiveType @@ -14019,17 +14472,17 @@ var file_product_proto_depIdxs = []int32{ 218, // 155: productv1.CategoryFilter.pagination:type_name -> productv1.Pagination 223, // 156: productv1.Animal.cat:type_name -> productv1.Cat 224, // 157: productv1.Animal.dog:type_name -> productv1.Dog - 252, // 158: productv1.SearchInput.limit:type_name -> google.protobuf.Int32Value + 258, // 158: productv1.SearchInput.limit:type_name -> google.protobuf.Int32Value 184, // 159: productv1.SearchResult.product:type_name -> productv1.Product 187, // 160: productv1.SearchResult.user:type_name -> productv1.User 196, // 161: productv1.SearchResult.category:type_name -> productv1.Category - 253, // 162: productv1.NullableFieldsType.optional_string:type_name -> google.protobuf.StringValue - 252, // 163: productv1.NullableFieldsType.optional_int:type_name -> google.protobuf.Int32Value - 254, // 164: productv1.NullableFieldsType.optional_float:type_name -> google.protobuf.DoubleValue - 255, // 165: productv1.NullableFieldsType.optional_boolean:type_name -> google.protobuf.BoolValue - 253, // 166: productv1.NullableFieldsFilter.name:type_name -> google.protobuf.StringValue - 253, // 167: productv1.NullableFieldsFilter.optional_string:type_name -> google.protobuf.StringValue - 255, // 168: productv1.NullableFieldsFilter.include_nulls:type_name -> google.protobuf.BoolValue + 259, // 162: productv1.NullableFieldsType.optional_string:type_name -> google.protobuf.StringValue + 258, // 163: productv1.NullableFieldsType.optional_int:type_name -> google.protobuf.Int32Value + 260, // 164: productv1.NullableFieldsType.optional_float:type_name -> google.protobuf.DoubleValue + 261, // 165: productv1.NullableFieldsType.optional_boolean:type_name -> google.protobuf.BoolValue + 259, // 166: productv1.NullableFieldsFilter.name:type_name -> google.protobuf.StringValue + 259, // 167: productv1.NullableFieldsFilter.optional_string:type_name -> google.protobuf.StringValue + 261, // 168: productv1.NullableFieldsFilter.include_nulls:type_name -> google.protobuf.BoolValue 18, // 169: productv1.BlogPost.optional_tags:type_name -> productv1.ListOfString 18, // 170: productv1.BlogPost.keywords:type_name -> productv1.ListOfString 10, // 171: productv1.BlogPost.ratings:type_name -> productv1.ListOfFloat @@ -14044,10 +14497,10 @@ var file_product_proto_depIdxs = []int32{ 20, // 180: productv1.BlogPost.mentioned_users:type_name -> productv1.ListOfUser 11, // 181: productv1.BlogPost.category_groups:type_name -> productv1.ListOfListOfCategory 14, // 182: productv1.BlogPost.contributor_teams:type_name -> productv1.ListOfListOfUser - 253, // 183: productv1.BlogPostFilter.title:type_name -> google.protobuf.StringValue - 255, // 184: productv1.BlogPostFilter.has_categories:type_name -> google.protobuf.BoolValue - 252, // 185: productv1.BlogPostFilter.min_tags:type_name -> google.protobuf.Int32Value - 253, // 186: productv1.Author.email:type_name -> google.protobuf.StringValue + 259, // 183: productv1.BlogPostFilter.title:type_name -> google.protobuf.StringValue + 261, // 184: productv1.BlogPostFilter.has_categories:type_name -> google.protobuf.BoolValue + 258, // 185: productv1.BlogPostFilter.min_tags:type_name -> google.protobuf.Int32Value + 259, // 186: productv1.Author.email:type_name -> google.protobuf.StringValue 18, // 187: productv1.Author.social_links:type_name -> productv1.ListOfString 13, // 188: productv1.Author.teams_by_project:type_name -> productv1.ListOfListOfString 13, // 189: productv1.Author.collaborations:type_name -> productv1.ListOfListOfString @@ -14058,16 +14511,16 @@ var file_product_proto_depIdxs = []int32{ 14, // 194: productv1.Author.author_groups:type_name -> productv1.ListOfListOfUser 11, // 195: productv1.Author.category_preferences:type_name -> productv1.ListOfListOfCategory 14, // 196: productv1.Author.project_teams:type_name -> productv1.ListOfListOfUser - 253, // 197: productv1.AuthorFilter.name:type_name -> google.protobuf.StringValue - 255, // 198: productv1.AuthorFilter.has_teams:type_name -> google.protobuf.BoolValue - 252, // 199: productv1.AuthorFilter.skill_count:type_name -> google.protobuf.Int32Value - 253, // 200: productv1.TestContainer.description:type_name -> google.protobuf.StringValue - 225, // 201: productv1.ActionResult.action_success:type_name -> productv1.ActionSuccess - 226, // 202: productv1.ActionResult.action_error:type_name -> productv1.ActionError - 253, // 203: productv1.NullableFieldsInput.optional_string:type_name -> google.protobuf.StringValue - 252, // 204: productv1.NullableFieldsInput.optional_int:type_name -> google.protobuf.Int32Value - 254, // 205: productv1.NullableFieldsInput.optional_float:type_name -> google.protobuf.DoubleValue - 255, // 206: productv1.NullableFieldsInput.optional_boolean:type_name -> google.protobuf.BoolValue + 259, // 197: productv1.AuthorFilter.name:type_name -> google.protobuf.StringValue + 261, // 198: productv1.AuthorFilter.has_teams:type_name -> google.protobuf.BoolValue + 258, // 199: productv1.AuthorFilter.skill_count:type_name -> google.protobuf.Int32Value + 259, // 200: productv1.TestContainer.description:type_name -> google.protobuf.StringValue + 231, // 201: productv1.ActionResult.action_success:type_name -> productv1.ActionSuccess + 232, // 202: productv1.ActionResult.action_error:type_name -> productv1.ActionError + 259, // 203: productv1.NullableFieldsInput.optional_string:type_name -> google.protobuf.StringValue + 258, // 204: productv1.NullableFieldsInput.optional_int:type_name -> google.protobuf.Int32Value + 260, // 205: productv1.NullableFieldsInput.optional_float:type_name -> google.protobuf.DoubleValue + 261, // 206: productv1.NullableFieldsInput.optional_boolean:type_name -> google.protobuf.BoolValue 18, // 207: productv1.BlogPostInput.optional_tags:type_name -> productv1.ListOfString 18, // 208: productv1.BlogPostInput.keywords:type_name -> productv1.ListOfString 10, // 209: productv1.BlogPostInput.ratings:type_name -> productv1.ListOfFloat @@ -14079,11 +14532,11 @@ var file_product_proto_depIdxs = []int32{ 9, // 215: productv1.BlogPostInput.related_categories:type_name -> productv1.ListOfCategoryInput 21, // 216: productv1.BlogPostInput.contributors:type_name -> productv1.ListOfUserInput 12, // 217: productv1.BlogPostInput.category_groups:type_name -> productv1.ListOfListOfCategoryInput - 253, // 218: productv1.AuthorInput.email:type_name -> google.protobuf.StringValue + 259, // 218: productv1.AuthorInput.email:type_name -> google.protobuf.StringValue 18, // 219: productv1.AuthorInput.social_links:type_name -> productv1.ListOfString 13, // 220: productv1.AuthorInput.teams_by_project:type_name -> productv1.ListOfListOfString 13, // 221: productv1.AuthorInput.collaborations:type_name -> productv1.ListOfListOfString - 228, // 222: productv1.AuthorInput.favorite_categories:type_name -> productv1.CategoryInput + 234, // 222: productv1.AuthorInput.favorite_categories:type_name -> productv1.CategoryInput 15, // 223: productv1.AuthorInput.author_groups:type_name -> productv1.ListOfListOfUserInput 15, // 224: productv1.AuthorInput.project_teams:type_name -> productv1.ListOfListOfUserInput 210, // 225: productv1.ProductDetails.review_summary:type_name -> productv1.ActionResult @@ -14092,164 +14545,172 @@ var file_product_proto_depIdxs = []int32{ 218, // 228: productv1.FilterType.pagination:type_name -> productv1.Pagination 18, // 229: productv1.OrderLineInput.modifiers:type_name -> productv1.ListOfString 18, // 230: productv1.OrderLine.modifiers:type_name -> productv1.ListOfString - 253, // 231: productv1.Subcategory.description:type_name -> google.protobuf.StringValue + 259, // 231: productv1.Subcategory.description:type_name -> google.protobuf.StringValue 196, // 232: productv1.CategoryMetrics.related_category:type_name -> productv1.Category - 198, // 233: productv1.TestDetails.pet:type_name -> productv1.Animal - 210, // 234: productv1.TestDetails.status:type_name -> productv1.ActionResult - 0, // 235: productv1.CategoryInput.kind:type_name -> productv1.CategoryKind - 254, // 236: productv1.ProductCountFilter.min_price:type_name -> google.protobuf.DoubleValue - 254, // 237: productv1.ProductCountFilter.max_price:type_name -> google.protobuf.DoubleValue - 255, // 238: productv1.ProductCountFilter.in_stock:type_name -> google.protobuf.BoolValue - 253, // 239: productv1.ProductCountFilter.search_term:type_name -> google.protobuf.StringValue - 254, // 240: productv1.SubcategoryItemFilter.min_price:type_name -> google.protobuf.DoubleValue - 254, // 241: productv1.SubcategoryItemFilter.max_price:type_name -> google.protobuf.DoubleValue - 255, // 242: productv1.SubcategoryItemFilter.in_stock:type_name -> google.protobuf.BoolValue - 255, // 243: productv1.SubcategoryItemFilter.is_active:type_name -> google.protobuf.BoolValue - 253, // 244: productv1.SubcategoryItemFilter.search_term:type_name -> google.protobuf.StringValue - 1, // 245: productv1.ShippingEstimateInput.destination:type_name -> productv1.ShippingDestination - 255, // 246: productv1.ShippingEstimateInput.expedited:type_name -> google.protobuf.BoolValue - 206, // 247: productv1.ListOfAuthorFilter.List.items:type_name -> productv1.AuthorFilter - 213, // 248: productv1.ListOfAuthorInput.List.items:type_name -> productv1.AuthorInput - 203, // 249: productv1.ListOfBlogPost.List.items:type_name -> productv1.BlogPost - 204, // 250: productv1.ListOfBlogPostFilter.List.items:type_name -> productv1.BlogPostFilter - 212, // 251: productv1.ListOfBlogPostInput.List.items:type_name -> productv1.BlogPostInput - 196, // 252: productv1.ListOfCategory.List.items:type_name -> productv1.Category - 228, // 253: productv1.ListOfCategoryInput.List.items:type_name -> productv1.CategoryInput - 8, // 254: productv1.ListOfListOfCategory.List.items:type_name -> productv1.ListOfCategory - 9, // 255: productv1.ListOfListOfCategoryInput.List.items:type_name -> productv1.ListOfCategoryInput - 18, // 256: productv1.ListOfListOfString.List.items:type_name -> productv1.ListOfString - 20, // 257: productv1.ListOfListOfUser.List.items:type_name -> productv1.ListOfUser - 21, // 258: productv1.ListOfListOfUserInput.List.items:type_name -> productv1.ListOfUserInput - 220, // 259: productv1.ListOfOrderLine.List.items:type_name -> productv1.OrderLine - 184, // 260: productv1.ListOfProduct.List.items:type_name -> productv1.Product - 221, // 261: productv1.ListOfSubcategory.List.items:type_name -> productv1.Subcategory - 187, // 262: productv1.ListOfUser.List.items:type_name -> productv1.User - 208, // 263: productv1.ListOfUserInput.List.items:type_name -> productv1.UserInput - 23, // 264: productv1.ProductService.LookupProductById:input_type -> productv1.LookupProductByIdRequest - 26, // 265: productv1.ProductService.LookupStorageById:input_type -> productv1.LookupStorageByIdRequest - 29, // 266: productv1.ProductService.LookupWarehouseById:input_type -> productv1.LookupWarehouseByIdRequest - 111, // 267: productv1.ProductService.MutationBulkCreateAuthors:input_type -> productv1.MutationBulkCreateAuthorsRequest - 115, // 268: productv1.ProductService.MutationBulkCreateBlogPosts:input_type -> productv1.MutationBulkCreateBlogPostsRequest - 113, // 269: productv1.ProductService.MutationBulkUpdateAuthors:input_type -> productv1.MutationBulkUpdateAuthorsRequest - 117, // 270: productv1.ProductService.MutationBulkUpdateBlogPosts:input_type -> productv1.MutationBulkUpdateBlogPostsRequest - 107, // 271: productv1.ProductService.MutationCreateAuthor:input_type -> productv1.MutationCreateAuthorRequest - 103, // 272: productv1.ProductService.MutationCreateBlogPost:input_type -> productv1.MutationCreateBlogPostRequest - 99, // 273: productv1.ProductService.MutationCreateNullableFieldsType:input_type -> productv1.MutationCreateNullableFieldsTypeRequest - 95, // 274: productv1.ProductService.MutationCreateUser:input_type -> productv1.MutationCreateUserRequest - 97, // 275: productv1.ProductService.MutationPerformAction:input_type -> productv1.MutationPerformActionRequest - 109, // 276: productv1.ProductService.MutationUpdateAuthor:input_type -> productv1.MutationUpdateAuthorRequest - 105, // 277: productv1.ProductService.MutationUpdateBlogPost:input_type -> productv1.MutationUpdateBlogPostRequest - 101, // 278: productv1.ProductService.MutationUpdateNullableFieldsType:input_type -> productv1.MutationUpdateNullableFieldsTypeRequest - 85, // 279: productv1.ProductService.QueryAllAuthors:input_type -> productv1.QueryAllAuthorsRequest - 77, // 280: productv1.ProductService.QueryAllBlogPosts:input_type -> productv1.QueryAllBlogPostsRequest - 69, // 281: productv1.ProductService.QueryAllNullableFieldsTypes:input_type -> productv1.QueryAllNullableFieldsTypesRequest - 57, // 282: productv1.ProductService.QueryAllPets:input_type -> productv1.QueryAllPetsRequest - 79, // 283: productv1.ProductService.QueryAuthor:input_type -> productv1.QueryAuthorRequest - 81, // 284: productv1.ProductService.QueryAuthorById:input_type -> productv1.QueryAuthorByIdRequest - 83, // 285: productv1.ProductService.QueryAuthorsWithFilter:input_type -> productv1.QueryAuthorsWithFilterRequest - 71, // 286: productv1.ProductService.QueryBlogPost:input_type -> productv1.QueryBlogPostRequest - 73, // 287: productv1.ProductService.QueryBlogPostById:input_type -> productv1.QueryBlogPostByIdRequest - 75, // 288: productv1.ProductService.QueryBlogPostsWithFilter:input_type -> productv1.QueryBlogPostsWithFilterRequest - 87, // 289: productv1.ProductService.QueryBulkSearchAuthors:input_type -> productv1.QueryBulkSearchAuthorsRequest - 89, // 290: productv1.ProductService.QueryBulkSearchBlogPosts:input_type -> productv1.QueryBulkSearchBlogPostsRequest - 45, // 291: productv1.ProductService.QueryCalculateTotals:input_type -> productv1.QueryCalculateTotalsRequest - 47, // 292: productv1.ProductService.QueryCategories:input_type -> productv1.QueryCategoriesRequest - 49, // 293: productv1.ProductService.QueryCategoriesByKind:input_type -> productv1.QueryCategoriesByKindRequest - 51, // 294: productv1.ProductService.QueryCategoriesByKinds:input_type -> productv1.QueryCategoriesByKindsRequest - 43, // 295: productv1.ProductService.QueryComplexFilterType:input_type -> productv1.QueryComplexFilterTypeRequest - 53, // 296: productv1.ProductService.QueryFilterCategories:input_type -> productv1.QueryFilterCategoriesRequest - 35, // 297: productv1.ProductService.QueryNestedType:input_type -> productv1.QueryNestedTypeRequest - 63, // 298: productv1.ProductService.QueryNullableFieldsType:input_type -> productv1.QueryNullableFieldsTypeRequest - 65, // 299: productv1.ProductService.QueryNullableFieldsTypeById:input_type -> productv1.QueryNullableFieldsTypeByIdRequest - 67, // 300: productv1.ProductService.QueryNullableFieldsTypeWithFilter:input_type -> productv1.QueryNullableFieldsTypeWithFilterRequest - 55, // 301: productv1.ProductService.QueryRandomPet:input_type -> productv1.QueryRandomPetRequest - 61, // 302: productv1.ProductService.QueryRandomSearchResult:input_type -> productv1.QueryRandomSearchResultRequest - 37, // 303: productv1.ProductService.QueryRecursiveType:input_type -> productv1.QueryRecursiveTypeRequest - 59, // 304: productv1.ProductService.QuerySearch:input_type -> productv1.QuerySearchRequest - 91, // 305: productv1.ProductService.QueryTestContainer:input_type -> productv1.QueryTestContainerRequest - 93, // 306: productv1.ProductService.QueryTestContainers:input_type -> productv1.QueryTestContainersRequest - 39, // 307: productv1.ProductService.QueryTypeFilterWithArguments:input_type -> productv1.QueryTypeFilterWithArgumentsRequest - 41, // 308: productv1.ProductService.QueryTypeWithMultipleFilterFields:input_type -> productv1.QueryTypeWithMultipleFilterFieldsRequest - 33, // 309: productv1.ProductService.QueryUser:input_type -> productv1.QueryUserRequest - 31, // 310: productv1.ProductService.QueryUsers:input_type -> productv1.QueryUsersRequest - 156, // 311: productv1.ProductService.ResolveCategoryCategoryMetrics:input_type -> productv1.ResolveCategoryCategoryMetricsRequest - 166, // 312: productv1.ProductService.ResolveCategoryCategoryStatus:input_type -> productv1.ResolveCategoryCategoryStatusRequest - 161, // 313: productv1.ProductService.ResolveCategoryMascot:input_type -> productv1.ResolveCategoryMascotRequest - 176, // 314: productv1.ProductService.ResolveCategoryMetricsNormalizedScore:input_type -> productv1.ResolveCategoryMetricsNormalizedScoreRequest - 151, // 315: productv1.ProductService.ResolveCategoryPopularityScore:input_type -> productv1.ResolveCategoryPopularityScoreRequest - 146, // 316: productv1.ProductService.ResolveCategoryProductCount:input_type -> productv1.ResolveCategoryProductCountRequest - 131, // 317: productv1.ProductService.ResolveProductMascotRecommendation:input_type -> productv1.ResolveProductMascotRecommendationRequest - 141, // 318: productv1.ProductService.ResolveProductProductDetails:input_type -> productv1.ResolveProductProductDetailsRequest - 126, // 319: productv1.ProductService.ResolveProductRecommendedCategory:input_type -> productv1.ResolveProductRecommendedCategoryRequest - 121, // 320: productv1.ProductService.ResolveProductShippingEstimate:input_type -> productv1.ResolveProductShippingEstimateRequest - 136, // 321: productv1.ProductService.ResolveProductStockStatus:input_type -> productv1.ResolveProductStockStatusRequest - 171, // 322: productv1.ProductService.ResolveSubcategoryItemCount:input_type -> productv1.ResolveSubcategoryItemCountRequest - 181, // 323: productv1.ProductService.ResolveTestContainerDetails:input_type -> productv1.ResolveTestContainerDetailsRequest - 24, // 324: productv1.ProductService.LookupProductById:output_type -> productv1.LookupProductByIdResponse - 27, // 325: productv1.ProductService.LookupStorageById:output_type -> productv1.LookupStorageByIdResponse - 30, // 326: productv1.ProductService.LookupWarehouseById:output_type -> productv1.LookupWarehouseByIdResponse - 112, // 327: productv1.ProductService.MutationBulkCreateAuthors:output_type -> productv1.MutationBulkCreateAuthorsResponse - 116, // 328: productv1.ProductService.MutationBulkCreateBlogPosts:output_type -> productv1.MutationBulkCreateBlogPostsResponse - 114, // 329: productv1.ProductService.MutationBulkUpdateAuthors:output_type -> productv1.MutationBulkUpdateAuthorsResponse - 118, // 330: productv1.ProductService.MutationBulkUpdateBlogPosts:output_type -> productv1.MutationBulkUpdateBlogPostsResponse - 108, // 331: productv1.ProductService.MutationCreateAuthor:output_type -> productv1.MutationCreateAuthorResponse - 104, // 332: productv1.ProductService.MutationCreateBlogPost:output_type -> productv1.MutationCreateBlogPostResponse - 100, // 333: productv1.ProductService.MutationCreateNullableFieldsType:output_type -> productv1.MutationCreateNullableFieldsTypeResponse - 96, // 334: productv1.ProductService.MutationCreateUser:output_type -> productv1.MutationCreateUserResponse - 98, // 335: productv1.ProductService.MutationPerformAction:output_type -> productv1.MutationPerformActionResponse - 110, // 336: productv1.ProductService.MutationUpdateAuthor:output_type -> productv1.MutationUpdateAuthorResponse - 106, // 337: productv1.ProductService.MutationUpdateBlogPost:output_type -> productv1.MutationUpdateBlogPostResponse - 102, // 338: productv1.ProductService.MutationUpdateNullableFieldsType:output_type -> productv1.MutationUpdateNullableFieldsTypeResponse - 86, // 339: productv1.ProductService.QueryAllAuthors:output_type -> productv1.QueryAllAuthorsResponse - 78, // 340: productv1.ProductService.QueryAllBlogPosts:output_type -> productv1.QueryAllBlogPostsResponse - 70, // 341: productv1.ProductService.QueryAllNullableFieldsTypes:output_type -> productv1.QueryAllNullableFieldsTypesResponse - 58, // 342: productv1.ProductService.QueryAllPets:output_type -> productv1.QueryAllPetsResponse - 80, // 343: productv1.ProductService.QueryAuthor:output_type -> productv1.QueryAuthorResponse - 82, // 344: productv1.ProductService.QueryAuthorById:output_type -> productv1.QueryAuthorByIdResponse - 84, // 345: productv1.ProductService.QueryAuthorsWithFilter:output_type -> productv1.QueryAuthorsWithFilterResponse - 72, // 346: productv1.ProductService.QueryBlogPost:output_type -> productv1.QueryBlogPostResponse - 74, // 347: productv1.ProductService.QueryBlogPostById:output_type -> productv1.QueryBlogPostByIdResponse - 76, // 348: productv1.ProductService.QueryBlogPostsWithFilter:output_type -> productv1.QueryBlogPostsWithFilterResponse - 88, // 349: productv1.ProductService.QueryBulkSearchAuthors:output_type -> productv1.QueryBulkSearchAuthorsResponse - 90, // 350: productv1.ProductService.QueryBulkSearchBlogPosts:output_type -> productv1.QueryBulkSearchBlogPostsResponse - 46, // 351: productv1.ProductService.QueryCalculateTotals:output_type -> productv1.QueryCalculateTotalsResponse - 48, // 352: productv1.ProductService.QueryCategories:output_type -> productv1.QueryCategoriesResponse - 50, // 353: productv1.ProductService.QueryCategoriesByKind:output_type -> productv1.QueryCategoriesByKindResponse - 52, // 354: productv1.ProductService.QueryCategoriesByKinds:output_type -> productv1.QueryCategoriesByKindsResponse - 44, // 355: productv1.ProductService.QueryComplexFilterType:output_type -> productv1.QueryComplexFilterTypeResponse - 54, // 356: productv1.ProductService.QueryFilterCategories:output_type -> productv1.QueryFilterCategoriesResponse - 36, // 357: productv1.ProductService.QueryNestedType:output_type -> productv1.QueryNestedTypeResponse - 64, // 358: productv1.ProductService.QueryNullableFieldsType:output_type -> productv1.QueryNullableFieldsTypeResponse - 66, // 359: productv1.ProductService.QueryNullableFieldsTypeById:output_type -> productv1.QueryNullableFieldsTypeByIdResponse - 68, // 360: productv1.ProductService.QueryNullableFieldsTypeWithFilter:output_type -> productv1.QueryNullableFieldsTypeWithFilterResponse - 56, // 361: productv1.ProductService.QueryRandomPet:output_type -> productv1.QueryRandomPetResponse - 62, // 362: productv1.ProductService.QueryRandomSearchResult:output_type -> productv1.QueryRandomSearchResultResponse - 38, // 363: productv1.ProductService.QueryRecursiveType:output_type -> productv1.QueryRecursiveTypeResponse - 60, // 364: productv1.ProductService.QuerySearch:output_type -> productv1.QuerySearchResponse - 92, // 365: productv1.ProductService.QueryTestContainer:output_type -> productv1.QueryTestContainerResponse - 94, // 366: productv1.ProductService.QueryTestContainers:output_type -> productv1.QueryTestContainersResponse - 40, // 367: productv1.ProductService.QueryTypeFilterWithArguments:output_type -> productv1.QueryTypeFilterWithArgumentsResponse - 42, // 368: productv1.ProductService.QueryTypeWithMultipleFilterFields:output_type -> productv1.QueryTypeWithMultipleFilterFieldsResponse - 34, // 369: productv1.ProductService.QueryUser:output_type -> productv1.QueryUserResponse - 32, // 370: productv1.ProductService.QueryUsers:output_type -> productv1.QueryUsersResponse - 158, // 371: productv1.ProductService.ResolveCategoryCategoryMetrics:output_type -> productv1.ResolveCategoryCategoryMetricsResponse - 168, // 372: productv1.ProductService.ResolveCategoryCategoryStatus:output_type -> productv1.ResolveCategoryCategoryStatusResponse - 163, // 373: productv1.ProductService.ResolveCategoryMascot:output_type -> productv1.ResolveCategoryMascotResponse - 178, // 374: productv1.ProductService.ResolveCategoryMetricsNormalizedScore:output_type -> productv1.ResolveCategoryMetricsNormalizedScoreResponse - 153, // 375: productv1.ProductService.ResolveCategoryPopularityScore:output_type -> productv1.ResolveCategoryPopularityScoreResponse - 148, // 376: productv1.ProductService.ResolveCategoryProductCount:output_type -> productv1.ResolveCategoryProductCountResponse - 133, // 377: productv1.ProductService.ResolveProductMascotRecommendation:output_type -> productv1.ResolveProductMascotRecommendationResponse - 143, // 378: productv1.ProductService.ResolveProductProductDetails:output_type -> productv1.ResolveProductProductDetailsResponse - 128, // 379: productv1.ProductService.ResolveProductRecommendedCategory:output_type -> productv1.ResolveProductRecommendedCategoryResponse - 123, // 380: productv1.ProductService.ResolveProductShippingEstimate:output_type -> productv1.ResolveProductShippingEstimateResponse - 138, // 381: productv1.ProductService.ResolveProductStockStatus:output_type -> productv1.ResolveProductStockStatusResponse - 173, // 382: productv1.ProductService.ResolveSubcategoryItemCount:output_type -> productv1.ResolveSubcategoryItemCountResponse - 183, // 383: productv1.ProductService.ResolveTestContainerDetails:output_type -> productv1.ResolveTestContainerDetailsResponse - 324, // [324:384] is the sub-list for method output_type - 264, // [264:324] is the sub-list for method input_type - 264, // [264:264] is the sub-list for extension type_name - 264, // [264:264] is the sub-list for extension extendee - 0, // [0:264] is the sub-list for field type_name + 225, // 233: productv1.Cat.owner:type_name -> productv1.Owner + 228, // 234: productv1.Cat.breed:type_name -> productv1.CatBreed + 225, // 235: productv1.Dog.owner:type_name -> productv1.Owner + 229, // 236: productv1.Dog.breed:type_name -> productv1.DogBreed + 226, // 237: productv1.Owner.contact:type_name -> productv1.ContactInfo + 227, // 238: productv1.ContactInfo.address:type_name -> productv1.Address + 230, // 239: productv1.CatBreed.characteristics:type_name -> productv1.BreedCharacteristics + 230, // 240: productv1.DogBreed.characteristics:type_name -> productv1.BreedCharacteristics + 198, // 241: productv1.TestDetails.pet:type_name -> productv1.Animal + 210, // 242: productv1.TestDetails.status:type_name -> productv1.ActionResult + 0, // 243: productv1.CategoryInput.kind:type_name -> productv1.CategoryKind + 260, // 244: productv1.ProductCountFilter.min_price:type_name -> google.protobuf.DoubleValue + 260, // 245: productv1.ProductCountFilter.max_price:type_name -> google.protobuf.DoubleValue + 261, // 246: productv1.ProductCountFilter.in_stock:type_name -> google.protobuf.BoolValue + 259, // 247: productv1.ProductCountFilter.search_term:type_name -> google.protobuf.StringValue + 260, // 248: productv1.SubcategoryItemFilter.min_price:type_name -> google.protobuf.DoubleValue + 260, // 249: productv1.SubcategoryItemFilter.max_price:type_name -> google.protobuf.DoubleValue + 261, // 250: productv1.SubcategoryItemFilter.in_stock:type_name -> google.protobuf.BoolValue + 261, // 251: productv1.SubcategoryItemFilter.is_active:type_name -> google.protobuf.BoolValue + 259, // 252: productv1.SubcategoryItemFilter.search_term:type_name -> google.protobuf.StringValue + 1, // 253: productv1.ShippingEstimateInput.destination:type_name -> productv1.ShippingDestination + 261, // 254: productv1.ShippingEstimateInput.expedited:type_name -> google.protobuf.BoolValue + 206, // 255: productv1.ListOfAuthorFilter.List.items:type_name -> productv1.AuthorFilter + 213, // 256: productv1.ListOfAuthorInput.List.items:type_name -> productv1.AuthorInput + 203, // 257: productv1.ListOfBlogPost.List.items:type_name -> productv1.BlogPost + 204, // 258: productv1.ListOfBlogPostFilter.List.items:type_name -> productv1.BlogPostFilter + 212, // 259: productv1.ListOfBlogPostInput.List.items:type_name -> productv1.BlogPostInput + 196, // 260: productv1.ListOfCategory.List.items:type_name -> productv1.Category + 234, // 261: productv1.ListOfCategoryInput.List.items:type_name -> productv1.CategoryInput + 8, // 262: productv1.ListOfListOfCategory.List.items:type_name -> productv1.ListOfCategory + 9, // 263: productv1.ListOfListOfCategoryInput.List.items:type_name -> productv1.ListOfCategoryInput + 18, // 264: productv1.ListOfListOfString.List.items:type_name -> productv1.ListOfString + 20, // 265: productv1.ListOfListOfUser.List.items:type_name -> productv1.ListOfUser + 21, // 266: productv1.ListOfListOfUserInput.List.items:type_name -> productv1.ListOfUserInput + 220, // 267: productv1.ListOfOrderLine.List.items:type_name -> productv1.OrderLine + 184, // 268: productv1.ListOfProduct.List.items:type_name -> productv1.Product + 221, // 269: productv1.ListOfSubcategory.List.items:type_name -> productv1.Subcategory + 187, // 270: productv1.ListOfUser.List.items:type_name -> productv1.User + 208, // 271: productv1.ListOfUserInput.List.items:type_name -> productv1.UserInput + 23, // 272: productv1.ProductService.LookupProductById:input_type -> productv1.LookupProductByIdRequest + 26, // 273: productv1.ProductService.LookupStorageById:input_type -> productv1.LookupStorageByIdRequest + 29, // 274: productv1.ProductService.LookupWarehouseById:input_type -> productv1.LookupWarehouseByIdRequest + 111, // 275: productv1.ProductService.MutationBulkCreateAuthors:input_type -> productv1.MutationBulkCreateAuthorsRequest + 115, // 276: productv1.ProductService.MutationBulkCreateBlogPosts:input_type -> productv1.MutationBulkCreateBlogPostsRequest + 113, // 277: productv1.ProductService.MutationBulkUpdateAuthors:input_type -> productv1.MutationBulkUpdateAuthorsRequest + 117, // 278: productv1.ProductService.MutationBulkUpdateBlogPosts:input_type -> productv1.MutationBulkUpdateBlogPostsRequest + 107, // 279: productv1.ProductService.MutationCreateAuthor:input_type -> productv1.MutationCreateAuthorRequest + 103, // 280: productv1.ProductService.MutationCreateBlogPost:input_type -> productv1.MutationCreateBlogPostRequest + 99, // 281: productv1.ProductService.MutationCreateNullableFieldsType:input_type -> productv1.MutationCreateNullableFieldsTypeRequest + 95, // 282: productv1.ProductService.MutationCreateUser:input_type -> productv1.MutationCreateUserRequest + 97, // 283: productv1.ProductService.MutationPerformAction:input_type -> productv1.MutationPerformActionRequest + 109, // 284: productv1.ProductService.MutationUpdateAuthor:input_type -> productv1.MutationUpdateAuthorRequest + 105, // 285: productv1.ProductService.MutationUpdateBlogPost:input_type -> productv1.MutationUpdateBlogPostRequest + 101, // 286: productv1.ProductService.MutationUpdateNullableFieldsType:input_type -> productv1.MutationUpdateNullableFieldsTypeRequest + 85, // 287: productv1.ProductService.QueryAllAuthors:input_type -> productv1.QueryAllAuthorsRequest + 77, // 288: productv1.ProductService.QueryAllBlogPosts:input_type -> productv1.QueryAllBlogPostsRequest + 69, // 289: productv1.ProductService.QueryAllNullableFieldsTypes:input_type -> productv1.QueryAllNullableFieldsTypesRequest + 57, // 290: productv1.ProductService.QueryAllPets:input_type -> productv1.QueryAllPetsRequest + 79, // 291: productv1.ProductService.QueryAuthor:input_type -> productv1.QueryAuthorRequest + 81, // 292: productv1.ProductService.QueryAuthorById:input_type -> productv1.QueryAuthorByIdRequest + 83, // 293: productv1.ProductService.QueryAuthorsWithFilter:input_type -> productv1.QueryAuthorsWithFilterRequest + 71, // 294: productv1.ProductService.QueryBlogPost:input_type -> productv1.QueryBlogPostRequest + 73, // 295: productv1.ProductService.QueryBlogPostById:input_type -> productv1.QueryBlogPostByIdRequest + 75, // 296: productv1.ProductService.QueryBlogPostsWithFilter:input_type -> productv1.QueryBlogPostsWithFilterRequest + 87, // 297: productv1.ProductService.QueryBulkSearchAuthors:input_type -> productv1.QueryBulkSearchAuthorsRequest + 89, // 298: productv1.ProductService.QueryBulkSearchBlogPosts:input_type -> productv1.QueryBulkSearchBlogPostsRequest + 45, // 299: productv1.ProductService.QueryCalculateTotals:input_type -> productv1.QueryCalculateTotalsRequest + 47, // 300: productv1.ProductService.QueryCategories:input_type -> productv1.QueryCategoriesRequest + 49, // 301: productv1.ProductService.QueryCategoriesByKind:input_type -> productv1.QueryCategoriesByKindRequest + 51, // 302: productv1.ProductService.QueryCategoriesByKinds:input_type -> productv1.QueryCategoriesByKindsRequest + 43, // 303: productv1.ProductService.QueryComplexFilterType:input_type -> productv1.QueryComplexFilterTypeRequest + 53, // 304: productv1.ProductService.QueryFilterCategories:input_type -> productv1.QueryFilterCategoriesRequest + 35, // 305: productv1.ProductService.QueryNestedType:input_type -> productv1.QueryNestedTypeRequest + 63, // 306: productv1.ProductService.QueryNullableFieldsType:input_type -> productv1.QueryNullableFieldsTypeRequest + 65, // 307: productv1.ProductService.QueryNullableFieldsTypeById:input_type -> productv1.QueryNullableFieldsTypeByIdRequest + 67, // 308: productv1.ProductService.QueryNullableFieldsTypeWithFilter:input_type -> productv1.QueryNullableFieldsTypeWithFilterRequest + 55, // 309: productv1.ProductService.QueryRandomPet:input_type -> productv1.QueryRandomPetRequest + 61, // 310: productv1.ProductService.QueryRandomSearchResult:input_type -> productv1.QueryRandomSearchResultRequest + 37, // 311: productv1.ProductService.QueryRecursiveType:input_type -> productv1.QueryRecursiveTypeRequest + 59, // 312: productv1.ProductService.QuerySearch:input_type -> productv1.QuerySearchRequest + 91, // 313: productv1.ProductService.QueryTestContainer:input_type -> productv1.QueryTestContainerRequest + 93, // 314: productv1.ProductService.QueryTestContainers:input_type -> productv1.QueryTestContainersRequest + 39, // 315: productv1.ProductService.QueryTypeFilterWithArguments:input_type -> productv1.QueryTypeFilterWithArgumentsRequest + 41, // 316: productv1.ProductService.QueryTypeWithMultipleFilterFields:input_type -> productv1.QueryTypeWithMultipleFilterFieldsRequest + 33, // 317: productv1.ProductService.QueryUser:input_type -> productv1.QueryUserRequest + 31, // 318: productv1.ProductService.QueryUsers:input_type -> productv1.QueryUsersRequest + 156, // 319: productv1.ProductService.ResolveCategoryCategoryMetrics:input_type -> productv1.ResolveCategoryCategoryMetricsRequest + 166, // 320: productv1.ProductService.ResolveCategoryCategoryStatus:input_type -> productv1.ResolveCategoryCategoryStatusRequest + 161, // 321: productv1.ProductService.ResolveCategoryMascot:input_type -> productv1.ResolveCategoryMascotRequest + 176, // 322: productv1.ProductService.ResolveCategoryMetricsNormalizedScore:input_type -> productv1.ResolveCategoryMetricsNormalizedScoreRequest + 151, // 323: productv1.ProductService.ResolveCategoryPopularityScore:input_type -> productv1.ResolveCategoryPopularityScoreRequest + 146, // 324: productv1.ProductService.ResolveCategoryProductCount:input_type -> productv1.ResolveCategoryProductCountRequest + 131, // 325: productv1.ProductService.ResolveProductMascotRecommendation:input_type -> productv1.ResolveProductMascotRecommendationRequest + 141, // 326: productv1.ProductService.ResolveProductProductDetails:input_type -> productv1.ResolveProductProductDetailsRequest + 126, // 327: productv1.ProductService.ResolveProductRecommendedCategory:input_type -> productv1.ResolveProductRecommendedCategoryRequest + 121, // 328: productv1.ProductService.ResolveProductShippingEstimate:input_type -> productv1.ResolveProductShippingEstimateRequest + 136, // 329: productv1.ProductService.ResolveProductStockStatus:input_type -> productv1.ResolveProductStockStatusRequest + 171, // 330: productv1.ProductService.ResolveSubcategoryItemCount:input_type -> productv1.ResolveSubcategoryItemCountRequest + 181, // 331: productv1.ProductService.ResolveTestContainerDetails:input_type -> productv1.ResolveTestContainerDetailsRequest + 24, // 332: productv1.ProductService.LookupProductById:output_type -> productv1.LookupProductByIdResponse + 27, // 333: productv1.ProductService.LookupStorageById:output_type -> productv1.LookupStorageByIdResponse + 30, // 334: productv1.ProductService.LookupWarehouseById:output_type -> productv1.LookupWarehouseByIdResponse + 112, // 335: productv1.ProductService.MutationBulkCreateAuthors:output_type -> productv1.MutationBulkCreateAuthorsResponse + 116, // 336: productv1.ProductService.MutationBulkCreateBlogPosts:output_type -> productv1.MutationBulkCreateBlogPostsResponse + 114, // 337: productv1.ProductService.MutationBulkUpdateAuthors:output_type -> productv1.MutationBulkUpdateAuthorsResponse + 118, // 338: productv1.ProductService.MutationBulkUpdateBlogPosts:output_type -> productv1.MutationBulkUpdateBlogPostsResponse + 108, // 339: productv1.ProductService.MutationCreateAuthor:output_type -> productv1.MutationCreateAuthorResponse + 104, // 340: productv1.ProductService.MutationCreateBlogPost:output_type -> productv1.MutationCreateBlogPostResponse + 100, // 341: productv1.ProductService.MutationCreateNullableFieldsType:output_type -> productv1.MutationCreateNullableFieldsTypeResponse + 96, // 342: productv1.ProductService.MutationCreateUser:output_type -> productv1.MutationCreateUserResponse + 98, // 343: productv1.ProductService.MutationPerformAction:output_type -> productv1.MutationPerformActionResponse + 110, // 344: productv1.ProductService.MutationUpdateAuthor:output_type -> productv1.MutationUpdateAuthorResponse + 106, // 345: productv1.ProductService.MutationUpdateBlogPost:output_type -> productv1.MutationUpdateBlogPostResponse + 102, // 346: productv1.ProductService.MutationUpdateNullableFieldsType:output_type -> productv1.MutationUpdateNullableFieldsTypeResponse + 86, // 347: productv1.ProductService.QueryAllAuthors:output_type -> productv1.QueryAllAuthorsResponse + 78, // 348: productv1.ProductService.QueryAllBlogPosts:output_type -> productv1.QueryAllBlogPostsResponse + 70, // 349: productv1.ProductService.QueryAllNullableFieldsTypes:output_type -> productv1.QueryAllNullableFieldsTypesResponse + 58, // 350: productv1.ProductService.QueryAllPets:output_type -> productv1.QueryAllPetsResponse + 80, // 351: productv1.ProductService.QueryAuthor:output_type -> productv1.QueryAuthorResponse + 82, // 352: productv1.ProductService.QueryAuthorById:output_type -> productv1.QueryAuthorByIdResponse + 84, // 353: productv1.ProductService.QueryAuthorsWithFilter:output_type -> productv1.QueryAuthorsWithFilterResponse + 72, // 354: productv1.ProductService.QueryBlogPost:output_type -> productv1.QueryBlogPostResponse + 74, // 355: productv1.ProductService.QueryBlogPostById:output_type -> productv1.QueryBlogPostByIdResponse + 76, // 356: productv1.ProductService.QueryBlogPostsWithFilter:output_type -> productv1.QueryBlogPostsWithFilterResponse + 88, // 357: productv1.ProductService.QueryBulkSearchAuthors:output_type -> productv1.QueryBulkSearchAuthorsResponse + 90, // 358: productv1.ProductService.QueryBulkSearchBlogPosts:output_type -> productv1.QueryBulkSearchBlogPostsResponse + 46, // 359: productv1.ProductService.QueryCalculateTotals:output_type -> productv1.QueryCalculateTotalsResponse + 48, // 360: productv1.ProductService.QueryCategories:output_type -> productv1.QueryCategoriesResponse + 50, // 361: productv1.ProductService.QueryCategoriesByKind:output_type -> productv1.QueryCategoriesByKindResponse + 52, // 362: productv1.ProductService.QueryCategoriesByKinds:output_type -> productv1.QueryCategoriesByKindsResponse + 44, // 363: productv1.ProductService.QueryComplexFilterType:output_type -> productv1.QueryComplexFilterTypeResponse + 54, // 364: productv1.ProductService.QueryFilterCategories:output_type -> productv1.QueryFilterCategoriesResponse + 36, // 365: productv1.ProductService.QueryNestedType:output_type -> productv1.QueryNestedTypeResponse + 64, // 366: productv1.ProductService.QueryNullableFieldsType:output_type -> productv1.QueryNullableFieldsTypeResponse + 66, // 367: productv1.ProductService.QueryNullableFieldsTypeById:output_type -> productv1.QueryNullableFieldsTypeByIdResponse + 68, // 368: productv1.ProductService.QueryNullableFieldsTypeWithFilter:output_type -> productv1.QueryNullableFieldsTypeWithFilterResponse + 56, // 369: productv1.ProductService.QueryRandomPet:output_type -> productv1.QueryRandomPetResponse + 62, // 370: productv1.ProductService.QueryRandomSearchResult:output_type -> productv1.QueryRandomSearchResultResponse + 38, // 371: productv1.ProductService.QueryRecursiveType:output_type -> productv1.QueryRecursiveTypeResponse + 60, // 372: productv1.ProductService.QuerySearch:output_type -> productv1.QuerySearchResponse + 92, // 373: productv1.ProductService.QueryTestContainer:output_type -> productv1.QueryTestContainerResponse + 94, // 374: productv1.ProductService.QueryTestContainers:output_type -> productv1.QueryTestContainersResponse + 40, // 375: productv1.ProductService.QueryTypeFilterWithArguments:output_type -> productv1.QueryTypeFilterWithArgumentsResponse + 42, // 376: productv1.ProductService.QueryTypeWithMultipleFilterFields:output_type -> productv1.QueryTypeWithMultipleFilterFieldsResponse + 34, // 377: productv1.ProductService.QueryUser:output_type -> productv1.QueryUserResponse + 32, // 378: productv1.ProductService.QueryUsers:output_type -> productv1.QueryUsersResponse + 158, // 379: productv1.ProductService.ResolveCategoryCategoryMetrics:output_type -> productv1.ResolveCategoryCategoryMetricsResponse + 168, // 380: productv1.ProductService.ResolveCategoryCategoryStatus:output_type -> productv1.ResolveCategoryCategoryStatusResponse + 163, // 381: productv1.ProductService.ResolveCategoryMascot:output_type -> productv1.ResolveCategoryMascotResponse + 178, // 382: productv1.ProductService.ResolveCategoryMetricsNormalizedScore:output_type -> productv1.ResolveCategoryMetricsNormalizedScoreResponse + 153, // 383: productv1.ProductService.ResolveCategoryPopularityScore:output_type -> productv1.ResolveCategoryPopularityScoreResponse + 148, // 384: productv1.ProductService.ResolveCategoryProductCount:output_type -> productv1.ResolveCategoryProductCountResponse + 133, // 385: productv1.ProductService.ResolveProductMascotRecommendation:output_type -> productv1.ResolveProductMascotRecommendationResponse + 143, // 386: productv1.ProductService.ResolveProductProductDetails:output_type -> productv1.ResolveProductProductDetailsResponse + 128, // 387: productv1.ProductService.ResolveProductRecommendedCategory:output_type -> productv1.ResolveProductRecommendedCategoryResponse + 123, // 388: productv1.ProductService.ResolveProductShippingEstimate:output_type -> productv1.ResolveProductShippingEstimateResponse + 138, // 389: productv1.ProductService.ResolveProductStockStatus:output_type -> productv1.ResolveProductStockStatusResponse + 173, // 390: productv1.ProductService.ResolveSubcategoryItemCount:output_type -> productv1.ResolveSubcategoryItemCountResponse + 183, // 391: productv1.ProductService.ResolveTestContainerDetails:output_type -> productv1.ResolveTestContainerDetailsResponse + 332, // [332:392] is the sub-list for method output_type + 272, // [272:332] is the sub-list for method input_type + 272, // [272:272] is the sub-list for extension type_name + 272, // [272:272] is the sub-list for extension extendee + 0, // [0:272] is the sub-list for field type_name } func init() { file_product_proto_init() } @@ -14276,7 +14737,7 @@ func file_product_proto_init() { GoPackagePath: reflect.TypeOf(x{}).PkgPath(), RawDescriptor: unsafe.Slice(unsafe.StringData(file_product_proto_rawDesc), len(file_product_proto_rawDesc)), NumEnums: 2, - NumMessages: 250, + NumMessages: 256, NumExtensions: 0, NumServices: 1, }, diff --git a/v2/pkg/grpctest/testdata/products.graphqls b/v2/pkg/grpctest/testdata/products.graphqls index f054de862..47662bfed 100644 --- a/v2/pkg/grpctest/testdata/products.graphqls +++ b/v2/pkg/grpctest/testdata/products.graphqls @@ -169,6 +169,8 @@ type Cat implements Animal { name: String! kind: String! meowVolume: Int! + owner: Owner! + breed: CatBreed! } type Dog implements Animal { @@ -176,6 +178,47 @@ type Dog implements Animal { name: String! kind: String! barkVolume: Int! + owner: Owner! + breed: DogBreed! +} + +type Owner { + id: ID! + name: String! + contact: ContactInfo! +} + +type ContactInfo { + email: String! + phone: String! + address: Address +} + +type Address { + street: String! + city: String! + country: String! + zipCode: String! +} + +type CatBreed { + id: ID! + name: String! + origin: String! + characteristics: BreedCharacteristics! +} + +type DogBreed { + id: ID! + name: String! + origin: String! + characteristics: BreedCharacteristics! +} + +type BreedCharacteristics { + size: String! + temperament: String! + lifespan: String! } # Union Types for Testing diff --git a/v2/pkg/grpctest/util.go b/v2/pkg/grpctest/util.go new file mode 100644 index 000000000..b1a422f09 --- /dev/null +++ b/v2/pkg/grpctest/util.go @@ -0,0 +1,148 @@ +package grpctest + +import ( + "fmt" + + "github.com/wundergraph/graphql-go-tools/v2/pkg/grpctest/productv1" + "google.golang.org/protobuf/types/known/wrapperspb" +) + +// Helper function to create subcategories for a category +func createSubcategories(categoryId string, kind productv1.CategoryKind, count int) *productv1.ListOfSubcategory { + if count <= 0 { + return &productv1.ListOfSubcategory{ + List: &productv1.ListOfSubcategory_List{ + Items: []*productv1.Subcategory{}, + }, + } + } + + subcategories := make([]*productv1.Subcategory, 0, count) + for j := 1; j <= count; j++ { + subcategories = append(subcategories, &productv1.Subcategory{ + Id: fmt.Sprintf("%s-subcategory-%d", categoryId, j), + Name: fmt.Sprintf("%s Subcategory %d", kind.String(), j), + Description: &wrapperspb.StringValue{Value: fmt.Sprintf("Subcategory %d for %s", j, categoryId)}, + IsActive: true, + }) + } + + return &productv1.ListOfSubcategory{ + List: &productv1.ListOfSubcategory_List{ + Items: subcategories, + }, + } +} + +// Helper functions to convert input types to output types +func convertCategoryInputsToCategories(inputs []*productv1.CategoryInput) []*productv1.Category { + if inputs == nil { + return nil + } + results := make([]*productv1.Category, len(inputs)) + for i, input := range inputs { + results[i] = &productv1.Category{ + Id: fmt.Sprintf("cat-input-%d", i), + Name: input.GetName(), + Kind: input.GetKind(), + Subcategories: createSubcategories(fmt.Sprintf("cat-input-%d", i), input.GetKind(), i+1), + } + } + return results +} + +func convertCategoryInputListToCategories(inputs *productv1.ListOfCategoryInput) []*productv1.Category { + if inputs == nil || inputs.List == nil || inputs.List.Items == nil { + return nil + } + results := make([]*productv1.Category, len(inputs.List.Items)) + for i, input := range inputs.List.Items { + results[i] = &productv1.Category{ + Id: fmt.Sprintf("cat-list-input-%d", i), + Name: input.GetName(), + Kind: input.GetKind(), + Subcategories: createSubcategories(fmt.Sprintf("cat-list-input-%d", i), input.GetKind(), i+1), + } + } + return results +} + +func convertUserInputsToUsers(inputs *productv1.ListOfUserInput) []*productv1.User { + if inputs == nil || inputs.List == nil || inputs.List.Items == nil { + return nil + } + results := make([]*productv1.User, len(inputs.List.Items)) + for i, input := range inputs.List.Items { + results[i] = &productv1.User{ + Id: fmt.Sprintf("user-input-%d", i), + Name: input.GetName(), + } + } + return results +} + +func convertNestedUserInputsToUsers(nestedInputs *productv1.ListOfListOfUserInput) *productv1.ListOfListOfUser { + if nestedInputs == nil || nestedInputs.List == nil { + return &productv1.ListOfListOfUser{ + List: &productv1.ListOfListOfUser_List{ + Items: []*productv1.ListOfUser{}, + }, + } + } + + results := make([]*productv1.ListOfUser, len(nestedInputs.List.Items)) + for i, userList := range nestedInputs.List.Items { + users := make([]*productv1.User, len(userList.List.Items)) + for j, userInput := range userList.List.Items { + users[j] = &productv1.User{ + Id: fmt.Sprintf("nested-user-%d-%d", i, j), + Name: userInput.GetName(), + } + } + results[i] = &productv1.ListOfUser{ + List: &productv1.ListOfUser_List{ + Items: users, + }, + } + } + + return &productv1.ListOfListOfUser{ + List: &productv1.ListOfListOfUser_List{ + Items: results, + }, + } +} + +func convertNestedCategoryInputsToCategories(nestedInputs *productv1.ListOfListOfCategoryInput) *productv1.ListOfListOfCategory { + if nestedInputs == nil || nestedInputs.List == nil { + return &productv1.ListOfListOfCategory{ + List: &productv1.ListOfListOfCategory_List{ + Items: []*productv1.ListOfCategory{}, + }, + } + } + + results := make([]*productv1.ListOfCategory, len(nestedInputs.List.Items)) + for i, categoryList := range nestedInputs.List.Items { + categories := make([]*productv1.Category, len(categoryList.List.Items)) + for j, categoryInput := range categoryList.List.Items { + categories[j] = &productv1.Category{ + Id: fmt.Sprintf("nested-cat-%d-%d", i, j), + Name: categoryInput.GetName(), + Kind: categoryInput.GetKind(), + Subcategories: createSubcategories(fmt.Sprintf("nested-cat-%d-%d", i, j), categoryInput.GetKind(), j+1), + } + } + results[i] = &productv1.ListOfCategory{ + List: &productv1.ListOfCategory_List{ + Items: categories, + }, + } + } + + return &productv1.ListOfListOfCategory{ + List: &productv1.ListOfListOfCategory_List{ + Items: results, + }, + } +} From 33bbb70aa59e2604b5fa9a466fd4e999d24dd02f Mon Sep 17 00:00:00 2001 From: Ludwig Bedacht Date: Mon, 17 Nov 2025 13:41:22 +0100 Subject: [PATCH 52/72] chore: lint --- v2/pkg/grpctest/mockservice_enums.go | 3 ++- v2/pkg/grpctest/mockservice_lists.go | 3 ++- v2/pkg/grpctest/mockservice_nullable_fields.go | 3 ++- v2/pkg/grpctest/mockservice_resolve.go | 3 ++- v2/pkg/grpctest/util.go | 3 ++- 5 files changed, 10 insertions(+), 5 deletions(-) diff --git a/v2/pkg/grpctest/mockservice_enums.go b/v2/pkg/grpctest/mockservice_enums.go index 9e50197a7..184060084 100644 --- a/v2/pkg/grpctest/mockservice_enums.go +++ b/v2/pkg/grpctest/mockservice_enums.go @@ -4,8 +4,9 @@ import ( context "context" "fmt" - "github.com/wundergraph/graphql-go-tools/v2/pkg/grpctest/productv1" "google.golang.org/protobuf/types/known/wrapperspb" + + "github.com/wundergraph/graphql-go-tools/v2/pkg/grpctest/productv1" ) // Implementation for QueryCategories diff --git a/v2/pkg/grpctest/mockservice_lists.go b/v2/pkg/grpctest/mockservice_lists.go index 09eec065f..759b3cb1a 100644 --- a/v2/pkg/grpctest/mockservice_lists.go +++ b/v2/pkg/grpctest/mockservice_lists.go @@ -5,8 +5,9 @@ import ( "fmt" "math/rand" - "github.com/wundergraph/graphql-go-tools/v2/pkg/grpctest/productv1" "google.golang.org/protobuf/types/known/wrapperspb" + + "github.com/wundergraph/graphql-go-tools/v2/pkg/grpctest/productv1" ) // BlogPost query implementations diff --git a/v2/pkg/grpctest/mockservice_nullable_fields.go b/v2/pkg/grpctest/mockservice_nullable_fields.go index bac7255da..1d651ce7e 100644 --- a/v2/pkg/grpctest/mockservice_nullable_fields.go +++ b/v2/pkg/grpctest/mockservice_nullable_fields.go @@ -6,8 +6,9 @@ import ( "math" "math/rand" - "github.com/wundergraph/graphql-go-tools/v2/pkg/grpctest/productv1" "google.golang.org/protobuf/types/known/wrapperspb" + + "github.com/wundergraph/graphql-go-tools/v2/pkg/grpctest/productv1" ) // MutationCreateNullableFieldsType implements productv1.ProductServiceServer. diff --git a/v2/pkg/grpctest/mockservice_resolve.go b/v2/pkg/grpctest/mockservice_resolve.go index 6c0bd18cb..709189c4c 100644 --- a/v2/pkg/grpctest/mockservice_resolve.go +++ b/v2/pkg/grpctest/mockservice_resolve.go @@ -4,8 +4,9 @@ import ( context "context" "fmt" - "github.com/wundergraph/graphql-go-tools/v2/pkg/grpctest/productv1" "google.golang.org/protobuf/types/known/wrapperspb" + + "github.com/wundergraph/graphql-go-tools/v2/pkg/grpctest/productv1" ) // ResolveProductMascotRecommendation implements productv1.ProductServiceServer. diff --git a/v2/pkg/grpctest/util.go b/v2/pkg/grpctest/util.go index b1a422f09..606726e9f 100644 --- a/v2/pkg/grpctest/util.go +++ b/v2/pkg/grpctest/util.go @@ -3,8 +3,9 @@ package grpctest import ( "fmt" - "github.com/wundergraph/graphql-go-tools/v2/pkg/grpctest/productv1" "google.golang.org/protobuf/types/known/wrapperspb" + + "github.com/wundergraph/graphql-go-tools/v2/pkg/grpctest/productv1" ) // Helper function to create subcategories for a category From 730de1a956fa49abb5d80e32bf314142eeb57448 Mon Sep 17 00:00:00 2001 From: Ludwig Bedacht Date: Tue, 18 Nov 2025 10:47:12 +0100 Subject: [PATCH 53/72] chore: fix typo --- v2/pkg/grpctest/mockservice_enums.go | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/v2/pkg/grpctest/mockservice_enums.go b/v2/pkg/grpctest/mockservice_enums.go index 184060084..0158695b8 100644 --- a/v2/pkg/grpctest/mockservice_enums.go +++ b/v2/pkg/grpctest/mockservice_enums.go @@ -46,9 +46,9 @@ func (s *MockService) QueryCategoriesByKind(ctx context.Context, in *productv1.Q // Create 3 categories of the requested kind for i := 1; i <= 3; i++ { - subcategoties := make([]*productv1.Subcategory, 0, i) + subcategories := make([]*productv1.Subcategory, 0, i) for j := 1; j <= i; j++ { - subcategoties = append(subcategoties, &productv1.Subcategory{ + subcategories = append(subcategories, &productv1.Subcategory{ Id: fmt.Sprintf("%s-subcategory-%d", kind.String(), j), Name: fmt.Sprintf("%s Subcategory %d", kind.String(), j), Description: &wrapperspb.StringValue{Value: fmt.Sprintf("%s Subcategory %d", kind.String(), j)}, @@ -62,7 +62,7 @@ func (s *MockService) QueryCategoriesByKind(ctx context.Context, in *productv1.Q Kind: kind, Subcategories: &productv1.ListOfSubcategory{ List: &productv1.ListOfSubcategory_List{ - Items: subcategoties, + Items: subcategories, }, }, }) From 939574280d280f6932ded7c589c2a783f5744086 Mon Sep 17 00:00:00 2001 From: Ludwig Bedacht Date: Wed, 19 Nov 2025 11:51:12 +0100 Subject: [PATCH 54/72] chore: improve naming --- .../grpc_datasource/execution_plan.go | 31 ++++++++++--------- .../grpc_datasource/execution_plan_visitor.go | 23 +++++++------- .../execution_plan_visitor_federation.go | 6 ++-- 3 files changed, 30 insertions(+), 30 deletions(-) diff --git a/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go b/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go index 7f89996b2..3d332e544 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go +++ b/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go @@ -796,7 +796,7 @@ func (r *rpcPlanningContext) resolveServiceName(subgraphName string) string { return r.mapping.Service } -type resolvedField struct { +type resolverField struct { callerRef int parentTypeNode ast.Node fieldRef int @@ -818,7 +818,7 @@ type fragmentSelection struct { // enterResolverCompositeSelectionSet handles logic when entering a composite selection set for a given field resolver. // It appends the inline fragment selections to the resolved field and sets the fragment type. -func (r *rpcPlanningContext) enterResolverCompositeSelectionSet(oneOfType OneOfType, selectionSetRef int, resolvedField *resolvedField) { +func (r *rpcPlanningContext) enterResolverCompositeSelectionSet(oneOfType OneOfType, selectionSetRef int, resolvedField *resolverField) { inlineFragSelections := r.operation.SelectionSetInlineFragmentSelections(selectionSetRef) if len(inlineFragSelections) == 0 { return @@ -881,7 +881,7 @@ func (r *rpcPlanningContext) getMemberTypes(node ast.Node) ([]string, error) { } // setResolvedField sets the resolved field for a given field definition reference. -func (r *rpcPlanningContext) setResolvedField(walker *astvisitor.Walker, fieldDefRef int, fieldArgs []int, fieldPath ast.Path, resolvedField *resolvedField) error { +func (r *rpcPlanningContext) setResolvedField(walker *astvisitor.Walker, fieldDefRef int, fieldArgs []int, fieldPath ast.Path, resolvedField *resolverField) error { // We need to resolve the context fields for the given field definition reference. contextFields, err := r.resolveContextFields(walker, fieldDefRef) if err != nil { @@ -1031,23 +1031,24 @@ func (r *rpcPlanningContext) filterIDFieldsFunc(o ast.ObjectTypeDefinition, fiel type resolveRPCCallConfig struct { resolveConfig *ResolveRPCTypeField - resolvedField *resolvedField + resolvedField *resolverField contextMessage *RPCMessage fieldArgsMessage *RPCMessage } -func (r *rpcPlanningContext) resolveRequiredFields(typeName string, resolvedField *resolvedField) (*RPCMessage, error) { +// buildFieldResolverTypeMessage builds the message for a given field resolver type. +// When a field resolver returns a complex or composite type, we need to build a message for the type. +func (r *rpcPlanningContext) buildFieldResolverTypeMessage(typeName string, resolverField *resolverField) (*RPCMessage, error) { message := &RPCMessage{ Name: typeName, } - // TODO: handle composite types. - if len(resolvedField.fragmentSelections) > 0 { - message.FieldSelectionSet = make(RPCFieldSelectionSet, len(resolvedField.fragmentSelections)) - message.OneOfType = resolvedField.fragmentType - message.MemberTypes = resolvedField.memberTypes + if len(resolverField.fragmentSelections) > 0 { + message.FieldSelectionSet = make(RPCFieldSelectionSet, len(resolverField.fragmentSelections)) + message.OneOfType = resolverField.fragmentType + message.MemberTypes = resolverField.memberTypes - for _, fragmentSelection := range resolvedField.fragmentSelections { + for _, fragmentSelection := range resolverField.fragmentSelections { inlineFragmentTypeNode, found := r.definition.NodeByNameStr(fragmentSelection.typeName) if !found { return nil, fmt.Errorf("unable to build composite field: underlying fragment type node not found for type %s", fragmentSelection.typeName) @@ -1064,7 +1065,7 @@ func (r *rpcPlanningContext) resolveRequiredFields(typeName string, resolvedFiel return message, nil } - if resolvedField.fieldsSelectionSetRef == ast.InvalidRef { + if resolverField.fieldsSelectionSetRef == ast.InvalidRef { return nil, errors.New("unable to resolve required fields: no fields selection set found") } @@ -1073,7 +1074,7 @@ func (r *rpcPlanningContext) resolveRequiredFields(typeName string, resolvedFiel return nil, fmt.Errorf("parent type node not found for type %s", typeName) } - fieldRefs := r.operation.SelectionSetFieldRefs(resolvedField.fieldsSelectionSetRef) + fieldRefs := r.operation.SelectionSetFieldRefs(resolverField.fieldsSelectionSetRef) message.Fields = make(RPCFields, 0, len(fieldRefs)) for _, fieldRef := range fieldRefs { @@ -1171,7 +1172,7 @@ func (r *rpcPlanningContext) fieldDefinitionForType(fieldName, typeName string) } // createResolverRPCCalls creates a new call for each resolved field. -func (r *rpcPlanningContext) createResolverRPCCalls(subgraphName string, resolvedFields []resolvedField) ([]RPCCall, error) { +func (r *rpcPlanningContext) createResolverRPCCalls(subgraphName string, resolvedFields []resolverField) ([]RPCCall, error) { // We need to create a new call for each resolved field. calls := make([]RPCCall, 0, len(resolvedFields)) @@ -1259,7 +1260,7 @@ func (r *rpcPlanningContext) newResolveRPCCall(config *resolveRPCCallConfig) (RP var responseFieldsMessage *RPCMessage if dataType == DataTypeMessage { var err error - responseFieldsMessage, err = r.resolveRequiredFields( + responseFieldsMessage, err = r.buildFieldResolverTypeMessage( r.definition.ResolveTypeNameString(underlyingTypeRef), resolvedField, ) diff --git a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor.go b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor.go index 40ddfda92..df8d17759 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor.go +++ b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor.go @@ -56,7 +56,7 @@ type rpcPlanVisitor struct { parentCallID int fieldResolverAncestors stack[int] - resolvedFields []resolvedField + resolverFields []resolverField fieldPath ast.Path } @@ -77,7 +77,7 @@ func newRPCPlanVisitor(config rpcPlanVisitorConfig) *rpcPlanVisitor { subgraphName: cases.Title(language.Und, cases.NoLower).String(config.subgraphName), mapping: config.mapping, operationFieldRef: ast.InvalidRef, - resolvedFields: make([]resolvedField, 0), + resolverFields: make([]resolverField, 0), parentCallID: ast.InvalidRef, fieldResolverAncestors: newStack[int](0), fieldPath: make(ast.Path, 0), @@ -112,18 +112,18 @@ func (r *rpcPlanVisitor) EnterDocument(operation *ast.Document, definition *ast. // LeaveDocument implements astvisitor.DocumentVisitor. func (r *rpcPlanVisitor) LeaveDocument(_, _ *ast.Document) { - if len(r.resolvedFields) == 0 { + if len(r.resolverFields) == 0 { return } - calls, err := r.planCtx.createResolverRPCCalls(r.subgraphName, r.resolvedFields) + calls, err := r.planCtx.createResolverRPCCalls(r.subgraphName, r.resolverFields) if err != nil { r.walker.StopWithInternalErr(err) return } r.plan.Calls = append(r.plan.Calls, calls...) - r.resolvedFields = nil + r.resolverFields = nil } // EnterOperationDefinition implements astvisitor.EnterOperationDefinitionVisitor. @@ -202,14 +202,14 @@ func (r *rpcPlanVisitor) EnterSelectionSet(ref int) { return } - resolvedFieldAncestor := r.fieldResolverAncestors.peek() + resolverFieldAncestor := r.fieldResolverAncestors.peek() if compositType := r.planCtx.getCompositeType(r.walker.EnclosingTypeDefinition); compositType != OneOfTypeNone { memberTypes, err := r.planCtx.getMemberTypes(r.walker.EnclosingTypeDefinition) if err != nil { r.walker.StopWithInternalErr(err) return } - resolvedField := &r.resolvedFields[resolvedFieldAncestor] + resolvedField := &r.resolverFields[resolverFieldAncestor] resolvedField.memberTypes = memberTypes resolvedField.fieldsSelectionSetRef = ast.InvalidRef @@ -217,8 +217,7 @@ func (r *rpcPlanVisitor) EnterSelectionSet(ref int) { return } - // TODO: handle nested resolved fields. - r.resolvedFields[resolvedFieldAncestor].fieldsSelectionSetRef = ref + r.resolverFields[resolverFieldAncestor].fieldsSelectionSetRef = ref return } @@ -444,7 +443,7 @@ func (r *rpcPlanVisitor) enterFieldResolver(ref int, fieldDefRef int) { // We need to make sure to handle a hierarchy of arguments in order to perform parallel calls in order to retrieve the data. fieldArgs := r.operation.FieldArguments(ref) // We don't want to add fields from the selection set to the actual call - resolvedField := resolvedField{ + resolvedField := resolverField{ callerRef: r.parentCallID, parentTypeNode: r.walker.EnclosingTypeDefinition, fieldRef: ref, @@ -457,8 +456,8 @@ func (r *rpcPlanVisitor) enterFieldResolver(ref int, fieldDefRef int) { return } - r.resolvedFields = append(r.resolvedFields, resolvedField) - r.fieldResolverAncestors.push(len(r.resolvedFields) - 1) + r.resolverFields = append(r.resolverFields, resolvedField) + r.fieldResolverAncestors.push(len(r.resolverFields) - 1) r.fieldPath = r.fieldPath.WithFieldNameItem(r.operation.FieldNameBytes(ref)) // In case of nested fields with arguments, we need to increment the related call ID. diff --git a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor_federation.go b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor_federation.go index 2b1a34157..7fc92a1a7 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor_federation.go +++ b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor_federation.go @@ -52,7 +52,7 @@ type rpcPlanVisitorFederation struct { parentCallID int fieldResolverAncestors stack[int] - resolvedFields []resolvedField + resolvedFields []resolverField fieldPath ast.Path } @@ -69,7 +69,7 @@ func newRPCPlanVisitorFederation(config rpcPlanVisitorConfig) *rpcPlanVisitorFed entityInlineFragmentRef: ast.InvalidRef, }, federationConfigData: parseFederationConfigData(config.federationConfigs), - resolvedFields: make([]resolvedField, 0), + resolvedFields: make([]resolverField, 0), fieldResolverAncestors: newStack[int](0), parentCallID: ast.InvalidRef, fieldPath: ast.Path{}.WithFieldNameItem([]byte("result")), @@ -407,7 +407,7 @@ func (r *rpcPlanVisitorFederation) enterFieldResolver(ref int, fieldDefRef int) // We need to make sure to handle a hierarchy of arguments in order to perform parallel calls in order to retrieve the data. fieldArgs := r.operation.FieldArguments(ref) // We don't want to add fields from the selection set to the actual call - resolvedField := resolvedField{ + resolvedField := resolverField{ callerRef: r.parentCallID, parentTypeNode: r.walker.EnclosingTypeDefinition, fieldRef: ref, From 8517bf338833cd3671bb6871128460546406436d Mon Sep 17 00:00:00 2001 From: Ludwig Bedacht Date: Wed, 19 Nov 2025 11:57:17 +0100 Subject: [PATCH 55/72] chore: fix pagination --- v2/pkg/grpctest/mockservice_enums.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/v2/pkg/grpctest/mockservice_enums.go b/v2/pkg/grpctest/mockservice_enums.go index 0158695b8..c03c7533b 100644 --- a/v2/pkg/grpctest/mockservice_enums.go +++ b/v2/pkg/grpctest/mockservice_enums.go @@ -123,7 +123,7 @@ func (s *MockService) QueryFilterCategories(ctx context.Context, in *productv1.Q page := int(pagination.GetPage()) perPage := int(pagination.GetPerPage()) - if page > 0 && perPage > 0 && len(categories) > perPage { + if page > 0 && perPage > 0 { startIdx := (page - 1) * perPage endIdx := startIdx + perPage From b77dba29c51ffca8ff81fc96e653d579b5ab5023 Mon Sep 17 00:00:00 2001 From: Ludwig Bedacht Date: Wed, 19 Nov 2025 11:58:40 +0100 Subject: [PATCH 56/72] chore: fix imports --- v2/pkg/grpctest/mockservice.go | 2 +- v2/pkg/grpctest/mockservice_enums.go | 2 +- v2/pkg/grpctest/mockservice_lists.go | 2 +- v2/pkg/grpctest/mockservice_lookup.go | 2 +- v2/pkg/grpctest/mockservice_nullable_fields.go | 2 +- v2/pkg/grpctest/mockservice_resolve.go | 2 +- 6 files changed, 6 insertions(+), 6 deletions(-) diff --git a/v2/pkg/grpctest/mockservice.go b/v2/pkg/grpctest/mockservice.go index d98c6ba69..ae9d45ffb 100644 --- a/v2/pkg/grpctest/mockservice.go +++ b/v2/pkg/grpctest/mockservice.go @@ -1,7 +1,7 @@ package grpctest import ( - context "context" + "context" "fmt" "math/rand" "strconv" diff --git a/v2/pkg/grpctest/mockservice_enums.go b/v2/pkg/grpctest/mockservice_enums.go index c03c7533b..15e590025 100644 --- a/v2/pkg/grpctest/mockservice_enums.go +++ b/v2/pkg/grpctest/mockservice_enums.go @@ -1,7 +1,7 @@ package grpctest import ( - context "context" + "context" "fmt" "google.golang.org/protobuf/types/known/wrapperspb" diff --git a/v2/pkg/grpctest/mockservice_lists.go b/v2/pkg/grpctest/mockservice_lists.go index 759b3cb1a..9963861f1 100644 --- a/v2/pkg/grpctest/mockservice_lists.go +++ b/v2/pkg/grpctest/mockservice_lists.go @@ -1,7 +1,7 @@ package grpctest import ( - context "context" + "context" "fmt" "math/rand" diff --git a/v2/pkg/grpctest/mockservice_lookup.go b/v2/pkg/grpctest/mockservice_lookup.go index e431c137c..afebcbcb6 100644 --- a/v2/pkg/grpctest/mockservice_lookup.go +++ b/v2/pkg/grpctest/mockservice_lookup.go @@ -1,7 +1,7 @@ package grpctest import ( - context "context" + "context" "fmt" "math/rand" diff --git a/v2/pkg/grpctest/mockservice_nullable_fields.go b/v2/pkg/grpctest/mockservice_nullable_fields.go index 1d651ce7e..e112e5ca4 100644 --- a/v2/pkg/grpctest/mockservice_nullable_fields.go +++ b/v2/pkg/grpctest/mockservice_nullable_fields.go @@ -1,7 +1,7 @@ package grpctest import ( - context "context" + "context" "fmt" "math" "math/rand" diff --git a/v2/pkg/grpctest/mockservice_resolve.go b/v2/pkg/grpctest/mockservice_resolve.go index 709189c4c..107364ffe 100644 --- a/v2/pkg/grpctest/mockservice_resolve.go +++ b/v2/pkg/grpctest/mockservice_resolve.go @@ -1,7 +1,7 @@ package grpctest import ( - context "context" + "context" "fmt" "google.golang.org/protobuf/types/known/wrapperspb" From 87ea8e566a1bb5b986179302f436467a4751bd53 Mon Sep 17 00:00:00 2001 From: Ludwig Bedacht Date: Wed, 19 Nov 2025 13:15:51 +0100 Subject: [PATCH 57/72] chore: improve handling for potential null fields --- v2/pkg/grpctest/util.go | 16 +++++++++------- 1 file changed, 9 insertions(+), 7 deletions(-) diff --git a/v2/pkg/grpctest/util.go b/v2/pkg/grpctest/util.go index 606726e9f..849112a00 100644 --- a/v2/pkg/grpctest/util.go +++ b/v2/pkg/grpctest/util.go @@ -91,10 +91,11 @@ func convertNestedUserInputsToUsers(nestedInputs *productv1.ListOfListOfUserInpu } } - results := make([]*productv1.ListOfUser, len(nestedInputs.List.Items)) - for i, userList := range nestedInputs.List.Items { - users := make([]*productv1.User, len(userList.List.Items)) - for j, userInput := range userList.List.Items { + results := make([]*productv1.ListOfUser, len(nestedInputs.GetList().GetItems())) + for i, userList := range nestedInputs.GetList().GetItems() { + userListItems := userList.GetList().GetItems() + users := make([]*productv1.User, len(userListItems)) + for j, userInput := range userListItems { users[j] = &productv1.User{ Id: fmt.Sprintf("nested-user-%d-%d", i, j), Name: userInput.GetName(), @@ -124,9 +125,10 @@ func convertNestedCategoryInputsToCategories(nestedInputs *productv1.ListOfListO } results := make([]*productv1.ListOfCategory, len(nestedInputs.List.Items)) - for i, categoryList := range nestedInputs.List.Items { - categories := make([]*productv1.Category, len(categoryList.List.Items)) - for j, categoryInput := range categoryList.List.Items { + for i, categoryList := range nestedInputs.GetList().GetItems() { + categoryListItems := categoryList.GetList().GetItems() + categories := make([]*productv1.Category, len(categoryListItems)) + for j, categoryInput := range categoryListItems { categories[j] = &productv1.Category{ Id: fmt.Sprintf("nested-cat-%d-%d", i, j), Name: categoryInput.GetName(), From 6c0bd4095e470619bc3c4aa28c309bb938715f7a Mon Sep 17 00:00:00 2001 From: Ludwig Bedacht Date: Fri, 21 Nov 2025 10:43:29 +0100 Subject: [PATCH 58/72] chore: remove exists --- .../datasource/grpc_datasource/execution_plan.go | 15 ++++++++++----- 1 file changed, 10 insertions(+), 5 deletions(-) diff --git a/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go b/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go index 3d332e544..1e47f97db 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go +++ b/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go @@ -1133,8 +1133,8 @@ func (r *rpcPlanningContext) buildCompositeField(inlineFragmentNode ast.Node, fr result := make([]RPCField, 0, len(fieldRefs)) for _, fieldRef := range fieldRefs { - fieldDef, found := r.fieldDefinitionForType(r.operation.FieldNameString(fieldRef), fragmentSelection.typeName) - if !found { + fieldDef := r.fieldDefinitionForType(r.operation.FieldNameString(fieldRef), fragmentSelection.typeName) + if fieldDef == ast.InvalidRef { return nil, fmt.Errorf("unable to build composite field: field definition not found for field %s", r.operation.FieldNameString(fieldRef)) } @@ -1162,13 +1162,18 @@ func (r *rpcPlanningContext) buildCompositeField(inlineFragmentNode ast.Node, fr return result, nil } -func (r *rpcPlanningContext) fieldDefinitionForType(fieldName, typeName string) (ref int, exists bool) { +func (r *rpcPlanningContext) fieldDefinitionForType(fieldName, typeName string) int { node, found := r.definition.NodeByNameStr(typeName) if !found { - return ast.InvalidRef, false + return ast.InvalidRef + } + + if ref, found := r.definition.NodeFieldDefinitionByName(node, unsafebytes.StringToBytes(fieldName)); found { + return ref } - return r.definition.NodeFieldDefinitionByName(node, unsafebytes.StringToBytes(fieldName)) + return ast.InvalidRef + } // createResolverRPCCalls creates a new call for each resolved field. From 331f49abed6d1ba223ab4e453ab29fab8d2f3a54 Mon Sep 17 00:00:00 2001 From: Ludwig Bedacht Date: Fri, 21 Nov 2025 10:57:34 +0100 Subject: [PATCH 59/72] chore: better function name --- v2/pkg/engine/datasource/grpc_datasource/execution_plan.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go b/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go index 1e47f97db..ac833f46d 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go +++ b/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go @@ -1133,7 +1133,7 @@ func (r *rpcPlanningContext) buildCompositeField(inlineFragmentNode ast.Node, fr result := make([]RPCField, 0, len(fieldRefs)) for _, fieldRef := range fieldRefs { - fieldDef := r.fieldDefinitionForType(r.operation.FieldNameString(fieldRef), fragmentSelection.typeName) + fieldDef := r.fieldDefinitionRefForType(r.operation.FieldNameString(fieldRef), fragmentSelection.typeName) if fieldDef == ast.InvalidRef { return nil, fmt.Errorf("unable to build composite field: field definition not found for field %s", r.operation.FieldNameString(fieldRef)) } @@ -1162,7 +1162,7 @@ func (r *rpcPlanningContext) buildCompositeField(inlineFragmentNode ast.Node, fr return result, nil } -func (r *rpcPlanningContext) fieldDefinitionForType(fieldName, typeName string) int { +func (r *rpcPlanningContext) fieldDefinitionRefForType(fieldName, typeName string) int { node, found := r.definition.NodeByNameStr(typeName) if !found { return ast.InvalidRef From 6ed7f6a176501ef7bfb655412895549cdf48556a Mon Sep 17 00:00:00 2001 From: Ludwig Bedacht Date: Fri, 21 Nov 2025 13:29:33 +0100 Subject: [PATCH 60/72] chore: update function comment --- v2/pkg/engine/datasource/grpc_datasource/execution_plan.go | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go b/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go index ac833f46d..4dcee860b 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go +++ b/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go @@ -849,7 +849,8 @@ func (r *rpcPlanningContext) isFieldResolver(fieldRef int, isRootField bool) boo return len(r.operation.FieldArguments(fieldRef)) > 0 } -// getCompositeType returns the most recent composite type from the type definitions. +// getCompositeType checks whether the node is an interface or union type. +// It returns OneOfTypeNone for non-composite types. func (r *rpcPlanningContext) getCompositeType(node ast.Node) OneOfType { switch node.Kind { case ast.NodeKindInterfaceTypeDefinition: From d9d3692329db97be36e1fd00d62971bc871a6d1b Mon Sep 17 00:00:00 2001 From: Ludwig Bedacht Date: Mon, 24 Nov 2025 14:23:51 +0100 Subject: [PATCH 61/72] chore: ignore nested field resolvers in composite types --- .../grpc_datasource/execution_plan.go | 16 +- .../execution_plan_field_resolvers_test.go | 316 ++++++++++++++++++ .../grpc_datasource/execution_plan_test.go | 40 ++- .../execution_plan_visitor_federation.go | 1 - 4 files changed, 357 insertions(+), 16 deletions(-) diff --git a/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go b/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go index 4dcee860b..a109d75c2 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go +++ b/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go @@ -760,7 +760,7 @@ func (r *rpcPlanningContext) buildFieldMessage(fieldTypeNode ast.Node, fieldRef return nil, fmt.Errorf("unable to resolve type node for inline fragment %s", typeName) } - fields, err := r.buildCompositeField(inlineFragmentTypeNode, fragmentSelection{ + fields, err := r.buildCompositeFields(inlineFragmentTypeNode, fragmentSelection{ typeName: typeName, selectionSetRef: selectionSetRef, }) @@ -777,6 +777,10 @@ func (r *rpcPlanningContext) buildFieldMessage(fieldTypeNode ast.Node, fieldRef } for _, fieldRef := range fieldRefs { + if r.isFieldResolver(fieldRef, false) { + continue + } + field, err := r.buildRequiredField(fieldTypeNode, fieldRef) if err != nil { return nil, err @@ -1055,7 +1059,7 @@ func (r *rpcPlanningContext) buildFieldResolverTypeMessage(typeName string, reso return nil, fmt.Errorf("unable to build composite field: underlying fragment type node not found for type %s", fragmentSelection.typeName) } - fields, err := r.buildCompositeField(inlineFragmentTypeNode, fragmentSelection) + fields, err := r.buildCompositeFields(inlineFragmentTypeNode, fragmentSelection) if err != nil { return nil, err } @@ -1129,11 +1133,17 @@ func (r *rpcPlanningContext) buildRequiredField(typeNode ast.Node, fieldRef int) return field, nil } -func (r *rpcPlanningContext) buildCompositeField(inlineFragmentNode ast.Node, fragmentSelection fragmentSelection) ([]RPCField, error) { +// buildCompositeFields creates fields for a given inline fragment node and its selection set. +// It returns a list of fields that have been composed from the inputs. +func (r *rpcPlanningContext) buildCompositeFields(inlineFragmentNode ast.Node, fragmentSelection fragmentSelection) ([]RPCField, error) { fieldRefs := r.operation.SelectionSetFieldRefs(fragmentSelection.selectionSetRef) result := make([]RPCField, 0, len(fieldRefs)) for _, fieldRef := range fieldRefs { + if r.isFieldResolver(fieldRef, false) { + continue + } + fieldDef := r.fieldDefinitionRefForType(r.operation.FieldNameString(fieldRef), fragmentSelection.typeName) if fieldDef == ast.InvalidRef { return nil, fmt.Errorf("unable to build composite field: field definition not found for field %s", r.operation.FieldNameString(fieldRef)) diff --git a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_field_resolvers_test.go b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_field_resolvers_test.go index 17cd47dd5..9f18f1342 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_field_resolvers_test.go +++ b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_field_resolvers_test.go @@ -2,6 +2,11 @@ package grpcdatasource import ( "testing" + + "github.com/wundergraph/graphql-go-tools/v2/pkg/ast" + "github.com/wundergraph/graphql-go-tools/v2/pkg/astvalidation" + "github.com/wundergraph/graphql-go-tools/v2/pkg/internal/unsafeparser" + "github.com/wundergraph/graphql-go-tools/v2/pkg/operationreport" ) func TestExecutionPlanFieldResolvers(t *testing.T) { @@ -2187,3 +2192,314 @@ func TestExecutionPlanFieldResolvers_WithCompositeTypes(t *testing.T) { }) } } + +func TestExecutionPlanFieldResolvers_CustomSchemas(t *testing.T) { + tests := []struct { + name string + operation string + schema ast.Document + subgraphName string + mapping *GRPCMapping + expectedPlan *RPCExecutionPlan + expectedError string + }{ + { + name: "Should not include nested resolvers in composite type selection set when building the execution plan", + subgraphName: "Foo", + operation: ` + query FooQuery($foo: String!, $baz: String!) { + foo { + fooResolver(foo: $foo) { + ... on Baz { + bazResolver(baz: $baz) + } + } + } + }`, + schema: schemaWithNestedResolverAndCompositeType(t), + mapping: mappingWithNestedResolverAndCompositeType(t), + expectedPlan: &RPCExecutionPlan{ + Calls: []RPCCall{ + { + ServiceName: "Foo", + MethodName: "QueryFoo", + Request: RPCMessage{ + Name: "QueryFooRequest", + }, + Response: RPCMessage{ + Name: "QueryFooResponse", + Fields: []RPCField{ + { + Name: "foo", + ProtoTypeName: DataTypeMessage, + JSONPath: "foo", + Message: &RPCMessage{ + Name: "Foo", + Fields: RPCFields{}, + }, + }, + }, + }, + }, + { + Kind: CallKindResolve, + DependentCalls: []int{0}, + ResponsePath: buildPath("foo.fooResolver"), + ServiceName: "Foo", + MethodName: "ResolveFooFooResolver", + Request: RPCMessage{ + Name: "ResolveFooFooResolverRequest", + Fields: []RPCField{ + { + Name: "context", + ProtoTypeName: DataTypeMessage, + Repeated: true, + Message: &RPCMessage{ + Name: "ResolveFooFooResolverContext", + Fields: []RPCField{ + { + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", + ResolvePath: buildPath("foo.id"), + }, + }, + }, + }, + { + Name: "field_args", + ProtoTypeName: DataTypeMessage, + Message: &RPCMessage{ + Name: "ResolveFooFooResolverArgs", + Fields: []RPCField{ + { + Name: "foo", + ProtoTypeName: DataTypeString, + JSONPath: "foo", + }, + }, + }, + }, + }, + }, + Response: RPCMessage{ + Name: "ResolveFooFooResolverResponse", + Fields: []RPCField{ + { + Name: "result", + ProtoTypeName: DataTypeMessage, + Repeated: true, + JSONPath: "result", + Message: &RPCMessage{ + Name: "ResolveFooFooResolverResult", + Fields: RPCFields{ + { + Name: "foo_resolver", + ProtoTypeName: DataTypeMessage, + JSONPath: "fooResolver", + Message: &RPCMessage{ + Name: "Bar", + FieldSelectionSet: RPCFieldSelectionSet{"Baz": {}}, + OneOfType: OneOfTypeInterface, + MemberTypes: []string{"Baz"}, + }, + }, + }, + }, + }, + }, + }, + }, + { + Kind: CallKindResolve, + DependentCalls: []int{1}, + ResponsePath: buildPath("foo.fooResolver.bazResolver"), + ServiceName: "Foo", + MethodName: "ResolveBazBazResolver", + Request: RPCMessage{ + Name: "ResolveBazBazResolverRequest", + Fields: []RPCField{ + { + Name: "context", + ProtoTypeName: DataTypeMessage, + Repeated: true, + Message: &RPCMessage{ + Name: "ResolveBazBazResolverContext", + Fields: []RPCField{ + { + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", + ResolvePath: buildPath("foo.fooResolver.id"), + }, + }, + }, + }, + { + Name: "field_args", + ProtoTypeName: DataTypeMessage, + Message: &RPCMessage{ + Name: "ResolveBazBazResolverArgs", + Fields: []RPCField{ + { + Name: "baz", + ProtoTypeName: DataTypeString, + JSONPath: "baz", + }, + }, + }, + }, + }, + }, + Response: RPCMessage{ + Name: "ResolveBazBazResolverResponse", + Fields: []RPCField{ + { + Name: "result", + ProtoTypeName: DataTypeMessage, + Repeated: true, + JSONPath: "result", + Message: &RPCMessage{ + Name: "ResolveBazBazResolverResult", + Fields: RPCFields{ + { + Name: "baz_resolver", + ProtoTypeName: DataTypeString, + JSONPath: "bazResolver", + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + } + + for _, tt := range tests { + + operation := unsafeparser.ParseGraphqlDocumentString(tt.operation) + + t.Run(tt.name, func(t *testing.T) { + t.Parallel() + runTestWithConfig(t, testCase{ + expectedPlan: tt.expectedPlan, + expectedError: tt.expectedError, + }, testConfig{ + subgraphName: tt.subgraphName, + mapping: tt.mapping, + schemaDoc: tt.schema, + operationDoc: operation, + }) + }) + } +} + +func schemaWithNestedResolverAndCompositeType(t *testing.T) ast.Document { + schema := ` + +scalar connect__FieldSet +directive @connect__fieldResolver(context: connect__FieldSet!) on FIELD_DEFINITION + +schema { + query: Query +} + +type Foo { + id: ID! + fooResolver(foo: String!): Bar! @connect__fieldResolver(context: "id") +} + +interface Bar { + id: ID! +} + +type Baz implements Bar { + id: ID! + bazResolver(baz: String!): String! @connect__fieldResolver(context: "id") +} + +type Query { + foo: Foo! +}` + doc := unsafeparser.ParseGraphqlDocumentStringWithBaseSchema(schema) + + report := &operationreport.Report{} + astvalidation.DefaultDefinitionValidator().Validate(&doc, report) + if report.HasErrors() { + t.Fatalf("failed to validate schema: %s", report.Error()) + } + + return doc +} + +func mappingWithNestedResolverAndCompositeType(_ *testing.T) *GRPCMapping { + return &GRPCMapping{ + Service: "Foo", + QueryRPCs: RPCConfigMap[RPCConfig]{ + "foo": { + RPC: "QueryFoo", + Request: "QueryFooRequest", + Response: "QueryFooResponse", + }, + }, + Fields: map[string]FieldMap{ + "Query": { + "foo": { + TargetName: "foo", + }, + }, + "Foo": { + "id": { + TargetName: "id", + }, + "fooResolver": { + TargetName: "foo_resolver", + }, + }, + "Baz": { + "id": { + TargetName: "id", + }, + "bazResolver": { + TargetName: "baz_resolver", + }, + }, + "Bar": { + "id": { + TargetName: "id", + }, + }, + }, + ResolveRPCs: RPCConfigMap[ResolveRPCMapping]{ + "Foo": { + "fooResolver": { + FieldMappingData: FieldMapData{ + TargetName: "foo_resolver", + ArgumentMappings: FieldArgumentMap{ + "foo": "foo", + }, + }, + RPC: "ResolveFooFooResolver", + Request: "ResolveFooFooResolverRequest", + Response: "ResolveFooFooResolverResponse", + }, + }, + "Baz": { + "bazResolver": { + FieldMappingData: FieldMapData{ + TargetName: "baz_resolver", + ArgumentMappings: FieldArgumentMap{ + "baz": "baz", + }, + }, + RPC: "ResolveBazBazResolver", + Request: "ResolveBazBazResolverRequest", + Response: "ResolveBazBazResolverResponse", + }, + }, + }, + } +} diff --git a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_test.go b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_test.go index 760a3e786..d63dbd6fe 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_test.go +++ b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_test.go @@ -20,22 +20,20 @@ type testCase struct { expectedError string } -func runTest(t *testing.T, testCase testCase) { - // Parse the GraphQL schema - schemaDoc := grpctest.MustGraphQLSchema(t) - - // Parse the GraphQL query - queryDoc, report := astparser.ParseGraphqlDocumentString(testCase.query) - if report.HasErrors() { - t.Fatalf("failed to parse query: %s", report.Error()) - } +type testConfig struct { + subgraphName string + mapping *GRPCMapping + schemaDoc ast.Document + operationDoc ast.Document +} +func runTestWithConfig(t *testing.T, testCase testCase, testConfig testConfig) { rpcPlanVisitor := newRPCPlanVisitor(rpcPlanVisitorConfig{ - subgraphName: "Products", - mapping: testMapping(), + subgraphName: testConfig.subgraphName, + mapping: testConfig.mapping, }) - plan, err := rpcPlanVisitor.PlanOperation(&queryDoc, &schemaDoc) + plan, err := rpcPlanVisitor.PlanOperation(&testConfig.operationDoc, &testConfig.schemaDoc) if err != nil { require.NotEmpty(t, testCase.expectedError, "expected error to be empty, got: %s", err.Error()) @@ -50,6 +48,24 @@ func runTest(t *testing.T, testCase testCase) { } } +func runTest(t *testing.T, testCase testCase) { + // Parse the GraphQL schema + schemaDoc := grpctest.MustGraphQLSchema(t) + + // Parse the GraphQL query + queryDoc, report := astparser.ParseGraphqlDocumentString(testCase.query) + if report.HasErrors() { + t.Fatalf("failed to parse query: %s", report.Error()) + } + + runTestWithConfig(t, testCase, testConfig{ + subgraphName: "Products", + mapping: testMapping(), + schemaDoc: schemaDoc, + operationDoc: queryDoc, + }) +} + // buildPath builds a path from a string which is a dot-separated list of field names. func buildPath(path string) ast.Path { b := make([]byte, len(path)) diff --git a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor_federation.go b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor_federation.go index 7fc92a1a7..a6d5b70fa 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor_federation.go +++ b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor_federation.go @@ -208,7 +208,6 @@ func (r *rpcPlanVisitorFederation) EnterSelectionSet(ref int) { return } - // TODO: handle nested resolved fields. r.resolvedFields[resolvedFieldAncestor].fieldsSelectionSetRef = ref return } From 48dc314d11dc939c21b3f910464305b682b57d76 Mon Sep 17 00:00:00 2001 From: Ludwig Bedacht Date: Mon, 24 Nov 2025 14:31:08 +0100 Subject: [PATCH 62/72] chore: fix error message --- v2/pkg/engine/datasource/grpc_datasource/execution_plan.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go b/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go index a109d75c2..c736cad71 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go +++ b/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go @@ -1157,7 +1157,7 @@ func (r *rpcPlanningContext) buildCompositeFields(inlineFragmentNode ast.Node, f if field.ProtoTypeName == DataTypeMessage && r.operation.FieldHasSelections(fieldRef) { fieldTypeNode, found := r.definition.ResolveNodeFromTypeRef(r.definition.FieldDefinitionType(fieldDef)) if !found { - return nil, fmt.Errorf("unable to build required field: unable to resolve field type node for field %s", r.operation.FieldNameString(fieldRef)) + return nil, fmt.Errorf("unable to build composite field: unable to resolve field type node for field %s", r.operation.FieldNameString(fieldRef)) } message, err := r.buildFieldMessage(fieldTypeNode, fieldRef) From cbab2991b3e748ae109e38098b37b45c8ecc42d3 Mon Sep 17 00:00:00 2001 From: Ludwig Bedacht Date: Mon, 24 Nov 2025 15:05:03 +0100 Subject: [PATCH 63/72] make test parallel --- .../grpc_datasource/execution_plan_field_resolvers_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_field_resolvers_test.go b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_field_resolvers_test.go index 9f18f1342..9ff60b3f9 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_field_resolvers_test.go +++ b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_field_resolvers_test.go @@ -2194,6 +2194,7 @@ func TestExecutionPlanFieldResolvers_WithCompositeTypes(t *testing.T) { } func TestExecutionPlanFieldResolvers_CustomSchemas(t *testing.T) { + t.Parallel() tests := []struct { name string operation string @@ -2379,7 +2380,6 @@ func TestExecutionPlanFieldResolvers_CustomSchemas(t *testing.T) { } for _, tt := range tests { - operation := unsafeparser.ParseGraphqlDocumentString(tt.operation) t.Run(tt.name, func(t *testing.T) { From 2be4f0667f9790d8e23fae08adf40a81d30c4cca Mon Sep 17 00:00:00 2001 From: Ludwig Bedacht Date: Mon, 24 Nov 2025 16:49:01 +0100 Subject: [PATCH 64/72] chore: add comments --- v2/pkg/engine/datasource/grpc_datasource/execution_plan.go | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go b/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go index c736cad71..5a51aa2d9 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go +++ b/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go @@ -1048,6 +1048,7 @@ func (r *rpcPlanningContext) buildFieldResolverTypeMessage(typeName string, reso Name: typeName, } + // If the resolved field returns a composite type we need to handle the selection set for the inline fragment. if len(resolverField.fragmentSelections) > 0 { message.FieldSelectionSet = make(RPCFieldSelectionSet, len(resolverField.fragmentSelections)) message.OneOfType = resolverField.fragmentType @@ -1070,10 +1071,13 @@ func (r *rpcPlanningContext) buildFieldResolverTypeMessage(typeName string, reso return message, nil } + // field resolvers which return a non scalar type must have a selection set. + // If we don't have a selection set we return an error. if resolverField.fieldsSelectionSetRef == ast.InvalidRef { return nil, errors.New("unable to resolve required fields: no fields selection set found") } + // If the resolved field does not return a composite type we handle the selection set for the required field. parentTypeNode, found := r.definition.NodeByNameStr(typeName) if !found { return nil, fmt.Errorf("parent type node not found for type %s", typeName) From 6779a3f52377b36fdf13151443370cbd172a5e20 Mon Sep 17 00:00:00 2001 From: Ludwig Bedacht Date: Mon, 24 Nov 2025 17:07:50 +0100 Subject: [PATCH 65/72] chore: return error instead --- v2/pkg/engine/datasource/grpc_datasource/execution_plan.go | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go b/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go index 5a51aa2d9..3c267ae59 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go +++ b/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go @@ -716,10 +716,12 @@ func (r *rpcPlanningContext) buildInputMessageField(typeRef int, fieldName, json return field, nil } +// buildFieldMessage builds a message from a field definition. +// It handles complex and composite types. func (r *rpcPlanningContext) buildFieldMessage(fieldTypeNode ast.Node, fieldRef int) (*RPCMessage, error) { field := r.operation.Fields[fieldRef] if !field.HasSelections { - return nil, nil + return nil, fmt.Errorf("unable to build field message: field %s has no selections", r.operation.FieldAliasOrNameString(fieldRef)) } fieldRefs := make([]int, 0) From 9171864cafcca84598c7e66eb61ad47f3e901359 Mon Sep 17 00:00:00 2001 From: Ludwig Bedacht Date: Tue, 25 Nov 2025 10:42:54 +0100 Subject: [PATCH 66/72] chore: simplify LeaveField logic --- v2/pkg/astvisitor/visitor.go | 1 + .../grpc_datasource/execution_plan_visitor.go | 31 +++++++++++++------ .../execution_plan_visitor_federation.go | 23 ++++++++------ 3 files changed, 35 insertions(+), 20 deletions(-) diff --git a/v2/pkg/astvisitor/visitor.go b/v2/pkg/astvisitor/visitor.go index 86f284c72..3f0beeb6e 100644 --- a/v2/pkg/astvisitor/visitor.go +++ b/v2/pkg/astvisitor/visitor.go @@ -4021,6 +4021,7 @@ func (w *Walker) FieldDefinitionDirectiveArgumentValueByName(field int, directiv } // InRootField returns true if the current field is a root field. +// This helper function can be used in EnterField and LeaveField. func (w *Walker) InRootField() bool { return w.CurrentKind == ast.NodeKindField && len(w.Ancestors) == 2 && diff --git a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor.go b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor.go index df8d17759..d99f7e1ee 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor.go +++ b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor.go @@ -409,23 +409,34 @@ func (r *rpcPlanVisitor) EnterField(ref int) { // LeaveField implements astvisitor.FieldVisitor. func (r *rpcPlanVisitor) LeaveField(ref int) { r.fieldPath = r.fieldPath.RemoveLastItem() - r.fieldResolverAncestors.pop() + inRootField := r.walker.InRootField() + + if inRootField { + // When leaving a root field, this indicates that we are done with the current call. + // We need to add the current call to the plan and reset the current call. + r.finalizeCall() + // RPC call is done, we can return. + return + } + + if r.planCtx.isFieldResolver(ref, inRootField) { + // Pop the field resolver ancestor only when leaving a field resolver field. + r.fieldResolverAncestors.pop() - // If we are not in the operation field, we can increment the response field index. - if !r.walker.InRootField() { // If the field has arguments, we need to decrement the related call ID. // This is because we can also have nested arguments, which require the underlying field to be resolved // by values provided by the parent call. - if r.operation.FieldHasArguments(ref) { - r.parentCallID-- - } - - r.planInfo.currentResponseFieldIndex++ - return + r.parentCallID-- } + // If we are not in the operation field, we can increment the response field index. + r.planInfo.currentResponseFieldIndex++ +} + +// finalizeCall finalizes the current call and resets the current call. +func (r *rpcPlanVisitor) finalizeCall() { r.plan.Calls[r.currentCallID] = *r.currentCall - r.currentCall = &RPCCall{} + r.currentCall = nil r.currentCallID++ if r.currentCallID < len(r.operationFieldRefs) { diff --git a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor_federation.go b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor_federation.go index a6d5b70fa..4d69d5f20 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor_federation.go +++ b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor_federation.go @@ -381,21 +381,24 @@ func (r *rpcPlanVisitorFederation) EnterField(ref int) { // LeaveField implements astvisitor.FieldVisitor. func (r *rpcPlanVisitorFederation) LeaveField(ref int) { r.fieldPath = r.fieldPath.RemoveLastItem() - r.fieldResolverAncestors.pop() - // If we are not in the operation field, we can increment the response field index. - if !r.walker.InRootField() { + + inRootField := r.walker.InRootField() + if inRootField { + return + } + + if r.planCtx.isFieldResolver(ref, inRootField) { + // Pop the field resolver ancestor only when leaving a field resolver field. + r.fieldResolverAncestors.pop() + // If the field has arguments, we need to decrement the related call ID. // This is because we can also have nested arguments, which require the underlying field to be resolved // by values provided by the parent call. - if r.operation.FieldHasArguments(ref) { - r.parentCallID-- - } - - r.planInfo.currentResponseFieldIndex++ - return + r.parentCallID-- } - r.planInfo.currentResponseFieldIndex = 0 + // If we are not in the operation field, we can increment the response field index. + r.planInfo.currentResponseFieldIndex++ } // enterFieldResolver enters a field resolver. From 08044c28850d2ca263fa15738bc211b79312c9a1 Mon Sep 17 00:00:00 2001 From: Ludwig Bedacht Date: Tue, 25 Nov 2025 10:44:12 +0100 Subject: [PATCH 67/72] chore: improve comment --- .../datasource/grpc_datasource/execution_plan_visitor.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor.go b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor.go index d99f7e1ee..c801446f1 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor.go +++ b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor.go @@ -412,8 +412,8 @@ func (r *rpcPlanVisitor) LeaveField(ref int) { inRootField := r.walker.InRootField() if inRootField { - // When leaving a root field, this indicates that we are done with the current call. - // We need to add the current call to the plan and reset the current call. + // Leaving a root field means the current RPC call is complete. + // Add the current call to the plan and reset the call state. r.finalizeCall() // RPC call is done, we can return. return From 881ca363dc5e7692451e7f1df92e2f85362c9bbc Mon Sep 17 00:00:00 2001 From: Ludwig Bedacht Date: Tue, 25 Nov 2025 12:56:32 +0100 Subject: [PATCH 68/72] chore: skip node on duplicate field --- .../engine/datasource/grpc_datasource/execution_plan_visitor.go | 1 + .../grpc_datasource/execution_plan_visitor_federation.go | 2 +- 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor.go b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor.go index c801446f1..b75cf80a1 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor.go +++ b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor.go @@ -375,6 +375,7 @@ func (r *rpcPlanVisitor) EnterField(ref int) { // prevent duplicate fields fieldAlias := r.operation.FieldAliasString(ref) if r.planInfo.currentResponseMessage.Fields.Exists(fieldName, fieldAlias) { + r.walker.SkipNode() return } diff --git a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor_federation.go b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor_federation.go index 4d69d5f20..f18ccac4d 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor_federation.go +++ b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor_federation.go @@ -346,7 +346,7 @@ func (r *rpcPlanVisitorFederation) EnterField(ref int) { // prevent duplicate fields fieldAlias := r.operation.FieldAliasString(ref) if r.planInfo.currentResponseMessage.Fields.Exists(fieldName, fieldAlias) { - r.fieldPath = r.fieldPath.WithFieldNameItem([]byte{}) + r.walker.SkipNode() return } From 37df47a142a9d565c00061c1fdd5896ca1a4c993 Mon Sep 17 00:00:00 2001 From: Ludwig Bedacht Date: Wed, 26 Nov 2025 14:16:37 +0100 Subject: [PATCH 69/72] fix: invalid index when skipping duplicate field --- .../execution_plan_federation_test.go | 121 +++++++++ .../grpc_datasource/execution_plan_test.go | 253 +++++++++++------- .../execution_plan_visitor_federation.go | 13 +- 3 files changed, 290 insertions(+), 97 deletions(-) diff --git a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_federation_test.go b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_federation_test.go index 2d8f76fce..7f19058c3 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_federation_test.go +++ b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_federation_test.go @@ -862,6 +862,127 @@ func TestEntityKeys(t *testing.T) { }, }, }, + { + name: "Should create an execution plan for an entity lookup with a key field and nested field", + query: `query EntityLookup($representations: [_Any!]!) { _entities(representations: $representations) { ... on User { __typename id name address { street } } } }`, + schema: testFederationSchemaString(` + type Query { + _entities(representations: [_Any!]!): [_Entity]! + } + type User @key(fields: "id") { + id: ID! + name: String! + address: Address! + } + + type Address { + id: ID! + street: String! + } + `, []string{"User"}), + mapping: &GRPCMapping{ + Service: "Products", + EntityRPCs: map[string][]EntityRPCConfig{ + "User": { + { + Key: "id", + RPCConfig: RPCConfig{ + RPC: "LookupUserById", + Request: "LookupUserByIdRequest", + Response: "LookupUserByIdResponse", + }, + }, + }, + }, + }, + federationConfigs: plan.FederationFieldConfigurations{ + { + TypeName: "User", + SelectionSet: "id", + }, + }, + + expectedPlan: &RPCExecutionPlan{ + Calls: []RPCCall{ + { + ServiceName: "Products", + MethodName: "LookupUserById", + Kind: CallKindEntity, + // Define the structure of the request message + Request: RPCMessage{ + Name: "LookupUserByIdRequest", + Fields: []RPCField{ + { + Name: "keys", + ProtoTypeName: DataTypeMessage, + Repeated: true, + JSONPath: "representations", + Message: &RPCMessage{ + Name: "LookupUserByIdKey", + MemberTypes: []string{"User"}, + Fields: []RPCField{ + { + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", + }, + }, + }, + }, + }, + }, + // Define the structure of the response message + Response: RPCMessage{ + Name: "LookupUserByIdResponse", + Fields: []RPCField{ + { + Name: "result", + ProtoTypeName: DataTypeMessage, + Repeated: true, + JSONPath: "_entities", + Message: &RPCMessage{ + Name: "User", + Fields: []RPCField{ + { + Name: "__typename", + ProtoTypeName: DataTypeString, + JSONPath: "__typename", + StaticValue: "User", + }, + { + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", + }, + { + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", + }, + { + Name: "address", + ProtoTypeName: DataTypeMessage, + JSONPath: "address", + Message: &RPCMessage{ + Name: "Address", + Fields: []RPCField{ + { + Name: "street", + ProtoTypeName: DataTypeString, + JSONPath: "street", + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, } for _, tt := range tests { diff --git a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_test.go b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_test.go index d63dbd6fe..94c22cdc4 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_test.go +++ b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_test.go @@ -1,7 +1,6 @@ package grpcdatasource import ( - "strings" "testing" "github.com/google/go-cmp/cmp" @@ -9,9 +8,10 @@ import ( "github.com/wundergraph/graphql-go-tools/v2/pkg/ast" "github.com/wundergraph/graphql-go-tools/v2/pkg/astparser" + "github.com/wundergraph/graphql-go-tools/v2/pkg/asttransform" "github.com/wundergraph/graphql-go-tools/v2/pkg/astvalidation" "github.com/wundergraph/graphql-go-tools/v2/pkg/grpctest" - "github.com/wundergraph/graphql-go-tools/v2/pkg/operationreport" + "github.com/wundergraph/graphql-go-tools/v2/pkg/internal/unsafeparser" ) type testCase struct { @@ -102,7 +102,7 @@ func TestQueryExecutionPlans(t *testing.T) { }{ { name: "Should include typename when requested", - query: `query UsersWithTypename { users { __typename id name } }`, + query: `query UsersWithTypename { users { __typename id __typename name } }`, mapping: testMapping(), expectedPlan: &RPCExecutionPlan{ Calls: []RPCCall{ @@ -1155,6 +1155,137 @@ func TestQueryExecutionPlans(t *testing.T) { } } +func TestQueryExecutionPlans_WithConfig(t *testing.T) { + t.Parallel() + tests := []struct { + name string + operation string + schema ast.Document + subgraphName string + mapping *GRPCMapping + expectedPlan *RPCExecutionPlan + expectedError string + }{ + { + name: "Should create an execution plan for a query with duplicated and nested fields", + operation: ` + query DuplicatedAndNestedFieldsQuery { + users { + __typename + id + __typename + name + address { + street + } + } + }`, + schema: testSchema(t, ` + type Query { + users: [User!]! + } + + type User { + id: ID! + name: String! + address: Address! + } + + type Address { + street: String! + }`), + subgraphName: "Products", + mapping: &GRPCMapping{ + Service: "Products", + QueryRPCs: map[string]RPCConfig{ + "users": { + RPC: "QueryUsers", + Request: "QueryUsersRequest", + Response: "QueryUsersResponse", + }, + }, + // No need to define fields as we don't have camel case in any of them. + }, + expectedPlan: &RPCExecutionPlan{ + Calls: []RPCCall{ + { + ServiceName: "Products", + MethodName: "QueryUsers", + Request: RPCMessage{ + Name: "QueryUsersRequest", + }, + Response: RPCMessage{ + Name: "QueryUsersResponse", + Fields: []RPCField{ + { + Name: "users", + ProtoTypeName: DataTypeMessage, + Repeated: true, + JSONPath: "users", + Message: &RPCMessage{ + Name: "User", + Fields: []RPCField{ + { + Name: "__typename", + ProtoTypeName: DataTypeString, + JSONPath: "__typename", + StaticValue: "User", + }, + { + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", + }, + { + Name: "name", + ProtoTypeName: DataTypeString, + JSONPath: "name", + }, + { + Name: "address", + ProtoTypeName: DataTypeMessage, + JSONPath: "address", + Message: &RPCMessage{ + Name: "Address", + Fields: []RPCField{ + { + Name: "street", + ProtoTypeName: DataTypeString, + JSONPath: "street", + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + } + + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + t.Parallel() + + runTestWithConfig(t, testCase{ + + query: tt.operation, + expectedPlan: tt.expectedPlan, + expectedError: tt.expectedError, + }, testConfig{ + subgraphName: tt.subgraphName, + mapping: tt.mapping, + schemaDoc: tt.schema, + operationDoc: unsafeparser.ParseGraphqlDocumentString(tt.operation), + }) + }) + } +} + func TestProductExecutionPlan(t *testing.T) { t.Parallel() tests := []struct { @@ -1365,53 +1496,12 @@ func TestProductExecutionPlan(t *testing.T) { for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { t.Parallel() - report := &operationreport.Report{} - // Parse the GraphQL schema - schemaDoc := grpctest.MustGraphQLSchema(t) - astvalidation.DefaultDefinitionValidator().Validate(&schemaDoc, report) - if report.HasErrors() { - t.Fatalf("failed to validate schema: %s", report.Error()) - } - - // Parse the GraphQL query - queryDoc, queryReport := astparser.ParseGraphqlDocumentString(tt.query) - if queryReport.HasErrors() { - t.Fatalf("failed to parse query: %s", queryReport.Error()) - } - - astvalidation.DefaultOperationValidator().Validate(&queryDoc, &schemaDoc, report) - if report.HasErrors() { - t.Fatalf("failed to validate query: %s", report.Error()) - } - - planner, err := NewPlanner("Products", testMapping(), nil) - if err != nil { - t.Fatalf("failed to create planner: %s", err) - } - outPlan, err := planner.PlanOperation(&queryDoc, &schemaDoc) - if err != nil { - t.Fatalf("failed to plan operation: %s", err) - } - - if tt.expectedError != "" { - if err == nil { - t.Fatalf("expected error, got nil") - } - if !strings.Contains(err.Error(), tt.expectedError) { - t.Fatalf("expected error to contain %q, got %q", tt.expectedError, err.Error()) - } - return - } - - if err != nil { - t.Fatalf("unexpected error: %s", err) - } - - diff := cmp.Diff(tt.expectedPlan, outPlan) - if diff != "" { - t.Fatalf("execution plan mismatch: %s", diff) - } + runTest(t, testCase{ + query: tt.query, + expectedPlan: tt.expectedPlan, + expectedError: tt.expectedError, + }) }) } } @@ -2475,55 +2565,34 @@ func TestProductExecutionPlanWithAliases(t *testing.T) { for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { t.Parallel() - report := &operationreport.Report{} - // Parse the GraphQL schema - schemaDoc := grpctest.MustGraphQLSchema(t) - - astvalidation.DefaultDefinitionValidator().Validate(&schemaDoc, report) - if report.HasErrors() { - t.Fatalf("failed to validate schema: %s", report.Error()) - } - - // Parse the GraphQL query - queryDoc, queryReport := astparser.ParseGraphqlDocumentString(tt.query) - if queryReport.HasErrors() { - t.Fatalf("failed to parse query: %s", queryReport.Error()) - } - astvalidation.DefaultOperationValidator().Validate(&queryDoc, &schemaDoc, report) - if report.HasErrors() { - t.Fatalf("failed to validate query: %s", report.Error()) - } + runTest(t, testCase{ + query: tt.query, + expectedPlan: tt.expectedPlan, + expectedError: tt.expectedError, + }) + }) + } - planner, err := NewPlanner("Products", testMapping(), nil) - if err != nil { - t.Fatalf("failed to create planner: %s", err) - } +} - outPlan, err := planner.PlanOperation(&queryDoc, &schemaDoc) - if err != nil { - t.Fatalf("failed to plan operation: %s", err) - } +func testSchema(t *testing.T, schema string) ast.Document { + t.Helper() - if tt.expectedError != "" { - if err == nil { - t.Fatalf("expected error, got nil") - } - if !strings.Contains(err.Error(), tt.expectedError) { - t.Fatalf("expected error to contain %q, got %q", tt.expectedError, err.Error()) - } - return - } + doc, report := astparser.ParseGraphqlDocumentString(schema) + if report.HasErrors() { + t.Fatalf("failed to parse schema: %s", report.Error()) + } - if err != nil { - t.Fatalf("unexpected error: %s", err) - } + if err := asttransform.MergeDefinitionWithBaseSchema(&doc); err != nil { + t.Fatalf("failed to merge schema: %s", err) + } - diff := cmp.Diff(tt.expectedPlan, outPlan) - if diff != "" { - t.Fatalf("execution plan mismatch: %s", diff) - } - }) + astvalidation.DefaultDefinitionValidator().Validate(&doc, &report) + if report.HasErrors() { + t.Fatalf("failed to validate schema: %s", report.Error()) } + return doc + } diff --git a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor_federation.go b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor_federation.go index f18ccac4d..8c59f1ee6 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor_federation.go +++ b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor_federation.go @@ -221,14 +221,16 @@ func (r *rpcPlanVisitorFederation) EnterSelectionSet(ref int) { r.planInfo.currentResponseMessage.Fields[r.planInfo.currentResponseFieldIndex].Message = r.planCtx.newMessageFromSelectionSet(r.walker.EnclosingTypeDefinition, ref) } - if r.IsEntityInlineFragment(r.walker.Ancestor()) { - r.planInfo.currentResponseMessage.Fields[r.planInfo.currentResponseFieldIndex].Message.AppendTypeNameField(r.entityInfo.typeName) - } - // Add the current response message to the ancestors and set the current response message to the current field message r.planInfo.responseMessageAncestors = append(r.planInfo.responseMessageAncestors, r.planInfo.currentResponseMessage) r.planInfo.currentResponseMessage = r.planInfo.currentResponseMessage.Fields[r.planInfo.currentResponseFieldIndex].Message + // Ensure that the entity inline fragment message has a typename field, + // to map the json data after receiving the response. + if r.IsEntityInlineFragment(r.walker.Ancestor()) { + r.planInfo.currentResponseMessage.AppendTypeNameField(r.entityInfo.typeName) + } + // Check if the ancestor type is a composite type (interface or union) // and set the oneof type and member types. if err := r.handleCompositeType(r.walker.Ancestor()); err != nil { @@ -243,7 +245,8 @@ func (r *rpcPlanVisitorFederation) EnterSelectionSet(ref int) { // when leaving the selection set. r.planInfo.responseFieldIndexAncestors = append(r.planInfo.responseFieldIndexAncestors, r.planInfo.currentResponseFieldIndex) - r.planInfo.currentResponseFieldIndex = 0 // reset the field index for the current selection set + // Reset the field index for the current selection set to the length of the current response message fields. + r.planInfo.currentResponseFieldIndex = len(r.planInfo.currentResponseMessage.Fields) } func (r *rpcPlanVisitorFederation) handleCompositeType(node ast.Node) error { From 4b04f320056611b1dc6bf2dd18d883deda48ed67 Mon Sep 17 00:00:00 2001 From: Ludwig Bedacht Date: Wed, 26 Nov 2025 14:46:31 +0100 Subject: [PATCH 70/72] chore: more improvements --- .../grpc_datasource/execution_plan_field_resolvers_test.go | 4 ++-- .../datasource/grpc_datasource/execution_plan_visitor.go | 3 +++ .../grpc_datasource/execution_plan_visitor_federation.go | 3 +++ 3 files changed, 8 insertions(+), 2 deletions(-) diff --git a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_field_resolvers_test.go b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_field_resolvers_test.go index 9ff60b3f9..35a8eb54a 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_field_resolvers_test.go +++ b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_field_resolvers_test.go @@ -19,7 +19,7 @@ func TestExecutionPlanFieldResolvers(t *testing.T) { }{ { name: "Should create an execution plan for a query with a field resolver", - query: "query CategoriesWithFieldResolvers($whoop: ProductCountFilter) { categories { id name kind productCount(filters: $whoop) } }", + query: "query CategoriesWithFieldResolvers($whoop: ProductCountFilter) { categories { id productCount(filters: $whoop) name kind } }", expectedPlan: &RPCExecutionPlan{ Calls: []RPCCall{ { @@ -908,7 +908,7 @@ func TestExecutionPlanFieldResolvers_WithNestedResolvers(t *testing.T) { }{ { name: "Should create an execution plan for a query with nested field resolvers", - query: "query CategoriesWithNestedResolvers($metricType: String, $baseline: Float!) { categories { categoryMetrics(metricType: $metricType) { id metricType value normalizedScore(baseline: $baseline) } } }", + query: "query CategoriesWithNestedResolvers($metricType: String, $baseline: Float!) { categories { categoryMetrics(metricType: $metricType) { id normalizedScore(baseline: $baseline) metricType value } } }", expectedPlan: &RPCExecutionPlan{ Calls: []RPCCall{ { diff --git a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor.go b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor.go index b75cf80a1..e1072781e 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor.go +++ b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor.go @@ -428,6 +428,9 @@ func (r *rpcPlanVisitor) LeaveField(ref int) { // This is because we can also have nested arguments, which require the underlying field to be resolved // by values provided by the parent call. r.parentCallID-- + + // We handle field resolvers differently, so we don't want to increment the response field index. + return } // If we are not in the operation field, we can increment the response field index. diff --git a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor_federation.go b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor_federation.go index 8c59f1ee6..d0b0ddbbb 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor_federation.go +++ b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor_federation.go @@ -398,6 +398,9 @@ func (r *rpcPlanVisitorFederation) LeaveField(ref int) { // This is because we can also have nested arguments, which require the underlying field to be resolved // by values provided by the parent call. r.parentCallID-- + + // We handle field resolvers differently, so we don't want to increment the response field index. + return } // If we are not in the operation field, we can increment the response field index. From ab44255e6af288746c4feb3dabf2d884275b6178 Mon Sep 17 00:00:00 2001 From: Ludwig Bedacht Date: Fri, 28 Nov 2025 16:05:15 +0100 Subject: [PATCH 71/72] fix: incorrect error propagation --- .../datasource/grpc_datasource/compiler.go | 33 ++++++++++++++++--- .../grpc_datasource/execution_plan.go | 19 ++++++++--- .../grpc_datasource/execution_plan_visitor.go | 2 -- .../grpc_datasource/grpc_datasource.go | 6 ++-- 4 files changed, 45 insertions(+), 15 deletions(-) diff --git a/v2/pkg/engine/datasource/grpc_datasource/compiler.go b/v2/pkg/engine/datasource/grpc_datasource/compiler.go index a22e6a6b9..a9261ee13 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/compiler.go +++ b/v2/pkg/engine/datasource/grpc_datasource/compiler.go @@ -625,9 +625,7 @@ func (p *RPCCompiler) buildProtoMessageWithContext(inputMessage Message, rpcMess if err != nil { return nil, err } - - // Set the context and args fields - p.setMessageValue(rootMessage, contextSchemaField.Name, protoref.ValueOfList(contextList)) + // Set the args field p.setMessageValue(rootMessage, argsRPCField.Name, protoref.ValueOfMessage(args)) return rootMessage, nil @@ -736,7 +734,7 @@ func (p *RPCCompiler) resolveDataForPath(messsage protoref.Message, path ast.Pat switch fd.Kind() { case protoref.MessageKind: if fd.IsList() { - return []protoref.Value{field} + return []protoref.Value{protoref.ValueOfList(field.List())} } return p.resolveDataForPath(field.Message(), path[1:]) @@ -835,8 +833,33 @@ func (p *RPCCompiler) newEmptyListMessageByName(msg protoref.Message, name strin return msg.Mutable(msg.Descriptor().Fields().ByName(protoref.Name(name))).List() } -func (p *RPCCompiler) setMessageValue(message protoref.Message, fieldName string, value protoref.Value) { +func (p *RPCCompiler) setMessageValue(message protoref.Message, fieldName string, value protoref.Value) error { + fd := message.Descriptor().Fields().ByName(protoref.Name(fieldName)) + if fd == nil { + return fmt.Errorf("field %s not found in message %s", fieldName, message.Descriptor().Name()) + } + + // If we are setting a list value here, we need to create a copy of the list + // because the field descriptor is included in the type check, so we cannot asign it using `Set` directly. + if fd.IsList() { + list := message.Mutable(fd).List() + source, ok := value.Interface().(protoref.List) + if !ok { + return fmt.Errorf("value is not a list") + } + + p.copyListValues(source, list) + return nil + } + message.Set(message.Descriptor().Fields().ByName(protoref.Name(fieldName)), value) + return nil +} + +func (p *RPCCompiler) copyListValues(source protoref.List, destination protoref.List) { + for i := range source.Len() { + destination.Append(source.Get(i)) + } } // buildProtoMessage recursively builds a protobuf message from an RPCMessage definition diff --git a/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go b/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go index 3c267ae59..c6589f45d 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go +++ b/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go @@ -825,6 +825,13 @@ type fragmentSelection struct { // enterResolverCompositeSelectionSet handles logic when entering a composite selection set for a given field resolver. // It appends the inline fragment selections to the resolved field and sets the fragment type. func (r *rpcPlanningContext) enterResolverCompositeSelectionSet(oneOfType OneOfType, selectionSetRef int, resolvedField *resolverField) { + resolvedField.fieldsSelectionSetRef = ast.InvalidRef + + // In case of an interface we can select individual fields from the interface without having to use an inline fragment. + if len(r.operation.SelectionSetFieldRefs(selectionSetRef)) > 0 { + resolvedField.fieldsSelectionSetRef = selectionSetRef + } + inlineFragSelections := r.operation.SelectionSetInlineFragmentSelections(selectionSetRef) if len(inlineFragSelections) == 0 { return @@ -1050,6 +1057,12 @@ func (r *rpcPlanningContext) buildFieldResolverTypeMessage(typeName string, reso Name: typeName, } + // field resolvers which return a non scalar type must have a selection set. + // If we don't have a selection set we return an error. + if len(resolverField.fragmentSelections) == 0 && resolverField.fieldsSelectionSetRef == ast.InvalidRef { + return nil, errors.New("unable to resolve required fields: no fields selection set found") + } + // If the resolved field returns a composite type we need to handle the selection set for the inline fragment. if len(resolverField.fragmentSelections) > 0 { message.FieldSelectionSet = make(RPCFieldSelectionSet, len(resolverField.fragmentSelections)) @@ -1069,14 +1082,10 @@ func (r *rpcPlanningContext) buildFieldResolverTypeMessage(typeName string, reso message.FieldSelectionSet[fragmentSelection.typeName] = fields } - - return message, nil } - // field resolvers which return a non scalar type must have a selection set. - // If we don't have a selection set we return an error. if resolverField.fieldsSelectionSetRef == ast.InvalidRef { - return nil, errors.New("unable to resolve required fields: no fields selection set found") + return message, nil } // If the resolved field does not return a composite type we handle the selection set for the required field. diff --git a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor.go b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor.go index e1072781e..307bba52b 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor.go +++ b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor.go @@ -211,8 +211,6 @@ func (r *rpcPlanVisitor) EnterSelectionSet(ref int) { } resolvedField := &r.resolverFields[resolverFieldAncestor] resolvedField.memberTypes = memberTypes - resolvedField.fieldsSelectionSetRef = ast.InvalidRef - r.planCtx.enterResolverCompositeSelectionSet(compositType, ref, resolvedField) return } diff --git a/v2/pkg/engine/datasource/grpc_datasource/grpc_datasource.go b/v2/pkg/engine/datasource/grpc_datasource/grpc_datasource.go index 8a196cbc6..cb114c014 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/grpc_datasource.go +++ b/v2/pkg/engine/datasource/grpc_datasource/grpc_datasource.go @@ -149,9 +149,8 @@ func (d *DataSource) Load(ctx context.Context, input []byte, out *bytes.Buffer) } if err := errGrp.Wait(); err != nil { - out.Write(builder.writeErrorBytes(err)) failed = true - return nil + return err } for _, result := range results { @@ -169,7 +168,8 @@ func (d *DataSource) Load(ctx context.Context, input []byte, out *bytes.Buffer) return nil }); err != nil || failed { - return err + out.Write(builder.writeErrorBytes(err)) + return nil } data := builder.toDataObject(root) From 805a7e19cd5ecdb0f6bfd474c75cb9dfc71920d3 Mon Sep 17 00:00:00 2001 From: Ludwig Bedacht Date: Fri, 28 Nov 2025 16:31:43 +0100 Subject: [PATCH 72/72] fix: properly include introspection typename --- .../grpc_datasource/execution_plan.go | 9 +- .../execution_plan_field_resolvers_test.go | 115 ++++++++++++++++++ 2 files changed, 119 insertions(+), 5 deletions(-) diff --git a/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go b/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go index c6589f45d..d1f20f304 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go +++ b/v2/pkg/engine/datasource/grpc_datasource/execution_plan.go @@ -826,6 +826,7 @@ type fragmentSelection struct { // It appends the inline fragment selections to the resolved field and sets the fragment type. func (r *rpcPlanningContext) enterResolverCompositeSelectionSet(oneOfType OneOfType, selectionSetRef int, resolvedField *resolverField) { resolvedField.fieldsSelectionSetRef = ast.InvalidRef + resolvedField.fragmentType = oneOfType // In case of an interface we can select individual fields from the interface without having to use an inline fragment. if len(r.operation.SelectionSetFieldRefs(selectionSetRef)) > 0 { @@ -849,8 +850,6 @@ func (r *rpcPlanningContext) enterResolverCompositeSelectionSet(oneOfType OneOfT selectionSetRef: inlinFragSelectionSetRef, }) } - - resolvedField.fragmentType = oneOfType } // isFieldResolver checks if a field is a field resolver. @@ -1054,7 +1053,9 @@ type resolveRPCCallConfig struct { // When a field resolver returns a complex or composite type, we need to build a message for the type. func (r *rpcPlanningContext) buildFieldResolverTypeMessage(typeName string, resolverField *resolverField) (*RPCMessage, error) { message := &RPCMessage{ - Name: typeName, + Name: typeName, + OneOfType: resolverField.fragmentType, + MemberTypes: resolverField.memberTypes, } // field resolvers which return a non scalar type must have a selection set. @@ -1066,8 +1067,6 @@ func (r *rpcPlanningContext) buildFieldResolverTypeMessage(typeName string, reso // If the resolved field returns a composite type we need to handle the selection set for the inline fragment. if len(resolverField.fragmentSelections) > 0 { message.FieldSelectionSet = make(RPCFieldSelectionSet, len(resolverField.fragmentSelections)) - message.OneOfType = resolverField.fragmentType - message.MemberTypes = resolverField.memberTypes for _, fragmentSelection := range resolverField.fragmentSelections { inlineFragmentTypeNode, found := r.definition.NodeByNameStr(fragmentSelection.typeName) diff --git a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_field_resolvers_test.go b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_field_resolvers_test.go index 35a8eb54a..5f702f6ff 100644 --- a/v2/pkg/engine/datasource/grpc_datasource/execution_plan_field_resolvers_test.go +++ b/v2/pkg/engine/datasource/grpc_datasource/execution_plan_field_resolvers_test.go @@ -2377,6 +2377,121 @@ func TestExecutionPlanFieldResolvers_CustomSchemas(t *testing.T) { }, }, }, + { + name: "Should correctly include typename field when providing empty selection set", + subgraphName: "Foo", + operation: ` + query FooQuery($foo: String!, $baz: String!) { + foo { + fooResolver(foo: $foo) { + __typename + } + } + }`, + schema: schemaWithNestedResolverAndCompositeType(t), + mapping: mappingWithNestedResolverAndCompositeType(t), + expectedPlan: &RPCExecutionPlan{ + Calls: []RPCCall{ + { + ServiceName: "Foo", + MethodName: "QueryFoo", + Request: RPCMessage{ + Name: "QueryFooRequest", + }, + Response: RPCMessage{ + Name: "QueryFooResponse", + Fields: []RPCField{ + { + Name: "foo", + ProtoTypeName: DataTypeMessage, + JSONPath: "foo", + Message: &RPCMessage{ + Name: "Foo", + Fields: RPCFields{}, + }, + }, + }, + }, + }, + { + Kind: CallKindResolve, + DependentCalls: []int{0}, + ResponsePath: buildPath("foo.fooResolver"), + ServiceName: "Foo", + MethodName: "ResolveFooFooResolver", + Request: RPCMessage{ + Name: "ResolveFooFooResolverRequest", + Fields: []RPCField{ + { + Name: "context", + ProtoTypeName: DataTypeMessage, + Repeated: true, + Message: &RPCMessage{ + Name: "ResolveFooFooResolverContext", + Fields: []RPCField{ + { + Name: "id", + ProtoTypeName: DataTypeString, + JSONPath: "id", + ResolvePath: buildPath("foo.id"), + }, + }, + }, + }, + { + Name: "field_args", + ProtoTypeName: DataTypeMessage, + Message: &RPCMessage{ + Name: "ResolveFooFooResolverArgs", + Fields: []RPCField{ + { + Name: "foo", + ProtoTypeName: DataTypeString, + JSONPath: "foo", + }, + }, + }, + }, + }, + }, + Response: RPCMessage{ + Name: "ResolveFooFooResolverResponse", + Fields: []RPCField{ + { + Name: "result", + ProtoTypeName: DataTypeMessage, + Repeated: true, + JSONPath: "result", + Message: &RPCMessage{ + Name: "ResolveFooFooResolverResult", + Fields: RPCFields{ + { + Name: "foo_resolver", + ProtoTypeName: DataTypeMessage, + JSONPath: "fooResolver", + Message: &RPCMessage{ + Name: "Bar", + OneOfType: OneOfTypeInterface, + MemberTypes: []string{"Baz"}, + Fields: RPCFields{ + { + Name: "__typename", + ProtoTypeName: DataTypeString, + JSONPath: "__typename", + StaticValue: "Bar", + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, } for _, tt := range tests {