diff --git a/cmd/bb_replicator/main.go b/cmd/bb_replicator/main.go index bd53eac9..e10310a8 100644 --- a/cmd/bb_replicator/main.go +++ b/cmd/bb_replicator/main.go @@ -65,11 +65,12 @@ func main() { replicator_pb.RegisterReplicatorServer(s, replication.NewReplicatorServer(replicator)) }, siblingsGroup, + grpcClientFactory, ); err != nil { return util.StatusWrap(err, "gRPC server failure") } - lifecycleState.MarkReadyAndWait(siblingsGroup) + lifecycleState.MarkReadyAndWait(siblingsGroup, grpcClientFactory) return nil }) } diff --git a/cmd/bb_storage/BUILD.bazel b/cmd/bb_storage/BUILD.bazel index beee4d44..89618a94 100644 --- a/cmd/bb_storage/BUILD.bazel +++ b/cmd/bb_storage/BUILD.bazel @@ -8,6 +8,7 @@ go_library( visibility = ["//visibility:private"], deps = [ "//pkg/auth", + "//pkg/auth/configuration", "//pkg/blobstore", "//pkg/blobstore/configuration", "//pkg/blobstore/grpcservers", diff --git a/cmd/bb_storage/main.go b/cmd/bb_storage/main.go index ed8045a5..0347f722 100644 --- a/cmd/bb_storage/main.go +++ b/cmd/bb_storage/main.go @@ -6,6 +6,7 @@ import ( remoteexecution "github.com/bazelbuild/remote-apis/build/bazel/remote/execution/v2" "github.com/buildbarn/bb-storage/pkg/auth" + auth_configuration "github.com/buildbarn/bb-storage/pkg/auth/configuration" "github.com/buildbarn/bb-storage/pkg/blobstore" blobstore_configuration "github.com/buildbarn/bb-storage/pkg/blobstore/configuration" "github.com/buildbarn/bb-storage/pkg/blobstore/grpcservers" @@ -56,7 +57,8 @@ func main() { configuration.ContentAddressableStorage, blobstore_configuration.NewCASBlobAccessCreator( grpcClientFactory, - int(configuration.MaximumMessageSizeBytes))) + int(configuration.MaximumMessageSizeBytes)), + grpcClientFactory) if err != nil { return util.StatusWrap(err, "Failed to create Content Addressable Storage") } @@ -75,7 +77,8 @@ func main() { blobstore_configuration.NewACBlobAccessCreator( contentAddressableStorageInfo, grpcClientFactory, - int(configuration.MaximumMessageSizeBytes))) + int(configuration.MaximumMessageSizeBytes)), + grpcClientFactory) if err != nil { return util.StatusWrap(err, "Failed to create Action Cache") } @@ -94,7 +97,8 @@ func main() { configuration.IndirectContentAddressableStorage, blobstore_configuration.NewICASBlobAccessCreator( grpcClientFactory, - int(configuration.MaximumMessageSizeBytes))) + int(configuration.MaximumMessageSizeBytes)), + grpcClientFactory) if err != nil { return util.StatusWrap(err, "Failed to create Indirect Content Addressable Storage") } @@ -109,7 +113,8 @@ func main() { configuration.InitialSizeClassCache, blobstore_configuration.NewISCCBlobAccessCreator( grpcClientFactory, - int(configuration.MaximumMessageSizeBytes))) + int(configuration.MaximumMessageSizeBytes)), + grpcClientFactory) if err != nil { return util.StatusWrap(err, "Failed to create Initial Size Class Cache") } @@ -124,7 +129,8 @@ func main() { configuration.FileSystemAccessCache, blobstore_configuration.NewFSACBlobAccessCreator( grpcClientFactory, - int(configuration.MaximumMessageSizeBytes))) + int(configuration.MaximumMessageSizeBytes)), + grpcClientFactory) if err != nil { return util.StatusWrap(err, "Failed to create File System Access Cache") } @@ -148,7 +154,7 @@ func main() { if err != nil { return err } - executeAuthorizer, err := auth.DefaultAuthorizerFactory.NewAuthorizerFromConfiguration(configuration.GetExecuteAuthorizer()) + executeAuthorizer, err := auth_configuration.DefaultAuthorizerFactory.NewAuthorizerFromConfiguration(configuration.GetExecuteAuthorizer(), grpcClientFactory) if err != nil { return util.StatusWrap(err, "Failed to create execute authorizer") } @@ -210,26 +216,27 @@ func main() { } }, siblingsGroup, + grpcClientFactory, ); err != nil { return util.StatusWrap(err, "gRPC server failure") } - lifecycleState.MarkReadyAndWait(siblingsGroup) + lifecycleState.MarkReadyAndWait(siblingsGroup, grpcClientFactory) return nil }) } -func newNonScannableBlobAccess(dependenciesGroup program.Group, configuration *bb_storage.NonScannableBlobAccessConfiguration, creator blobstore_configuration.BlobAccessCreator) (blobstore_configuration.BlobAccessInfo, blobstore.BlobAccess, []auth.Authorizer, auth.Authorizer, error) { +func newNonScannableBlobAccess(dependenciesGroup program.Group, configuration *bb_storage.NonScannableBlobAccessConfiguration, creator blobstore_configuration.BlobAccessCreator, grpcClientFactory bb_grpc.ClientFactory) (blobstore_configuration.BlobAccessInfo, blobstore.BlobAccess, []auth.Authorizer, auth.Authorizer, error) { info, err := blobstore_configuration.NewBlobAccessFromConfiguration(dependenciesGroup, configuration.Backend, creator) if err != nil { return blobstore_configuration.BlobAccessInfo{}, nil, nil, nil, err } - getAuthorizer, err := auth.DefaultAuthorizerFactory.NewAuthorizerFromConfiguration(configuration.GetAuthorizer) + getAuthorizer, err := auth_configuration.DefaultAuthorizerFactory.NewAuthorizerFromConfiguration(configuration.GetAuthorizer, grpcClientFactory) if err != nil { return blobstore_configuration.BlobAccessInfo{}, nil, nil, nil, util.StatusWrap(err, "Failed to create Get() authorizer") } - putAuthorizer, err := auth.DefaultAuthorizerFactory.NewAuthorizerFromConfiguration(configuration.PutAuthorizer) + putAuthorizer, err := auth_configuration.DefaultAuthorizerFactory.NewAuthorizerFromConfiguration(configuration.PutAuthorizer, grpcClientFactory) if err != nil { return blobstore_configuration.BlobAccessInfo{}, nil, nil, nil, util.StatusWrap(err, "Failed to create Put() authorizer") } @@ -241,21 +248,21 @@ func newNonScannableBlobAccess(dependenciesGroup program.Group, configuration *b nil } -func newScannableBlobAccess(dependenciesGroup program.Group, configuration *bb_storage.ScannableBlobAccessConfiguration, creator blobstore_configuration.BlobAccessCreator) (blobstore_configuration.BlobAccessInfo, blobstore.BlobAccess, []auth.Authorizer, error) { +func newScannableBlobAccess(dependenciesGroup program.Group, configuration *bb_storage.ScannableBlobAccessConfiguration, creator blobstore_configuration.BlobAccessCreator, grpcClientFactory bb_grpc.ClientFactory) (blobstore_configuration.BlobAccessInfo, blobstore.BlobAccess, []auth.Authorizer, error) { info, err := blobstore_configuration.NewBlobAccessFromConfiguration(dependenciesGroup, configuration.Backend, creator) if err != nil { return blobstore_configuration.BlobAccessInfo{}, nil, nil, err } - getAuthorizer, err := auth.DefaultAuthorizerFactory.NewAuthorizerFromConfiguration(configuration.GetAuthorizer) + getAuthorizer, err := auth_configuration.DefaultAuthorizerFactory.NewAuthorizerFromConfiguration(configuration.GetAuthorizer, grpcClientFactory) if err != nil { return blobstore_configuration.BlobAccessInfo{}, nil, nil, util.StatusWrap(err, "Failed to create Get() authorizer") } - putAuthorizer, err := auth.DefaultAuthorizerFactory.NewAuthorizerFromConfiguration(configuration.PutAuthorizer) + putAuthorizer, err := auth_configuration.DefaultAuthorizerFactory.NewAuthorizerFromConfiguration(configuration.PutAuthorizer, grpcClientFactory) if err != nil { return blobstore_configuration.BlobAccessInfo{}, nil, nil, util.StatusWrap(err, "Failed to create Put() authorizer") } - findMissingAuthorizer, err := auth.DefaultAuthorizerFactory.NewAuthorizerFromConfiguration(configuration.FindMissingAuthorizer) + findMissingAuthorizer, err := auth_configuration.DefaultAuthorizerFactory.NewAuthorizerFromConfiguration(configuration.FindMissingAuthorizer, grpcClientFactory) if err != nil { return blobstore_configuration.BlobAccessInfo{}, nil, nil, util.StatusWrap(err, "Failed to create FindMissing() authorizer") } diff --git a/internal/mock/BUILD.bazel b/internal/mock/BUILD.bazel index 1a064243..057f0c94 100644 --- a/internal/mock/BUILD.bazel +++ b/internal/mock/BUILD.bazel @@ -23,6 +23,7 @@ gomock( out = "auth.go", interfaces = [ "Authorizer", + "RequestHeadersAuthenticator", ], library = "//pkg/auth", mockgen_model_library = "@org_uber_go_mock//mockgen/model", diff --git a/pkg/auth/BUILD.bazel b/pkg/auth/BUILD.bazel index 27c91604..d8ac8622 100644 --- a/pkg/auth/BUILD.bazel +++ b/pkg/auth/BUILD.bazel @@ -6,24 +6,29 @@ go_library( "any_authorizer.go", "authentication_metadata.go", "authorizer.go", - "authorizer_factory.go", "jmespath_expression_authorizer.go", + "remote_authorizer.go", + "remote_request_headers_authenticator.go", + "request_headers_authenticator.go", "static_authorizer.go", ], importpath = "github.com/buildbarn/bb-storage/pkg/auth", visibility = ["//visibility:public"], deps = [ + "//pkg/clock", "//pkg/digest", + "//pkg/eviction", "//pkg/otel", "//pkg/proto/auth", - "//pkg/proto/configuration/auth", "//pkg/util", "@com_github_jmespath_go_jmespath//:go-jmespath", "@io_opentelemetry_go_otel//attribute", + "@org_golang_google_grpc//:grpc", "@org_golang_google_grpc//codes", "@org_golang_google_grpc//status", "@org_golang_google_protobuf//encoding/protojson", "@org_golang_google_protobuf//proto", + "@org_golang_google_protobuf//types/known/structpb", ], ) @@ -33,21 +38,28 @@ go_test( "any_authorizer_test.go", "authentication_metadata_test.go", "jmespath_expression_authorizer_test.go", + "remote_authorizer_test.go", + "remote_request_headers_authenticator_test.go", "static_authorizer_test.go", ], deps = [ ":auth", "//internal/mock", "//pkg/digest", + "//pkg/eviction", "//pkg/proto/auth", "//pkg/testutil", "@com_github_jmespath_go_jmespath//:go-jmespath", "@com_github_stretchr_testify//require", "@io_opentelemetry_go_otel//attribute", "@io_opentelemetry_go_proto_otlp//common/v1:common", + "@org_golang_google_grpc//:grpc", "@org_golang_google_grpc//codes", "@org_golang_google_grpc//status", + "@org_golang_google_protobuf//proto", + "@org_golang_google_protobuf//types/known/emptypb", "@org_golang_google_protobuf//types/known/structpb", + "@org_golang_google_protobuf//types/known/timestamppb", "@org_uber_go_mock//gomock", ], ) diff --git a/pkg/auth/configuration/BUILD.bazel b/pkg/auth/configuration/BUILD.bazel new file mode 100644 index 00000000..d441ac78 --- /dev/null +++ b/pkg/auth/configuration/BUILD.bazel @@ -0,0 +1,21 @@ +load("@rules_go//go:def.bzl", "go_library") + +go_library( + name = "configuration", + srcs = ["authorizer_factory.go"], + importpath = "github.com/buildbarn/bb-storage/pkg/auth/configuration", + visibility = ["//visibility:public"], + deps = [ + "//pkg/auth", + "//pkg/clock", + "//pkg/digest", + "//pkg/eviction", + "//pkg/grpc", + "//pkg/proto/configuration/auth", + "//pkg/util", + "@com_github_jmespath_go_jmespath//:go-jmespath", + "@org_golang_google_grpc//codes", + "@org_golang_google_grpc//status", + "@org_golang_google_protobuf//encoding/protojson", + ], +) diff --git a/pkg/auth/authorizer_factory.go b/pkg/auth/configuration/authorizer_factory.go similarity index 64% rename from pkg/auth/authorizer_factory.go rename to pkg/auth/configuration/authorizer_factory.go index f2e8c1bd..96bb69b5 100644 --- a/pkg/auth/authorizer_factory.go +++ b/pkg/auth/configuration/authorizer_factory.go @@ -1,7 +1,11 @@ -package auth +package configuration import ( + "github.com/buildbarn/bb-storage/pkg/auth" + "github.com/buildbarn/bb-storage/pkg/clock" "github.com/buildbarn/bb-storage/pkg/digest" + "github.com/buildbarn/bb-storage/pkg/eviction" + "github.com/buildbarn/bb-storage/pkg/grpc" pb "github.com/buildbarn/bb-storage/pkg/proto/configuration/auth" "github.com/buildbarn/bb-storage/pkg/util" "github.com/jmespath/go-jmespath" @@ -16,7 +20,7 @@ import ( type AuthorizerFactory interface { // NewAuthorizerFromConfiguration constructs an authorizer based on // options specified in a configuration message. - NewAuthorizerFromConfiguration(configuration *pb.AuthorizerConfiguration) (Authorizer, error) + NewAuthorizerFromConfiguration(configuration *pb.AuthorizerConfiguration, grpcClientFactory grpc.ClientFactory) (auth.Authorizer, error) } // DefaultAuthorizerFactory constructs deduplicated authorizers based on @@ -29,15 +33,15 @@ type BaseAuthorizerFactory struct{} // NewAuthorizerFromConfiguration constructs an authorizer based on // options specified in a configuration message. -func (f BaseAuthorizerFactory) NewAuthorizerFromConfiguration(config *pb.AuthorizerConfiguration) (Authorizer, error) { +func (f BaseAuthorizerFactory) NewAuthorizerFromConfiguration(config *pb.AuthorizerConfiguration, grpcClientFactory grpc.ClientFactory) (auth.Authorizer, error) { if config == nil { return nil, status.Error(codes.InvalidArgument, "Authorizer configuration not specified") } switch policy := config.Policy.(type) { case *pb.AuthorizerConfiguration_Allow: - return NewStaticAuthorizer(func(in digest.InstanceName) bool { return true }), nil + return auth.NewStaticAuthorizer(func(in digest.InstanceName) bool { return true }), nil case *pb.AuthorizerConfiguration_Deny: - return NewStaticAuthorizer(func(in digest.InstanceName) bool { return false }), nil + return auth.NewStaticAuthorizer(func(in digest.InstanceName) bool { return false }), nil case *pb.AuthorizerConfiguration_InstanceNamePrefix: trie := digest.NewInstanceNameTrie() for _, i := range policy.InstanceNamePrefix.AllowedInstanceNamePrefixes { @@ -47,13 +51,29 @@ func (f BaseAuthorizerFactory) NewAuthorizerFromConfiguration(config *pb.Authori } trie.Set(instanceNamePrefix, 0) } - return NewStaticAuthorizer(trie.ContainsPrefix), nil + return auth.NewStaticAuthorizer(trie.ContainsPrefix), nil case *pb.AuthorizerConfiguration_JmespathExpression: expression, err := jmespath.Compile(policy.JmespathExpression) if err != nil { return nil, util.StatusWrapWithCode(err, codes.InvalidArgument, "Failed to compile JMESPath expression") } - return NewJMESPathExpressionAuthorizer(expression), nil + return auth.NewJMESPathExpressionAuthorizer(expression), nil + case *pb.AuthorizerConfiguration_Remote: + grpcClient, err := grpcClientFactory.NewClientFromConfiguration(policy.Remote.Endpoint) + if err != nil { + return nil, util.StatusWrap(err, "Failed to create authorizer RPC client") + } + evictionSet, err := eviction.NewSetFromConfiguration[auth.RemoteAuthorizerCacheKey](policy.Remote.CacheReplacementPolicy) + if err != nil { + return nil, util.StatusWrap(err, "Cache replacement policy for remote authorization") + } + return auth.NewRemoteAuthorizer( + grpcClient, + policy.Remote.Scope, + clock.SystemClock, + eviction.NewMetricsSet(evictionSet, "remote_authorizer"), + int(policy.Remote.MaximumCacheSize), + ), nil default: return nil, status.Error(codes.InvalidArgument, "Unknown authorizer configuration") } @@ -62,7 +82,7 @@ func (f BaseAuthorizerFactory) NewAuthorizerFromConfiguration(config *pb.Authori type deduplicatingAuthorizerFactory struct { base AuthorizerFactory // Keys are protojson-encoded pb.AuthorizerConfigurations - known map[string]Authorizer + known map[string]auth.Authorizer } // NewDeduplicatingAuthorizerFactory creates a new AuthorizerFactory @@ -71,19 +91,19 @@ type deduplicatingAuthorizerFactory struct { func NewDeduplicatingAuthorizerFactory(base AuthorizerFactory) AuthorizerFactory { return &deduplicatingAuthorizerFactory{ base: base, - known: make(map[string]Authorizer), + known: make(map[string]auth.Authorizer), } } // NewAuthorizerFromConfiguration creates an Authorizer based on the passed configuration. -func (af *deduplicatingAuthorizerFactory) NewAuthorizerFromConfiguration(config *pb.AuthorizerConfiguration) (Authorizer, error) { +func (af *deduplicatingAuthorizerFactory) NewAuthorizerFromConfiguration(config *pb.AuthorizerConfiguration, grpcClientFactory grpc.ClientFactory) (auth.Authorizer, error) { keyBytes, err := protojson.Marshal(config) key := string(keyBytes) if err != nil { return nil, err } if _, ok := af.known[key]; !ok { - a, err := af.base.NewAuthorizerFromConfiguration(config) + a, err := af.base.NewAuthorizerFromConfiguration(config, grpcClientFactory) if err != nil { return nil, err } diff --git a/pkg/auth/remote_authorizer.go b/pkg/auth/remote_authorizer.go new file mode 100644 index 00000000..4e8ee731 --- /dev/null +++ b/pkg/auth/remote_authorizer.go @@ -0,0 +1,199 @@ +package auth + +import ( + "context" + "crypto/sha256" + "sync" + "time" + + "github.com/buildbarn/bb-storage/pkg/clock" + "github.com/buildbarn/bb-storage/pkg/digest" + "github.com/buildbarn/bb-storage/pkg/eviction" + auth_pb "github.com/buildbarn/bb-storage/pkg/proto/auth" + "github.com/buildbarn/bb-storage/pkg/util" + "google.golang.org/grpc" + "google.golang.org/grpc/codes" + "google.golang.org/grpc/status" + "google.golang.org/protobuf/proto" + "google.golang.org/protobuf/types/known/structpb" +) + +type remoteAuthorizer struct { + remoteAuthClient auth_pb.AuthorizerClient + scope *structpb.Value + + clock clock.Clock + maximumCacheSize int + + lock sync.Mutex + // The channel is closed when the response is ready. If the response is + // missing from cachedResponses, the request failed and should be retried. + pendingRequests map[RemoteAuthorizerCacheKey]<-chan struct{} + cachedResponses map[RemoteAuthorizerCacheKey]remoteAuthorizerResponse + evictionSet eviction.Set[RemoteAuthorizerCacheKey] +} + +// RemoteAuthorizerCacheKey is the key type for the cache inside +// remoteAuthorizer. +type RemoteAuthorizerCacheKey [sha256.Size]byte + +type remoteAuthorizerResponse struct { + expirationTime time.Time + err error +} + +// NewRemoteAuthorizer creates a new Authorizer which asks a remote gRPC +// service for authorize response. The result from the remote service is +// cached. +func NewRemoteAuthorizer( + client grpc.ClientConnInterface, + scope *structpb.Value, + clock clock.Clock, + evictionSet eviction.Set[RemoteAuthorizerCacheKey], + maximumCacheSize int, +) Authorizer { + return &remoteAuthorizer{ + remoteAuthClient: auth_pb.NewAuthorizerClient(client), + scope: scope, + + clock: clock, + maximumCacheSize: maximumCacheSize, + + pendingRequests: make(map[RemoteAuthorizerCacheKey]<-chan struct{}), + cachedResponses: make(map[RemoteAuthorizerCacheKey]remoteAuthorizerResponse), + evictionSet: evictionSet, + } +} + +func (a *remoteAuthorizer) Authorize(ctx context.Context, instanceNames []digest.InstanceName) []error { + errs := make([]error, 0, len(instanceNames)) + for _, instanceName := range instanceNames { + errs = append(errs, a.authorizeSingle(ctx, instanceName)) + } + return errs +} + +func (a *remoteAuthorizer) authorizeSingle(ctx context.Context, instanceName digest.InstanceName) error { + authenticationMetadata := AuthenticationMetadataFromContext(ctx) + request := &auth_pb.AuthorizeRequest{ + AuthenticationMetadata: authenticationMetadata.GetFullProto(), + Scope: a.scope, + InstanceName: instanceName.String(), + } + if a.maximumCacheSize == 0 { + // Cache disabled, skip request deduplication. + response, err := a.authorizeRemotely(ctx, request) + if err != nil { + return err + } + return response.err + } + + requestBytes, err := proto.Marshal(request) + if err != nil { + return util.StatusWrapWithCode(err, codes.PermissionDenied, "Failed to marshal authorize request") + } + // Hash the request to use as a cache key to save memory. + requestKey := sha256.Sum256(requestBytes) + + for { + now := a.clock.Now() + a.lock.Lock() + if response, ok := a.getAndTouchCacheEntry(requestKey); ok && response.expirationTime.After(now) { + a.lock.Unlock() + return response.err + } + // No valid cache entry available. Deduplicate requests. + responseReadyChan, ok := a.pendingRequests[requestKey] + if !ok { + // No pending request. Create one. + responseReadyChan := make(chan struct{}) + a.pendingRequests[requestKey] = responseReadyChan + a.lock.Unlock() + // Perform the remote authentication request. + response, err := a.authorizeRemotely(ctx, request) + a.lock.Lock() + delete(a.pendingRequests, requestKey) + close(responseReadyChan) + if err != nil { + a.lock.Unlock() + return err + } + a.insertCacheEntry(requestKey, response) + a.lock.Unlock() + return response.err + } + // A remote request is pending, wait for it to finish. + a.lock.Unlock() + select { + case <-ctx.Done(): + return util.StatusFromContext(ctx) + case <-responseReadyChan: + } + // Check whether the remote authentication call succeeded. + a.lock.Lock() + response, ok := a.cachedResponses[requestKey] + a.lock.Unlock() + if ok { + // Note that the expiration time is not checked, as the + // response is as fresh as it can be. + return response.err + } + // The remote authentication call failed. Retry. + } +} + +func (a *remoteAuthorizer) getAndTouchCacheEntry(requestKey RemoteAuthorizerCacheKey) (remoteAuthorizerResponse, bool) { + if entry, ok := a.cachedResponses[requestKey]; ok { + // Cache contains a matching entry. + a.evictionSet.Touch(requestKey) + return entry, true + } + return remoteAuthorizerResponse{}, false +} + +func (a *remoteAuthorizer) insertCacheEntry(requestKey RemoteAuthorizerCacheKey, response remoteAuthorizerResponse) { + if _, ok := a.cachedResponses[requestKey]; ok { + a.evictionSet.Touch(requestKey) + } else { + // Cache contains no matching entry. Free up space, so that the + // caller may insert a new entry. + for len(a.cachedResponses) > 0 && len(a.cachedResponses) >= a.maximumCacheSize { + delete(a.cachedResponses, a.evictionSet.Peek()) + a.evictionSet.Remove() + } + a.evictionSet.Insert(requestKey) + } + a.cachedResponses[requestKey] = response +} + +func (a *remoteAuthorizer) authorizeRemotely(ctx context.Context, request *auth_pb.AuthorizeRequest) (remoteAuthorizerResponse, error) { + ret := remoteAuthorizerResponse{ + // The default expirationTime has already passed. + expirationTime: time.Time{}, + } + + response, err := a.remoteAuthClient.Authorize(ctx, request) + if err != nil { + return ret, util.StatusWrapWithCode(err, codes.PermissionDenied, "Remote authorization failed") + } + + // An invalid expiration time indicates that the response should not be cached. + if response.CacheExpirationTime != nil { + if err := response.CacheExpirationTime.CheckValid(); err != nil { + return ret, util.StatusWrapWithCode(err, codes.PermissionDenied, "Invalid authorization expiration time") + } + // Note that the expiration time might still be valid for non-allow verdicts. + ret.expirationTime = response.GetCacheExpirationTime().AsTime() + } + + switch verdict := response.Verdict.(type) { + case *auth_pb.AuthorizeResponse_Allow: + // noop + case *auth_pb.AuthorizeResponse_Deny: + ret.err = status.Error(codes.PermissionDenied, verdict.Deny) + default: + ret.err = status.Error(codes.PermissionDenied, "Invalid authorize verdict") + } + return ret, nil +} diff --git a/pkg/auth/remote_authorizer_test.go b/pkg/auth/remote_authorizer_test.go new file mode 100644 index 00000000..38cee3e1 --- /dev/null +++ b/pkg/auth/remote_authorizer_test.go @@ -0,0 +1,440 @@ +package auth_test + +import ( + "context" + "strings" + "testing" + "time" + + "github.com/buildbarn/bb-storage/internal/mock" + "github.com/buildbarn/bb-storage/pkg/auth" + "github.com/buildbarn/bb-storage/pkg/digest" + "github.com/buildbarn/bb-storage/pkg/eviction" + auth_pb "github.com/buildbarn/bb-storage/pkg/proto/auth" + "github.com/buildbarn/bb-storage/pkg/testutil" + "github.com/stretchr/testify/require" + "go.uber.org/mock/gomock" + "google.golang.org/grpc" + "google.golang.org/grpc/codes" + "google.golang.org/grpc/status" + "google.golang.org/protobuf/proto" + "google.golang.org/protobuf/types/known/emptypb" + "google.golang.org/protobuf/types/known/structpb" + "google.golang.org/protobuf/types/known/timestamppb" +) + +func TestRemoteAuthorizerFailure(t *testing.T) { + ctrl, ctx := gomock.WithContext(context.Background(), t) + + client := mock.NewMockClientConnInterface(ctrl) + clock := mock.NewMockClock(ctrl) + + t.Run("BackendFailure", func(t *testing.T) { + client.EXPECT().Invoke( + ctx, "/buildbarn.auth.Authorizer/Authorize", gomock.Any(), gomock.Any(), gomock.Any(), + ).Return(status.Error(codes.Unavailable, "Server offline")) + clock.EXPECT().Now().Return(time.Unix(1000, 0)) + + authorizer := auth.NewRemoteAuthorizer( + client, + structpb.NewStringValue("auth-scope"), + clock, + eviction.NewLRUSet[auth.RemoteAuthorizerCacheKey](), + 100, + ) + errs := authorizer.Authorize(ctx, []digest.InstanceName{digest.MustNewInstanceName("allowed")}) + require.Len(t, errs, 1) + testutil.RequireEqualStatus( + t, + status.Error(codes.PermissionDenied, "Remote authorization failed: Server offline"), + errs[0]) + }) + + t.Run("InvalidVerdict", func(t *testing.T) { + client.EXPECT().Invoke( + ctx, "/buildbarn.auth.Authorizer/Authorize", gomock.Any(), gomock.Any(), gomock.Any(), + ).DoAndReturn(func(ctx context.Context, method string, args, reply interface{}, opts ...grpc.CallOption) error { + proto.Merge(reply.(proto.Message), &auth_pb.AuthorizeResponse{}) + return nil + }) + clock.EXPECT().Now().Return(time.Unix(1000, 0)) + + authorizer := auth.NewRemoteAuthorizer( + client, + structpb.NewStringValue("auth-scope"), + clock, + eviction.NewLRUSet[auth.RemoteAuthorizerCacheKey](), + 100, + ) + errs := authorizer.Authorize(ctx, []digest.InstanceName{digest.MustNewInstanceName("allowed")}) + require.Len(t, errs, 1) + testutil.RequireEqualStatus( + t, + status.Error(codes.PermissionDenied, "Invalid authorize verdict"), + errs[0]) + }) +} + +func TestRemoteAuthorizerSuccess(t *testing.T) { + ctrl, ctx := gomock.WithContext(context.Background(), t) + authCtx := auth.NewContextWithAuthenticationMetadata( + ctx, + auth.MustNewAuthenticationMetadataFromProto(&auth_pb.AuthenticationMetadata{ + Public: structpb.NewStringValue("I'm here"), + }), + ) + + remoteService := func(ctx context.Context, method string, args, reply interface{}, opts ...grpc.CallOption) error { + instanceName := args.(*auth_pb.AuthorizeRequest).InstanceName + if strings.HasPrefix(instanceName, "allow") { + proto.Merge(reply.(proto.Message), &auth_pb.AuthorizeResponse{ + Verdict: &auth_pb.AuthorizeResponse_Allow{ + Allow: &emptypb.Empty{}, + }, + CacheExpirationTime: timestamppb.New(time.Unix(1002, 0)), + }) + } else if strings.HasPrefix(instanceName, "deny") { + proto.Merge(reply.(proto.Message), &auth_pb.AuthorizeResponse{ + Verdict: &auth_pb.AuthorizeResponse_Deny{ + Deny: instanceName, + }, + CacheExpirationTime: timestamppb.New(time.Unix(1002, 0)), + }) + } + return nil + } + + testAuthorizeAllow := func(authorizer auth.Authorizer, instanceName string) { + errs := authorizer.Authorize(ctx, []digest.InstanceName{digest.MustNewInstanceName(instanceName)}) + require.Len(t, errs, 1) + require.NoError(t, errs[0]) + } + testAuthorizeDeny := func(authorizer auth.Authorizer, instanceName string) { + errs := authorizer.Authorize(ctx, []digest.InstanceName{digest.MustNewInstanceName(instanceName)}) + require.Len(t, errs, 1) + testutil.RequireEqualStatus( + t, + status.Error(codes.PermissionDenied, instanceName), + errs[0]) + } + + t.Run("Success", func(t *testing.T) { + client := mock.NewMockClientConnInterface(ctrl) + clock := mock.NewMockClock(ctrl) + + client.EXPECT().Invoke( + gomock.Any(), + "/buildbarn.auth.Authorizer/Authorize", + testutil.EqProto(t, &auth_pb.AuthorizeRequest{ + AuthenticationMetadata: &auth_pb.AuthenticationMetadata{ + Public: structpb.NewStringValue("I'm here"), + }, + Scope: structpb.NewStringValue("auth-scope"), + InstanceName: "deny-success", + }), + gomock.Any(), + gomock.Any(), + ).DoAndReturn(remoteService) + client.EXPECT().Invoke( + gomock.Any(), + "/buildbarn.auth.Authorizer/Authorize", + testutil.EqProto(t, &auth_pb.AuthorizeRequest{ + AuthenticationMetadata: &auth_pb.AuthenticationMetadata{ + Public: structpb.NewStringValue("I'm here"), + }, + Scope: structpb.NewStringValue("auth-scope"), + InstanceName: "allow-success", + }), + gomock.Any(), + gomock.Any(), + ).DoAndReturn(remoteService) + clock.EXPECT().Now().Return(time.Unix(1000, 0)).Times(2) + + authorizer := auth.NewRemoteAuthorizer( + client, + structpb.NewStringValue("auth-scope"), + clock, + eviction.NewLRUSet[auth.RemoteAuthorizerCacheKey](), + 100, + ) + errs := authorizer.Authorize(authCtx, []digest.InstanceName{ + digest.MustNewInstanceName("deny-success"), + digest.MustNewInstanceName("allow-success"), + }) + // The returned errors should be in the same order as the request instance names. + require.Len(t, errs, 2) + testutil.RequireEqualStatus( + t, + status.Error(codes.PermissionDenied, "deny-success"), + errs[0]) + require.NoError(t, errs[1]) + }) + + t.Run("ExpireResponses", func(t *testing.T) { + client := mock.NewMockClientConnInterface(ctrl) + clock := mock.NewMockClock(ctrl) + + authorizer := auth.NewRemoteAuthorizer( + client, + structpb.NewStringValue("auth-scope"), + clock, + eviction.NewLRUSet[auth.RemoteAuthorizerCacheKey](), + 100, + ) + + // First request should hit the backend. + // Second request should hit the cache. + // Third request should hit the backend again, as the cache entry has expired. + for _, timestamp := range []int64{1000, 1001, 1002} { + if timestamp != 1001 { + client.EXPECT().Invoke( + ctx, "/buildbarn.auth.Authorizer/Authorize", gomock.Any(), gomock.Any(), gomock.Any(), + ).DoAndReturn(remoteService).Times(2) // Times 2 for both allow and deny calls. + } + clock.EXPECT().Now().Return(time.Unix(timestamp, 0)) + testAuthorizeAllow(authorizer, "allow") + clock.EXPECT().Now().Return(time.Unix(timestamp, 0)) + testAuthorizeDeny(authorizer, "deny") + } + }) + + t.Run("CacheKeyContent", func(t *testing.T) { + client := mock.NewMockClientConnInterface(ctrl) + clock := mock.NewMockClock(ctrl) + + clock.EXPECT().Now().Return(time.Unix(1000, 0)).AnyTimes() + + authorizer := auth.NewRemoteAuthorizer( + client, + structpb.NewStringValue("auth-scope"), + clock, + eviction.NewLRUSet[auth.RemoteAuthorizerCacheKey](), + 100, + ) + + // First call uncached. + client.EXPECT().Invoke(gomock.Any(), "/buildbarn.auth.Authorizer/Authorize", gomock.Any(), gomock.Any(), gomock.Any()).DoAndReturn(remoteService) + errs := authorizer.Authorize(ctx, []digest.InstanceName{digest.MustNewInstanceName("allow")}) + require.Len(t, errs, 1) + require.NoError(t, errs[0]) + // Different instanceName, not cached. + client.EXPECT().Invoke(gomock.Any(), "/buildbarn.auth.Authorizer/Authorize", gomock.Any(), gomock.Any(), gomock.Any()).DoAndReturn(remoteService) + errs = authorizer.Authorize(ctx, []digest.InstanceName{digest.MustNewInstanceName("allow2")}) + require.Len(t, errs, 1) + require.NoError(t, errs[0]) + // Different authMetadata, not cached. + client.EXPECT().Invoke(gomock.Any(), "/buildbarn.auth.Authorizer/Authorize", gomock.Any(), gomock.Any(), gomock.Any()).DoAndReturn(remoteService) + errs = authorizer.Authorize(authCtx, []digest.InstanceName{digest.MustNewInstanceName("allow")}) + require.Len(t, errs, 1) + require.NoError(t, errs[0]) + // Different context instance, should be cached. + ctxOther := context.WithValue(ctx, "unused-key", "value") + errs = authorizer.Authorize(ctxOther, []digest.InstanceName{digest.MustNewInstanceName("allow")}) + require.Len(t, errs, 1) + require.NoError(t, errs[0]) + }) + + t.Run("MaxCacheSizeAndLRUEviction", func(t *testing.T) { + client := mock.NewMockClientConnInterface(ctrl) + clock := mock.NewMockClock(ctrl) + + clock.EXPECT().Now().Return(time.Unix(1000, 0)).AnyTimes() + + authorizer := auth.NewRemoteAuthorizer( + client, + structpb.NewStringValue("auth-scope"), + clock, + eviction.NewLRUSet[auth.RemoteAuthorizerCacheKey](), + 2, // Only two spaces in this test. + ) + + // 1 uncached. + client.EXPECT().Invoke(gomock.Any(), "/buildbarn.auth.Authorizer/Authorize", gomock.Any(), gomock.Any(), gomock.Any()).DoAndReturn(remoteService) + testAuthorizeAllow(authorizer, "allow1") + // 2 uncached. + client.EXPECT().Invoke(gomock.Any(), "/buildbarn.auth.Authorizer/Authorize", gomock.Any(), gomock.Any(), gomock.Any()).DoAndReturn(remoteService) + testAuthorizeDeny(authorizer, "deny2") + // 1 should be cached. + testAuthorizeAllow(authorizer, "allow1") + // 3 uncached, replacing 2. + client.EXPECT().Invoke(gomock.Any(), "/buildbarn.auth.Authorizer/Authorize", gomock.Any(), gomock.Any(), gomock.Any()).DoAndReturn(remoteService) + testAuthorizeAllow(authorizer, "allow3") + // 1 should still be cached. + testAuthorizeAllow(authorizer, "allow1") + // 2 should have been evicted. + client.EXPECT().Invoke(gomock.Any(), "/buildbarn.auth.Authorizer/Authorize", gomock.Any(), gomock.Any(), gomock.Any()).DoAndReturn(remoteService) + testAuthorizeDeny(authorizer, "deny2") + }) + + t.Run("DeduplicateCalls", func(t *testing.T) { + client := mock.NewMockClientConnInterface(ctrl) + clock := mock.NewMockClock(ctrl) + + authCalled := make(chan struct{}) + authRelease := make(map[string]chan struct{}) + + client.EXPECT().Invoke( + ctx, "/buildbarn.auth.Authorizer/Authorize", gomock.Any(), gomock.Any(), gomock.Any(), + ).DoAndReturn(func(ctx context.Context, method string, args, reply interface{}, opts ...grpc.CallOption) error { + name := args.(*auth_pb.AuthorizeRequest).InstanceName + proto.Merge(reply.(proto.Message), &auth_pb.AuthorizeResponse{}) + authCalled <- struct{}{} + <-authRelease[name] + return nil + }).Times(2) // name1 and name2 + + clock.EXPECT().Now().Return(time.Unix(1000, 0)).AnyTimes() + + authorizer := auth.NewRemoteAuthorizer( + client, + structpb.NewStringValue("auth-scope"), + clock, + eviction.NewLRUSet[auth.RemoteAuthorizerCacheKey](), + 100, + ) + doAuth := func(name string, done chan<- struct{}) { + errs := authorizer.Authorize(ctx, []digest.InstanceName{digest.MustNewInstanceName(name)}) + require.Len(t, errs, 1) + defer close(done) + testutil.RequireEqualStatus( + t, + status.Error(codes.PermissionDenied, "Invalid authorize verdict"), + errs[0]) + } + authRelease["name1"] = make(chan struct{}) + authRelease["name2"] = make(chan struct{}) + done1a := make(chan struct{}) + done1b := make(chan struct{}) + done1c := make(chan struct{}) + done2 := make(chan struct{}) + go doAuth("name1", done1a) + <-authCalled + go doAuth("name2", done2) + <-authCalled + go doAuth("name1", done1b) + ctx1c, cancel1c := context.WithCancel(ctx) + go func() { + errs := authorizer.Authorize(ctx1c, []digest.InstanceName{digest.MustNewInstanceName("name1")}) + require.Len(t, errs, 1) + testutil.RequireEqualStatus( + t, + status.Error(codes.Canceled, "context canceled"), + errs[0]) + close(done1c) + }() + // Nothing done yet. + time.Sleep(100 * time.Millisecond) + select { + case <-done1a: + t.Error("done1a too early") + case <-done1b: + t.Error("done1b too early") + case <-done1c: + t.Error("done1c too early") + case <-done2: + t.Error("done2 too early") + default: + // Noop. + } + cancel1c() + <-done1c + close(authRelease["name2"]) + // name1 still blocked. + time.Sleep(100 * time.Millisecond) + select { + case <-done1a: + t.Error("done1a too early") + case <-done1b: + t.Error("done1b too early") + case <-done2: + // Noop. + } + close(authRelease["name1"]) + <-done1a + <-done1b + }) + + t.Run("SkipDeduplicateErrors", func(t *testing.T) { + client := mock.NewMockClientConnInterface(ctrl) + clock := mock.NewMockClock(ctrl) + + authCalled := make(chan struct{}) + authRelease := make(map[string]chan struct{}) + + client.EXPECT().Invoke( + ctx, "/buildbarn.auth.Authorizer/Authorize", gomock.Any(), gomock.Any(), gomock.Any(), + ).DoAndReturn(func(ctx context.Context, method string, args, reply interface{}, opts ...grpc.CallOption) error { + name := args.(*auth_pb.AuthorizeRequest).InstanceName + proto.Merge(reply.(proto.Message), &auth_pb.AuthorizeResponse{}) + authCalled <- struct{}{} + <-authRelease[name] + return status.Error(codes.DataLoss, "Data loss") + }) + + client.EXPECT().Invoke( + ctx, "/buildbarn.auth.Authorizer/Authorize", gomock.Any(), gomock.Any(), gomock.Any(), + ).DoAndReturn(func(ctx context.Context, method string, args, reply interface{}, opts ...grpc.CallOption) error { + name := args.(*auth_pb.AuthorizeRequest).InstanceName + proto.Merge(reply.(proto.Message), &auth_pb.AuthorizeResponse{}) + authCalled <- struct{}{} + <-authRelease[name] + return nil + }) + + clock.EXPECT().Now().Return(time.Unix(1000, 0)).AnyTimes() + + authorizer := auth.NewRemoteAuthorizer( + client, + structpb.NewStringValue("auth-scope"), + clock, + eviction.NewLRUSet[auth.RemoteAuthorizerCacheKey](), + 100, + ) + doAuth := func(name string, done chan<- struct{}, verdict string) { + errs := authorizer.Authorize(ctx, []digest.InstanceName{digest.MustNewInstanceName(name)}) + require.Len(t, errs, 1) + defer close(done) + testutil.RequireEqualStatus( + t, + status.Error(codes.PermissionDenied, verdict), + errs[0]) + } + + authRelease["token1"] = make(chan struct{}) + done1a := make(chan struct{}) + done1b := make(chan struct{}) + done1c := make(chan struct{}) + go doAuth("token1", done1a, "Remote authorization failed: Data loss") + <-authCalled // token1a + go doAuth("token1", done1b, "Invalid authorize verdict") + go doAuth("token1", done1c, "Invalid authorize verdict") + // Nothing done yet. + time.Sleep(100 * time.Millisecond) + select { + case <-done1a: + t.Error("done1a too early") + case <-done1b: + t.Error("done1b too early") + case <-done1c: + t.Error("done1c too early") + case <-authCalled: + t.Error("authCalled second time too early") + default: + // Noop. + } + close(authRelease["token1"]) + // token1 still blocked. + time.Sleep(100 * time.Millisecond) + select { + case <-done1b: + t.Error("done1b too early") + case <-done1c: + t.Error("done1c too early") + case <-authCalled: + // token1b released. + // Noop. + } + <-done1b + <-done1c + }) +} diff --git a/pkg/auth/remote_request_headers_authenticator.go b/pkg/auth/remote_request_headers_authenticator.go new file mode 100644 index 00000000..c47cb0b0 --- /dev/null +++ b/pkg/auth/remote_request_headers_authenticator.go @@ -0,0 +1,199 @@ +package auth + +import ( + "context" + "crypto/sha256" + "sync" + "time" + + "github.com/buildbarn/bb-storage/pkg/clock" + "github.com/buildbarn/bb-storage/pkg/eviction" + auth_pb "github.com/buildbarn/bb-storage/pkg/proto/auth" + "github.com/buildbarn/bb-storage/pkg/util" + "google.golang.org/grpc" + "google.golang.org/grpc/codes" + "google.golang.org/grpc/status" + "google.golang.org/protobuf/proto" + "google.golang.org/protobuf/types/known/structpb" +) + +type remoteRequestHeadersAuthenticator struct { + remoteAuthClient auth_pb.AuthenticationClient + scope *structpb.Value + + clock clock.Clock + maximumCacheSize int + + lock sync.Mutex + // The channel is closed when the response is ready. If the response is + // missing from cachedResponses, the request failed and should be retried. + pendingRequests map[RemoteRequestHeadersAuthenticatorCacheKey]<-chan struct{} + cachedResponses map[RemoteRequestHeadersAuthenticatorCacheKey]remoteAuthResponse + evictionSet eviction.Set[RemoteRequestHeadersAuthenticatorCacheKey] +} + +// RemoteRequestHeadersAuthenticatorCacheKey is the key type for the cache +// inside remoteRequestHeadersAuthenticator. +type RemoteRequestHeadersAuthenticatorCacheKey [sha256.Size]byte + +type remoteAuthResponse struct { + expirationTime time.Time + authMetadata *AuthenticationMetadata + err error +} + +// NewRemoteRequestHeadersAuthenticator creates a new +// RequestHeadersAuthenticator for incoming requests that forwards headers to a +// remote service for authentication. The result from the remote service is +// cached. +func NewRemoteRequestHeadersAuthenticator( + client grpc.ClientConnInterface, + scope *structpb.Value, + clock clock.Clock, + evictionSet eviction.Set[RemoteRequestHeadersAuthenticatorCacheKey], + maximumCacheSize int, +) RequestHeadersAuthenticator { + return &remoteRequestHeadersAuthenticator{ + remoteAuthClient: auth_pb.NewAuthenticationClient(client), + scope: scope, + + clock: clock, + maximumCacheSize: maximumCacheSize, + + pendingRequests: make(map[RemoteRequestHeadersAuthenticatorCacheKey]<-chan struct{}), + cachedResponses: make(map[RemoteRequestHeadersAuthenticatorCacheKey]remoteAuthResponse), + evictionSet: evictionSet, + } +} + +func (a *remoteRequestHeadersAuthenticator) Authenticate(ctx context.Context, headers map[string][]string) (*AuthenticationMetadata, error) { + request := &auth_pb.AuthenticateRequest{ + RequestMetadata: make(map[string]*auth_pb.AuthenticateRequest_ValueList, len(headers)), + Scope: a.scope, + } + for headerKey, headerValues := range headers { + request.RequestMetadata[headerKey] = &auth_pb.AuthenticateRequest_ValueList{ + Value: headerValues, + } + } + if a.maximumCacheSize == 0 { + // Cache disabled, skip request deduplication. + response, err := a.authenticateRemotely(ctx, request) + if err != nil { + return nil, err + } + return response.authMetadata, response.err + } + + requestBytes, err := proto.Marshal(request) + if err != nil { + return nil, util.StatusWrapWithCode(err, codes.Unauthenticated, "Failed to marshal authenticate request") + } + // Hash the request to use as a cache key to both save memory and avoid + // keeping credentials in the memory. + requestKey := sha256.Sum256(requestBytes) + + for { + now := a.clock.Now() + a.lock.Lock() + if response, ok := a.getAndTouchCacheEntry(requestKey); ok && response.expirationTime.After(now) { + a.lock.Unlock() + return response.authMetadata, response.err + } + // No valid cache entry available. Deduplicate requests. + responseReadyChan, ok := a.pendingRequests[requestKey] + if !ok { + // No pending request. Create one. + responseReadyChan := make(chan struct{}) + a.pendingRequests[requestKey] = responseReadyChan + a.lock.Unlock() + // Perform the remote authentication request. + response, err := a.authenticateRemotely(ctx, request) + a.lock.Lock() + delete(a.pendingRequests, requestKey) + close(responseReadyChan) + if err != nil { + a.lock.Unlock() + return nil, err + } + a.insertCacheEntry(requestKey, response) + a.lock.Unlock() + return response.authMetadata, response.err + } + // A remote request is pending, wait for it to finish. + a.lock.Unlock() + select { + case <-ctx.Done(): + return nil, util.StatusFromContext(ctx) + case <-responseReadyChan: + } + // Check whether the remote authentication call succeeded. + a.lock.Lock() + response, ok := a.cachedResponses[requestKey] + a.lock.Unlock() + if ok { + // Note that the expiration time is not checked, as the + // response is as fresh as it can be. + return response.authMetadata, response.err + } + // The remote authentication call failed. Retry. + } +} + +func (a *remoteRequestHeadersAuthenticator) getAndTouchCacheEntry(requestKey RemoteRequestHeadersAuthenticatorCacheKey) (remoteAuthResponse, bool) { + if entry, ok := a.cachedResponses[requestKey]; ok { + // Cache contains a matching entry. + a.evictionSet.Touch(requestKey) + return entry, true + } + return remoteAuthResponse{}, false +} + +func (a *remoteRequestHeadersAuthenticator) insertCacheEntry(requestKey RemoteRequestHeadersAuthenticatorCacheKey, response remoteAuthResponse) { + if _, ok := a.cachedResponses[requestKey]; ok { + a.evictionSet.Touch(requestKey) + } else { + // Cache contains no matching entry. Free up space, so that the + // caller may insert a new entry. + for len(a.cachedResponses) >= a.maximumCacheSize { + delete(a.cachedResponses, a.evictionSet.Peek()) + a.evictionSet.Remove() + } + a.evictionSet.Insert(requestKey) + } + a.cachedResponses[requestKey] = response +} + +func (a *remoteRequestHeadersAuthenticator) authenticateRemotely(ctx context.Context, request *auth_pb.AuthenticateRequest) (remoteAuthResponse, error) { + ret := remoteAuthResponse{ + // The default expirationTime has already passed. + expirationTime: time.Time{}, + } + + response, err := a.remoteAuthClient.Authenticate(ctx, request) + if err != nil { + return ret, util.StatusWrapWithCode(err, codes.Unauthenticated, "Remote authentication failed") + } + + // An invalid expiration time indicates that the response should not be cached. + if response.CacheExpirationTime != nil { + if err := response.CacheExpirationTime.CheckValid(); err != nil { + return ret, util.StatusWrapWithCode(err, codes.Unauthenticated, "Invalid authentication expiration time") + } + // Note that the expiration time might still be valid for non-allow verdicts. + ret.expirationTime = response.CacheExpirationTime.AsTime() + } + + switch verdict := response.Verdict.(type) { + case *auth_pb.AuthenticateResponse_Allow: + ret.authMetadata, err = NewAuthenticationMetadataFromProto(verdict.Allow) + if err != nil { + ret.err = util.StatusWrapWithCode(err, codes.Unauthenticated, "Bad authentication response") + } + case *auth_pb.AuthenticateResponse_Deny: + ret.err = status.Error(codes.Unauthenticated, verdict.Deny) + default: + ret.err = status.Error(codes.Unauthenticated, "Invalid authentication verdict") + } + return ret, nil +} diff --git a/pkg/auth/remote_request_headers_authenticator_test.go b/pkg/auth/remote_request_headers_authenticator_test.go new file mode 100644 index 00000000..0b4cb7ed --- /dev/null +++ b/pkg/auth/remote_request_headers_authenticator_test.go @@ -0,0 +1,414 @@ +package auth_test + +import ( + "context" + "strings" + "testing" + "time" + + "github.com/buildbarn/bb-storage/internal/mock" + "github.com/buildbarn/bb-storage/pkg/auth" + "github.com/buildbarn/bb-storage/pkg/eviction" + auth_pb "github.com/buildbarn/bb-storage/pkg/proto/auth" + "github.com/buildbarn/bb-storage/pkg/testutil" + "github.com/stretchr/testify/require" + "go.uber.org/mock/gomock" + "google.golang.org/grpc" + "google.golang.org/grpc/codes" + "google.golang.org/grpc/status" + "google.golang.org/protobuf/proto" + "google.golang.org/protobuf/types/known/structpb" + "google.golang.org/protobuf/types/known/timestamppb" +) + +func TestRemoteRequestHeadersAuthenticatorFailure(t *testing.T) { + ctrl, ctx := gomock.WithContext(context.Background(), t) + + t.Run("BackendFailure", func(t *testing.T) { + client := mock.NewMockClientConnInterface(ctrl) + clock := mock.NewMockClock(ctrl) + + client.EXPECT().Invoke( + ctx, "/buildbarn.auth.Authentication/Authenticate", gomock.Any(), gomock.Any(), gomock.Any(), + ).Return(status.Error(codes.Unavailable, "Server offline")) + clock.EXPECT().Now().Return(time.Unix(1000, 0)) + + authenticator := auth.NewRemoteRequestHeadersAuthenticator( + client, + structpb.NewStringValue("auth-scope"), + clock, + eviction.NewLRUSet[auth.RemoteRequestHeadersAuthenticatorCacheKey](), + 100, + ) + // authMetadata, err := authenticator.Authenticate(ctx) + _, err := authenticator.Authenticate(ctx, map[string][]string{"Authorization": {"token", "token2"}}) + testutil.RequireEqualStatus( + t, + status.Error(codes.Unauthenticated, "Remote authentication failed: Server offline"), + err) + }) + + t.Run("InvalidVerdict", func(t *testing.T) { + client := mock.NewMockClientConnInterface(ctrl) + clock := mock.NewMockClock(ctrl) + + client.EXPECT().Invoke( + ctx, "/buildbarn.auth.Authentication/Authenticate", gomock.Any(), gomock.Any(), gomock.Any(), + ).DoAndReturn(func(ctx context.Context, method string, args, reply interface{}, opts ...grpc.CallOption) error { + proto.Merge(reply.(proto.Message), &auth_pb.AuthenticateResponse{}) + return nil + }) + clock.EXPECT().Now().Return(time.Unix(1000, 0)) + + authenticator := auth.NewRemoteRequestHeadersAuthenticator( + client, + structpb.NewStringValue("auth-scope"), + clock, + eviction.NewLRUSet[auth.RemoteRequestHeadersAuthenticatorCacheKey](), + 100, + ) + // authMetadata, err := authenticator.Authenticate(ctx) + _, err := authenticator.Authenticate(ctx, map[string][]string{"Authorization": {"token", "token2"}}) + testutil.RequireEqualStatus( + t, + status.Error(codes.Unauthenticated, "Invalid authentication verdict"), + err) + }) +} + +func TestRemoteRequestHeadersAuthenticatorSuccess(t *testing.T) { + ctrl, ctx := gomock.WithContext(context.Background(), t) + + remoteService := func(ctx context.Context, method string, args, reply interface{}, opts ...grpc.CallOption) error { + token := args.(*auth_pb.AuthenticateRequest).RequestMetadata["Authorization"].Value[0] + if strings.HasPrefix(token, "allow") { + proto.Merge(reply.(proto.Message), &auth_pb.AuthenticateResponse{ + Verdict: &auth_pb.AuthenticateResponse_Allow{ + Allow: &auth_pb.AuthenticationMetadata{ + Public: structpb.NewStringValue("You're totally who you say you are: " + token), + }, + }, + CacheExpirationTime: timestamppb.New(time.Unix(1002, 0)), + }) + } else if strings.HasPrefix(token, "deny") { + proto.Merge(reply.(proto.Message), &auth_pb.AuthenticateResponse{ + Verdict: &auth_pb.AuthenticateResponse_Deny{ + Deny: "You are an alien: " + token, + }, + CacheExpirationTime: timestamppb.New(time.Unix(1002, 0)), + }) + } + return nil + } + + authenticateAllowFunc := func(authenticator auth.RequestHeadersAuthenticator, token string) { + authMetadata, err := authenticator.Authenticate(ctx, map[string][]string{"Authorization": {token}}) + require.NoError(t, err) + require.Equal(t, map[string]any{ + "public": "You're totally who you say you are: " + token, + }, authMetadata.GetRaw()) + } + authenticateDenyFunc := func(authenticator auth.RequestHeadersAuthenticator, token string) { + _, err := authenticator.Authenticate(ctx, map[string][]string{"Authorization": {token}}) + testutil.RequireEqualStatus( + t, + status.Error(codes.Unauthenticated, "You are an alien: "+token), + err) + } + + t.Run("SuccessAllow", func(t *testing.T) { + client := mock.NewMockClientConnInterface(ctrl) + clock := mock.NewMockClock(ctrl) + + client.EXPECT().Invoke( + ctx, + "/buildbarn.auth.Authentication/Authenticate", + testutil.EqProto(t, &auth_pb.AuthenticateRequest{ + RequestMetadata: map[string]*auth_pb.AuthenticateRequest_ValueList{ + "Authorization": { + Value: []string{"allow1", "token2"}, + }, + }, + Scope: structpb.NewStringValue("auth-scope"), + }), + gomock.Any(), + gomock.Any(), + ).DoAndReturn(remoteService) + clock.EXPECT().Now().Return(time.Unix(1000, 0)) + + authenticator := auth.NewRemoteRequestHeadersAuthenticator( + client, + structpb.NewStringValue("auth-scope"), + clock, + eviction.NewLRUSet[auth.RemoteRequestHeadersAuthenticatorCacheKey](), + 100, + ) + authMetadata, err := authenticator.Authenticate(ctx, map[string][]string{"Authorization": {"allow1", "token2"}}) + require.NoError(t, err) + require.Equal(t, map[string]any{ + "public": "You're totally who you say you are: allow1", + }, authMetadata.GetRaw()) + }) + + t.Run("SuccessDeny", func(t *testing.T) { + client := mock.NewMockClientConnInterface(ctrl) + clock := mock.NewMockClock(ctrl) + + client.EXPECT().Invoke( + ctx, + "/buildbarn.auth.Authentication/Authenticate", + testutil.EqProto(t, &auth_pb.AuthenticateRequest{ + RequestMetadata: map[string]*auth_pb.AuthenticateRequest_ValueList{ + "Authorization": { + Value: []string{"deny3", "token4"}, + }, + }, + Scope: structpb.NewStringValue("auth-scope"), + }), + gomock.Any(), + gomock.Any(), + ).DoAndReturn(remoteService) + clock.EXPECT().Now().Return(time.Unix(1000, 0)) + + authenticator := auth.NewRemoteRequestHeadersAuthenticator( + client, + structpb.NewStringValue("auth-scope"), + clock, + eviction.NewLRUSet[auth.RemoteRequestHeadersAuthenticatorCacheKey](), + 100, + ) + _, err := authenticator.Authenticate(ctx, map[string][]string{"Authorization": {"deny3", "token4"}}) + testutil.RequireEqualStatus( + t, + status.Error(codes.Unauthenticated, "You are an alien: deny3"), + err) + }) + + t.Run("ExpireResponses", func(t *testing.T) { + client := mock.NewMockClientConnInterface(ctrl) + clock := mock.NewMockClock(ctrl) + + authenticator := auth.NewRemoteRequestHeadersAuthenticator( + client, + structpb.NewStringValue("auth-scope"), + clock, + eviction.NewLRUSet[auth.RemoteRequestHeadersAuthenticatorCacheKey](), + 100, + ) + + // First request should hit the backend. + // Second request should hit the cache. + // Third request should hit the backend again, as the cache entry has expired. + for _, timestamp := range []int64{1000, 1001, 1002} { + if timestamp != 1001 { + client.EXPECT().Invoke( + ctx, "/buildbarn.auth.Authentication/Authenticate", gomock.Any(), gomock.Any(), gomock.Any(), + ).DoAndReturn(remoteService).Times(2) // Times 2 for both allow and deny calls. + } + clock.EXPECT().Now().Return(time.Unix(timestamp, 0)) + authenticateAllowFunc(authenticator, "allow") + clock.EXPECT().Now().Return(time.Unix(timestamp, 0)) + authenticateDenyFunc(authenticator, "deny") + } + }) + + t.Run("MaxCacheSizeAndLRUEviction", func(t *testing.T) { + client := mock.NewMockClientConnInterface(ctrl) + clock := mock.NewMockClock(ctrl) + + clock.EXPECT().Now().Return(time.Unix(1000, 0)).AnyTimes() + + authenticator := auth.NewRemoteRequestHeadersAuthenticator( + client, + structpb.NewStringValue("auth-scope"), + clock, + eviction.NewLRUSet[auth.RemoteRequestHeadersAuthenticatorCacheKey](), + 2, // Only two spaces in this test. + ) + + // 1 uncached. + client.EXPECT().Invoke(ctx, "/buildbarn.auth.Authentication/Authenticate", gomock.Any(), gomock.Any(), gomock.Any()).DoAndReturn(remoteService) + authenticateAllowFunc(authenticator, "allow1") + // 2 uncached. + client.EXPECT().Invoke(ctx, "/buildbarn.auth.Authentication/Authenticate", gomock.Any(), gomock.Any(), gomock.Any()).DoAndReturn(remoteService) + authenticateDenyFunc(authenticator, "deny2") + // 1 should be cached. + authenticateAllowFunc(authenticator, "allow1") + // 3 uncached, replacing 2. + client.EXPECT().Invoke(ctx, "/buildbarn.auth.Authentication/Authenticate", gomock.Any(), gomock.Any(), gomock.Any()).DoAndReturn(remoteService) + authenticateAllowFunc(authenticator, "allow3") + // 1 should still be cached. + authenticateAllowFunc(authenticator, "allow1") + // 2 should have been evicted. + client.EXPECT().Invoke(ctx, "/buildbarn.auth.Authentication/Authenticate", gomock.Any(), gomock.Any(), gomock.Any()).DoAndReturn(remoteService) + authenticateDenyFunc(authenticator, "deny2") + }) + + t.Run("DeduplicateCalls", func(t *testing.T) { + client := mock.NewMockClientConnInterface(ctrl) + clock := mock.NewMockClock(ctrl) + + authCalled := make(chan struct{}) + authRelease := make(map[string]chan struct{}) + + client.EXPECT().Invoke( + ctx, "/buildbarn.auth.Authentication/Authenticate", gomock.Any(), gomock.Any(), gomock.Any(), + ).DoAndReturn(func(ctx context.Context, method string, args, reply interface{}, opts ...grpc.CallOption) error { + token := args.(*auth_pb.AuthenticateRequest).RequestMetadata["Authorization"].Value[0] + proto.Merge(reply.(proto.Message), &auth_pb.AuthenticateResponse{}) + authCalled <- struct{}{} + <-authRelease[token] + return nil + }).Times(2) // token1 and token2 + + clock.EXPECT().Now().Return(time.Unix(1000, 0)).AnyTimes() + + authenticator := auth.NewRemoteRequestHeadersAuthenticator( + client, + structpb.NewStringValue("auth-scope"), + clock, + eviction.NewLRUSet[auth.RemoteRequestHeadersAuthenticatorCacheKey](), + 100, + ) + doAuth := func(token string, done chan<- struct{}) { + _, err := authenticator.Authenticate(ctx, map[string][]string{"Authorization": {token}}) + testutil.RequireEqualStatus( + t, + status.Error(codes.Unauthenticated, "Invalid authentication verdict"), + err) + close(done) + } + authRelease["token1"] = make(chan struct{}) + authRelease["token2"] = make(chan struct{}) + done1a := make(chan struct{}) + done1b := make(chan struct{}) + done1c := make(chan struct{}) + done2 := make(chan struct{}) + go doAuth("token1", done1a) + <-authCalled + go doAuth("token2", done2) + <-authCalled + go doAuth("token1", done1b) + ctx1c, cancel1c := context.WithCancel(ctx) + go func() { + _, err := authenticator.Authenticate(ctx1c, map[string][]string{"Authorization": {"token1"}}) + testutil.RequireEqualStatus( + t, + status.Error(codes.Canceled, "context canceled"), + err) + close(done1c) + }() + // Nothing done yet. + time.Sleep(100 * time.Millisecond) + select { + case <-done1a: + t.Error("done1a too early") + case <-done1b: + t.Error("done1b too early") + case <-done1c: + t.Error("done1c too early") + case <-done2: + t.Error("done2 too early") + default: + // Noop. + } + cancel1c() + <-done1c + close(authRelease["token2"]) + // token1 still blocked. + time.Sleep(100 * time.Millisecond) + select { + case <-done1a: + t.Error("done1a too early") + case <-done1b: + t.Error("done1b too early") + case <-done2: + // Noop. + } + close(authRelease["token1"]) + <-done1a + <-done1b + }) + + t.Run("SkipDeduplicateErrors", func(t *testing.T) { + client := mock.NewMockClientConnInterface(ctrl) + clock := mock.NewMockClock(ctrl) + + authCalled := make(chan struct{}) + authRelease := make(map[string]chan struct{}) + + client.EXPECT().Invoke( + ctx, "/buildbarn.auth.Authentication/Authenticate", gomock.Any(), gomock.Any(), gomock.Any(), + ).DoAndReturn(func(ctx context.Context, method string, args, reply interface{}, opts ...grpc.CallOption) error { + token := args.(*auth_pb.AuthenticateRequest).RequestMetadata["Authorization"].Value[0] + proto.Merge(reply.(proto.Message), &auth_pb.AuthenticateResponse{}) + authCalled <- struct{}{} + <-authRelease[token] + return status.Error(codes.DataLoss, "Data loss") + }) + client.EXPECT().Invoke( + ctx, "/buildbarn.auth.Authentication/Authenticate", gomock.Any(), gomock.Any(), gomock.Any(), + ).DoAndReturn(func(ctx context.Context, method string, args, reply interface{}, opts ...grpc.CallOption) error { + token := args.(*auth_pb.AuthenticateRequest).RequestMetadata["Authorization"].Value[0] + proto.Merge(reply.(proto.Message), &auth_pb.AuthenticateResponse{}) + authCalled <- struct{}{} + <-authRelease[token] + return nil + }) + + clock.EXPECT().Now().Return(time.Unix(1000, 0)).AnyTimes() + + authenticator := auth.NewRemoteRequestHeadersAuthenticator( + client, + structpb.NewStringValue("auth-scope"), + clock, + eviction.NewLRUSet[auth.RemoteRequestHeadersAuthenticatorCacheKey](), + 100, + ) + doAuth := func(token string, done chan<- struct{}, verdict string) { + _, err := authenticator.Authenticate(ctx, map[string][]string{"Authorization": {token}}) + defer close(done) + testutil.RequireEqualStatus( + t, + status.Error(codes.Unauthenticated, verdict), + err) + } + + authRelease["token1"] = make(chan struct{}) + done1a := make(chan struct{}) + done1b := make(chan struct{}) + done1c := make(chan struct{}) + go doAuth("token1", done1a, "Remote authentication failed: Data loss") + <-authCalled // token1a + go doAuth("token1", done1b, "Invalid authentication verdict") + go doAuth("token1", done1c, "Invalid authentication verdict") + // Nothing done yet. + time.Sleep(100 * time.Millisecond) + select { + case <-done1a: + t.Error("done1a too early") + case <-done1b: + t.Error("done1b too early") + case <-done1c: + t.Error("done1c too early") + case <-authCalled: + t.Error("authCalled second time too early") + default: + // Noop. + } + close(authRelease["token1"]) + // token1 still blocked. + time.Sleep(100 * time.Millisecond) + select { + case <-done1b: + t.Error("done1b too early") + case <-done1c: + t.Error("done1c too early") + case <-authCalled: + // token1b released. + // Noop. + } + <-done1b + <-done1c + }) +} diff --git a/pkg/auth/request_headers_authenticator.go b/pkg/auth/request_headers_authenticator.go new file mode 100644 index 00000000..263e8d98 --- /dev/null +++ b/pkg/auth/request_headers_authenticator.go @@ -0,0 +1,11 @@ +package auth + +import ( + "context" +) + +// RequestHeadersAuthenticator can be used to grant or deny access to a server +// based on request headers, typically from an HTTP or gRPC request. +type RequestHeadersAuthenticator interface { + Authenticate(ctx context.Context, headers map[string][]string) (*AuthenticationMetadata, error) +} diff --git a/pkg/global/apply_configuration.go b/pkg/global/apply_configuration.go index 1aca56ab..5fe9073f 100644 --- a/pkg/global/apply_configuration.go +++ b/pkg/global/apply_configuration.go @@ -58,7 +58,7 @@ type LifecycleState struct { // MarkReadyAndWait can be called to report that the program has started // successfully. The application should now be reported as being healthy // and ready, and receive incoming requests if applicable. -func (ls *LifecycleState) MarkReadyAndWait(group program.Group) { +func (ls *LifecycleState) MarkReadyAndWait(group program.Group, grpcClientFactory bb_grpc.ClientFactory) { // Start a diagnostics web server that exposes Prometheus // metrics and provides a health check endpoint. if ls.config != nil { @@ -77,7 +77,8 @@ func (ls *LifecycleState) MarkReadyAndWait(group program.Group) { bb_http.NewServersFromConfigurationAndServe( ls.config.HttpServers, bb_http.NewMetricsHandler(router, "Diagnostics"), - group) + group, + grpcClientFactory) } } diff --git a/pkg/grpc/BUILD.bazel b/pkg/grpc/BUILD.bazel index 7d2b7107..ce529cf7 100644 --- a/pkg/grpc/BUILD.bazel +++ b/pkg/grpc/BUILD.bazel @@ -14,7 +14,6 @@ go_library( "deduplicating_client_factory.go", "deny_authenticator.go", "jmespath_extractor.go", - "jwt_authenticator.go", "lazy_client_dialer.go", "metadata_adding_interceptor.go", "metadata_extracting_and_forwarding_interceptor.go", @@ -27,6 +26,7 @@ go_library( "peer_transport_credentials_linux.go", "proto_trace_attributes_extractor.go", "proxy_dialer.go", + "request_headers_authenticator.go", "request_metadata_tracing_interceptor.go", "server.go", "tls_client_certificate_authenticator.go", @@ -36,6 +36,7 @@ go_library( deps = [ "//pkg/auth", "//pkg/clock", + "//pkg/eviction", "//pkg/jwt", "//pkg/program", "//pkg/proto/auth", @@ -100,6 +101,7 @@ go_test( "metadata_forwarding_and_reusing_interceptor_test.go", "peer_credentials_authenticator_test.go", "proto_trace_attributes_extractor_test.go", + "request_headers_authenticator_test.go", "request_metadata_tracing_interceptor_test.go", "tls_client_certificate_authenticator_test.go", ] + select({ diff --git a/pkg/grpc/authenticator.go b/pkg/grpc/authenticator.go index 79b282ec..05470f05 100644 --- a/pkg/grpc/authenticator.go +++ b/pkg/grpc/authenticator.go @@ -6,6 +6,7 @@ import ( "github.com/buildbarn/bb-storage/pkg/auth" "github.com/buildbarn/bb-storage/pkg/clock" + "github.com/buildbarn/bb-storage/pkg/eviction" "github.com/buildbarn/bb-storage/pkg/jwt" "github.com/buildbarn/bb-storage/pkg/program" configuration "github.com/buildbarn/bb-storage/pkg/proto/configuration/grpc" @@ -25,7 +26,7 @@ type Authenticator interface { // NewAuthenticatorFromConfiguration creates a tree of Authenticator // objects based on a configuration file. -func NewAuthenticatorFromConfiguration(policy *configuration.AuthenticationPolicy, group program.Group) (Authenticator, bool, bool, error) { +func NewAuthenticatorFromConfiguration(policy *configuration.AuthenticationPolicy, group program.Group, grpcClientFactory ClientFactory) (Authenticator, bool, bool, error) { if policy == nil { return nil, false, false, status.Error(codes.InvalidArgument, "Authentication policy not specified") } @@ -41,7 +42,7 @@ func NewAuthenticatorFromConfiguration(policy *configuration.AuthenticationPolic needsPeerTransportCredentials := false requestTLSClientCertificate := false for _, childConfiguration := range policyKind.Any.Policies { - child, childNeedsPeerTransportCredentials, childRequestTLSClientCertificate, err := NewAuthenticatorFromConfiguration(childConfiguration, group) + child, childNeedsPeerTransportCredentials, childRequestTLSClientCertificate, err := NewAuthenticatorFromConfiguration(childConfiguration, group, grpcClientFactory) if err != nil { return nil, false, false, err } @@ -55,7 +56,7 @@ func NewAuthenticatorFromConfiguration(policy *configuration.AuthenticationPolic needsPeerTransportCredentials := false requestTLSClientCertificate := false for _, childConfiguration := range policyKind.All.Policies { - child, childNeedsPeerTransportCredentials, childRequestTLSClientCertificate, err := NewAuthenticatorFromConfiguration(childConfiguration, group) + child, childNeedsPeerTransportCredentials, childRequestTLSClientCertificate, err := NewAuthenticatorFromConfiguration(childConfiguration, group, grpcClientFactory) if err != nil { return nil, false, false, err } @@ -90,14 +91,54 @@ func NewAuthenticatorFromConfiguration(policy *configuration.AuthenticationPolic if err != nil { return nil, false, false, util.StatusWrap(err, "Failed to create authorization header parser for JWT authentication policy") } - return NewJWTAuthenticator(authorizationHeaderParser), false, false, nil + return NewRequestHeadersAuthenticator(authorizationHeaderParser, []string{jwt.AuthorizationHeaderName}), false, false, nil case *configuration.AuthenticationPolicy_PeerCredentialsJmespathExpression: metadataExtractor, err := jmespath.Compile(policyKind.PeerCredentialsJmespathExpression) if err != nil { return nil, false, false, util.StatusWrap(err, "Failed to compile peer credentials metadata extraction JMESPath expression") } return NewPeerCredentialsAuthenticator(metadataExtractor), true, false, nil + case *configuration.AuthenticationPolicy_Remote: + // TODO: With auth.RequestHeadersPolicy = oneof {auth.Jwt, auth.Remote} + // in the .proto definitions, the HTTP and gRPC authentication policy + // code could be unified. Unfortunately, that creates the .proto + // dependency cycle below: + // + // grpc.ServerConfiguration -> + // grpc.AuthenticationPolicy -> + // auth.RequestHeadersAuthenticator -> + // auth.RemoteAuthenticator -> + // grpc.ClientConfiguration + // + // Resolving this requires splitting `grpc.proto` into `grpc_client.proto`, + // `grpc_server.proto` and `grpc_tracing_method.proto`. + authenticator, err := NewRemoteRequestHeadersAuthenticatorFromConfiguration(policyKind.Remote, grpcClientFactory) + if err != nil { + return nil, false, false, err + } + return NewRequestHeadersAuthenticator(authenticator, policyKind.Remote.Headers), false, false, nil default: return nil, false, false, status.Error(codes.InvalidArgument, "Configuration did not contain an authentication policy type") } } + +// NewRemoteRequestHeadersAuthenticatorFromConfiguration creates an +// Authenticator that forwards authentication requests to a remote gRPC service. +// This is a convenient way to integrate custom authentication processes. +func NewRemoteRequestHeadersAuthenticatorFromConfiguration(configuration *configuration.RemoteAuthenticationPolicy, grpcClientFactory ClientFactory) (auth.RequestHeadersAuthenticator, error) { + grpcClient, err := grpcClientFactory.NewClientFromConfiguration(configuration.Endpoint) + if err != nil { + return nil, util.StatusWrap(err, "Failed to create authenticator RPC client") + } + evictionSet, err := eviction.NewSetFromConfiguration[auth.RemoteRequestHeadersAuthenticatorCacheKey](configuration.CacheReplacementPolicy) + if err != nil { + return nil, util.StatusWrap(err, "Cache replacement policy for remote authentication") + } + return auth.NewRemoteRequestHeadersAuthenticator( + grpcClient, + configuration.Scope, + clock.SystemClock, + eviction.NewMetricsSet(evictionSet, "remote_authenticator"), + int(configuration.MaximumCacheSize), + ), nil +} diff --git a/pkg/grpc/jwt_authenticator.go b/pkg/grpc/jwt_authenticator.go deleted file mode 100644 index e82af509..00000000 --- a/pkg/grpc/jwt_authenticator.go +++ /dev/null @@ -1,37 +0,0 @@ -package grpc - -import ( - "context" - - "github.com/buildbarn/bb-storage/pkg/auth" - "github.com/buildbarn/bb-storage/pkg/jwt" - - "google.golang.org/grpc/codes" - "google.golang.org/grpc/metadata" - "google.golang.org/grpc/status" -) - -type jwtAuthenticator struct { - authorizationHeaderParser *jwt.AuthorizationHeaderParser -} - -// NewJWTAuthenticator creates an authenticator for incoming gRPC -// requests that validates requests that contain an "Authorization" of -// shape "Bearer ${jwt}", where ${jwt} is a valid JSON Web Token. -func NewJWTAuthenticator(authorizationHeaderParser *jwt.AuthorizationHeaderParser) Authenticator { - return &jwtAuthenticator{ - authorizationHeaderParser: authorizationHeaderParser, - } -} - -func (a *jwtAuthenticator) Authenticate(ctx context.Context) (*auth.AuthenticationMetadata, error) { - grpcMetadata, ok := metadata.FromIncomingContext(ctx) - if !ok { - return nil, status.Error(codes.Unauthenticated, "Not called from within an incoming gRPC context") - } - metadata, ok := a.authorizationHeaderParser.ParseAuthorizationHeaders(grpcMetadata.Get("authorization")) - if !ok { - return nil, status.Error(codes.Unauthenticated, "No valid authorization header containing a bearer token provided") - } - return metadata, nil -} diff --git a/pkg/grpc/request_headers_authenticator.go b/pkg/grpc/request_headers_authenticator.go new file mode 100644 index 00000000..1af147b1 --- /dev/null +++ b/pkg/grpc/request_headers_authenticator.go @@ -0,0 +1,42 @@ +package grpc + +import ( + "context" + + "github.com/buildbarn/bb-storage/pkg/auth" + "google.golang.org/grpc/codes" + "google.golang.org/grpc/metadata" + "google.golang.org/grpc/status" +) + +type requestHeadersAuthenticator struct { + authenticator auth.RequestHeadersAuthenticator + headerKeys []string +} + +// NewRequestHeadersAuthenticator creates a new Authenticator for incoming gRPC +// requests that forwards configured headers to a remote service for +// authentication. The result from the remote service is cached. +func NewRequestHeadersAuthenticator( + authenticator auth.RequestHeadersAuthenticator, + headerKeys []string, +) Authenticator { + return &requestHeadersAuthenticator{ + authenticator: authenticator, + headerKeys: headerKeys, + } +} + +func (a *requestHeadersAuthenticator) Authenticate(ctx context.Context) (*auth.AuthenticationMetadata, error) { + metadata, ok := metadata.FromIncomingContext(ctx) + if !ok { + return nil, status.Error(codes.Unauthenticated, "Not called from within an incoming gRPC context") + } + requestHeaders := make(map[string][]string, len(a.headerKeys)) + for _, key := range a.headerKeys { + if values := metadata.Get(key); len(values) != 0 { + requestHeaders[key] = values + } + } + return a.authenticator.Authenticate(ctx, requestHeaders) +} diff --git a/pkg/grpc/request_headers_authenticator_test.go b/pkg/grpc/request_headers_authenticator_test.go new file mode 100644 index 00000000..a1650b9f --- /dev/null +++ b/pkg/grpc/request_headers_authenticator_test.go @@ -0,0 +1,76 @@ +package grpc_test + +import ( + "context" + "testing" + + "github.com/buildbarn/bb-storage/internal/mock" + "github.com/buildbarn/bb-storage/pkg/auth" + bb_grpc "github.com/buildbarn/bb-storage/pkg/grpc" + auth_pb "github.com/buildbarn/bb-storage/pkg/proto/auth" + "github.com/buildbarn/bb-storage/pkg/testutil" + "github.com/stretchr/testify/require" + "go.uber.org/mock/gomock" + "google.golang.org/grpc/codes" + "google.golang.org/grpc/metadata" + "google.golang.org/grpc/status" + "google.golang.org/protobuf/types/known/structpb" +) + +func TestRequestHeadersAuthenticator(t *testing.T) { + ctrl, ctx := gomock.WithContext(context.Background(), t) + md := metadata.New( + map[string]string{ + "Authorization": "token", + "OtherHeader": "Other", + "UnusedHeader": "DontUseMe", + }, + ) + md.Append("OtherHeader", "Other2") + grpcCtx := metadata.NewIncomingContext(ctx, md) + backend := mock.NewMockRequestHeadersAuthenticator(ctrl) + + t.Run("BackendSuccess", func(t *testing.T) { + backend.EXPECT().Authenticate( + grpcCtx, map[string][]string{ + "Authorization": {"token"}, + "OtherHEADER": {"Other", "Other2"}, + // MissingHeader is not part of the request. + // UnusedHeader should not be passed. + }, + ).Return(auth.MustNewAuthenticationMetadataFromProto(&auth_pb.AuthenticationMetadata{ + Public: structpb.NewStringValue("You're totally who you say you are"), + }), nil) + + authenticator := bb_grpc.NewRequestHeadersAuthenticator( + backend, + []string{ + "Authorization", + "OtherHEADER", + "MissingHeader", + }, + ) + metadata, err := authenticator.Authenticate(grpcCtx) + require.NoError(t, err) + require.Equal(t, map[string]any{ + "public": "You're totally who you say you are", + }, metadata.GetRaw()) + }) + + t.Run("BackendFailure", func(t *testing.T) { + headerKeys := []string{} + backend.EXPECT().Authenticate( + grpcCtx, map[string][]string{}, + ).Return(nil, status.Error(codes.Unavailable, "Server offline")) + + authenticator := bb_grpc.NewRequestHeadersAuthenticator( + backend, + headerKeys, + ) + _, err := authenticator.Authenticate(grpcCtx) + testutil.RequireEqualStatus( + t, + status.Error(codes.Unavailable, "Server offline"), + err) + }) +} diff --git a/pkg/grpc/server.go b/pkg/grpc/server.go index d2440b9f..278b129a 100644 --- a/pkg/grpc/server.go +++ b/pkg/grpc/server.go @@ -33,10 +33,10 @@ func init() { // based on a configuration stored in a list of Protobuf messages. It // then lets all of these gRPC servers listen on the network addresses // of UNIX socket paths provided. -func NewServersFromConfigurationAndServe(configurations []*configuration.ServerConfiguration, registrationFunc func(grpc.ServiceRegistrar), group program.Group) error { +func NewServersFromConfigurationAndServe(configurations []*configuration.ServerConfiguration, registrationFunc func(grpc.ServiceRegistrar), group program.Group, grpcClientFactory ClientFactory) error { for _, configuration := range configurations { // Create an authenticator for requests. - authenticator, needsPeerTransportCredentials, requestTLSClientCertificate, err := NewAuthenticatorFromConfiguration(configuration.AuthenticationPolicy, group) + authenticator, needsPeerTransportCredentials, requestTLSClientCertificate, err := NewAuthenticatorFromConfiguration(configuration.AuthenticationPolicy, group, grpcClientFactory) if err != nil { return err } diff --git a/pkg/http/BUILD.bazel b/pkg/http/BUILD.bazel index 1ab0ea5b..864a9cb8 100644 --- a/pkg/http/BUILD.bazel +++ b/pkg/http/BUILD.bazel @@ -11,10 +11,10 @@ go_library( "configuration.go", "deny_authenticator.go", "header_adding_round_tripper.go", - "jwt_authenticator.go", "metrics_handler.go", "metrics_round_tripper.go", "oidc_authenticator.go", + "request_headers_authenticator.go", "server.go", "status_code.go", ], @@ -23,6 +23,7 @@ go_library( deps = [ "//pkg/auth", "//pkg/clock", + "//pkg/grpc", "//pkg/jwt", "//pkg/program", "//pkg/proto/configuration/http", @@ -49,6 +50,7 @@ go_test( "allow_authenticator_test.go", "deny_authenticator_test.go", "oidc_authenticator_test.go", + "request_headers_authenticator_test.go", ], deps = [ ":http", diff --git a/pkg/http/authenticator.go b/pkg/http/authenticator.go index 6db0f7f5..37a015e2 100644 --- a/pkg/http/authenticator.go +++ b/pkg/http/authenticator.go @@ -9,6 +9,7 @@ import ( "github.com/buildbarn/bb-storage/pkg/auth" "github.com/buildbarn/bb-storage/pkg/clock" + "github.com/buildbarn/bb-storage/pkg/grpc" "github.com/buildbarn/bb-storage/pkg/jwt" "github.com/buildbarn/bb-storage/pkg/program" configuration "github.com/buildbarn/bb-storage/pkg/proto/configuration/http" @@ -31,7 +32,7 @@ type Authenticator interface { // NewAuthenticatorFromConfiguration creates a tree of Authenticator // objects based on a configuration file. -func NewAuthenticatorFromConfiguration(policy *configuration.AuthenticationPolicy, group program.Group) (Authenticator, error) { +func NewAuthenticatorFromConfiguration(policy *configuration.AuthenticationPolicy, group program.Group, grpcClientFactory grpc.ClientFactory) (Authenticator, error) { if policy == nil { return nil, status.Error(codes.InvalidArgument, "Authentication policy not specified") } @@ -45,7 +46,7 @@ func NewAuthenticatorFromConfiguration(policy *configuration.AuthenticationPolic case *configuration.AuthenticationPolicy_Any: children := make([]Authenticator, 0, len(policyKind.Any.Policies)) for _, childConfiguration := range policyKind.Any.Policies { - child, err := NewAuthenticatorFromConfiguration(childConfiguration, group) + child, err := NewAuthenticatorFromConfiguration(childConfiguration, group, grpcClientFactory) if err != nil { return nil, err } @@ -59,7 +60,11 @@ func NewAuthenticatorFromConfiguration(policy *configuration.AuthenticationPolic if err != nil { return nil, util.StatusWrap(err, "Failed to create authorization header parser for JWT authentication policy") } - return NewJWTAuthenticator(authorizationHeaderParser), nil + authenticator, err := NewRequestHeadersAuthenticator(authorizationHeaderParser, []string{jwt.AuthorizationHeaderName}) + if err != nil { + return nil, util.StatusWrap(err, "Failed to create request headers authenticator for JWT authentication policy") + } + return authenticator, nil case *configuration.AuthenticationPolicy_Oidc: // Select a name and encryption key for the session // state cookie. Even though the configuration has a @@ -119,11 +124,30 @@ func NewAuthenticatorFromConfiguration(policy *configuration.AuthenticationPolic cookieAEAD, clock.SystemClock) case *configuration.AuthenticationPolicy_AcceptHeader: - base, err := NewAuthenticatorFromConfiguration(policyKind.AcceptHeader.Policy, group) + base, err := NewAuthenticatorFromConfiguration(policyKind.AcceptHeader.Policy, group, grpcClientFactory) if err != nil { return nil, err } return NewAcceptHeaderAuthenticator(base, policyKind.AcceptHeader.MediaTypes), nil + case *configuration.AuthenticationPolicy_Remote: + // TODO: With auth.RequestHeadersPolicy = oneof {auth.Jwt, auth.Remote} + // in the .proto definitions, the HTTP and gRPC authentication policy + // code could be unified. Unfortunately, that creates the .proto + // dependency cycle below: + // + // grpc.ServerConfiguration -> + // grpc.AuthenticationPolicy -> + // auth.RequestHeadersAuthenticator -> + // auth.RemoteAuthenticator -> + // grpc.ClientConfiguration + // + // Resolving this requires splitting `grpc.proto` into `grpc_client.proto`, + // `grpc_server.proto` and `grpc_tracing_method.proto`. + backend, err := grpc.NewRemoteRequestHeadersAuthenticatorFromConfiguration(policyKind.Remote, grpcClientFactory) + if err != nil { + return nil, err + } + return NewRequestHeadersAuthenticator(backend, policyKind.Remote.Headers) default: return nil, status.Error(codes.InvalidArgument, "Configuration did not contain an authentication policy type") } diff --git a/pkg/http/jwt_authenticator.go b/pkg/http/jwt_authenticator.go deleted file mode 100644 index 9e0565bf..00000000 --- a/pkg/http/jwt_authenticator.go +++ /dev/null @@ -1,32 +0,0 @@ -package http - -import ( - "net/http" - - "github.com/buildbarn/bb-storage/pkg/auth" - "github.com/buildbarn/bb-storage/pkg/jwt" - - "google.golang.org/grpc/codes" - "google.golang.org/grpc/status" -) - -type jwtAuthenticator struct { - authorizationHeaderParser *jwt.AuthorizationHeaderParser -} - -// NewJWTAuthenticator creates an authenticator for incoming HTTP -// requests that validates requests that contain an "Authorization" of -// shape "Bearer ${jwt}", where ${jwt} is a valid JSON Web Token. -func NewJWTAuthenticator(authorizationHeaderParser *jwt.AuthorizationHeaderParser) Authenticator { - return &jwtAuthenticator{ - authorizationHeaderParser: authorizationHeaderParser, - } -} - -func (a *jwtAuthenticator) Authenticate(w http.ResponseWriter, r *http.Request) (*auth.AuthenticationMetadata, error) { - metadata, ok := a.authorizationHeaderParser.ParseAuthorizationHeaders(r.Header["Authorization"]) - if !ok { - return nil, status.Error(codes.Unauthenticated, "No valid authorization header containing a bearer token provided") - } - return metadata, nil -} diff --git a/pkg/http/request_headers_authenticator.go b/pkg/http/request_headers_authenticator.go new file mode 100644 index 00000000..b77c2d1f --- /dev/null +++ b/pkg/http/request_headers_authenticator.go @@ -0,0 +1,42 @@ +package http + +import ( + "net/http" + + "github.com/buildbarn/bb-storage/pkg/auth" + "google.golang.org/grpc/codes" + "google.golang.org/grpc/status" +) + +type requestHeadersAuthenticator struct { + backend auth.RequestHeadersAuthenticator + headerKeys []string +} + +// NewRequestHeadersAuthenticator creates a new Authenticator for incoming gRPC +// requests that forwards configured headers to a remote service for +// authentication. The result from the remote service is cached. +func NewRequestHeadersAuthenticator( + backend auth.RequestHeadersAuthenticator, + headerKeys []string, +) (Authenticator, error) { + for _, key := range headerKeys { + if canonicalHeaderKey := http.CanonicalHeaderKey(key); canonicalHeaderKey != key { + return nil, status.Errorf(codes.InvalidArgument, "Header key %#v is not canonical, did you mean %#v?", key, canonicalHeaderKey) + } + } + return &requestHeadersAuthenticator{ + backend: backend, + headerKeys: headerKeys, + }, nil +} + +func (a *requestHeadersAuthenticator) Authenticate(w http.ResponseWriter, r *http.Request) (*auth.AuthenticationMetadata, error) { + requestHeaders := make(map[string][]string, len(a.headerKeys)) + for _, key := range a.headerKeys { + if values, ok := r.Header[key]; ok { + requestHeaders[key] = values + } + } + return a.backend.Authenticate(r.Context(), requestHeaders) +} diff --git a/pkg/http/request_headers_authenticator_test.go b/pkg/http/request_headers_authenticator_test.go new file mode 100644 index 00000000..7675ee5f --- /dev/null +++ b/pkg/http/request_headers_authenticator_test.go @@ -0,0 +1,95 @@ +package http_test + +import ( + "context" + "net/http" + "testing" + + "github.com/buildbarn/bb-storage/internal/mock" + "github.com/buildbarn/bb-storage/pkg/auth" + bb_http "github.com/buildbarn/bb-storage/pkg/http" + auth_pb "github.com/buildbarn/bb-storage/pkg/proto/auth" + "github.com/buildbarn/bb-storage/pkg/testutil" + "github.com/stretchr/testify/require" + "go.uber.org/mock/gomock" + "google.golang.org/grpc/codes" + "google.golang.org/grpc/status" + "google.golang.org/protobuf/types/known/structpb" +) + +func TestRequestHeadersAuthenticator(t *testing.T) { + ctrl, ctx := gomock.WithContext(context.Background(), t) + + backend := mock.NewMockRequestHeadersAuthenticator(ctrl) + + t.Run("BackendSuccess", func(t *testing.T) { + r, err := http.NewRequestWithContext(ctx, http.MethodGet, "/path", nil) + require.NoError(t, err) + r.Header.Set("aUTHORIZATION", "token") + r.Header["Other-Header"] = []string{"Other", "Other2"} + r.Header.Set("Unused-Header", "DontUseMe") + + backend.EXPECT().Authenticate( + ctx, map[string][]string{ + "Authorization": {"token"}, + "Other-Header": {"Other", "Other2"}, + // MissingHeader is not part of the request. + // UnusedHeader should not be passed. + }, + ).Return(auth.MustNewAuthenticationMetadataFromProto(&auth_pb.AuthenticationMetadata{ + Public: structpb.NewStringValue("You're totally who you say you are"), + }), nil) + + authenticator, err := bb_http.NewRequestHeadersAuthenticator( + backend, + []string{ + "Authorization", + "Other-Header", + "Missing-Header", + }, + ) + require.NoError(t, err) + + metadata, err := authenticator.Authenticate(nil, r) + require.NoError(t, err) + require.Equal(t, map[string]any{ + "public": "You're totally who you say you are", + }, metadata.GetRaw()) + }) + + t.Run("BackendFailure", func(t *testing.T) { + r, err := http.NewRequestWithContext(ctx, http.MethodGet, "/path", nil) + require.NoError(t, err) + backend.EXPECT().Authenticate( + ctx, map[string][]string{}, + ).Return(nil, status.Error(codes.Unauthenticated, "Server offline")) + + authenticator, err := bb_http.NewRequestHeadersAuthenticator( + backend, + []string{}, + ) + require.NoError(t, err) + + _, err = authenticator.Authenticate(nil, r) + testutil.RequireEqualStatus( + t, + status.Error(codes.Unauthenticated, "Server offline"), + err) + }) + + // The current implementation forwards headers in canonical form, so don't + // allow configuring headers in other forms as that may confuse the users. + t.Run("OnlyAcceptCanonicalHeaders", func(t *testing.T) { + _, err := bb_http.NewRequestHeadersAuthenticator( + backend, + []string{"Non-CANONICAL-Header"}, + ) + testutil.RequireEqualStatus( + t, + status.Error( + codes.InvalidArgument, + "Header key \"Non-CANONICAL-Header\" is not canonical, did you mean \"Non-Canonical-Header\"?", + ), + err) + }) +} diff --git a/pkg/http/server.go b/pkg/http/server.go index 82593c27..7ceb47c0 100644 --- a/pkg/http/server.go +++ b/pkg/http/server.go @@ -4,6 +4,7 @@ import ( "context" "net/http" + "github.com/buildbarn/bb-storage/pkg/grpc" "github.com/buildbarn/bb-storage/pkg/program" configuration "github.com/buildbarn/bb-storage/pkg/proto/configuration/http" "github.com/buildbarn/bb-storage/pkg/util" @@ -13,10 +14,10 @@ import ( // program.Group, based on a configuration message. The web servers are // automatically terminated if the context associated with the group is // canceled. -func NewServersFromConfigurationAndServe(configurations []*configuration.ServerConfiguration, handler http.Handler, group program.Group) { +func NewServersFromConfigurationAndServe(configurations []*configuration.ServerConfiguration, handler http.Handler, group program.Group, grpcClientFactory grpc.ClientFactory) { group.Go(func(ctx context.Context, siblingsGroup, dependenciesGroup program.Group) error { for _, configuration := range configurations { - authenticator, err := NewAuthenticatorFromConfiguration(configuration.AuthenticationPolicy, dependenciesGroup) + authenticator, err := NewAuthenticatorFromConfiguration(configuration.AuthenticationPolicy, dependenciesGroup, grpcClientFactory) if err != nil { return err } diff --git a/pkg/jwt/authorization_header_parser.go b/pkg/jwt/authorization_header_parser.go index 3b155fd8..cd3ac7c2 100644 --- a/pkg/jwt/authorization_header_parser.go +++ b/pkg/jwt/authorization_header_parser.go @@ -1,6 +1,7 @@ package jwt import ( + "context" "encoding/base64" "encoding/json" "log" @@ -13,8 +14,14 @@ import ( "github.com/buildbarn/bb-storage/pkg/clock" "github.com/buildbarn/bb-storage/pkg/eviction" "github.com/jmespath/go-jmespath" + "google.golang.org/grpc/codes" + "google.golang.org/grpc/status" ) +// AuthorizationHeaderName is the name of the HTTP header that contains +// the JSON Web Token. +const AuthorizationHeaderName = "Authorization" + // Pattern of authorization headers from which to extract a JSON Web Token. var jwtHeaderPattern = regexp.MustCompile("^Bearer\\s+(([-_a-zA-Z0-9]+)\\.([-_a-zA-Z0-9]+))\\.([-_a-zA-Z0-9]+)$") @@ -83,6 +90,15 @@ func jsonNumberAsTimestamp(n *json.Number) (time.Time, error) { return time.Unix(int64(i), int64(frac*1e9)), nil } +// Authenticate is the implementation of RequestHeadersAuthenticator.Authenticate. +func (a *AuthorizationHeaderParser) Authenticate(ctx context.Context, headers map[string][]string) (*auth.AuthenticationMetadata, error) { + metadata, ok := a.ParseAuthorizationHeaders(headers[AuthorizationHeaderName]) + if !ok { + return nil, status.Error(codes.Unauthenticated, "No valid authorization header containing a bearer token provided") + } + return metadata, nil +} + func (a *AuthorizationHeaderParser) parseSingleAuthorizationHeader(header string, now time.Time) response { match := jwtHeaderPattern.FindStringSubmatch(header) if match == nil { diff --git a/pkg/proto/auth/BUILD.bazel b/pkg/proto/auth/BUILD.bazel index 7981cb33..4fd756b1 100644 --- a/pkg/proto/auth/BUILD.bazel +++ b/pkg/proto/auth/BUILD.bazel @@ -8,12 +8,18 @@ proto_library( visibility = ["//visibility:public"], deps = [ "@opentelemetry-proto//:common_proto", + "@protobuf//:empty_proto", "@protobuf//:struct_proto", + "@protobuf//:timestamp_proto", ], ) go_proto_library( name = "auth_go_proto", + compilers = [ + "@rules_go//proto:go_proto", + "@rules_go//proto:go_grpc_v2", + ], importpath = "github.com/buildbarn/bb-storage/pkg/proto/auth", proto = ":auth_proto", visibility = ["//visibility:public"], diff --git a/pkg/proto/auth/auth.pb.go b/pkg/proto/auth/auth.pb.go index a719640f..76b1b81a 100644 --- a/pkg/proto/auth/auth.pb.go +++ b/pkg/proto/auth/auth.pb.go @@ -10,7 +10,9 @@ import ( v1 "go.opentelemetry.io/proto/otlp/common/v1" protoreflect "google.golang.org/protobuf/reflect/protoreflect" protoimpl "google.golang.org/protobuf/runtime/protoimpl" + emptypb "google.golang.org/protobuf/types/known/emptypb" structpb "google.golang.org/protobuf/types/known/structpb" + timestamppb "google.golang.org/protobuf/types/known/timestamppb" reflect "reflect" sync "sync" unsafe "unsafe" @@ -83,34 +85,443 @@ func (x *AuthenticationMetadata) GetPrivate() *structpb.Value { return nil } +type AuthenticateRequest struct { + state protoimpl.MessageState `protogen:"open.v1"` + RequestMetadata map[string]*AuthenticateRequest_ValueList `protobuf:"bytes,1,rep,name=request_metadata,json=requestMetadata,proto3" json:"request_metadata,omitempty" protobuf_key:"bytes,1,opt,name=key" protobuf_val:"bytes,2,opt,name=value"` + Scope *structpb.Value `protobuf:"bytes,2,opt,name=scope,proto3" json:"scope,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *AuthenticateRequest) Reset() { + *x = AuthenticateRequest{} + mi := &file_pkg_proto_auth_auth_proto_msgTypes[1] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *AuthenticateRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*AuthenticateRequest) ProtoMessage() {} + +func (x *AuthenticateRequest) ProtoReflect() protoreflect.Message { + mi := &file_pkg_proto_auth_auth_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 AuthenticateRequest.ProtoReflect.Descriptor instead. +func (*AuthenticateRequest) Descriptor() ([]byte, []int) { + return file_pkg_proto_auth_auth_proto_rawDescGZIP(), []int{1} +} + +func (x *AuthenticateRequest) GetRequestMetadata() map[string]*AuthenticateRequest_ValueList { + if x != nil { + return x.RequestMetadata + } + return nil +} + +func (x *AuthenticateRequest) GetScope() *structpb.Value { + if x != nil { + return x.Scope + } + return nil +} + +type AuthenticateResponse struct { + state protoimpl.MessageState `protogen:"open.v1"` + // Types that are valid to be assigned to Verdict: + // + // *AuthenticateResponse_Allow + // *AuthenticateResponse_Deny + Verdict isAuthenticateResponse_Verdict `protobuf_oneof:"verdict"` + CacheExpirationTime *timestamppb.Timestamp `protobuf:"bytes,3,opt,name=cache_expiration_time,json=cacheExpirationTime,proto3" json:"cache_expiration_time,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *AuthenticateResponse) Reset() { + *x = AuthenticateResponse{} + mi := &file_pkg_proto_auth_auth_proto_msgTypes[2] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *AuthenticateResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*AuthenticateResponse) ProtoMessage() {} + +func (x *AuthenticateResponse) ProtoReflect() protoreflect.Message { + mi := &file_pkg_proto_auth_auth_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 AuthenticateResponse.ProtoReflect.Descriptor instead. +func (*AuthenticateResponse) Descriptor() ([]byte, []int) { + return file_pkg_proto_auth_auth_proto_rawDescGZIP(), []int{2} +} + +func (x *AuthenticateResponse) GetVerdict() isAuthenticateResponse_Verdict { + if x != nil { + return x.Verdict + } + return nil +} + +func (x *AuthenticateResponse) GetAllow() *AuthenticationMetadata { + if x != nil { + if x, ok := x.Verdict.(*AuthenticateResponse_Allow); ok { + return x.Allow + } + } + return nil +} + +func (x *AuthenticateResponse) GetDeny() string { + if x != nil { + if x, ok := x.Verdict.(*AuthenticateResponse_Deny); ok { + return x.Deny + } + } + return "" +} + +func (x *AuthenticateResponse) GetCacheExpirationTime() *timestamppb.Timestamp { + if x != nil { + return x.CacheExpirationTime + } + return nil +} + +type isAuthenticateResponse_Verdict interface { + isAuthenticateResponse_Verdict() +} + +type AuthenticateResponse_Allow struct { + Allow *AuthenticationMetadata `protobuf:"bytes,1,opt,name=allow,proto3,oneof"` +} + +type AuthenticateResponse_Deny struct { + Deny string `protobuf:"bytes,2,opt,name=deny,proto3,oneof"` +} + +func (*AuthenticateResponse_Allow) isAuthenticateResponse_Verdict() {} + +func (*AuthenticateResponse_Deny) isAuthenticateResponse_Verdict() {} + +type AuthorizeRequest struct { + state protoimpl.MessageState `protogen:"open.v1"` + AuthenticationMetadata *AuthenticationMetadata `protobuf:"bytes,1,opt,name=authentication_metadata,json=authenticationMetadata,proto3" json:"authentication_metadata,omitempty"` + Scope *structpb.Value `protobuf:"bytes,2,opt,name=scope,proto3" json:"scope,omitempty"` + InstanceName string `protobuf:"bytes,3,opt,name=instance_name,json=instanceName,proto3" json:"instance_name,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *AuthorizeRequest) Reset() { + *x = AuthorizeRequest{} + mi := &file_pkg_proto_auth_auth_proto_msgTypes[3] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *AuthorizeRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*AuthorizeRequest) ProtoMessage() {} + +func (x *AuthorizeRequest) ProtoReflect() protoreflect.Message { + mi := &file_pkg_proto_auth_auth_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 AuthorizeRequest.ProtoReflect.Descriptor instead. +func (*AuthorizeRequest) Descriptor() ([]byte, []int) { + return file_pkg_proto_auth_auth_proto_rawDescGZIP(), []int{3} +} + +func (x *AuthorizeRequest) GetAuthenticationMetadata() *AuthenticationMetadata { + if x != nil { + return x.AuthenticationMetadata + } + return nil +} + +func (x *AuthorizeRequest) GetScope() *structpb.Value { + if x != nil { + return x.Scope + } + return nil +} + +func (x *AuthorizeRequest) GetInstanceName() string { + if x != nil { + return x.InstanceName + } + return "" +} + +type AuthorizeResponse struct { + state protoimpl.MessageState `protogen:"open.v1"` + // Types that are valid to be assigned to Verdict: + // + // *AuthorizeResponse_Allow + // *AuthorizeResponse_Deny + Verdict isAuthorizeResponse_Verdict `protobuf_oneof:"verdict"` + CacheExpirationTime *timestamppb.Timestamp `protobuf:"bytes,3,opt,name=cache_expiration_time,json=cacheExpirationTime,proto3" json:"cache_expiration_time,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *AuthorizeResponse) Reset() { + *x = AuthorizeResponse{} + mi := &file_pkg_proto_auth_auth_proto_msgTypes[4] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *AuthorizeResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*AuthorizeResponse) ProtoMessage() {} + +func (x *AuthorizeResponse) ProtoReflect() protoreflect.Message { + mi := &file_pkg_proto_auth_auth_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 AuthorizeResponse.ProtoReflect.Descriptor instead. +func (*AuthorizeResponse) Descriptor() ([]byte, []int) { + return file_pkg_proto_auth_auth_proto_rawDescGZIP(), []int{4} +} + +func (x *AuthorizeResponse) GetVerdict() isAuthorizeResponse_Verdict { + if x != nil { + return x.Verdict + } + return nil +} + +func (x *AuthorizeResponse) GetAllow() *emptypb.Empty { + if x != nil { + if x, ok := x.Verdict.(*AuthorizeResponse_Allow); ok { + return x.Allow + } + } + return nil +} + +func (x *AuthorizeResponse) GetDeny() string { + if x != nil { + if x, ok := x.Verdict.(*AuthorizeResponse_Deny); ok { + return x.Deny + } + } + return "" +} + +func (x *AuthorizeResponse) GetCacheExpirationTime() *timestamppb.Timestamp { + if x != nil { + return x.CacheExpirationTime + } + return nil +} + +type isAuthorizeResponse_Verdict interface { + isAuthorizeResponse_Verdict() +} + +type AuthorizeResponse_Allow struct { + Allow *emptypb.Empty `protobuf:"bytes,1,opt,name=allow,proto3,oneof"` +} + +type AuthorizeResponse_Deny struct { + Deny string `protobuf:"bytes,2,opt,name=deny,proto3,oneof"` +} + +func (*AuthorizeResponse_Allow) isAuthorizeResponse_Verdict() {} + +func (*AuthorizeResponse_Deny) isAuthorizeResponse_Verdict() {} + +type AuthenticateRequest_ValueList struct { + state protoimpl.MessageState `protogen:"open.v1"` + Value []string `protobuf:"bytes,1,rep,name=value,proto3" json:"value,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *AuthenticateRequest_ValueList) Reset() { + *x = AuthenticateRequest_ValueList{} + mi := &file_pkg_proto_auth_auth_proto_msgTypes[5] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *AuthenticateRequest_ValueList) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*AuthenticateRequest_ValueList) ProtoMessage() {} + +func (x *AuthenticateRequest_ValueList) ProtoReflect() protoreflect.Message { + mi := &file_pkg_proto_auth_auth_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 AuthenticateRequest_ValueList.ProtoReflect.Descriptor instead. +func (*AuthenticateRequest_ValueList) Descriptor() ([]byte, []int) { + return file_pkg_proto_auth_auth_proto_rawDescGZIP(), []int{1, 0} +} + +func (x *AuthenticateRequest_ValueList) GetValue() []string { + if x != nil { + return x.Value + } + return nil +} + var File_pkg_proto_auth_auth_proto protoreflect.FileDescriptor var file_pkg_proto_auth_auth_proto_rawDesc = string([]byte{ 0x0a, 0x19, 0x70, 0x6b, 0x67, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x61, 0x75, 0x74, 0x68, 0x2f, 0x61, 0x75, 0x74, 0x68, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x0e, 0x62, 0x75, 0x69, - 0x6c, 0x64, 0x62, 0x61, 0x72, 0x6e, 0x2e, 0x61, 0x75, 0x74, 0x68, 0x1a, 0x1c, 0x67, 0x6f, 0x6f, - 0x67, 0x6c, 0x65, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2f, 0x73, 0x74, 0x72, - 0x75, 0x63, 0x74, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x2a, 0x6f, 0x70, 0x65, 0x6e, 0x74, - 0x65, 0x6c, 0x65, 0x6d, 0x65, 0x74, 0x72, 0x79, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x63, - 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2f, 0x76, 0x31, 0x2f, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, - 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x22, 0xd2, 0x01, 0x0a, 0x16, 0x41, 0x75, 0x74, 0x68, 0x65, 0x6e, - 0x74, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, - 0x12, 0x2e, 0x0a, 0x06, 0x70, 0x75, 0x62, 0x6c, 0x69, 0x63, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, - 0x32, 0x16, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, - 0x75, 0x66, 0x2e, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x52, 0x06, 0x70, 0x75, 0x62, 0x6c, 0x69, 0x63, - 0x12, 0x56, 0x0a, 0x12, 0x74, 0x72, 0x61, 0x63, 0x69, 0x6e, 0x67, 0x5f, 0x61, 0x74, 0x74, 0x72, - 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x27, 0x2e, 0x6f, - 0x70, 0x65, 0x6e, 0x74, 0x65, 0x6c, 0x65, 0x6d, 0x65, 0x74, 0x72, 0x79, 0x2e, 0x70, 0x72, 0x6f, - 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x4b, 0x65, 0x79, - 0x56, 0x61, 0x6c, 0x75, 0x65, 0x52, 0x11, 0x74, 0x72, 0x61, 0x63, 0x69, 0x6e, 0x67, 0x41, 0x74, - 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x12, 0x30, 0x0a, 0x07, 0x70, 0x72, 0x69, 0x76, - 0x61, 0x74, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x16, 0x2e, 0x67, 0x6f, 0x6f, 0x67, - 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x56, 0x61, 0x6c, 0x75, - 0x65, 0x52, 0x07, 0x70, 0x72, 0x69, 0x76, 0x61, 0x74, 0x65, 0x42, 0x30, 0x5a, 0x2e, 0x67, 0x69, - 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x62, 0x61, - 0x72, 0x6e, 0x2f, 0x62, 0x62, 0x2d, 0x73, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x2f, 0x70, 0x6b, - 0x67, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x61, 0x75, 0x74, 0x68, 0x62, 0x06, 0x70, 0x72, - 0x6f, 0x74, 0x6f, 0x33, + 0x6c, 0x64, 0x62, 0x61, 0x72, 0x6e, 0x2e, 0x61, 0x75, 0x74, 0x68, 0x1a, 0x1b, 0x67, 0x6f, 0x6f, + 0x67, 0x6c, 0x65, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2f, 0x65, 0x6d, 0x70, + 0x74, 0x79, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x1c, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, + 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2f, 0x73, 0x74, 0x72, 0x75, 0x63, 0x74, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x1f, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2f, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2f, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, + 0x70, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x2a, 0x6f, 0x70, 0x65, 0x6e, 0x74, 0x65, 0x6c, + 0x65, 0x6d, 0x65, 0x74, 0x72, 0x79, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x63, 0x6f, 0x6d, + 0x6d, 0x6f, 0x6e, 0x2f, 0x76, 0x31, 0x2f, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x22, 0xd2, 0x01, 0x0a, 0x16, 0x41, 0x75, 0x74, 0x68, 0x65, 0x6e, 0x74, 0x69, + 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x12, 0x2e, + 0x0a, 0x06, 0x70, 0x75, 0x62, 0x6c, 0x69, 0x63, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x16, + 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, + 0x2e, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x52, 0x06, 0x70, 0x75, 0x62, 0x6c, 0x69, 0x63, 0x12, 0x56, + 0x0a, 0x12, 0x74, 0x72, 0x61, 0x63, 0x69, 0x6e, 0x67, 0x5f, 0x61, 0x74, 0x74, 0x72, 0x69, 0x62, + 0x75, 0x74, 0x65, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x27, 0x2e, 0x6f, 0x70, 0x65, + 0x6e, 0x74, 0x65, 0x6c, 0x65, 0x6d, 0x65, 0x74, 0x72, 0x79, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x4b, 0x65, 0x79, 0x56, 0x61, + 0x6c, 0x75, 0x65, 0x52, 0x11, 0x74, 0x72, 0x61, 0x63, 0x69, 0x6e, 0x67, 0x41, 0x74, 0x74, 0x72, + 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x12, 0x30, 0x0a, 0x07, 0x70, 0x72, 0x69, 0x76, 0x61, 0x74, + 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x16, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x52, + 0x07, 0x70, 0x72, 0x69, 0x76, 0x61, 0x74, 0x65, 0x22, 0xbe, 0x02, 0x0a, 0x13, 0x41, 0x75, 0x74, + 0x68, 0x65, 0x6e, 0x74, 0x69, 0x63, 0x61, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x12, 0x63, 0x0a, 0x10, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x6d, 0x65, 0x74, 0x61, + 0x64, 0x61, 0x74, 0x61, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x38, 0x2e, 0x62, 0x75, 0x69, + 0x6c, 0x64, 0x62, 0x61, 0x72, 0x6e, 0x2e, 0x61, 0x75, 0x74, 0x68, 0x2e, 0x41, 0x75, 0x74, 0x68, + 0x65, 0x6e, 0x74, 0x69, 0x63, 0x61, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x2e, + 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x45, + 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0f, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x4d, 0x65, 0x74, + 0x61, 0x64, 0x61, 0x74, 0x61, 0x12, 0x2c, 0x0a, 0x05, 0x73, 0x63, 0x6f, 0x70, 0x65, 0x18, 0x02, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x16, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x52, 0x05, 0x73, 0x63, + 0x6f, 0x70, 0x65, 0x1a, 0x21, 0x0a, 0x09, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x4c, 0x69, 0x73, 0x74, + 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x01, 0x20, 0x03, 0x28, 0x09, 0x52, + 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x1a, 0x71, 0x0a, 0x14, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, + 0x74, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, + 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, + 0x12, 0x43, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x2d, 0x2e, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x62, 0x61, 0x72, 0x6e, 0x2e, 0x61, 0x75, 0x74, 0x68, + 0x2e, 0x41, 0x75, 0x74, 0x68, 0x65, 0x6e, 0x74, 0x69, 0x63, 0x61, 0x74, 0x65, 0x52, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x2e, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x4c, 0x69, 0x73, 0x74, 0x52, 0x05, + 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0xc7, 0x01, 0x0a, 0x14, 0x41, 0x75, + 0x74, 0x68, 0x65, 0x6e, 0x74, 0x69, 0x63, 0x61, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, + 0x73, 0x65, 0x12, 0x3e, 0x0a, 0x05, 0x61, 0x6c, 0x6c, 0x6f, 0x77, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x26, 0x2e, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x62, 0x61, 0x72, 0x6e, 0x2e, 0x61, 0x75, + 0x74, 0x68, 0x2e, 0x41, 0x75, 0x74, 0x68, 0x65, 0x6e, 0x74, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, + 0x6e, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x48, 0x00, 0x52, 0x05, 0x61, 0x6c, 0x6c, + 0x6f, 0x77, 0x12, 0x14, 0x0a, 0x04, 0x64, 0x65, 0x6e, 0x79, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, + 0x48, 0x00, 0x52, 0x04, 0x64, 0x65, 0x6e, 0x79, 0x12, 0x4e, 0x0a, 0x15, 0x63, 0x61, 0x63, 0x68, + 0x65, 0x5f, 0x65, 0x78, 0x70, 0x69, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x74, 0x69, 0x6d, + 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, + 0x61, 0x6d, 0x70, 0x52, 0x13, 0x63, 0x61, 0x63, 0x68, 0x65, 0x45, 0x78, 0x70, 0x69, 0x72, 0x61, + 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x69, 0x6d, 0x65, 0x42, 0x09, 0x0a, 0x07, 0x76, 0x65, 0x72, 0x64, + 0x69, 0x63, 0x74, 0x22, 0xc6, 0x01, 0x0a, 0x10, 0x41, 0x75, 0x74, 0x68, 0x6f, 0x72, 0x69, 0x7a, + 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x5f, 0x0a, 0x17, 0x61, 0x75, 0x74, 0x68, + 0x65, 0x6e, 0x74, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x6d, 0x65, 0x74, 0x61, 0x64, + 0x61, 0x74, 0x61, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x26, 0x2e, 0x62, 0x75, 0x69, 0x6c, + 0x64, 0x62, 0x61, 0x72, 0x6e, 0x2e, 0x61, 0x75, 0x74, 0x68, 0x2e, 0x41, 0x75, 0x74, 0x68, 0x65, + 0x6e, 0x74, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, + 0x61, 0x52, 0x16, 0x61, 0x75, 0x74, 0x68, 0x65, 0x6e, 0x74, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, + 0x6e, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x12, 0x2c, 0x0a, 0x05, 0x73, 0x63, 0x6f, + 0x70, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x16, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, + 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x56, 0x61, 0x6c, 0x75, 0x65, + 0x52, 0x05, 0x73, 0x63, 0x6f, 0x70, 0x65, 0x12, 0x23, 0x0a, 0x0d, 0x69, 0x6e, 0x73, 0x74, 0x61, + 0x6e, 0x63, 0x65, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, + 0x69, 0x6e, 0x73, 0x74, 0x61, 0x6e, 0x63, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x22, 0xb4, 0x01, 0x0a, + 0x11, 0x41, 0x75, 0x74, 0x68, 0x6f, 0x72, 0x69, 0x7a, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, + 0x73, 0x65, 0x12, 0x2e, 0x0a, 0x05, 0x61, 0x6c, 0x6c, 0x6f, 0x77, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x16, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x62, 0x75, 0x66, 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x48, 0x00, 0x52, 0x05, 0x61, 0x6c, 0x6c, + 0x6f, 0x77, 0x12, 0x14, 0x0a, 0x04, 0x64, 0x65, 0x6e, 0x79, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, + 0x48, 0x00, 0x52, 0x04, 0x64, 0x65, 0x6e, 0x79, 0x12, 0x4e, 0x0a, 0x15, 0x63, 0x61, 0x63, 0x68, + 0x65, 0x5f, 0x65, 0x78, 0x70, 0x69, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x74, 0x69, 0x6d, + 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, + 0x61, 0x6d, 0x70, 0x52, 0x13, 0x63, 0x61, 0x63, 0x68, 0x65, 0x45, 0x78, 0x70, 0x69, 0x72, 0x61, + 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x69, 0x6d, 0x65, 0x42, 0x09, 0x0a, 0x07, 0x76, 0x65, 0x72, 0x64, + 0x69, 0x63, 0x74, 0x32, 0x6d, 0x0a, 0x0e, 0x41, 0x75, 0x74, 0x68, 0x65, 0x6e, 0x74, 0x69, 0x63, + 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x5b, 0x0a, 0x0c, 0x41, 0x75, 0x74, 0x68, 0x65, 0x6e, 0x74, + 0x69, 0x63, 0x61, 0x74, 0x65, 0x12, 0x23, 0x2e, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x62, 0x61, 0x72, + 0x6e, 0x2e, 0x61, 0x75, 0x74, 0x68, 0x2e, 0x41, 0x75, 0x74, 0x68, 0x65, 0x6e, 0x74, 0x69, 0x63, + 0x61, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x24, 0x2e, 0x62, 0x75, 0x69, + 0x6c, 0x64, 0x62, 0x61, 0x72, 0x6e, 0x2e, 0x61, 0x75, 0x74, 0x68, 0x2e, 0x41, 0x75, 0x74, 0x68, + 0x65, 0x6e, 0x74, 0x69, 0x63, 0x61, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, + 0x22, 0x00, 0x32, 0x60, 0x0a, 0x0a, 0x41, 0x75, 0x74, 0x68, 0x6f, 0x72, 0x69, 0x7a, 0x65, 0x72, + 0x12, 0x52, 0x0a, 0x09, 0x41, 0x75, 0x74, 0x68, 0x6f, 0x72, 0x69, 0x7a, 0x65, 0x12, 0x20, 0x2e, + 0x62, 0x75, 0x69, 0x6c, 0x64, 0x62, 0x61, 0x72, 0x6e, 0x2e, 0x61, 0x75, 0x74, 0x68, 0x2e, 0x41, + 0x75, 0x74, 0x68, 0x6f, 0x72, 0x69, 0x7a, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, + 0x21, 0x2e, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x62, 0x61, 0x72, 0x6e, 0x2e, 0x61, 0x75, 0x74, 0x68, + 0x2e, 0x41, 0x75, 0x74, 0x68, 0x6f, 0x72, 0x69, 0x7a, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, + 0x73, 0x65, 0x22, 0x00, 0x42, 0x30, 0x5a, 0x2e, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, + 0x6f, 0x6d, 0x2f, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x62, 0x61, 0x72, 0x6e, 0x2f, 0x62, 0x62, 0x2d, + 0x73, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x2f, 0x70, 0x6b, 0x67, 0x2f, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2f, 0x61, 0x75, 0x74, 0x68, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, }) var ( @@ -125,21 +536,42 @@ func file_pkg_proto_auth_auth_proto_rawDescGZIP() []byte { return file_pkg_proto_auth_auth_proto_rawDescData } -var file_pkg_proto_auth_auth_proto_msgTypes = make([]protoimpl.MessageInfo, 1) +var file_pkg_proto_auth_auth_proto_msgTypes = make([]protoimpl.MessageInfo, 7) var file_pkg_proto_auth_auth_proto_goTypes = []any{ - (*AuthenticationMetadata)(nil), // 0: buildbarn.auth.AuthenticationMetadata - (*structpb.Value)(nil), // 1: google.protobuf.Value - (*v1.KeyValue)(nil), // 2: opentelemetry.proto.common.v1.KeyValue + (*AuthenticationMetadata)(nil), // 0: buildbarn.auth.AuthenticationMetadata + (*AuthenticateRequest)(nil), // 1: buildbarn.auth.AuthenticateRequest + (*AuthenticateResponse)(nil), // 2: buildbarn.auth.AuthenticateResponse + (*AuthorizeRequest)(nil), // 3: buildbarn.auth.AuthorizeRequest + (*AuthorizeResponse)(nil), // 4: buildbarn.auth.AuthorizeResponse + (*AuthenticateRequest_ValueList)(nil), // 5: buildbarn.auth.AuthenticateRequest.ValueList + nil, // 6: buildbarn.auth.AuthenticateRequest.RequestMetadataEntry + (*structpb.Value)(nil), // 7: google.protobuf.Value + (*v1.KeyValue)(nil), // 8: opentelemetry.proto.common.v1.KeyValue + (*timestamppb.Timestamp)(nil), // 9: google.protobuf.Timestamp + (*emptypb.Empty)(nil), // 10: google.protobuf.Empty } var file_pkg_proto_auth_auth_proto_depIdxs = []int32{ - 1, // 0: buildbarn.auth.AuthenticationMetadata.public:type_name -> google.protobuf.Value - 2, // 1: buildbarn.auth.AuthenticationMetadata.tracing_attributes:type_name -> opentelemetry.proto.common.v1.KeyValue - 1, // 2: buildbarn.auth.AuthenticationMetadata.private:type_name -> google.protobuf.Value - 3, // [3:3] is the sub-list for method output_type - 3, // [3:3] is the sub-list for method input_type - 3, // [3:3] is the sub-list for extension type_name - 3, // [3:3] is the sub-list for extension extendee - 0, // [0:3] is the sub-list for field type_name + 7, // 0: buildbarn.auth.AuthenticationMetadata.public:type_name -> google.protobuf.Value + 8, // 1: buildbarn.auth.AuthenticationMetadata.tracing_attributes:type_name -> opentelemetry.proto.common.v1.KeyValue + 7, // 2: buildbarn.auth.AuthenticationMetadata.private:type_name -> google.protobuf.Value + 6, // 3: buildbarn.auth.AuthenticateRequest.request_metadata:type_name -> buildbarn.auth.AuthenticateRequest.RequestMetadataEntry + 7, // 4: buildbarn.auth.AuthenticateRequest.scope:type_name -> google.protobuf.Value + 0, // 5: buildbarn.auth.AuthenticateResponse.allow:type_name -> buildbarn.auth.AuthenticationMetadata + 9, // 6: buildbarn.auth.AuthenticateResponse.cache_expiration_time:type_name -> google.protobuf.Timestamp + 0, // 7: buildbarn.auth.AuthorizeRequest.authentication_metadata:type_name -> buildbarn.auth.AuthenticationMetadata + 7, // 8: buildbarn.auth.AuthorizeRequest.scope:type_name -> google.protobuf.Value + 10, // 9: buildbarn.auth.AuthorizeResponse.allow:type_name -> google.protobuf.Empty + 9, // 10: buildbarn.auth.AuthorizeResponse.cache_expiration_time:type_name -> google.protobuf.Timestamp + 5, // 11: buildbarn.auth.AuthenticateRequest.RequestMetadataEntry.value:type_name -> buildbarn.auth.AuthenticateRequest.ValueList + 1, // 12: buildbarn.auth.Authentication.Authenticate:input_type -> buildbarn.auth.AuthenticateRequest + 3, // 13: buildbarn.auth.Authorizer.Authorize:input_type -> buildbarn.auth.AuthorizeRequest + 2, // 14: buildbarn.auth.Authentication.Authenticate:output_type -> buildbarn.auth.AuthenticateResponse + 4, // 15: buildbarn.auth.Authorizer.Authorize:output_type -> buildbarn.auth.AuthorizeResponse + 14, // [14:16] is the sub-list for method output_type + 12, // [12:14] is the sub-list for method input_type + 12, // [12:12] is the sub-list for extension type_name + 12, // [12:12] is the sub-list for extension extendee + 0, // [0:12] is the sub-list for field type_name } func init() { file_pkg_proto_auth_auth_proto_init() } @@ -147,15 +579,23 @@ func file_pkg_proto_auth_auth_proto_init() { if File_pkg_proto_auth_auth_proto != nil { return } + file_pkg_proto_auth_auth_proto_msgTypes[2].OneofWrappers = []any{ + (*AuthenticateResponse_Allow)(nil), + (*AuthenticateResponse_Deny)(nil), + } + file_pkg_proto_auth_auth_proto_msgTypes[4].OneofWrappers = []any{ + (*AuthorizeResponse_Allow)(nil), + (*AuthorizeResponse_Deny)(nil), + } type x struct{} out := protoimpl.TypeBuilder{ File: protoimpl.DescBuilder{ GoPackagePath: reflect.TypeOf(x{}).PkgPath(), RawDescriptor: unsafe.Slice(unsafe.StringData(file_pkg_proto_auth_auth_proto_rawDesc), len(file_pkg_proto_auth_auth_proto_rawDesc)), NumEnums: 0, - NumMessages: 1, + NumMessages: 7, NumExtensions: 0, - NumServices: 0, + NumServices: 2, }, GoTypes: file_pkg_proto_auth_auth_proto_goTypes, DependencyIndexes: file_pkg_proto_auth_auth_proto_depIdxs, diff --git a/pkg/proto/auth/auth.proto b/pkg/proto/auth/auth.proto index 1cd64ae7..90f36b7b 100644 --- a/pkg/proto/auth/auth.proto +++ b/pkg/proto/auth/auth.proto @@ -2,7 +2,9 @@ syntax = "proto3"; package buildbarn.auth; +import "google/protobuf/empty.proto"; import "google/protobuf/struct.proto"; +import "google/protobuf/timestamp.proto"; import "opentelemetry/proto/common/v1/common.proto"; option go_package = "github.com/buildbarn/bb-storage/pkg/proto/auth"; @@ -27,3 +29,74 @@ message AuthenticationMetadata { // be accessed by JMESPathExpressionAuthorizer. google.protobuf.Value private = 3; } + +service Authentication { + // Authenticates a request that the client has received. + rpc Authenticate(AuthenticateRequest) returns (AuthenticateResponse) {} +} + +message AuthenticateRequest { + message ValueList { + repeated string value = 1; + } + + // The HTTP or gRPC request headers that the client wants to authenticate. + // This metadata is forwarded as part of the request because the communication + // with the authentication service might have its own authentication and + // authorizing system. + map request_metadata = 1; + + // A description of for example what subsystem is being accessed and what + // kind of request is being made. + google.protobuf.Value scope = 2; +} + +message AuthenticateResponse { + oneof verdict { + // Allow the request. + // + // The value of this field is used as authentication metadata in + // literal form. + AuthenticationMetadata allow = 1; + + // Deny the request by returning UNAUTHENTICATED with a fixed error message + // to the client. + string deny = 2; + } + + // The last point in time this response can be reused for the same request. + // A null value means not to be cached for future requests. + google.protobuf.Timestamp cache_expiration_time = 3; +} + +service Authorizer { + // Authorizes a request that the client has received. + rpc Authorize(AuthorizeRequest) returns (AuthorizeResponse) {} +} + +message AuthorizeRequest { + // Metadata returned by the authenticator. + AuthenticationMetadata authentication_metadata = 1; + + // A description of for example what subsystem is being accessed and what + // kind of request is being made. + google.protobuf.Value scope = 2; + + // The instance name to evaluate. + string instance_name = 3; +} + +message AuthorizeResponse { + oneof verdict { + // Allow the request. + google.protobuf.Empty allow = 1; + + // Deny the request by returning PERMISSION_DENIED with a fixed error + // message to the client. + string deny = 2; + } + + // The last point in time this response can be reused for the same request. + // A null value means not to be cached for future requests. + google.protobuf.Timestamp cache_expiration_time = 3; +} diff --git a/pkg/proto/auth/auth_grpc.pb.go b/pkg/proto/auth/auth_grpc.pb.go new file mode 100644 index 00000000..e93e6645 --- /dev/null +++ b/pkg/proto/auth/auth_grpc.pb.go @@ -0,0 +1,219 @@ +// Code generated by protoc-gen-go-grpc. DO NOT EDIT. +// versions: +// - protoc-gen-go-grpc v1.5.1 +// - protoc v5.29.3 +// source: pkg/proto/auth/auth.proto + +package auth + +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 ( + Authentication_Authenticate_FullMethodName = "/buildbarn.auth.Authentication/Authenticate" +) + +// AuthenticationClient is the client API for Authentication 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. +type AuthenticationClient interface { + Authenticate(ctx context.Context, in *AuthenticateRequest, opts ...grpc.CallOption) (*AuthenticateResponse, error) +} + +type authenticationClient struct { + cc grpc.ClientConnInterface +} + +func NewAuthenticationClient(cc grpc.ClientConnInterface) AuthenticationClient { + return &authenticationClient{cc} +} + +func (c *authenticationClient) Authenticate(ctx context.Context, in *AuthenticateRequest, opts ...grpc.CallOption) (*AuthenticateResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(AuthenticateResponse) + err := c.cc.Invoke(ctx, Authentication_Authenticate_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +// AuthenticationServer is the server API for Authentication service. +// All implementations should embed UnimplementedAuthenticationServer +// for forward compatibility. +type AuthenticationServer interface { + Authenticate(context.Context, *AuthenticateRequest) (*AuthenticateResponse, error) +} + +// UnimplementedAuthenticationServer should 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 UnimplementedAuthenticationServer struct{} + +func (UnimplementedAuthenticationServer) Authenticate(context.Context, *AuthenticateRequest) (*AuthenticateResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method Authenticate not implemented") +} +func (UnimplementedAuthenticationServer) testEmbeddedByValue() {} + +// UnsafeAuthenticationServer may be embedded to opt out of forward compatibility for this service. +// Use of this interface is not recommended, as added methods to AuthenticationServer will +// result in compilation errors. +type UnsafeAuthenticationServer interface { + mustEmbedUnimplementedAuthenticationServer() +} + +func RegisterAuthenticationServer(s grpc.ServiceRegistrar, srv AuthenticationServer) { + // If the following call pancis, it indicates UnimplementedAuthenticationServer 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(&Authentication_ServiceDesc, srv) +} + +func _Authentication_Authenticate_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(AuthenticateRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(AuthenticationServer).Authenticate(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: Authentication_Authenticate_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(AuthenticationServer).Authenticate(ctx, req.(*AuthenticateRequest)) + } + return interceptor(ctx, in, info, handler) +} + +// Authentication_ServiceDesc is the grpc.ServiceDesc for Authentication service. +// It's only intended for direct use with grpc.RegisterService, +// and not to be introspected or modified (even as a copy) +var Authentication_ServiceDesc = grpc.ServiceDesc{ + ServiceName: "buildbarn.auth.Authentication", + HandlerType: (*AuthenticationServer)(nil), + Methods: []grpc.MethodDesc{ + { + MethodName: "Authenticate", + Handler: _Authentication_Authenticate_Handler, + }, + }, + Streams: []grpc.StreamDesc{}, + Metadata: "pkg/proto/auth/auth.proto", +} + +const ( + Authorizer_Authorize_FullMethodName = "/buildbarn.auth.Authorizer/Authorize" +) + +// AuthorizerClient is the client API for Authorizer 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. +type AuthorizerClient interface { + Authorize(ctx context.Context, in *AuthorizeRequest, opts ...grpc.CallOption) (*AuthorizeResponse, error) +} + +type authorizerClient struct { + cc grpc.ClientConnInterface +} + +func NewAuthorizerClient(cc grpc.ClientConnInterface) AuthorizerClient { + return &authorizerClient{cc} +} + +func (c *authorizerClient) Authorize(ctx context.Context, in *AuthorizeRequest, opts ...grpc.CallOption) (*AuthorizeResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(AuthorizeResponse) + err := c.cc.Invoke(ctx, Authorizer_Authorize_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +// AuthorizerServer is the server API for Authorizer service. +// All implementations should embed UnimplementedAuthorizerServer +// for forward compatibility. +type AuthorizerServer interface { + Authorize(context.Context, *AuthorizeRequest) (*AuthorizeResponse, error) +} + +// UnimplementedAuthorizerServer should 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 UnimplementedAuthorizerServer struct{} + +func (UnimplementedAuthorizerServer) Authorize(context.Context, *AuthorizeRequest) (*AuthorizeResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method Authorize not implemented") +} +func (UnimplementedAuthorizerServer) testEmbeddedByValue() {} + +// UnsafeAuthorizerServer may be embedded to opt out of forward compatibility for this service. +// Use of this interface is not recommended, as added methods to AuthorizerServer will +// result in compilation errors. +type UnsafeAuthorizerServer interface { + mustEmbedUnimplementedAuthorizerServer() +} + +func RegisterAuthorizerServer(s grpc.ServiceRegistrar, srv AuthorizerServer) { + // If the following call pancis, it indicates UnimplementedAuthorizerServer 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(&Authorizer_ServiceDesc, srv) +} + +func _Authorizer_Authorize_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(AuthorizeRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(AuthorizerServer).Authorize(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: Authorizer_Authorize_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(AuthorizerServer).Authorize(ctx, req.(*AuthorizeRequest)) + } + return interceptor(ctx, in, info, handler) +} + +// Authorizer_ServiceDesc is the grpc.ServiceDesc for Authorizer service. +// It's only intended for direct use with grpc.RegisterService, +// and not to be introspected or modified (even as a copy) +var Authorizer_ServiceDesc = grpc.ServiceDesc{ + ServiceName: "buildbarn.auth.Authorizer", + HandlerType: (*AuthorizerServer)(nil), + Methods: []grpc.MethodDesc{ + { + MethodName: "Authorize", + Handler: _Authorizer_Authorize_Handler, + }, + }, + Streams: []grpc.StreamDesc{}, + Metadata: "pkg/proto/auth/auth.proto", +} diff --git a/pkg/proto/configuration/auth/BUILD.bazel b/pkg/proto/configuration/auth/BUILD.bazel index 886ad4d1..efc01038 100644 --- a/pkg/proto/configuration/auth/BUILD.bazel +++ b/pkg/proto/configuration/auth/BUILD.bazel @@ -6,7 +6,12 @@ proto_library( name = "auth_proto", srcs = ["auth.proto"], visibility = ["//visibility:public"], - deps = ["@protobuf//:empty_proto"], + deps = [ + "//pkg/proto/configuration/eviction:eviction_proto", + "//pkg/proto/configuration/grpc:grpc_proto", + "@protobuf//:empty_proto", + "@protobuf//:struct_proto", + ], ) go_proto_library( @@ -14,6 +19,10 @@ go_proto_library( importpath = "github.com/buildbarn/bb-storage/pkg/proto/configuration/auth", proto = ":auth_proto", visibility = ["//visibility:public"], + deps = [ + "//pkg/proto/configuration/eviction", + "//pkg/proto/configuration/grpc", + ], ) go_library( diff --git a/pkg/proto/configuration/auth/auth.pb.go b/pkg/proto/configuration/auth/auth.pb.go index 9bcd6aa4..ae4767ab 100644 --- a/pkg/proto/configuration/auth/auth.pb.go +++ b/pkg/proto/configuration/auth/auth.pb.go @@ -7,9 +7,12 @@ package auth import ( + eviction "github.com/buildbarn/bb-storage/pkg/proto/configuration/eviction" + grpc "github.com/buildbarn/bb-storage/pkg/proto/configuration/grpc" protoreflect "google.golang.org/protobuf/reflect/protoreflect" protoimpl "google.golang.org/protobuf/runtime/protoimpl" emptypb "google.golang.org/protobuf/types/known/emptypb" + structpb "google.golang.org/protobuf/types/known/structpb" reflect "reflect" sync "sync" unsafe "unsafe" @@ -30,6 +33,7 @@ type AuthorizerConfiguration struct { // *AuthorizerConfiguration_InstanceNamePrefix // *AuthorizerConfiguration_Deny // *AuthorizerConfiguration_JmespathExpression + // *AuthorizerConfiguration_Remote Policy isAuthorizerConfiguration_Policy `protobuf_oneof:"policy"` unknownFields protoimpl.UnknownFields sizeCache protoimpl.SizeCache @@ -108,6 +112,15 @@ func (x *AuthorizerConfiguration) GetJmespathExpression() string { return "" } +func (x *AuthorizerConfiguration) GetRemote() *RemoteAuthorizer { + if x != nil { + if x, ok := x.Policy.(*AuthorizerConfiguration_Remote); ok { + return x.Remote + } + } + return nil +} + type isAuthorizerConfiguration_Policy interface { isAuthorizerConfiguration_Policy() } @@ -128,6 +141,10 @@ type AuthorizerConfiguration_JmespathExpression struct { JmespathExpression string `protobuf:"bytes,4,opt,name=jmespath_expression,json=jmespathExpression,proto3,oneof"` } +type AuthorizerConfiguration_Remote struct { + Remote *RemoteAuthorizer `protobuf:"bytes,7,opt,name=remote,proto3,oneof"` +} + func (*AuthorizerConfiguration_Allow) isAuthorizerConfiguration_Policy() {} func (*AuthorizerConfiguration_InstanceNamePrefix) isAuthorizerConfiguration_Policy() {} @@ -136,6 +153,8 @@ func (*AuthorizerConfiguration_Deny) isAuthorizerConfiguration_Policy() {} func (*AuthorizerConfiguration_JmespathExpression) isAuthorizerConfiguration_Policy() {} +func (*AuthorizerConfiguration_Remote) isAuthorizerConfiguration_Policy() {} + type InstanceNameAuthorizer struct { state protoimpl.MessageState `protogen:"open.v1"` AllowedInstanceNamePrefixes []string `protobuf:"bytes,1,rep,name=allowed_instance_name_prefixes,json=allowedInstanceNamePrefixes,proto3" json:"allowed_instance_name_prefixes,omitempty"` @@ -180,6 +199,74 @@ func (x *InstanceNameAuthorizer) GetAllowedInstanceNamePrefixes() []string { return nil } +type RemoteAuthorizer struct { + state protoimpl.MessageState `protogen:"open.v1"` + Endpoint *grpc.ClientConfiguration `protobuf:"bytes,1,opt,name=endpoint,proto3" json:"endpoint,omitempty"` + Scope *structpb.Value `protobuf:"bytes,2,opt,name=scope,proto3" json:"scope,omitempty"` + MaximumCacheSize int32 `protobuf:"varint,3,opt,name=maximum_cache_size,json=maximumCacheSize,proto3" json:"maximum_cache_size,omitempty"` + CacheReplacementPolicy eviction.CacheReplacementPolicy `protobuf:"varint,4,opt,name=cache_replacement_policy,json=cacheReplacementPolicy,proto3,enum=buildbarn.configuration.eviction.CacheReplacementPolicy" json:"cache_replacement_policy,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *RemoteAuthorizer) Reset() { + *x = RemoteAuthorizer{} + mi := &file_pkg_proto_configuration_auth_auth_proto_msgTypes[2] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *RemoteAuthorizer) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*RemoteAuthorizer) ProtoMessage() {} + +func (x *RemoteAuthorizer) ProtoReflect() protoreflect.Message { + mi := &file_pkg_proto_configuration_auth_auth_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 RemoteAuthorizer.ProtoReflect.Descriptor instead. +func (*RemoteAuthorizer) Descriptor() ([]byte, []int) { + return file_pkg_proto_configuration_auth_auth_proto_rawDescGZIP(), []int{2} +} + +func (x *RemoteAuthorizer) GetEndpoint() *grpc.ClientConfiguration { + if x != nil { + return x.Endpoint + } + return nil +} + +func (x *RemoteAuthorizer) GetScope() *structpb.Value { + if x != nil { + return x.Scope + } + return nil +} + +func (x *RemoteAuthorizer) GetMaximumCacheSize() int32 { + if x != nil { + return x.MaximumCacheSize + } + return 0 +} + +func (x *RemoteAuthorizer) GetCacheReplacementPolicy() eviction.CacheReplacementPolicy { + if x != nil { + return x.CacheReplacementPolicy + } + return eviction.CacheReplacementPolicy(0) +} + var File_pkg_proto_configuration_auth_auth_proto protoreflect.FileDescriptor var file_pkg_proto_configuration_auth_auth_proto_rawDesc = string([]byte{ @@ -189,35 +276,67 @@ var file_pkg_proto_configuration_auth_auth_proto_rawDesc = string([]byte{ 0x62, 0x61, 0x72, 0x6e, 0x2e, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x61, 0x75, 0x74, 0x68, 0x1a, 0x1b, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2f, 0x65, 0x6d, 0x70, 0x74, 0x79, 0x2e, 0x70, - 0x72, 0x6f, 0x74, 0x6f, 0x22, 0x9e, 0x02, 0x0a, 0x17, 0x41, 0x75, 0x74, 0x68, 0x6f, 0x72, 0x69, - 0x7a, 0x65, 0x72, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, - 0x12, 0x2e, 0x0a, 0x05, 0x61, 0x6c, 0x6c, 0x6f, 0x77, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, - 0x16, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, - 0x66, 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x48, 0x00, 0x52, 0x05, 0x61, 0x6c, 0x6c, 0x6f, 0x77, - 0x12, 0x68, 0x0a, 0x14, 0x69, 0x6e, 0x73, 0x74, 0x61, 0x6e, 0x63, 0x65, 0x5f, 0x6e, 0x61, 0x6d, - 0x65, 0x5f, 0x70, 0x72, 0x65, 0x66, 0x69, 0x78, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x34, - 0x2e, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x62, 0x61, 0x72, 0x6e, 0x2e, 0x63, 0x6f, 0x6e, 0x66, 0x69, - 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x61, 0x75, 0x74, 0x68, 0x2e, 0x49, 0x6e, - 0x73, 0x74, 0x61, 0x6e, 0x63, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x41, 0x75, 0x74, 0x68, 0x6f, 0x72, - 0x69, 0x7a, 0x65, 0x72, 0x48, 0x00, 0x52, 0x12, 0x69, 0x6e, 0x73, 0x74, 0x61, 0x6e, 0x63, 0x65, - 0x4e, 0x61, 0x6d, 0x65, 0x50, 0x72, 0x65, 0x66, 0x69, 0x78, 0x12, 0x2c, 0x0a, 0x04, 0x64, 0x65, - 0x6e, 0x79, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x16, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, - 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, - 0x48, 0x00, 0x52, 0x04, 0x64, 0x65, 0x6e, 0x79, 0x12, 0x31, 0x0a, 0x13, 0x6a, 0x6d, 0x65, 0x73, - 0x70, 0x61, 0x74, 0x68, 0x5f, 0x65, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x18, - 0x04, 0x20, 0x01, 0x28, 0x09, 0x48, 0x00, 0x52, 0x12, 0x6a, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x74, - 0x68, 0x45, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x42, 0x08, 0x0a, 0x06, 0x70, - 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x22, 0x5d, 0x0a, 0x16, 0x49, 0x6e, 0x73, 0x74, 0x61, 0x6e, 0x63, - 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x41, 0x75, 0x74, 0x68, 0x6f, 0x72, 0x69, 0x7a, 0x65, 0x72, 0x12, - 0x43, 0x0a, 0x1e, 0x61, 0x6c, 0x6c, 0x6f, 0x77, 0x65, 0x64, 0x5f, 0x69, 0x6e, 0x73, 0x74, 0x61, - 0x6e, 0x63, 0x65, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x5f, 0x70, 0x72, 0x65, 0x66, 0x69, 0x78, 0x65, - 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x09, 0x52, 0x1b, 0x61, 0x6c, 0x6c, 0x6f, 0x77, 0x65, 0x64, - 0x49, 0x6e, 0x73, 0x74, 0x61, 0x6e, 0x63, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x50, 0x72, 0x65, 0x66, - 0x69, 0x78, 0x65, 0x73, 0x42, 0x3e, 0x5a, 0x3c, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, - 0x6f, 0x6d, 0x2f, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x62, 0x61, 0x72, 0x6e, 0x2f, 0x62, 0x62, 0x2d, - 0x73, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x2f, 0x70, 0x6b, 0x67, 0x2f, 0x70, 0x72, 0x6f, 0x74, - 0x6f, 0x2f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2f, - 0x61, 0x75, 0x74, 0x68, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, + 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x1c, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2f, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2f, 0x73, 0x74, 0x72, 0x75, 0x63, 0x74, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x1a, 0x2f, 0x70, 0x6b, 0x67, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x63, 0x6f, + 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2f, 0x65, 0x76, 0x69, 0x63, + 0x74, 0x69, 0x6f, 0x6e, 0x2f, 0x65, 0x76, 0x69, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x1a, 0x27, 0x70, 0x6b, 0x67, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x63, + 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2f, 0x67, 0x72, 0x70, + 0x63, 0x2f, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x22, 0xe8, 0x02, 0x0a, + 0x17, 0x41, 0x75, 0x74, 0x68, 0x6f, 0x72, 0x69, 0x7a, 0x65, 0x72, 0x43, 0x6f, 0x6e, 0x66, 0x69, + 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x2e, 0x0a, 0x05, 0x61, 0x6c, 0x6c, 0x6f, + 0x77, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x16, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x48, + 0x00, 0x52, 0x05, 0x61, 0x6c, 0x6c, 0x6f, 0x77, 0x12, 0x68, 0x0a, 0x14, 0x69, 0x6e, 0x73, 0x74, + 0x61, 0x6e, 0x63, 0x65, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x5f, 0x70, 0x72, 0x65, 0x66, 0x69, 0x78, + 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x34, 0x2e, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x62, 0x61, + 0x72, 0x6e, 0x2e, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, + 0x2e, 0x61, 0x75, 0x74, 0x68, 0x2e, 0x49, 0x6e, 0x73, 0x74, 0x61, 0x6e, 0x63, 0x65, 0x4e, 0x61, + 0x6d, 0x65, 0x41, 0x75, 0x74, 0x68, 0x6f, 0x72, 0x69, 0x7a, 0x65, 0x72, 0x48, 0x00, 0x52, 0x12, + 0x69, 0x6e, 0x73, 0x74, 0x61, 0x6e, 0x63, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x50, 0x72, 0x65, 0x66, + 0x69, 0x78, 0x12, 0x2c, 0x0a, 0x04, 0x64, 0x65, 0x6e, 0x79, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x16, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, + 0x75, 0x66, 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x48, 0x00, 0x52, 0x04, 0x64, 0x65, 0x6e, 0x79, + 0x12, 0x31, 0x0a, 0x13, 0x6a, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x74, 0x68, 0x5f, 0x65, 0x78, 0x70, + 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x48, 0x00, 0x52, + 0x12, 0x6a, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x74, 0x68, 0x45, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73, + 0x69, 0x6f, 0x6e, 0x12, 0x48, 0x0a, 0x06, 0x72, 0x65, 0x6d, 0x6f, 0x74, 0x65, 0x18, 0x07, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x2e, 0x2e, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x62, 0x61, 0x72, 0x6e, 0x2e, + 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x61, 0x75, + 0x74, 0x68, 0x2e, 0x52, 0x65, 0x6d, 0x6f, 0x74, 0x65, 0x41, 0x75, 0x74, 0x68, 0x6f, 0x72, 0x69, + 0x7a, 0x65, 0x72, 0x48, 0x00, 0x52, 0x06, 0x72, 0x65, 0x6d, 0x6f, 0x74, 0x65, 0x42, 0x08, 0x0a, + 0x06, 0x70, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x22, 0x5d, 0x0a, 0x16, 0x49, 0x6e, 0x73, 0x74, 0x61, + 0x6e, 0x63, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x41, 0x75, 0x74, 0x68, 0x6f, 0x72, 0x69, 0x7a, 0x65, + 0x72, 0x12, 0x43, 0x0a, 0x1e, 0x61, 0x6c, 0x6c, 0x6f, 0x77, 0x65, 0x64, 0x5f, 0x69, 0x6e, 0x73, + 0x74, 0x61, 0x6e, 0x63, 0x65, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x5f, 0x70, 0x72, 0x65, 0x66, 0x69, + 0x78, 0x65, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x09, 0x52, 0x1b, 0x61, 0x6c, 0x6c, 0x6f, 0x77, + 0x65, 0x64, 0x49, 0x6e, 0x73, 0x74, 0x61, 0x6e, 0x63, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x50, 0x72, + 0x65, 0x66, 0x69, 0x78, 0x65, 0x73, 0x22, 0xb1, 0x02, 0x0a, 0x10, 0x52, 0x65, 0x6d, 0x6f, 0x74, + 0x65, 0x41, 0x75, 0x74, 0x68, 0x6f, 0x72, 0x69, 0x7a, 0x65, 0x72, 0x12, 0x4d, 0x0a, 0x08, 0x65, + 0x6e, 0x64, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x31, 0x2e, + 0x62, 0x75, 0x69, 0x6c, 0x64, 0x62, 0x61, 0x72, 0x6e, 0x2e, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, + 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x43, 0x6c, 0x69, + 0x65, 0x6e, 0x74, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, + 0x52, 0x08, 0x65, 0x6e, 0x64, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x12, 0x2c, 0x0a, 0x05, 0x73, 0x63, + 0x6f, 0x70, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x16, 0x2e, 0x67, 0x6f, 0x6f, 0x67, + 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x56, 0x61, 0x6c, 0x75, + 0x65, 0x52, 0x05, 0x73, 0x63, 0x6f, 0x70, 0x65, 0x12, 0x2c, 0x0a, 0x12, 0x6d, 0x61, 0x78, 0x69, + 0x6d, 0x75, 0x6d, 0x5f, 0x63, 0x61, 0x63, 0x68, 0x65, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x18, 0x03, + 0x20, 0x01, 0x28, 0x05, 0x52, 0x10, 0x6d, 0x61, 0x78, 0x69, 0x6d, 0x75, 0x6d, 0x43, 0x61, 0x63, + 0x68, 0x65, 0x53, 0x69, 0x7a, 0x65, 0x12, 0x72, 0x0a, 0x18, 0x63, 0x61, 0x63, 0x68, 0x65, 0x5f, + 0x72, 0x65, 0x70, 0x6c, 0x61, 0x63, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x5f, 0x70, 0x6f, 0x6c, 0x69, + 0x63, 0x79, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x38, 0x2e, 0x62, 0x75, 0x69, 0x6c, 0x64, + 0x62, 0x61, 0x72, 0x6e, 0x2e, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, + 0x6f, 0x6e, 0x2e, 0x65, 0x76, 0x69, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x43, 0x61, 0x63, 0x68, + 0x65, 0x52, 0x65, 0x70, 0x6c, 0x61, 0x63, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x50, 0x6f, 0x6c, 0x69, + 0x63, 0x79, 0x52, 0x16, 0x63, 0x61, 0x63, 0x68, 0x65, 0x52, 0x65, 0x70, 0x6c, 0x61, 0x63, 0x65, + 0x6d, 0x65, 0x6e, 0x74, 0x50, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x42, 0x3e, 0x5a, 0x3c, 0x67, 0x69, + 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x62, 0x61, + 0x72, 0x6e, 0x2f, 0x62, 0x62, 0x2d, 0x73, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x2f, 0x70, 0x6b, + 0x67, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, + 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2f, 0x61, 0x75, 0x74, 0x68, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x33, }) var ( @@ -232,21 +351,29 @@ func file_pkg_proto_configuration_auth_auth_proto_rawDescGZIP() []byte { return file_pkg_proto_configuration_auth_auth_proto_rawDescData } -var file_pkg_proto_configuration_auth_auth_proto_msgTypes = make([]protoimpl.MessageInfo, 2) +var file_pkg_proto_configuration_auth_auth_proto_msgTypes = make([]protoimpl.MessageInfo, 3) var file_pkg_proto_configuration_auth_auth_proto_goTypes = []any{ - (*AuthorizerConfiguration)(nil), // 0: buildbarn.configuration.auth.AuthorizerConfiguration - (*InstanceNameAuthorizer)(nil), // 1: buildbarn.configuration.auth.InstanceNameAuthorizer - (*emptypb.Empty)(nil), // 2: google.protobuf.Empty + (*AuthorizerConfiguration)(nil), // 0: buildbarn.configuration.auth.AuthorizerConfiguration + (*InstanceNameAuthorizer)(nil), // 1: buildbarn.configuration.auth.InstanceNameAuthorizer + (*RemoteAuthorizer)(nil), // 2: buildbarn.configuration.auth.RemoteAuthorizer + (*emptypb.Empty)(nil), // 3: google.protobuf.Empty + (*grpc.ClientConfiguration)(nil), // 4: buildbarn.configuration.grpc.ClientConfiguration + (*structpb.Value)(nil), // 5: google.protobuf.Value + (eviction.CacheReplacementPolicy)(0), // 6: buildbarn.configuration.eviction.CacheReplacementPolicy } var file_pkg_proto_configuration_auth_auth_proto_depIdxs = []int32{ - 2, // 0: buildbarn.configuration.auth.AuthorizerConfiguration.allow:type_name -> google.protobuf.Empty + 3, // 0: buildbarn.configuration.auth.AuthorizerConfiguration.allow:type_name -> google.protobuf.Empty 1, // 1: buildbarn.configuration.auth.AuthorizerConfiguration.instance_name_prefix:type_name -> buildbarn.configuration.auth.InstanceNameAuthorizer - 2, // 2: buildbarn.configuration.auth.AuthorizerConfiguration.deny:type_name -> google.protobuf.Empty - 3, // [3:3] is the sub-list for method output_type - 3, // [3:3] is the sub-list for method input_type - 3, // [3:3] is the sub-list for extension type_name - 3, // [3:3] is the sub-list for extension extendee - 0, // [0:3] is the sub-list for field type_name + 3, // 2: buildbarn.configuration.auth.AuthorizerConfiguration.deny:type_name -> google.protobuf.Empty + 2, // 3: buildbarn.configuration.auth.AuthorizerConfiguration.remote:type_name -> buildbarn.configuration.auth.RemoteAuthorizer + 4, // 4: buildbarn.configuration.auth.RemoteAuthorizer.endpoint:type_name -> buildbarn.configuration.grpc.ClientConfiguration + 5, // 5: buildbarn.configuration.auth.RemoteAuthorizer.scope:type_name -> google.protobuf.Value + 6, // 6: buildbarn.configuration.auth.RemoteAuthorizer.cache_replacement_policy:type_name -> buildbarn.configuration.eviction.CacheReplacementPolicy + 7, // [7:7] is the sub-list for method output_type + 7, // [7:7] is the sub-list for method input_type + 7, // [7:7] is the sub-list for extension type_name + 7, // [7:7] is the sub-list for extension extendee + 0, // [0:7] is the sub-list for field type_name } func init() { file_pkg_proto_configuration_auth_auth_proto_init() } @@ -259,6 +386,7 @@ func file_pkg_proto_configuration_auth_auth_proto_init() { (*AuthorizerConfiguration_InstanceNamePrefix)(nil), (*AuthorizerConfiguration_Deny)(nil), (*AuthorizerConfiguration_JmespathExpression)(nil), + (*AuthorizerConfiguration_Remote)(nil), } type x struct{} out := protoimpl.TypeBuilder{ @@ -266,7 +394,7 @@ func file_pkg_proto_configuration_auth_auth_proto_init() { GoPackagePath: reflect.TypeOf(x{}).PkgPath(), RawDescriptor: unsafe.Slice(unsafe.StringData(file_pkg_proto_configuration_auth_auth_proto_rawDesc), len(file_pkg_proto_configuration_auth_auth_proto_rawDesc)), NumEnums: 0, - NumMessages: 2, + NumMessages: 3, NumExtensions: 0, NumServices: 0, }, diff --git a/pkg/proto/configuration/auth/auth.proto b/pkg/proto/configuration/auth/auth.proto index 1ea7c037..57dff231 100644 --- a/pkg/proto/configuration/auth/auth.proto +++ b/pkg/proto/configuration/auth/auth.proto @@ -3,6 +3,9 @@ syntax = "proto3"; package buildbarn.configuration.auth; import "google/protobuf/empty.proto"; +import "google/protobuf/struct.proto"; +import "pkg/proto/configuration/eviction/eviction.proto"; +import "pkg/proto/configuration/grpc/grpc.proto"; option go_package = "github.com/buildbarn/bb-storage/pkg/proto/configuration/auth"; @@ -36,6 +39,9 @@ message AuthorizerConfiguration { // "instanceName" corresponds to the REv2 instance name that was // part of the client request. string jmespath_expression = 4; + + // Forwards the authorizing task to a remote server. + RemoteAuthorizer remote = 7; } } @@ -44,3 +50,22 @@ message InstanceNameAuthorizer { // The empty string may be used to indicate all instance names. repeated string allowed_instance_name_prefixes = 1; } + +message RemoteAuthorizer { + // The remote buildbarn.auth.AuthorizerServer to connect. + buildbarn.configuration.grpc.ClientConfiguration endpoint = 1; + + // A description of, for example, what subsystem is being accessed and what + // kind of request is being made. This passed to a + // buildbarn.auth.AuthorizeRequest in literal form. + google.protobuf.Value scope = 2; + + // Maximum number of authorizer responses to cache in memory. This speeds + // up successive requests made with the same metadata. + int32 maximum_cache_size = 3; + + // The cache replacement policy that should be applied to the token + // cache. It is advised that this is set to LEAST_RECENTLY_USED. + buildbarn.configuration.eviction.CacheReplacementPolicy + cache_replacement_policy = 4; +} diff --git a/pkg/proto/configuration/grpc/BUILD.bazel b/pkg/proto/configuration/grpc/BUILD.bazel index 39b17cda..9528aa97 100644 --- a/pkg/proto/configuration/grpc/BUILD.bazel +++ b/pkg/proto/configuration/grpc/BUILD.bazel @@ -8,6 +8,7 @@ proto_library( visibility = ["//visibility:public"], deps = [ "//pkg/proto/auth:auth_proto", + "//pkg/proto/configuration/eviction:eviction_proto", "//pkg/proto/configuration/jwt:jwt_proto", "//pkg/proto/configuration/tls:tls_proto", "@protobuf//:duration_proto", @@ -23,6 +24,7 @@ go_proto_library( visibility = ["//visibility:public"], deps = [ "//pkg/proto/auth", + "//pkg/proto/configuration/eviction", "//pkg/proto/configuration/jwt", "//pkg/proto/configuration/tls", ], diff --git a/pkg/proto/configuration/grpc/grpc.pb.go b/pkg/proto/configuration/grpc/grpc.pb.go index d93946cc..d78baf08 100644 --- a/pkg/proto/configuration/grpc/grpc.pb.go +++ b/pkg/proto/configuration/grpc/grpc.pb.go @@ -8,6 +8,7 @@ package grpc import ( auth "github.com/buildbarn/bb-storage/pkg/proto/auth" + eviction "github.com/buildbarn/bb-storage/pkg/proto/configuration/eviction" jwt "github.com/buildbarn/bb-storage/pkg/proto/configuration/jwt" tls "github.com/buildbarn/bb-storage/pkg/proto/configuration/tls" protoreflect "google.golang.org/protobuf/reflect/protoreflect" @@ -572,6 +573,7 @@ type AuthenticationPolicy struct { // *AuthenticationPolicy_TlsClientCertificate // *AuthenticationPolicy_Jwt // *AuthenticationPolicy_PeerCredentialsJmespathExpression + // *AuthenticationPolicy_Remote Policy isAuthenticationPolicy_Policy `protobuf_oneof:"policy"` unknownFields protoimpl.UnknownFields sizeCache protoimpl.SizeCache @@ -677,6 +679,15 @@ func (x *AuthenticationPolicy) GetPeerCredentialsJmespathExpression() string { return "" } +func (x *AuthenticationPolicy) GetRemote() *RemoteAuthenticationPolicy { + if x != nil { + if x, ok := x.Policy.(*AuthenticationPolicy_Remote); ok { + return x.Remote + } + } + return nil +} + type isAuthenticationPolicy_Policy interface { isAuthenticationPolicy_Policy() } @@ -709,6 +720,10 @@ type AuthenticationPolicy_PeerCredentialsJmespathExpression struct { PeerCredentialsJmespathExpression string `protobuf:"bytes,6,opt,name=peer_credentials_jmespath_expression,json=peerCredentialsJmespathExpression,proto3,oneof"` } +type AuthenticationPolicy_Remote struct { + Remote *RemoteAuthenticationPolicy `protobuf:"bytes,8,opt,name=remote,proto3,oneof"` +} + func (*AuthenticationPolicy_Allow) isAuthenticationPolicy_Policy() {} func (*AuthenticationPolicy_Any) isAuthenticationPolicy_Policy() {} @@ -723,6 +738,8 @@ func (*AuthenticationPolicy_Jwt) isAuthenticationPolicy_Policy() {} func (*AuthenticationPolicy_PeerCredentialsJmespathExpression) isAuthenticationPolicy_Policy() {} +func (*AuthenticationPolicy_Remote) isAuthenticationPolicy_Policy() {} + type AnyAuthenticationPolicy struct { state protoimpl.MessageState `protogen:"open.v1"` Policies []*AuthenticationPolicy `protobuf:"bytes,1,rep,name=policies,proto3" json:"policies,omitempty"` @@ -871,6 +888,82 @@ func (x *TLSClientCertificateAuthenticationPolicy) GetMetadataExtractionJmespath return "" } +type RemoteAuthenticationPolicy struct { + state protoimpl.MessageState `protogen:"open.v1"` + Headers []string `protobuf:"bytes,1,rep,name=headers,proto3" json:"headers,omitempty"` + Endpoint *ClientConfiguration `protobuf:"bytes,2,opt,name=endpoint,proto3" json:"endpoint,omitempty"` + Scope *structpb.Value `protobuf:"bytes,3,opt,name=scope,proto3" json:"scope,omitempty"` + MaximumCacheSize int32 `protobuf:"varint,4,opt,name=maximum_cache_size,json=maximumCacheSize,proto3" json:"maximum_cache_size,omitempty"` + CacheReplacementPolicy eviction.CacheReplacementPolicy `protobuf:"varint,5,opt,name=cache_replacement_policy,json=cacheReplacementPolicy,proto3,enum=buildbarn.configuration.eviction.CacheReplacementPolicy" json:"cache_replacement_policy,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *RemoteAuthenticationPolicy) Reset() { + *x = RemoteAuthenticationPolicy{} + mi := &file_pkg_proto_configuration_grpc_grpc_proto_msgTypes[10] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *RemoteAuthenticationPolicy) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*RemoteAuthenticationPolicy) ProtoMessage() {} + +func (x *RemoteAuthenticationPolicy) ProtoReflect() protoreflect.Message { + mi := &file_pkg_proto_configuration_grpc_grpc_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 RemoteAuthenticationPolicy.ProtoReflect.Descriptor instead. +func (*RemoteAuthenticationPolicy) Descriptor() ([]byte, []int) { + return file_pkg_proto_configuration_grpc_grpc_proto_rawDescGZIP(), []int{10} +} + +func (x *RemoteAuthenticationPolicy) GetHeaders() []string { + if x != nil { + return x.Headers + } + return nil +} + +func (x *RemoteAuthenticationPolicy) GetEndpoint() *ClientConfiguration { + if x != nil { + return x.Endpoint + } + return nil +} + +func (x *RemoteAuthenticationPolicy) GetScope() *structpb.Value { + if x != nil { + return x.Scope + } + return nil +} + +func (x *RemoteAuthenticationPolicy) GetMaximumCacheSize() int32 { + if x != nil { + return x.MaximumCacheSize + } + return 0 +} + +func (x *RemoteAuthenticationPolicy) GetCacheReplacementPolicy() eviction.CacheReplacementPolicy { + if x != nil { + return x.CacheReplacementPolicy + } + return eviction.CacheReplacementPolicy(0) +} + type TracingMethodConfiguration struct { state protoimpl.MessageState `protogen:"open.v1"` AttributesFromFirstRequestMessage []string `protobuf:"bytes,1,rep,name=attributes_from_first_request_message,json=attributesFromFirstRequestMessage,proto3" json:"attributes_from_first_request_message,omitempty"` @@ -881,7 +974,7 @@ type TracingMethodConfiguration struct { func (x *TracingMethodConfiguration) Reset() { *x = TracingMethodConfiguration{} - mi := &file_pkg_proto_configuration_grpc_grpc_proto_msgTypes[10] + mi := &file_pkg_proto_configuration_grpc_grpc_proto_msgTypes[11] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -893,7 +986,7 @@ func (x *TracingMethodConfiguration) String() string { func (*TracingMethodConfiguration) ProtoMessage() {} func (x *TracingMethodConfiguration) ProtoReflect() protoreflect.Message { - mi := &file_pkg_proto_configuration_grpc_grpc_proto_msgTypes[10] + mi := &file_pkg_proto_configuration_grpc_grpc_proto_msgTypes[11] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -906,7 +999,7 @@ func (x *TracingMethodConfiguration) ProtoReflect() protoreflect.Message { // Deprecated: Use TracingMethodConfiguration.ProtoReflect.Descriptor instead. func (*TracingMethodConfiguration) Descriptor() ([]byte, []int) { - return file_pkg_proto_configuration_grpc_grpc_proto_rawDescGZIP(), []int{10} + return file_pkg_proto_configuration_grpc_grpc_proto_rawDescGZIP(), []int{11} } func (x *TracingMethodConfiguration) GetAttributesFromFirstRequestMessage() []string { @@ -933,7 +1026,7 @@ type ClientConfiguration_HeaderValues struct { func (x *ClientConfiguration_HeaderValues) Reset() { *x = ClientConfiguration_HeaderValues{} - mi := &file_pkg_proto_configuration_grpc_grpc_proto_msgTypes[11] + mi := &file_pkg_proto_configuration_grpc_grpc_proto_msgTypes[12] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -945,7 +1038,7 @@ func (x *ClientConfiguration_HeaderValues) String() string { func (*ClientConfiguration_HeaderValues) ProtoMessage() {} func (x *ClientConfiguration_HeaderValues) ProtoReflect() protoreflect.Message { - mi := &file_pkg_proto_configuration_grpc_grpc_proto_msgTypes[11] + mi := &file_pkg_proto_configuration_grpc_grpc_proto_msgTypes[12] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -989,276 +1082,306 @@ var file_pkg_proto_configuration_grpc_grpc_proto_rawDesc = string([]byte{ 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x1c, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2f, 0x73, 0x74, 0x72, 0x75, 0x63, 0x74, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x19, 0x70, 0x6b, 0x67, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x61, 0x75, - 0x74, 0x68, 0x2f, 0x61, 0x75, 0x74, 0x68, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x25, 0x70, + 0x74, 0x68, 0x2f, 0x61, 0x75, 0x74, 0x68, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x2f, 0x70, 0x6b, 0x67, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, - 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2f, 0x6a, 0x77, 0x74, 0x2f, 0x6a, 0x77, 0x74, 0x2e, 0x70, - 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x25, 0x70, 0x6b, 0x67, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, - 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2f, 0x74, 0x6c, - 0x73, 0x2f, 0x74, 0x6c, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x22, 0xce, 0x07, 0x0a, 0x13, - 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, - 0x69, 0x6f, 0x6e, 0x12, 0x18, 0x0a, 0x07, 0x61, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x18, 0x01, - 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x61, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x12, 0x42, 0x0a, - 0x03, 0x74, 0x6c, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x30, 0x2e, 0x62, 0x75, 0x69, - 0x6c, 0x64, 0x62, 0x61, 0x72, 0x6e, 0x2e, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, - 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x74, 0x6c, 0x73, 0x2e, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x43, - 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x03, 0x74, 0x6c, - 0x73, 0x12, 0x58, 0x0a, 0x09, 0x6b, 0x65, 0x65, 0x70, 0x61, 0x6c, 0x69, 0x76, 0x65, 0x18, 0x03, - 0x20, 0x01, 0x28, 0x0b, 0x32, 0x3a, 0x2e, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x62, 0x61, 0x72, 0x6e, - 0x2e, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x67, - 0x72, 0x70, 0x63, 0x2e, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x4b, 0x65, 0x65, 0x70, 0x61, 0x6c, - 0x69, 0x76, 0x65, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, - 0x52, 0x09, 0x6b, 0x65, 0x65, 0x70, 0x61, 0x6c, 0x69, 0x76, 0x65, 0x12, 0x61, 0x0a, 0x0c, 0x61, - 0x64, 0x64, 0x5f, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x18, 0x05, 0x20, 0x03, 0x28, - 0x0b, 0x32, 0x3e, 0x2e, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x62, 0x61, 0x72, 0x6e, 0x2e, 0x63, 0x6f, - 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x67, 0x72, 0x70, 0x63, - 0x2e, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, - 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x56, 0x61, 0x6c, 0x75, 0x65, - 0x73, 0x52, 0x0b, 0x61, 0x64, 0x64, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x12, 0x47, - 0x0a, 0x20, 0x61, 0x64, 0x64, 0x5f, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x5f, 0x6a, - 0x6d, 0x65, 0x73, 0x70, 0x61, 0x74, 0x68, 0x5f, 0x65, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, - 0x6f, 0x6e, 0x18, 0x0c, 0x20, 0x01, 0x28, 0x09, 0x52, 0x1d, 0x61, 0x64, 0x64, 0x4d, 0x65, 0x74, - 0x61, 0x64, 0x61, 0x74, 0x61, 0x4a, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x74, 0x68, 0x45, 0x78, 0x70, - 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x4c, 0x0a, 0x05, 0x6f, 0x61, 0x75, 0x74, 0x68, - 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x36, 0x2e, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x62, 0x61, - 0x72, 0x6e, 0x2e, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, - 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x4f, 0x41, 0x75, 0x74, - 0x68, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x05, - 0x6f, 0x61, 0x75, 0x74, 0x68, 0x12, 0x39, 0x0a, 0x19, 0x69, 0x6e, 0x69, 0x74, 0x69, 0x61, 0x6c, - 0x5f, 0x77, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x5f, 0x62, 0x79, 0x74, - 0x65, 0x73, 0x18, 0x08, 0x20, 0x01, 0x28, 0x05, 0x52, 0x16, 0x69, 0x6e, 0x69, 0x74, 0x69, 0x61, - 0x6c, 0x57, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x53, 0x69, 0x7a, 0x65, 0x42, 0x79, 0x74, 0x65, 0x73, - 0x12, 0x42, 0x0a, 0x1e, 0x69, 0x6e, 0x69, 0x74, 0x69, 0x61, 0x6c, 0x5f, 0x63, 0x6f, 0x6e, 0x6e, - 0x5f, 0x77, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x5f, 0x62, 0x79, 0x74, - 0x65, 0x73, 0x18, 0x09, 0x20, 0x01, 0x28, 0x05, 0x52, 0x1a, 0x69, 0x6e, 0x69, 0x74, 0x69, 0x61, - 0x6c, 0x43, 0x6f, 0x6e, 0x6e, 0x57, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x53, 0x69, 0x7a, 0x65, 0x42, - 0x79, 0x74, 0x65, 0x73, 0x12, 0x1b, 0x0a, 0x09, 0x70, 0x72, 0x6f, 0x78, 0x79, 0x5f, 0x75, 0x72, - 0x6c, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x70, 0x72, 0x6f, 0x78, 0x79, 0x55, 0x72, - 0x6c, 0x12, 0x58, 0x0a, 0x07, 0x74, 0x72, 0x61, 0x63, 0x69, 0x6e, 0x67, 0x18, 0x0b, 0x20, 0x03, + 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2f, 0x65, 0x76, 0x69, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x2f, + 0x65, 0x76, 0x69, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x25, + 0x70, 0x6b, 0x67, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, + 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2f, 0x6a, 0x77, 0x74, 0x2f, 0x6a, 0x77, 0x74, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x25, 0x70, 0x6b, 0x67, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x2f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2f, 0x74, + 0x6c, 0x73, 0x2f, 0x74, 0x6c, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x22, 0xce, 0x07, 0x0a, + 0x13, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, + 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x18, 0x0a, 0x07, 0x61, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x61, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x12, 0x42, + 0x0a, 0x03, 0x74, 0x6c, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x30, 0x2e, 0x62, 0x75, + 0x69, 0x6c, 0x64, 0x62, 0x61, 0x72, 0x6e, 0x2e, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, + 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x74, 0x6c, 0x73, 0x2e, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, + 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x03, 0x74, + 0x6c, 0x73, 0x12, 0x58, 0x0a, 0x09, 0x6b, 0x65, 0x65, 0x70, 0x61, 0x6c, 0x69, 0x76, 0x65, 0x18, + 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x3a, 0x2e, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x62, 0x61, 0x72, + 0x6e, 0x2e, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, + 0x67, 0x72, 0x70, 0x63, 0x2e, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x4b, 0x65, 0x65, 0x70, 0x61, + 0x6c, 0x69, 0x76, 0x65, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, + 0x6e, 0x52, 0x09, 0x6b, 0x65, 0x65, 0x70, 0x61, 0x6c, 0x69, 0x76, 0x65, 0x12, 0x61, 0x0a, 0x0c, + 0x61, 0x64, 0x64, 0x5f, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x18, 0x05, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x3e, 0x2e, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x62, 0x61, 0x72, 0x6e, 0x2e, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, - 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x54, 0x72, 0x61, 0x63, 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, - 0x72, 0x79, 0x52, 0x07, 0x74, 0x72, 0x61, 0x63, 0x69, 0x6e, 0x67, 0x12, 0x4d, 0x0a, 0x16, 0x64, - 0x65, 0x66, 0x61, 0x75, 0x6c, 0x74, 0x5f, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x5f, 0x63, - 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x0d, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x67, 0x6f, - 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x53, 0x74, - 0x72, 0x75, 0x63, 0x74, 0x52, 0x14, 0x64, 0x65, 0x66, 0x61, 0x75, 0x6c, 0x74, 0x53, 0x65, 0x72, - 0x76, 0x69, 0x63, 0x65, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x1a, 0x3e, 0x0a, 0x0c, 0x48, 0x65, - 0x61, 0x64, 0x65, 0x72, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x73, 0x12, 0x16, 0x0a, 0x06, 0x68, 0x65, - 0x61, 0x64, 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x68, 0x65, 0x61, 0x64, - 0x65, 0x72, 0x12, 0x16, 0x0a, 0x06, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x73, 0x18, 0x02, 0x20, 0x03, - 0x28, 0x09, 0x52, 0x06, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x73, 0x1a, 0x74, 0x0a, 0x0c, 0x54, 0x72, - 0x61, 0x63, 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, - 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x4e, 0x0a, 0x05, - 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x38, 0x2e, 0x62, 0x75, - 0x69, 0x6c, 0x64, 0x62, 0x61, 0x72, 0x6e, 0x2e, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, - 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x54, 0x72, 0x61, 0x63, 0x69, - 0x6e, 0x67, 0x4d, 0x65, 0x74, 0x68, 0x6f, 0x64, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, - 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, - 0x4a, 0x04, 0x08, 0x04, 0x10, 0x05, 0x4a, 0x04, 0x08, 0x07, 0x10, 0x08, 0x22, 0xb6, 0x01, 0x0a, - 0x1c, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x4b, 0x65, 0x65, 0x70, 0x61, 0x6c, 0x69, 0x76, 0x65, - 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x2d, 0x0a, - 0x04, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x67, 0x6f, - 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x44, 0x75, - 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x04, 0x74, 0x69, 0x6d, 0x65, 0x12, 0x33, 0x0a, 0x07, - 0x74, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, - 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, - 0x44, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x07, 0x74, 0x69, 0x6d, 0x65, 0x6f, 0x75, - 0x74, 0x12, 0x32, 0x0a, 0x15, 0x70, 0x65, 0x72, 0x6d, 0x69, 0x74, 0x5f, 0x77, 0x69, 0x74, 0x68, - 0x6f, 0x75, 0x74, 0x5f, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x18, 0x03, 0x20, 0x01, 0x28, 0x08, - 0x52, 0x13, 0x70, 0x65, 0x72, 0x6d, 0x69, 0x74, 0x57, 0x69, 0x74, 0x68, 0x6f, 0x75, 0x74, 0x53, - 0x74, 0x72, 0x65, 0x61, 0x6d, 0x22, 0xcb, 0x01, 0x0a, 0x18, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, - 0x4f, 0x41, 0x75, 0x74, 0x68, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, - 0x6f, 0x6e, 0x12, 0x56, 0x0a, 0x1a, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x5f, 0x64, 0x65, 0x66, - 0x61, 0x75, 0x6c, 0x74, 0x5f, 0x63, 0x72, 0x65, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x61, 0x6c, 0x73, - 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x16, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, - 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x48, 0x00, - 0x52, 0x18, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x44, 0x65, 0x66, 0x61, 0x75, 0x6c, 0x74, 0x43, - 0x72, 0x65, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x61, 0x6c, 0x73, 0x12, 0x30, 0x0a, 0x13, 0x73, 0x65, - 0x72, 0x76, 0x69, 0x63, 0x65, 0x5f, 0x61, 0x63, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x5f, 0x6b, 0x65, - 0x79, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x48, 0x00, 0x52, 0x11, 0x73, 0x65, 0x72, 0x76, 0x69, - 0x63, 0x65, 0x41, 0x63, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x4b, 0x65, 0x79, 0x12, 0x16, 0x0a, 0x06, - 0x73, 0x63, 0x6f, 0x70, 0x65, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x09, 0x52, 0x06, 0x73, 0x63, - 0x6f, 0x70, 0x65, 0x73, 0x42, 0x0d, 0x0a, 0x0b, 0x63, 0x72, 0x65, 0x64, 0x65, 0x6e, 0x74, 0x69, - 0x61, 0x6c, 0x73, 0x22, 0xf7, 0x07, 0x0a, 0x13, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x43, 0x6f, - 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x29, 0x0a, 0x10, 0x6c, - 0x69, 0x73, 0x74, 0x65, 0x6e, 0x5f, 0x61, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x65, 0x73, 0x18, - 0x01, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0f, 0x6c, 0x69, 0x73, 0x74, 0x65, 0x6e, 0x41, 0x64, 0x64, - 0x72, 0x65, 0x73, 0x73, 0x65, 0x73, 0x12, 0x21, 0x0a, 0x0c, 0x6c, 0x69, 0x73, 0x74, 0x65, 0x6e, - 0x5f, 0x70, 0x61, 0x74, 0x68, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0b, 0x6c, 0x69, - 0x73, 0x74, 0x65, 0x6e, 0x50, 0x61, 0x74, 0x68, 0x73, 0x12, 0x42, 0x0a, 0x03, 0x74, 0x6c, 0x73, - 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x30, 0x2e, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x62, 0x61, - 0x72, 0x6e, 0x2e, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, - 0x2e, 0x74, 0x6c, 0x73, 0x2e, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x43, 0x6f, 0x6e, 0x66, 0x69, - 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x03, 0x74, 0x6c, 0x73, 0x12, 0x67, 0x0a, - 0x15, 0x61, 0x75, 0x74, 0x68, 0x65, 0x6e, 0x74, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, - 0x70, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x32, 0x2e, 0x62, + 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x56, 0x61, 0x6c, 0x75, + 0x65, 0x73, 0x52, 0x0b, 0x61, 0x64, 0x64, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x12, + 0x47, 0x0a, 0x20, 0x61, 0x64, 0x64, 0x5f, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x5f, + 0x6a, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x74, 0x68, 0x5f, 0x65, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73, + 0x69, 0x6f, 0x6e, 0x18, 0x0c, 0x20, 0x01, 0x28, 0x09, 0x52, 0x1d, 0x61, 0x64, 0x64, 0x4d, 0x65, + 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x4a, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x74, 0x68, 0x45, 0x78, + 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x4c, 0x0a, 0x05, 0x6f, 0x61, 0x75, 0x74, + 0x68, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x36, 0x2e, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x62, + 0x61, 0x72, 0x6e, 0x2e, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, + 0x6e, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x4f, 0x41, 0x75, + 0x74, 0x68, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, + 0x05, 0x6f, 0x61, 0x75, 0x74, 0x68, 0x12, 0x39, 0x0a, 0x19, 0x69, 0x6e, 0x69, 0x74, 0x69, 0x61, + 0x6c, 0x5f, 0x77, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x5f, 0x62, 0x79, + 0x74, 0x65, 0x73, 0x18, 0x08, 0x20, 0x01, 0x28, 0x05, 0x52, 0x16, 0x69, 0x6e, 0x69, 0x74, 0x69, + 0x61, 0x6c, 0x57, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x53, 0x69, 0x7a, 0x65, 0x42, 0x79, 0x74, 0x65, + 0x73, 0x12, 0x42, 0x0a, 0x1e, 0x69, 0x6e, 0x69, 0x74, 0x69, 0x61, 0x6c, 0x5f, 0x63, 0x6f, 0x6e, + 0x6e, 0x5f, 0x77, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x5f, 0x62, 0x79, + 0x74, 0x65, 0x73, 0x18, 0x09, 0x20, 0x01, 0x28, 0x05, 0x52, 0x1a, 0x69, 0x6e, 0x69, 0x74, 0x69, + 0x61, 0x6c, 0x43, 0x6f, 0x6e, 0x6e, 0x57, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x53, 0x69, 0x7a, 0x65, + 0x42, 0x79, 0x74, 0x65, 0x73, 0x12, 0x1b, 0x0a, 0x09, 0x70, 0x72, 0x6f, 0x78, 0x79, 0x5f, 0x75, + 0x72, 0x6c, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x70, 0x72, 0x6f, 0x78, 0x79, 0x55, + 0x72, 0x6c, 0x12, 0x58, 0x0a, 0x07, 0x74, 0x72, 0x61, 0x63, 0x69, 0x6e, 0x67, 0x18, 0x0b, 0x20, + 0x03, 0x28, 0x0b, 0x32, 0x3e, 0x2e, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x62, 0x61, 0x72, 0x6e, 0x2e, + 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x67, 0x72, + 0x70, 0x63, 0x2e, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, + 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x54, 0x72, 0x61, 0x63, 0x69, 0x6e, 0x67, 0x45, 0x6e, + 0x74, 0x72, 0x79, 0x52, 0x07, 0x74, 0x72, 0x61, 0x63, 0x69, 0x6e, 0x67, 0x12, 0x4d, 0x0a, 0x16, + 0x64, 0x65, 0x66, 0x61, 0x75, 0x6c, 0x74, 0x5f, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x5f, + 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x0d, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x67, + 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x53, + 0x74, 0x72, 0x75, 0x63, 0x74, 0x52, 0x14, 0x64, 0x65, 0x66, 0x61, 0x75, 0x6c, 0x74, 0x53, 0x65, + 0x72, 0x76, 0x69, 0x63, 0x65, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x1a, 0x3e, 0x0a, 0x0c, 0x48, + 0x65, 0x61, 0x64, 0x65, 0x72, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x73, 0x12, 0x16, 0x0a, 0x06, 0x68, + 0x65, 0x61, 0x64, 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x68, 0x65, 0x61, + 0x64, 0x65, 0x72, 0x12, 0x16, 0x0a, 0x06, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x73, 0x18, 0x02, 0x20, + 0x03, 0x28, 0x09, 0x52, 0x06, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x73, 0x1a, 0x74, 0x0a, 0x0c, 0x54, + 0x72, 0x61, 0x63, 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, + 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x4e, 0x0a, + 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x38, 0x2e, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x62, 0x61, 0x72, 0x6e, 0x2e, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, - 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x41, 0x75, 0x74, 0x68, - 0x65, 0x6e, 0x74, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x6f, 0x6c, 0x69, 0x63, 0x79, - 0x52, 0x14, 0x61, 0x75, 0x74, 0x68, 0x65, 0x6e, 0x74, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, - 0x50, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x12, 0x4c, 0x0a, 0x23, 0x6d, 0x61, 0x78, 0x69, 0x6d, 0x75, - 0x6d, 0x5f, 0x72, 0x65, 0x63, 0x65, 0x69, 0x76, 0x65, 0x64, 0x5f, 0x6d, 0x65, 0x73, 0x73, 0x61, - 0x67, 0x65, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x5f, 0x62, 0x79, 0x74, 0x65, 0x73, 0x18, 0x05, 0x20, - 0x01, 0x28, 0x03, 0x52, 0x1f, 0x6d, 0x61, 0x78, 0x69, 0x6d, 0x75, 0x6d, 0x52, 0x65, 0x63, 0x65, - 0x69, 0x76, 0x65, 0x64, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x53, 0x69, 0x7a, 0x65, 0x42, - 0x79, 0x74, 0x65, 0x73, 0x12, 0x80, 0x01, 0x0a, 0x1c, 0x6b, 0x65, 0x65, 0x70, 0x61, 0x6c, 0x69, - 0x76, 0x65, 0x5f, 0x65, 0x6e, 0x66, 0x6f, 0x72, 0x63, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x5f, 0x70, - 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x3e, 0x2e, 0x62, 0x75, - 0x69, 0x6c, 0x64, 0x62, 0x61, 0x72, 0x6e, 0x2e, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, - 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x53, 0x65, 0x72, 0x76, 0x65, - 0x72, 0x4b, 0x65, 0x65, 0x70, 0x61, 0x6c, 0x69, 0x76, 0x65, 0x45, 0x6e, 0x66, 0x6f, 0x72, 0x63, - 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x50, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x52, 0x1a, 0x6b, 0x65, 0x65, - 0x70, 0x61, 0x6c, 0x69, 0x76, 0x65, 0x45, 0x6e, 0x66, 0x6f, 0x72, 0x63, 0x65, 0x6d, 0x65, 0x6e, - 0x74, 0x50, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x12, 0x30, 0x0a, 0x14, 0x68, 0x65, 0x61, 0x6c, 0x74, - 0x68, 0x5f, 0x63, 0x68, 0x65, 0x63, 0x6b, 0x5f, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x18, - 0x07, 0x20, 0x01, 0x28, 0x09, 0x52, 0x12, 0x68, 0x65, 0x61, 0x6c, 0x74, 0x68, 0x43, 0x68, 0x65, - 0x63, 0x6b, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x12, 0x39, 0x0a, 0x19, 0x69, 0x6e, 0x69, - 0x74, 0x69, 0x61, 0x6c, 0x5f, 0x77, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x5f, 0x73, 0x69, 0x7a, 0x65, - 0x5f, 0x62, 0x79, 0x74, 0x65, 0x73, 0x18, 0x08, 0x20, 0x01, 0x28, 0x05, 0x52, 0x16, 0x69, 0x6e, - 0x69, 0x74, 0x69, 0x61, 0x6c, 0x57, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x53, 0x69, 0x7a, 0x65, 0x42, - 0x79, 0x74, 0x65, 0x73, 0x12, 0x42, 0x0a, 0x1e, 0x69, 0x6e, 0x69, 0x74, 0x69, 0x61, 0x6c, 0x5f, - 0x63, 0x6f, 0x6e, 0x6e, 0x5f, 0x77, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x5f, 0x73, 0x69, 0x7a, 0x65, - 0x5f, 0x62, 0x79, 0x74, 0x65, 0x73, 0x18, 0x09, 0x20, 0x01, 0x28, 0x05, 0x52, 0x1a, 0x69, 0x6e, - 0x69, 0x74, 0x69, 0x61, 0x6c, 0x43, 0x6f, 0x6e, 0x6e, 0x57, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x53, - 0x69, 0x7a, 0x65, 0x42, 0x79, 0x74, 0x65, 0x73, 0x12, 0x58, 0x0a, 0x07, 0x74, 0x72, 0x61, 0x63, - 0x69, 0x6e, 0x67, 0x18, 0x0a, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x3e, 0x2e, 0x62, 0x75, 0x69, 0x6c, + 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x54, 0x72, 0x61, 0x63, + 0x69, 0x6e, 0x67, 0x4d, 0x65, 0x74, 0x68, 0x6f, 0x64, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, + 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, + 0x01, 0x4a, 0x04, 0x08, 0x04, 0x10, 0x05, 0x4a, 0x04, 0x08, 0x07, 0x10, 0x08, 0x22, 0xb6, 0x01, + 0x0a, 0x1c, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x4b, 0x65, 0x65, 0x70, 0x61, 0x6c, 0x69, 0x76, + 0x65, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x2d, + 0x0a, 0x04, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x67, + 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x44, + 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x04, 0x74, 0x69, 0x6d, 0x65, 0x12, 0x33, 0x0a, + 0x07, 0x74, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, + 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, + 0x2e, 0x44, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x07, 0x74, 0x69, 0x6d, 0x65, 0x6f, + 0x75, 0x74, 0x12, 0x32, 0x0a, 0x15, 0x70, 0x65, 0x72, 0x6d, 0x69, 0x74, 0x5f, 0x77, 0x69, 0x74, + 0x68, 0x6f, 0x75, 0x74, 0x5f, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x18, 0x03, 0x20, 0x01, 0x28, + 0x08, 0x52, 0x13, 0x70, 0x65, 0x72, 0x6d, 0x69, 0x74, 0x57, 0x69, 0x74, 0x68, 0x6f, 0x75, 0x74, + 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x22, 0xcb, 0x01, 0x0a, 0x18, 0x43, 0x6c, 0x69, 0x65, 0x6e, + 0x74, 0x4f, 0x41, 0x75, 0x74, 0x68, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, + 0x69, 0x6f, 0x6e, 0x12, 0x56, 0x0a, 0x1a, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x5f, 0x64, 0x65, + 0x66, 0x61, 0x75, 0x6c, 0x74, 0x5f, 0x63, 0x72, 0x65, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x61, 0x6c, + 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x16, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x48, + 0x00, 0x52, 0x18, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x44, 0x65, 0x66, 0x61, 0x75, 0x6c, 0x74, + 0x43, 0x72, 0x65, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x61, 0x6c, 0x73, 0x12, 0x30, 0x0a, 0x13, 0x73, + 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x5f, 0x61, 0x63, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x5f, 0x6b, + 0x65, 0x79, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x48, 0x00, 0x52, 0x11, 0x73, 0x65, 0x72, 0x76, + 0x69, 0x63, 0x65, 0x41, 0x63, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x4b, 0x65, 0x79, 0x12, 0x16, 0x0a, + 0x06, 0x73, 0x63, 0x6f, 0x70, 0x65, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x09, 0x52, 0x06, 0x73, + 0x63, 0x6f, 0x70, 0x65, 0x73, 0x42, 0x0d, 0x0a, 0x0b, 0x63, 0x72, 0x65, 0x64, 0x65, 0x6e, 0x74, + 0x69, 0x61, 0x6c, 0x73, 0x22, 0xf7, 0x07, 0x0a, 0x13, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x43, + 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x29, 0x0a, 0x10, + 0x6c, 0x69, 0x73, 0x74, 0x65, 0x6e, 0x5f, 0x61, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x65, 0x73, + 0x18, 0x01, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0f, 0x6c, 0x69, 0x73, 0x74, 0x65, 0x6e, 0x41, 0x64, + 0x64, 0x72, 0x65, 0x73, 0x73, 0x65, 0x73, 0x12, 0x21, 0x0a, 0x0c, 0x6c, 0x69, 0x73, 0x74, 0x65, + 0x6e, 0x5f, 0x70, 0x61, 0x74, 0x68, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0b, 0x6c, + 0x69, 0x73, 0x74, 0x65, 0x6e, 0x50, 0x61, 0x74, 0x68, 0x73, 0x12, 0x42, 0x0a, 0x03, 0x74, 0x6c, + 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x30, 0x2e, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x62, + 0x61, 0x72, 0x6e, 0x2e, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, + 0x6e, 0x2e, 0x74, 0x6c, 0x73, 0x2e, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x43, 0x6f, 0x6e, 0x66, + 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x03, 0x74, 0x6c, 0x73, 0x12, 0x67, + 0x0a, 0x15, 0x61, 0x75, 0x74, 0x68, 0x65, 0x6e, 0x74, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, + 0x5f, 0x70, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x32, 0x2e, + 0x62, 0x75, 0x69, 0x6c, 0x64, 0x62, 0x61, 0x72, 0x6e, 0x2e, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, + 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x41, 0x75, 0x74, + 0x68, 0x65, 0x6e, 0x74, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x6f, 0x6c, 0x69, 0x63, + 0x79, 0x52, 0x14, 0x61, 0x75, 0x74, 0x68, 0x65, 0x6e, 0x74, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, + 0x6e, 0x50, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x12, 0x4c, 0x0a, 0x23, 0x6d, 0x61, 0x78, 0x69, 0x6d, + 0x75, 0x6d, 0x5f, 0x72, 0x65, 0x63, 0x65, 0x69, 0x76, 0x65, 0x64, 0x5f, 0x6d, 0x65, 0x73, 0x73, + 0x61, 0x67, 0x65, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x5f, 0x62, 0x79, 0x74, 0x65, 0x73, 0x18, 0x05, + 0x20, 0x01, 0x28, 0x03, 0x52, 0x1f, 0x6d, 0x61, 0x78, 0x69, 0x6d, 0x75, 0x6d, 0x52, 0x65, 0x63, + 0x65, 0x69, 0x76, 0x65, 0x64, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x53, 0x69, 0x7a, 0x65, + 0x42, 0x79, 0x74, 0x65, 0x73, 0x12, 0x80, 0x01, 0x0a, 0x1c, 0x6b, 0x65, 0x65, 0x70, 0x61, 0x6c, + 0x69, 0x76, 0x65, 0x5f, 0x65, 0x6e, 0x66, 0x6f, 0x72, 0x63, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x5f, + 0x70, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x3e, 0x2e, 0x62, + 0x75, 0x69, 0x6c, 0x64, 0x62, 0x61, 0x72, 0x6e, 0x2e, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, + 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x53, 0x65, 0x72, 0x76, + 0x65, 0x72, 0x4b, 0x65, 0x65, 0x70, 0x61, 0x6c, 0x69, 0x76, 0x65, 0x45, 0x6e, 0x66, 0x6f, 0x72, + 0x63, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x50, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x52, 0x1a, 0x6b, 0x65, + 0x65, 0x70, 0x61, 0x6c, 0x69, 0x76, 0x65, 0x45, 0x6e, 0x66, 0x6f, 0x72, 0x63, 0x65, 0x6d, 0x65, + 0x6e, 0x74, 0x50, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x12, 0x30, 0x0a, 0x14, 0x68, 0x65, 0x61, 0x6c, + 0x74, 0x68, 0x5f, 0x63, 0x68, 0x65, 0x63, 0x6b, 0x5f, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, + 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, 0x52, 0x12, 0x68, 0x65, 0x61, 0x6c, 0x74, 0x68, 0x43, 0x68, + 0x65, 0x63, 0x6b, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x12, 0x39, 0x0a, 0x19, 0x69, 0x6e, + 0x69, 0x74, 0x69, 0x61, 0x6c, 0x5f, 0x77, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x5f, 0x73, 0x69, 0x7a, + 0x65, 0x5f, 0x62, 0x79, 0x74, 0x65, 0x73, 0x18, 0x08, 0x20, 0x01, 0x28, 0x05, 0x52, 0x16, 0x69, + 0x6e, 0x69, 0x74, 0x69, 0x61, 0x6c, 0x57, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x53, 0x69, 0x7a, 0x65, + 0x42, 0x79, 0x74, 0x65, 0x73, 0x12, 0x42, 0x0a, 0x1e, 0x69, 0x6e, 0x69, 0x74, 0x69, 0x61, 0x6c, + 0x5f, 0x63, 0x6f, 0x6e, 0x6e, 0x5f, 0x77, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x5f, 0x73, 0x69, 0x7a, + 0x65, 0x5f, 0x62, 0x79, 0x74, 0x65, 0x73, 0x18, 0x09, 0x20, 0x01, 0x28, 0x05, 0x52, 0x1a, 0x69, + 0x6e, 0x69, 0x74, 0x69, 0x61, 0x6c, 0x43, 0x6f, 0x6e, 0x6e, 0x57, 0x69, 0x6e, 0x64, 0x6f, 0x77, + 0x53, 0x69, 0x7a, 0x65, 0x42, 0x79, 0x74, 0x65, 0x73, 0x12, 0x58, 0x0a, 0x07, 0x74, 0x72, 0x61, + 0x63, 0x69, 0x6e, 0x67, 0x18, 0x0a, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x3e, 0x2e, 0x62, 0x75, 0x69, + 0x6c, 0x64, 0x62, 0x61, 0x72, 0x6e, 0x2e, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, + 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, + 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x54, 0x72, + 0x61, 0x63, 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x07, 0x74, 0x72, 0x61, 0x63, + 0x69, 0x6e, 0x67, 0x12, 0x6a, 0x0a, 0x14, 0x6b, 0x65, 0x65, 0x70, 0x61, 0x6c, 0x69, 0x76, 0x65, + 0x5f, 0x70, 0x61, 0x72, 0x61, 0x6d, 0x65, 0x74, 0x65, 0x72, 0x73, 0x18, 0x0b, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x37, 0x2e, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x62, 0x61, 0x72, 0x6e, 0x2e, 0x63, 0x6f, + 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x67, 0x72, 0x70, 0x63, + 0x2e, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x4b, 0x65, 0x65, 0x70, 0x61, 0x6c, 0x69, 0x76, 0x65, + 0x50, 0x61, 0x72, 0x61, 0x6d, 0x65, 0x74, 0x65, 0x72, 0x73, 0x52, 0x13, 0x6b, 0x65, 0x65, 0x70, + 0x61, 0x6c, 0x69, 0x76, 0x65, 0x50, 0x61, 0x72, 0x61, 0x6d, 0x65, 0x74, 0x65, 0x72, 0x73, 0x12, + 0x27, 0x0a, 0x0f, 0x73, 0x74, 0x6f, 0x70, 0x5f, 0x67, 0x72, 0x61, 0x63, 0x65, 0x66, 0x75, 0x6c, + 0x6c, 0x79, 0x18, 0x0c, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0e, 0x73, 0x74, 0x6f, 0x70, 0x47, 0x72, + 0x61, 0x63, 0x65, 0x66, 0x75, 0x6c, 0x6c, 0x79, 0x1a, 0x74, 0x0a, 0x0c, 0x54, 0x72, 0x61, 0x63, + 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x4e, 0x0a, 0x05, 0x76, 0x61, + 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x38, 0x2e, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x62, 0x61, 0x72, 0x6e, 0x2e, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, - 0x69, 0x6f, 0x6e, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x43, - 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x54, 0x72, 0x61, - 0x63, 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x07, 0x74, 0x72, 0x61, 0x63, 0x69, - 0x6e, 0x67, 0x12, 0x6a, 0x0a, 0x14, 0x6b, 0x65, 0x65, 0x70, 0x61, 0x6c, 0x69, 0x76, 0x65, 0x5f, - 0x70, 0x61, 0x72, 0x61, 0x6d, 0x65, 0x74, 0x65, 0x72, 0x73, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x0b, - 0x32, 0x37, 0x2e, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x62, 0x61, 0x72, 0x6e, 0x2e, 0x63, 0x6f, 0x6e, - 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, - 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x4b, 0x65, 0x65, 0x70, 0x61, 0x6c, 0x69, 0x76, 0x65, 0x50, - 0x61, 0x72, 0x61, 0x6d, 0x65, 0x74, 0x65, 0x72, 0x73, 0x52, 0x13, 0x6b, 0x65, 0x65, 0x70, 0x61, - 0x6c, 0x69, 0x76, 0x65, 0x50, 0x61, 0x72, 0x61, 0x6d, 0x65, 0x74, 0x65, 0x72, 0x73, 0x12, 0x27, - 0x0a, 0x0f, 0x73, 0x74, 0x6f, 0x70, 0x5f, 0x67, 0x72, 0x61, 0x63, 0x65, 0x66, 0x75, 0x6c, 0x6c, - 0x79, 0x18, 0x0c, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0e, 0x73, 0x74, 0x6f, 0x70, 0x47, 0x72, 0x61, - 0x63, 0x65, 0x66, 0x75, 0x6c, 0x6c, 0x79, 0x1a, 0x74, 0x0a, 0x0c, 0x54, 0x72, 0x61, 0x63, 0x69, - 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, - 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x4e, 0x0a, 0x05, 0x76, 0x61, 0x6c, - 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x38, 0x2e, 0x62, 0x75, 0x69, 0x6c, 0x64, - 0x62, 0x61, 0x72, 0x6e, 0x2e, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, - 0x6f, 0x6e, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x54, 0x72, 0x61, 0x63, 0x69, 0x6e, 0x67, 0x4d, - 0x65, 0x74, 0x68, 0x6f, 0x64, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, - 0x6f, 0x6e, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x8c, 0x01, - 0x0a, 0x20, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x4b, 0x65, 0x65, 0x70, 0x61, 0x6c, 0x69, 0x76, - 0x65, 0x45, 0x6e, 0x66, 0x6f, 0x72, 0x63, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x50, 0x6f, 0x6c, 0x69, - 0x63, 0x79, 0x12, 0x34, 0x0a, 0x08, 0x6d, 0x69, 0x6e, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x01, - 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, - 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x44, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, - 0x07, 0x6d, 0x69, 0x6e, 0x54, 0x69, 0x6d, 0x65, 0x12, 0x32, 0x0a, 0x15, 0x70, 0x65, 0x72, 0x6d, - 0x69, 0x74, 0x5f, 0x77, 0x69, 0x74, 0x68, 0x6f, 0x75, 0x74, 0x5f, 0x73, 0x74, 0x72, 0x65, 0x61, - 0x6d, 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, 0x52, 0x13, 0x70, 0x65, 0x72, 0x6d, 0x69, 0x74, 0x57, - 0x69, 0x74, 0x68, 0x6f, 0x75, 0x74, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x22, 0xe7, 0x02, 0x0a, - 0x19, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x4b, 0x65, 0x65, 0x70, 0x61, 0x6c, 0x69, 0x76, 0x65, - 0x50, 0x61, 0x72, 0x61, 0x6d, 0x65, 0x74, 0x65, 0x72, 0x73, 0x12, 0x49, 0x0a, 0x13, 0x6d, 0x61, - 0x78, 0x5f, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x6c, - 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, - 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x44, 0x75, 0x72, 0x61, 0x74, 0x69, - 0x6f, 0x6e, 0x52, 0x11, 0x6d, 0x61, 0x78, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, - 0x6e, 0x49, 0x64, 0x6c, 0x65, 0x12, 0x47, 0x0a, 0x12, 0x6d, 0x61, 0x78, 0x5f, 0x63, 0x6f, 0x6e, - 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x61, 0x67, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, - 0x0b, 0x32, 0x19, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, - 0x62, 0x75, 0x66, 0x2e, 0x44, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x10, 0x6d, 0x61, - 0x78, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x41, 0x67, 0x65, 0x12, 0x52, - 0x0a, 0x18, 0x6d, 0x61, 0x78, 0x5f, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, - 0x5f, 0x61, 0x67, 0x65, 0x5f, 0x67, 0x72, 0x61, 0x63, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, - 0x32, 0x19, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, - 0x75, 0x66, 0x2e, 0x44, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x15, 0x6d, 0x61, 0x78, - 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x41, 0x67, 0x65, 0x47, 0x72, 0x61, - 0x63, 0x65, 0x12, 0x2d, 0x0a, 0x04, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, - 0x32, 0x19, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, - 0x75, 0x66, 0x2e, 0x44, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x04, 0x74, 0x69, 0x6d, - 0x65, 0x12, 0x33, 0x0a, 0x07, 0x74, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x18, 0x05, 0x20, 0x01, + 0x69, 0x6f, 0x6e, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x54, 0x72, 0x61, 0x63, 0x69, 0x6e, 0x67, + 0x4d, 0x65, 0x74, 0x68, 0x6f, 0x64, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, + 0x69, 0x6f, 0x6e, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x8c, + 0x01, 0x0a, 0x20, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x4b, 0x65, 0x65, 0x70, 0x61, 0x6c, 0x69, + 0x76, 0x65, 0x45, 0x6e, 0x66, 0x6f, 0x72, 0x63, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x50, 0x6f, 0x6c, + 0x69, 0x63, 0x79, 0x12, 0x34, 0x0a, 0x08, 0x6d, 0x69, 0x6e, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x44, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, + 0x52, 0x07, 0x6d, 0x69, 0x6e, 0x54, 0x69, 0x6d, 0x65, 0x12, 0x32, 0x0a, 0x15, 0x70, 0x65, 0x72, + 0x6d, 0x69, 0x74, 0x5f, 0x77, 0x69, 0x74, 0x68, 0x6f, 0x75, 0x74, 0x5f, 0x73, 0x74, 0x72, 0x65, + 0x61, 0x6d, 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, 0x52, 0x13, 0x70, 0x65, 0x72, 0x6d, 0x69, 0x74, + 0x57, 0x69, 0x74, 0x68, 0x6f, 0x75, 0x74, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x22, 0xe7, 0x02, + 0x0a, 0x19, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x4b, 0x65, 0x65, 0x70, 0x61, 0x6c, 0x69, 0x76, + 0x65, 0x50, 0x61, 0x72, 0x61, 0x6d, 0x65, 0x74, 0x65, 0x72, 0x73, 0x12, 0x49, 0x0a, 0x13, 0x6d, + 0x61, 0x78, 0x5f, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, + 0x6c, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, + 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x44, 0x75, 0x72, 0x61, 0x74, + 0x69, 0x6f, 0x6e, 0x52, 0x11, 0x6d, 0x61, 0x78, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, + 0x6f, 0x6e, 0x49, 0x64, 0x6c, 0x65, 0x12, 0x47, 0x0a, 0x12, 0x6d, 0x61, 0x78, 0x5f, 0x63, 0x6f, + 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x61, 0x67, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, - 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x44, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x07, 0x74, - 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x22, 0xb8, 0x04, 0x0a, 0x14, 0x41, 0x75, 0x74, 0x68, 0x65, - 0x6e, 0x74, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x12, - 0x3e, 0x0a, 0x05, 0x61, 0x6c, 0x6c, 0x6f, 0x77, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x26, - 0x2e, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x62, 0x61, 0x72, 0x6e, 0x2e, 0x61, 0x75, 0x74, 0x68, 0x2e, - 0x41, 0x75, 0x74, 0x68, 0x65, 0x6e, 0x74, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x4d, 0x65, - 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x48, 0x00, 0x52, 0x05, 0x61, 0x6c, 0x6c, 0x6f, 0x77, 0x12, - 0x49, 0x0a, 0x03, 0x61, 0x6e, 0x79, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x35, 0x2e, 0x62, + 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x44, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x10, 0x6d, + 0x61, 0x78, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x41, 0x67, 0x65, 0x12, + 0x52, 0x0a, 0x18, 0x6d, 0x61, 0x78, 0x5f, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, + 0x6e, 0x5f, 0x61, 0x67, 0x65, 0x5f, 0x67, 0x72, 0x61, 0x63, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x19, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x62, 0x75, 0x66, 0x2e, 0x44, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x15, 0x6d, 0x61, + 0x78, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x41, 0x67, 0x65, 0x47, 0x72, + 0x61, 0x63, 0x65, 0x12, 0x2d, 0x0a, 0x04, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x19, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x62, 0x75, 0x66, 0x2e, 0x44, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x04, 0x74, 0x69, + 0x6d, 0x65, 0x12, 0x33, 0x0a, 0x07, 0x74, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x18, 0x05, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x44, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x07, + 0x74, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x22, 0x8c, 0x05, 0x0a, 0x14, 0x41, 0x75, 0x74, 0x68, + 0x65, 0x6e, 0x74, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x6f, 0x6c, 0x69, 0x63, 0x79, + 0x12, 0x3e, 0x0a, 0x05, 0x61, 0x6c, 0x6c, 0x6f, 0x77, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x26, 0x2e, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x62, 0x61, 0x72, 0x6e, 0x2e, 0x61, 0x75, 0x74, 0x68, + 0x2e, 0x41, 0x75, 0x74, 0x68, 0x65, 0x6e, 0x74, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x4d, + 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x48, 0x00, 0x52, 0x05, 0x61, 0x6c, 0x6c, 0x6f, 0x77, + 0x12, 0x49, 0x0a, 0x03, 0x61, 0x6e, 0x79, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x35, 0x2e, + 0x62, 0x75, 0x69, 0x6c, 0x64, 0x62, 0x61, 0x72, 0x6e, 0x2e, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, + 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x41, 0x6e, 0x79, + 0x41, 0x75, 0x74, 0x68, 0x65, 0x6e, 0x74, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x6f, + 0x6c, 0x69, 0x63, 0x79, 0x48, 0x00, 0x52, 0x03, 0x61, 0x6e, 0x79, 0x12, 0x49, 0x0a, 0x03, 0x61, + 0x6c, 0x6c, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x35, 0x2e, 0x62, 0x75, 0x69, 0x6c, 0x64, + 0x62, 0x61, 0x72, 0x6e, 0x2e, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, + 0x6f, 0x6e, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x41, 0x6c, 0x6c, 0x41, 0x75, 0x74, 0x68, 0x65, + 0x6e, 0x74, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x48, + 0x00, 0x52, 0x03, 0x61, 0x6c, 0x6c, 0x12, 0x14, 0x0a, 0x04, 0x64, 0x65, 0x6e, 0x79, 0x18, 0x03, + 0x20, 0x01, 0x28, 0x09, 0x48, 0x00, 0x52, 0x04, 0x64, 0x65, 0x6e, 0x79, 0x12, 0x7e, 0x0a, 0x16, + 0x74, 0x6c, 0x73, 0x5f, 0x63, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x5f, 0x63, 0x65, 0x72, 0x74, 0x69, + 0x66, 0x69, 0x63, 0x61, 0x74, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x46, 0x2e, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x62, 0x61, 0x72, 0x6e, 0x2e, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, - 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x41, 0x6e, 0x79, 0x41, - 0x75, 0x74, 0x68, 0x65, 0x6e, 0x74, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x6f, 0x6c, - 0x69, 0x63, 0x79, 0x48, 0x00, 0x52, 0x03, 0x61, 0x6e, 0x79, 0x12, 0x49, 0x0a, 0x03, 0x61, 0x6c, - 0x6c, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x35, 0x2e, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x62, - 0x61, 0x72, 0x6e, 0x2e, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, - 0x6e, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x41, 0x6c, 0x6c, 0x41, 0x75, 0x74, 0x68, 0x65, 0x6e, - 0x74, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x48, 0x00, - 0x52, 0x03, 0x61, 0x6c, 0x6c, 0x12, 0x14, 0x0a, 0x04, 0x64, 0x65, 0x6e, 0x79, 0x18, 0x03, 0x20, - 0x01, 0x28, 0x09, 0x48, 0x00, 0x52, 0x04, 0x64, 0x65, 0x6e, 0x79, 0x12, 0x7e, 0x0a, 0x16, 0x74, - 0x6c, 0x73, 0x5f, 0x63, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x5f, 0x63, 0x65, 0x72, 0x74, 0x69, 0x66, - 0x69, 0x63, 0x61, 0x74, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x46, 0x2e, 0x62, 0x75, - 0x69, 0x6c, 0x64, 0x62, 0x61, 0x72, 0x6e, 0x2e, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, - 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x54, 0x4c, 0x53, 0x43, 0x6c, - 0x69, 0x65, 0x6e, 0x74, 0x43, 0x65, 0x72, 0x74, 0x69, 0x66, 0x69, 0x63, 0x61, 0x74, 0x65, 0x41, - 0x75, 0x74, 0x68, 0x65, 0x6e, 0x74, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x6f, 0x6c, - 0x69, 0x63, 0x79, 0x48, 0x00, 0x52, 0x14, 0x74, 0x6c, 0x73, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, - 0x43, 0x65, 0x72, 0x74, 0x69, 0x66, 0x69, 0x63, 0x61, 0x74, 0x65, 0x12, 0x57, 0x0a, 0x03, 0x6a, - 0x77, 0x74, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x43, 0x2e, 0x62, 0x75, 0x69, 0x6c, 0x64, + 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x54, 0x4c, 0x53, 0x43, + 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x43, 0x65, 0x72, 0x74, 0x69, 0x66, 0x69, 0x63, 0x61, 0x74, 0x65, + 0x41, 0x75, 0x74, 0x68, 0x65, 0x6e, 0x74, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x6f, + 0x6c, 0x69, 0x63, 0x79, 0x48, 0x00, 0x52, 0x14, 0x74, 0x6c, 0x73, 0x43, 0x6c, 0x69, 0x65, 0x6e, + 0x74, 0x43, 0x65, 0x72, 0x74, 0x69, 0x66, 0x69, 0x63, 0x61, 0x74, 0x65, 0x12, 0x57, 0x0a, 0x03, + 0x6a, 0x77, 0x74, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x43, 0x2e, 0x62, 0x75, 0x69, 0x6c, + 0x64, 0x62, 0x61, 0x72, 0x6e, 0x2e, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, + 0x69, 0x6f, 0x6e, 0x2e, 0x6a, 0x77, 0x74, 0x2e, 0x41, 0x75, 0x74, 0x68, 0x6f, 0x72, 0x69, 0x7a, + 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x50, 0x61, 0x72, 0x73, 0x65, + 0x72, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x48, 0x00, + 0x52, 0x03, 0x6a, 0x77, 0x74, 0x12, 0x51, 0x0a, 0x24, 0x70, 0x65, 0x65, 0x72, 0x5f, 0x63, 0x72, + 0x65, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x61, 0x6c, 0x73, 0x5f, 0x6a, 0x6d, 0x65, 0x73, 0x70, 0x61, + 0x74, 0x68, 0x5f, 0x65, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x06, 0x20, + 0x01, 0x28, 0x09, 0x48, 0x00, 0x52, 0x21, 0x70, 0x65, 0x65, 0x72, 0x43, 0x72, 0x65, 0x64, 0x65, + 0x6e, 0x74, 0x69, 0x61, 0x6c, 0x73, 0x4a, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x74, 0x68, 0x45, 0x78, + 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x52, 0x0a, 0x06, 0x72, 0x65, 0x6d, 0x6f, + 0x74, 0x65, 0x18, 0x08, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x38, 0x2e, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x62, 0x61, 0x72, 0x6e, 0x2e, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, - 0x6f, 0x6e, 0x2e, 0x6a, 0x77, 0x74, 0x2e, 0x41, 0x75, 0x74, 0x68, 0x6f, 0x72, 0x69, 0x7a, 0x61, - 0x74, 0x69, 0x6f, 0x6e, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x50, 0x61, 0x72, 0x73, 0x65, 0x72, - 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x48, 0x00, 0x52, - 0x03, 0x6a, 0x77, 0x74, 0x12, 0x51, 0x0a, 0x24, 0x70, 0x65, 0x65, 0x72, 0x5f, 0x63, 0x72, 0x65, - 0x64, 0x65, 0x6e, 0x74, 0x69, 0x61, 0x6c, 0x73, 0x5f, 0x6a, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x74, - 0x68, 0x5f, 0x65, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x06, 0x20, 0x01, - 0x28, 0x09, 0x48, 0x00, 0x52, 0x21, 0x70, 0x65, 0x65, 0x72, 0x43, 0x72, 0x65, 0x64, 0x65, 0x6e, - 0x74, 0x69, 0x61, 0x6c, 0x73, 0x4a, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x74, 0x68, 0x45, 0x78, 0x70, - 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x42, 0x08, 0x0a, 0x06, 0x70, 0x6f, 0x6c, 0x69, 0x63, - 0x79, 0x22, 0x69, 0x0a, 0x17, 0x41, 0x6e, 0x79, 0x41, 0x75, 0x74, 0x68, 0x65, 0x6e, 0x74, 0x69, + 0x6f, 0x6e, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x52, 0x65, 0x6d, 0x6f, 0x74, 0x65, 0x41, 0x75, + 0x74, 0x68, 0x65, 0x6e, 0x74, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x6f, 0x6c, 0x69, + 0x63, 0x79, 0x48, 0x00, 0x52, 0x06, 0x72, 0x65, 0x6d, 0x6f, 0x74, 0x65, 0x42, 0x08, 0x0a, 0x06, + 0x70, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x22, 0x69, 0x0a, 0x17, 0x41, 0x6e, 0x79, 0x41, 0x75, 0x74, + 0x68, 0x65, 0x6e, 0x74, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x6f, 0x6c, 0x69, 0x63, + 0x79, 0x12, 0x4e, 0x0a, 0x08, 0x70, 0x6f, 0x6c, 0x69, 0x63, 0x69, 0x65, 0x73, 0x18, 0x01, 0x20, + 0x03, 0x28, 0x0b, 0x32, 0x32, 0x2e, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x62, 0x61, 0x72, 0x6e, 0x2e, + 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x67, 0x72, + 0x70, 0x63, 0x2e, 0x41, 0x75, 0x74, 0x68, 0x65, 0x6e, 0x74, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, + 0x6e, 0x50, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x52, 0x08, 0x70, 0x6f, 0x6c, 0x69, 0x63, 0x69, 0x65, + 0x73, 0x22, 0x69, 0x0a, 0x17, 0x41, 0x6c, 0x6c, 0x41, 0x75, 0x74, 0x68, 0x65, 0x6e, 0x74, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x12, 0x4e, 0x0a, 0x08, 0x70, 0x6f, 0x6c, 0x69, 0x63, 0x69, 0x65, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x32, 0x2e, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x62, 0x61, 0x72, 0x6e, 0x2e, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x41, 0x75, 0x74, 0x68, 0x65, 0x6e, 0x74, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x6f, 0x6c, 0x69, - 0x63, 0x79, 0x52, 0x08, 0x70, 0x6f, 0x6c, 0x69, 0x63, 0x69, 0x65, 0x73, 0x22, 0x69, 0x0a, 0x17, - 0x41, 0x6c, 0x6c, 0x41, 0x75, 0x74, 0x68, 0x65, 0x6e, 0x74, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, - 0x6e, 0x50, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x12, 0x4e, 0x0a, 0x08, 0x70, 0x6f, 0x6c, 0x69, 0x63, - 0x69, 0x65, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x32, 0x2e, 0x62, 0x75, 0x69, 0x6c, - 0x64, 0x62, 0x61, 0x72, 0x6e, 0x2e, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, - 0x69, 0x6f, 0x6e, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x41, 0x75, 0x74, 0x68, 0x65, 0x6e, 0x74, - 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x52, 0x08, 0x70, - 0x6f, 0x6c, 0x69, 0x63, 0x69, 0x65, 0x73, 0x22, 0x93, 0x02, 0x0a, 0x28, 0x54, 0x4c, 0x53, 0x43, - 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x43, 0x65, 0x72, 0x74, 0x69, 0x66, 0x69, 0x63, 0x61, 0x74, 0x65, - 0x41, 0x75, 0x74, 0x68, 0x65, 0x6e, 0x74, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x6f, - 0x6c, 0x69, 0x63, 0x79, 0x12, 0x44, 0x0a, 0x1e, 0x63, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x5f, 0x63, - 0x65, 0x72, 0x74, 0x69, 0x66, 0x69, 0x63, 0x61, 0x74, 0x65, 0x5f, 0x61, 0x75, 0x74, 0x68, 0x6f, - 0x72, 0x69, 0x74, 0x69, 0x65, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x1c, 0x63, 0x6c, - 0x69, 0x65, 0x6e, 0x74, 0x43, 0x65, 0x72, 0x74, 0x69, 0x66, 0x69, 0x63, 0x61, 0x74, 0x65, 0x41, - 0x75, 0x74, 0x68, 0x6f, 0x72, 0x69, 0x74, 0x69, 0x65, 0x73, 0x12, 0x44, 0x0a, 0x1e, 0x76, 0x61, - 0x6c, 0x69, 0x64, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x6a, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x74, - 0x68, 0x5f, 0x65, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, - 0x28, 0x09, 0x52, 0x1c, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x4a, 0x6d, - 0x65, 0x73, 0x70, 0x61, 0x74, 0x68, 0x45, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, - 0x12, 0x55, 0x0a, 0x27, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x5f, 0x65, 0x78, 0x74, - 0x72, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x6a, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x74, 0x68, - 0x5f, 0x65, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x04, 0x20, 0x01, 0x28, - 0x09, 0x52, 0x24, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x45, 0x78, 0x74, 0x72, 0x61, - 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x4a, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x74, 0x68, 0x45, 0x78, 0x70, - 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x4a, 0x04, 0x08, 0x02, 0x10, 0x03, 0x22, 0xc2, 0x01, - 0x0a, 0x1a, 0x54, 0x72, 0x61, 0x63, 0x69, 0x6e, 0x67, 0x4d, 0x65, 0x74, 0x68, 0x6f, 0x64, 0x43, - 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x50, 0x0a, 0x25, - 0x61, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x5f, 0x66, 0x72, 0x6f, 0x6d, 0x5f, - 0x66, 0x69, 0x72, 0x73, 0x74, 0x5f, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x6d, 0x65, - 0x73, 0x73, 0x61, 0x67, 0x65, 0x18, 0x01, 0x20, 0x03, 0x28, 0x09, 0x52, 0x21, 0x61, 0x74, 0x74, + 0x63, 0x79, 0x52, 0x08, 0x70, 0x6f, 0x6c, 0x69, 0x63, 0x69, 0x65, 0x73, 0x22, 0x93, 0x02, 0x0a, + 0x28, 0x54, 0x4c, 0x53, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x43, 0x65, 0x72, 0x74, 0x69, 0x66, + 0x69, 0x63, 0x61, 0x74, 0x65, 0x41, 0x75, 0x74, 0x68, 0x65, 0x6e, 0x74, 0x69, 0x63, 0x61, 0x74, + 0x69, 0x6f, 0x6e, 0x50, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x12, 0x44, 0x0a, 0x1e, 0x63, 0x6c, 0x69, + 0x65, 0x6e, 0x74, 0x5f, 0x63, 0x65, 0x72, 0x74, 0x69, 0x66, 0x69, 0x63, 0x61, 0x74, 0x65, 0x5f, + 0x61, 0x75, 0x74, 0x68, 0x6f, 0x72, 0x69, 0x74, 0x69, 0x65, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x1c, 0x63, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x43, 0x65, 0x72, 0x74, 0x69, 0x66, 0x69, + 0x63, 0x61, 0x74, 0x65, 0x41, 0x75, 0x74, 0x68, 0x6f, 0x72, 0x69, 0x74, 0x69, 0x65, 0x73, 0x12, + 0x44, 0x0a, 0x1e, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x6a, 0x6d, + 0x65, 0x73, 0x70, 0x61, 0x74, 0x68, 0x5f, 0x65, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, + 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x1c, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, + 0x69, 0x6f, 0x6e, 0x4a, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x74, 0x68, 0x45, 0x78, 0x70, 0x72, 0x65, + 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x55, 0x0a, 0x27, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, + 0x61, 0x5f, 0x65, 0x78, 0x74, 0x72, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x6a, 0x6d, 0x65, + 0x73, 0x70, 0x61, 0x74, 0x68, 0x5f, 0x65, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, + 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x24, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, + 0x45, 0x78, 0x74, 0x72, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x4a, 0x6d, 0x65, 0x73, 0x70, 0x61, + 0x74, 0x68, 0x45, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x4a, 0x04, 0x08, 0x02, + 0x10, 0x03, 0x22, 0xd5, 0x02, 0x0a, 0x1a, 0x52, 0x65, 0x6d, 0x6f, 0x74, 0x65, 0x41, 0x75, 0x74, + 0x68, 0x65, 0x6e, 0x74, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x6f, 0x6c, 0x69, 0x63, + 0x79, 0x12, 0x18, 0x0a, 0x07, 0x68, 0x65, 0x61, 0x64, 0x65, 0x72, 0x73, 0x18, 0x01, 0x20, 0x03, + 0x28, 0x09, 0x52, 0x07, 0x68, 0x65, 0x61, 0x64, 0x65, 0x72, 0x73, 0x12, 0x4d, 0x0a, 0x08, 0x65, + 0x6e, 0x64, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x31, 0x2e, + 0x62, 0x75, 0x69, 0x6c, 0x64, 0x62, 0x61, 0x72, 0x6e, 0x2e, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, + 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x43, 0x6c, 0x69, + 0x65, 0x6e, 0x74, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, + 0x52, 0x08, 0x65, 0x6e, 0x64, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x12, 0x2c, 0x0a, 0x05, 0x73, 0x63, + 0x6f, 0x70, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x16, 0x2e, 0x67, 0x6f, 0x6f, 0x67, + 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x56, 0x61, 0x6c, 0x75, + 0x65, 0x52, 0x05, 0x73, 0x63, 0x6f, 0x70, 0x65, 0x12, 0x2c, 0x0a, 0x12, 0x6d, 0x61, 0x78, 0x69, + 0x6d, 0x75, 0x6d, 0x5f, 0x63, 0x61, 0x63, 0x68, 0x65, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x18, 0x04, + 0x20, 0x01, 0x28, 0x05, 0x52, 0x10, 0x6d, 0x61, 0x78, 0x69, 0x6d, 0x75, 0x6d, 0x43, 0x61, 0x63, + 0x68, 0x65, 0x53, 0x69, 0x7a, 0x65, 0x12, 0x72, 0x0a, 0x18, 0x63, 0x61, 0x63, 0x68, 0x65, 0x5f, + 0x72, 0x65, 0x70, 0x6c, 0x61, 0x63, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x5f, 0x70, 0x6f, 0x6c, 0x69, + 0x63, 0x79, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x38, 0x2e, 0x62, 0x75, 0x69, 0x6c, 0x64, + 0x62, 0x61, 0x72, 0x6e, 0x2e, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, + 0x6f, 0x6e, 0x2e, 0x65, 0x76, 0x69, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x43, 0x61, 0x63, 0x68, + 0x65, 0x52, 0x65, 0x70, 0x6c, 0x61, 0x63, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x50, 0x6f, 0x6c, 0x69, + 0x63, 0x79, 0x52, 0x16, 0x63, 0x61, 0x63, 0x68, 0x65, 0x52, 0x65, 0x70, 0x6c, 0x61, 0x63, 0x65, + 0x6d, 0x65, 0x6e, 0x74, 0x50, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x22, 0xc2, 0x01, 0x0a, 0x1a, 0x54, + 0x72, 0x61, 0x63, 0x69, 0x6e, 0x67, 0x4d, 0x65, 0x74, 0x68, 0x6f, 0x64, 0x43, 0x6f, 0x6e, 0x66, + 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x50, 0x0a, 0x25, 0x61, 0x74, 0x74, + 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x5f, 0x66, 0x72, 0x6f, 0x6d, 0x5f, 0x66, 0x69, 0x72, + 0x73, 0x74, 0x5f, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x6d, 0x65, 0x73, 0x73, 0x61, + 0x67, 0x65, 0x18, 0x01, 0x20, 0x03, 0x28, 0x09, 0x52, 0x21, 0x61, 0x74, 0x74, 0x72, 0x69, 0x62, + 0x75, 0x74, 0x65, 0x73, 0x46, 0x72, 0x6f, 0x6d, 0x46, 0x69, 0x72, 0x73, 0x74, 0x52, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x52, 0x0a, 0x26, 0x61, + 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x5f, 0x66, 0x72, 0x6f, 0x6d, 0x5f, 0x66, + 0x69, 0x72, 0x73, 0x74, 0x5f, 0x72, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x5f, 0x6d, 0x65, + 0x73, 0x73, 0x61, 0x67, 0x65, 0x18, 0x02, 0x20, 0x03, 0x28, 0x09, 0x52, 0x22, 0x61, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x46, 0x72, 0x6f, 0x6d, 0x46, 0x69, 0x72, 0x73, 0x74, - 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x52, - 0x0a, 0x26, 0x61, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x5f, 0x66, 0x72, 0x6f, - 0x6d, 0x5f, 0x66, 0x69, 0x72, 0x73, 0x74, 0x5f, 0x72, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, - 0x5f, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x18, 0x02, 0x20, 0x03, 0x28, 0x09, 0x52, 0x22, - 0x61, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x46, 0x72, 0x6f, 0x6d, 0x46, 0x69, - 0x72, 0x73, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x4d, 0x65, 0x73, 0x73, 0x61, - 0x67, 0x65, 0x42, 0x3e, 0x5a, 0x3c, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, - 0x2f, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x62, 0x61, 0x72, 0x6e, 0x2f, 0x62, 0x62, 0x2d, 0x73, 0x74, - 0x6f, 0x72, 0x61, 0x67, 0x65, 0x2f, 0x70, 0x6b, 0x67, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, - 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2f, 0x67, 0x72, - 0x70, 0x63, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, + 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x42, + 0x3e, 0x5a, 0x3c, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x62, 0x75, + 0x69, 0x6c, 0x64, 0x62, 0x61, 0x72, 0x6e, 0x2f, 0x62, 0x62, 0x2d, 0x73, 0x74, 0x6f, 0x72, 0x61, + 0x67, 0x65, 0x2f, 0x70, 0x6b, 0x67, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x63, 0x6f, 0x6e, + 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2f, 0x67, 0x72, 0x70, 0x63, 0x62, + 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, }) var ( @@ -1273,7 +1396,7 @@ func file_pkg_proto_configuration_grpc_grpc_proto_rawDescGZIP() []byte { return file_pkg_proto_configuration_grpc_grpc_proto_rawDescData } -var file_pkg_proto_configuration_grpc_grpc_proto_msgTypes = make([]protoimpl.MessageInfo, 14) +var file_pkg_proto_configuration_grpc_grpc_proto_msgTypes = make([]protoimpl.MessageInfo, 15) var file_pkg_proto_configuration_grpc_grpc_proto_goTypes = []any{ (*ClientConfiguration)(nil), // 0: buildbarn.configuration.grpc.ClientConfiguration (*ClientKeepaliveConfiguration)(nil), // 1: buildbarn.configuration.grpc.ClientKeepaliveConfiguration @@ -1285,53 +1408,60 @@ var file_pkg_proto_configuration_grpc_grpc_proto_goTypes = []any{ (*AnyAuthenticationPolicy)(nil), // 7: buildbarn.configuration.grpc.AnyAuthenticationPolicy (*AllAuthenticationPolicy)(nil), // 8: buildbarn.configuration.grpc.AllAuthenticationPolicy (*TLSClientCertificateAuthenticationPolicy)(nil), // 9: buildbarn.configuration.grpc.TLSClientCertificateAuthenticationPolicy - (*TracingMethodConfiguration)(nil), // 10: buildbarn.configuration.grpc.TracingMethodConfiguration - (*ClientConfiguration_HeaderValues)(nil), // 11: buildbarn.configuration.grpc.ClientConfiguration.HeaderValues - nil, // 12: buildbarn.configuration.grpc.ClientConfiguration.TracingEntry - nil, // 13: buildbarn.configuration.grpc.ServerConfiguration.TracingEntry - (*tls.ClientConfiguration)(nil), // 14: buildbarn.configuration.tls.ClientConfiguration - (*structpb.Struct)(nil), // 15: google.protobuf.Struct - (*durationpb.Duration)(nil), // 16: google.protobuf.Duration - (*emptypb.Empty)(nil), // 17: google.protobuf.Empty - (*tls.ServerConfiguration)(nil), // 18: buildbarn.configuration.tls.ServerConfiguration - (*auth.AuthenticationMetadata)(nil), // 19: buildbarn.auth.AuthenticationMetadata - (*jwt.AuthorizationHeaderParserConfiguration)(nil), // 20: buildbarn.configuration.jwt.AuthorizationHeaderParserConfiguration + (*RemoteAuthenticationPolicy)(nil), // 10: buildbarn.configuration.grpc.RemoteAuthenticationPolicy + (*TracingMethodConfiguration)(nil), // 11: buildbarn.configuration.grpc.TracingMethodConfiguration + (*ClientConfiguration_HeaderValues)(nil), // 12: buildbarn.configuration.grpc.ClientConfiguration.HeaderValues + nil, // 13: buildbarn.configuration.grpc.ClientConfiguration.TracingEntry + nil, // 14: buildbarn.configuration.grpc.ServerConfiguration.TracingEntry + (*tls.ClientConfiguration)(nil), // 15: buildbarn.configuration.tls.ClientConfiguration + (*structpb.Struct)(nil), // 16: google.protobuf.Struct + (*durationpb.Duration)(nil), // 17: google.protobuf.Duration + (*emptypb.Empty)(nil), // 18: google.protobuf.Empty + (*tls.ServerConfiguration)(nil), // 19: buildbarn.configuration.tls.ServerConfiguration + (*auth.AuthenticationMetadata)(nil), // 20: buildbarn.auth.AuthenticationMetadata + (*jwt.AuthorizationHeaderParserConfiguration)(nil), // 21: buildbarn.configuration.jwt.AuthorizationHeaderParserConfiguration + (*structpb.Value)(nil), // 22: google.protobuf.Value + (eviction.CacheReplacementPolicy)(0), // 23: buildbarn.configuration.eviction.CacheReplacementPolicy } var file_pkg_proto_configuration_grpc_grpc_proto_depIdxs = []int32{ - 14, // 0: buildbarn.configuration.grpc.ClientConfiguration.tls:type_name -> buildbarn.configuration.tls.ClientConfiguration + 15, // 0: buildbarn.configuration.grpc.ClientConfiguration.tls:type_name -> buildbarn.configuration.tls.ClientConfiguration 1, // 1: buildbarn.configuration.grpc.ClientConfiguration.keepalive:type_name -> buildbarn.configuration.grpc.ClientKeepaliveConfiguration - 11, // 2: buildbarn.configuration.grpc.ClientConfiguration.add_metadata:type_name -> buildbarn.configuration.grpc.ClientConfiguration.HeaderValues + 12, // 2: buildbarn.configuration.grpc.ClientConfiguration.add_metadata:type_name -> buildbarn.configuration.grpc.ClientConfiguration.HeaderValues 2, // 3: buildbarn.configuration.grpc.ClientConfiguration.oauth:type_name -> buildbarn.configuration.grpc.ClientOAuthConfiguration - 12, // 4: buildbarn.configuration.grpc.ClientConfiguration.tracing:type_name -> buildbarn.configuration.grpc.ClientConfiguration.TracingEntry - 15, // 5: buildbarn.configuration.grpc.ClientConfiguration.default_service_config:type_name -> google.protobuf.Struct - 16, // 6: buildbarn.configuration.grpc.ClientKeepaliveConfiguration.time:type_name -> google.protobuf.Duration - 16, // 7: buildbarn.configuration.grpc.ClientKeepaliveConfiguration.timeout:type_name -> google.protobuf.Duration - 17, // 8: buildbarn.configuration.grpc.ClientOAuthConfiguration.google_default_credentials:type_name -> google.protobuf.Empty - 18, // 9: buildbarn.configuration.grpc.ServerConfiguration.tls:type_name -> buildbarn.configuration.tls.ServerConfiguration + 13, // 4: buildbarn.configuration.grpc.ClientConfiguration.tracing:type_name -> buildbarn.configuration.grpc.ClientConfiguration.TracingEntry + 16, // 5: buildbarn.configuration.grpc.ClientConfiguration.default_service_config:type_name -> google.protobuf.Struct + 17, // 6: buildbarn.configuration.grpc.ClientKeepaliveConfiguration.time:type_name -> google.protobuf.Duration + 17, // 7: buildbarn.configuration.grpc.ClientKeepaliveConfiguration.timeout:type_name -> google.protobuf.Duration + 18, // 8: buildbarn.configuration.grpc.ClientOAuthConfiguration.google_default_credentials:type_name -> google.protobuf.Empty + 19, // 9: buildbarn.configuration.grpc.ServerConfiguration.tls:type_name -> buildbarn.configuration.tls.ServerConfiguration 6, // 10: buildbarn.configuration.grpc.ServerConfiguration.authentication_policy:type_name -> buildbarn.configuration.grpc.AuthenticationPolicy 4, // 11: buildbarn.configuration.grpc.ServerConfiguration.keepalive_enforcement_policy:type_name -> buildbarn.configuration.grpc.ServerKeepaliveEnforcementPolicy - 13, // 12: buildbarn.configuration.grpc.ServerConfiguration.tracing:type_name -> buildbarn.configuration.grpc.ServerConfiguration.TracingEntry + 14, // 12: buildbarn.configuration.grpc.ServerConfiguration.tracing:type_name -> buildbarn.configuration.grpc.ServerConfiguration.TracingEntry 5, // 13: buildbarn.configuration.grpc.ServerConfiguration.keepalive_parameters:type_name -> buildbarn.configuration.grpc.ServerKeepaliveParameters - 16, // 14: buildbarn.configuration.grpc.ServerKeepaliveEnforcementPolicy.min_time:type_name -> google.protobuf.Duration - 16, // 15: buildbarn.configuration.grpc.ServerKeepaliveParameters.max_connection_idle:type_name -> google.protobuf.Duration - 16, // 16: buildbarn.configuration.grpc.ServerKeepaliveParameters.max_connection_age:type_name -> google.protobuf.Duration - 16, // 17: buildbarn.configuration.grpc.ServerKeepaliveParameters.max_connection_age_grace:type_name -> google.protobuf.Duration - 16, // 18: buildbarn.configuration.grpc.ServerKeepaliveParameters.time:type_name -> google.protobuf.Duration - 16, // 19: buildbarn.configuration.grpc.ServerKeepaliveParameters.timeout:type_name -> google.protobuf.Duration - 19, // 20: buildbarn.configuration.grpc.AuthenticationPolicy.allow:type_name -> buildbarn.auth.AuthenticationMetadata + 17, // 14: buildbarn.configuration.grpc.ServerKeepaliveEnforcementPolicy.min_time:type_name -> google.protobuf.Duration + 17, // 15: buildbarn.configuration.grpc.ServerKeepaliveParameters.max_connection_idle:type_name -> google.protobuf.Duration + 17, // 16: buildbarn.configuration.grpc.ServerKeepaliveParameters.max_connection_age:type_name -> google.protobuf.Duration + 17, // 17: buildbarn.configuration.grpc.ServerKeepaliveParameters.max_connection_age_grace:type_name -> google.protobuf.Duration + 17, // 18: buildbarn.configuration.grpc.ServerKeepaliveParameters.time:type_name -> google.protobuf.Duration + 17, // 19: buildbarn.configuration.grpc.ServerKeepaliveParameters.timeout:type_name -> google.protobuf.Duration + 20, // 20: buildbarn.configuration.grpc.AuthenticationPolicy.allow:type_name -> buildbarn.auth.AuthenticationMetadata 7, // 21: buildbarn.configuration.grpc.AuthenticationPolicy.any:type_name -> buildbarn.configuration.grpc.AnyAuthenticationPolicy 8, // 22: buildbarn.configuration.grpc.AuthenticationPolicy.all:type_name -> buildbarn.configuration.grpc.AllAuthenticationPolicy 9, // 23: buildbarn.configuration.grpc.AuthenticationPolicy.tls_client_certificate:type_name -> buildbarn.configuration.grpc.TLSClientCertificateAuthenticationPolicy - 20, // 24: buildbarn.configuration.grpc.AuthenticationPolicy.jwt:type_name -> buildbarn.configuration.jwt.AuthorizationHeaderParserConfiguration - 6, // 25: buildbarn.configuration.grpc.AnyAuthenticationPolicy.policies:type_name -> buildbarn.configuration.grpc.AuthenticationPolicy - 6, // 26: buildbarn.configuration.grpc.AllAuthenticationPolicy.policies:type_name -> buildbarn.configuration.grpc.AuthenticationPolicy - 10, // 27: buildbarn.configuration.grpc.ClientConfiguration.TracingEntry.value:type_name -> buildbarn.configuration.grpc.TracingMethodConfiguration - 10, // 28: buildbarn.configuration.grpc.ServerConfiguration.TracingEntry.value:type_name -> buildbarn.configuration.grpc.TracingMethodConfiguration - 29, // [29:29] is the sub-list for method output_type - 29, // [29:29] 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 + 21, // 24: buildbarn.configuration.grpc.AuthenticationPolicy.jwt:type_name -> buildbarn.configuration.jwt.AuthorizationHeaderParserConfiguration + 10, // 25: buildbarn.configuration.grpc.AuthenticationPolicy.remote:type_name -> buildbarn.configuration.grpc.RemoteAuthenticationPolicy + 6, // 26: buildbarn.configuration.grpc.AnyAuthenticationPolicy.policies:type_name -> buildbarn.configuration.grpc.AuthenticationPolicy + 6, // 27: buildbarn.configuration.grpc.AllAuthenticationPolicy.policies:type_name -> buildbarn.configuration.grpc.AuthenticationPolicy + 0, // 28: buildbarn.configuration.grpc.RemoteAuthenticationPolicy.endpoint:type_name -> buildbarn.configuration.grpc.ClientConfiguration + 22, // 29: buildbarn.configuration.grpc.RemoteAuthenticationPolicy.scope:type_name -> google.protobuf.Value + 23, // 30: buildbarn.configuration.grpc.RemoteAuthenticationPolicy.cache_replacement_policy:type_name -> buildbarn.configuration.eviction.CacheReplacementPolicy + 11, // 31: buildbarn.configuration.grpc.ClientConfiguration.TracingEntry.value:type_name -> buildbarn.configuration.grpc.TracingMethodConfiguration + 11, // 32: buildbarn.configuration.grpc.ServerConfiguration.TracingEntry.value:type_name -> buildbarn.configuration.grpc.TracingMethodConfiguration + 33, // [33:33] is the sub-list for method output_type + 33, // [33:33] 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_pkg_proto_configuration_grpc_grpc_proto_init() } @@ -1351,6 +1481,7 @@ func file_pkg_proto_configuration_grpc_grpc_proto_init() { (*AuthenticationPolicy_TlsClientCertificate)(nil), (*AuthenticationPolicy_Jwt)(nil), (*AuthenticationPolicy_PeerCredentialsJmespathExpression)(nil), + (*AuthenticationPolicy_Remote)(nil), } type x struct{} out := protoimpl.TypeBuilder{ @@ -1358,7 +1489,7 @@ func file_pkg_proto_configuration_grpc_grpc_proto_init() { GoPackagePath: reflect.TypeOf(x{}).PkgPath(), RawDescriptor: unsafe.Slice(unsafe.StringData(file_pkg_proto_configuration_grpc_grpc_proto_rawDesc), len(file_pkg_proto_configuration_grpc_grpc_proto_rawDesc)), NumEnums: 0, - NumMessages: 14, + NumMessages: 15, NumExtensions: 0, NumServices: 0, }, diff --git a/pkg/proto/configuration/grpc/grpc.proto b/pkg/proto/configuration/grpc/grpc.proto index 8a3564f5..43212702 100644 --- a/pkg/proto/configuration/grpc/grpc.proto +++ b/pkg/proto/configuration/grpc/grpc.proto @@ -6,6 +6,7 @@ import "google/protobuf/duration.proto"; import "google/protobuf/empty.proto"; import "google/protobuf/struct.proto"; import "pkg/proto/auth/auth.proto"; +import "pkg/proto/configuration/eviction/eviction.proto"; import "pkg/proto/configuration/jwt/jwt.proto"; import "pkg/proto/configuration/tls/tls.proto"; @@ -365,6 +366,13 @@ message AuthenticationPolicy { // LOCAL_PEERCRED is used, which is typically capable of returning // up to 16 group memberships. string peer_credentials_jmespath_expression = 6; + + // Forwards the authentication task to a remote server. This can be a + // convenient way to integrate custom authentication processes, for example + // if a proxy has already performed authentication where the resulting HTTP + // request's headers need to be verified and processed in a custom way to + // produce buildbarn.auth.AuthenticationMetadata. + RemoteAuthenticationPolicy remote = 8; } } @@ -440,6 +448,28 @@ message TLSClientCertificateAuthenticationPolicy { string metadata_extraction_jmespath_expression = 4; } +message RemoteAuthenticationPolicy { + // Headers from the incoming gRPC request to forward to the remote + // authenticator. + repeated string headers = 1; + + // The remote buildbarn.auth.AuthenticationServer to grant or deny access the + // gRPC request. + ClientConfiguration endpoint = 2; + + // Additional literal input to the remote authenticator. + google.protobuf.Value scope = 3; + + // Maximum number of authentication responses to cache in memory. This speeds + // up successive requests made with the same metadata. + int32 maximum_cache_size = 4; + + // The cache replacement policy that should be applied to the token + // cache. It is advised that this is set to LEAST_RECENTLY_USED. + buildbarn.configuration.eviction.CacheReplacementPolicy + cache_replacement_policy = 5; +} + message TracingMethodConfiguration { // Names of fields to extract from the request message of an RPC, and // convert into trace span attributes. The resulting attributes will diff --git a/pkg/proto/configuration/http/BUILD.bazel b/pkg/proto/configuration/http/BUILD.bazel index 1f6b5d0f..cd62289f 100644 --- a/pkg/proto/configuration/http/BUILD.bazel +++ b/pkg/proto/configuration/http/BUILD.bazel @@ -8,6 +8,7 @@ proto_library( visibility = ["//visibility:public"], deps = [ "//pkg/proto/auth:auth_proto", + "//pkg/proto/configuration/grpc:grpc_proto", "//pkg/proto/configuration/jwt:jwt_proto", "//pkg/proto/configuration/tls:tls_proto", ], @@ -20,6 +21,7 @@ go_proto_library( visibility = ["//visibility:public"], deps = [ "//pkg/proto/auth", + "//pkg/proto/configuration/grpc", "//pkg/proto/configuration/jwt", "//pkg/proto/configuration/tls", ], diff --git a/pkg/proto/configuration/http/http.pb.go b/pkg/proto/configuration/http/http.pb.go index 98610e9b..04dddd98 100644 --- a/pkg/proto/configuration/http/http.pb.go +++ b/pkg/proto/configuration/http/http.pb.go @@ -8,6 +8,7 @@ package http import ( auth "github.com/buildbarn/bb-storage/pkg/proto/auth" + grpc "github.com/buildbarn/bb-storage/pkg/proto/configuration/grpc" jwt "github.com/buildbarn/bb-storage/pkg/proto/configuration/jwt" tls "github.com/buildbarn/bb-storage/pkg/proto/configuration/tls" protoreflect "google.golang.org/protobuf/reflect/protoreflect" @@ -162,6 +163,7 @@ type AuthenticationPolicy struct { // *AuthenticationPolicy_Jwt // *AuthenticationPolicy_Oidc // *AuthenticationPolicy_AcceptHeader + // *AuthenticationPolicy_Remote Policy isAuthenticationPolicy_Policy `protobuf_oneof:"policy"` unknownFields protoimpl.UnknownFields sizeCache protoimpl.SizeCache @@ -258,6 +260,15 @@ func (x *AuthenticationPolicy) GetAcceptHeader() *AcceptHeaderAuthenticationPoli return nil } +func (x *AuthenticationPolicy) GetRemote() *grpc.RemoteAuthenticationPolicy { + if x != nil { + if x, ok := x.Policy.(*AuthenticationPolicy_Remote); ok { + return x.Remote + } + } + return nil +} + type isAuthenticationPolicy_Policy interface { isAuthenticationPolicy_Policy() } @@ -286,6 +297,10 @@ type AuthenticationPolicy_AcceptHeader struct { AcceptHeader *AcceptHeaderAuthenticationPolicy `protobuf:"bytes,6,opt,name=accept_header,json=acceptHeader,proto3,oneof"` } +type AuthenticationPolicy_Remote struct { + Remote *grpc.RemoteAuthenticationPolicy `protobuf:"bytes,7,opt,name=remote,proto3,oneof"` +} + func (*AuthenticationPolicy_Allow) isAuthenticationPolicy_Policy() {} func (*AuthenticationPolicy_Any) isAuthenticationPolicy_Policy() {} @@ -298,6 +313,8 @@ func (*AuthenticationPolicy_Oidc) isAuthenticationPolicy_Policy() {} func (*AuthenticationPolicy_AcceptHeader) isAuthenticationPolicy_Policy() {} +func (*AuthenticationPolicy_Remote) isAuthenticationPolicy_Policy() {} + type AnyAuthenticationPolicy struct { state protoimpl.MessageState `protogen:"open.v1"` Policies []*AuthenticationPolicy `protobuf:"bytes,1,rep,name=policies,proto3" json:"policies,omitempty"` @@ -571,128 +588,136 @@ var file_pkg_proto_configuration_http_http_proto_rawDesc = string([]byte{ 0x62, 0x61, 0x72, 0x6e, 0x2e, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x68, 0x74, 0x74, 0x70, 0x1a, 0x19, 0x70, 0x6b, 0x67, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x61, 0x75, 0x74, 0x68, 0x2f, 0x61, 0x75, 0x74, 0x68, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x1a, 0x27, 0x70, 0x6b, 0x67, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x63, 0x6f, + 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2f, 0x67, 0x72, 0x70, 0x63, + 0x2f, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x25, 0x70, 0x6b, 0x67, + 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, + 0x74, 0x69, 0x6f, 0x6e, 0x2f, 0x6a, 0x77, 0x74, 0x2f, 0x6a, 0x77, 0x74, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x25, 0x70, 0x6b, 0x67, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x63, 0x6f, - 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2f, 0x6a, 0x77, 0x74, 0x2f, - 0x6a, 0x77, 0x74, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x25, 0x70, 0x6b, 0x67, 0x2f, 0x70, - 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, - 0x6f, 0x6e, 0x2f, 0x74, 0x6c, 0x73, 0x2f, 0x74, 0x6c, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, - 0x22, 0xbc, 0x02, 0x0a, 0x13, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x43, 0x6f, 0x6e, 0x66, 0x69, - 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x42, 0x0a, 0x03, 0x74, 0x6c, 0x73, 0x18, - 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x30, 0x2e, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x62, 0x61, 0x72, - 0x6e, 0x2e, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, - 0x74, 0x6c, 0x73, 0x2e, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, - 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x03, 0x74, 0x6c, 0x73, 0x12, 0x1b, 0x0a, 0x09, - 0x70, 0x72, 0x6f, 0x78, 0x79, 0x5f, 0x75, 0x72, 0x6c, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, - 0x08, 0x70, 0x72, 0x6f, 0x78, 0x79, 0x55, 0x72, 0x6c, 0x12, 0x5f, 0x0a, 0x0b, 0x61, 0x64, 0x64, - 0x5f, 0x68, 0x65, 0x61, 0x64, 0x65, 0x72, 0x73, 0x18, 0x05, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x3e, - 0x2e, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x62, 0x61, 0x72, 0x6e, 0x2e, 0x63, 0x6f, 0x6e, 0x66, 0x69, - 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x68, 0x74, 0x74, 0x70, 0x2e, 0x43, 0x6c, + 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2f, 0x74, 0x6c, 0x73, 0x2f, + 0x74, 0x6c, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x22, 0xbc, 0x02, 0x0a, 0x13, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, - 0x6e, 0x2e, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x73, 0x52, 0x0a, - 0x61, 0x64, 0x64, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x73, 0x12, 0x23, 0x0a, 0x0d, 0x64, 0x69, - 0x73, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x68, 0x74, 0x74, 0x70, 0x32, 0x18, 0x06, 0x20, 0x01, 0x28, - 0x08, 0x52, 0x0c, 0x64, 0x69, 0x73, 0x61, 0x62, 0x6c, 0x65, 0x48, 0x74, 0x74, 0x70, 0x32, 0x1a, - 0x3e, 0x0a, 0x0c, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x73, 0x12, - 0x16, 0x0a, 0x06, 0x68, 0x65, 0x61, 0x64, 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, - 0x06, 0x68, 0x65, 0x61, 0x64, 0x65, 0x72, 0x12, 0x16, 0x0a, 0x06, 0x76, 0x61, 0x6c, 0x75, 0x65, - 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x09, 0x52, 0x06, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x73, 0x22, - 0xed, 0x01, 0x0a, 0x13, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, - 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x29, 0x0a, 0x10, 0x6c, 0x69, 0x73, 0x74, 0x65, - 0x6e, 0x5f, 0x61, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x65, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, - 0x09, 0x52, 0x0f, 0x6c, 0x69, 0x73, 0x74, 0x65, 0x6e, 0x41, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, - 0x65, 0x73, 0x12, 0x67, 0x0a, 0x15, 0x61, 0x75, 0x74, 0x68, 0x65, 0x6e, 0x74, 0x69, 0x63, 0x61, - 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x70, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x18, 0x02, 0x20, 0x01, 0x28, - 0x0b, 0x32, 0x32, 0x2e, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x62, 0x61, 0x72, 0x6e, 0x2e, 0x63, 0x6f, - 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x68, 0x74, 0x74, 0x70, - 0x2e, 0x41, 0x75, 0x74, 0x68, 0x65, 0x6e, 0x74, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x50, - 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x52, 0x14, 0x61, 0x75, 0x74, 0x68, 0x65, 0x6e, 0x74, 0x69, 0x63, - 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x12, 0x42, 0x0a, 0x03, 0x74, - 0x6c, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x30, 0x2e, 0x62, 0x75, 0x69, 0x6c, 0x64, - 0x62, 0x61, 0x72, 0x6e, 0x2e, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, - 0x6f, 0x6e, 0x2e, 0x74, 0x6c, 0x73, 0x2e, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x43, 0x6f, 0x6e, - 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x03, 0x74, 0x6c, 0x73, 0x22, - 0xcf, 0x03, 0x0a, 0x14, 0x41, 0x75, 0x74, 0x68, 0x65, 0x6e, 0x74, 0x69, 0x63, 0x61, 0x74, 0x69, - 0x6f, 0x6e, 0x50, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x12, 0x3e, 0x0a, 0x05, 0x61, 0x6c, 0x6c, 0x6f, - 0x77, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x26, 0x2e, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x62, - 0x61, 0x72, 0x6e, 0x2e, 0x61, 0x75, 0x74, 0x68, 0x2e, 0x41, 0x75, 0x74, 0x68, 0x65, 0x6e, 0x74, - 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x48, - 0x00, 0x52, 0x05, 0x61, 0x6c, 0x6c, 0x6f, 0x77, 0x12, 0x49, 0x0a, 0x03, 0x61, 0x6e, 0x79, 0x18, - 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x35, 0x2e, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x62, 0x61, 0x72, - 0x6e, 0x2e, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, - 0x68, 0x74, 0x74, 0x70, 0x2e, 0x41, 0x6e, 0x79, 0x41, 0x75, 0x74, 0x68, 0x65, 0x6e, 0x74, 0x69, - 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x48, 0x00, 0x52, 0x03, - 0x61, 0x6e, 0x79, 0x12, 0x14, 0x0a, 0x04, 0x64, 0x65, 0x6e, 0x79, 0x18, 0x03, 0x20, 0x01, 0x28, - 0x09, 0x48, 0x00, 0x52, 0x04, 0x64, 0x65, 0x6e, 0x79, 0x12, 0x57, 0x0a, 0x03, 0x6a, 0x77, 0x74, - 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x43, 0x2e, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x62, 0x61, - 0x72, 0x6e, 0x2e, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, - 0x2e, 0x6a, 0x77, 0x74, 0x2e, 0x41, 0x75, 0x74, 0x68, 0x6f, 0x72, 0x69, 0x7a, 0x61, 0x74, 0x69, - 0x6f, 0x6e, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x50, 0x61, 0x72, 0x73, 0x65, 0x72, 0x43, 0x6f, - 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x48, 0x00, 0x52, 0x03, 0x6a, - 0x77, 0x74, 0x12, 0x4c, 0x0a, 0x04, 0x6f, 0x69, 0x64, 0x63, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, - 0x32, 0x36, 0x2e, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x62, 0x61, 0x72, 0x6e, 0x2e, 0x63, 0x6f, 0x6e, - 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x68, 0x74, 0x74, 0x70, 0x2e, - 0x4f, 0x49, 0x44, 0x43, 0x41, 0x75, 0x74, 0x68, 0x65, 0x6e, 0x74, 0x69, 0x63, 0x61, 0x74, 0x69, - 0x6f, 0x6e, 0x50, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x48, 0x00, 0x52, 0x04, 0x6f, 0x69, 0x64, 0x63, - 0x12, 0x65, 0x0a, 0x0d, 0x61, 0x63, 0x63, 0x65, 0x70, 0x74, 0x5f, 0x68, 0x65, 0x61, 0x64, 0x65, - 0x72, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x3e, 0x2e, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x62, + 0x6e, 0x12, 0x42, 0x0a, 0x03, 0x74, 0x6c, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x30, + 0x2e, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x62, 0x61, 0x72, 0x6e, 0x2e, 0x63, 0x6f, 0x6e, 0x66, 0x69, + 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x74, 0x6c, 0x73, 0x2e, 0x43, 0x6c, 0x69, + 0x65, 0x6e, 0x74, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, + 0x52, 0x03, 0x74, 0x6c, 0x73, 0x12, 0x1b, 0x0a, 0x09, 0x70, 0x72, 0x6f, 0x78, 0x79, 0x5f, 0x75, + 0x72, 0x6c, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x70, 0x72, 0x6f, 0x78, 0x79, 0x55, + 0x72, 0x6c, 0x12, 0x5f, 0x0a, 0x0b, 0x61, 0x64, 0x64, 0x5f, 0x68, 0x65, 0x61, 0x64, 0x65, 0x72, + 0x73, 0x18, 0x05, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x3e, 0x2e, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x62, 0x61, 0x72, 0x6e, 0x2e, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, - 0x6e, 0x2e, 0x68, 0x74, 0x74, 0x70, 0x2e, 0x41, 0x63, 0x63, 0x65, 0x70, 0x74, 0x48, 0x65, 0x61, - 0x64, 0x65, 0x72, 0x41, 0x75, 0x74, 0x68, 0x65, 0x6e, 0x74, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, - 0x6e, 0x50, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x48, 0x00, 0x52, 0x0c, 0x61, 0x63, 0x63, 0x65, 0x70, - 0x74, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x42, 0x08, 0x0a, 0x06, 0x70, 0x6f, 0x6c, 0x69, 0x63, - 0x79, 0x22, 0x69, 0x0a, 0x17, 0x41, 0x6e, 0x79, 0x41, 0x75, 0x74, 0x68, 0x65, 0x6e, 0x74, 0x69, - 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x12, 0x4e, 0x0a, 0x08, - 0x70, 0x6f, 0x6c, 0x69, 0x63, 0x69, 0x65, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x32, + 0x6e, 0x2e, 0x68, 0x74, 0x74, 0x70, 0x2e, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x43, 0x6f, 0x6e, + 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x48, 0x65, 0x61, 0x64, 0x65, + 0x72, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x73, 0x52, 0x0a, 0x61, 0x64, 0x64, 0x48, 0x65, 0x61, 0x64, + 0x65, 0x72, 0x73, 0x12, 0x23, 0x0a, 0x0d, 0x64, 0x69, 0x73, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x68, + 0x74, 0x74, 0x70, 0x32, 0x18, 0x06, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0c, 0x64, 0x69, 0x73, 0x61, + 0x62, 0x6c, 0x65, 0x48, 0x74, 0x74, 0x70, 0x32, 0x1a, 0x3e, 0x0a, 0x0c, 0x48, 0x65, 0x61, 0x64, + 0x65, 0x72, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x73, 0x12, 0x16, 0x0a, 0x06, 0x68, 0x65, 0x61, 0x64, + 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x68, 0x65, 0x61, 0x64, 0x65, 0x72, + 0x12, 0x16, 0x0a, 0x06, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x09, + 0x52, 0x06, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x73, 0x22, 0xed, 0x01, 0x0a, 0x13, 0x53, 0x65, 0x72, + 0x76, 0x65, 0x72, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, + 0x12, 0x29, 0x0a, 0x10, 0x6c, 0x69, 0x73, 0x74, 0x65, 0x6e, 0x5f, 0x61, 0x64, 0x64, 0x72, 0x65, + 0x73, 0x73, 0x65, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0f, 0x6c, 0x69, 0x73, 0x74, + 0x65, 0x6e, 0x41, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x65, 0x73, 0x12, 0x67, 0x0a, 0x15, 0x61, + 0x75, 0x74, 0x68, 0x65, 0x6e, 0x74, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x70, 0x6f, + 0x6c, 0x69, 0x63, 0x79, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x32, 0x2e, 0x62, 0x75, 0x69, + 0x6c, 0x64, 0x62, 0x61, 0x72, 0x6e, 0x2e, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, + 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x68, 0x74, 0x74, 0x70, 0x2e, 0x41, 0x75, 0x74, 0x68, 0x65, 0x6e, + 0x74, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x52, 0x14, + 0x61, 0x75, 0x74, 0x68, 0x65, 0x6e, 0x74, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x6f, + 0x6c, 0x69, 0x63, 0x79, 0x12, 0x42, 0x0a, 0x03, 0x74, 0x6c, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x30, 0x2e, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x62, 0x61, 0x72, 0x6e, 0x2e, 0x63, 0x6f, + 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x74, 0x6c, 0x73, 0x2e, + 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, + 0x69, 0x6f, 0x6e, 0x52, 0x03, 0x74, 0x6c, 0x73, 0x22, 0xa3, 0x04, 0x0a, 0x14, 0x41, 0x75, 0x74, + 0x68, 0x65, 0x6e, 0x74, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x6f, 0x6c, 0x69, 0x63, + 0x79, 0x12, 0x3e, 0x0a, 0x05, 0x61, 0x6c, 0x6c, 0x6f, 0x77, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x26, 0x2e, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x62, 0x61, 0x72, 0x6e, 0x2e, 0x61, 0x75, 0x74, + 0x68, 0x2e, 0x41, 0x75, 0x74, 0x68, 0x65, 0x6e, 0x74, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, + 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x48, 0x00, 0x52, 0x05, 0x61, 0x6c, 0x6c, 0x6f, + 0x77, 0x12, 0x49, 0x0a, 0x03, 0x61, 0x6e, 0x79, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x35, 0x2e, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x62, 0x61, 0x72, 0x6e, 0x2e, 0x63, 0x6f, 0x6e, 0x66, 0x69, - 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x68, 0x74, 0x74, 0x70, 0x2e, 0x41, 0x75, - 0x74, 0x68, 0x65, 0x6e, 0x74, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x6f, 0x6c, 0x69, - 0x63, 0x79, 0x52, 0x08, 0x70, 0x6f, 0x6c, 0x69, 0x63, 0x69, 0x65, 0x73, 0x22, 0x84, 0x04, 0x0a, - 0x18, 0x4f, 0x49, 0x44, 0x43, 0x41, 0x75, 0x74, 0x68, 0x65, 0x6e, 0x74, 0x69, 0x63, 0x61, 0x74, - 0x69, 0x6f, 0x6e, 0x50, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x12, 0x1b, 0x0a, 0x09, 0x63, 0x6c, 0x69, - 0x65, 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x63, 0x6c, - 0x69, 0x65, 0x6e, 0x74, 0x49, 0x64, 0x12, 0x23, 0x0a, 0x0d, 0x63, 0x6c, 0x69, 0x65, 0x6e, 0x74, - 0x5f, 0x73, 0x65, 0x63, 0x72, 0x65, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x63, - 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x53, 0x65, 0x63, 0x72, 0x65, 0x74, 0x12, 0x3c, 0x0a, 0x1a, 0x61, - 0x75, 0x74, 0x68, 0x6f, 0x72, 0x69, 0x7a, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x65, 0x6e, 0x64, - 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x5f, 0x75, 0x72, 0x6c, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, - 0x18, 0x61, 0x75, 0x74, 0x68, 0x6f, 0x72, 0x69, 0x7a, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x45, 0x6e, - 0x64, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x55, 0x72, 0x6c, 0x12, 0x2c, 0x0a, 0x12, 0x74, 0x6f, 0x6b, - 0x65, 0x6e, 0x5f, 0x65, 0x6e, 0x64, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x5f, 0x75, 0x72, 0x6c, 0x18, - 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x10, 0x74, 0x6f, 0x6b, 0x65, 0x6e, 0x45, 0x6e, 0x64, 0x70, - 0x6f, 0x69, 0x6e, 0x74, 0x55, 0x72, 0x6c, 0x12, 0x33, 0x0a, 0x16, 0x75, 0x73, 0x65, 0x72, 0x5f, - 0x69, 0x6e, 0x66, 0x6f, 0x5f, 0x65, 0x6e, 0x64, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x5f, 0x75, 0x72, - 0x6c, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x13, 0x75, 0x73, 0x65, 0x72, 0x49, 0x6e, 0x66, - 0x6f, 0x45, 0x6e, 0x64, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x55, 0x72, 0x6c, 0x12, 0x55, 0x0a, 0x27, - 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x5f, 0x65, 0x78, 0x74, 0x72, 0x61, 0x63, 0x74, - 0x69, 0x6f, 0x6e, 0x5f, 0x6a, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x74, 0x68, 0x5f, 0x65, 0x78, 0x70, - 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x24, 0x6d, - 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x45, 0x78, 0x74, 0x72, 0x61, 0x63, 0x74, 0x69, 0x6f, - 0x6e, 0x4a, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x74, 0x68, 0x45, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73, - 0x69, 0x6f, 0x6e, 0x12, 0x21, 0x0a, 0x0c, 0x72, 0x65, 0x64, 0x69, 0x72, 0x65, 0x63, 0x74, 0x5f, - 0x75, 0x72, 0x6c, 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x72, 0x65, 0x64, 0x69, 0x72, - 0x65, 0x63, 0x74, 0x55, 0x72, 0x6c, 0x12, 0x16, 0x0a, 0x06, 0x73, 0x63, 0x6f, 0x70, 0x65, 0x73, - 0x18, 0x08, 0x20, 0x03, 0x28, 0x09, 0x52, 0x06, 0x73, 0x63, 0x6f, 0x70, 0x65, 0x73, 0x12, 0x1f, - 0x0a, 0x0b, 0x63, 0x6f, 0x6f, 0x6b, 0x69, 0x65, 0x5f, 0x73, 0x65, 0x65, 0x64, 0x18, 0x09, 0x20, - 0x01, 0x28, 0x0c, 0x52, 0x0a, 0x63, 0x6f, 0x6f, 0x6b, 0x69, 0x65, 0x53, 0x65, 0x65, 0x64, 0x12, - 0x52, 0x0a, 0x0b, 0x68, 0x74, 0x74, 0x70, 0x5f, 0x63, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x18, 0x0a, - 0x20, 0x01, 0x28, 0x0b, 0x32, 0x31, 0x2e, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x62, 0x61, 0x72, 0x6e, - 0x2e, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x68, - 0x74, 0x74, 0x70, 0x2e, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, - 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0a, 0x68, 0x74, 0x74, 0x70, 0x43, 0x6c, 0x69, - 0x65, 0x6e, 0x74, 0x22, 0x8f, 0x01, 0x0a, 0x20, 0x41, 0x63, 0x63, 0x65, 0x70, 0x74, 0x48, 0x65, - 0x61, 0x64, 0x65, 0x72, 0x41, 0x75, 0x74, 0x68, 0x65, 0x6e, 0x74, 0x69, 0x63, 0x61, 0x74, 0x69, - 0x6f, 0x6e, 0x50, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x12, 0x1f, 0x0a, 0x0b, 0x6d, 0x65, 0x64, 0x69, - 0x61, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0a, 0x6d, - 0x65, 0x64, 0x69, 0x61, 0x54, 0x79, 0x70, 0x65, 0x73, 0x12, 0x4a, 0x0a, 0x06, 0x70, 0x6f, 0x6c, - 0x69, 0x63, 0x79, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x32, 0x2e, 0x62, 0x75, 0x69, 0x6c, + 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x68, 0x74, 0x74, 0x70, 0x2e, 0x41, 0x6e, + 0x79, 0x41, 0x75, 0x74, 0x68, 0x65, 0x6e, 0x74, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x50, + 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x48, 0x00, 0x52, 0x03, 0x61, 0x6e, 0x79, 0x12, 0x14, 0x0a, 0x04, + 0x64, 0x65, 0x6e, 0x79, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x48, 0x00, 0x52, 0x04, 0x64, 0x65, + 0x6e, 0x79, 0x12, 0x57, 0x0a, 0x03, 0x6a, 0x77, 0x74, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x43, 0x2e, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x62, 0x61, 0x72, 0x6e, 0x2e, 0x63, 0x6f, 0x6e, 0x66, + 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x6a, 0x77, 0x74, 0x2e, 0x41, 0x75, + 0x74, 0x68, 0x6f, 0x72, 0x69, 0x7a, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x48, 0x65, 0x61, 0x64, 0x65, + 0x72, 0x50, 0x61, 0x72, 0x73, 0x65, 0x72, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, + 0x74, 0x69, 0x6f, 0x6e, 0x48, 0x00, 0x52, 0x03, 0x6a, 0x77, 0x74, 0x12, 0x4c, 0x0a, 0x04, 0x6f, + 0x69, 0x64, 0x63, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x36, 0x2e, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x62, 0x61, 0x72, 0x6e, 0x2e, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, - 0x69, 0x6f, 0x6e, 0x2e, 0x68, 0x74, 0x74, 0x70, 0x2e, 0x41, 0x75, 0x74, 0x68, 0x65, 0x6e, 0x74, - 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x52, 0x06, 0x70, - 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x42, 0x3e, 0x5a, 0x3c, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, - 0x63, 0x6f, 0x6d, 0x2f, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x62, 0x61, 0x72, 0x6e, 0x2f, 0x62, 0x62, - 0x2d, 0x73, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x2f, 0x70, 0x6b, 0x67, 0x2f, 0x70, 0x72, 0x6f, - 0x74, 0x6f, 0x2f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, - 0x2f, 0x68, 0x74, 0x74, 0x70, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, + 0x69, 0x6f, 0x6e, 0x2e, 0x68, 0x74, 0x74, 0x70, 0x2e, 0x4f, 0x49, 0x44, 0x43, 0x41, 0x75, 0x74, + 0x68, 0x65, 0x6e, 0x74, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x6f, 0x6c, 0x69, 0x63, + 0x79, 0x48, 0x00, 0x52, 0x04, 0x6f, 0x69, 0x64, 0x63, 0x12, 0x65, 0x0a, 0x0d, 0x61, 0x63, 0x63, + 0x65, 0x70, 0x74, 0x5f, 0x68, 0x65, 0x61, 0x64, 0x65, 0x72, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x3e, 0x2e, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x62, 0x61, 0x72, 0x6e, 0x2e, 0x63, 0x6f, 0x6e, + 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x68, 0x74, 0x74, 0x70, 0x2e, + 0x41, 0x63, 0x63, 0x65, 0x70, 0x74, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x41, 0x75, 0x74, 0x68, + 0x65, 0x6e, 0x74, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x6f, 0x6c, 0x69, 0x63, 0x79, + 0x48, 0x00, 0x52, 0x0c, 0x61, 0x63, 0x63, 0x65, 0x70, 0x74, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, + 0x12, 0x52, 0x0a, 0x06, 0x72, 0x65, 0x6d, 0x6f, 0x74, 0x65, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x38, 0x2e, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x62, 0x61, 0x72, 0x6e, 0x2e, 0x63, 0x6f, 0x6e, + 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, + 0x52, 0x65, 0x6d, 0x6f, 0x74, 0x65, 0x41, 0x75, 0x74, 0x68, 0x65, 0x6e, 0x74, 0x69, 0x63, 0x61, + 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x48, 0x00, 0x52, 0x06, 0x72, 0x65, + 0x6d, 0x6f, 0x74, 0x65, 0x42, 0x08, 0x0a, 0x06, 0x70, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x22, 0x69, + 0x0a, 0x17, 0x41, 0x6e, 0x79, 0x41, 0x75, 0x74, 0x68, 0x65, 0x6e, 0x74, 0x69, 0x63, 0x61, 0x74, + 0x69, 0x6f, 0x6e, 0x50, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x12, 0x4e, 0x0a, 0x08, 0x70, 0x6f, 0x6c, + 0x69, 0x63, 0x69, 0x65, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x32, 0x2e, 0x62, 0x75, + 0x69, 0x6c, 0x64, 0x62, 0x61, 0x72, 0x6e, 0x2e, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, + 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x68, 0x74, 0x74, 0x70, 0x2e, 0x41, 0x75, 0x74, 0x68, 0x65, + 0x6e, 0x74, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x52, + 0x08, 0x70, 0x6f, 0x6c, 0x69, 0x63, 0x69, 0x65, 0x73, 0x22, 0x84, 0x04, 0x0a, 0x18, 0x4f, 0x49, + 0x44, 0x43, 0x41, 0x75, 0x74, 0x68, 0x65, 0x6e, 0x74, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, + 0x50, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x12, 0x1b, 0x0a, 0x09, 0x63, 0x6c, 0x69, 0x65, 0x6e, 0x74, + 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x63, 0x6c, 0x69, 0x65, 0x6e, + 0x74, 0x49, 0x64, 0x12, 0x23, 0x0a, 0x0d, 0x63, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x5f, 0x73, 0x65, + 0x63, 0x72, 0x65, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x63, 0x6c, 0x69, 0x65, + 0x6e, 0x74, 0x53, 0x65, 0x63, 0x72, 0x65, 0x74, 0x12, 0x3c, 0x0a, 0x1a, 0x61, 0x75, 0x74, 0x68, + 0x6f, 0x72, 0x69, 0x7a, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x65, 0x6e, 0x64, 0x70, 0x6f, 0x69, + 0x6e, 0x74, 0x5f, 0x75, 0x72, 0x6c, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x18, 0x61, 0x75, + 0x74, 0x68, 0x6f, 0x72, 0x69, 0x7a, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x45, 0x6e, 0x64, 0x70, 0x6f, + 0x69, 0x6e, 0x74, 0x55, 0x72, 0x6c, 0x12, 0x2c, 0x0a, 0x12, 0x74, 0x6f, 0x6b, 0x65, 0x6e, 0x5f, + 0x65, 0x6e, 0x64, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x5f, 0x75, 0x72, 0x6c, 0x18, 0x04, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x10, 0x74, 0x6f, 0x6b, 0x65, 0x6e, 0x45, 0x6e, 0x64, 0x70, 0x6f, 0x69, 0x6e, + 0x74, 0x55, 0x72, 0x6c, 0x12, 0x33, 0x0a, 0x16, 0x75, 0x73, 0x65, 0x72, 0x5f, 0x69, 0x6e, 0x66, + 0x6f, 0x5f, 0x65, 0x6e, 0x64, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x5f, 0x75, 0x72, 0x6c, 0x18, 0x05, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x13, 0x75, 0x73, 0x65, 0x72, 0x49, 0x6e, 0x66, 0x6f, 0x45, 0x6e, + 0x64, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x55, 0x72, 0x6c, 0x12, 0x55, 0x0a, 0x27, 0x6d, 0x65, 0x74, + 0x61, 0x64, 0x61, 0x74, 0x61, 0x5f, 0x65, 0x78, 0x74, 0x72, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, + 0x5f, 0x6a, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x74, 0x68, 0x5f, 0x65, 0x78, 0x70, 0x72, 0x65, 0x73, + 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x24, 0x6d, 0x65, 0x74, 0x61, + 0x64, 0x61, 0x74, 0x61, 0x45, 0x78, 0x74, 0x72, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x4a, 0x6d, + 0x65, 0x73, 0x70, 0x61, 0x74, 0x68, 0x45, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, + 0x12, 0x21, 0x0a, 0x0c, 0x72, 0x65, 0x64, 0x69, 0x72, 0x65, 0x63, 0x74, 0x5f, 0x75, 0x72, 0x6c, + 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x72, 0x65, 0x64, 0x69, 0x72, 0x65, 0x63, 0x74, + 0x55, 0x72, 0x6c, 0x12, 0x16, 0x0a, 0x06, 0x73, 0x63, 0x6f, 0x70, 0x65, 0x73, 0x18, 0x08, 0x20, + 0x03, 0x28, 0x09, 0x52, 0x06, 0x73, 0x63, 0x6f, 0x70, 0x65, 0x73, 0x12, 0x1f, 0x0a, 0x0b, 0x63, + 0x6f, 0x6f, 0x6b, 0x69, 0x65, 0x5f, 0x73, 0x65, 0x65, 0x64, 0x18, 0x09, 0x20, 0x01, 0x28, 0x0c, + 0x52, 0x0a, 0x63, 0x6f, 0x6f, 0x6b, 0x69, 0x65, 0x53, 0x65, 0x65, 0x64, 0x12, 0x52, 0x0a, 0x0b, + 0x68, 0x74, 0x74, 0x70, 0x5f, 0x63, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x18, 0x0a, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x31, 0x2e, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x62, 0x61, 0x72, 0x6e, 0x2e, 0x63, 0x6f, + 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x68, 0x74, 0x74, 0x70, + 0x2e, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, + 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0a, 0x68, 0x74, 0x74, 0x70, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, + 0x22, 0x8f, 0x01, 0x0a, 0x20, 0x41, 0x63, 0x63, 0x65, 0x70, 0x74, 0x48, 0x65, 0x61, 0x64, 0x65, + 0x72, 0x41, 0x75, 0x74, 0x68, 0x65, 0x6e, 0x74, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x50, + 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x12, 0x1f, 0x0a, 0x0b, 0x6d, 0x65, 0x64, 0x69, 0x61, 0x5f, 0x74, + 0x79, 0x70, 0x65, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0a, 0x6d, 0x65, 0x64, 0x69, + 0x61, 0x54, 0x79, 0x70, 0x65, 0x73, 0x12, 0x4a, 0x0a, 0x06, 0x70, 0x6f, 0x6c, 0x69, 0x63, 0x79, + 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x32, 0x2e, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x62, 0x61, + 0x72, 0x6e, 0x2e, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, + 0x2e, 0x68, 0x74, 0x74, 0x70, 0x2e, 0x41, 0x75, 0x74, 0x68, 0x65, 0x6e, 0x74, 0x69, 0x63, 0x61, + 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x52, 0x06, 0x70, 0x6f, 0x6c, 0x69, + 0x63, 0x79, 0x42, 0x3e, 0x5a, 0x3c, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, + 0x2f, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x62, 0x61, 0x72, 0x6e, 0x2f, 0x62, 0x62, 0x2d, 0x73, 0x74, + 0x6f, 0x72, 0x61, 0x67, 0x65, 0x2f, 0x70, 0x6b, 0x67, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, + 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2f, 0x68, 0x74, + 0x74, 0x70, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, }) var ( @@ -720,6 +745,7 @@ var file_pkg_proto_configuration_http_http_proto_goTypes = []any{ (*tls.ServerConfiguration)(nil), // 8: buildbarn.configuration.tls.ServerConfiguration (*auth.AuthenticationMetadata)(nil), // 9: buildbarn.auth.AuthenticationMetadata (*jwt.AuthorizationHeaderParserConfiguration)(nil), // 10: buildbarn.configuration.jwt.AuthorizationHeaderParserConfiguration + (*grpc.RemoteAuthenticationPolicy)(nil), // 11: buildbarn.configuration.grpc.RemoteAuthenticationPolicy } var file_pkg_proto_configuration_http_http_proto_depIdxs = []int32{ 7, // 0: buildbarn.configuration.http.ClientConfiguration.tls:type_name -> buildbarn.configuration.tls.ClientConfiguration @@ -731,14 +757,15 @@ var file_pkg_proto_configuration_http_http_proto_depIdxs = []int32{ 10, // 6: buildbarn.configuration.http.AuthenticationPolicy.jwt:type_name -> buildbarn.configuration.jwt.AuthorizationHeaderParserConfiguration 4, // 7: buildbarn.configuration.http.AuthenticationPolicy.oidc:type_name -> buildbarn.configuration.http.OIDCAuthenticationPolicy 5, // 8: buildbarn.configuration.http.AuthenticationPolicy.accept_header:type_name -> buildbarn.configuration.http.AcceptHeaderAuthenticationPolicy - 2, // 9: buildbarn.configuration.http.AnyAuthenticationPolicy.policies:type_name -> buildbarn.configuration.http.AuthenticationPolicy - 0, // 10: buildbarn.configuration.http.OIDCAuthenticationPolicy.http_client:type_name -> buildbarn.configuration.http.ClientConfiguration - 2, // 11: buildbarn.configuration.http.AcceptHeaderAuthenticationPolicy.policy:type_name -> buildbarn.configuration.http.AuthenticationPolicy - 12, // [12:12] is the sub-list for method output_type - 12, // [12:12] is the sub-list for method input_type - 12, // [12:12] is the sub-list for extension type_name - 12, // [12:12] is the sub-list for extension extendee - 0, // [0:12] is the sub-list for field type_name + 11, // 9: buildbarn.configuration.http.AuthenticationPolicy.remote:type_name -> buildbarn.configuration.grpc.RemoteAuthenticationPolicy + 2, // 10: buildbarn.configuration.http.AnyAuthenticationPolicy.policies:type_name -> buildbarn.configuration.http.AuthenticationPolicy + 0, // 11: buildbarn.configuration.http.OIDCAuthenticationPolicy.http_client:type_name -> buildbarn.configuration.http.ClientConfiguration + 2, // 12: buildbarn.configuration.http.AcceptHeaderAuthenticationPolicy.policy:type_name -> buildbarn.configuration.http.AuthenticationPolicy + 13, // [13:13] is the sub-list for method output_type + 13, // [13:13] is the sub-list for method input_type + 13, // [13:13] is the sub-list for extension type_name + 13, // [13:13] is the sub-list for extension extendee + 0, // [0:13] is the sub-list for field type_name } func init() { file_pkg_proto_configuration_http_http_proto_init() } @@ -753,6 +780,7 @@ func file_pkg_proto_configuration_http_http_proto_init() { (*AuthenticationPolicy_Jwt)(nil), (*AuthenticationPolicy_Oidc)(nil), (*AuthenticationPolicy_AcceptHeader)(nil), + (*AuthenticationPolicy_Remote)(nil), } type x struct{} out := protoimpl.TypeBuilder{ diff --git a/pkg/proto/configuration/http/http.proto b/pkg/proto/configuration/http/http.proto index f3faabb1..bd2e4e34 100644 --- a/pkg/proto/configuration/http/http.proto +++ b/pkg/proto/configuration/http/http.proto @@ -3,6 +3,7 @@ syntax = "proto3"; package buildbarn.configuration.http; import "pkg/proto/auth/auth.proto"; +import "pkg/proto/configuration/grpc/grpc.proto"; import "pkg/proto/configuration/jwt/jwt.proto"; import "pkg/proto/configuration/tls/tls.proto"; @@ -92,6 +93,13 @@ message AuthenticationPolicy { // limit OpenID Connect authentication to requests originating from // a web browser. AcceptHeaderAuthenticationPolicy accept_header = 6; + + // Forwards the authentication task to a remote server. This can be a + // convenient way to integrate custom authentication processes, for example + // if a proxy has already performed authentication where the resulting HTTP + // request's headers need to be verified and processed in a custom way to + // produce buildbarn.auth.AuthenticationMetadata. + buildbarn.configuration.grpc.RemoteAuthenticationPolicy remote = 7; } }