From 68ebdafbaadc61dd407dadbeba27b4f953779845 Mon Sep 17 00:00:00 2001 From: Joseph Schorr Date: Fri, 5 Aug 2022 16:38:57 -0400 Subject: [PATCH] Implement DispatchLookupSubjects for finding all reachable subjects of a particular type for a resource and permission First part of work for #261 --- internal/dispatch/caching/caching.go | 85 ++- .../dispatch/caching/cachingdispatch_test.go | 4 + internal/dispatch/caching/delegate.go | 4 + internal/dispatch/dispatch.go | 27 + internal/dispatch/graph/graph.go | 35 ++ .../dispatch/graph/lookupsubjects_test.go | 216 ++++++++ internal/dispatch/remote/cluster.go | 35 ++ internal/dispatch/stream.go | 8 +- internal/graph/lookupsubjects.go | 498 ++++++++++++++++++ internal/graph/reachableresources.go | 4 +- pkg/proto/dispatch/v1/00_zerolog.go | 13 + pkg/proto/dispatch/v1/dispatch.pb.go | 343 +++++++++--- pkg/proto/dispatch/v1/dispatch.pb.validate.go | 355 +++++++++++++ pkg/proto/dispatch/v1/dispatch_grpc.pb.go | 63 +++ proto/internal/dispatch/v1/dispatch.proto | 17 + 15 files changed, 1626 insertions(+), 81 deletions(-) create mode 100644 internal/dispatch/graph/lookupsubjects_test.go create mode 100644 internal/graph/lookupsubjects.go diff --git a/internal/dispatch/caching/caching.go b/internal/dispatch/caching/caching.go index 0390683869..c20395348e 100644 --- a/internal/dispatch/caching/caching.go +++ b/internal/dispatch/caching/caching.go @@ -35,6 +35,8 @@ type Dispatcher struct { lookupFromCacheCounter prometheus.Counter reachableResourcesTotalCounter prometheus.Counter reachableResourcesFromCacheCounter prometheus.Counter + lookupSubjectsTotalCounter prometheus.Counter + lookupSubjectsFromCacheCounter prometheus.Counter cacheHits prometheus.CounterFunc cacheMisses prometheus.CounterFunc @@ -54,10 +56,15 @@ type reachableResourcesResultEntry struct { responses []*v1.DispatchReachableResourcesResponse } +type lookupSubjectsResultEntry struct { + responses []*v1.DispatchLookupSubjectsResponse +} + var ( checkResultEntryCost = int64(unsafe.Sizeof(checkResultEntry{})) lookupResultEntryEmptyCost = int64(unsafe.Sizeof(lookupResultEntry{})) reachbleResourcesEntryEmptyCost = int64(unsafe.Sizeof(reachableResourcesResultEntry{})) + lookupSubjectsEntryEmptyCost = int64(unsafe.Sizeof(lookupSubjectsResultEntry{})) ) // NewCachingDispatcher creates a new dispatch.Dispatcher which delegates dispatch requests @@ -122,6 +129,17 @@ func NewCachingDispatcher( Name: "reachable_resources_from_cache_total", }) + lookupSubjectsTotalCounter := prometheus.NewCounter(prometheus.CounterOpts{ + Namespace: prometheusNamespace, + Subsystem: prometheusSubsystem, + Name: "lookup_subjects_total", + }) + lookupSubjectsFromCacheCounter := prometheus.NewCounter(prometheus.CounterOpts{ + Namespace: prometheusNamespace, + Subsystem: prometheusSubsystem, + Name: "lookup_subjects_from_cache_total", + }) + cacheHitsTotal := prometheus.NewCounterFunc(prometheus.CounterOpts{ Namespace: prometheusNamespace, Subsystem: prometheusSubsystem, @@ -178,6 +196,14 @@ func NewCachingDispatcher( if err != nil { return nil, fmt.Errorf(errCachingInitialization, err) } + err = prometheus.Register(lookupSubjectsTotalCounter) + if err != nil { + return nil, fmt.Errorf(errCachingInitialization, err) + } + err = prometheus.Register(lookupSubjectsFromCacheCounter) + if err != nil { + return nil, fmt.Errorf(errCachingInitialization, err) + } // Export some ristretto metrics err = prometheus.Register(cacheHitsTotal) @@ -212,6 +238,8 @@ func NewCachingDispatcher( lookupFromCacheCounter: lookupFromCacheCounter, reachableResourcesTotalCounter: reachableResourcesTotalCounter, reachableResourcesFromCacheCounter: reachableResourcesFromCacheCounter, + lookupSubjectsTotalCounter: lookupSubjectsTotalCounter, + lookupSubjectsFromCacheCounter: lookupSubjectsFromCacheCounter, cacheHits: cacheHitsTotal, cacheMisses: cacheMissesTotal, costAddedBytes: costAddedBytes, @@ -343,7 +371,7 @@ func (cd *Dispatcher) DispatchReachableResources(req *v1.DispatchReachableResour wrapped := &dispatch.WrappedDispatchStream[*v1.DispatchReachableResourcesResponse]{ Stream: stream, Ctx: stream.Context(), - Processor: func(result *v1.DispatchReachableResourcesResponse) (*v1.DispatchReachableResourcesResponse, error) { + Processor: func(result *v1.DispatchReachableResourcesResponse) (*v1.DispatchReachableResourcesResponse, bool, error) { mu.Lock() defer mu.Unlock() @@ -357,7 +385,7 @@ func (cd *Dispatcher) DispatchReachableResources(req *v1.DispatchReachableResour for _, id := range result.Resource.ResourceIds { estimatedSize += int64(len(id)) } - return result, nil + return result, true, nil }, } @@ -372,6 +400,59 @@ func (cd *Dispatcher) DispatchReachableResources(req *v1.DispatchReachableResour return err } +// DispatchLookupSubjects implements dispatch.LookupSubjects interface and does not do any caching yet. +func (cd *Dispatcher) DispatchLookupSubjects(req *v1.DispatchLookupSubjectsRequest, stream dispatch.LookupSubjectsStream) error { + cd.lookupSubjectsTotalCounter.Inc() + + requestKey := dispatch.LookupSubjectsRequestToKey(req) + if cachedResultRaw, found := cd.c.Get(requestKey); found { + cachedResult := cachedResultRaw.(lookupSubjectsResultEntry) + cd.lookupSubjectsFromCacheCounter.Inc() + for _, result := range cachedResult.responses { + err := stream.Publish(result) + if err != nil { + return err + } + } + + return nil + } + + var mu sync.Mutex + estimatedSize := lookupSubjectsEntryEmptyCost + toCacheResults := []*v1.DispatchLookupSubjectsResponse{} + wrapped := &dispatch.WrappedDispatchStream[*v1.DispatchLookupSubjectsResponse]{ + Stream: stream, + Ctx: stream.Context(), + Processor: func(result *v1.DispatchLookupSubjectsResponse) (*v1.DispatchLookupSubjectsResponse, bool, error) { + mu.Lock() + defer mu.Unlock() + + adjustedResult := proto.Clone(result).(*v1.DispatchLookupSubjectsResponse) + adjustedResult.Metadata.CachedDispatchCount = adjustedResult.Metadata.DispatchCount + adjustedResult.Metadata.DispatchCount = 0 + adjustedResult.Metadata.DebugInfo = nil + + toCacheResults = append(toCacheResults, adjustedResult) + + for _, id := range result.FoundSubjectIds { + estimatedSize += int64(len(id)) + } + return result, true, nil + }, + } + + err := cd.d.DispatchLookupSubjects(req, wrapped) + + // We only want to cache the result if there was no error + if err == nil { + toCache := lookupSubjectsResultEntry{toCacheResults} + cd.c.Set(requestKey, toCache, estimatedSize) + } + + return err +} + func (cd *Dispatcher) Close() error { prometheus.Unregister(cd.checkTotalCounter) prometheus.Unregister(cd.lookupTotalCounter) diff --git a/internal/dispatch/caching/cachingdispatch_test.go b/internal/dispatch/caching/cachingdispatch_test.go index 8213ffb026..3db7bc792f 100644 --- a/internal/dispatch/caching/cachingdispatch_test.go +++ b/internal/dispatch/caching/cachingdispatch_test.go @@ -150,6 +150,10 @@ func (ddm delegateDispatchMock) DispatchReachableResources(req *v1.DispatchReach return nil } +func (ddm delegateDispatchMock) DispatchLookupSubjects(req *v1.DispatchLookupSubjectsRequest, stream dispatch.LookupSubjectsStream) error { + return nil +} + func (ddm delegateDispatchMock) Close() error { return nil } diff --git a/internal/dispatch/caching/delegate.go b/internal/dispatch/caching/delegate.go index 3c09a6f332..9cfe7577de 100644 --- a/internal/dispatch/caching/delegate.go +++ b/internal/dispatch/caching/delegate.go @@ -35,4 +35,8 @@ func (fd fakeDelegate) DispatchReachableResources(req *v1.DispatchReachableResou panic(errMessage) } +func (fd fakeDelegate) DispatchLookupSubjects(req *v1.DispatchLookupSubjectsRequest, stream dispatch.LookupSubjectsStream) error { + panic(errMessage) +} + var _ dispatch.Dispatcher = fakeDelegate{} diff --git a/internal/dispatch/dispatch.go b/internal/dispatch/dispatch.go index 7133187f94..394a17c84b 100644 --- a/internal/dispatch/dispatch.go +++ b/internal/dispatch/dispatch.go @@ -22,6 +22,7 @@ type Dispatcher interface { Expand Lookup ReachableResources + LookupSubjects // Close closes the dispatcher. Close() error @@ -60,6 +61,18 @@ type ReachableResources interface { ) error } +// LookupSubjectsStream is an alias for the stream to which found subjects will be written. +type LookupSubjectsStream = Stream[*v1.DispatchLookupSubjectsResponse] + +// LookupSubjects interface describes just the methods required to dispatch lookup subjects requests. +type LookupSubjects interface { + // DispatchLookupSubjects submits a single lookup subjects request, writing its results to the specified stream. + DispatchLookupSubjects( + req *v1.DispatchLookupSubjectsRequest, + stream LookupSubjectsStream, + ) error +} + // HasMetadata is an interface for requests containing resolver metadata. type HasMetadata interface { zerolog.LogObjectMarshaler @@ -90,6 +103,7 @@ const ( lookupPrefix cachePrefix = "l" expandPrefix cachePrefix = "e" reachableResourcesPrefix cachePrefix = "rr" + lookupSubjectsPrefix cachePrefix = "ls" ) var cachePrefixes = []cachePrefix{checkViaRelationPrefix, checkViaCanonicalPrefix, lookupPrefix, expandPrefix, reachableResourcesPrefix} @@ -132,3 +146,16 @@ func ReachableResourcesRequestToKey(req *v1.DispatchReachableResourcesRequest) s req.Metadata.AtRevision, ) } + +// LookupSubjectsRequestToKey converts a lookup subjects request into a cache key +func LookupSubjectsRequestToKey(req *v1.DispatchLookupSubjectsRequest) string { + return fmt.Sprintf("%s//%s#%s@%s#%s:[%s]@%s", + lookupSubjectsPrefix, + req.ResourceRelation.Namespace, + req.ResourceRelation.Relation, + req.SubjectRelation.Namespace, + req.SubjectRelation.Relation, + strings.Join(req.ResourceIds, ","), + req.Metadata.AtRevision, + ) +} diff --git a/internal/dispatch/graph/graph.go b/internal/dispatch/graph/graph.go index c5d68805a3..d6b423ae91 100644 --- a/internal/dispatch/graph/graph.go +++ b/internal/dispatch/graph/graph.go @@ -31,6 +31,7 @@ func NewLocalOnlyDispatcher(concurrencyLimit uint16) dispatch.Dispatcher { d.expander = graph.NewConcurrentExpander(d) d.lookupHandler = graph.NewConcurrentLookup(d, d, concurrencyLimit) d.reachableResourcesHandler = graph.NewConcurrentReachableResources(d, concurrencyLimit) + d.lookupSubjectsHandler = graph.NewConcurrentLookupSubjects(d, concurrencyLimit) return d } @@ -42,12 +43,14 @@ func NewDispatcher(redispatcher dispatch.Dispatcher, concurrencyLimit uint16) di expander := graph.NewConcurrentExpander(redispatcher) lookupHandler := graph.NewConcurrentLookup(redispatcher, redispatcher, concurrencyLimit) reachableResourcesHandler := graph.NewConcurrentReachableResources(redispatcher, concurrencyLimit) + lookupSubjectsHandler := graph.NewConcurrentLookupSubjects(redispatcher, concurrencyLimit) return &localDispatcher{ checker: checker, expander: expander, lookupHandler: lookupHandler, reachableResourcesHandler: reachableResourcesHandler, + lookupSubjectsHandler: lookupSubjectsHandler, } } @@ -56,6 +59,7 @@ type localDispatcher struct { expander *graph.ConcurrentExpander lookupHandler *graph.ConcurrentLookup reachableResourcesHandler *graph.ConcurrentReachableResources + lookupSubjectsHandler *graph.ConcurrentLookupSubjects } func (ld *localDispatcher) loadNamespace(ctx context.Context, nsName string, revision decimal.Decimal) (*core.NamespaceDefinition, error) { @@ -281,6 +285,37 @@ func (ld *localDispatcher) DispatchReachableResources( return ld.reachableResourcesHandler.ReachableResources(validatedReq, wrappedStream) } +// DispatchLookupSubjects implements dispatch.LookupSubjects interface +func (ld *localDispatcher) DispatchLookupSubjects( + req *v1.DispatchLookupSubjectsRequest, + stream dispatch.LookupSubjectsStream, +) error { + ctx, span := tracer.Start(stream.Context(), "DispatchLookupSubjects", trace.WithAttributes( + attribute.Stringer("resource-type", stringableRelRef{req.ResourceRelation}), + attribute.Stringer("subject-type", stringableRelRef{req.SubjectRelation}), + attribute.StringSlice("resource-ids", req.ResourceIds), + )) + defer span.End() + + err := dispatch.CheckDepth(ctx, req) + if err != nil { + return err + } + + revision, err := decimal.NewFromString(req.Metadata.AtRevision) + if err != nil { + return err + } + + validatedReq := graph.ValidatedLookupSubjectsRequest{ + DispatchLookupSubjectsRequest: req, + Revision: revision, + } + + wrappedStream := dispatch.StreamWithContext(ctx, stream) + return ld.lookupSubjectsHandler.LookupSubjects(validatedReq, wrappedStream) +} + func (ld *localDispatcher) Close() error { return nil } diff --git a/internal/dispatch/graph/lookupsubjects_test.go b/internal/dispatch/graph/lookupsubjects_test.go new file mode 100644 index 0000000000..383ef3402a --- /dev/null +++ b/internal/dispatch/graph/lookupsubjects_test.go @@ -0,0 +1,216 @@ +package graph + +import ( + "context" + "fmt" + "sort" + "testing" + + v1_api "github.com/authzed/authzed-go/proto/authzed/api/v1" + + "github.com/rs/zerolog/log" + "github.com/shopspring/decimal" + "github.com/stretchr/testify/require" + + "github.com/authzed/spicedb/internal/datastore/memdb" + "github.com/authzed/spicedb/internal/dispatch" + datastoremw "github.com/authzed/spicedb/internal/middleware/datastore" + "github.com/authzed/spicedb/internal/testfixtures" + "github.com/authzed/spicedb/pkg/datastore" + v1 "github.com/authzed/spicedb/pkg/proto/dispatch/v1" +) + +func TestSimpleLookupSubjects(t *testing.T) { + testCases := []struct { + resourceType string + resourceID string + permission string + subjectType string + subjectRelation string + expectedSubjects []string + }{ + { + "document", + "masterplan", + "view", + "user", + "...", + []string{"auditor", "chief_financial_officer", "eng_lead", "legal", "owner", "product_manager", "vp_product"}, + }, + { + "document", + "masterplan", + "edit", + "user", + "...", + []string{"product_manager"}, + }, + { + "document", + "masterplan", + "view_and_edit", + "user", + "...", + []string{}, + }, + { + "document", + "specialplan", + "view", + "user", + "...", + []string{"multiroleguy"}, + }, + { + "document", + "specialplan", + "edit", + "user", + "...", + []string{"multiroleguy"}, + }, + { + "document", + "specialplan", + "viewer_and_editor", + "user", + "...", + []string{"multiroleguy", "missingrolegal"}, + }, + { + "document", + "specialplan", + "view_and_edit", + "user", + "...", + []string{"multiroleguy"}, + }, + { + "folder", + "company", + "view", + "user", + "...", + []string{"auditor", "legal", "owner"}, + }, + { + "folder", + "strategy", + "view", + "user", + "...", + []string{"auditor", "legal", "owner", "vp_product"}, + }, + { + "document", + "masterplan", + "parent", + "folder", + "...", + []string{"plans", "strategy"}, + }, + { + "document", + "masterplan", + "view", + "folder", + "...", + []string{}, + }, + } + + for _, tc := range testCases { + t.Run(fmt.Sprintf("simple-lookup-subjects:%s:%s:%s:%s:%s", tc.resourceType, tc.resourceID, tc.permission, tc.subjectType, tc.subjectRelation), func(t *testing.T) { + require := require.New(t) + + ctx, dis, revision := newLocalDispatcher(require) + stream := dispatch.NewCollectingDispatchStream[*v1.DispatchLookupSubjectsResponse](ctx) + + err := dis.DispatchLookupSubjects(&v1.DispatchLookupSubjectsRequest{ + ResourceRelation: RR(tc.resourceType, tc.permission), + ResourceIds: []string{tc.resourceID}, + SubjectRelation: RR(tc.subjectType, tc.subjectRelation), + Metadata: &v1.ResolverMeta{ + AtRevision: revision.String(), + DepthRemaining: 50, + }, + }, stream) + + require.NoError(err) + + foundSubjectIds := []string{} + for _, result := range stream.Results() { + foundSubjectIds = append(foundSubjectIds, result.FoundSubjectIds...) + } + + sort.Strings(foundSubjectIds) + sort.Strings(tc.expectedSubjects) + require.Equal(tc.expectedSubjects, foundSubjectIds) + + // Ensure every subject found has access. + for _, subjectId := range foundSubjectIds { + checkResult, err := dis.DispatchCheck(ctx, &v1.DispatchCheckRequest{ + ResourceAndRelation: ONR(tc.resourceType, tc.resourceID, tc.permission), + Subject: ONR(tc.subjectType, subjectId, tc.subjectRelation), + Metadata: &v1.ResolverMeta{ + AtRevision: revision.String(), + DepthRemaining: 50, + }, + }) + + require.NoError(err) + require.Equal(v1.DispatchCheckResponse_MEMBER, checkResult.Membership) + } + }) + } +} + +func TestLookupSubjectsMaxDepth(t *testing.T) { + require := require.New(t) + + rawDS, err := memdb.NewMemdbDatastore(0, 0, memdb.DisableGC) + require.NoError(err) + + ds, _ := testfixtures.StandardDatastoreWithSchema(rawDS, require) + + mutations := []*v1_api.RelationshipUpdate{{ + Operation: v1_api.RelationshipUpdate_OPERATION_CREATE, + Relationship: &v1_api.Relationship{ + Resource: &v1_api.ObjectReference{ + ObjectType: "folder", + ObjectId: "oops", + }, + Relation: "owner", + Subject: &v1_api.SubjectReference{ + Object: &v1_api.ObjectReference{ + ObjectType: "folder", + ObjectId: "oops", + }, + OptionalRelation: "owner", + }, + }, + }} + + ctx := log.Logger.WithContext(datastoremw.ContextWithHandle(context.Background())) + require.NoError(datastoremw.SetInContext(ctx, ds)) + + revision, err := ds.ReadWriteTx(ctx, func(ctx context.Context, rwt datastore.ReadWriteTransaction) error { + return rwt.WriteRelationships(mutations) + }) + require.NoError(err) + require.True(revision.GreaterThan(decimal.Zero)) + + dis := NewLocalOnlyDispatcher(10) + stream := dispatch.NewCollectingDispatchStream[*v1.DispatchLookupSubjectsResponse](ctx) + + err = dis.DispatchLookupSubjects(&v1.DispatchLookupSubjectsRequest{ + ResourceRelation: RR("folder", "owner"), + ResourceIds: []string{"oops"}, + SubjectRelation: RR("user", "..."), + Metadata: &v1.ResolverMeta{ + AtRevision: revision.String(), + DepthRemaining: 50, + }, + }, stream) + require.Error(err) +} diff --git a/internal/dispatch/remote/cluster.go b/internal/dispatch/remote/cluster.go index f07e484bf9..aa14b807e8 100644 --- a/internal/dispatch/remote/cluster.go +++ b/internal/dispatch/remote/cluster.go @@ -20,6 +20,7 @@ type clusterClient interface { DispatchExpand(ctx context.Context, req *v1.DispatchExpandRequest, opts ...grpc.CallOption) (*v1.DispatchExpandResponse, error) DispatchLookup(ctx context.Context, req *v1.DispatchLookupRequest, opts ...grpc.CallOption) (*v1.DispatchLookupResponse, error) DispatchReachableResources(ctx context.Context, in *v1.DispatchReachableResourcesRequest, opts ...grpc.CallOption) (v1.DispatchService_DispatchReachableResourcesClient, error) + DispatchLookupSubjects(ctx context.Context, in *v1.DispatchLookupSubjectsRequest, opts ...grpc.CallOption) (v1.DispatchService_DispatchLookupSubjectsClient, error) } // NewClusterDispatcher creates a dispatcher implementation that uses the provided client @@ -120,6 +121,40 @@ func (cr *clusterDispatcher) DispatchReachableResources( } } +func (cr *clusterDispatcher) DispatchLookupSubjects( + req *v1.DispatchLookupSubjectsRequest, + stream dispatch.LookupSubjectsStream, +) error { + ctx := context.WithValue(stream.Context(), balancer.CtxKey, []byte(dispatch.LookupSubjectsRequestToKey(req))) + stream = dispatch.StreamWithContext(ctx, stream) + + err := dispatch.CheckDepth(ctx, req) + if err != nil { + return err + } + + client, err := cr.clusterClient.DispatchLookupSubjects(ctx, req) + if err != nil { + return err + } + + for { + result, err := client.Recv() + if errors.Is(err, io.EOF) { + return nil + } + + if err != nil { + return err + } + + serr := stream.Publish(result) + if serr != nil { + return serr + } + } +} + func (cr *clusterDispatcher) Close() error { return nil } diff --git a/internal/dispatch/stream.go b/internal/dispatch/stream.go index 24f7b445cc..ebb4305fb3 100644 --- a/internal/dispatch/stream.go +++ b/internal/dispatch/stream.go @@ -82,7 +82,7 @@ func (s *CollectingDispatchStream[T]) Publish(result T) error { type WrappedDispatchStream[T any] struct { Stream Stream[T] Ctx context.Context - Processor func(result T) (T, error) + Processor func(result T) (T, bool, error) } func (s *WrappedDispatchStream[T]) Publish(result T) error { @@ -90,10 +90,14 @@ func (s *WrappedDispatchStream[T]) Publish(result T) error { return s.Stream.Publish(result) } - processed, err := s.Processor(result) + processed, ok, err := s.Processor(result) if err != nil { return err } + if !ok { + return nil + } + return s.Stream.Publish(processed) } diff --git a/internal/graph/lookupsubjects.go b/internal/graph/lookupsubjects.go new file mode 100644 index 0000000000..8278f155e9 --- /dev/null +++ b/internal/graph/lookupsubjects.go @@ -0,0 +1,498 @@ +package graph + +import ( + "context" + "errors" + "fmt" + "sync" + + "github.com/rs/zerolog/log" + "github.com/shopspring/decimal" + "golang.org/x/sync/errgroup" + + "github.com/authzed/spicedb/internal/dispatch" + datastoremw "github.com/authzed/spicedb/internal/middleware/datastore" + "github.com/authzed/spicedb/internal/namespace" + "github.com/authzed/spicedb/internal/util" + "github.com/authzed/spicedb/pkg/datastore" + core "github.com/authzed/spicedb/pkg/proto/core/v1" + v1 "github.com/authzed/spicedb/pkg/proto/dispatch/v1" + "github.com/authzed/spicedb/pkg/tuple" +) + +// ValidatedLookupSubjectsRequest represents a request after it has been validated and parsed for internal +// consumption. +type ValidatedLookupSubjectsRequest struct { + *v1.DispatchLookupSubjectsRequest + Revision decimal.Decimal +} + +// NewConcurrentLookupSubjects creates an instance of ConcurrentLookupSubjects. +func NewConcurrentLookupSubjects(d dispatch.LookupSubjects, concurrencyLimit uint16) *ConcurrentLookupSubjects { + return &ConcurrentLookupSubjects{d, concurrencyLimit} +} + +type ConcurrentLookupSubjects struct { + d dispatch.LookupSubjects + concurrencyLimit uint16 +} + +func (cl *ConcurrentLookupSubjects) LookupSubjects( + req ValidatedLookupSubjectsRequest, + stream dispatch.LookupSubjectsStream, +) error { + ctx := stream.Context() + + if len(req.ResourceIds) == 0 { + return fmt.Errorf("no resources ids given to lookupsubjects dispatch") + } + + // If the resource type matches the subject type, yield directly. + if req.SubjectRelation.Namespace == req.ResourceRelation.Namespace && + req.SubjectRelation.Relation == req.ResourceRelation.Relation { + err := stream.Publish(&v1.DispatchLookupSubjectsResponse{ + FoundSubjectIds: req.ResourceIds, + Metadata: emptyMetadata, + }) + if err != nil { + return err + } + } + + ds := datastoremw.MustFromContext(ctx) + reader := ds.SnapshotReader(req.Revision) + _, relation, err := namespace.ReadNamespaceAndRelation( + ctx, + req.ResourceRelation.Namespace, + req.ResourceRelation.Relation, + reader) + if err != nil { + return err + } + + if relation.UsersetRewrite == nil { + // Direct lookup of subjects. + return cl.lookupDirectSubjects(ctx, req, stream, relation, reader) + } + + return cl.lookupViaRewrite(ctx, req, stream, relation.UsersetRewrite) +} + +func (cl *ConcurrentLookupSubjects) lookupDirectSubjects( + ctx context.Context, + req ValidatedLookupSubjectsRequest, + stream dispatch.LookupSubjectsStream, + relation *core.Relation, + reader datastore.Reader, +) error { + // TODO(jschorr): use type information to skip subject relations that cannot reach the subject type. + it, err := reader.QueryRelationships(ctx, datastore.RelationshipsFilter{ + ResourceType: req.ResourceRelation.Namespace, + OptionalResourceRelation: req.ResourceRelation.Relation, + OptionalResourceIds: req.ResourceIds, + }) + if err != nil { + return err + } + defer it.Close() + + toDispatchByType := tuple.NewONRByTypeSet() + var foundSubjectIds []string + for tpl := it.Next(); tpl != nil; tpl = it.Next() { + if it.Err() != nil { + return err + } + + if tpl.Subject.Namespace == req.SubjectRelation.Namespace && + tpl.Subject.Relation == req.SubjectRelation.Relation { + foundSubjectIds = append(foundSubjectIds, tpl.Subject.ObjectId) + } + + if tpl.Subject.Relation != tuple.Ellipsis { + toDispatchByType.Add(tpl.Subject) + } + } + + if len(foundSubjectIds) > 0 { + err := stream.Publish(&v1.DispatchLookupSubjectsResponse{ + FoundSubjectIds: foundSubjectIds, + Metadata: emptyMetadata, + }) + if err != nil { + return err + } + } + + return cl.dispatchTo(ctx, req, toDispatchByType, stream) +} + +func (cl *ConcurrentLookupSubjects) lookupViaComputed( + ctx context.Context, + parentRequest ValidatedLookupSubjectsRequest, + parentStream dispatch.LookupSubjectsStream, + cu *core.ComputedUserset, +) error { + ds := datastoremw.MustFromContext(ctx).SnapshotReader(parentRequest.Revision) + err := namespace.CheckNamespaceAndRelation(ctx, parentRequest.ResourceRelation.Namespace, cu.Relation, true, ds) + if err != nil { + if errors.As(err, &namespace.ErrRelationNotFound{}) { + return nil + } + + return err + } + + stream := &dispatch.WrappedDispatchStream[*v1.DispatchLookupSubjectsResponse]{ + Stream: parentStream, + Ctx: ctx, + Processor: func(result *v1.DispatchLookupSubjectsResponse) (*v1.DispatchLookupSubjectsResponse, bool, error) { + return &v1.DispatchLookupSubjectsResponse{ + FoundSubjectIds: result.FoundSubjectIds, + Metadata: addCallToResponseMetadata(result.Metadata), + }, true, nil + }, + } + + return cl.d.DispatchLookupSubjects(&v1.DispatchLookupSubjectsRequest{ + ResourceRelation: &core.RelationReference{ + Namespace: parentRequest.ResourceRelation.Namespace, + Relation: cu.Relation, + }, + ResourceIds: parentRequest.ResourceIds, + SubjectRelation: parentRequest.SubjectRelation, + Metadata: &v1.ResolverMeta{ + AtRevision: parentRequest.Revision.String(), + DepthRemaining: parentRequest.Metadata.DepthRemaining - 1, + }, + }, stream) +} + +func (cl *ConcurrentLookupSubjects) lookupViaTupleToUserset( + ctx context.Context, + parentRequest ValidatedLookupSubjectsRequest, + parentStream dispatch.LookupSubjectsStream, + ttu *core.TupleToUserset, +) error { + ds := datastoremw.MustFromContext(ctx).SnapshotReader(parentRequest.Revision) + it, err := ds.QueryRelationships(ctx, datastore.RelationshipsFilter{ + ResourceType: parentRequest.ResourceRelation.Namespace, + OptionalResourceRelation: ttu.Tupleset.Relation, + OptionalResourceIds: parentRequest.ResourceIds, + }) + if err != nil { + return err + } + defer it.Close() + + toDispatchByTuplesetType := tuple.NewONRByTypeSet() + for tpl := it.Next(); tpl != nil; tpl = it.Next() { + if it.Err() != nil { + return err + } + + toDispatchByTuplesetType.Add(tpl.Subject) + } + + // Map the found subject types by the computed userset relation, so that we dispatch to it. + toDispatchByComputedRelationType, err := toDispatchByTuplesetType.Map(func(resourceType *core.RelationReference) (*core.RelationReference, error) { + err := namespace.CheckNamespaceAndRelation(ctx, resourceType.Namespace, ttu.ComputedUserset.Relation, false, ds) + if err != nil { + if errors.As(err, &namespace.ErrRelationNotFound{}) { + return nil, nil + } + + return nil, err + } + + return &core.RelationReference{ + Namespace: resourceType.Namespace, + Relation: ttu.ComputedUserset.Relation, + }, nil + }) + if err != nil { + return err + } + + return cl.dispatchTo(ctx, parentRequest, toDispatchByComputedRelationType, parentStream) +} + +func (cl *ConcurrentLookupSubjects) lookupViaRewrite( + ctx context.Context, + req ValidatedLookupSubjectsRequest, + stream dispatch.LookupSubjectsStream, + usr *core.UsersetRewrite, +) error { + switch rw := usr.RewriteOperation.(type) { + case *core.UsersetRewrite_Union: + log.Ctx(ctx).Trace().Msg("union") + return cl.lookupSetOperation(ctx, req, rw.Union, newLookupSubjectsUnion(stream)) + case *core.UsersetRewrite_Intersection: + log.Ctx(ctx).Trace().Msg("intersection") + return cl.lookupSetOperation(ctx, req, rw.Intersection, newLookupSubjectsIntersection(stream)) + case *core.UsersetRewrite_Exclusion: + log.Ctx(ctx).Trace().Msg("exclusion") + return cl.lookupSetOperation(ctx, req, rw.Exclusion, newLookupSubjectsExclusion(stream)) + default: + return fmt.Errorf("unknown kind of rewrite in lookup subjects") + } +} + +func (cl *ConcurrentLookupSubjects) lookupSetOperation( + ctx context.Context, + req ValidatedLookupSubjectsRequest, + so *core.SetOperation, + reducer lookupSubjectsReducer, +) error { + cancelCtx, checkCancel := context.WithCancel(ctx) + defer checkCancel() + + g, subCtx := errgroup.WithContext(cancelCtx) + g.SetLimit(int(cl.concurrencyLimit)) + + for index, childOneof := range so.Child { + stream := reducer.ForIndex(subCtx, index) + + switch child := childOneof.ChildType.(type) { + case *core.SetOperation_Child_XThis: + return errors.New("use of _this is unsupported; please rewrite your schema") + + case *core.SetOperation_Child_ComputedUserset: + g.Go(func() error { + return cl.lookupViaComputed(subCtx, req, stream, child.ComputedUserset) + }) + + case *core.SetOperation_Child_UsersetRewrite: + g.Go(func() error { + return cl.lookupViaRewrite(subCtx, req, stream, child.UsersetRewrite) + }) + + case *core.SetOperation_Child_TupleToUserset: + g.Go(func() error { + return cl.lookupViaTupleToUserset(subCtx, req, stream, child.TupleToUserset) + }) + + case *core.SetOperation_Child_XNil: + // Purposely do nothing. + continue + + default: + return fmt.Errorf("unknown set operation child `%T` in expand", child) + } + } + + // Wait for all dispatched operations to complete. + err := g.Wait() + if err != nil { + return err + } + + return reducer.CompletedChildOperations() +} + +const dispatchChunkSize = 100 + +func (cl *ConcurrentLookupSubjects) dispatchTo( + ctx context.Context, + parentRequest ValidatedLookupSubjectsRequest, + toDispatchByType *tuple.ONRByTypeSet, + parentStream dispatch.LookupSubjectsStream, +) error { + if toDispatchByType.IsEmpty() { + return nil + } + + cancelCtx, checkCancel := context.WithCancel(ctx) + defer checkCancel() + + g, subCtx := errgroup.WithContext(cancelCtx) + g.SetLimit(int(cl.concurrencyLimit)) + + stream := &dispatch.WrappedDispatchStream[*v1.DispatchLookupSubjectsResponse]{ + Stream: parentStream, + Ctx: subCtx, + Processor: func(result *v1.DispatchLookupSubjectsResponse) (*v1.DispatchLookupSubjectsResponse, bool, error) { + return &v1.DispatchLookupSubjectsResponse{ + FoundSubjectIds: result.FoundSubjectIds, + Metadata: addCallToResponseMetadata(result.Metadata), + }, true, nil + }, + } + + toDispatchByType.ForEachType(func(resourceType *core.RelationReference, resourceIds []string) { + chunkCount := (len(resourceIds) / dispatchChunkSize) + 1 + for chunkIndex := 0; chunkIndex < chunkCount; chunkIndex++ { + g.Go(func() error { + return cl.d.DispatchLookupSubjects(&v1.DispatchLookupSubjectsRequest{ + ResourceRelation: resourceType, + ResourceIds: resourceIds, + SubjectRelation: parentRequest.SubjectRelation, + Metadata: &v1.ResolverMeta{ + AtRevision: parentRequest.Revision.String(), + DepthRemaining: parentRequest.Metadata.DepthRemaining - 1, + }, + }, stream) + }) + } + }) + + return g.Wait() +} + +type lookupSubjectsReducer interface { + ForIndex(ctx context.Context, setOperationIndex int) dispatch.LookupSubjectsStream + CompletedChildOperations() error +} + +// Union +type lookupSubjectsUnion struct { + parentStream dispatch.LookupSubjectsStream + encountered map[string]struct{} + mu sync.Mutex +} + +func newLookupSubjectsUnion(parentStream dispatch.LookupSubjectsStream) *lookupSubjectsUnion { + return &lookupSubjectsUnion{ + parentStream: parentStream, + encountered: map[string]struct{}{}, + mu: sync.Mutex{}, + } +} + +func (lsu *lookupSubjectsUnion) ForIndex(ctx context.Context, setOperationIndex int) dispatch.LookupSubjectsStream { + return &dispatch.WrappedDispatchStream[*v1.DispatchLookupSubjectsResponse]{ + Stream: lsu.parentStream, + Ctx: ctx, + Processor: func(result *v1.DispatchLookupSubjectsResponse) (*v1.DispatchLookupSubjectsResponse, bool, error) { + lsu.mu.Lock() + defer lsu.mu.Unlock() + + filtered := make([]string, 0, len(result.FoundSubjectIds)) + for _, subjectId := range result.FoundSubjectIds { + _, ok := lsu.encountered[subjectId] + if !ok { + filtered = append(filtered, subjectId) + lsu.encountered[subjectId] = struct{}{} + } + } + + if len(filtered) == 0 { + return nil, false, nil + } + + return &v1.DispatchLookupSubjectsResponse{ + FoundSubjectIds: filtered, + Metadata: result.Metadata, + }, true, nil + }, + } +} + +func (lsu *lookupSubjectsUnion) CompletedChildOperations() error { + return nil +} + +// Intersection +type lookupSubjectsIntersection struct { + parentStream dispatch.LookupSubjectsStream + collectors map[int]*dispatch.CollectingDispatchStream[*v1.DispatchLookupSubjectsResponse] + mu sync.Mutex +} + +func newLookupSubjectsIntersection(parentStream dispatch.LookupSubjectsStream) *lookupSubjectsIntersection { + return &lookupSubjectsIntersection{ + parentStream: parentStream, + collectors: map[int]*dispatch.CollectingDispatchStream[*v1.DispatchLookupSubjectsResponse]{}, + mu: sync.Mutex{}, + } +} + +func (lsi *lookupSubjectsIntersection) ForIndex(ctx context.Context, setOperationIndex int) dispatch.LookupSubjectsStream { + lsi.mu.Lock() + defer lsi.mu.Unlock() + collector := dispatch.NewCollectingDispatchStream[*v1.DispatchLookupSubjectsResponse](ctx) + lsi.collectors[setOperationIndex] = collector + return collector +} + +func (lsi *lookupSubjectsIntersection) CompletedChildOperations() error { + var foundSubjectIds *util.Set[string] + metadata := emptyMetadata + + for index := 0; index < len(lsi.collectors); index++ { + collector, ok := lsi.collectors[index] + if !ok { + return fmt.Errorf("missing collector for index %d", index) + } + + results := util.NewSet[string]() + for _, result := range collector.Results() { + metadata = combineResponseMetadata(metadata, result.Metadata) + results.Extend(result.FoundSubjectIds) + } + + if index == 0 { + foundSubjectIds = results + } else { + foundSubjectIds.IntersectionDifference(results) + if foundSubjectIds.IsEmpty() { + return nil + } + } + } + + return lsi.parentStream.Publish(&v1.DispatchLookupSubjectsResponse{ + FoundSubjectIds: foundSubjectIds.AsSlice(), + Metadata: metadata, + }) +} + +// Exclusion +type lookupSubjectsExclusion struct { + parentStream dispatch.LookupSubjectsStream + collectors map[int]*dispatch.CollectingDispatchStream[*v1.DispatchLookupSubjectsResponse] + mu sync.Mutex +} + +func newLookupSubjectsExclusion(parentStream dispatch.LookupSubjectsStream) *lookupSubjectsExclusion { + return &lookupSubjectsExclusion{ + parentStream: parentStream, + collectors: map[int]*dispatch.CollectingDispatchStream[*v1.DispatchLookupSubjectsResponse]{}, + mu: sync.Mutex{}, + } +} + +func (lse *lookupSubjectsExclusion) ForIndex(ctx context.Context, setOperationIndex int) dispatch.LookupSubjectsStream { + lse.mu.Lock() + defer lse.mu.Unlock() + collector := dispatch.NewCollectingDispatchStream[*v1.DispatchLookupSubjectsResponse](ctx) + lse.collectors[setOperationIndex] = collector + return collector +} + +func (lse *lookupSubjectsExclusion) CompletedChildOperations() error { + var foundSubjectIds *util.Set[string] + metadata := emptyMetadata + + for index := 0; index < len(lse.collectors); index++ { + collector := lse.collectors[index] + results := util.NewSet[string]() + for _, result := range collector.Results() { + metadata = combineResponseMetadata(metadata, result.Metadata) + results.Extend(result.FoundSubjectIds) + } + + if index == 0 { + foundSubjectIds = results + } else { + foundSubjectIds.RemoveAll(results) + if foundSubjectIds.IsEmpty() { + return nil + } + } + } + + return lse.parentStream.Publish(&v1.DispatchLookupSubjectsResponse{ + FoundSubjectIds: foundSubjectIds.AsSlice(), + Metadata: metadata, + }) +} diff --git a/internal/graph/reachableresources.go b/internal/graph/reachableresources.go index a527db6a86..897d68b503 100644 --- a/internal/graph/reachableresources.go +++ b/internal/graph/reachableresources.go @@ -387,7 +387,7 @@ func (crr *ConcurrentReachableResources) redispatchOrReport( stream := &dispatch.WrappedDispatchStream[*v1.DispatchReachableResourcesResponse]{ Stream: parentStream, Ctx: ctx, - Processor: func(result *v1.DispatchReachableResourcesResponse) (*v1.DispatchReachableResourcesResponse, error) { + Processor: func(result *v1.DispatchReachableResourcesResponse) (*v1.DispatchReachableResourcesResponse, bool, error) { // If the entrypoint is not a direct result, then a check is required to determine // whether the resource actually has permission. status := result.Resource.ResultStatus @@ -401,7 +401,7 @@ func (crr *ConcurrentReachableResources) redispatchOrReport( ResultStatus: status, }, Metadata: addCallToResponseMetadata(result.Metadata), - }, nil + }, true, nil }, } diff --git a/pkg/proto/dispatch/v1/00_zerolog.go b/pkg/proto/dispatch/v1/00_zerolog.go index 46870e4aca..5ae3035655 100644 --- a/pkg/proto/dispatch/v1/00_zerolog.go +++ b/pkg/proto/dispatch/v1/00_zerolog.go @@ -55,6 +55,14 @@ func (lr *DispatchReachableResourcesRequest) MarshalZerologObject(e *zerolog.Eve e.Array("subject-ids", strArray(lr.SubjectIds)) } +// MarshalZerologObject implements zerolog object marshalling. +func (ls *DispatchLookupSubjectsRequest) MarshalZerologObject(e *zerolog.Event) { + e.Object("metadata", ls.Metadata) + e.Str("resource-type", fmt.Sprintf("%s#%s", ls.ResourceRelation.Namespace, ls.ResourceRelation.Relation)) + e.Str("subject-type", fmt.Sprintf("%s#%s", ls.SubjectRelation.Namespace, ls.SubjectRelation.Relation)) + e.Array("resource-ids", strArray(ls.ResourceIds)) +} + type strArray []string type onArray []*core.RelationReference @@ -86,6 +94,11 @@ func (cr *DispatchLookupResponse) MarshalZerologObject(e *zerolog.Event) { e.Object("metadata", cr.Metadata) } +// MarshalZerologObject implements zerolog object marshalling. +func (cs *DispatchLookupSubjectsResponse) MarshalZerologObject(e *zerolog.Event) { + e.Object("metadata", cs.Metadata) +} + // MarshalZerologObject implements zerolog object marshalling. func (cr *ResolverMeta) MarshalZerologObject(e *zerolog.Event) { e.Str("revision", cr.AtRevision) diff --git a/pkg/proto/dispatch/v1/dispatch.pb.go b/pkg/proto/dispatch/v1/dispatch.pb.go index 79fc1dc626..042b91558f 100644 --- a/pkg/proto/dispatch/v1/dispatch.pb.go +++ b/pkg/proto/dispatch/v1/dispatch.pb.go @@ -261,7 +261,7 @@ func (x CheckDebugTrace_RelationType) Number() protoreflect.EnumNumber { // Deprecated: Use CheckDebugTrace_RelationType.Descriptor instead. func (CheckDebugTrace_RelationType) EnumDescriptor() ([]byte, []int) { - return file_dispatch_v1_dispatch_proto_rawDescGZIP(), []int{12, 0} + return file_dispatch_v1_dispatch_proto_rawDescGZIP(), []int{14, 0} } type DispatchCheckRequest struct { @@ -839,6 +839,132 @@ func (x *DispatchReachableResourcesResponse) GetMetadata() *ResponseMeta { return nil } +type DispatchLookupSubjectsRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Metadata *ResolverMeta `protobuf:"bytes,1,opt,name=metadata,proto3" json:"metadata,omitempty"` + ResourceRelation *v1.RelationReference `protobuf:"bytes,2,opt,name=resource_relation,json=resourceRelation,proto3" json:"resource_relation,omitempty"` + ResourceIds []string `protobuf:"bytes,3,rep,name=resource_ids,json=resourceIds,proto3" json:"resource_ids,omitempty"` + SubjectRelation *v1.RelationReference `protobuf:"bytes,4,opt,name=subject_relation,json=subjectRelation,proto3" json:"subject_relation,omitempty"` +} + +func (x *DispatchLookupSubjectsRequest) Reset() { + *x = DispatchLookupSubjectsRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_dispatch_v1_dispatch_proto_msgTypes[9] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *DispatchLookupSubjectsRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*DispatchLookupSubjectsRequest) ProtoMessage() {} + +func (x *DispatchLookupSubjectsRequest) ProtoReflect() protoreflect.Message { + mi := &file_dispatch_v1_dispatch_proto_msgTypes[9] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use DispatchLookupSubjectsRequest.ProtoReflect.Descriptor instead. +func (*DispatchLookupSubjectsRequest) Descriptor() ([]byte, []int) { + return file_dispatch_v1_dispatch_proto_rawDescGZIP(), []int{9} +} + +func (x *DispatchLookupSubjectsRequest) GetMetadata() *ResolverMeta { + if x != nil { + return x.Metadata + } + return nil +} + +func (x *DispatchLookupSubjectsRequest) GetResourceRelation() *v1.RelationReference { + if x != nil { + return x.ResourceRelation + } + return nil +} + +func (x *DispatchLookupSubjectsRequest) GetResourceIds() []string { + if x != nil { + return x.ResourceIds + } + return nil +} + +func (x *DispatchLookupSubjectsRequest) GetSubjectRelation() *v1.RelationReference { + if x != nil { + return x.SubjectRelation + } + return nil +} + +type DispatchLookupSubjectsResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + FoundSubjectIds []string `protobuf:"bytes,1,rep,name=found_subject_ids,json=foundSubjectIds,proto3" json:"found_subject_ids,omitempty"` + Metadata *ResponseMeta `protobuf:"bytes,2,opt,name=metadata,proto3" json:"metadata,omitempty"` +} + +func (x *DispatchLookupSubjectsResponse) Reset() { + *x = DispatchLookupSubjectsResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_dispatch_v1_dispatch_proto_msgTypes[10] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *DispatchLookupSubjectsResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*DispatchLookupSubjectsResponse) ProtoMessage() {} + +func (x *DispatchLookupSubjectsResponse) ProtoReflect() protoreflect.Message { + mi := &file_dispatch_v1_dispatch_proto_msgTypes[10] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use DispatchLookupSubjectsResponse.ProtoReflect.Descriptor instead. +func (*DispatchLookupSubjectsResponse) Descriptor() ([]byte, []int) { + return file_dispatch_v1_dispatch_proto_rawDescGZIP(), []int{10} +} + +func (x *DispatchLookupSubjectsResponse) GetFoundSubjectIds() []string { + if x != nil { + return x.FoundSubjectIds + } + return nil +} + +func (x *DispatchLookupSubjectsResponse) GetMetadata() *ResponseMeta { + if x != nil { + return x.Metadata + } + return nil +} + type ResolverMeta struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -851,7 +977,7 @@ type ResolverMeta struct { func (x *ResolverMeta) Reset() { *x = ResolverMeta{} if protoimpl.UnsafeEnabled { - mi := &file_dispatch_v1_dispatch_proto_msgTypes[9] + mi := &file_dispatch_v1_dispatch_proto_msgTypes[11] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -864,7 +990,7 @@ func (x *ResolverMeta) String() string { func (*ResolverMeta) ProtoMessage() {} func (x *ResolverMeta) ProtoReflect() protoreflect.Message { - mi := &file_dispatch_v1_dispatch_proto_msgTypes[9] + mi := &file_dispatch_v1_dispatch_proto_msgTypes[11] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -877,7 +1003,7 @@ func (x *ResolverMeta) ProtoReflect() protoreflect.Message { // Deprecated: Use ResolverMeta.ProtoReflect.Descriptor instead. func (*ResolverMeta) Descriptor() ([]byte, []int) { - return file_dispatch_v1_dispatch_proto_rawDescGZIP(), []int{9} + return file_dispatch_v1_dispatch_proto_rawDescGZIP(), []int{11} } func (x *ResolverMeta) GetAtRevision() string { @@ -908,7 +1034,7 @@ type ResponseMeta struct { func (x *ResponseMeta) Reset() { *x = ResponseMeta{} if protoimpl.UnsafeEnabled { - mi := &file_dispatch_v1_dispatch_proto_msgTypes[10] + mi := &file_dispatch_v1_dispatch_proto_msgTypes[12] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -921,7 +1047,7 @@ func (x *ResponseMeta) String() string { func (*ResponseMeta) ProtoMessage() {} func (x *ResponseMeta) ProtoReflect() protoreflect.Message { - mi := &file_dispatch_v1_dispatch_proto_msgTypes[10] + mi := &file_dispatch_v1_dispatch_proto_msgTypes[12] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -934,7 +1060,7 @@ func (x *ResponseMeta) ProtoReflect() protoreflect.Message { // Deprecated: Use ResponseMeta.ProtoReflect.Descriptor instead. func (*ResponseMeta) Descriptor() ([]byte, []int) { - return file_dispatch_v1_dispatch_proto_rawDescGZIP(), []int{10} + return file_dispatch_v1_dispatch_proto_rawDescGZIP(), []int{12} } func (x *ResponseMeta) GetDispatchCount() uint32 { @@ -976,7 +1102,7 @@ type DebugInformation struct { func (x *DebugInformation) Reset() { *x = DebugInformation{} if protoimpl.UnsafeEnabled { - mi := &file_dispatch_v1_dispatch_proto_msgTypes[11] + mi := &file_dispatch_v1_dispatch_proto_msgTypes[13] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -989,7 +1115,7 @@ func (x *DebugInformation) String() string { func (*DebugInformation) ProtoMessage() {} func (x *DebugInformation) ProtoReflect() protoreflect.Message { - mi := &file_dispatch_v1_dispatch_proto_msgTypes[11] + mi := &file_dispatch_v1_dispatch_proto_msgTypes[13] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1002,7 +1128,7 @@ func (x *DebugInformation) ProtoReflect() protoreflect.Message { // Deprecated: Use DebugInformation.ProtoReflect.Descriptor instead. func (*DebugInformation) Descriptor() ([]byte, []int) { - return file_dispatch_v1_dispatch_proto_rawDescGZIP(), []int{11} + return file_dispatch_v1_dispatch_proto_rawDescGZIP(), []int{13} } func (x *DebugInformation) GetCheck() *CheckDebugTrace { @@ -1027,7 +1153,7 @@ type CheckDebugTrace struct { func (x *CheckDebugTrace) Reset() { *x = CheckDebugTrace{} if protoimpl.UnsafeEnabled { - mi := &file_dispatch_v1_dispatch_proto_msgTypes[12] + mi := &file_dispatch_v1_dispatch_proto_msgTypes[14] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1040,7 +1166,7 @@ func (x *CheckDebugTrace) String() string { func (*CheckDebugTrace) ProtoMessage() {} func (x *CheckDebugTrace) ProtoReflect() protoreflect.Message { - mi := &file_dispatch_v1_dispatch_proto_msgTypes[12] + mi := &file_dispatch_v1_dispatch_proto_msgTypes[14] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1053,7 +1179,7 @@ func (x *CheckDebugTrace) ProtoReflect() protoreflect.Message { // Deprecated: Use CheckDebugTrace.ProtoReflect.Descriptor instead. func (*CheckDebugTrace) Descriptor() ([]byte, []int) { - return file_dispatch_v1_dispatch_proto_rawDescGZIP(), []int{12} + return file_dispatch_v1_dispatch_proto_rawDescGZIP(), []int{14} } func (x *CheckDebugTrace) GetRequest() *DispatchCheckRequest { @@ -1240,6 +1366,33 @@ var file_dispatch_v1_dispatch_proto_rawDesc = []byte{ 0x63, 0x65, 0x12, 0x35, 0x0a, 0x08, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x64, 0x69, 0x73, 0x70, 0x61, 0x74, 0x63, 0x68, 0x2e, 0x76, 0x31, 0x2e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x4d, 0x65, 0x74, 0x61, 0x52, + 0x08, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x22, 0xa7, 0x02, 0x0a, 0x1d, 0x44, 0x69, + 0x73, 0x70, 0x61, 0x74, 0x63, 0x68, 0x4c, 0x6f, 0x6f, 0x6b, 0x75, 0x70, 0x53, 0x75, 0x62, 0x6a, + 0x65, 0x63, 0x74, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x3f, 0x0a, 0x08, 0x6d, + 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, + 0x64, 0x69, 0x73, 0x70, 0x61, 0x74, 0x63, 0x68, 0x2e, 0x76, 0x31, 0x2e, 0x52, 0x65, 0x73, 0x6f, + 0x6c, 0x76, 0x65, 0x72, 0x4d, 0x65, 0x74, 0x61, 0x42, 0x08, 0xfa, 0x42, 0x05, 0x8a, 0x01, 0x02, + 0x10, 0x01, 0x52, 0x08, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x12, 0x51, 0x0a, 0x11, + 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x72, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, + 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x76, + 0x31, 0x2e, 0x52, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x66, 0x65, 0x72, 0x65, + 0x6e, 0x63, 0x65, 0x42, 0x08, 0xfa, 0x42, 0x05, 0x8a, 0x01, 0x02, 0x10, 0x01, 0x52, 0x10, 0x72, + 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x52, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, + 0x21, 0x0a, 0x0c, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x69, 0x64, 0x73, 0x18, + 0x03, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0b, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x49, + 0x64, 0x73, 0x12, 0x4f, 0x0a, 0x10, 0x73, 0x75, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x5f, 0x72, 0x65, + 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x63, + 0x6f, 0x72, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x52, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, + 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65, 0x42, 0x08, 0xfa, 0x42, 0x05, 0x8a, 0x01, 0x02, + 0x10, 0x01, 0x52, 0x0f, 0x73, 0x75, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x52, 0x65, 0x6c, 0x61, 0x74, + 0x69, 0x6f, 0x6e, 0x22, 0x83, 0x01, 0x0a, 0x1e, 0x44, 0x69, 0x73, 0x70, 0x61, 0x74, 0x63, 0x68, + 0x4c, 0x6f, 0x6f, 0x6b, 0x75, 0x70, 0x53, 0x75, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x73, 0x52, 0x65, + 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x2a, 0x0a, 0x11, 0x66, 0x6f, 0x75, 0x6e, 0x64, 0x5f, + 0x73, 0x75, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x5f, 0x69, 0x64, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, + 0x09, 0x52, 0x0f, 0x66, 0x6f, 0x75, 0x6e, 0x64, 0x53, 0x75, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x49, + 0x64, 0x73, 0x12, 0x35, 0x0a, 0x08, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x18, 0x02, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x64, 0x69, 0x73, 0x70, 0x61, 0x74, 0x63, 0x68, 0x2e, + 0x76, 0x31, 0x2e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x4d, 0x65, 0x74, 0x61, 0x52, 0x08, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x22, 0x7d, 0x0a, 0x0c, 0x52, 0x65, 0x73, 0x6f, 0x6c, 0x76, 0x65, 0x72, 0x4d, 0x65, 0x74, 0x61, 0x12, 0x3b, 0x0a, 0x0b, 0x61, 0x74, 0x5f, 0x72, 0x65, 0x76, 0x69, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x42, 0x1a, @@ -1290,7 +1443,7 @@ var file_dispatch_v1_dispatch_proto_rawDesc = []byte{ 0x73, 0x22, 0x39, 0x0a, 0x0c, 0x52, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x79, 0x70, 0x65, 0x12, 0x0b, 0x0a, 0x07, 0x55, 0x4e, 0x4b, 0x4e, 0x4f, 0x57, 0x4e, 0x10, 0x00, 0x12, 0x0c, 0x0a, 0x08, 0x52, 0x45, 0x4c, 0x41, 0x54, 0x49, 0x4f, 0x4e, 0x10, 0x01, 0x12, 0x0e, 0x0a, 0x0a, - 0x50, 0x45, 0x52, 0x4d, 0x49, 0x53, 0x53, 0x49, 0x4f, 0x4e, 0x10, 0x02, 0x32, 0xa9, 0x03, 0x0a, + 0x50, 0x45, 0x52, 0x4d, 0x49, 0x53, 0x53, 0x49, 0x4f, 0x4e, 0x10, 0x02, 0x32, 0xa0, 0x04, 0x0a, 0x0f, 0x44, 0x69, 0x73, 0x70, 0x61, 0x74, 0x63, 0x68, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x12, 0x58, 0x0a, 0x0d, 0x44, 0x69, 0x73, 0x70, 0x61, 0x74, 0x63, 0x68, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x12, 0x21, 0x2e, 0x64, 0x69, 0x73, 0x70, 0x61, 0x74, 0x63, 0x68, 0x2e, 0x76, 0x31, 0x2e, @@ -1317,18 +1470,26 @@ var file_dispatch_v1_dispatch_proto_rawDesc = []byte{ 0x65, 0x73, 0x74, 0x1a, 0x2f, 0x2e, 0x64, 0x69, 0x73, 0x70, 0x61, 0x74, 0x63, 0x68, 0x2e, 0x76, 0x31, 0x2e, 0x44, 0x69, 0x73, 0x70, 0x61, 0x74, 0x63, 0x68, 0x52, 0x65, 0x61, 0x63, 0x68, 0x61, 0x62, 0x6c, 0x65, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, - 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x30, 0x01, 0x42, 0xaa, 0x01, 0x0a, 0x0f, 0x63, 0x6f, 0x6d, - 0x2e, 0x64, 0x69, 0x73, 0x70, 0x61, 0x74, 0x63, 0x68, 0x2e, 0x76, 0x31, 0x42, 0x0d, 0x44, 0x69, - 0x73, 0x70, 0x61, 0x74, 0x63, 0x68, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x50, 0x01, 0x5a, 0x3b, 0x67, - 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x61, 0x75, 0x74, 0x68, 0x7a, 0x65, - 0x64, 0x2f, 0x73, 0x70, 0x69, 0x63, 0x65, 0x64, 0x62, 0x2f, 0x70, 0x6b, 0x67, 0x2f, 0x70, 0x72, - 0x6f, 0x74, 0x6f, 0x2f, 0x64, 0x69, 0x73, 0x70, 0x61, 0x74, 0x63, 0x68, 0x2f, 0x76, 0x31, 0x3b, - 0x64, 0x69, 0x73, 0x70, 0x61, 0x74, 0x63, 0x68, 0x76, 0x31, 0xa2, 0x02, 0x03, 0x44, 0x58, 0x58, - 0xaa, 0x02, 0x0b, 0x44, 0x69, 0x73, 0x70, 0x61, 0x74, 0x63, 0x68, 0x2e, 0x56, 0x31, 0xca, 0x02, - 0x0b, 0x44, 0x69, 0x73, 0x70, 0x61, 0x74, 0x63, 0x68, 0x5c, 0x56, 0x31, 0xe2, 0x02, 0x17, 0x44, - 0x69, 0x73, 0x70, 0x61, 0x74, 0x63, 0x68, 0x5c, 0x56, 0x31, 0x5c, 0x47, 0x50, 0x42, 0x4d, 0x65, - 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0xea, 0x02, 0x0c, 0x44, 0x69, 0x73, 0x70, 0x61, 0x74, 0x63, - 0x68, 0x3a, 0x3a, 0x56, 0x31, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, + 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x30, 0x01, 0x12, 0x75, 0x0a, 0x16, 0x44, 0x69, 0x73, 0x70, + 0x61, 0x74, 0x63, 0x68, 0x4c, 0x6f, 0x6f, 0x6b, 0x75, 0x70, 0x53, 0x75, 0x62, 0x6a, 0x65, 0x63, + 0x74, 0x73, 0x12, 0x2a, 0x2e, 0x64, 0x69, 0x73, 0x70, 0x61, 0x74, 0x63, 0x68, 0x2e, 0x76, 0x31, + 0x2e, 0x44, 0x69, 0x73, 0x70, 0x61, 0x74, 0x63, 0x68, 0x4c, 0x6f, 0x6f, 0x6b, 0x75, 0x70, 0x53, + 0x75, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2b, + 0x2e, 0x64, 0x69, 0x73, 0x70, 0x61, 0x74, 0x63, 0x68, 0x2e, 0x76, 0x31, 0x2e, 0x44, 0x69, 0x73, + 0x70, 0x61, 0x74, 0x63, 0x68, 0x4c, 0x6f, 0x6f, 0x6b, 0x75, 0x70, 0x53, 0x75, 0x62, 0x6a, 0x65, + 0x63, 0x74, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x30, 0x01, 0x42, + 0xaa, 0x01, 0x0a, 0x0f, 0x63, 0x6f, 0x6d, 0x2e, 0x64, 0x69, 0x73, 0x70, 0x61, 0x74, 0x63, 0x68, + 0x2e, 0x76, 0x31, 0x42, 0x0d, 0x44, 0x69, 0x73, 0x70, 0x61, 0x74, 0x63, 0x68, 0x50, 0x72, 0x6f, + 0x74, 0x6f, 0x50, 0x01, 0x5a, 0x3b, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, + 0x2f, 0x61, 0x75, 0x74, 0x68, 0x7a, 0x65, 0x64, 0x2f, 0x73, 0x70, 0x69, 0x63, 0x65, 0x64, 0x62, + 0x2f, 0x70, 0x6b, 0x67, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x64, 0x69, 0x73, 0x70, 0x61, + 0x74, 0x63, 0x68, 0x2f, 0x76, 0x31, 0x3b, 0x64, 0x69, 0x73, 0x70, 0x61, 0x74, 0x63, 0x68, 0x76, + 0x31, 0xa2, 0x02, 0x03, 0x44, 0x58, 0x58, 0xaa, 0x02, 0x0b, 0x44, 0x69, 0x73, 0x70, 0x61, 0x74, + 0x63, 0x68, 0x2e, 0x56, 0x31, 0xca, 0x02, 0x0b, 0x44, 0x69, 0x73, 0x70, 0x61, 0x74, 0x63, 0x68, + 0x5c, 0x56, 0x31, 0xe2, 0x02, 0x17, 0x44, 0x69, 0x73, 0x70, 0x61, 0x74, 0x63, 0x68, 0x5c, 0x56, + 0x31, 0x5c, 0x47, 0x50, 0x42, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0xea, 0x02, 0x0c, + 0x44, 0x69, 0x73, 0x70, 0x61, 0x74, 0x63, 0x68, 0x3a, 0x3a, 0x56, 0x31, 0x62, 0x06, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x33, } var ( @@ -1344,7 +1505,7 @@ func file_dispatch_v1_dispatch_proto_rawDescGZIP() []byte { } var file_dispatch_v1_dispatch_proto_enumTypes = make([]protoimpl.EnumInfo, 5) -var file_dispatch_v1_dispatch_proto_msgTypes = make([]protoimpl.MessageInfo, 13) +var file_dispatch_v1_dispatch_proto_msgTypes = make([]protoimpl.MessageInfo, 15) var file_dispatch_v1_dispatch_proto_goTypes = []interface{}{ (DispatchCheckRequest_DebugSetting)(0), // 0: dispatch.v1.DispatchCheckRequest.DebugSetting (DispatchCheckResponse_Membership)(0), // 1: dispatch.v1.DispatchCheckResponse.Membership @@ -1360,57 +1521,65 @@ var file_dispatch_v1_dispatch_proto_goTypes = []interface{}{ (*DispatchReachableResourcesRequest)(nil), // 11: dispatch.v1.DispatchReachableResourcesRequest (*ReachableResource)(nil), // 12: dispatch.v1.ReachableResource (*DispatchReachableResourcesResponse)(nil), // 13: dispatch.v1.DispatchReachableResourcesResponse - (*ResolverMeta)(nil), // 14: dispatch.v1.ResolverMeta - (*ResponseMeta)(nil), // 15: dispatch.v1.ResponseMeta - (*DebugInformation)(nil), // 16: dispatch.v1.DebugInformation - (*CheckDebugTrace)(nil), // 17: dispatch.v1.CheckDebugTrace - (*v1.ObjectAndRelation)(nil), // 18: core.v1.ObjectAndRelation - (*v1.RelationTupleTreeNode)(nil), // 19: core.v1.RelationTupleTreeNode - (*v1.RelationReference)(nil), // 20: core.v1.RelationReference + (*DispatchLookupSubjectsRequest)(nil), // 14: dispatch.v1.DispatchLookupSubjectsRequest + (*DispatchLookupSubjectsResponse)(nil), // 15: dispatch.v1.DispatchLookupSubjectsResponse + (*ResolverMeta)(nil), // 16: dispatch.v1.ResolverMeta + (*ResponseMeta)(nil), // 17: dispatch.v1.ResponseMeta + (*DebugInformation)(nil), // 18: dispatch.v1.DebugInformation + (*CheckDebugTrace)(nil), // 19: dispatch.v1.CheckDebugTrace + (*v1.ObjectAndRelation)(nil), // 20: core.v1.ObjectAndRelation + (*v1.RelationTupleTreeNode)(nil), // 21: core.v1.RelationTupleTreeNode + (*v1.RelationReference)(nil), // 22: core.v1.RelationReference } var file_dispatch_v1_dispatch_proto_depIdxs = []int32{ - 14, // 0: dispatch.v1.DispatchCheckRequest.metadata:type_name -> dispatch.v1.ResolverMeta - 18, // 1: dispatch.v1.DispatchCheckRequest.resource_and_relation:type_name -> core.v1.ObjectAndRelation - 18, // 2: dispatch.v1.DispatchCheckRequest.subject:type_name -> core.v1.ObjectAndRelation + 16, // 0: dispatch.v1.DispatchCheckRequest.metadata:type_name -> dispatch.v1.ResolverMeta + 20, // 1: dispatch.v1.DispatchCheckRequest.resource_and_relation:type_name -> core.v1.ObjectAndRelation + 20, // 2: dispatch.v1.DispatchCheckRequest.subject:type_name -> core.v1.ObjectAndRelation 0, // 3: dispatch.v1.DispatchCheckRequest.debug:type_name -> dispatch.v1.DispatchCheckRequest.DebugSetting - 15, // 4: dispatch.v1.DispatchCheckResponse.metadata:type_name -> dispatch.v1.ResponseMeta + 17, // 4: dispatch.v1.DispatchCheckResponse.metadata:type_name -> dispatch.v1.ResponseMeta 1, // 5: dispatch.v1.DispatchCheckResponse.membership:type_name -> dispatch.v1.DispatchCheckResponse.Membership - 14, // 6: dispatch.v1.DispatchExpandRequest.metadata:type_name -> dispatch.v1.ResolverMeta - 18, // 7: dispatch.v1.DispatchExpandRequest.resource_and_relation:type_name -> core.v1.ObjectAndRelation + 16, // 6: dispatch.v1.DispatchExpandRequest.metadata:type_name -> dispatch.v1.ResolverMeta + 20, // 7: dispatch.v1.DispatchExpandRequest.resource_and_relation:type_name -> core.v1.ObjectAndRelation 2, // 8: dispatch.v1.DispatchExpandRequest.expansion_mode:type_name -> dispatch.v1.DispatchExpandRequest.ExpansionMode - 15, // 9: dispatch.v1.DispatchExpandResponse.metadata:type_name -> dispatch.v1.ResponseMeta - 19, // 10: dispatch.v1.DispatchExpandResponse.tree_node:type_name -> core.v1.RelationTupleTreeNode - 14, // 11: dispatch.v1.DispatchLookupRequest.metadata:type_name -> dispatch.v1.ResolverMeta - 20, // 12: dispatch.v1.DispatchLookupRequest.object_relation:type_name -> core.v1.RelationReference - 18, // 13: dispatch.v1.DispatchLookupRequest.subject:type_name -> core.v1.ObjectAndRelation - 20, // 14: dispatch.v1.DispatchLookupRequest.direct_stack:type_name -> core.v1.RelationReference - 20, // 15: dispatch.v1.DispatchLookupRequest.ttu_stack:type_name -> core.v1.RelationReference - 15, // 16: dispatch.v1.DispatchLookupResponse.metadata:type_name -> dispatch.v1.ResponseMeta - 18, // 17: dispatch.v1.DispatchLookupResponse.resolved_onrs:type_name -> core.v1.ObjectAndRelation - 14, // 18: dispatch.v1.DispatchReachableResourcesRequest.metadata:type_name -> dispatch.v1.ResolverMeta - 20, // 19: dispatch.v1.DispatchReachableResourcesRequest.resource_relation:type_name -> core.v1.RelationReference - 20, // 20: dispatch.v1.DispatchReachableResourcesRequest.subject_relation:type_name -> core.v1.RelationReference + 17, // 9: dispatch.v1.DispatchExpandResponse.metadata:type_name -> dispatch.v1.ResponseMeta + 21, // 10: dispatch.v1.DispatchExpandResponse.tree_node:type_name -> core.v1.RelationTupleTreeNode + 16, // 11: dispatch.v1.DispatchLookupRequest.metadata:type_name -> dispatch.v1.ResolverMeta + 22, // 12: dispatch.v1.DispatchLookupRequest.object_relation:type_name -> core.v1.RelationReference + 20, // 13: dispatch.v1.DispatchLookupRequest.subject:type_name -> core.v1.ObjectAndRelation + 22, // 14: dispatch.v1.DispatchLookupRequest.direct_stack:type_name -> core.v1.RelationReference + 22, // 15: dispatch.v1.DispatchLookupRequest.ttu_stack:type_name -> core.v1.RelationReference + 17, // 16: dispatch.v1.DispatchLookupResponse.metadata:type_name -> dispatch.v1.ResponseMeta + 20, // 17: dispatch.v1.DispatchLookupResponse.resolved_onrs:type_name -> core.v1.ObjectAndRelation + 16, // 18: dispatch.v1.DispatchReachableResourcesRequest.metadata:type_name -> dispatch.v1.ResolverMeta + 22, // 19: dispatch.v1.DispatchReachableResourcesRequest.resource_relation:type_name -> core.v1.RelationReference + 22, // 20: dispatch.v1.DispatchReachableResourcesRequest.subject_relation:type_name -> core.v1.RelationReference 3, // 21: dispatch.v1.ReachableResource.result_status:type_name -> dispatch.v1.ReachableResource.ResultStatus 12, // 22: dispatch.v1.DispatchReachableResourcesResponse.resource:type_name -> dispatch.v1.ReachableResource - 15, // 23: dispatch.v1.DispatchReachableResourcesResponse.metadata:type_name -> dispatch.v1.ResponseMeta - 16, // 24: dispatch.v1.ResponseMeta.debug_info:type_name -> dispatch.v1.DebugInformation - 17, // 25: dispatch.v1.DebugInformation.check:type_name -> dispatch.v1.CheckDebugTrace - 5, // 26: dispatch.v1.CheckDebugTrace.request:type_name -> dispatch.v1.DispatchCheckRequest - 4, // 27: dispatch.v1.CheckDebugTrace.resource_relation_type:type_name -> dispatch.v1.CheckDebugTrace.RelationType - 17, // 28: dispatch.v1.CheckDebugTrace.sub_problems:type_name -> dispatch.v1.CheckDebugTrace - 5, // 29: dispatch.v1.DispatchService.DispatchCheck:input_type -> dispatch.v1.DispatchCheckRequest - 7, // 30: dispatch.v1.DispatchService.DispatchExpand:input_type -> dispatch.v1.DispatchExpandRequest - 9, // 31: dispatch.v1.DispatchService.DispatchLookup:input_type -> dispatch.v1.DispatchLookupRequest - 11, // 32: dispatch.v1.DispatchService.DispatchReachableResources:input_type -> dispatch.v1.DispatchReachableResourcesRequest - 6, // 33: dispatch.v1.DispatchService.DispatchCheck:output_type -> dispatch.v1.DispatchCheckResponse - 8, // 34: dispatch.v1.DispatchService.DispatchExpand:output_type -> dispatch.v1.DispatchExpandResponse - 10, // 35: dispatch.v1.DispatchService.DispatchLookup:output_type -> dispatch.v1.DispatchLookupResponse - 13, // 36: dispatch.v1.DispatchService.DispatchReachableResources:output_type -> dispatch.v1.DispatchReachableResourcesResponse - 33, // [33:37] is the sub-list for method output_type - 29, // [29:33] is the sub-list for method input_type - 29, // [29:29] is the sub-list for extension type_name - 29, // [29:29] is the sub-list for extension extendee - 0, // [0:29] is the sub-list for field type_name + 17, // 23: dispatch.v1.DispatchReachableResourcesResponse.metadata:type_name -> dispatch.v1.ResponseMeta + 16, // 24: dispatch.v1.DispatchLookupSubjectsRequest.metadata:type_name -> dispatch.v1.ResolverMeta + 22, // 25: dispatch.v1.DispatchLookupSubjectsRequest.resource_relation:type_name -> core.v1.RelationReference + 22, // 26: dispatch.v1.DispatchLookupSubjectsRequest.subject_relation:type_name -> core.v1.RelationReference + 17, // 27: dispatch.v1.DispatchLookupSubjectsResponse.metadata:type_name -> dispatch.v1.ResponseMeta + 18, // 28: dispatch.v1.ResponseMeta.debug_info:type_name -> dispatch.v1.DebugInformation + 19, // 29: dispatch.v1.DebugInformation.check:type_name -> dispatch.v1.CheckDebugTrace + 5, // 30: dispatch.v1.CheckDebugTrace.request:type_name -> dispatch.v1.DispatchCheckRequest + 4, // 31: dispatch.v1.CheckDebugTrace.resource_relation_type:type_name -> dispatch.v1.CheckDebugTrace.RelationType + 19, // 32: dispatch.v1.CheckDebugTrace.sub_problems:type_name -> dispatch.v1.CheckDebugTrace + 5, // 33: dispatch.v1.DispatchService.DispatchCheck:input_type -> dispatch.v1.DispatchCheckRequest + 7, // 34: dispatch.v1.DispatchService.DispatchExpand:input_type -> dispatch.v1.DispatchExpandRequest + 9, // 35: dispatch.v1.DispatchService.DispatchLookup:input_type -> dispatch.v1.DispatchLookupRequest + 11, // 36: dispatch.v1.DispatchService.DispatchReachableResources:input_type -> dispatch.v1.DispatchReachableResourcesRequest + 14, // 37: dispatch.v1.DispatchService.DispatchLookupSubjects:input_type -> dispatch.v1.DispatchLookupSubjectsRequest + 6, // 38: dispatch.v1.DispatchService.DispatchCheck:output_type -> dispatch.v1.DispatchCheckResponse + 8, // 39: dispatch.v1.DispatchService.DispatchExpand:output_type -> dispatch.v1.DispatchExpandResponse + 10, // 40: dispatch.v1.DispatchService.DispatchLookup:output_type -> dispatch.v1.DispatchLookupResponse + 13, // 41: dispatch.v1.DispatchService.DispatchReachableResources:output_type -> dispatch.v1.DispatchReachableResourcesResponse + 15, // 42: dispatch.v1.DispatchService.DispatchLookupSubjects:output_type -> dispatch.v1.DispatchLookupSubjectsResponse + 38, // [38:43] is the sub-list for method output_type + 33, // [33:38] is the sub-list for method input_type + 33, // [33:33] is the sub-list for extension type_name + 33, // [33:33] is the sub-list for extension extendee + 0, // [0:33] is the sub-list for field type_name } func init() { file_dispatch_v1_dispatch_proto_init() } @@ -1528,7 +1697,7 @@ func file_dispatch_v1_dispatch_proto_init() { } } file_dispatch_v1_dispatch_proto_msgTypes[9].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ResolverMeta); i { + switch v := v.(*DispatchLookupSubjectsRequest); i { case 0: return &v.state case 1: @@ -1540,7 +1709,7 @@ func file_dispatch_v1_dispatch_proto_init() { } } file_dispatch_v1_dispatch_proto_msgTypes[10].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ResponseMeta); i { + switch v := v.(*DispatchLookupSubjectsResponse); i { case 0: return &v.state case 1: @@ -1552,7 +1721,7 @@ func file_dispatch_v1_dispatch_proto_init() { } } file_dispatch_v1_dispatch_proto_msgTypes[11].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*DebugInformation); i { + switch v := v.(*ResolverMeta); i { case 0: return &v.state case 1: @@ -1564,6 +1733,30 @@ func file_dispatch_v1_dispatch_proto_init() { } } file_dispatch_v1_dispatch_proto_msgTypes[12].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ResponseMeta); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_dispatch_v1_dispatch_proto_msgTypes[13].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*DebugInformation); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_dispatch_v1_dispatch_proto_msgTypes[14].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*CheckDebugTrace); i { case 0: return &v.state @@ -1582,7 +1775,7 @@ func file_dispatch_v1_dispatch_proto_init() { GoPackagePath: reflect.TypeOf(x{}).PkgPath(), RawDescriptor: file_dispatch_v1_dispatch_proto_rawDesc, NumEnums: 5, - NumMessages: 13, + NumMessages: 15, NumExtensions: 0, NumServices: 1, }, diff --git a/pkg/proto/dispatch/v1/dispatch.pb.validate.go b/pkg/proto/dispatch/v1/dispatch.pb.validate.go index d3b32f06c6..89791f4a68 100644 --- a/pkg/proto/dispatch/v1/dispatch.pb.validate.go +++ b/pkg/proto/dispatch/v1/dispatch.pb.validate.go @@ -1687,6 +1687,361 @@ var _ interface { ErrorName() string } = DispatchReachableResourcesResponseValidationError{} +// Validate checks the field values on DispatchLookupSubjectsRequest with the +// rules defined in the proto definition for this message. If any rules are +// violated, the first error encountered is returned, or nil if there are no violations. +func (m *DispatchLookupSubjectsRequest) Validate() error { + return m.validate(false) +} + +// ValidateAll checks the field values on DispatchLookupSubjectsRequest with +// the rules defined in the proto definition for this message. If any rules +// are violated, the result is a list of violation errors wrapped in +// DispatchLookupSubjectsRequestMultiError, or nil if none found. +func (m *DispatchLookupSubjectsRequest) ValidateAll() error { + return m.validate(true) +} + +func (m *DispatchLookupSubjectsRequest) validate(all bool) error { + if m == nil { + return nil + } + + var errors []error + + if m.GetMetadata() == nil { + err := DispatchLookupSubjectsRequestValidationError{ + field: "Metadata", + reason: "value is required", + } + if !all { + return err + } + errors = append(errors, err) + } + + if all { + switch v := interface{}(m.GetMetadata()).(type) { + case interface{ ValidateAll() error }: + if err := v.ValidateAll(); err != nil { + errors = append(errors, DispatchLookupSubjectsRequestValidationError{ + field: "Metadata", + reason: "embedded message failed validation", + cause: err, + }) + } + case interface{ Validate() error }: + if err := v.Validate(); err != nil { + errors = append(errors, DispatchLookupSubjectsRequestValidationError{ + field: "Metadata", + reason: "embedded message failed validation", + cause: err, + }) + } + } + } else if v, ok := interface{}(m.GetMetadata()).(interface{ Validate() error }); ok { + if err := v.Validate(); err != nil { + return DispatchLookupSubjectsRequestValidationError{ + field: "Metadata", + reason: "embedded message failed validation", + cause: err, + } + } + } + + if m.GetResourceRelation() == nil { + err := DispatchLookupSubjectsRequestValidationError{ + field: "ResourceRelation", + reason: "value is required", + } + if !all { + return err + } + errors = append(errors, err) + } + + if all { + switch v := interface{}(m.GetResourceRelation()).(type) { + case interface{ ValidateAll() error }: + if err := v.ValidateAll(); err != nil { + errors = append(errors, DispatchLookupSubjectsRequestValidationError{ + field: "ResourceRelation", + reason: "embedded message failed validation", + cause: err, + }) + } + case interface{ Validate() error }: + if err := v.Validate(); err != nil { + errors = append(errors, DispatchLookupSubjectsRequestValidationError{ + field: "ResourceRelation", + reason: "embedded message failed validation", + cause: err, + }) + } + } + } else if v, ok := interface{}(m.GetResourceRelation()).(interface{ Validate() error }); ok { + if err := v.Validate(); err != nil { + return DispatchLookupSubjectsRequestValidationError{ + field: "ResourceRelation", + reason: "embedded message failed validation", + cause: err, + } + } + } + + if m.GetSubjectRelation() == nil { + err := DispatchLookupSubjectsRequestValidationError{ + field: "SubjectRelation", + reason: "value is required", + } + if !all { + return err + } + errors = append(errors, err) + } + + if all { + switch v := interface{}(m.GetSubjectRelation()).(type) { + case interface{ ValidateAll() error }: + if err := v.ValidateAll(); err != nil { + errors = append(errors, DispatchLookupSubjectsRequestValidationError{ + field: "SubjectRelation", + reason: "embedded message failed validation", + cause: err, + }) + } + case interface{ Validate() error }: + if err := v.Validate(); err != nil { + errors = append(errors, DispatchLookupSubjectsRequestValidationError{ + field: "SubjectRelation", + reason: "embedded message failed validation", + cause: err, + }) + } + } + } else if v, ok := interface{}(m.GetSubjectRelation()).(interface{ Validate() error }); ok { + if err := v.Validate(); err != nil { + return DispatchLookupSubjectsRequestValidationError{ + field: "SubjectRelation", + reason: "embedded message failed validation", + cause: err, + } + } + } + + if len(errors) > 0 { + return DispatchLookupSubjectsRequestMultiError(errors) + } + + return nil +} + +// DispatchLookupSubjectsRequestMultiError is an error wrapping multiple +// validation errors returned by DispatchLookupSubjectsRequest.ValidateAll() +// if the designated constraints aren't met. +type DispatchLookupSubjectsRequestMultiError []error + +// Error returns a concatenation of all the error messages it wraps. +func (m DispatchLookupSubjectsRequestMultiError) Error() string { + var msgs []string + for _, err := range m { + msgs = append(msgs, err.Error()) + } + return strings.Join(msgs, "; ") +} + +// AllErrors returns a list of validation violation errors. +func (m DispatchLookupSubjectsRequestMultiError) AllErrors() []error { return m } + +// DispatchLookupSubjectsRequestValidationError is the validation error +// returned by DispatchLookupSubjectsRequest.Validate if the designated +// constraints aren't met. +type DispatchLookupSubjectsRequestValidationError struct { + field string + reason string + cause error + key bool +} + +// Field function returns field value. +func (e DispatchLookupSubjectsRequestValidationError) Field() string { return e.field } + +// Reason function returns reason value. +func (e DispatchLookupSubjectsRequestValidationError) Reason() string { return e.reason } + +// Cause function returns cause value. +func (e DispatchLookupSubjectsRequestValidationError) Cause() error { return e.cause } + +// Key function returns key value. +func (e DispatchLookupSubjectsRequestValidationError) Key() bool { return e.key } + +// ErrorName returns error name. +func (e DispatchLookupSubjectsRequestValidationError) ErrorName() string { + return "DispatchLookupSubjectsRequestValidationError" +} + +// Error satisfies the builtin error interface +func (e DispatchLookupSubjectsRequestValidationError) Error() string { + cause := "" + if e.cause != nil { + cause = fmt.Sprintf(" | caused by: %v", e.cause) + } + + key := "" + if e.key { + key = "key for " + } + + return fmt.Sprintf( + "invalid %sDispatchLookupSubjectsRequest.%s: %s%s", + key, + e.field, + e.reason, + cause) +} + +var _ error = DispatchLookupSubjectsRequestValidationError{} + +var _ interface { + Field() string + Reason() string + Key() bool + Cause() error + ErrorName() string +} = DispatchLookupSubjectsRequestValidationError{} + +// Validate checks the field values on DispatchLookupSubjectsResponse with the +// rules defined in the proto definition for this message. If any rules are +// violated, the first error encountered is returned, or nil if there are no violations. +func (m *DispatchLookupSubjectsResponse) Validate() error { + return m.validate(false) +} + +// ValidateAll checks the field values on DispatchLookupSubjectsResponse with +// the rules defined in the proto definition for this message. If any rules +// are violated, the result is a list of violation errors wrapped in +// DispatchLookupSubjectsResponseMultiError, or nil if none found. +func (m *DispatchLookupSubjectsResponse) ValidateAll() error { + return m.validate(true) +} + +func (m *DispatchLookupSubjectsResponse) validate(all bool) error { + if m == nil { + return nil + } + + var errors []error + + if all { + switch v := interface{}(m.GetMetadata()).(type) { + case interface{ ValidateAll() error }: + if err := v.ValidateAll(); err != nil { + errors = append(errors, DispatchLookupSubjectsResponseValidationError{ + field: "Metadata", + reason: "embedded message failed validation", + cause: err, + }) + } + case interface{ Validate() error }: + if err := v.Validate(); err != nil { + errors = append(errors, DispatchLookupSubjectsResponseValidationError{ + field: "Metadata", + reason: "embedded message failed validation", + cause: err, + }) + } + } + } else if v, ok := interface{}(m.GetMetadata()).(interface{ Validate() error }); ok { + if err := v.Validate(); err != nil { + return DispatchLookupSubjectsResponseValidationError{ + field: "Metadata", + reason: "embedded message failed validation", + cause: err, + } + } + } + + if len(errors) > 0 { + return DispatchLookupSubjectsResponseMultiError(errors) + } + + return nil +} + +// DispatchLookupSubjectsResponseMultiError is an error wrapping multiple +// validation errors returned by DispatchLookupSubjectsResponse.ValidateAll() +// if the designated constraints aren't met. +type DispatchLookupSubjectsResponseMultiError []error + +// Error returns a concatenation of all the error messages it wraps. +func (m DispatchLookupSubjectsResponseMultiError) Error() string { + var msgs []string + for _, err := range m { + msgs = append(msgs, err.Error()) + } + return strings.Join(msgs, "; ") +} + +// AllErrors returns a list of validation violation errors. +func (m DispatchLookupSubjectsResponseMultiError) AllErrors() []error { return m } + +// DispatchLookupSubjectsResponseValidationError is the validation error +// returned by DispatchLookupSubjectsResponse.Validate if the designated +// constraints aren't met. +type DispatchLookupSubjectsResponseValidationError struct { + field string + reason string + cause error + key bool +} + +// Field function returns field value. +func (e DispatchLookupSubjectsResponseValidationError) Field() string { return e.field } + +// Reason function returns reason value. +func (e DispatchLookupSubjectsResponseValidationError) Reason() string { return e.reason } + +// Cause function returns cause value. +func (e DispatchLookupSubjectsResponseValidationError) Cause() error { return e.cause } + +// Key function returns key value. +func (e DispatchLookupSubjectsResponseValidationError) Key() bool { return e.key } + +// ErrorName returns error name. +func (e DispatchLookupSubjectsResponseValidationError) ErrorName() string { + return "DispatchLookupSubjectsResponseValidationError" +} + +// Error satisfies the builtin error interface +func (e DispatchLookupSubjectsResponseValidationError) Error() string { + cause := "" + if e.cause != nil { + cause = fmt.Sprintf(" | caused by: %v", e.cause) + } + + key := "" + if e.key { + key = "key for " + } + + return fmt.Sprintf( + "invalid %sDispatchLookupSubjectsResponse.%s: %s%s", + key, + e.field, + e.reason, + cause) +} + +var _ error = DispatchLookupSubjectsResponseValidationError{} + +var _ interface { + Field() string + Reason() string + Key() bool + Cause() error + ErrorName() string +} = DispatchLookupSubjectsResponseValidationError{} + // Validate checks the field values on ResolverMeta with the rules defined in // the proto definition for this message. If any rules are violated, the first // error encountered is returned, or nil if there are no violations. diff --git a/pkg/proto/dispatch/v1/dispatch_grpc.pb.go b/pkg/proto/dispatch/v1/dispatch_grpc.pb.go index f55da1c6ef..2648a5e631 100644 --- a/pkg/proto/dispatch/v1/dispatch_grpc.pb.go +++ b/pkg/proto/dispatch/v1/dispatch_grpc.pb.go @@ -26,6 +26,7 @@ type DispatchServiceClient interface { DispatchExpand(ctx context.Context, in *DispatchExpandRequest, opts ...grpc.CallOption) (*DispatchExpandResponse, error) DispatchLookup(ctx context.Context, in *DispatchLookupRequest, opts ...grpc.CallOption) (*DispatchLookupResponse, error) DispatchReachableResources(ctx context.Context, in *DispatchReachableResourcesRequest, opts ...grpc.CallOption) (DispatchService_DispatchReachableResourcesClient, error) + DispatchLookupSubjects(ctx context.Context, in *DispatchLookupSubjectsRequest, opts ...grpc.CallOption) (DispatchService_DispatchLookupSubjectsClient, error) } type dispatchServiceClient struct { @@ -95,6 +96,38 @@ func (x *dispatchServiceDispatchReachableResourcesClient) Recv() (*DispatchReach return m, nil } +func (c *dispatchServiceClient) DispatchLookupSubjects(ctx context.Context, in *DispatchLookupSubjectsRequest, opts ...grpc.CallOption) (DispatchService_DispatchLookupSubjectsClient, error) { + stream, err := c.cc.NewStream(ctx, &DispatchService_ServiceDesc.Streams[1], "/dispatch.v1.DispatchService/DispatchLookupSubjects", opts...) + if err != nil { + return nil, err + } + x := &dispatchServiceDispatchLookupSubjectsClient{stream} + if err := x.ClientStream.SendMsg(in); err != nil { + return nil, err + } + if err := x.ClientStream.CloseSend(); err != nil { + return nil, err + } + return x, nil +} + +type DispatchService_DispatchLookupSubjectsClient interface { + Recv() (*DispatchLookupSubjectsResponse, error) + grpc.ClientStream +} + +type dispatchServiceDispatchLookupSubjectsClient struct { + grpc.ClientStream +} + +func (x *dispatchServiceDispatchLookupSubjectsClient) Recv() (*DispatchLookupSubjectsResponse, error) { + m := new(DispatchLookupSubjectsResponse) + if err := x.ClientStream.RecvMsg(m); err != nil { + return nil, err + } + return m, nil +} + // DispatchServiceServer is the server API for DispatchService service. // All implementations must embed UnimplementedDispatchServiceServer // for forward compatibility @@ -103,6 +136,7 @@ type DispatchServiceServer interface { DispatchExpand(context.Context, *DispatchExpandRequest) (*DispatchExpandResponse, error) DispatchLookup(context.Context, *DispatchLookupRequest) (*DispatchLookupResponse, error) DispatchReachableResources(*DispatchReachableResourcesRequest, DispatchService_DispatchReachableResourcesServer) error + DispatchLookupSubjects(*DispatchLookupSubjectsRequest, DispatchService_DispatchLookupSubjectsServer) error mustEmbedUnimplementedDispatchServiceServer() } @@ -122,6 +156,9 @@ func (UnimplementedDispatchServiceServer) DispatchLookup(context.Context, *Dispa func (UnimplementedDispatchServiceServer) DispatchReachableResources(*DispatchReachableResourcesRequest, DispatchService_DispatchReachableResourcesServer) error { return status.Errorf(codes.Unimplemented, "method DispatchReachableResources not implemented") } +func (UnimplementedDispatchServiceServer) DispatchLookupSubjects(*DispatchLookupSubjectsRequest, DispatchService_DispatchLookupSubjectsServer) error { + return status.Errorf(codes.Unimplemented, "method DispatchLookupSubjects not implemented") +} func (UnimplementedDispatchServiceServer) mustEmbedUnimplementedDispatchServiceServer() {} // UnsafeDispatchServiceServer may be embedded to opt out of forward compatibility for this service. @@ -210,6 +247,27 @@ func (x *dispatchServiceDispatchReachableResourcesServer) Send(m *DispatchReacha return x.ServerStream.SendMsg(m) } +func _DispatchService_DispatchLookupSubjects_Handler(srv interface{}, stream grpc.ServerStream) error { + m := new(DispatchLookupSubjectsRequest) + if err := stream.RecvMsg(m); err != nil { + return err + } + return srv.(DispatchServiceServer).DispatchLookupSubjects(m, &dispatchServiceDispatchLookupSubjectsServer{stream}) +} + +type DispatchService_DispatchLookupSubjectsServer interface { + Send(*DispatchLookupSubjectsResponse) error + grpc.ServerStream +} + +type dispatchServiceDispatchLookupSubjectsServer struct { + grpc.ServerStream +} + +func (x *dispatchServiceDispatchLookupSubjectsServer) Send(m *DispatchLookupSubjectsResponse) error { + return x.ServerStream.SendMsg(m) +} + // DispatchService_ServiceDesc is the grpc.ServiceDesc for DispatchService service. // It's only intended for direct use with grpc.RegisterService, // and not to be introspected or modified (even as a copy) @@ -236,6 +294,11 @@ var DispatchService_ServiceDesc = grpc.ServiceDesc{ Handler: _DispatchService_DispatchReachableResources_Handler, ServerStreams: true, }, + { + StreamName: "DispatchLookupSubjects", + Handler: _DispatchService_DispatchLookupSubjects_Handler, + ServerStreams: true, + }, }, Metadata: "dispatch/v1/dispatch.proto", } diff --git a/proto/internal/dispatch/v1/dispatch.proto b/proto/internal/dispatch/v1/dispatch.proto index 0c9f303217..b9df367033 100644 --- a/proto/internal/dispatch/v1/dispatch.proto +++ b/proto/internal/dispatch/v1/dispatch.proto @@ -11,6 +11,7 @@ service DispatchService { rpc DispatchExpand(DispatchExpandRequest) returns (DispatchExpandResponse) {} rpc DispatchLookup(DispatchLookupRequest) returns (DispatchLookupResponse) {} rpc DispatchReachableResources(DispatchReachableResourcesRequest) returns (stream DispatchReachableResourcesResponse) {} + rpc DispatchLookupSubjects(DispatchLookupSubjectsRequest) returns (stream DispatchLookupSubjectsResponse) {} } message DispatchCheckRequest { @@ -111,6 +112,22 @@ message DispatchReachableResourcesResponse { ResponseMeta metadata = 2; } +message DispatchLookupSubjectsRequest { + ResolverMeta metadata = 1 [ (validate.rules).message.required = true ]; + + core.v1.RelationReference resource_relation = 2 + [ (validate.rules).message.required = true ]; + repeated string resource_ids = 3; + + core.v1.RelationReference subject_relation = 4 + [ (validate.rules).message.required = true ]; +} + +message DispatchLookupSubjectsResponse { + repeated string found_subject_ids = 1; + ResponseMeta metadata = 2; +} + message ResolverMeta { string at_revision = 1 [ (validate.rules).string = { pattern : "^[0-9]+(\\.[0-9]+)?$",