diff --git a/internal/datastore/context.go b/internal/datastore/context.go index 4ef59874a0..ef452a8417 100644 --- a/internal/datastore/context.go +++ b/internal/datastore/context.go @@ -85,6 +85,10 @@ func (p *ctxProxy) Features(ctx context.Context) (*datastore.Features, error) { return p.delegate.Features(SeparateContextWithTracing(ctx)) } +func (p *ctxProxy) OfflineFeatures() (*datastore.Features, error) { + return p.delegate.OfflineFeatures() +} + func (p *ctxProxy) Statistics(ctx context.Context) (datastore.Stats, error) { return p.delegate.Statistics(SeparateContextWithTracing(ctx)) } diff --git a/internal/datastore/crdb/crdb.go b/internal/datastore/crdb/crdb.go index c4cee4efd8..af48a91cb4 100644 --- a/internal/datastore/crdb/crdb.go +++ b/internal/datastore/crdb/crdb.go @@ -48,23 +48,31 @@ const ( Engine = "cockroachdb" tableNamespace = "namespace_config" tableTuple = "relation_tuple" + tableTupleWithIntegrity = "relation_tuple_with_integrity" tableTransactions = "transactions" tableCaveat = "caveat" tableRelationshipCounter = "relationship_counter" - colNamespace = "namespace" - colConfig = "serialized_config" - colTimestamp = "timestamp" - colTransactionKey = "key" - colObjectID = "object_id" - colRelation = "relation" - colUsersetNamespace = "userset_namespace" - colUsersetObjectID = "userset_object_id" - colUsersetRelation = "userset_relation" - colCaveatName = "name" - colCaveatDefinition = "definition" - colCaveatContextName = "caveat_name" - colCaveatContext = "caveat_context" + colNamespace = "namespace" + colConfig = "serialized_config" + colTimestamp = "timestamp" + colTransactionKey = "key" + + colObjectID = "object_id" + colRelation = "relation" + + colUsersetNamespace = "userset_namespace" + colUsersetObjectID = "userset_object_id" + colUsersetRelation = "userset_relation" + + colCaveatName = "name" + colCaveatDefinition = "definition" + colCaveatContextName = "caveat_name" + colCaveatContext = "caveat_context" + + colIntegrityHash = "integrity_hash" + colIntegrityKeyID = "integrity_key_id" + colCounterName = "name" colCounterSerializedFilter = "serialized_filter" colCounterCurrentCount = "current_count" @@ -190,6 +198,7 @@ func newCRDBDatastore(ctx context.Context, url string, options ...Option) (datas transactionNowQuery: transactionNowQuery, analyzeBeforeStatistics: config.analyzeBeforeStatistics, filterMaximumIDCount: config.filterMaximumIDCount, + supportsIntegrity: config.withIntegrity, } ds.RemoteClockRevisions.SetNowFunc(ds.headRevisionInternal) @@ -282,18 +291,19 @@ type crdbDatastore struct { ctx context.Context cancel context.CancelFunc filterMaximumIDCount uint16 + supportsIntegrity bool } func (cds *crdbDatastore) SnapshotReader(rev datastore.Revision) datastore.Reader { executor := common.QueryExecutor{ - Executor: pgxcommon.NewPGXExecutor(cds.readPool), + Executor: pgxcommon.NewPGXExecutorWithIntegrityOption(cds.readPool, cds.supportsIntegrity), } fromBuilder := func(query sq.SelectBuilder, fromStr string) sq.SelectBuilder { return query.From(fromStr + " AS OF SYSTEM TIME " + rev.String()) } - return &crdbReader{cds.readPool, executor, noOverlapKeyer, nil, fromBuilder, cds.filterMaximumIDCount} + return &crdbReader{cds.readPool, executor, noOverlapKeyer, nil, fromBuilder, cds.filterMaximumIDCount, cds.tableTupleName(), cds.supportsIntegrity} } func (cds *crdbDatastore) ReadWriteTx( @@ -311,7 +321,7 @@ func (cds *crdbDatastore) ReadWriteTx( err := cds.writePool.BeginFunc(ctx, func(tx pgx.Tx) error { querier := pgxcommon.QuerierFuncsFor(tx) executor := common.QueryExecutor{ - Executor: pgxcommon.NewPGXExecutor(querier), + Executor: pgxcommon.NewPGXExecutorWithIntegrityOption(querier, cds.supportsIntegrity), } rwt := &crdbReadWriteTXN{ @@ -324,6 +334,8 @@ func (cds *crdbDatastore) ReadWriteTx( return query.From(fromStr) }, cds.filterMaximumIDCount, + cds.tableTupleName(), + cds.supportsIntegrity, }, tx, 0, @@ -442,6 +454,22 @@ func (cds *crdbDatastore) headRevisionInternal(ctx context.Context) (datastore.R return hlcNow, fnErr } +func (cds *crdbDatastore) OfflineFeatures() (*datastore.Features, error) { + if cds.supportsIntegrity { + return &datastore.Features{ + IntegrityData: datastore.Feature{ + Status: datastore.FeatureSupported, + }, + }, nil + } + + return &datastore.Features{ + IntegrityData: datastore.Feature{ + Status: datastore.FeatureUnsupported, + }, + }, nil +} + func (cds *crdbDatastore) Features(ctx context.Context) (*datastore.Features, error) { features, _, err := cds.featureGroup.Do(ctx, "", func(ictx context.Context) (*datastore.Features, error) { return cds.features(ictx) @@ -449,8 +477,23 @@ func (cds *crdbDatastore) Features(ctx context.Context) (*datastore.Features, er return features, err } +func (cds *crdbDatastore) SupportsIntegrity() bool { + return cds.supportsIntegrity +} + +func (cds *crdbDatastore) tableTupleName() string { + if cds.supportsIntegrity { + return tableTupleWithIntegrity + } + + return tableTuple +} + func (cds *crdbDatastore) features(ctx context.Context) (*datastore.Features, error) { - var features datastore.Features + features := datastore.Features{} + if cds.supportsIntegrity { + features.IntegrityData.Status = datastore.FeatureSupported + } head, err := cds.HeadRevision(ctx) if err != nil { @@ -465,14 +508,14 @@ func (cds *crdbDatastore) features(ctx context.Context) (*datastore.Features, er _ = cds.writePool.ExecFunc(streamCtx, func(ctx context.Context, tag pgconn.CommandTag, err error) error { if err != nil && errors.Is(err, context.Canceled) { - features.Watch.Enabled = true + features.Watch.Status = datastore.FeatureSupported features.Watch.Reason = "" } else if err != nil { - features.Watch.Enabled = false + features.Watch.Status = datastore.FeatureUnsupported features.Watch.Reason = fmt.Sprintf("Range feeds must be enabled in CockroachDB and the user must have permission to create them in order to enable the Watch API: %s", err.Error()) } return nil - }, fmt.Sprintf(cds.beginChangefeedQuery, tableTuple, head, "1s")) + }, fmt.Sprintf(cds.beginChangefeedQuery, cds.tableTupleName(), head, "1s")) <-streamCtx.Done() diff --git a/internal/datastore/crdb/crdb_test.go b/internal/datastore/crdb/crdb_test.go index 097176a9f3..53194badc2 100644 --- a/internal/datastore/crdb/crdb_test.go +++ b/internal/datastore/crdb/crdb_test.go @@ -10,8 +10,10 @@ import ( "crypto/rand" "crypto/x509" "crypto/x509/pkix" + "encoding/hex" "encoding/pem" "fmt" + "math" "math/big" "net" "os" @@ -24,14 +26,24 @@ import ( "github.com/jackc/pgx/v5/pgxpool" "github.com/ory/dockertest/v3" "github.com/stretchr/testify/require" + "google.golang.org/protobuf/types/known/timestamppb" crdbmigrations "github.com/authzed/spicedb/internal/datastore/crdb/migrations" "github.com/authzed/spicedb/internal/datastore/crdb/pool" + "github.com/authzed/spicedb/internal/datastore/proxy" "github.com/authzed/spicedb/internal/datastore/revisions" + "github.com/authzed/spicedb/internal/testfixtures" testdatastore "github.com/authzed/spicedb/internal/testserver/datastore" "github.com/authzed/spicedb/pkg/datastore" "github.com/authzed/spicedb/pkg/datastore/test" "github.com/authzed/spicedb/pkg/migrate" + core "github.com/authzed/spicedb/pkg/proto/core/v1" + "github.com/authzed/spicedb/pkg/tuple" +) + +const ( + veryLargeGCWindow = 90000 * time.Second + veryLargeGCInterval = 90000 * time.Second ) // Implement the TestableDatastore interface @@ -41,7 +53,7 @@ func (cds *crdbDatastore) ExampleRetryableError() error { } } -func TestCRDBDatastore(t *testing.T) { +func TestCRDBDatastoreWithoutIntegrity(t *testing.T) { b := testdatastore.RunCRDBForTesting(t, "") test.All(t, test.DatastoreTesterFunc(func(revisionQuantization, gcInterval, gcWindow time.Duration, watchBufferLength uint16) (datastore.Datastore, error) { ctx := context.Background() @@ -111,6 +123,69 @@ func TestCRDBDatastoreWithFollowerReads(t *testing.T) { } } +var defaultKeyForTesting = proxy.KeyConfig{ + ID: "defaultfortest", + Bytes: (func() []byte { + b, err := hex.DecodeString("000102030405060708090A0B0C0D0E0FF0E0D0C0B0A090807060504030201000") + if err != nil { + panic(err) + } + return b + })(), + ExpiredAt: nil, +} + +func TestCRDBDatastoreWithIntegrity(t *testing.T) { + b := testdatastore.RunCRDBForTesting(t, "") + + test.All(t, test.DatastoreTesterFunc(func(revisionQuantization, gcInterval, gcWindow time.Duration, watchBufferLength uint16) (datastore.Datastore, error) { + ctx := context.Background() + ds := b.NewDatastore(t, func(engine, uri string) datastore.Datastore { + ds, err := NewCRDBDatastore( + ctx, + uri, + GCWindow(gcWindow), + RevisionQuantization(revisionQuantization), + WatchBufferLength(watchBufferLength), + OverlapStrategy(overlapStrategyPrefix), + DebugAnalyzeBeforeStatistics(), + WithIntegrity(true), + ) + require.NoError(t, err) + + wrapped, err := proxy.NewRelationshipIntegrityProxy(ds, defaultKeyForTesting, nil) + require.NoError(t, err) + return wrapped + }) + + return ds, nil + })) + + unwrappedTester := test.DatastoreTesterFunc(func(revisionQuantization, gcInterval, gcWindow time.Duration, watchBufferLength uint16) (datastore.Datastore, error) { + ctx := context.Background() + ds := b.NewDatastore(t, func(engine, uri string) datastore.Datastore { + ds, err := NewCRDBDatastore( + ctx, + uri, + GCWindow(gcWindow), + RevisionQuantization(revisionQuantization), + WatchBufferLength(watchBufferLength), + OverlapStrategy(overlapStrategyPrefix), + DebugAnalyzeBeforeStatistics(), + WithIntegrity(true), + ) + require.NoError(t, err) + return ds + }) + + return ds, nil + }) + + t.Run("TestRelationshipIntegrityInfo", func(t *testing.T) { RelationshipIntegrityInfoTest(t, unwrappedTester) }) + t.Run("TestBulkRelationshipIntegrityInfo", func(t *testing.T) { BulkRelationshipIntegrityInfoTest(t, unwrappedTester) }) + t.Run("TestWatchRelationshipIntegrity", func(t *testing.T) { RelationshipIntegrityWatchTest(t, unwrappedTester) }) +} + func TestWatchFeatureDetection(t *testing.T) { pool, err := dockertest.NewPool("") require.NoError(t, err) @@ -171,10 +246,10 @@ func TestWatchFeatureDetection(t *testing.T) { features, err := ds.Features(ctx) require.NoError(t, err) - require.Equal(t, tt.expectEnabled, features.Watch.Enabled) + require.Equal(t, tt.expectEnabled, features.Watch.Status == datastore.FeatureSupported) require.Contains(t, features.Watch.Reason, tt.expectMessage) - if !features.Watch.Enabled { + if features.Watch.Status == datastore.FeatureSupported { headRevision, err := ds.HeadRevision(ctx) require.NoError(t, err) @@ -348,3 +423,169 @@ func newCRDBWithUser(t *testing.T, pool *dockertest.Pool) (adminConn *pgx.Conn, return } + +func RelationshipIntegrityInfoTest(t *testing.T, tester test.DatastoreTester) { + require := require.New(t) + + rawDS, err := tester.New(0, veryLargeGCInterval, veryLargeGCWindow, 1) + require.NoError(err) + + ds, _ := testfixtures.StandardDatastoreWithSchema(rawDS, require) + ctx := context.Background() + + // Write a relationship with integrity information. + timestamp := time.Now().UTC() + + _, err = ds.ReadWriteTx(ctx, func(ctx context.Context, rwt datastore.ReadWriteTransaction) error { + tpl := tuple.MustParse("document:foo#viewer@user:tom") + tpl.Integrity = &core.RelationshipIntegrity{ + KeyId: "key1", + Hash: []byte("hash1"), + HashedAt: timestamppb.New(timestamp), + } + return rwt.WriteRelationships(ctx, []*core.RelationTupleUpdate{ + tuple.Create(tpl), + }) + }) + require.NoError(err) + + // Read the relationship back and ensure the integrity information is present. + headRev, err := ds.HeadRevision(ctx) + require.NoError(err) + + reader := ds.SnapshotReader(headRev) + iter, err := reader.QueryRelationships(ctx, datastore.RelationshipsFilter{ + OptionalResourceType: "document", + OptionalResourceIds: []string{"foo"}, + OptionalResourceRelation: "viewer", + }) + require.NoError(err) + t.Cleanup(iter.Close) + + tpl := iter.Next() + require.NotNil(tpl) + + require.NotNil(tpl.Integrity) + require.Equal("key1", tpl.Integrity.KeyId) + require.Equal([]byte("hash1"), tpl.Integrity.Hash) + + require.LessOrEqual(math.Abs(float64(timestamp.Sub(tpl.Integrity.HashedAt.AsTime()).Milliseconds())), 1000.0) + + iter.Close() +} + +type fakeSource struct { + tpl *core.RelationTuple +} + +func (f *fakeSource) Next(ctx context.Context) (*core.RelationTuple, error) { + if f.tpl == nil { + return nil, nil + } + + tpl := f.tpl + f.tpl = nil + return tpl, nil +} + +func BulkRelationshipIntegrityInfoTest(t *testing.T, tester test.DatastoreTester) { + require := require.New(t) + + rawDS, err := tester.New(0, veryLargeGCInterval, veryLargeGCWindow, 1) + require.NoError(err) + + ds, _ := testfixtures.StandardDatastoreWithSchema(rawDS, require) + ctx := context.Background() + + // Write a relationship with integrity information. + timestamp := time.Now().UTC() + + _, err = ds.ReadWriteTx(ctx, func(ctx context.Context, rwt datastore.ReadWriteTransaction) error { + tpl := tuple.MustParse("document:foo#viewer@user:tom") + tpl.Integrity = &core.RelationshipIntegrity{ + KeyId: "key1", + Hash: []byte("hash1"), + HashedAt: timestamppb.New(timestamp), + } + + _, err := rwt.BulkLoad(ctx, &fakeSource{tpl}) + return err + }) + require.NoError(err) + + // Read the relationship back and ensure the integrity information is present. + headRev, err := ds.HeadRevision(ctx) + require.NoError(err) + + reader := ds.SnapshotReader(headRev) + iter, err := reader.QueryRelationships(ctx, datastore.RelationshipsFilter{ + OptionalResourceType: "document", + OptionalResourceIds: []string{"foo"}, + OptionalResourceRelation: "viewer", + }) + require.NoError(err) + t.Cleanup(iter.Close) + + tpl := iter.Next() + require.NotNil(tpl) + + require.NotNil(tpl.Integrity) + require.Equal("key1", tpl.Integrity.KeyId) + require.Equal([]byte("hash1"), tpl.Integrity.Hash) + + require.LessOrEqual(math.Abs(float64(timestamp.Sub(tpl.Integrity.HashedAt.AsTime()).Milliseconds())), 1000.0) + + iter.Close() +} + +func RelationshipIntegrityWatchTest(t *testing.T, tester test.DatastoreTester) { + require := require.New(t) + + rawDS, err := tester.New(0, veryLargeGCInterval, veryLargeGCWindow, 1) + require.NoError(err) + + ds, rev := testfixtures.StandardDatastoreWithSchema(rawDS, require) + ctx := context.Background() + + // Write a relationship with integrity information. + timestamp := time.Now().UTC() + + _, err = ds.ReadWriteTx(ctx, func(ctx context.Context, rwt datastore.ReadWriteTransaction) error { + tpl := tuple.MustParse("document:foo#viewer@user:tom") + tpl.Integrity = &core.RelationshipIntegrity{ + KeyId: "key1", + Hash: []byte("hash1"), + HashedAt: timestamppb.New(timestamp), + } + return rwt.WriteRelationships(ctx, []*core.RelationTupleUpdate{ + tuple.Create(tpl), + }) + }) + require.NoError(err) + + // Ensure the watch API returns the integrity information. + opts := datastore.WatchOptions{ + Content: datastore.WatchRelationships, + WatchBufferLength: 128, + WatchBufferWriteTimeout: 1 * time.Minute, + } + + changes, errchan := ds.Watch(ctx, rev, opts) + select { + case change, ok := <-changes: + if !ok { + require.Fail("Timed out waiting for ErrWatchDisconnected") + } + + tpl := change.RelationshipChanges[0].Tuple + require.NotNil(tpl.Integrity) + require.Equal("key1", tpl.Integrity.KeyId) + require.Equal([]byte("hash1"), tpl.Integrity.Hash) + + require.LessOrEqual(math.Abs(float64(timestamp.Sub(tpl.Integrity.HashedAt.AsTime()).Milliseconds())), 1000.0) + case err := <-errchan: + require.Failf("Failed waiting for changes with error", "error: %v", err) + case <-time.NewTimer(10 * time.Second).C: + require.Fail("Timed out") + } +} diff --git a/internal/datastore/crdb/migrations/zz_migration.0007_add_integrity_relationtuple_table.go b/internal/datastore/crdb/migrations/zz_migration.0007_add_integrity_relationtuple_table.go new file mode 100644 index 0000000000..5107340eb8 --- /dev/null +++ b/internal/datastore/crdb/migrations/zz_migration.0007_add_integrity_relationtuple_table.go @@ -0,0 +1,46 @@ +package migrations + +import ( + "context" + "fmt" + + "github.com/jackc/pgx/v5" +) + +const ( + createRelationTupleWithIntegrityTable = `CREATE TABLE relation_tuple_with_integrity ( + namespace VARCHAR NOT NULL, + object_id VARCHAR NOT NULL, + relation VARCHAR NOT NULL, + userset_namespace VARCHAR NOT NULL, + userset_object_id VARCHAR NOT NULL, + userset_relation VARCHAR NOT NULL, + caveat_name VARCHAR, + caveat_context JSONB, + timestamp TIMESTAMP WITHOUT TIME ZONE DEFAULT now() NOT NULL, + integrity_hash BYTEA NOT NULL, + integrity_key_id VARCHAR(255) NOT NULL, + CONSTRAINT pk_relation_tuple PRIMARY KEY (namespace, object_id, relation, userset_namespace, userset_object_id, userset_relation) +);` + + createIntegrityTupleIndex = `CREATE INDEX ix_relation_tuple_with_integrity ON relation_tuple_with_integrity (namespace, object_id, relation, userset_namespace, userset_object_id, userset_relation) STORING (integrity_key_id, integrity_hash, timestamp, caveat_name, caveat_context);` +) + +func init() { + err := CRDBMigrations.Register("add-integrity-relationtuple-table", "add-relationship-counters-table", addIntegrityColumns, noAtomicMigration) + if err != nil { + panic("failed to register migration: " + err.Error()) + } +} + +func addIntegrityColumns(ctx context.Context, conn *pgx.Conn) error { + if _, err := conn.Exec(ctx, createRelationTupleWithIntegrityTable); err != nil { + return fmt.Errorf("failed to create relation_tuple_with_integrity table: %w", err) + } + + if _, err := conn.Exec(ctx, createIntegrityTupleIndex); err != nil { + return fmt.Errorf("failed to create index on relation_tuple_with_integrity table: %w", err) + } + + return nil +} diff --git a/internal/datastore/crdb/options.go b/internal/datastore/crdb/options.go index bf561309ab..aa1dc4459a 100644 --- a/internal/datastore/crdb/options.go +++ b/internal/datastore/crdb/options.go @@ -26,6 +26,7 @@ type crdbOptions struct { analyzeBeforeStatistics bool filterMaximumIDCount uint16 enablePrometheusStats bool + withIntegrity bool } const ( @@ -52,6 +53,7 @@ const ( defaultEnableConnectionBalancing = true defaultConnectRate = 100 * time.Millisecond defaultFilterMaximumIDCount = 100 + defaultWithIntegrity = false ) // Option provides the facility to configure how clients within the CRDB @@ -74,6 +76,7 @@ func generateConfig(options []Option) (crdbOptions, error) { enableConnectionBalancing: defaultEnableConnectionBalancing, connectRate: defaultConnectRate, filterMaximumIDCount: defaultFilterMaximumIDCount, + withIntegrity: defaultWithIntegrity, } for _, option := range options { @@ -330,3 +333,8 @@ func DebugAnalyzeBeforeStatistics() Option { func FilterMaximumIDCount(filterMaximumIDCount uint16) Option { return func(po *crdbOptions) { po.filterMaximumIDCount = filterMaximumIDCount } } + +// WithIntegrity marks whether the datastore should store and return integrity information. +func WithIntegrity(withIntegrity bool) Option { + return func(po *crdbOptions) { po.withIntegrity = withIntegrity } +} diff --git a/internal/datastore/crdb/reader.go b/internal/datastore/crdb/reader.go index 312f86e282..41e3c91fe0 100644 --- a/internal/datastore/crdb/reader.go +++ b/internal/datastore/crdb/reader.go @@ -27,17 +27,6 @@ const ( var ( queryReadNamespace = psql.Select(colConfig, colTimestamp) - queryTuples = psql.Select( - colNamespace, - colObjectID, - colRelation, - colUsersetNamespace, - colUsersetObjectID, - colUsersetRelation, - colCaveatContextName, - colCaveatContext, - ) - countTuples = psql.Select("count(*)") schema = common.NewSchemaInformation( @@ -66,6 +55,8 @@ type crdbReader struct { overlapKeySet keySet fromBuilder func(query sq.SelectBuilder, fromStr string) sq.SelectBuilder filterMaximumIDCount uint16 + tupleTableName string + withIntegrity bool } func (cr *crdbReader) CountRelationships(ctx context.Context, name string) (int, error) { @@ -83,7 +74,7 @@ func (cr *crdbReader) CountRelationships(ctx context.Context, name string) (int, return 0, err } - query := cr.fromBuilder(countTuples, tableTuple) + query := cr.fromBuilder(countTuples, cr.tupleTableName) builder, err := common.NewSchemaQueryFilterer(schema, query, cr.filterMaximumIDCount).FilterWithRelationshipsFilter(relFilter) if err != nil { return 0, err @@ -192,6 +183,35 @@ func (cr *crdbReader) ListAllNamespaces(ctx context.Context) ([]datastore.Revisi return nsDefs, nil } +func (cr *crdbReader) queryTuples() sq.SelectBuilder { + if cr.withIntegrity { + return psql.Select( + colNamespace, + colObjectID, + colRelation, + colUsersetNamespace, + colUsersetObjectID, + colUsersetRelation, + colCaveatContextName, + colCaveatContext, + colIntegrityKeyID, + colIntegrityHash, + colTimestamp, + ) + } + + return psql.Select( + colNamespace, + colObjectID, + colRelation, + colUsersetNamespace, + colUsersetObjectID, + colUsersetRelation, + colCaveatContextName, + colCaveatContext, + ) +} + func (cr *crdbReader) LookupNamespacesWithNames(ctx context.Context, nsNames []string) ([]datastore.RevisionedNamespace, error) { if len(nsNames) == 0 { return nil, nil @@ -208,7 +228,7 @@ func (cr *crdbReader) QueryRelationships( filter datastore.RelationshipsFilter, opts ...options.QueryOptionsOption, ) (iter datastore.RelationshipIterator, err error) { - query := cr.fromBuilder(queryTuples, tableTuple) + query := cr.fromBuilder(cr.queryTuples(), cr.tupleTableName) qBuilder, err := common.NewSchemaQueryFilterer(schema, query, cr.filterMaximumIDCount).FilterWithRelationshipsFilter(filter) if err != nil { return nil, err @@ -222,7 +242,7 @@ func (cr *crdbReader) ReverseQueryRelationships( subjectsFilter datastore.SubjectsFilter, opts ...options.ReverseQueryOptionsOption, ) (iter datastore.RelationshipIterator, err error) { - query := cr.fromBuilder(queryTuples, tableTuple) + query := cr.fromBuilder(cr.queryTuples(), cr.tupleTableName) qBuilder, err := common.NewSchemaQueryFilterer(schema, query, cr.filterMaximumIDCount). FilterWithSubjectsSelectors(subjectsFilter.AsSelector()) if err != nil { diff --git a/internal/datastore/crdb/readwrite.go b/internal/datastore/crdb/readwrite.go index a3fd8917e1..6b1be9e5c1 100644 --- a/internal/datastore/crdb/readwrite.go +++ b/internal/datastore/crdb/readwrite.go @@ -18,6 +18,7 @@ import ( "github.com/authzed/spicedb/pkg/datastore" "github.com/authzed/spicedb/pkg/datastore/options" core "github.com/authzed/spicedb/pkg/proto/core/v1" + "github.com/authzed/spicedb/pkg/spiceerrors" ) const ( @@ -50,7 +51,7 @@ type crdbReadWriteTXN struct { } var ( - upsertTupleSuffix = fmt.Sprintf( + upsertTupleSuffixWithoutIntegrity = fmt.Sprintf( "ON CONFLICT (%s,%s,%s,%s,%s,%s) DO UPDATE SET %s = now(), %s = excluded.%s, %s = excluded.%s WHERE (relation_tuple.%s <> excluded.%s OR relation_tuple.%s <> excluded.%s)", colNamespace, colObjectID, @@ -69,21 +70,29 @@ var ( colCaveatContext, ) - queryWriteTuple = psql.Insert(tableTuple).Columns( + upsertTupleSuffixWithIntegrity = fmt.Sprintf( + "ON CONFLICT (%s,%s,%s,%s,%s,%s) DO UPDATE SET %s = now(), %s = excluded.%s, %s = excluded.%s, %s = excluded.%s, %s = excluded.%s WHERE (relation_tuple_with_integrity.%s <> excluded.%s OR relation_tuple_with_integrity.%s <> excluded.%s)", colNamespace, colObjectID, colRelation, colUsersetNamespace, colUsersetObjectID, colUsersetRelation, + colTimestamp, + colCaveatContextName, colCaveatContextName, colCaveatContext, + colCaveatContext, + colIntegrityKeyID, + colIntegrityKeyID, + colIntegrityHash, + colIntegrityHash, + colCaveatContextName, + colCaveatContextName, + colCaveatContext, + colCaveatContext, ) - queryTouchTuple = queryWriteTuple.Suffix(upsertTupleSuffix) - - queryDeleteTuples = psql.Delete(tableTuple) - queryTouchTransaction = fmt.Sprintf( "INSERT INTO %s (%s) VALUES ($1::text) ON CONFLICT (%s) DO UPDATE SET %s = now()", tableTransactions, @@ -104,6 +113,50 @@ var ( queryDeleteCounter = psql.Delete(tableRelationshipCounter) ) +func (rwt *crdbReadWriteTXN) insertQuery() sq.InsertBuilder { + return psql.Insert(rwt.tupleTableName) +} + +func (rwt *crdbReadWriteTXN) queryDeleteTuples() sq.DeleteBuilder { + return psql.Delete(rwt.tupleTableName) +} + +func (rwt *crdbReadWriteTXN) queryWriteTuple() sq.InsertBuilder { + if rwt.withIntegrity { + return rwt.insertQuery().Columns( + colNamespace, + colObjectID, + colRelation, + colUsersetNamespace, + colUsersetObjectID, + colUsersetRelation, + colCaveatContextName, + colCaveatContext, + colIntegrityKeyID, + colIntegrityHash, + ) + } + + return rwt.insertQuery().Columns( + colNamespace, + colObjectID, + colRelation, + colUsersetNamespace, + colUsersetObjectID, + colUsersetRelation, + colCaveatContextName, + colCaveatContext, + ) +} + +func (rwt *crdbReadWriteTXN) queryTouchTuple() sq.InsertBuilder { + if rwt.withIntegrity { + return rwt.queryWriteTuple().Suffix(upsertTupleSuffixWithIntegrity) + } + + return rwt.queryWriteTuple().Suffix(upsertTupleSuffixWithoutIntegrity) +} + func (rwt *crdbReadWriteTXN) RegisterCounter(ctx context.Context, name string, filter *core.RelationshipFilter) error { counters, err := rwt.lookupCounters(ctx, name) if err != nil { @@ -201,13 +254,13 @@ func (rwt *crdbReadWriteTXN) StoreCounterValue(ctx context.Context, name string, } func (rwt *crdbReadWriteTXN) WriteRelationships(ctx context.Context, mutations []*core.RelationTupleUpdate) error { - bulkWrite := queryWriteTuple + bulkWrite := rwt.queryWriteTuple() var bulkWriteCount int64 - bulkTouch := queryTouchTuple + bulkTouch := rwt.queryTouchTuple() var bulkTouchCount int64 - bulkDelete := queryDeleteTuples + bulkDelete := rwt.queryDeleteTuples() bulkDeleteOr := sq.Or{} var bulkDeleteCount int64 @@ -222,36 +275,49 @@ func (rwt *crdbReadWriteTXN) WriteRelationships(ctx context.Context, mutations [ caveatContext = rel.Caveat.Context.AsMap() } + var integrityKeyID *string + var integrityHash []byte + + if rel.Integrity != nil { + if !rwt.withIntegrity { + return spiceerrors.MustBugf("attempted to write a relationship with integrity, but the datastore does not support integrity") + } + + integrityKeyID = &rel.Integrity.KeyId + integrityHash = rel.Integrity.Hash + } else if rwt.withIntegrity { + return spiceerrors.MustBugf("attempted to write a relationship without integrity, but the datastore requires integrity") + } + + values := []any{ + rel.ResourceAndRelation.Namespace, + rel.ResourceAndRelation.ObjectId, + rel.ResourceAndRelation.Relation, + rel.Subject.Namespace, + rel.Subject.ObjectId, + rel.Subject.Relation, + caveatName, + caveatContext, + } + + if rwt.withIntegrity { + values = append(values, integrityKeyID, integrityHash) + } + rwt.addOverlapKey(rel.ResourceAndRelation.Namespace) rwt.addOverlapKey(rel.Subject.Namespace) switch mutation.Operation { case core.RelationTupleUpdate_TOUCH: rwt.relCountChange++ - bulkTouch = bulkTouch.Values( - rel.ResourceAndRelation.Namespace, - rel.ResourceAndRelation.ObjectId, - rel.ResourceAndRelation.Relation, - rel.Subject.Namespace, - rel.Subject.ObjectId, - rel.Subject.Relation, - caveatName, - caveatContext, - ) + bulkTouch = bulkTouch.Values(values...) bulkTouchCount++ + case core.RelationTupleUpdate_CREATE: rwt.relCountChange++ - bulkWrite = bulkWrite.Values( - rel.ResourceAndRelation.Namespace, - rel.ResourceAndRelation.ObjectId, - rel.ResourceAndRelation.Relation, - rel.Subject.Namespace, - rel.Subject.ObjectId, - rel.Subject.Relation, - caveatName, - caveatContext, - ) + bulkWrite = bulkWrite.Values(values...) bulkWriteCount++ + case core.RelationTupleUpdate_DELETE: rwt.relCountChange-- bulkDeleteOr = append(bulkDeleteOr, exactRelationshipClause(rel)) @@ -310,7 +376,7 @@ func exactRelationshipClause(r *core.RelationTuple) sq.Eq { func (rwt *crdbReadWriteTXN) DeleteRelationships(ctx context.Context, filter *v1.RelationshipFilter, opts ...options.DeleteOptionsOption) (bool, error) { // Add clauses for the ResourceFilter - query := queryDeleteTuples + query := rwt.queryDeleteTuples() if filter.ResourceType != "" { query = query.Where(sq.Eq{colNamespace: filter.ResourceType}) @@ -428,7 +494,7 @@ func (rwt *crdbReadWriteTXN) DeleteNamespaces(ctx context.Context, nsNames ...st return fmt.Errorf(errUnableToDeleteConfig, err) } - deleteTupleSQL, deleteTupleArgs, err := queryDeleteTuples.Where(sq.Or(tplClauses)).ToSql() + deleteTupleSQL, deleteTupleArgs, err := rwt.queryDeleteTuples().Where(sq.Or(tplClauses)).ToSql() if err != nil { return fmt.Errorf(errUnableToDeleteConfig, err) } @@ -455,8 +521,26 @@ var copyCols = []string{ colCaveatContext, } +var copyColsWithIntegrity = []string{ + colNamespace, + colObjectID, + colRelation, + colUsersetNamespace, + colUsersetObjectID, + colUsersetRelation, + colCaveatContextName, + colCaveatContext, + colIntegrityKeyID, + colIntegrityHash, + colTimestamp, +} + func (rwt *crdbReadWriteTXN) BulkLoad(ctx context.Context, iter datastore.BulkWriteRelationshipSource) (uint64, error) { - return pgxcommon.BulkLoad(ctx, rwt.tx, tableTuple, copyCols, iter) + if rwt.withIntegrity { + return pgxcommon.BulkLoad(ctx, rwt.tx, rwt.tupleTableName, copyColsWithIntegrity, iter) + } + + return pgxcommon.BulkLoad(ctx, rwt.tx, rwt.tupleTableName, copyCols, iter) } var _ datastore.ReadWriteTransaction = &crdbReadWriteTXN{} diff --git a/internal/datastore/crdb/stats.go b/internal/datastore/crdb/stats.go index 5da5ed355e..ea3d3e6706 100644 --- a/internal/datastore/crdb/stats.go +++ b/internal/datastore/crdb/stats.go @@ -55,7 +55,7 @@ func (cds *crdbDatastore) Statistics(ctx context.Context) (datastore.Stats, erro if cds.analyzeBeforeStatistics { if err := cds.readPool.BeginTxFunc(ctx, pgx.TxOptions{AccessMode: pgx.ReadOnly}, func(tx pgx.Tx) error { - if _, err := tx.Exec(ctx, "ANALYZE "+tableTuple); err != nil { + if _, err := tx.Exec(ctx, "ANALYZE "+cds.tableTupleName()); err != nil { return fmt.Errorf("unable to analyze tuple table: %w", err) } @@ -125,7 +125,7 @@ func (cds *crdbDatastore) Statistics(ctx context.Context) (datastore.Stats, erro log.Warn().Bool("has-rows", hasRows).Msg("unable to find row count in statistics query result") return nil - }, "SHOW STATISTICS FOR TABLE relation_tuple;"); err != nil { + }, "SHOW STATISTICS FOR TABLE "+cds.tableTupleName()); err != nil { return datastore.Stats{}, fmt.Errorf("unable to query unique estimated row count: %w", err) } diff --git a/internal/datastore/crdb/watch.go b/internal/datastore/crdb/watch.go index 68383d8a55..8ee9125265 100644 --- a/internal/datastore/crdb/watch.go +++ b/internal/datastore/crdb/watch.go @@ -11,6 +11,7 @@ import ( "time" "github.com/prometheus/client_golang/prometheus" + "google.golang.org/protobuf/types/known/timestamppb" "github.com/authzed/spicedb/internal/datastore/common" "github.com/authzed/spicedb/internal/datastore/crdb/pool" @@ -50,6 +51,10 @@ type changeDetails struct { RelationshipCaveatContext map[string]any `json:"caveat_context"` RelationshipCaveatName string `json:"caveat_name"` + + IntegrityKeyID *string `json:"integrity_key_id"` + IntegrityHashAsHex *string `json:"integrity_hash"` + TimestampAsString *string `json:"timestamp"` } } @@ -68,7 +73,7 @@ func (cds *crdbDatastore) Watch(ctx context.Context, afterRevision datastore.Rev return updates, errs } - if !features.Watch.Enabled { + if features.Watch.Status != datastore.FeatureSupported { errs <- datastore.NewWatchDisabledErr(fmt.Sprintf("%s. See https://spicedb.dev/d/enable-watch-api-crdb", features.Watch.Reason)) return updates, errs } @@ -107,7 +112,7 @@ func (cds *crdbDatastore) watch( tableNames := make([]string, 0, 3) if opts.Content&datastore.WatchRelationships == datastore.WatchRelationships { - tableNames = append(tableNames, tableTuple) + tableNames = append(tableNames, cds.tableTupleName()) } if opts.Content&datastore.WatchSchema == datastore.WatchSchema { tableNames = append(tableNames, tableNamespace) @@ -240,7 +245,7 @@ func (cds *crdbDatastore) watch( } switch tableName { - case tableTuple: + case cds.tableTupleName(): var caveatName string var caveatContext map[string]any if details.After != nil && details.After.RelationshipCaveatName != "" { @@ -253,6 +258,30 @@ func (cds *crdbDatastore) watch( return } + var integrity *core.RelationshipIntegrity + + if details.After != nil && details.After.IntegrityKeyID != nil && details.After.IntegrityHashAsHex != nil && details.After.TimestampAsString != nil { + hexString := *details.After.IntegrityHashAsHex + hashBytes, err := hex.DecodeString(hexString[2:]) // drop the \x + if err != nil { + sendError(fmt.Errorf("could not decode hash bytes: %w", err)) + return + } + + timestampString := *details.After.TimestampAsString + parsedTime, err := time.Parse("2006-01-02T15:04:05.999999999", timestampString) + if err != nil { + sendError(fmt.Errorf("could not parse timestamp: %w", err)) + return + } + + integrity = &core.RelationshipIntegrity{ + KeyId: *details.After.IntegrityKeyID, + Hash: hashBytes, + HashedAt: timestamppb.New(parsedTime), + } + } + tuple := &core.RelationTuple{ ResourceAndRelation: &core.ObjectAndRelation{ Namespace: pkValues[0], @@ -264,7 +293,8 @@ func (cds *crdbDatastore) watch( ObjectId: pkValues[4], Relation: pkValues[5], }, - Caveat: ctxCaveat, + Caveat: ctxCaveat, + Integrity: integrity, } rev, err := revisions.HLCRevisionFromString(details.Updated) diff --git a/internal/datastore/memdb/memdb.go b/internal/datastore/memdb/memdb.go index e23a7f3560..30dc13fe0b 100644 --- a/internal/datastore/memdb/memdb.go +++ b/internal/datastore/memdb/memdb.go @@ -133,6 +133,10 @@ func (mdb *memdbDatastore) SnapshotReader(dr datastore.Revision) datastore.Reade return &memdbReader{noopTryLocker{}, txSrc, nil} } +func (mdb *memdbDatastore) SupportsIntegrity() bool { + return true +} + func (mdb *memdbDatastore) ReadWriteTx( ctx context.Context, f datastore.TxUserFunc, @@ -296,8 +300,19 @@ func (mdb *memdbDatastore) ReadyState(_ context.Context) (datastore.ReadyState, }, nil } +func (mdb *memdbDatastore) OfflineFeatures() (*datastore.Features, error) { + return &datastore.Features{ + Watch: datastore.Feature{ + Status: datastore.FeatureSupported, + }, + IntegrityData: datastore.Feature{ + Status: datastore.FeatureSupported, + }, + }, nil +} + func (mdb *memdbDatastore) Features(_ context.Context) (*datastore.Features, error) { - return &datastore.Features{Watch: datastore.Feature{Enabled: true}}, nil + return mdb.OfflineFeatures() } func (mdb *memdbDatastore) Close() error { diff --git a/internal/datastore/memdb/readwrite.go b/internal/datastore/memdb/readwrite.go index 6cabafe21a..5598fea52f 100644 --- a/internal/datastore/memdb/readwrite.go +++ b/internal/datastore/memdb/readwrite.go @@ -34,6 +34,18 @@ func (rwt *memdbReadWriteTx) WriteRelationships(_ context.Context, mutations []* return rwt.write(tx, mutations...) } +func (rwt *memdbReadWriteTx) toIntegrity(mutation *core.RelationTupleUpdate) *relationshipIntegrity { + var ri *relationshipIntegrity + if mutation.Tuple.Integrity != nil { + ri = &relationshipIntegrity{ + keyID: mutation.Tuple.Integrity.KeyId, + hash: mutation.Tuple.Integrity.Hash, + timestamp: mutation.Tuple.Integrity.HashedAt.AsTime(), + } + } + return ri +} + // Caller must already hold the concurrent access lock! func (rwt *memdbReadWriteTx) write(tx *memdb.Txn, mutations ...*core.RelationTupleUpdate) error { // Apply the mutations @@ -46,6 +58,7 @@ func (rwt *memdbReadWriteTx) write(tx *memdb.Txn, mutations ...*core.RelationTup mutation.Tuple.Subject.ObjectId, mutation.Tuple.Subject.Relation, rwt.toCaveatReference(mutation), + rwt.toIntegrity(mutation), } found, err := tx.First( diff --git a/internal/datastore/memdb/schema.go b/internal/datastore/memdb/schema.go index 27b85872dd..4ef504f23c 100644 --- a/internal/datastore/memdb/schema.go +++ b/internal/datastore/memdb/schema.go @@ -1,11 +1,14 @@ package memdb import ( + "time" + v1 "github.com/authzed/authzed-go/proto/authzed/api/v1" "github.com/hashicorp/go-memdb" "github.com/jzelinskie/stringz" "github.com/rs/zerolog" "google.golang.org/protobuf/types/known/structpb" + "google.golang.org/protobuf/types/known/timestamppb" "github.com/authzed/spicedb/pkg/datastore" core "github.com/authzed/spicedb/pkg/proto/core/v1" @@ -51,6 +54,25 @@ type relationship struct { subjectObjectID string subjectRelation string caveat *contextualizedCaveat + integrity *relationshipIntegrity +} + +type relationshipIntegrity struct { + keyID string + hash []byte + timestamp time.Time +} + +func (ri relationshipIntegrity) MarshalZerologObject(e *zerolog.Event) { + e.Str("keyID", ri.keyID).Bytes("hash", ri.hash).Time("timestamp", ri.timestamp) +} + +func (ri relationshipIntegrity) RelationshipIntegrity() *core.RelationshipIntegrity { + return &core.RelationshipIntegrity{ + KeyId: ri.keyID, + Hash: ri.hash, + HashedAt: timestamppb.New(ri.timestamp), + } } type contextualizedCaveat struct { @@ -107,6 +129,12 @@ func (r relationship) RelationTuple() (*core.RelationTuple, error) { if err != nil { return nil, err } + + var ig *core.RelationshipIntegrity + if r.integrity != nil { + ig = r.integrity.RelationshipIntegrity() + } + return &core.RelationTuple{ ResourceAndRelation: &core.ObjectAndRelation{ Namespace: r.namespace, @@ -118,7 +146,8 @@ func (r relationship) RelationTuple() (*core.RelationTuple, error) { ObjectId: r.subjectObjectID, Relation: r.subjectRelation, }, - Caveat: cr, + Caveat: cr, + Integrity: ig, }, nil } diff --git a/internal/datastore/mysql/datastore.go b/internal/datastore/mysql/datastore.go index 93602cb161..ed1fc7749a 100644 --- a/internal/datastore/mysql/datastore.go +++ b/internal/datastore/mysql/datastore.go @@ -563,7 +563,18 @@ func (mds *Datastore) ReadyState(ctx context.Context) (datastore.ReadyState, err } func (mds *Datastore) Features(_ context.Context) (*datastore.Features, error) { - return &datastore.Features{Watch: datastore.Feature{Enabled: true}}, nil + return mds.OfflineFeatures() +} + +func (mds *Datastore) OfflineFeatures() (*datastore.Features, error) { + return &datastore.Features{ + Watch: datastore.Feature{ + Status: datastore.FeatureSupported, + }, + IntegrityData: datastore.Feature{ + Status: datastore.FeatureUnsupported, + }, + }, nil } // isSeeded determines if the backing database has been seeded diff --git a/internal/datastore/postgres/common/bulk.go b/internal/datastore/postgres/common/bulk.go index 51de5a8073..d7c8e8d8fd 100644 --- a/internal/datastore/postgres/common/bulk.go +++ b/internal/datastore/postgres/common/bulk.go @@ -16,6 +16,7 @@ type tupleSourceAdapter struct { current *core.RelationTuple err error valuesBuffer []any + colNames []string } // Next returns true if there is another row and makes the next row data @@ -44,6 +45,12 @@ func (tg *tupleSourceAdapter) Values() ([]any, error) { tg.valuesBuffer[6] = caveatName tg.valuesBuffer[7] = caveatContext + if len(tg.colNames) > 8 && tg.current.Integrity != nil { + tg.valuesBuffer[8] = tg.current.Integrity.KeyId + tg.valuesBuffer[9] = tg.current.Integrity.Hash + tg.valuesBuffer[10] = tg.current.Integrity.HashedAt.AsTime() + } + return tg.valuesBuffer, nil } @@ -63,7 +70,8 @@ func BulkLoad( adapter := &tupleSourceAdapter{ source: iter, ctx: ctx, - valuesBuffer: make([]any, 8), + valuesBuffer: make([]any, len(colNames)), + colNames: colNames, } copied, err := tx.CopyFrom(ctx, pgx.Identifier{tupleTableName}, colNames, adapter) return uint64(copied), err diff --git a/internal/datastore/postgres/common/pgx.go b/internal/datastore/postgres/common/pgx.go index 47f3a4c474..94bdf08c66 100644 --- a/internal/datastore/postgres/common/pgx.go +++ b/internal/datastore/postgres/common/pgx.go @@ -17,6 +17,7 @@ import ( "github.com/rs/zerolog" "go.opentelemetry.io/otel/attribute" "go.opentelemetry.io/otel/trace" + "google.golang.org/protobuf/types/known/timestamppb" "github.com/authzed/spicedb/internal/datastore/common" log "github.com/authzed/spicedb/internal/logging" @@ -29,12 +30,19 @@ const errUnableToQueryTuples = "unable to query tuples: %w" func NewPGXExecutor(querier DBFuncQuerier) common.ExecuteQueryFunc { return func(ctx context.Context, sql string, args []any) ([]*corev1.RelationTuple, error) { span := trace.SpanFromContext(ctx) - return queryTuples(ctx, sql, args, span, querier) + return queryTuples(ctx, sql, args, span, querier, false) + } +} + +func NewPGXExecutorWithIntegrityOption(querier DBFuncQuerier, withIntegrity bool) common.ExecuteQueryFunc { + return func(ctx context.Context, sql string, args []any) ([]*corev1.RelationTuple, error) { + span := trace.SpanFromContext(ctx) + return queryTuples(ctx, sql, args, span, querier, withIntegrity) } } // queryTuples queries tuples for the given query and transaction. -func queryTuples(ctx context.Context, sqlStatement string, args []any, span trace.Span, tx DBFuncQuerier) ([]*corev1.RelationTuple, error) { +func queryTuples(ctx context.Context, sqlStatement string, args []any, span trace.Span, tx DBFuncQuerier, withIntegrity bool) ([]*corev1.RelationTuple, error) { var tuples []*corev1.RelationTuple err := tx.QueryFunc(ctx, func(ctx context.Context, rows pgx.Rows) error { span.AddEvent("Query issued to database") @@ -46,24 +54,53 @@ func queryTuples(ctx context.Context, sqlStatement string, args []any, span trac } var caveatName sql.NullString var caveatCtx map[string]any - err := rows.Scan( - &nextTuple.ResourceAndRelation.Namespace, - &nextTuple.ResourceAndRelation.ObjectId, - &nextTuple.ResourceAndRelation.Relation, - &nextTuple.Subject.Namespace, - &nextTuple.Subject.ObjectId, - &nextTuple.Subject.Relation, - &caveatName, - &caveatCtx, - ) - if err != nil { - return fmt.Errorf(errUnableToQueryTuples, fmt.Errorf("scan err: %w", err)) + + if withIntegrity { + var integrityKeyID string + var integrityHash []byte + var timestamp time.Time + + if err := rows.Scan( + &nextTuple.ResourceAndRelation.Namespace, + &nextTuple.ResourceAndRelation.ObjectId, + &nextTuple.ResourceAndRelation.Relation, + &nextTuple.Subject.Namespace, + &nextTuple.Subject.ObjectId, + &nextTuple.Subject.Relation, + &caveatName, + &caveatCtx, + &integrityKeyID, + &integrityHash, + ×tamp, + ); err != nil { + return fmt.Errorf(errUnableToQueryTuples, fmt.Errorf("scan err: %w", err)) + } + + nextTuple.Integrity = &corev1.RelationshipIntegrity{ + KeyId: integrityKeyID, + Hash: integrityHash, + HashedAt: timestamppb.New(timestamp), + } + } else { + if err := rows.Scan( + &nextTuple.ResourceAndRelation.Namespace, + &nextTuple.ResourceAndRelation.ObjectId, + &nextTuple.ResourceAndRelation.Relation, + &nextTuple.Subject.Namespace, + &nextTuple.Subject.ObjectId, + &nextTuple.Subject.Relation, + &caveatName, + &caveatCtx, + ); err != nil { + return fmt.Errorf(errUnableToQueryTuples, fmt.Errorf("scan err: %w", err)) + } } - nextTuple.Caveat, err = common.ContextualizedCaveatFrom(caveatName.String, caveatCtx) + caveat, err := common.ContextualizedCaveatFrom(caveatName.String, caveatCtx) if err != nil { return fmt.Errorf(errUnableToQueryTuples, fmt.Errorf("unable to fetch caveat context: %w", err)) } + nextTuple.Caveat = caveat tuples = append(tuples, nextTuple) } if err := rows.Err(); err != nil { diff --git a/internal/datastore/postgres/postgres.go b/internal/datastore/postgres/postgres.go index cd73907fb0..717b67f864 100644 --- a/internal/datastore/postgres/postgres.go +++ b/internal/datastore/postgres/postgres.go @@ -674,8 +674,30 @@ func (pgd *pgDatastore) ReadyState(ctx context.Context) (datastore.ReadyState, e }, nil } -func (pgd *pgDatastore) Features(_ context.Context) (*datastore.Features, error) { - return &datastore.Features{Watch: datastore.Feature{Enabled: pgd.watchEnabled}}, nil +func (pgd *pgDatastore) Features(ctx context.Context) (*datastore.Features, error) { + return pgd.OfflineFeatures() +} + +func (pgd *pgDatastore) OfflineFeatures() (*datastore.Features, error) { + if pgd.watchEnabled { + return &datastore.Features{ + Watch: datastore.Feature{ + Status: datastore.FeatureSupported, + }, + IntegrityData: datastore.Feature{ + Status: datastore.FeatureUnsupported, + }, + }, nil + } + + return &datastore.Features{ + Watch: datastore.Feature{ + Status: datastore.FeatureUnsupported, + }, + IntegrityData: datastore.Feature{ + Status: datastore.FeatureUnsupported, + }, + }, nil } func buildLivingObjectFilterForRevision(revision postgresRevision) queryFilterer { diff --git a/internal/datastore/proxy/observable.go b/internal/datastore/proxy/observable.go index 51dd358985..d1a1ebe952 100644 --- a/internal/datastore/proxy/observable.go +++ b/internal/datastore/proxy/observable.go @@ -120,6 +120,10 @@ func (p *observableProxy) Features(ctx context.Context) (*datastore.Features, er return p.delegate.Features(ctx) } +func (p *observableProxy) OfflineFeatures() (*datastore.Features, error) { + return p.delegate.OfflineFeatures() +} + func (p *observableProxy) Statistics(ctx context.Context) (datastore.Stats, error) { ctx, closer := observe(ctx, "Statistics") defer closer() diff --git a/internal/datastore/proxy/proxy_test/mock.go b/internal/datastore/proxy/proxy_test/mock.go index 634028c51e..0b5af2254e 100644 --- a/internal/datastore/proxy/proxy_test/mock.go +++ b/internal/datastore/proxy/proxy_test/mock.go @@ -70,6 +70,11 @@ func (dm *MockDatastore) Features(_ context.Context) (*datastore.Features, error return args.Get(0).(*datastore.Features), args.Error(1) } +func (dm *MockDatastore) OfflineFeatures() (*datastore.Features, error) { + args := dm.Called() + return args.Get(0).(*datastore.Features), args.Error(1) +} + func (dm *MockDatastore) Statistics(_ context.Context) (datastore.Stats, error) { args := dm.Called() return args.Get(0).(datastore.Stats), args.Error(1) diff --git a/internal/datastore/proxy/relationshipintegrity.go b/internal/datastore/proxy/relationshipintegrity.go new file mode 100644 index 0000000000..6a09ab669b --- /dev/null +++ b/internal/datastore/proxy/relationshipintegrity.go @@ -0,0 +1,472 @@ +package proxy + +import ( + "context" + "crypto/hmac" + "crypto/sha256" + "fmt" + "hash" + "sync" + "time" + + "google.golang.org/protobuf/types/known/timestamppb" + + log "github.com/authzed/spicedb/internal/logging" + "github.com/authzed/spicedb/pkg/datastore" + "github.com/authzed/spicedb/pkg/datastore/options" + corev1 "github.com/authzed/spicedb/pkg/proto/core/v1" + "github.com/authzed/spicedb/pkg/spiceerrors" + "github.com/authzed/spicedb/pkg/tuple" +) + +// KeyConfig is a configuration for a key used to sign relationships. +type KeyConfig struct { + // ID is the unique identifier for the key. + ID string + + // ExpiredAt is the time at which the key is no longer valid, if any. + ExpiredAt *time.Time + + // Bytes is the raw key material. + Bytes []byte +} + +type hmacConfig struct { + keyID string + expiredAt *time.Time + pool sync.Pool +} + +var ( + versionByte = byte(0x01) + hashLength = 16 +) + +// NewRelationshipIntegrityProxy creates a new datastore proxy that ensures the integrity of +// relationships by using HMACs to sign the data. The current key is used to sign new data, +// and the expired keys are used to verify old data, if any. +func NewRelationshipIntegrityProxy(ds datastore.Datastore, currentKey KeyConfig, expiredKeys []KeyConfig) (datastore.Datastore, error) { + // Ensure the datastore supports integrity. + features, err := ds.OfflineFeatures() + if err != nil { + return nil, err + } + + if features.IntegrityData.Status != datastore.FeatureSupported { + return nil, spiceerrors.MustBugf("datastore does not support relationship integrity") + } + + if len(currentKey.Bytes) == 0 { + return nil, fmt.Errorf("contents of the current key file cannot be empty") + } + + if len(currentKey.ID) == 0 { + return nil, fmt.Errorf("current key ID cannot be empty") + } + + currentKeyHMAC := &hmacConfig{ + keyID: currentKey.ID, + expiredAt: currentKey.ExpiredAt, + pool: poolForKey(currentKey.Bytes), + } + + if currentKey.ExpiredAt != nil { + return nil, spiceerrors.MustBugf("current key cannot have an expiration") + } + + keysByID := make(map[string]*hmacConfig, len(expiredKeys)+1) + keysByID[currentKey.ID] = currentKeyHMAC + + expiredKeyIDs := make([]string, 0, len(expiredKeys)) + for _, key := range expiredKeys { + if len(key.Bytes) == 0 { + return nil, fmt.Errorf("expired key cannot be empty") + } + + if len(key.ID) == 0 { + return nil, fmt.Errorf("expired key ID cannot be empty") + } + + if key.ExpiredAt == nil { + return nil, fmt.Errorf("expired key missing expiration time") + } + + if _, ok := keysByID[key.ID]; ok { + return nil, fmt.Errorf("found duplicate key ID: %s", key.ID) + } + + keysByID[key.ID] = &hmacConfig{ + keyID: key.ID, + expiredAt: key.ExpiredAt, + pool: poolForKey(key.Bytes), + } + + expiredKeyIDs = append(expiredKeyIDs, key.ID) + } + + log.Debug(). + Str("current_key_id", currentKey.ID). + Strs("expired_key_ids", expiredKeyIDs). + Msgf("created relationship integrity proxy") + + return &relationshipIntegrityProxy{ + ds: ds, + primaryKey: currentKeyHMAC, + keysByID: keysByID, + }, nil +} + +func poolForKey(key []byte) sync.Pool { + return sync.Pool{ + New: func() any { + return hmac.New(sha256.New, key) + }, + } +} + +type relationshipIntegrityProxy struct { + ds datastore.Datastore + primaryKey *hmacConfig + keysByID map[string]*hmacConfig +} + +func (r *relationshipIntegrityProxy) lookupKey(keyID string) (*hmacConfig, error) { + key, ok := r.keysByID[keyID] + if !ok { + return nil, fmt.Errorf("key not found: %s", keyID) + } + + return key, nil +} + +// computeRelationshipHash computes the HMAC hash of a relationship tuple. +func computeRelationshipHash(tpl *corev1.RelationTuple, key *hmacConfig) ([]byte, error) { + bytes, err := tuple.CanonicalBytes(tpl) + if err != nil { + return nil, err + } + + hasher := key.pool.Get().(hash.Hash) + defer key.pool.Put(hasher) + + hasher.Reset() + if _, err := hasher.Write(bytes); err != nil { + return nil, err + } + + return hasher.Sum(nil)[:hashLength], nil +} + +func (r *relationshipIntegrityProxy) SnapshotReader(rev datastore.Revision) datastore.Reader { + return relationshipIntegrityReader{ + parent: r, + wrapped: r.ds.SnapshotReader(rev), + } +} + +func (r *relationshipIntegrityProxy) ReadWriteTx(ctx context.Context, f datastore.TxUserFunc, opts ...options.RWTOptionsOption) (datastore.Revision, error) { + return r.ds.ReadWriteTx(ctx, func(ctx context.Context, tx datastore.ReadWriteTransaction) error { + return f(ctx, &relationshipIntegrityTx{ + ReadWriteTransaction: tx, + parent: r, + }) + }, opts...) +} + +func (r *relationshipIntegrityProxy) CheckRevision(ctx context.Context, revision datastore.Revision) error { + return r.ds.CheckRevision(ctx, revision) +} + +func (r *relationshipIntegrityProxy) Close() error { + return r.ds.Close() +} + +func (r *relationshipIntegrityProxy) Features(ctx context.Context) (*datastore.Features, error) { + return r.ds.Features(ctx) +} + +func (r *relationshipIntegrityProxy) OfflineFeatures() (*datastore.Features, error) { + return r.ds.OfflineFeatures() +} + +func (r *relationshipIntegrityProxy) HeadRevision(ctx context.Context) (datastore.Revision, error) { + return r.ds.HeadRevision(ctx) +} + +func (r *relationshipIntegrityProxy) OptimizedRevision(ctx context.Context) (datastore.Revision, error) { + return r.ds.OptimizedRevision(ctx) +} + +func (r *relationshipIntegrityProxy) ReadyState(ctx context.Context) (datastore.ReadyState, error) { + return r.ds.ReadyState(ctx) +} + +func (r *relationshipIntegrityProxy) RevisionFromString(serialized string) (datastore.Revision, error) { + return r.ds.RevisionFromString(serialized) +} + +func (r *relationshipIntegrityProxy) Statistics(ctx context.Context) (datastore.Stats, error) { + return r.ds.Statistics(ctx) +} + +func (r *relationshipIntegrityProxy) validateRelationTuple(tpl *corev1.RelationTuple) error { + // Ensure the relationship has integrity data. + if tpl.Integrity == nil || len(tpl.Integrity.Hash) == 0 || tpl.Integrity.KeyId == "" { + str, err := tuple.String(tpl) + if err != nil { + return err + } + + return fmt.Errorf("relationship %s is missing required integrity data", str) + } + + hashWithoutByte := tpl.Integrity.Hash[1:] + if tpl.Integrity.Hash[0] != versionByte || len(hashWithoutByte) != hashLength { + return fmt.Errorf("relationship %s has invalid integrity data", tpl) + } + + // Validate the integrity of the relationship. + key, err := r.lookupKey(tpl.Integrity.KeyId) + if err != nil { + return err + } + + if key.expiredAt != nil && key.expiredAt.Before(tpl.Integrity.HashedAt.AsTime()) { + return fmt.Errorf("relationship %s is signed by an expired key", tpl) + } + + computedHash, err := computeRelationshipHash(tpl, key) + if err != nil { + return err + } + + if !hmac.Equal(computedHash, hashWithoutByte) { + str, err := tuple.String(tpl) + if err != nil { + return err + } + + return fmt.Errorf("relationship %s has invalid integrity hash", str) + } + + // NOTE: The caller expects the integrity to be nil, so the proxy sets it to nil here. + tpl.Integrity = nil + return nil +} + +func (r *relationshipIntegrityProxy) Watch(ctx context.Context, afterRevision datastore.Revision, options datastore.WatchOptions) (<-chan *datastore.RevisionChanges, <-chan error) { + resultsChan, errChan := r.ds.Watch(ctx, afterRevision, options) + checkedResultsChan := make(chan *datastore.RevisionChanges) + checkedErrChan := make(chan error, 1) + + go func() { + defer close(checkedResultsChan) + defer close(checkedErrChan) + + for { + select { + case result := <-resultsChan: + for _, rel := range result.RelationshipChanges { + if rel.Operation != corev1.RelationTupleUpdate_DELETE { + err := r.validateRelationTuple(rel.Tuple) + if err != nil { + checkedErrChan <- err + return + } + } + } + checkedResultsChan <- result + + case err := <-errChan: + checkedErrChan <- err + return + } + } + }() + + return checkedResultsChan, checkedErrChan +} + +func (r *relationshipIntegrityProxy) Unwrap() datastore.Datastore { + return r.ds +} + +type relationshipIntegrityReader struct { + parent *relationshipIntegrityProxy + wrapped datastore.Reader +} + +func (r relationshipIntegrityReader) QueryRelationships(ctx context.Context, filter datastore.RelationshipsFilter, options ...options.QueryOptionsOption) (datastore.RelationshipIterator, error) { + it, err := r.wrapped.QueryRelationships(ctx, filter, options...) + if err != nil { + return nil, err + } + + return &relationshipIntegrityIterator{ + parent: r, + wrapped: it, + }, nil +} + +func (r relationshipIntegrityReader) ReverseQueryRelationships(ctx context.Context, subjectsFilter datastore.SubjectsFilter, options ...options.ReverseQueryOptionsOption) (datastore.RelationshipIterator, error) { + it, err := r.wrapped.ReverseQueryRelationships(ctx, subjectsFilter, options...) + if err != nil { + return nil, err + } + + return &relationshipIntegrityIterator{ + parent: r, + wrapped: it, + }, nil +} + +func (r relationshipIntegrityReader) CountRelationships(ctx context.Context, name string) (int, error) { + return r.wrapped.CountRelationships(ctx, name) +} + +func (r relationshipIntegrityReader) ListAllCaveats(ctx context.Context) ([]datastore.RevisionedDefinition[*corev1.CaveatDefinition], error) { + return r.wrapped.ListAllCaveats(ctx) +} + +func (r relationshipIntegrityReader) ListAllNamespaces(ctx context.Context) ([]datastore.RevisionedDefinition[*corev1.NamespaceDefinition], error) { + return r.wrapped.ListAllNamespaces(ctx) +} + +func (r relationshipIntegrityReader) LookupCaveatsWithNames(ctx context.Context, names []string) ([]datastore.RevisionedDefinition[*corev1.CaveatDefinition], error) { + return r.wrapped.LookupCaveatsWithNames(ctx, names) +} + +func (r relationshipIntegrityReader) LookupCounters(ctx context.Context) ([]datastore.RelationshipCounter, error) { + return r.wrapped.LookupCounters(ctx) +} + +func (r relationshipIntegrityReader) LookupNamespacesWithNames(ctx context.Context, nsNames []string) ([]datastore.RevisionedDefinition[*corev1.NamespaceDefinition], error) { + return r.wrapped.LookupNamespacesWithNames(ctx, nsNames) +} + +func (r relationshipIntegrityReader) ReadCaveatByName(ctx context.Context, name string) (caveat *corev1.CaveatDefinition, lastWritten datastore.Revision, err error) { + return r.wrapped.ReadCaveatByName(ctx, name) +} + +func (r relationshipIntegrityReader) ReadNamespaceByName(ctx context.Context, nsName string) (ns *corev1.NamespaceDefinition, lastWritten datastore.Revision, err error) { + return r.wrapped.ReadNamespaceByName(ctx, nsName) +} + +type relationshipIntegrityIterator struct { + parent relationshipIntegrityReader + wrapped datastore.RelationshipIterator + err error +} + +func (r *relationshipIntegrityIterator) Close() { + r.wrapped.Close() +} + +func (r *relationshipIntegrityIterator) Cursor() (options.Cursor, error) { + return r.wrapped.Cursor() +} + +func (r *relationshipIntegrityIterator) Err() error { + if r.err != nil { + return r.err + } + + return r.wrapped.Err() +} + +func (r *relationshipIntegrityIterator) Next() *corev1.RelationTuple { + tpl := r.wrapped.Next() + if tpl == nil { + return nil + } + + err := r.parent.parent.validateRelationTuple(tpl) + if err != nil { + r.err = err + return nil + } + + return tpl +} + +type relationshipIntegrityTx struct { + datastore.ReadWriteTransaction + + parent *relationshipIntegrityProxy +} + +func (r *relationshipIntegrityTx) WriteRelationships( + ctx context.Context, + mutations []*corev1.RelationTupleUpdate, +) error { + // Add integrity data to the relationships. + key := r.parent.primaryKey + hashedAt := timestamppb.Now() + + updated := make([]*corev1.RelationTupleUpdate, 0, len(mutations)) + for _, mutation := range mutations { + if mutation.Tuple.Integrity != nil { + return spiceerrors.MustBugf("relationship %s already has integrity data", mutation.Tuple) + } + + hash, err := computeRelationshipHash(mutation.Tuple, key) + if err != nil { + return err + } + + // NOTE: Callers expect to be able to reuse the tuple, so we need to clone it. + cloned := mutation.CloneVT() + cloned.Tuple.Integrity = &corev1.RelationshipIntegrity{ + HashedAt: hashedAt, + Hash: append([]byte{versionByte}, hash...), + KeyId: key.keyID, + } + updated = append(updated, cloned) + } + + return r.ReadWriteTransaction.WriteRelationships(ctx, updated) +} + +func (r *relationshipIntegrityTx) BulkLoad( + ctx context.Context, + iter datastore.BulkWriteRelationshipSource, +) (uint64, error) { + wrapped := &integrityAddingBulkLoadInterator{iter, r.parent} + return r.ReadWriteTransaction.BulkLoad(ctx, wrapped) +} + +type integrityAddingBulkLoadInterator struct { + wrapped datastore.BulkWriteRelationshipSource + parent *relationshipIntegrityProxy +} + +func (w integrityAddingBulkLoadInterator) Next(ctx context.Context) (*corev1.RelationTuple, error) { + tpl, err := w.wrapped.Next(ctx) + if err != nil { + return nil, err + } + + if tpl == nil { + return nil, nil + } + + key := w.parent.primaryKey + hashedAt := timestamppb.Now() + + hash, err := computeRelationshipHash(tpl, key) + if err != nil { + return nil, err + } + + if tpl.Integrity != nil { + return nil, spiceerrors.MustBugf("relationship %s already has integrity data", tpl) + } + + tpl.Integrity = &corev1.RelationshipIntegrity{ + HashedAt: hashedAt, + Hash: append([]byte{versionByte}, hash...), + KeyId: key.keyID, + } + + return tpl, nil +} diff --git a/internal/datastore/proxy/relationshipintegrity_test.go b/internal/datastore/proxy/relationshipintegrity_test.go new file mode 100644 index 0000000000..603b9ad98c --- /dev/null +++ b/internal/datastore/proxy/relationshipintegrity_test.go @@ -0,0 +1,416 @@ +package proxy + +import ( + "context" + "encoding/hex" + "fmt" + "testing" + "time" + + "github.com/stretchr/testify/require" + "google.golang.org/protobuf/types/known/timestamppb" + + "github.com/authzed/spicedb/internal/datastore/memdb" + "github.com/authzed/spicedb/pkg/datastore" + core "github.com/authzed/spicedb/pkg/proto/core/v1" + "github.com/authzed/spicedb/pkg/tuple" +) + +var DefaultKeyForTesting = KeyConfig{ + ID: "defaultfortest", + Bytes: (func() []byte { + b, err := hex.DecodeString("000102030405060708090A0B0C0D0E0FF0E0D0C0B0A090807060504030201000") + if err != nil { + panic(err) + } + return b + })(), + ExpiredAt: nil, +} + +var toBeExpiredKeyForTesting = KeyConfig{ + ID: "expiredkeyfortest", + Bytes: (func() []byte { + b, err := hex.DecodeString("000102030405060708090A0B0C0D0E0FF0E0D0C0B0A090807060504030201222") + if err != nil { + panic(err) + } + return b + })(), +} + +var expiredKeyForTesting = KeyConfig{ + ID: "expiredkeyfortest", + Bytes: (func() []byte { + b, err := hex.DecodeString("000102030405060708090A0B0C0D0E0FF0E0D0C0B0A090807060504030201222") + if err != nil { + panic(err) + } + return b + })(), + ExpiredAt: (func() *time.Time { + t, err := time.Parse("2006-01-02", "2021-01-01") + if err != nil { + panic(err) + } + return &t + })(), +} + +func TestWriteWithPredefinedIntegrity(t *testing.T) { + ds, err := memdb.NewMemdbDatastore(0, 5*time.Second, 1*time.Hour) + require.NoError(t, err) + + pds, err := NewRelationshipIntegrityProxy(ds, DefaultKeyForTesting, nil) + require.NoError(t, err) + + require.Panics(t, func() { + _, _ = pds.ReadWriteTx(context.Background(), func(ctx context.Context, tx datastore.ReadWriteTransaction) error { + tpl := tuple.MustParse("resource:foo#viewer@user:tom") + tpl.Integrity = &core.RelationshipIntegrity{} + return tx.WriteRelationships(context.Background(), []*core.RelationTupleUpdate{ + tuple.Create(tpl), + }) + }) + }) +} + +func TestReadWithMissingIntegrity(t *testing.T) { + ds, err := memdb.NewMemdbDatastore(0, 5*time.Second, 1*time.Hour) + require.NoError(t, err) + + // Write a relationship to the underlying datastore without integrity information. + _, err = ds.ReadWriteTx(context.Background(), func(ctx context.Context, tx datastore.ReadWriteTransaction) error { + tpl := tuple.MustParse("resource:foo#viewer@user:tom") + return tx.WriteRelationships(context.Background(), []*core.RelationTupleUpdate{ + tuple.Create(tpl), + }) + }) + require.NoError(t, err) + + // Attempt to read, which should return an error. + pds, err := NewRelationshipIntegrityProxy(ds, DefaultKeyForTesting, nil) + require.NoError(t, err) + + headRev, err := pds.HeadRevision(context.Background()) + require.NoError(t, err) + + reader := pds.SnapshotReader(headRev) + iter, err := reader.QueryRelationships( + context.Background(), + datastore.RelationshipsFilter{OptionalResourceType: "resource"}, + ) + require.NoError(t, err) + + found := iter.Next() + require.Nil(t, found) + require.Error(t, iter.Err()) + require.ErrorContains(t, iter.Err(), "is missing required integrity data") + + iter.Close() +} + +func TestBasicIntegrityFailureDueToInvalidHashVersion(t *testing.T) { + ds, err := memdb.NewMemdbDatastore(0, 5*time.Second, 1*time.Hour) + require.NoError(t, err) + + pds, err := NewRelationshipIntegrityProxy(ds, DefaultKeyForTesting, nil) + require.NoError(t, err) + + // Write some relationships. + _, err = pds.ReadWriteTx(context.Background(), func(ctx context.Context, tx datastore.ReadWriteTransaction) error { + return tx.WriteRelationships(context.Background(), []*core.RelationTupleUpdate{ + tuple.Create(tuple.MustParse("resource:foo#viewer@user:tom")), + tuple.Create(tuple.MustParse("resource:foo#viewer@user:fred")), + tuple.Touch(tuple.MustParse("resource:bar#viewer@user:sarah")), + }) + }) + require.NoError(t, err) + + // Insert an invalid integrity hash for one of the relationships to be invalid by bypassing + // the proxy. + _, err = ds.ReadWriteTx(context.Background(), func(ctx context.Context, tx datastore.ReadWriteTransaction) error { + invalidTpl := tuple.MustParse("resource:foo#viewer@user:jimmy") + invalidTpl.Integrity = &core.RelationshipIntegrity{ + KeyId: "defaultfortest", + Hash: []byte("invalidhash"), + HashedAt: timestamppb.Now(), + } + + return tx.WriteRelationships(context.Background(), []*core.RelationTupleUpdate{ + tuple.Create(invalidTpl), + }) + }) + require.NoError(t, err) + + // Read them back and ensure the read fails. + headRev, err := pds.HeadRevision(context.Background()) + require.NoError(t, err) + + reader := pds.SnapshotReader(headRev) + iter, err := reader.QueryRelationships( + context.Background(), + datastore.RelationshipsFilter{OptionalResourceType: "resource"}, + ) + t.Cleanup(iter.Close) + require.NoError(t, err) + + var foundError error + for { + rel := iter.Next() + if rel == nil { + break + } + + err := iter.Err() + if err != nil { + foundError = err + break + } + } + + if foundError == nil { + foundError = iter.Err() + } + + require.Error(t, foundError) + require.ErrorContains(t, foundError, "has invalid integrity data") +} + +func TestBasicIntegrityFailureDueToInvalidHashSignature(t *testing.T) { + ds, err := memdb.NewMemdbDatastore(0, 5*time.Second, 1*time.Hour) + require.NoError(t, err) + + pds, err := NewRelationshipIntegrityProxy(ds, DefaultKeyForTesting, nil) + require.NoError(t, err) + + // Write some relationships. + _, err = pds.ReadWriteTx(context.Background(), func(ctx context.Context, tx datastore.ReadWriteTransaction) error { + return tx.WriteRelationships(context.Background(), []*core.RelationTupleUpdate{ + tuple.Create(tuple.MustParse("resource:foo#viewer@user:tom")), + tuple.Create(tuple.MustParse("resource:foo#viewer@user:fred")), + tuple.Touch(tuple.MustParse("resource:bar#viewer@user:sarah")), + }) + }) + require.NoError(t, err) + + // Insert an invalid integrity hash for one of the relationships to be invalid by bypassing + // the + _, err = ds.ReadWriteTx(context.Background(), func(ctx context.Context, tx datastore.ReadWriteTransaction) error { + invalidTpl := tuple.MustParse("resource:foo#viewer@user:jimmy") + invalidTpl.Integrity = &core.RelationshipIntegrity{ + KeyId: "defaultfortest", + Hash: append([]byte{0x01}, []byte("someinvalidhashaasd")[0:hashLength]...), + HashedAt: timestamppb.Now(), + } + + return tx.WriteRelationships(context.Background(), []*core.RelationTupleUpdate{ + tuple.Create(invalidTpl), + }) + }) + require.NoError(t, err) + + // Read them back and ensure the read fails. + headRev, err := pds.HeadRevision(context.Background()) + require.NoError(t, err) + + reader := pds.SnapshotReader(headRev) + iter, err := reader.QueryRelationships( + context.Background(), + datastore.RelationshipsFilter{OptionalResourceType: "resource"}, + ) + t.Cleanup(iter.Close) + require.NoError(t, err) + + var foundError error + for { + rel := iter.Next() + if rel == nil { + break + } + + err := iter.Err() + if err != nil { + foundError = err + break + } + } + + if foundError == nil { + foundError = iter.Err() + } + + require.Error(t, foundError) + require.ErrorContains(t, foundError, "has invalid integrity hash") +} + +func TestBasicIntegrityFailureDueToWriteWithExpiredKey(t *testing.T) { + ds, err := memdb.NewMemdbDatastore(0, 5*time.Second, 1*time.Hour) + require.NoError(t, err) + + // Create a proxy with the to-be-expired key and write some relationships. + epds, err := NewRelationshipIntegrityProxy(ds, toBeExpiredKeyForTesting, nil) + require.NoError(t, err) + + // Write some relationships. + _, err = epds.ReadWriteTx(context.Background(), func(ctx context.Context, tx datastore.ReadWriteTransaction) error { + return tx.WriteRelationships(context.Background(), []*core.RelationTupleUpdate{ + tuple.Create(tuple.MustParse("resource:foo#viewer@user:tom")), + tuple.Create(tuple.MustParse("resource:foo#viewer@user:fred")), + tuple.Touch(tuple.MustParse("resource:bar#viewer@user:sarah")), + }) + }) + require.NoError(t, err) + + pds, err := NewRelationshipIntegrityProxy(ds, DefaultKeyForTesting, []KeyConfig{ + expiredKeyForTesting, + }) + require.NoError(t, err) + + // Read them back and ensure the read fails. + headRev, err := pds.HeadRevision(context.Background()) + require.NoError(t, err) + + reader := pds.SnapshotReader(headRev) + iter, err := reader.QueryRelationships( + context.Background(), + datastore.RelationshipsFilter{OptionalResourceType: "resource"}, + ) + t.Cleanup(iter.Close) + require.NoError(t, err) + + var foundError error + for { + rel := iter.Next() + if rel == nil { + break + } + + err := iter.Err() + if err != nil { + foundError = err + break + } + } + + if foundError == nil { + foundError = iter.Err() + } + + require.Error(t, foundError) + require.ErrorContains(t, foundError, "is signed by an expired key") +} + +func TestWatchIntegrityFailureDueToInvalidHashSignature(t *testing.T) { + ds, err := memdb.NewMemdbDatastore(0, 5*time.Second, 1*time.Hour) + require.NoError(t, err) + + headRev, err := ds.HeadRevision(context.Background()) + require.NoError(t, err) + + pds, err := NewRelationshipIntegrityProxy(ds, DefaultKeyForTesting, nil) + require.NoError(t, err) + + watchEvents, errChan := pds.Watch(context.Background(), headRev, datastore.WatchJustRelationships()) + + // Insert an invalid integrity hash for one of the relationships to be invalid by bypassing + // the proxy. + _, err = ds.ReadWriteTx(context.Background(), func(ctx context.Context, tx datastore.ReadWriteTransaction) error { + invalidTpl := tuple.MustParse("resource:foo#viewer@user:jimmy") + invalidTpl.Integrity = &core.RelationshipIntegrity{ + KeyId: "defaultfortest", + Hash: append([]byte{0x01}, []byte("someinvalidhashaasd")[0:hashLength]...), + HashedAt: timestamppb.Now(), + } + + return tx.WriteRelationships(context.Background(), []*core.RelationTupleUpdate{ + tuple.Create(invalidTpl), + }) + }) + require.NoError(t, err) + + // Ensure a watch error is raised. + select { + case <-watchEvents: + require.Fail(t, "watch event received") + + case err := <-errChan: + require.Error(t, err) + require.ErrorContains(t, err, "has invalid integrity hash") + + case <-time.After(5 * time.Second): + require.Fail(t, "timeout waiting for watch error") + } +} + +func BenchmarkQueryRelsWithIntegrity(b *testing.B) { + for _, withIntegrity := range []bool{true, false} { + b.Run(fmt.Sprintf("withIntegrity=%t", withIntegrity), func(b *testing.B) { + ds, err := memdb.NewMemdbDatastore(0, 5*time.Second, 1*time.Hour) + require.NoError(b, err) + + pds, err := NewRelationshipIntegrityProxy(ds, DefaultKeyForTesting, nil) + require.NoError(b, err) + + _, err = pds.ReadWriteTx(context.Background(), func(ctx context.Context, tx datastore.ReadWriteTransaction) error { + for i := 0; i < 1000; i++ { + tpl := tuple.MustParse(fmt.Sprintf("resource:foo#viewer@user:user-%d", i)) + if err := tx.WriteRelationships(context.Background(), []*core.RelationTupleUpdate{ + tuple.Create(tpl), + }); err != nil { + return err + } + } + + return nil + }) + require.NoError(b, err) + + headRev, err := pds.HeadRevision(context.Background()) + require.NoError(b, err) + + b.ResetTimer() + for i := 0; i < b.N; i++ { + var reader datastore.Reader + if withIntegrity { + reader = pds.SnapshotReader(headRev) + } else { + reader = ds.SnapshotReader(headRev) + } + iter, err := reader.QueryRelationships( + context.Background(), + datastore.RelationshipsFilter{OptionalResourceType: "resource"}, + ) + require.NoError(b, err) + + for { + err := iter.Err() + if err != nil { + require.NoError(b, err) + } + + rel := iter.Next() + if rel == nil { + break + } + } + + iter.Close() + } + b.StopTimer() + }) + } +} + +func BenchmarkComputeRelationshipHash(b *testing.B) { + config := &hmacConfig{ + keyID: "defaultfortest", + pool: poolForKey(DefaultKeyForTesting.Bytes), + } + + tpl := tuple.MustParse("resource:foo#viewer@user:tom") + for i := 0; i < b.N; i++ { + _, err := computeRelationshipHash(tpl, config) + require.NoError(b, err) + } +} diff --git a/internal/datastore/proxy/replicated_test.go b/internal/datastore/proxy/replicated_test.go index 39a23c90ab..e882998e9d 100644 --- a/internal/datastore/proxy/replicated_test.go +++ b/internal/datastore/proxy/replicated_test.go @@ -134,6 +134,10 @@ func (f fakeDatastore) Features(_ context.Context) (*datastore.Features, error) return nil, nil } +func (f fakeDatastore) OfflineFeatures() (*datastore.Features, error) { + return nil, nil +} + func (f fakeDatastore) Statistics(_ context.Context) (datastore.Stats, error) { return datastore.Stats{}, nil } diff --git a/internal/datastore/proxy/schemacaching/watchingcache_test.go b/internal/datastore/proxy/schemacaching/watchingcache_test.go index 5ed4333901..0e8ee2fc65 100644 --- a/internal/datastore/proxy/schemacaching/watchingcache_test.go +++ b/internal/datastore/proxy/schemacaching/watchingcache_test.go @@ -504,6 +504,10 @@ func (*fakeDatastore) Features(context.Context) (*datastore.Features, error) { return nil, fmt.Errorf("not implemented") } +func (*fakeDatastore) OfflineFeatures() (*datastore.Features, error) { + return nil, fmt.Errorf("not implemented") +} + func (*fakeDatastore) OptimizedRevision(context.Context) (datastore.Revision, error) { return nil, fmt.Errorf("not implemented") } diff --git a/internal/datastore/proxy/singleflight.go b/internal/datastore/proxy/singleflight.go index 70dc348f72..ee02f32e1d 100644 --- a/internal/datastore/proxy/singleflight.go +++ b/internal/datastore/proxy/singleflight.go @@ -71,6 +71,10 @@ func (p *singleflightProxy) Features(ctx context.Context) (*datastore.Features, return p.delegate.Features(ctx) } +func (p *singleflightProxy) OfflineFeatures() (*datastore.Features, error) { + return p.delegate.OfflineFeatures() +} + func (p *singleflightProxy) ReadyState(ctx context.Context) (datastore.ReadyState, error) { return p.delegate.ReadyState(ctx) } diff --git a/internal/datastore/spanner/spanner.go b/internal/datastore/spanner/spanner.go index dd85bc4650..d9bfcbf360 100644 --- a/internal/datastore/spanner/spanner.go +++ b/internal/datastore/spanner/spanner.go @@ -308,8 +308,19 @@ func (sd *spannerDatastore) ReadyState(ctx context.Context) (datastore.ReadyStat }, nil } -func (sd *spannerDatastore) Features(_ context.Context) (*datastore.Features, error) { - return &datastore.Features{Watch: datastore.Feature{Enabled: true}}, nil +func (sd *spannerDatastore) Features(ctx context.Context) (*datastore.Features, error) { + return sd.OfflineFeatures() +} + +func (sd *spannerDatastore) OfflineFeatures() (*datastore.Features, error) { + return &datastore.Features{ + Watch: datastore.Feature{ + Status: datastore.FeatureSupported, + }, + IntegrityData: datastore.Feature{ + Status: datastore.FeatureUnsupported, + }, + }, nil } func (sd *spannerDatastore) Close() error { diff --git a/internal/services/v1/experimental.go b/internal/services/v1/experimental.go index cf5966544f..e1e9e563c0 100644 --- a/internal/services/v1/experimental.go +++ b/internal/services/v1/experimental.go @@ -164,6 +164,7 @@ func (a *bulkLoadAdapter) Next(_ context.Context) (*core.RelationTuple, error) { } a.current.Caveat = &a.caveat + a.current.Integrity = nil tuple.CopyRelationshipToRelationTuple(a.currentBatch[a.numSent], &a.current) if err := relationships.ValidateOneRelationship( diff --git a/internal/testfixtures/generator.go b/internal/testfixtures/generator.go index a1382b762d..751f28271e 100644 --- a/internal/testfixtures/generator.go +++ b/internal/testfixtures/generator.go @@ -61,6 +61,8 @@ func (btg *BulkTupleGenerator) Next(_ context.Context) (*core.RelationTuple, err btg.remaining-- btg.current.ResourceAndRelation.ObjectId = strconv.Itoa(btg.remaining) btg.current.Subject.ObjectId = strconv.Itoa(btg.remaining) + btg.current.Caveat = nil + btg.current.Integrity = nil return &btg.current, nil } diff --git a/pkg/cmd/datastore/datastore.go b/pkg/cmd/datastore/datastore.go index f350caa31e..6bffafba84 100644 --- a/pkg/cmd/datastore/datastore.go +++ b/pkg/cmd/datastore/datastore.go @@ -2,8 +2,10 @@ package datastore import ( "context" + "encoding/json" "errors" "fmt" + "os" "strings" "time" @@ -150,6 +152,11 @@ type Config struct { // MySQL TablePrefix string `debugmap:"visible"` + // Relationship Integrity + RelationshipIntegrityEnabled bool `debugmap:"visible"` + RelationshipIntegrityCurrentKey RelIntegrityKey `debugmap:"visible"` + RelationshipIntegrityExpiredKeys []string `debugmap:"visible"` + // Internal WatchBufferLength uint16 `debugmap:"visible"` WatchBufferWriteTimeout time.Duration `debugmap:"visible"` @@ -159,6 +166,12 @@ type Config struct { MigrationPhase string `debugmap:"visible"` } +//go:generate go run github.com/ecordell/optgen -sensitive-field-name-matches uri,secure -output zz_generated.relintegritykey.options.go . RelIntegrityKey +type RelIntegrityKey struct { + KeyID string `debugmap:"visible"` + KeyFilename string `debugmap:"visible"` +} + // RegisterDatastoreFlags adds datastore flags to a cobra command. func RegisterDatastoreFlags(cmd *cobra.Command, opts *Config) error { return RegisterDatastoreFlagsWithPrefix(cmd.Flags(), "", opts) @@ -233,6 +246,11 @@ func RegisterDatastoreFlagsWithPrefix(flagSet *pflag.FlagSet, prefix string, opt flagSet.DurationVar(&opts.WatchBufferWriteTimeout, flagName("datastore-watch-buffer-write-timeout"), 1*time.Second, "how long the watch buffer should queue before forcefully disconnecting the reader") flagSet.DurationVar(&opts.WatchConnectTimeout, flagName("datastore-watch-connect-timeout"), 1*time.Second, "how long the watch connection should wait before timing out (cockroachdb driver only)") + flagSet.BoolVar(&opts.RelationshipIntegrityEnabled, flagName("datastore-relationship-integrity-enabled"), false, "enables relationship integrity checks. only supported on CRDB") + flagSet.StringVar(&opts.RelationshipIntegrityCurrentKey.KeyID, flagName("datastore-relationship-integrity-current-key-id"), "", "current key id for relationship integrity checks") + flagSet.StringVar(&opts.RelationshipIntegrityCurrentKey.KeyFilename, flagName("datastore-relationship-integrity-current-key-filename"), "", "current key filename for relationship integrity checks") + flagSet.StringArrayVar(&opts.RelationshipIntegrityExpiredKeys, flagName("datastore-relationship-integrity-expired-keys"), []string{}, "config for expired keys for relationship integrity checks") + // disabling stats is only for tests flagSet.BoolVar(&opts.DisableStats, flagName("datastore-disable-stats"), false, "disable recording relationship counts to the stats table") if err := flagSet.MarkHidden(flagName("datastore-disable-stats")); err != nil { @@ -254,43 +272,46 @@ func RegisterDatastoreFlagsWithPrefix(flagSet *pflag.FlagSet, prefix string, opt func DefaultDatastoreConfig() *Config { return &Config{ - Engine: MemoryEngine, - GCWindow: 24 * time.Hour, - LegacyFuzzing: -1, - RevisionQuantization: 5 * time.Second, - MaxRevisionStalenessPercent: .1, // 10% - ReadConnPool: *DefaultReadConnPool(), - WriteConnPool: *DefaultWriteConnPool(), - ReadReplicaConnPool: *DefaultReadConnPool(), - ReadReplicaURIs: []string{}, - ReadOnly: false, - MaxRetries: 10, - OverlapKey: "key", - OverlapStrategy: "static", - ConnectRate: 100 * time.Millisecond, - EnableConnectionBalancing: true, - GCInterval: 3 * time.Minute, - GCMaxOperationTime: 1 * time.Minute, - WatchBufferLength: 1024, - WatchBufferWriteTimeout: 1 * time.Second, - WatchConnectTimeout: 1 * time.Second, - EnableDatastoreMetrics: true, - DisableStats: false, - BootstrapFiles: []string{}, - BootstrapTimeout: 10 * time.Second, - BootstrapOverwrite: false, - RequestHedgingEnabled: false, - RequestHedgingInitialSlowValue: 10000000, - RequestHedgingMaxRequests: 1_000_000, - RequestHedgingQuantile: 0.95, - SpannerCredentialsFile: "", - SpannerEmulatorHost: "", - TablePrefix: "", - MigrationPhase: "", - FollowerReadDelay: 4_800 * time.Millisecond, - SpannerMinSessions: 100, - SpannerMaxSessions: 400, - FilterMaximumIDCount: 100, + Engine: MemoryEngine, + GCWindow: 24 * time.Hour, + LegacyFuzzing: -1, + RevisionQuantization: 5 * time.Second, + MaxRevisionStalenessPercent: .1, // 10% + ReadConnPool: *DefaultReadConnPool(), + WriteConnPool: *DefaultWriteConnPool(), + ReadReplicaConnPool: *DefaultReadConnPool(), + ReadReplicaURIs: []string{}, + ReadOnly: false, + MaxRetries: 10, + OverlapKey: "key", + OverlapStrategy: "static", + ConnectRate: 100 * time.Millisecond, + EnableConnectionBalancing: true, + GCInterval: 3 * time.Minute, + GCMaxOperationTime: 1 * time.Minute, + WatchBufferLength: 1024, + WatchBufferWriteTimeout: 1 * time.Second, + WatchConnectTimeout: 1 * time.Second, + EnableDatastoreMetrics: true, + DisableStats: false, + BootstrapFiles: []string{}, + BootstrapTimeout: 10 * time.Second, + BootstrapOverwrite: false, + RequestHedgingEnabled: false, + RequestHedgingInitialSlowValue: 10000000, + RequestHedgingMaxRequests: 1_000_000, + RequestHedgingQuantile: 0.95, + SpannerCredentialsFile: "", + SpannerEmulatorHost: "", + TablePrefix: "", + MigrationPhase: "", + FollowerReadDelay: 4_800 * time.Millisecond, + SpannerMinSessions: 100, + SpannerMaxSessions: 400, + FilterMaximumIDCount: 100, + RelationshipIntegrityEnabled: false, + RelationshipIntegrityCurrentKey: RelIntegrityKey{}, + RelationshipIntegrityExpiredKeys: []string{}, } } @@ -382,9 +403,68 @@ func NewDatastore(ctx context.Context, options ...ConfigOption) (datastore.Datas log.Ctx(ctx).Warn().Msg("setting the datastore to read-only") ds = proxy.NewReadonlyDatastore(ds) } + + if opts.RelationshipIntegrityEnabled { + log.Ctx(ctx).Info().Msg("enabling relationship integrity checks") + + keyBytes, err := os.ReadFile(opts.RelationshipIntegrityCurrentKey.KeyFilename) + if err != nil { + return nil, fmt.Errorf("error in opening current key file: %w", err) + } + + currentKey := proxy.KeyConfig{ + ID: opts.RelationshipIntegrityCurrentKey.KeyID, + Bytes: keyBytes, + } + + expiredKeys, err := readExpiredKeys(opts.RelationshipIntegrityExpiredKeys) + if err != nil { + return nil, fmt.Errorf("error in reading expired keys: %w", err) + } + + wrapped, err := proxy.NewRelationshipIntegrityProxy(ds, currentKey, expiredKeys) + if err != nil { + return nil, fmt.Errorf("error in configuring relationship integrity checks: %w", err) + } + + ds = wrapped + } + return ds, nil } +type expiredKeyStruct struct { + KeyID string `json:"key_id"` + KeyFilename string `json:"key_filename"` + ExpiredAt time.Time `json:"expired_at"` +} + +func readExpiredKeys(expiredKeyStrings []string) ([]proxy.KeyConfig, error) { + expiredKeys := make([]proxy.KeyConfig, 0, len(expiredKeyStrings)) + for index, keyString := range expiredKeyStrings { + key := expiredKeyStruct{} + err := json.Unmarshal([]byte(keyString), &key) + if err != nil { + return nil, fmt.Errorf("error in unmarshalling expired key #%d: %w", index+1, err) + } + + keyBytes, err := os.ReadFile(key.KeyFilename) + if err != nil { + return nil, fmt.Errorf("error in opening current key file: %w", err) + } + + expiredAt := key.ExpiredAt + expiredKey := proxy.KeyConfig{ + ID: key.KeyID, + Bytes: keyBytes, + ExpiredAt: &expiredAt, + } + expiredKeys = append(expiredKeys, expiredKey) + } + + return expiredKeys, nil +} + func newCRDBDatastore(ctx context.Context, opts Config) (datastore.Datastore, error) { if len(opts.ReadReplicaURIs) > 0 { return nil, errors.New("read replicas are not supported for the CockroachDB datastore engine") @@ -419,6 +499,7 @@ func newCRDBDatastore(ctx context.Context, opts Config) (datastore.Datastore, er crdb.WithEnableConnectionBalancing(opts.EnableConnectionBalancing), crdb.ConnectRate(opts.ConnectRate), crdb.FilterMaximumIDCount(opts.FilterMaximumIDCount), + crdb.WithIntegrity(opts.RelationshipIntegrityEnabled), ) } diff --git a/pkg/cmd/datastore/zz_generated.options.go b/pkg/cmd/datastore/zz_generated.options.go index 7f52d8e1ef..ea4e793075 100644 --- a/pkg/cmd/datastore/zz_generated.options.go +++ b/pkg/cmd/datastore/zz_generated.options.go @@ -69,6 +69,9 @@ func (c *Config) ToOption() ConfigOption { to.SpannerMinSessions = c.SpannerMinSessions to.SpannerMaxSessions = c.SpannerMaxSessions to.TablePrefix = c.TablePrefix + to.RelationshipIntegrityEnabled = c.RelationshipIntegrityEnabled + to.RelationshipIntegrityCurrentKey = c.RelationshipIntegrityCurrentKey + to.RelationshipIntegrityExpiredKeys = c.RelationshipIntegrityExpiredKeys to.WatchBufferLength = c.WatchBufferLength to.WatchBufferWriteTimeout = c.WatchBufferWriteTimeout to.WatchConnectTimeout = c.WatchConnectTimeout @@ -116,6 +119,9 @@ func (c Config) DebugMap() map[string]any { debugMap["SpannerMinSessions"] = helpers.DebugValue(c.SpannerMinSessions, false) debugMap["SpannerMaxSessions"] = helpers.DebugValue(c.SpannerMaxSessions, false) debugMap["TablePrefix"] = helpers.DebugValue(c.TablePrefix, false) + debugMap["RelationshipIntegrityEnabled"] = helpers.DebugValue(c.RelationshipIntegrityEnabled, false) + debugMap["RelationshipIntegrityCurrentKey"] = helpers.DebugValue(c.RelationshipIntegrityCurrentKey, false) + debugMap["RelationshipIntegrityExpiredKeys"] = helpers.DebugValue(c.RelationshipIntegrityExpiredKeys, false) debugMap["WatchBufferLength"] = helpers.DebugValue(c.WatchBufferLength, false) debugMap["WatchBufferWriteTimeout"] = helpers.DebugValue(c.WatchBufferWriteTimeout, false) debugMap["WatchConnectTimeout"] = helpers.DebugValue(c.WatchConnectTimeout, false) @@ -433,6 +439,34 @@ func WithTablePrefix(tablePrefix string) ConfigOption { } } +// WithRelationshipIntegrityEnabled returns an option that can set RelationshipIntegrityEnabled on a Config +func WithRelationshipIntegrityEnabled(relationshipIntegrityEnabled bool) ConfigOption { + return func(c *Config) { + c.RelationshipIntegrityEnabled = relationshipIntegrityEnabled + } +} + +// WithRelationshipIntegrityCurrentKey returns an option that can set RelationshipIntegrityCurrentKey on a Config +func WithRelationshipIntegrityCurrentKey(relationshipIntegrityCurrentKey RelIntegrityKey) ConfigOption { + return func(c *Config) { + c.RelationshipIntegrityCurrentKey = relationshipIntegrityCurrentKey + } +} + +// WithRelationshipIntegrityExpiredKeys returns an option that can append RelationshipIntegrityExpiredKeyss to Config.RelationshipIntegrityExpiredKeys +func WithRelationshipIntegrityExpiredKeys(relationshipIntegrityExpiredKeys string) ConfigOption { + return func(c *Config) { + c.RelationshipIntegrityExpiredKeys = append(c.RelationshipIntegrityExpiredKeys, relationshipIntegrityExpiredKeys) + } +} + +// SetRelationshipIntegrityExpiredKeys returns an option that can set RelationshipIntegrityExpiredKeys on a Config +func SetRelationshipIntegrityExpiredKeys(relationshipIntegrityExpiredKeys []string) ConfigOption { + return func(c *Config) { + c.RelationshipIntegrityExpiredKeys = relationshipIntegrityExpiredKeys + } +} + // WithWatchBufferLength returns an option that can set WatchBufferLength on a Config func WithWatchBufferLength(watchBufferLength uint16) ConfigOption { return func(c *Config) { diff --git a/pkg/cmd/datastore/zz_generated.relintegritykey.options.go b/pkg/cmd/datastore/zz_generated.relintegritykey.options.go new file mode 100644 index 0000000000..875ecefa64 --- /dev/null +++ b/pkg/cmd/datastore/zz_generated.relintegritykey.options.go @@ -0,0 +1,74 @@ +// Code generated by github.com/ecordell/optgen. DO NOT EDIT. +package datastore + +import ( + defaults "github.com/creasty/defaults" + helpers "github.com/ecordell/optgen/helpers" +) + +type RelIntegrityKeyOption func(r *RelIntegrityKey) + +// NewRelIntegrityKeyWithOptions creates a new RelIntegrityKey with the passed in options set +func NewRelIntegrityKeyWithOptions(opts ...RelIntegrityKeyOption) *RelIntegrityKey { + r := &RelIntegrityKey{} + for _, o := range opts { + o(r) + } + return r +} + +// NewRelIntegrityKeyWithOptionsAndDefaults creates a new RelIntegrityKey with the passed in options set starting from the defaults +func NewRelIntegrityKeyWithOptionsAndDefaults(opts ...RelIntegrityKeyOption) *RelIntegrityKey { + r := &RelIntegrityKey{} + defaults.MustSet(r) + for _, o := range opts { + o(r) + } + return r +} + +// ToOption returns a new RelIntegrityKeyOption that sets the values from the passed in RelIntegrityKey +func (r *RelIntegrityKey) ToOption() RelIntegrityKeyOption { + return func(to *RelIntegrityKey) { + to.KeyID = r.KeyID + to.KeyFilename = r.KeyFilename + } +} + +// DebugMap returns a map form of RelIntegrityKey for debugging +func (r RelIntegrityKey) DebugMap() map[string]any { + debugMap := map[string]any{} + debugMap["KeyID"] = helpers.DebugValue(r.KeyID, false) + debugMap["KeyFilename"] = helpers.DebugValue(r.KeyFilename, false) + return debugMap +} + +// RelIntegrityKeyWithOptions configures an existing RelIntegrityKey with the passed in options set +func RelIntegrityKeyWithOptions(r *RelIntegrityKey, opts ...RelIntegrityKeyOption) *RelIntegrityKey { + for _, o := range opts { + o(r) + } + return r +} + +// WithOptions configures the receiver RelIntegrityKey with the passed in options set +func (r *RelIntegrityKey) WithOptions(opts ...RelIntegrityKeyOption) *RelIntegrityKey { + for _, o := range opts { + o(r) + } + return r +} + +// WithKeyID returns an option that can set KeyID on a RelIntegrityKey +func WithKeyID(keyID string) RelIntegrityKeyOption { + return func(r *RelIntegrityKey) { + r.KeyID = keyID + } +} + +// WithKeyFilename returns an option that can set KeyFilename on a RelIntegrityKey +func WithKeyFilename(keyFilename string) RelIntegrityKeyOption { + return func(r *RelIntegrityKey) { + r.KeyFilename = keyFilename + } +} diff --git a/pkg/cmd/server/server.go b/pkg/cmd/server/server.go index 16d2449913..ad569fb628 100644 --- a/pkg/cmd/server/server.go +++ b/pkg/cmd/server/server.go @@ -371,7 +371,7 @@ func (c *Config) Complete(ctx context.Context) (RunnableServer, error) { } watchServiceOption := services.WatchServiceEnabled - if !datastoreFeatures.Watch.Enabled { + if datastoreFeatures.Watch.Status != datastore.FeatureSupported { log.Ctx(ctx).Warn().Str("reason", datastoreFeatures.Watch.Reason).Msg("watch api disabled; underlying datastore does not support it") watchServiceOption = services.WatchServiceDisabled } diff --git a/pkg/datastore/datastore.go b/pkg/datastore/datastore.go index d709018010..11024ae5c9 100644 --- a/pkg/datastore/datastore.go +++ b/pkg/datastore/datastore.go @@ -575,9 +575,14 @@ type ReadOnlyDatastore interface { ReadyState(ctx context.Context) (ReadyState, error) // Features returns an object representing what features this - // datastore can support. + // datastore can support. Can make calls to the database, so should + // only be used when a connection is allowed. Features(ctx context.Context) (*Features, error) + // OfflineFeatures returns an object representing what features this + // datastore supports code-wise, without making any calls to the database. + OfflineFeatures() (*Features, error) + // Statistics returns relevant values about the data contained in this cluster. Statistics(ctx context.Context) (Stats, error) @@ -674,17 +679,37 @@ func UnwrapAs[T any](datastore Datastore) T { return ds } +// FeatureStatus are the possible statuses for a feature in the datastore. +type FeatureStatus int + +const ( + // FeatureStatusUnknown indicates that the status of the feature is unknown. + // This can be returned, for example, when a call is made to OfflineFeatures + // but the feature requires a call to the database to determine its status. + FeatureStatusUnknown FeatureStatus = iota + + // FeatureSupported indicates that the feature is supported by the datastore. + FeatureSupported + + // FeatureUnsupported indicates that the feature is not supported by the datastore. + FeatureUnsupported +) + // Feature represents a capability that a datastore can support, plus an // optional message explaining the feature is available (or not). type Feature struct { - Enabled bool - Reason string + Status FeatureStatus + Reason string } // Features holds values that represent what features a database can support. type Features struct { // Watch is enabled if the underlying datastore can support the Watch api. Watch Feature + + // IntegrityData is enabled if the underlying datastore supports retrieving and storing + // integrity information. + IntegrityData Feature } // ObjectTypeStat represents statistics for a single object type (namespace). diff --git a/pkg/datastore/datastore_test.go b/pkg/datastore/datastore_test.go index 1898d9aa77..0dd7786643 100644 --- a/pkg/datastore/datastore_test.go +++ b/pkg/datastore/datastore_test.go @@ -622,6 +622,10 @@ func (f fakeDatastore) Features(_ context.Context) (*Features, error) { return nil, nil } +func (f fakeDatastore) OfflineFeatures() (*Features, error) { + return nil, nil +} + func (f fakeDatastore) Statistics(_ context.Context) (Stats, error) { return Stats{}, nil } diff --git a/pkg/datastore/test/transactions.go b/pkg/datastore/test/transactions.go index b08d11472b..6e725054b7 100644 --- a/pkg/datastore/test/transactions.go +++ b/pkg/datastore/test/transactions.go @@ -34,7 +34,14 @@ func RetryTest(t *testing.T, tester DatastoreTester) { rawDS, err := tester.New(0, veryLargeGCInterval, veryLargeGCWindow, 1) require.NoError(err) - ds := rawDS.(TestableDatastore) + var ds TestableDatastore + if tds, ok := rawDS.(TestableDatastore); ok { + ds = tds + } else { + if uw, ok := rawDS.(datastore.UnwrappableDatastore); ok { + ds = uw.Unwrap().(TestableDatastore) + } + } ctx, cancel := context.WithTimeout(context.Background(), 1500*time.Millisecond) defer cancel() diff --git a/pkg/datastore/test/tuples.go b/pkg/datastore/test/tuples.go index d9429dd7a1..df92b10b1c 100644 --- a/pkg/datastore/test/tuples.go +++ b/pkg/datastore/test/tuples.go @@ -609,6 +609,7 @@ func CreateAlreadyExistingTest(t *testing.T, tester DatastoreTester) { } _, _ = ds.ReadWriteTx(ctx, f) _, err = ds.ReadWriteTx(ctx, f) + require.Error(err) grpcutil.RequireStatus(t, codes.AlreadyExists, err) } @@ -1584,7 +1585,9 @@ func ConcurrentWriteSerializationTest(t *testing.T, tester DatastoreTester) { OptionalResourceType: testResourceNamespace, }) iter.Close() - require.NoError(err) + if err != nil { + return err + } // We do NOT assert the error here because serialization problems can manifest as errors // on the individual writes. @@ -1595,10 +1598,11 @@ func ConcurrentWriteSerializationTest(t *testing.T, tester DatastoreTester) { close(waitToStart) }) <-waitToFinish - return err }) - require.NoError(err) + if err != nil { + panic(err) + } return nil }) @@ -1710,6 +1714,7 @@ func ensureTuplesStatus(ctx context.Context, require *require.Assertions, ds dat defer iter.Close() found := iter.Next() + require.NoError(iter.Err()) if mustExist { require.NotNil(found, "expected tuple %s", tuple.MustString(tpl)) diff --git a/pkg/proto/core/v1/core.pb.go b/pkg/proto/core/v1/core.pb.go index 7ced6d3f5d..e5c41eeced 100644 --- a/pkg/proto/core/v1/core.pb.go +++ b/pkg/proto/core/v1/core.pb.go @@ -12,6 +12,7 @@ import ( protoimpl "google.golang.org/protobuf/runtime/protoimpl" anypb "google.golang.org/protobuf/types/known/anypb" structpb "google.golang.org/protobuf/types/known/structpb" + timestamppb "google.golang.org/protobuf/types/known/timestamppb" reflect "reflect" sync "sync" ) @@ -72,7 +73,7 @@ func (x RelationTupleUpdate_Operation) Number() protoreflect.EnumNumber { // Deprecated: Use RelationTupleUpdate_Operation.Descriptor instead. func (RelationTupleUpdate_Operation) EnumDescriptor() ([]byte, []int) { - return file_core_v1_core_proto_rawDescGZIP(), []int{7, 0} + return file_core_v1_core_proto_rawDescGZIP(), []int{8, 0} } type SetOperationUserset_Operation int32 @@ -124,7 +125,7 @@ func (x SetOperationUserset_Operation) Number() protoreflect.EnumNumber { // Deprecated: Use SetOperationUserset_Operation.Descriptor instead. func (SetOperationUserset_Operation) EnumDescriptor() ([]byte, []int) { - return file_core_v1_core_proto_rawDescGZIP(), []int{9, 0} + return file_core_v1_core_proto_rawDescGZIP(), []int{10, 0} } type ReachabilityEntrypoint_ReachabilityEntrypointKind int32 @@ -182,7 +183,7 @@ func (x ReachabilityEntrypoint_ReachabilityEntrypointKind) Number() protoreflect // Deprecated: Use ReachabilityEntrypoint_ReachabilityEntrypointKind.Descriptor instead. func (ReachabilityEntrypoint_ReachabilityEntrypointKind) EnumDescriptor() ([]byte, []int) { - return file_core_v1_core_proto_rawDescGZIP(), []int{17, 0} + return file_core_v1_core_proto_rawDescGZIP(), []int{18, 0} } type ReachabilityEntrypoint_EntrypointResultStatus int32 @@ -235,7 +236,7 @@ func (x ReachabilityEntrypoint_EntrypointResultStatus) Number() protoreflect.Enu // Deprecated: Use ReachabilityEntrypoint_EntrypointResultStatus.Descriptor instead. func (ReachabilityEntrypoint_EntrypointResultStatus) EnumDescriptor() ([]byte, []int) { - return file_core_v1_core_proto_rawDescGZIP(), []int{17, 1} + return file_core_v1_core_proto_rawDescGZIP(), []int{18, 1} } type FunctionedTupleToUserset_Function int32 @@ -284,7 +285,7 @@ func (x FunctionedTupleToUserset_Function) Number() protoreflect.EnumNumber { // Deprecated: Use FunctionedTupleToUserset_Function.Descriptor instead. func (FunctionedTupleToUserset_Function) EnumDescriptor() ([]byte, []int) { - return file_core_v1_core_proto_rawDescGZIP(), []int{24, 0} + return file_core_v1_core_proto_rawDescGZIP(), []int{25, 0} } type ComputedUserset_Object int32 @@ -330,7 +331,7 @@ func (x ComputedUserset_Object) Number() protoreflect.EnumNumber { // Deprecated: Use ComputedUserset_Object.Descriptor instead. func (ComputedUserset_Object) EnumDescriptor() ([]byte, []int) { - return file_core_v1_core_proto_rawDescGZIP(), []int{25, 0} + return file_core_v1_core_proto_rawDescGZIP(), []int{26, 0} } type CaveatOperation_Operation int32 @@ -382,7 +383,7 @@ func (x CaveatOperation_Operation) Number() protoreflect.EnumNumber { // Deprecated: Use CaveatOperation_Operation.Descriptor instead. func (CaveatOperation_Operation) EnumDescriptor() ([]byte, []int) { - return file_core_v1_core_proto_rawDescGZIP(), []int{28, 0} + return file_core_v1_core_proto_rawDescGZIP(), []int{29, 0} } type RelationTuple struct { @@ -396,6 +397,8 @@ type RelationTuple struct { Subject *ObjectAndRelation `protobuf:"bytes,2,opt,name=subject,proto3" json:"subject,omitempty"` // * caveat is a reference to a the caveat that must be enforced over the tuple * Caveat *ContextualizedCaveat `protobuf:"bytes,3,opt,name=caveat,proto3" json:"caveat,omitempty"` + // * integrity holds (optional) information about the integrity of the tuple + Integrity *RelationshipIntegrity `protobuf:"bytes,4,opt,name=integrity,proto3" json:"integrity,omitempty"` } func (x *RelationTuple) Reset() { @@ -451,6 +454,79 @@ func (x *RelationTuple) GetCaveat() *ContextualizedCaveat { return nil } +func (x *RelationTuple) GetIntegrity() *RelationshipIntegrity { + if x != nil { + return x.Integrity + } + return nil +} + +type RelationshipIntegrity struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // * key_id is the key ID used to hash the tuple + KeyId string `protobuf:"bytes,1,opt,name=key_id,json=keyId,proto3" json:"key_id,omitempty"` + // * hash is the hash of the tuple + Hash []byte `protobuf:"bytes,2,opt,name=hash,proto3" json:"hash,omitempty"` + // * hashed_at is the timestamp when the tuple was hashed + HashedAt *timestamppb.Timestamp `protobuf:"bytes,3,opt,name=hashed_at,json=hashedAt,proto3" json:"hashed_at,omitempty"` +} + +func (x *RelationshipIntegrity) Reset() { + *x = RelationshipIntegrity{} + if protoimpl.UnsafeEnabled { + mi := &file_core_v1_core_proto_msgTypes[1] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *RelationshipIntegrity) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*RelationshipIntegrity) ProtoMessage() {} + +func (x *RelationshipIntegrity) ProtoReflect() protoreflect.Message { + mi := &file_core_v1_core_proto_msgTypes[1] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use RelationshipIntegrity.ProtoReflect.Descriptor instead. +func (*RelationshipIntegrity) Descriptor() ([]byte, []int) { + return file_core_v1_core_proto_rawDescGZIP(), []int{1} +} + +func (x *RelationshipIntegrity) GetKeyId() string { + if x != nil { + return x.KeyId + } + return "" +} + +func (x *RelationshipIntegrity) GetHash() []byte { + if x != nil { + return x.Hash + } + return nil +} + +func (x *RelationshipIntegrity) GetHashedAt() *timestamppb.Timestamp { + if x != nil { + return x.HashedAt + } + return nil +} + // * // ContextualizedCaveat represents a reference to a caveat used to by caveated tuples. // The context are key-value pairs that will be injected at evaluation time. @@ -468,7 +544,7 @@ type ContextualizedCaveat struct { func (x *ContextualizedCaveat) Reset() { *x = ContextualizedCaveat{} if protoimpl.UnsafeEnabled { - mi := &file_core_v1_core_proto_msgTypes[1] + mi := &file_core_v1_core_proto_msgTypes[2] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -481,7 +557,7 @@ func (x *ContextualizedCaveat) String() string { func (*ContextualizedCaveat) ProtoMessage() {} func (x *ContextualizedCaveat) ProtoReflect() protoreflect.Message { - mi := &file_core_v1_core_proto_msgTypes[1] + mi := &file_core_v1_core_proto_msgTypes[2] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -494,7 +570,7 @@ func (x *ContextualizedCaveat) ProtoReflect() protoreflect.Message { // Deprecated: Use ContextualizedCaveat.ProtoReflect.Descriptor instead. func (*ContextualizedCaveat) Descriptor() ([]byte, []int) { - return file_core_v1_core_proto_rawDescGZIP(), []int{1} + return file_core_v1_core_proto_rawDescGZIP(), []int{2} } func (x *ContextualizedCaveat) GetCaveatName() string { @@ -531,7 +607,7 @@ type CaveatDefinition struct { func (x *CaveatDefinition) Reset() { *x = CaveatDefinition{} if protoimpl.UnsafeEnabled { - mi := &file_core_v1_core_proto_msgTypes[2] + mi := &file_core_v1_core_proto_msgTypes[3] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -544,7 +620,7 @@ func (x *CaveatDefinition) String() string { func (*CaveatDefinition) ProtoMessage() {} func (x *CaveatDefinition) ProtoReflect() protoreflect.Message { - mi := &file_core_v1_core_proto_msgTypes[2] + mi := &file_core_v1_core_proto_msgTypes[3] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -557,7 +633,7 @@ func (x *CaveatDefinition) ProtoReflect() protoreflect.Message { // Deprecated: Use CaveatDefinition.ProtoReflect.Descriptor instead. func (*CaveatDefinition) Descriptor() ([]byte, []int) { - return file_core_v1_core_proto_rawDescGZIP(), []int{2} + return file_core_v1_core_proto_rawDescGZIP(), []int{3} } func (x *CaveatDefinition) GetName() string { @@ -607,7 +683,7 @@ type CaveatTypeReference struct { func (x *CaveatTypeReference) Reset() { *x = CaveatTypeReference{} if protoimpl.UnsafeEnabled { - mi := &file_core_v1_core_proto_msgTypes[3] + mi := &file_core_v1_core_proto_msgTypes[4] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -620,7 +696,7 @@ func (x *CaveatTypeReference) String() string { func (*CaveatTypeReference) ProtoMessage() {} func (x *CaveatTypeReference) ProtoReflect() protoreflect.Message { - mi := &file_core_v1_core_proto_msgTypes[3] + mi := &file_core_v1_core_proto_msgTypes[4] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -633,7 +709,7 @@ func (x *CaveatTypeReference) ProtoReflect() protoreflect.Message { // Deprecated: Use CaveatTypeReference.ProtoReflect.Descriptor instead. func (*CaveatTypeReference) Descriptor() ([]byte, []int) { - return file_core_v1_core_proto_rawDescGZIP(), []int{3} + return file_core_v1_core_proto_rawDescGZIP(), []int{4} } func (x *CaveatTypeReference) GetTypeName() string { @@ -666,7 +742,7 @@ type ObjectAndRelation struct { func (x *ObjectAndRelation) Reset() { *x = ObjectAndRelation{} if protoimpl.UnsafeEnabled { - mi := &file_core_v1_core_proto_msgTypes[4] + mi := &file_core_v1_core_proto_msgTypes[5] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -679,7 +755,7 @@ func (x *ObjectAndRelation) String() string { func (*ObjectAndRelation) ProtoMessage() {} func (x *ObjectAndRelation) ProtoReflect() protoreflect.Message { - mi := &file_core_v1_core_proto_msgTypes[4] + mi := &file_core_v1_core_proto_msgTypes[5] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -692,7 +768,7 @@ func (x *ObjectAndRelation) ProtoReflect() protoreflect.Message { // Deprecated: Use ObjectAndRelation.ProtoReflect.Descriptor instead. func (*ObjectAndRelation) Descriptor() ([]byte, []int) { - return file_core_v1_core_proto_rawDescGZIP(), []int{4} + return file_core_v1_core_proto_rawDescGZIP(), []int{5} } func (x *ObjectAndRelation) GetNamespace() string { @@ -730,7 +806,7 @@ type RelationReference struct { func (x *RelationReference) Reset() { *x = RelationReference{} if protoimpl.UnsafeEnabled { - mi := &file_core_v1_core_proto_msgTypes[5] + mi := &file_core_v1_core_proto_msgTypes[6] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -743,7 +819,7 @@ func (x *RelationReference) String() string { func (*RelationReference) ProtoMessage() {} func (x *RelationReference) ProtoReflect() protoreflect.Message { - mi := &file_core_v1_core_proto_msgTypes[5] + mi := &file_core_v1_core_proto_msgTypes[6] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -756,7 +832,7 @@ func (x *RelationReference) ProtoReflect() protoreflect.Message { // Deprecated: Use RelationReference.ProtoReflect.Descriptor instead. func (*RelationReference) Descriptor() ([]byte, []int) { - return file_core_v1_core_proto_rawDescGZIP(), []int{5} + return file_core_v1_core_proto_rawDescGZIP(), []int{6} } func (x *RelationReference) GetNamespace() string { @@ -784,7 +860,7 @@ type Zookie struct { func (x *Zookie) Reset() { *x = Zookie{} if protoimpl.UnsafeEnabled { - mi := &file_core_v1_core_proto_msgTypes[6] + mi := &file_core_v1_core_proto_msgTypes[7] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -797,7 +873,7 @@ func (x *Zookie) String() string { func (*Zookie) ProtoMessage() {} func (x *Zookie) ProtoReflect() protoreflect.Message { - mi := &file_core_v1_core_proto_msgTypes[6] + mi := &file_core_v1_core_proto_msgTypes[7] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -810,7 +886,7 @@ func (x *Zookie) ProtoReflect() protoreflect.Message { // Deprecated: Use Zookie.ProtoReflect.Descriptor instead. func (*Zookie) Descriptor() ([]byte, []int) { - return file_core_v1_core_proto_rawDescGZIP(), []int{6} + return file_core_v1_core_proto_rawDescGZIP(), []int{7} } func (x *Zookie) GetToken() string { @@ -832,7 +908,7 @@ type RelationTupleUpdate struct { func (x *RelationTupleUpdate) Reset() { *x = RelationTupleUpdate{} if protoimpl.UnsafeEnabled { - mi := &file_core_v1_core_proto_msgTypes[7] + mi := &file_core_v1_core_proto_msgTypes[8] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -845,7 +921,7 @@ func (x *RelationTupleUpdate) String() string { func (*RelationTupleUpdate) ProtoMessage() {} func (x *RelationTupleUpdate) ProtoReflect() protoreflect.Message { - mi := &file_core_v1_core_proto_msgTypes[7] + mi := &file_core_v1_core_proto_msgTypes[8] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -858,7 +934,7 @@ func (x *RelationTupleUpdate) ProtoReflect() protoreflect.Message { // Deprecated: Use RelationTupleUpdate.ProtoReflect.Descriptor instead. func (*RelationTupleUpdate) Descriptor() ([]byte, []int) { - return file_core_v1_core_proto_rawDescGZIP(), []int{7} + return file_core_v1_core_proto_rawDescGZIP(), []int{8} } func (x *RelationTupleUpdate) GetOperation() RelationTupleUpdate_Operation { @@ -892,7 +968,7 @@ type RelationTupleTreeNode struct { func (x *RelationTupleTreeNode) Reset() { *x = RelationTupleTreeNode{} if protoimpl.UnsafeEnabled { - mi := &file_core_v1_core_proto_msgTypes[8] + mi := &file_core_v1_core_proto_msgTypes[9] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -905,7 +981,7 @@ func (x *RelationTupleTreeNode) String() string { func (*RelationTupleTreeNode) ProtoMessage() {} func (x *RelationTupleTreeNode) ProtoReflect() protoreflect.Message { - mi := &file_core_v1_core_proto_msgTypes[8] + mi := &file_core_v1_core_proto_msgTypes[9] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -918,7 +994,7 @@ func (x *RelationTupleTreeNode) ProtoReflect() protoreflect.Message { // Deprecated: Use RelationTupleTreeNode.ProtoReflect.Descriptor instead. func (*RelationTupleTreeNode) Descriptor() ([]byte, []int) { - return file_core_v1_core_proto_rawDescGZIP(), []int{8} + return file_core_v1_core_proto_rawDescGZIP(), []int{9} } func (m *RelationTupleTreeNode) GetNodeType() isRelationTupleTreeNode_NodeType { @@ -984,7 +1060,7 @@ type SetOperationUserset struct { func (x *SetOperationUserset) Reset() { *x = SetOperationUserset{} if protoimpl.UnsafeEnabled { - mi := &file_core_v1_core_proto_msgTypes[9] + mi := &file_core_v1_core_proto_msgTypes[10] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -997,7 +1073,7 @@ func (x *SetOperationUserset) String() string { func (*SetOperationUserset) ProtoMessage() {} func (x *SetOperationUserset) ProtoReflect() protoreflect.Message { - mi := &file_core_v1_core_proto_msgTypes[9] + mi := &file_core_v1_core_proto_msgTypes[10] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1010,7 +1086,7 @@ func (x *SetOperationUserset) ProtoReflect() protoreflect.Message { // Deprecated: Use SetOperationUserset.ProtoReflect.Descriptor instead. func (*SetOperationUserset) Descriptor() ([]byte, []int) { - return file_core_v1_core_proto_rawDescGZIP(), []int{9} + return file_core_v1_core_proto_rawDescGZIP(), []int{10} } func (x *SetOperationUserset) GetOperation() SetOperationUserset_Operation { @@ -1039,7 +1115,7 @@ type DirectSubject struct { func (x *DirectSubject) Reset() { *x = DirectSubject{} if protoimpl.UnsafeEnabled { - mi := &file_core_v1_core_proto_msgTypes[10] + mi := &file_core_v1_core_proto_msgTypes[11] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1052,7 +1128,7 @@ func (x *DirectSubject) String() string { func (*DirectSubject) ProtoMessage() {} func (x *DirectSubject) ProtoReflect() protoreflect.Message { - mi := &file_core_v1_core_proto_msgTypes[10] + mi := &file_core_v1_core_proto_msgTypes[11] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1065,7 +1141,7 @@ func (x *DirectSubject) ProtoReflect() protoreflect.Message { // Deprecated: Use DirectSubject.ProtoReflect.Descriptor instead. func (*DirectSubject) Descriptor() ([]byte, []int) { - return file_core_v1_core_proto_rawDescGZIP(), []int{10} + return file_core_v1_core_proto_rawDescGZIP(), []int{11} } func (x *DirectSubject) GetSubject() *ObjectAndRelation { @@ -1093,7 +1169,7 @@ type DirectSubjects struct { func (x *DirectSubjects) Reset() { *x = DirectSubjects{} if protoimpl.UnsafeEnabled { - mi := &file_core_v1_core_proto_msgTypes[11] + mi := &file_core_v1_core_proto_msgTypes[12] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1106,7 +1182,7 @@ func (x *DirectSubjects) String() string { func (*DirectSubjects) ProtoMessage() {} func (x *DirectSubjects) ProtoReflect() protoreflect.Message { - mi := &file_core_v1_core_proto_msgTypes[11] + mi := &file_core_v1_core_proto_msgTypes[12] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1119,7 +1195,7 @@ func (x *DirectSubjects) ProtoReflect() protoreflect.Message { // Deprecated: Use DirectSubjects.ProtoReflect.Descriptor instead. func (*DirectSubjects) Descriptor() ([]byte, []int) { - return file_core_v1_core_proto_rawDescGZIP(), []int{11} + return file_core_v1_core_proto_rawDescGZIP(), []int{12} } func (x *DirectSubjects) GetSubjects() []*DirectSubject { @@ -1143,7 +1219,7 @@ type Metadata struct { func (x *Metadata) Reset() { *x = Metadata{} if protoimpl.UnsafeEnabled { - mi := &file_core_v1_core_proto_msgTypes[12] + mi := &file_core_v1_core_proto_msgTypes[13] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1156,7 +1232,7 @@ func (x *Metadata) String() string { func (*Metadata) ProtoMessage() {} func (x *Metadata) ProtoReflect() protoreflect.Message { - mi := &file_core_v1_core_proto_msgTypes[12] + mi := &file_core_v1_core_proto_msgTypes[13] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1169,7 +1245,7 @@ func (x *Metadata) ProtoReflect() protoreflect.Message { // Deprecated: Use Metadata.ProtoReflect.Descriptor instead. func (*Metadata) Descriptor() ([]byte, []int) { - return file_core_v1_core_proto_rawDescGZIP(), []int{12} + return file_core_v1_core_proto_rawDescGZIP(), []int{13} } func (x *Metadata) GetMetadataMessage() []*anypb.Any { @@ -1199,7 +1275,7 @@ type NamespaceDefinition struct { func (x *NamespaceDefinition) Reset() { *x = NamespaceDefinition{} if protoimpl.UnsafeEnabled { - mi := &file_core_v1_core_proto_msgTypes[13] + mi := &file_core_v1_core_proto_msgTypes[14] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1212,7 +1288,7 @@ func (x *NamespaceDefinition) String() string { func (*NamespaceDefinition) ProtoMessage() {} func (x *NamespaceDefinition) ProtoReflect() protoreflect.Message { - mi := &file_core_v1_core_proto_msgTypes[13] + mi := &file_core_v1_core_proto_msgTypes[14] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1225,7 +1301,7 @@ func (x *NamespaceDefinition) ProtoReflect() protoreflect.Message { // Deprecated: Use NamespaceDefinition.ProtoReflect.Descriptor instead. func (*NamespaceDefinition) Descriptor() ([]byte, []int) { - return file_core_v1_core_proto_rawDescGZIP(), []int{13} + return file_core_v1_core_proto_rawDescGZIP(), []int{14} } func (x *NamespaceDefinition) GetName() string { @@ -1282,7 +1358,7 @@ type Relation struct { func (x *Relation) Reset() { *x = Relation{} if protoimpl.UnsafeEnabled { - mi := &file_core_v1_core_proto_msgTypes[14] + mi := &file_core_v1_core_proto_msgTypes[15] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1295,7 +1371,7 @@ func (x *Relation) String() string { func (*Relation) ProtoMessage() {} func (x *Relation) ProtoReflect() protoreflect.Message { - mi := &file_core_v1_core_proto_msgTypes[14] + mi := &file_core_v1_core_proto_msgTypes[15] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1308,7 +1384,7 @@ func (x *Relation) ProtoReflect() protoreflect.Message { // Deprecated: Use Relation.ProtoReflect.Descriptor instead. func (*Relation) Descriptor() ([]byte, []int) { - return file_core_v1_core_proto_rawDescGZIP(), []int{14} + return file_core_v1_core_proto_rawDescGZIP(), []int{15} } func (x *Relation) GetName() string { @@ -1410,7 +1486,7 @@ type ReachabilityGraph struct { func (x *ReachabilityGraph) Reset() { *x = ReachabilityGraph{} if protoimpl.UnsafeEnabled { - mi := &file_core_v1_core_proto_msgTypes[15] + mi := &file_core_v1_core_proto_msgTypes[16] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1423,7 +1499,7 @@ func (x *ReachabilityGraph) String() string { func (*ReachabilityGraph) ProtoMessage() {} func (x *ReachabilityGraph) ProtoReflect() protoreflect.Message { - mi := &file_core_v1_core_proto_msgTypes[15] + mi := &file_core_v1_core_proto_msgTypes[16] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1436,7 +1512,7 @@ func (x *ReachabilityGraph) ProtoReflect() protoreflect.Message { // Deprecated: Use ReachabilityGraph.ProtoReflect.Descriptor instead. func (*ReachabilityGraph) Descriptor() ([]byte, []int) { - return file_core_v1_core_proto_rawDescGZIP(), []int{15} + return file_core_v1_core_proto_rawDescGZIP(), []int{16} } func (x *ReachabilityGraph) GetEntrypointsBySubjectType() map[string]*ReachabilityEntrypoints { @@ -1477,7 +1553,7 @@ type ReachabilityEntrypoints struct { func (x *ReachabilityEntrypoints) Reset() { *x = ReachabilityEntrypoints{} if protoimpl.UnsafeEnabled { - mi := &file_core_v1_core_proto_msgTypes[16] + mi := &file_core_v1_core_proto_msgTypes[17] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1490,7 +1566,7 @@ func (x *ReachabilityEntrypoints) String() string { func (*ReachabilityEntrypoints) ProtoMessage() {} func (x *ReachabilityEntrypoints) ProtoReflect() protoreflect.Message { - mi := &file_core_v1_core_proto_msgTypes[16] + mi := &file_core_v1_core_proto_msgTypes[17] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1503,7 +1579,7 @@ func (x *ReachabilityEntrypoints) ProtoReflect() protoreflect.Message { // Deprecated: Use ReachabilityEntrypoints.ProtoReflect.Descriptor instead. func (*ReachabilityEntrypoints) Descriptor() ([]byte, []int) { - return file_core_v1_core_proto_rawDescGZIP(), []int{16} + return file_core_v1_core_proto_rawDescGZIP(), []int{17} } func (x *ReachabilityEntrypoints) GetEntrypoints() []*ReachabilityEntrypoint { @@ -1558,7 +1634,7 @@ type ReachabilityEntrypoint struct { func (x *ReachabilityEntrypoint) Reset() { *x = ReachabilityEntrypoint{} if protoimpl.UnsafeEnabled { - mi := &file_core_v1_core_proto_msgTypes[17] + mi := &file_core_v1_core_proto_msgTypes[18] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1571,7 +1647,7 @@ func (x *ReachabilityEntrypoint) String() string { func (*ReachabilityEntrypoint) ProtoMessage() {} func (x *ReachabilityEntrypoint) ProtoReflect() protoreflect.Message { - mi := &file_core_v1_core_proto_msgTypes[17] + mi := &file_core_v1_core_proto_msgTypes[18] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1584,7 +1660,7 @@ func (x *ReachabilityEntrypoint) ProtoReflect() protoreflect.Message { // Deprecated: Use ReachabilityEntrypoint.ProtoReflect.Descriptor instead. func (*ReachabilityEntrypoint) Descriptor() ([]byte, []int) { - return file_core_v1_core_proto_rawDescGZIP(), []int{17} + return file_core_v1_core_proto_rawDescGZIP(), []int{18} } func (x *ReachabilityEntrypoint) GetKind() ReachabilityEntrypoint_ReachabilityEntrypointKind { @@ -1638,7 +1714,7 @@ type TypeInformation struct { func (x *TypeInformation) Reset() { *x = TypeInformation{} if protoimpl.UnsafeEnabled { - mi := &file_core_v1_core_proto_msgTypes[18] + mi := &file_core_v1_core_proto_msgTypes[19] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1651,7 +1727,7 @@ func (x *TypeInformation) String() string { func (*TypeInformation) ProtoMessage() {} func (x *TypeInformation) ProtoReflect() protoreflect.Message { - mi := &file_core_v1_core_proto_msgTypes[18] + mi := &file_core_v1_core_proto_msgTypes[19] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1664,7 +1740,7 @@ func (x *TypeInformation) ProtoReflect() protoreflect.Message { // Deprecated: Use TypeInformation.ProtoReflect.Descriptor instead. func (*TypeInformation) Descriptor() ([]byte, []int) { - return file_core_v1_core_proto_rawDescGZIP(), []int{18} + return file_core_v1_core_proto_rawDescGZIP(), []int{19} } func (x *TypeInformation) GetAllowedDirectRelations() []*AllowedRelation { @@ -1701,7 +1777,7 @@ type AllowedRelation struct { func (x *AllowedRelation) Reset() { *x = AllowedRelation{} if protoimpl.UnsafeEnabled { - mi := &file_core_v1_core_proto_msgTypes[19] + mi := &file_core_v1_core_proto_msgTypes[20] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1714,7 +1790,7 @@ func (x *AllowedRelation) String() string { func (*AllowedRelation) ProtoMessage() {} func (x *AllowedRelation) ProtoReflect() protoreflect.Message { - mi := &file_core_v1_core_proto_msgTypes[19] + mi := &file_core_v1_core_proto_msgTypes[20] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1727,7 +1803,7 @@ func (x *AllowedRelation) ProtoReflect() protoreflect.Message { // Deprecated: Use AllowedRelation.ProtoReflect.Descriptor instead. func (*AllowedRelation) Descriptor() ([]byte, []int) { - return file_core_v1_core_proto_rawDescGZIP(), []int{19} + return file_core_v1_core_proto_rawDescGZIP(), []int{20} } func (x *AllowedRelation) GetNamespace() string { @@ -1803,7 +1879,7 @@ type AllowedCaveat struct { func (x *AllowedCaveat) Reset() { *x = AllowedCaveat{} if protoimpl.UnsafeEnabled { - mi := &file_core_v1_core_proto_msgTypes[20] + mi := &file_core_v1_core_proto_msgTypes[21] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1816,7 +1892,7 @@ func (x *AllowedCaveat) String() string { func (*AllowedCaveat) ProtoMessage() {} func (x *AllowedCaveat) ProtoReflect() protoreflect.Message { - mi := &file_core_v1_core_proto_msgTypes[20] + mi := &file_core_v1_core_proto_msgTypes[21] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1829,7 +1905,7 @@ func (x *AllowedCaveat) ProtoReflect() protoreflect.Message { // Deprecated: Use AllowedCaveat.ProtoReflect.Descriptor instead. func (*AllowedCaveat) Descriptor() ([]byte, []int) { - return file_core_v1_core_proto_rawDescGZIP(), []int{20} + return file_core_v1_core_proto_rawDescGZIP(), []int{21} } func (x *AllowedCaveat) GetCaveatName() string { @@ -1856,7 +1932,7 @@ type UsersetRewrite struct { func (x *UsersetRewrite) Reset() { *x = UsersetRewrite{} if protoimpl.UnsafeEnabled { - mi := &file_core_v1_core_proto_msgTypes[21] + mi := &file_core_v1_core_proto_msgTypes[22] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1869,7 +1945,7 @@ func (x *UsersetRewrite) String() string { func (*UsersetRewrite) ProtoMessage() {} func (x *UsersetRewrite) ProtoReflect() protoreflect.Message { - mi := &file_core_v1_core_proto_msgTypes[21] + mi := &file_core_v1_core_proto_msgTypes[22] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1882,7 +1958,7 @@ func (x *UsersetRewrite) ProtoReflect() protoreflect.Message { // Deprecated: Use UsersetRewrite.ProtoReflect.Descriptor instead. func (*UsersetRewrite) Descriptor() ([]byte, []int) { - return file_core_v1_core_proto_rawDescGZIP(), []int{21} + return file_core_v1_core_proto_rawDescGZIP(), []int{22} } func (m *UsersetRewrite) GetRewriteOperation() isUsersetRewrite_RewriteOperation { @@ -1953,7 +2029,7 @@ type SetOperation struct { func (x *SetOperation) Reset() { *x = SetOperation{} if protoimpl.UnsafeEnabled { - mi := &file_core_v1_core_proto_msgTypes[22] + mi := &file_core_v1_core_proto_msgTypes[23] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1966,7 +2042,7 @@ func (x *SetOperation) String() string { func (*SetOperation) ProtoMessage() {} func (x *SetOperation) ProtoReflect() protoreflect.Message { - mi := &file_core_v1_core_proto_msgTypes[22] + mi := &file_core_v1_core_proto_msgTypes[23] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1979,7 +2055,7 @@ func (x *SetOperation) ProtoReflect() protoreflect.Message { // Deprecated: Use SetOperation.ProtoReflect.Descriptor instead. func (*SetOperation) Descriptor() ([]byte, []int) { - return file_core_v1_core_proto_rawDescGZIP(), []int{22} + return file_core_v1_core_proto_rawDescGZIP(), []int{23} } func (x *SetOperation) GetChild() []*SetOperation_Child { @@ -2002,7 +2078,7 @@ type TupleToUserset struct { func (x *TupleToUserset) Reset() { *x = TupleToUserset{} if protoimpl.UnsafeEnabled { - mi := &file_core_v1_core_proto_msgTypes[23] + mi := &file_core_v1_core_proto_msgTypes[24] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2015,7 +2091,7 @@ func (x *TupleToUserset) String() string { func (*TupleToUserset) ProtoMessage() {} func (x *TupleToUserset) ProtoReflect() protoreflect.Message { - mi := &file_core_v1_core_proto_msgTypes[23] + mi := &file_core_v1_core_proto_msgTypes[24] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2028,7 +2104,7 @@ func (x *TupleToUserset) ProtoReflect() protoreflect.Message { // Deprecated: Use TupleToUserset.ProtoReflect.Descriptor instead. func (*TupleToUserset) Descriptor() ([]byte, []int) { - return file_core_v1_core_proto_rawDescGZIP(), []int{23} + return file_core_v1_core_proto_rawDescGZIP(), []int{24} } func (x *TupleToUserset) GetTupleset() *TupleToUserset_Tupleset { @@ -2066,7 +2142,7 @@ type FunctionedTupleToUserset struct { func (x *FunctionedTupleToUserset) Reset() { *x = FunctionedTupleToUserset{} if protoimpl.UnsafeEnabled { - mi := &file_core_v1_core_proto_msgTypes[24] + mi := &file_core_v1_core_proto_msgTypes[25] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2079,7 +2155,7 @@ func (x *FunctionedTupleToUserset) String() string { func (*FunctionedTupleToUserset) ProtoMessage() {} func (x *FunctionedTupleToUserset) ProtoReflect() protoreflect.Message { - mi := &file_core_v1_core_proto_msgTypes[24] + mi := &file_core_v1_core_proto_msgTypes[25] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2092,7 +2168,7 @@ func (x *FunctionedTupleToUserset) ProtoReflect() protoreflect.Message { // Deprecated: Use FunctionedTupleToUserset.ProtoReflect.Descriptor instead. func (*FunctionedTupleToUserset) Descriptor() ([]byte, []int) { - return file_core_v1_core_proto_rawDescGZIP(), []int{24} + return file_core_v1_core_proto_rawDescGZIP(), []int{25} } func (x *FunctionedTupleToUserset) GetFunction() FunctionedTupleToUserset_Function { @@ -2136,7 +2212,7 @@ type ComputedUserset struct { func (x *ComputedUserset) Reset() { *x = ComputedUserset{} if protoimpl.UnsafeEnabled { - mi := &file_core_v1_core_proto_msgTypes[25] + mi := &file_core_v1_core_proto_msgTypes[26] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2149,7 +2225,7 @@ func (x *ComputedUserset) String() string { func (*ComputedUserset) ProtoMessage() {} func (x *ComputedUserset) ProtoReflect() protoreflect.Message { - mi := &file_core_v1_core_proto_msgTypes[25] + mi := &file_core_v1_core_proto_msgTypes[26] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2162,7 +2238,7 @@ func (x *ComputedUserset) ProtoReflect() protoreflect.Message { // Deprecated: Use ComputedUserset.ProtoReflect.Descriptor instead. func (*ComputedUserset) Descriptor() ([]byte, []int) { - return file_core_v1_core_proto_rawDescGZIP(), []int{25} + return file_core_v1_core_proto_rawDescGZIP(), []int{26} } func (x *ComputedUserset) GetObject() ComputedUserset_Object { @@ -2198,7 +2274,7 @@ type SourcePosition struct { func (x *SourcePosition) Reset() { *x = SourcePosition{} if protoimpl.UnsafeEnabled { - mi := &file_core_v1_core_proto_msgTypes[26] + mi := &file_core_v1_core_proto_msgTypes[27] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2211,7 +2287,7 @@ func (x *SourcePosition) String() string { func (*SourcePosition) ProtoMessage() {} func (x *SourcePosition) ProtoReflect() protoreflect.Message { - mi := &file_core_v1_core_proto_msgTypes[26] + mi := &file_core_v1_core_proto_msgTypes[27] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2224,7 +2300,7 @@ func (x *SourcePosition) ProtoReflect() protoreflect.Message { // Deprecated: Use SourcePosition.ProtoReflect.Descriptor instead. func (*SourcePosition) Descriptor() ([]byte, []int) { - return file_core_v1_core_proto_rawDescGZIP(), []int{26} + return file_core_v1_core_proto_rawDescGZIP(), []int{27} } func (x *SourcePosition) GetZeroIndexedLineNumber() uint64 { @@ -2256,7 +2332,7 @@ type CaveatExpression struct { func (x *CaveatExpression) Reset() { *x = CaveatExpression{} if protoimpl.UnsafeEnabled { - mi := &file_core_v1_core_proto_msgTypes[27] + mi := &file_core_v1_core_proto_msgTypes[28] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2269,7 +2345,7 @@ func (x *CaveatExpression) String() string { func (*CaveatExpression) ProtoMessage() {} func (x *CaveatExpression) ProtoReflect() protoreflect.Message { - mi := &file_core_v1_core_proto_msgTypes[27] + mi := &file_core_v1_core_proto_msgTypes[28] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2282,7 +2358,7 @@ func (x *CaveatExpression) ProtoReflect() protoreflect.Message { // Deprecated: Use CaveatExpression.ProtoReflect.Descriptor instead. func (*CaveatExpression) Descriptor() ([]byte, []int) { - return file_core_v1_core_proto_rawDescGZIP(), []int{27} + return file_core_v1_core_proto_rawDescGZIP(), []int{28} } func (m *CaveatExpression) GetOperationOrCaveat() isCaveatExpression_OperationOrCaveat { @@ -2334,7 +2410,7 @@ type CaveatOperation struct { func (x *CaveatOperation) Reset() { *x = CaveatOperation{} if protoimpl.UnsafeEnabled { - mi := &file_core_v1_core_proto_msgTypes[28] + mi := &file_core_v1_core_proto_msgTypes[29] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2347,7 +2423,7 @@ func (x *CaveatOperation) String() string { func (*CaveatOperation) ProtoMessage() {} func (x *CaveatOperation) ProtoReflect() protoreflect.Message { - mi := &file_core_v1_core_proto_msgTypes[28] + mi := &file_core_v1_core_proto_msgTypes[29] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2360,7 +2436,7 @@ func (x *CaveatOperation) ProtoReflect() protoreflect.Message { // Deprecated: Use CaveatOperation.ProtoReflect.Descriptor instead. func (*CaveatOperation) Descriptor() ([]byte, []int) { - return file_core_v1_core_proto_rawDescGZIP(), []int{28} + return file_core_v1_core_proto_rawDescGZIP(), []int{29} } func (x *CaveatOperation) GetOp() CaveatOperation_Operation { @@ -2400,7 +2476,7 @@ type RelationshipFilter struct { func (x *RelationshipFilter) Reset() { *x = RelationshipFilter{} if protoimpl.UnsafeEnabled { - mi := &file_core_v1_core_proto_msgTypes[29] + mi := &file_core_v1_core_proto_msgTypes[30] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2413,7 +2489,7 @@ func (x *RelationshipFilter) String() string { func (*RelationshipFilter) ProtoMessage() {} func (x *RelationshipFilter) ProtoReflect() protoreflect.Message { - mi := &file_core_v1_core_proto_msgTypes[29] + mi := &file_core_v1_core_proto_msgTypes[30] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2426,7 +2502,7 @@ func (x *RelationshipFilter) ProtoReflect() protoreflect.Message { // Deprecated: Use RelationshipFilter.ProtoReflect.Descriptor instead. func (*RelationshipFilter) Descriptor() ([]byte, []int) { - return file_core_v1_core_proto_rawDescGZIP(), []int{29} + return file_core_v1_core_proto_rawDescGZIP(), []int{30} } func (x *RelationshipFilter) GetResourceType() string { @@ -2481,7 +2557,7 @@ type SubjectFilter struct { func (x *SubjectFilter) Reset() { *x = SubjectFilter{} if protoimpl.UnsafeEnabled { - mi := &file_core_v1_core_proto_msgTypes[30] + mi := &file_core_v1_core_proto_msgTypes[31] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2494,7 +2570,7 @@ func (x *SubjectFilter) String() string { func (*SubjectFilter) ProtoMessage() {} func (x *SubjectFilter) ProtoReflect() protoreflect.Message { - mi := &file_core_v1_core_proto_msgTypes[30] + mi := &file_core_v1_core_proto_msgTypes[31] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2507,7 +2583,7 @@ func (x *SubjectFilter) ProtoReflect() protoreflect.Message { // Deprecated: Use SubjectFilter.ProtoReflect.Descriptor instead. func (*SubjectFilter) Descriptor() ([]byte, []int) { - return file_core_v1_core_proto_rawDescGZIP(), []int{30} + return file_core_v1_core_proto_rawDescGZIP(), []int{31} } func (x *SubjectFilter) GetSubjectType() string { @@ -2540,7 +2616,7 @@ type AllowedRelation_PublicWildcard struct { func (x *AllowedRelation_PublicWildcard) Reset() { *x = AllowedRelation_PublicWildcard{} if protoimpl.UnsafeEnabled { - mi := &file_core_v1_core_proto_msgTypes[34] + mi := &file_core_v1_core_proto_msgTypes[35] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2553,7 +2629,7 @@ func (x *AllowedRelation_PublicWildcard) String() string { func (*AllowedRelation_PublicWildcard) ProtoMessage() {} func (x *AllowedRelation_PublicWildcard) ProtoReflect() protoreflect.Message { - mi := &file_core_v1_core_proto_msgTypes[34] + mi := &file_core_v1_core_proto_msgTypes[35] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2566,7 +2642,7 @@ func (x *AllowedRelation_PublicWildcard) ProtoReflect() protoreflect.Message { // Deprecated: Use AllowedRelation_PublicWildcard.ProtoReflect.Descriptor instead. func (*AllowedRelation_PublicWildcard) Descriptor() ([]byte, []int) { - return file_core_v1_core_proto_rawDescGZIP(), []int{19, 0} + return file_core_v1_core_proto_rawDescGZIP(), []int{20, 0} } type SetOperation_Child struct { @@ -2595,7 +2671,7 @@ type SetOperation_Child struct { func (x *SetOperation_Child) Reset() { *x = SetOperation_Child{} if protoimpl.UnsafeEnabled { - mi := &file_core_v1_core_proto_msgTypes[35] + mi := &file_core_v1_core_proto_msgTypes[36] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2608,7 +2684,7 @@ func (x *SetOperation_Child) String() string { func (*SetOperation_Child) ProtoMessage() {} func (x *SetOperation_Child) ProtoReflect() protoreflect.Message { - mi := &file_core_v1_core_proto_msgTypes[35] + mi := &file_core_v1_core_proto_msgTypes[36] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2621,7 +2697,7 @@ func (x *SetOperation_Child) ProtoReflect() protoreflect.Message { // Deprecated: Use SetOperation_Child.ProtoReflect.Descriptor instead. func (*SetOperation_Child) Descriptor() ([]byte, []int) { - return file_core_v1_core_proto_rawDescGZIP(), []int{22, 0} + return file_core_v1_core_proto_rawDescGZIP(), []int{23, 0} } func (m *SetOperation_Child) GetChildType() isSetOperation_Child_ChildType { @@ -2736,7 +2812,7 @@ type SetOperation_Child_This struct { func (x *SetOperation_Child_This) Reset() { *x = SetOperation_Child_This{} if protoimpl.UnsafeEnabled { - mi := &file_core_v1_core_proto_msgTypes[36] + mi := &file_core_v1_core_proto_msgTypes[37] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2749,7 +2825,7 @@ func (x *SetOperation_Child_This) String() string { func (*SetOperation_Child_This) ProtoMessage() {} func (x *SetOperation_Child_This) ProtoReflect() protoreflect.Message { - mi := &file_core_v1_core_proto_msgTypes[36] + mi := &file_core_v1_core_proto_msgTypes[37] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2762,7 +2838,7 @@ func (x *SetOperation_Child_This) ProtoReflect() protoreflect.Message { // Deprecated: Use SetOperation_Child_This.ProtoReflect.Descriptor instead. func (*SetOperation_Child_This) Descriptor() ([]byte, []int) { - return file_core_v1_core_proto_rawDescGZIP(), []int{22, 0, 0} + return file_core_v1_core_proto_rawDescGZIP(), []int{23, 0, 0} } type SetOperation_Child_Nil struct { @@ -2774,7 +2850,7 @@ type SetOperation_Child_Nil struct { func (x *SetOperation_Child_Nil) Reset() { *x = SetOperation_Child_Nil{} if protoimpl.UnsafeEnabled { - mi := &file_core_v1_core_proto_msgTypes[37] + mi := &file_core_v1_core_proto_msgTypes[38] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2787,7 +2863,7 @@ func (x *SetOperation_Child_Nil) String() string { func (*SetOperation_Child_Nil) ProtoMessage() {} func (x *SetOperation_Child_Nil) ProtoReflect() protoreflect.Message { - mi := &file_core_v1_core_proto_msgTypes[37] + mi := &file_core_v1_core_proto_msgTypes[38] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2800,7 +2876,7 @@ func (x *SetOperation_Child_Nil) ProtoReflect() protoreflect.Message { // Deprecated: Use SetOperation_Child_Nil.ProtoReflect.Descriptor instead. func (*SetOperation_Child_Nil) Descriptor() ([]byte, []int) { - return file_core_v1_core_proto_rawDescGZIP(), []int{22, 0, 1} + return file_core_v1_core_proto_rawDescGZIP(), []int{23, 0, 1} } type TupleToUserset_Tupleset struct { @@ -2814,7 +2890,7 @@ type TupleToUserset_Tupleset struct { func (x *TupleToUserset_Tupleset) Reset() { *x = TupleToUserset_Tupleset{} if protoimpl.UnsafeEnabled { - mi := &file_core_v1_core_proto_msgTypes[38] + mi := &file_core_v1_core_proto_msgTypes[39] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2827,7 +2903,7 @@ func (x *TupleToUserset_Tupleset) String() string { func (*TupleToUserset_Tupleset) ProtoMessage() {} func (x *TupleToUserset_Tupleset) ProtoReflect() protoreflect.Message { - mi := &file_core_v1_core_proto_msgTypes[38] + mi := &file_core_v1_core_proto_msgTypes[39] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2840,7 +2916,7 @@ func (x *TupleToUserset_Tupleset) ProtoReflect() protoreflect.Message { // Deprecated: Use TupleToUserset_Tupleset.ProtoReflect.Descriptor instead. func (*TupleToUserset_Tupleset) Descriptor() ([]byte, []int) { - return file_core_v1_core_proto_rawDescGZIP(), []int{23, 0} + return file_core_v1_core_proto_rawDescGZIP(), []int{24, 0} } func (x *TupleToUserset_Tupleset) GetRelation() string { @@ -2861,7 +2937,7 @@ type FunctionedTupleToUserset_Tupleset struct { func (x *FunctionedTupleToUserset_Tupleset) Reset() { *x = FunctionedTupleToUserset_Tupleset{} if protoimpl.UnsafeEnabled { - mi := &file_core_v1_core_proto_msgTypes[39] + mi := &file_core_v1_core_proto_msgTypes[40] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2874,7 +2950,7 @@ func (x *FunctionedTupleToUserset_Tupleset) String() string { func (*FunctionedTupleToUserset_Tupleset) ProtoMessage() {} func (x *FunctionedTupleToUserset_Tupleset) ProtoReflect() protoreflect.Message { - mi := &file_core_v1_core_proto_msgTypes[39] + mi := &file_core_v1_core_proto_msgTypes[40] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2887,7 +2963,7 @@ func (x *FunctionedTupleToUserset_Tupleset) ProtoReflect() protoreflect.Message // Deprecated: Use FunctionedTupleToUserset_Tupleset.ProtoReflect.Descriptor instead. func (*FunctionedTupleToUserset_Tupleset) Descriptor() ([]byte, []int) { - return file_core_v1_core_proto_rawDescGZIP(), []int{24, 0} + return file_core_v1_core_proto_rawDescGZIP(), []int{25, 0} } func (x *FunctionedTupleToUserset_Tupleset) GetRelation() string { @@ -2908,7 +2984,7 @@ type SubjectFilter_RelationFilter struct { func (x *SubjectFilter_RelationFilter) Reset() { *x = SubjectFilter_RelationFilter{} if protoimpl.UnsafeEnabled { - mi := &file_core_v1_core_proto_msgTypes[40] + mi := &file_core_v1_core_proto_msgTypes[41] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2921,7 +2997,7 @@ func (x *SubjectFilter_RelationFilter) String() string { func (*SubjectFilter_RelationFilter) ProtoMessage() {} func (x *SubjectFilter_RelationFilter) ProtoReflect() protoreflect.Message { - mi := &file_core_v1_core_proto_msgTypes[40] + mi := &file_core_v1_core_proto_msgTypes[41] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2934,7 +3010,7 @@ func (x *SubjectFilter_RelationFilter) ProtoReflect() protoreflect.Message { // Deprecated: Use SubjectFilter_RelationFilter.ProtoReflect.Descriptor instead. func (*SubjectFilter_RelationFilter) Descriptor() ([]byte, []int) { - return file_core_v1_core_proto_rawDescGZIP(), []int{30, 0} + return file_core_v1_core_proto_rawDescGZIP(), []int{31, 0} } func (x *SubjectFilter_RelationFilter) GetRelation() string { @@ -2952,564 +3028,578 @@ var file_core_v1_core_proto_rawDesc = []byte{ 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2f, 0x61, 0x6e, 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, 0x17, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x65, - 0x2f, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x22, - 0xea, 0x01, 0x0a, 0x0d, 0x52, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x75, 0x70, 0x6c, - 0x65, 0x12, 0x58, 0x0a, 0x15, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x61, 0x6e, - 0x64, 0x5f, 0x72, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, - 0x32, 0x1a, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x4f, 0x62, 0x6a, 0x65, 0x63, - 0x74, 0x41, 0x6e, 0x64, 0x52, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x42, 0x08, 0xfa, 0x42, - 0x05, 0x8a, 0x01, 0x02, 0x10, 0x01, 0x52, 0x13, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, - 0x41, 0x6e, 0x64, 0x52, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x3e, 0x0a, 0x07, 0x73, - 0x75, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x63, - 0x6f, 0x72, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x4f, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x41, 0x6e, 0x64, - 0x52, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x42, 0x08, 0xfa, 0x42, 0x05, 0x8a, 0x01, 0x02, - 0x10, 0x01, 0x52, 0x07, 0x73, 0x75, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x12, 0x3f, 0x0a, 0x06, 0x63, - 0x61, 0x76, 0x65, 0x61, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x63, 0x6f, - 0x72, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x43, 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74, 0x75, 0x61, 0x6c, - 0x69, 0x7a, 0x65, 0x64, 0x43, 0x61, 0x76, 0x65, 0x61, 0x74, 0x42, 0x08, 0xfa, 0x42, 0x05, 0x8a, - 0x01, 0x02, 0x10, 0x00, 0x52, 0x06, 0x63, 0x61, 0x76, 0x65, 0x61, 0x74, 0x22, 0xab, 0x01, 0x0a, - 0x14, 0x43, 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74, 0x75, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x64, 0x43, - 0x61, 0x76, 0x65, 0x61, 0x74, 0x12, 0x56, 0x0a, 0x0b, 0x63, 0x61, 0x76, 0x65, 0x61, 0x74, 0x5f, - 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x42, 0x35, 0xfa, 0x42, 0x32, 0x72, - 0x30, 0x28, 0x80, 0x01, 0x32, 0x2b, 0x5e, 0x28, 0x28, 0x5b, 0x61, 0x2d, 0x7a, 0x41, 0x2d, 0x5a, - 0x30, 0x2d, 0x39, 0x5f, 0x5d, 0x5b, 0x61, 0x2d, 0x7a, 0x41, 0x2d, 0x5a, 0x30, 0x2d, 0x39, 0x2f, - 0x5f, 0x7c, 0x2d, 0x5d, 0x7b, 0x30, 0x2c, 0x31, 0x32, 0x37, 0x7d, 0x29, 0x7c, 0x5c, 0x2a, 0x29, - 0x24, 0x52, 0x0a, 0x63, 0x61, 0x76, 0x65, 0x61, 0x74, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x3b, 0x0a, - 0x07, 0x63, 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74, 0x18, 0x02, 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, 0x42, 0x08, 0xfa, 0x42, 0x05, 0x8a, 0x01, 0x02, 0x10, - 0x00, 0x52, 0x07, 0x63, 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74, 0x22, 0xd4, 0x03, 0x0a, 0x10, 0x43, - 0x61, 0x76, 0x65, 0x61, 0x74, 0x44, 0x65, 0x66, 0x69, 0x6e, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, - 0x49, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x42, 0x35, 0xfa, - 0x42, 0x32, 0x72, 0x30, 0x28, 0x80, 0x01, 0x32, 0x2b, 0x5e, 0x28, 0x28, 0x5b, 0x61, 0x2d, 0x7a, - 0x41, 0x2d, 0x5a, 0x30, 0x2d, 0x39, 0x5f, 0x5d, 0x5b, 0x61, 0x2d, 0x7a, 0x41, 0x2d, 0x5a, 0x30, - 0x2d, 0x39, 0x2f, 0x5f, 0x7c, 0x2d, 0x5d, 0x7b, 0x30, 0x2c, 0x31, 0x32, 0x37, 0x7d, 0x29, 0x7c, - 0x5c, 0x2a, 0x29, 0x24, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x3f, 0x0a, 0x15, 0x73, 0x65, - 0x72, 0x69, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x64, 0x5f, 0x65, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73, - 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x0a, 0xfa, 0x42, 0x07, 0x7a, 0x05, - 0x10, 0x00, 0x18, 0x80, 0x20, 0x52, 0x14, 0x73, 0x65, 0x72, 0x69, 0x61, 0x6c, 0x69, 0x7a, 0x65, - 0x64, 0x45, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x62, 0x0a, 0x0f, 0x70, - 0x61, 0x72, 0x61, 0x6d, 0x65, 0x74, 0x65, 0x72, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x73, 0x18, 0x03, - 0x20, 0x03, 0x28, 0x0b, 0x32, 0x2d, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x43, - 0x61, 0x76, 0x65, 0x61, 0x74, 0x44, 0x65, 0x66, 0x69, 0x6e, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x2e, - 0x50, 0x61, 0x72, 0x61, 0x6d, 0x65, 0x74, 0x65, 0x72, 0x54, 0x79, 0x70, 0x65, 0x73, 0x45, 0x6e, - 0x74, 0x72, 0x79, 0x42, 0x0a, 0xfa, 0x42, 0x07, 0x9a, 0x01, 0x04, 0x08, 0x01, 0x10, 0x14, 0x52, - 0x0e, 0x70, 0x61, 0x72, 0x61, 0x6d, 0x65, 0x74, 0x65, 0x72, 0x54, 0x79, 0x70, 0x65, 0x73, 0x12, - 0x2d, 0x0a, 0x08, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x18, 0x04, 0x20, 0x01, 0x28, - 0x0b, 0x32, 0x11, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x4d, 0x65, 0x74, 0x61, - 0x64, 0x61, 0x74, 0x61, 0x52, 0x08, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x12, 0x40, - 0x0a, 0x0f, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, - 0x6e, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x76, - 0x31, 0x2e, 0x53, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x50, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, - 0x52, 0x0e, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x50, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, - 0x1a, 0x5f, 0x0a, 0x13, 0x50, 0x61, 0x72, 0x61, 0x6d, 0x65, 0x74, 0x65, 0x72, 0x54, 0x79, 0x70, - 0x65, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, - 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x32, 0x0a, 0x05, 0x76, 0x61, 0x6c, - 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, - 0x76, 0x31, 0x2e, 0x43, 0x61, 0x76, 0x65, 0x61, 0x74, 0x54, 0x79, 0x70, 0x65, 0x52, 0x65, 0x66, - 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, - 0x01, 0x22, 0x7d, 0x0a, 0x13, 0x43, 0x61, 0x76, 0x65, 0x61, 0x74, 0x54, 0x79, 0x70, 0x65, 0x52, - 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65, 0x12, 0x1b, 0x0a, 0x09, 0x74, 0x79, 0x70, 0x65, - 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x74, 0x79, 0x70, - 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x49, 0x0a, 0x0b, 0x63, 0x68, 0x69, 0x6c, 0x64, 0x5f, 0x74, - 0x79, 0x70, 0x65, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x63, 0x6f, 0x72, - 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x43, 0x61, 0x76, 0x65, 0x61, 0x74, 0x54, 0x79, 0x70, 0x65, 0x52, - 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65, 0x42, 0x0a, 0xfa, 0x42, 0x07, 0x92, 0x01, 0x04, - 0x08, 0x00, 0x10, 0x01, 0x52, 0x0a, 0x63, 0x68, 0x69, 0x6c, 0x64, 0x54, 0x79, 0x70, 0x65, 0x73, - 0x22, 0x91, 0x02, 0x0a, 0x11, 0x4f, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x41, 0x6e, 0x64, 0x52, 0x65, - 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x66, 0x0a, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, - 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x42, 0x48, 0xfa, 0x42, 0x45, 0x72, 0x43, - 0x28, 0x80, 0x01, 0x32, 0x3e, 0x5e, 0x28, 0x5b, 0x61, 0x2d, 0x7a, 0x5d, 0x5b, 0x61, 0x2d, 0x7a, - 0x30, 0x2d, 0x39, 0x5f, 0x5d, 0x7b, 0x31, 0x2c, 0x36, 0x31, 0x7d, 0x5b, 0x61, 0x2d, 0x7a, 0x30, - 0x2d, 0x39, 0x5d, 0x2f, 0x29, 0x2a, 0x5b, 0x61, 0x2d, 0x7a, 0x5d, 0x5b, 0x61, 0x2d, 0x7a, 0x30, - 0x2d, 0x39, 0x5f, 0x5d, 0x7b, 0x31, 0x2c, 0x36, 0x32, 0x7d, 0x5b, 0x61, 0x2d, 0x7a, 0x30, 0x2d, - 0x39, 0x5d, 0x24, 0x52, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x46, - 0x0a, 0x09, 0x6f, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, - 0x09, 0x42, 0x29, 0xfa, 0x42, 0x26, 0x72, 0x24, 0x28, 0x80, 0x08, 0x32, 0x1f, 0x5e, 0x28, 0x28, - 0x5b, 0x61, 0x2d, 0x7a, 0x41, 0x2d, 0x5a, 0x30, 0x2d, 0x39, 0x2f, 0x5f, 0x7c, 0x5c, 0x2d, 0x3d, - 0x2b, 0x5d, 0x7b, 0x31, 0x2c, 0x7d, 0x29, 0x7c, 0x5c, 0x2a, 0x29, 0x24, 0x52, 0x08, 0x6f, 0x62, - 0x6a, 0x65, 0x63, 0x74, 0x49, 0x64, 0x12, 0x4c, 0x0a, 0x08, 0x72, 0x65, 0x6c, 0x61, 0x74, 0x69, - 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x42, 0x30, 0xfa, 0x42, 0x2d, 0x72, 0x2b, 0x28, - 0x40, 0x32, 0x27, 0x5e, 0x28, 0x5c, 0x2e, 0x5c, 0x2e, 0x5c, 0x2e, 0x7c, 0x5b, 0x61, 0x2d, 0x7a, - 0x5d, 0x5b, 0x61, 0x2d, 0x7a, 0x30, 0x2d, 0x39, 0x5f, 0x5d, 0x7b, 0x31, 0x2c, 0x36, 0x32, 0x7d, - 0x5b, 0x61, 0x2d, 0x7a, 0x30, 0x2d, 0x39, 0x5d, 0x29, 0x24, 0x52, 0x08, 0x72, 0x65, 0x6c, 0x61, - 0x74, 0x69, 0x6f, 0x6e, 0x22, 0xc9, 0x01, 0x0a, 0x11, 0x52, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, - 0x6e, 0x52, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65, 0x12, 0x66, 0x0a, 0x09, 0x6e, 0x61, - 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x42, 0x48, 0xfa, - 0x42, 0x45, 0x72, 0x43, 0x28, 0x80, 0x01, 0x32, 0x3e, 0x5e, 0x28, 0x5b, 0x61, 0x2d, 0x7a, 0x5d, - 0x5b, 0x61, 0x2d, 0x7a, 0x30, 0x2d, 0x39, 0x5f, 0x5d, 0x7b, 0x31, 0x2c, 0x36, 0x31, 0x7d, 0x5b, - 0x61, 0x2d, 0x7a, 0x30, 0x2d, 0x39, 0x5d, 0x2f, 0x29, 0x2a, 0x5b, 0x61, 0x2d, 0x7a, 0x5d, 0x5b, - 0x61, 0x2d, 0x7a, 0x30, 0x2d, 0x39, 0x5f, 0x5d, 0x7b, 0x31, 0x2c, 0x36, 0x32, 0x7d, 0x5b, 0x61, - 0x2d, 0x7a, 0x30, 0x2d, 0x39, 0x5d, 0x24, 0x52, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, - 0x63, 0x65, 0x12, 0x4c, 0x0a, 0x08, 0x72, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x03, - 0x20, 0x01, 0x28, 0x09, 0x42, 0x30, 0xfa, 0x42, 0x2d, 0x72, 0x2b, 0x28, 0x40, 0x32, 0x27, 0x5e, - 0x28, 0x5c, 0x2e, 0x5c, 0x2e, 0x5c, 0x2e, 0x7c, 0x5b, 0x61, 0x2d, 0x7a, 0x5d, 0x5b, 0x61, 0x2d, - 0x7a, 0x30, 0x2d, 0x39, 0x5f, 0x5d, 0x7b, 0x31, 0x2c, 0x36, 0x32, 0x7d, 0x5b, 0x61, 0x2d, 0x7a, - 0x30, 0x2d, 0x39, 0x5d, 0x29, 0x24, 0x52, 0x08, 0x72, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, - 0x22, 0x27, 0x0a, 0x06, 0x5a, 0x6f, 0x6f, 0x6b, 0x69, 0x65, 0x12, 0x1d, 0x0a, 0x05, 0x74, 0x6f, - 0x6b, 0x65, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x42, 0x07, 0xfa, 0x42, 0x04, 0x72, 0x02, - 0x20, 0x01, 0x52, 0x05, 0x74, 0x6f, 0x6b, 0x65, 0x6e, 0x22, 0xda, 0x01, 0x0a, 0x13, 0x52, 0x65, - 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x75, 0x70, 0x6c, 0x65, 0x55, 0x70, 0x64, 0x61, 0x74, - 0x65, 0x12, 0x4e, 0x0a, 0x09, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x01, - 0x20, 0x01, 0x28, 0x0e, 0x32, 0x26, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x52, - 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x75, 0x70, 0x6c, 0x65, 0x55, 0x70, 0x64, 0x61, - 0x74, 0x65, 0x2e, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x42, 0x08, 0xfa, 0x42, - 0x05, 0x82, 0x01, 0x02, 0x10, 0x01, 0x52, 0x09, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, - 0x6e, 0x12, 0x36, 0x0a, 0x05, 0x74, 0x75, 0x70, 0x6c, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, - 0x32, 0x16, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x52, 0x65, 0x6c, 0x61, 0x74, - 0x69, 0x6f, 0x6e, 0x54, 0x75, 0x70, 0x6c, 0x65, 0x42, 0x08, 0xfa, 0x42, 0x05, 0x8a, 0x01, 0x02, - 0x10, 0x01, 0x52, 0x05, 0x74, 0x75, 0x70, 0x6c, 0x65, 0x22, 0x3b, 0x0a, 0x09, 0x4f, 0x70, 0x65, - 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x0b, 0x0a, 0x07, 0x55, 0x4e, 0x4b, 0x4e, 0x4f, 0x57, - 0x4e, 0x10, 0x00, 0x12, 0x0a, 0x0a, 0x06, 0x43, 0x52, 0x45, 0x41, 0x54, 0x45, 0x10, 0x01, 0x12, - 0x09, 0x0a, 0x05, 0x54, 0x4f, 0x55, 0x43, 0x48, 0x10, 0x02, 0x12, 0x0a, 0x0a, 0x06, 0x44, 0x45, - 0x4c, 0x45, 0x54, 0x45, 0x10, 0x03, 0x22, 0xa9, 0x02, 0x0a, 0x15, 0x52, 0x65, 0x6c, 0x61, 0x74, - 0x69, 0x6f, 0x6e, 0x54, 0x75, 0x70, 0x6c, 0x65, 0x54, 0x72, 0x65, 0x65, 0x4e, 0x6f, 0x64, 0x65, - 0x12, 0x4b, 0x0a, 0x11, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6d, 0x65, 0x64, 0x69, 0x61, 0x74, 0x65, - 0x5f, 0x6e, 0x6f, 0x64, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x63, 0x6f, - 0x72, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x65, 0x74, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, - 0x6f, 0x6e, 0x55, 0x73, 0x65, 0x72, 0x73, 0x65, 0x74, 0x48, 0x00, 0x52, 0x10, 0x69, 0x6e, 0x74, - 0x65, 0x72, 0x6d, 0x65, 0x64, 0x69, 0x61, 0x74, 0x65, 0x4e, 0x6f, 0x64, 0x65, 0x12, 0x36, 0x0a, - 0x09, 0x6c, 0x65, 0x61, 0x66, 0x5f, 0x6e, 0x6f, 0x64, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, - 0x32, 0x17, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x44, 0x69, 0x72, 0x65, 0x63, - 0x74, 0x53, 0x75, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x73, 0x48, 0x00, 0x52, 0x08, 0x6c, 0x65, 0x61, - 0x66, 0x4e, 0x6f, 0x64, 0x65, 0x12, 0x36, 0x0a, 0x08, 0x65, 0x78, 0x70, 0x61, 0x6e, 0x64, 0x65, - 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x76, - 0x31, 0x2e, 0x4f, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x41, 0x6e, 0x64, 0x52, 0x65, 0x6c, 0x61, 0x74, - 0x69, 0x6f, 0x6e, 0x52, 0x08, 0x65, 0x78, 0x70, 0x61, 0x6e, 0x64, 0x65, 0x64, 0x12, 0x46, 0x0a, - 0x11, 0x63, 0x61, 0x76, 0x65, 0x61, 0x74, 0x5f, 0x65, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, - 0x6f, 0x6e, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, - 0x76, 0x31, 0x2e, 0x43, 0x61, 0x76, 0x65, 0x61, 0x74, 0x45, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73, - 0x69, 0x6f, 0x6e, 0x52, 0x10, 0x63, 0x61, 0x76, 0x65, 0x61, 0x74, 0x45, 0x78, 0x70, 0x72, 0x65, - 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x42, 0x0b, 0x0a, 0x09, 0x6e, 0x6f, 0x64, 0x65, 0x5f, 0x74, 0x79, - 0x70, 0x65, 0x22, 0xe2, 0x01, 0x0a, 0x13, 0x53, 0x65, 0x74, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, - 0x69, 0x6f, 0x6e, 0x55, 0x73, 0x65, 0x72, 0x73, 0x65, 0x74, 0x12, 0x44, 0x0a, 0x09, 0x6f, 0x70, - 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x26, 0x2e, - 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x65, 0x74, 0x4f, 0x70, 0x65, 0x72, 0x61, - 0x74, 0x69, 0x6f, 0x6e, 0x55, 0x73, 0x65, 0x72, 0x73, 0x65, 0x74, 0x2e, 0x4f, 0x70, 0x65, 0x72, - 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x09, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, - 0x12, 0x3f, 0x0a, 0x0b, 0x63, 0x68, 0x69, 0x6c, 0x64, 0x5f, 0x6e, 0x6f, 0x64, 0x65, 0x73, 0x18, - 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x76, 0x31, 0x2e, - 0x52, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x75, 0x70, 0x6c, 0x65, 0x54, 0x72, 0x65, - 0x65, 0x4e, 0x6f, 0x64, 0x65, 0x52, 0x0a, 0x63, 0x68, 0x69, 0x6c, 0x64, 0x4e, 0x6f, 0x64, 0x65, - 0x73, 0x22, 0x44, 0x0a, 0x09, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x0b, - 0x0a, 0x07, 0x49, 0x4e, 0x56, 0x41, 0x4c, 0x49, 0x44, 0x10, 0x00, 0x12, 0x09, 0x0a, 0x05, 0x55, - 0x4e, 0x49, 0x4f, 0x4e, 0x10, 0x01, 0x12, 0x10, 0x0a, 0x0c, 0x49, 0x4e, 0x54, 0x45, 0x52, 0x53, - 0x45, 0x43, 0x54, 0x49, 0x4f, 0x4e, 0x10, 0x02, 0x12, 0x0d, 0x0a, 0x09, 0x45, 0x58, 0x43, 0x4c, - 0x55, 0x53, 0x49, 0x4f, 0x4e, 0x10, 0x03, 0x22, 0x8d, 0x01, 0x0a, 0x0d, 0x44, 0x69, 0x72, 0x65, - 0x63, 0x74, 0x53, 0x75, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x12, 0x34, 0x0a, 0x07, 0x73, 0x75, 0x62, - 0x6a, 0x65, 0x63, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x63, 0x6f, 0x72, - 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x4f, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x41, 0x6e, 0x64, 0x52, 0x65, - 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x07, 0x73, 0x75, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x12, - 0x46, 0x0a, 0x11, 0x63, 0x61, 0x76, 0x65, 0x61, 0x74, 0x5f, 0x65, 0x78, 0x70, 0x72, 0x65, 0x73, - 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x63, 0x6f, 0x72, - 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x43, 0x61, 0x76, 0x65, 0x61, 0x74, 0x45, 0x78, 0x70, 0x72, 0x65, - 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x52, 0x10, 0x63, 0x61, 0x76, 0x65, 0x61, 0x74, 0x45, 0x78, 0x70, - 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x22, 0x44, 0x0a, 0x0e, 0x44, 0x69, 0x72, 0x65, 0x63, - 0x74, 0x53, 0x75, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x73, 0x12, 0x32, 0x0a, 0x08, 0x73, 0x75, 0x62, - 0x6a, 0x65, 0x63, 0x74, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x16, 0x2e, 0x63, 0x6f, - 0x72, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x44, 0x69, 0x72, 0x65, 0x63, 0x74, 0x53, 0x75, 0x62, 0x6a, - 0x65, 0x63, 0x74, 0x52, 0x08, 0x73, 0x75, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x73, 0x22, 0xb8, 0x01, - 0x0a, 0x08, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x12, 0xab, 0x01, 0x0a, 0x10, 0x6d, - 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x5f, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x18, - 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x14, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, - 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x41, 0x6e, 0x79, 0x42, 0x6a, 0xfa, 0x42, 0x67, - 0x92, 0x01, 0x64, 0x08, 0x01, 0x22, 0x60, 0x8a, 0x01, 0x02, 0x10, 0x01, 0xa2, 0x01, 0x58, 0x08, - 0x01, 0x12, 0x26, 0x74, 0x79, 0x70, 0x65, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x61, 0x70, - 0x69, 0x73, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x69, 0x6d, 0x70, 0x6c, 0x2e, 0x76, 0x31, 0x2e, 0x44, - 0x6f, 0x63, 0x43, 0x6f, 0x6d, 0x6d, 0x65, 0x6e, 0x74, 0x12, 0x2c, 0x74, 0x79, 0x70, 0x65, 0x2e, - 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x61, 0x70, 0x69, 0x73, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x69, - 0x6d, 0x70, 0x6c, 0x2e, 0x76, 0x31, 0x2e, 0x52, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x4d, - 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x52, 0x0f, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, - 0x61, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0x93, 0x02, 0x0a, 0x13, 0x4e, 0x61, 0x6d, - 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x44, 0x65, 0x66, 0x69, 0x6e, 0x69, 0x74, 0x69, 0x6f, 0x6e, - 0x12, 0x5c, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x42, 0x48, - 0xfa, 0x42, 0x45, 0x72, 0x43, 0x28, 0x80, 0x01, 0x32, 0x3e, 0x5e, 0x28, 0x5b, 0x61, 0x2d, 0x7a, + 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, 0x17, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, + 0x65, 0x2f, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x22, 0xb2, 0x02, 0x0a, 0x0d, 0x52, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x75, 0x70, + 0x6c, 0x65, 0x12, 0x58, 0x0a, 0x15, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x61, + 0x6e, 0x64, 0x5f, 0x72, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x1a, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x4f, 0x62, 0x6a, 0x65, + 0x63, 0x74, 0x41, 0x6e, 0x64, 0x52, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x42, 0x08, 0xfa, + 0x42, 0x05, 0x8a, 0x01, 0x02, 0x10, 0x01, 0x52, 0x13, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, + 0x65, 0x41, 0x6e, 0x64, 0x52, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x3e, 0x0a, 0x07, + 0x73, 0x75, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, + 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x4f, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x41, 0x6e, + 0x64, 0x52, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x42, 0x08, 0xfa, 0x42, 0x05, 0x8a, 0x01, + 0x02, 0x10, 0x01, 0x52, 0x07, 0x73, 0x75, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x12, 0x3f, 0x0a, 0x06, + 0x63, 0x61, 0x76, 0x65, 0x61, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x63, + 0x6f, 0x72, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x43, 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74, 0x75, 0x61, + 0x6c, 0x69, 0x7a, 0x65, 0x64, 0x43, 0x61, 0x76, 0x65, 0x61, 0x74, 0x42, 0x08, 0xfa, 0x42, 0x05, + 0x8a, 0x01, 0x02, 0x10, 0x00, 0x52, 0x06, 0x63, 0x61, 0x76, 0x65, 0x61, 0x74, 0x12, 0x46, 0x0a, + 0x09, 0x69, 0x6e, 0x74, 0x65, 0x67, 0x72, 0x69, 0x74, 0x79, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x1e, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x52, 0x65, 0x6c, 0x61, 0x74, + 0x69, 0x6f, 0x6e, 0x73, 0x68, 0x69, 0x70, 0x49, 0x6e, 0x74, 0x65, 0x67, 0x72, 0x69, 0x74, 0x79, + 0x42, 0x08, 0xfa, 0x42, 0x05, 0x8a, 0x01, 0x02, 0x10, 0x00, 0x52, 0x09, 0x69, 0x6e, 0x74, 0x65, + 0x67, 0x72, 0x69, 0x74, 0x79, 0x22, 0x7b, 0x0a, 0x15, 0x52, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, + 0x6e, 0x73, 0x68, 0x69, 0x70, 0x49, 0x6e, 0x74, 0x65, 0x67, 0x72, 0x69, 0x74, 0x79, 0x12, 0x15, + 0x0a, 0x06, 0x6b, 0x65, 0x79, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, + 0x6b, 0x65, 0x79, 0x49, 0x64, 0x12, 0x12, 0x0a, 0x04, 0x68, 0x61, 0x73, 0x68, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x0c, 0x52, 0x04, 0x68, 0x61, 0x73, 0x68, 0x12, 0x37, 0x0a, 0x09, 0x68, 0x61, 0x73, + 0x68, 0x65, 0x64, 0x5f, 0x61, 0x74, 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, 0x08, 0x68, 0x61, 0x73, 0x68, 0x65, 0x64, + 0x41, 0x74, 0x22, 0xab, 0x01, 0x0a, 0x14, 0x43, 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74, 0x75, 0x61, + 0x6c, 0x69, 0x7a, 0x65, 0x64, 0x43, 0x61, 0x76, 0x65, 0x61, 0x74, 0x12, 0x56, 0x0a, 0x0b, 0x63, + 0x61, 0x76, 0x65, 0x61, 0x74, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, + 0x42, 0x35, 0xfa, 0x42, 0x32, 0x72, 0x30, 0x28, 0x80, 0x01, 0x32, 0x2b, 0x5e, 0x28, 0x28, 0x5b, + 0x61, 0x2d, 0x7a, 0x41, 0x2d, 0x5a, 0x30, 0x2d, 0x39, 0x5f, 0x5d, 0x5b, 0x61, 0x2d, 0x7a, 0x41, + 0x2d, 0x5a, 0x30, 0x2d, 0x39, 0x2f, 0x5f, 0x7c, 0x2d, 0x5d, 0x7b, 0x30, 0x2c, 0x31, 0x32, 0x37, + 0x7d, 0x29, 0x7c, 0x5c, 0x2a, 0x29, 0x24, 0x52, 0x0a, 0x63, 0x61, 0x76, 0x65, 0x61, 0x74, 0x4e, + 0x61, 0x6d, 0x65, 0x12, 0x3b, 0x0a, 0x07, 0x63, 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74, 0x18, 0x02, + 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, 0x42, 0x08, 0xfa, + 0x42, 0x05, 0x8a, 0x01, 0x02, 0x10, 0x00, 0x52, 0x07, 0x63, 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74, + 0x22, 0xd4, 0x03, 0x0a, 0x10, 0x43, 0x61, 0x76, 0x65, 0x61, 0x74, 0x44, 0x65, 0x66, 0x69, 0x6e, + 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x49, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x09, 0x42, 0x35, 0xfa, 0x42, 0x32, 0x72, 0x30, 0x28, 0x80, 0x01, 0x32, 0x2b, 0x5e, + 0x28, 0x28, 0x5b, 0x61, 0x2d, 0x7a, 0x41, 0x2d, 0x5a, 0x30, 0x2d, 0x39, 0x5f, 0x5d, 0x5b, 0x61, + 0x2d, 0x7a, 0x41, 0x2d, 0x5a, 0x30, 0x2d, 0x39, 0x2f, 0x5f, 0x7c, 0x2d, 0x5d, 0x7b, 0x30, 0x2c, + 0x31, 0x32, 0x37, 0x7d, 0x29, 0x7c, 0x5c, 0x2a, 0x29, 0x24, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, + 0x12, 0x3f, 0x0a, 0x15, 0x73, 0x65, 0x72, 0x69, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x64, 0x5f, 0x65, + 0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x42, + 0x0a, 0xfa, 0x42, 0x07, 0x7a, 0x05, 0x10, 0x00, 0x18, 0x80, 0x20, 0x52, 0x14, 0x73, 0x65, 0x72, + 0x69, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x64, 0x45, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, + 0x6e, 0x12, 0x62, 0x0a, 0x0f, 0x70, 0x61, 0x72, 0x61, 0x6d, 0x65, 0x74, 0x65, 0x72, 0x5f, 0x74, + 0x79, 0x70, 0x65, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x2d, 0x2e, 0x63, 0x6f, 0x72, + 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x43, 0x61, 0x76, 0x65, 0x61, 0x74, 0x44, 0x65, 0x66, 0x69, 0x6e, + 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x50, 0x61, 0x72, 0x61, 0x6d, 0x65, 0x74, 0x65, 0x72, 0x54, + 0x79, 0x70, 0x65, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x42, 0x0a, 0xfa, 0x42, 0x07, 0x9a, 0x01, + 0x04, 0x08, 0x01, 0x10, 0x14, 0x52, 0x0e, 0x70, 0x61, 0x72, 0x61, 0x6d, 0x65, 0x74, 0x65, 0x72, + 0x54, 0x79, 0x70, 0x65, 0x73, 0x12, 0x2d, 0x0a, 0x08, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, + 0x61, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x11, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x76, + 0x31, 0x2e, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x52, 0x08, 0x6d, 0x65, 0x74, 0x61, + 0x64, 0x61, 0x74, 0x61, 0x12, 0x40, 0x0a, 0x0f, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x70, + 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, + 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x50, 0x6f, + 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0e, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x50, 0x6f, + 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x1a, 0x5f, 0x0a, 0x13, 0x50, 0x61, 0x72, 0x61, 0x6d, 0x65, + 0x74, 0x65, 0x72, 0x54, 0x79, 0x70, 0x65, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, + 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, + 0x32, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, + 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x43, 0x61, 0x76, 0x65, 0x61, 0x74, 0x54, + 0x79, 0x70, 0x65, 0x52, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65, 0x52, 0x05, 0x76, 0x61, + 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x7d, 0x0a, 0x13, 0x43, 0x61, 0x76, 0x65, 0x61, + 0x74, 0x54, 0x79, 0x70, 0x65, 0x52, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65, 0x12, 0x1b, + 0x0a, 0x09, 0x74, 0x79, 0x70, 0x65, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x08, 0x74, 0x79, 0x70, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x49, 0x0a, 0x0b, 0x63, + 0x68, 0x69, 0x6c, 0x64, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, + 0x32, 0x1c, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x43, 0x61, 0x76, 0x65, 0x61, + 0x74, 0x54, 0x79, 0x70, 0x65, 0x52, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65, 0x42, 0x0a, + 0xfa, 0x42, 0x07, 0x92, 0x01, 0x04, 0x08, 0x00, 0x10, 0x01, 0x52, 0x0a, 0x63, 0x68, 0x69, 0x6c, + 0x64, 0x54, 0x79, 0x70, 0x65, 0x73, 0x22, 0x91, 0x02, 0x0a, 0x11, 0x4f, 0x62, 0x6a, 0x65, 0x63, + 0x74, 0x41, 0x6e, 0x64, 0x52, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x66, 0x0a, 0x09, + 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x42, + 0x48, 0xfa, 0x42, 0x45, 0x72, 0x43, 0x28, 0x80, 0x01, 0x32, 0x3e, 0x5e, 0x28, 0x5b, 0x61, 0x2d, + 0x7a, 0x5d, 0x5b, 0x61, 0x2d, 0x7a, 0x30, 0x2d, 0x39, 0x5f, 0x5d, 0x7b, 0x31, 0x2c, 0x36, 0x31, + 0x7d, 0x5b, 0x61, 0x2d, 0x7a, 0x30, 0x2d, 0x39, 0x5d, 0x2f, 0x29, 0x2a, 0x5b, 0x61, 0x2d, 0x7a, 0x5d, 0x5b, 0x61, 0x2d, 0x7a, 0x30, 0x2d, 0x39, 0x5f, 0x5d, 0x7b, 0x31, 0x2c, 0x36, 0x32, 0x7d, - 0x5b, 0x61, 0x2d, 0x7a, 0x30, 0x2d, 0x39, 0x5d, 0x2f, 0x29, 0x2a, 0x5b, 0x61, 0x2d, 0x7a, 0x5d, + 0x5b, 0x61, 0x2d, 0x7a, 0x30, 0x2d, 0x39, 0x5d, 0x24, 0x52, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, + 0x70, 0x61, 0x63, 0x65, 0x12, 0x46, 0x0a, 0x09, 0x6f, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x5f, 0x69, + 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x42, 0x29, 0xfa, 0x42, 0x26, 0x72, 0x24, 0x28, 0x80, + 0x08, 0x32, 0x1f, 0x5e, 0x28, 0x28, 0x5b, 0x61, 0x2d, 0x7a, 0x41, 0x2d, 0x5a, 0x30, 0x2d, 0x39, + 0x2f, 0x5f, 0x7c, 0x5c, 0x2d, 0x3d, 0x2b, 0x5d, 0x7b, 0x31, 0x2c, 0x7d, 0x29, 0x7c, 0x5c, 0x2a, + 0x29, 0x24, 0x52, 0x08, 0x6f, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x49, 0x64, 0x12, 0x4c, 0x0a, 0x08, + 0x72, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x42, 0x30, + 0xfa, 0x42, 0x2d, 0x72, 0x2b, 0x28, 0x40, 0x32, 0x27, 0x5e, 0x28, 0x5c, 0x2e, 0x5c, 0x2e, 0x5c, + 0x2e, 0x7c, 0x5b, 0x61, 0x2d, 0x7a, 0x5d, 0x5b, 0x61, 0x2d, 0x7a, 0x30, 0x2d, 0x39, 0x5f, 0x5d, + 0x7b, 0x31, 0x2c, 0x36, 0x32, 0x7d, 0x5b, 0x61, 0x2d, 0x7a, 0x30, 0x2d, 0x39, 0x5d, 0x29, 0x24, + 0x52, 0x08, 0x72, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x22, 0xc9, 0x01, 0x0a, 0x11, 0x52, + 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65, + 0x12, 0x66, 0x0a, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x09, 0x42, 0x48, 0xfa, 0x42, 0x45, 0x72, 0x43, 0x28, 0x80, 0x01, 0x32, 0x3e, 0x5e, + 0x28, 0x5b, 0x61, 0x2d, 0x7a, 0x5d, 0x5b, 0x61, 0x2d, 0x7a, 0x30, 0x2d, 0x39, 0x5f, 0x5d, 0x7b, + 0x31, 0x2c, 0x36, 0x31, 0x7d, 0x5b, 0x61, 0x2d, 0x7a, 0x30, 0x2d, 0x39, 0x5d, 0x2f, 0x29, 0x2a, + 0x5b, 0x61, 0x2d, 0x7a, 0x5d, 0x5b, 0x61, 0x2d, 0x7a, 0x30, 0x2d, 0x39, 0x5f, 0x5d, 0x7b, 0x31, + 0x2c, 0x36, 0x32, 0x7d, 0x5b, 0x61, 0x2d, 0x7a, 0x30, 0x2d, 0x39, 0x5d, 0x24, 0x52, 0x09, 0x6e, + 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x4c, 0x0a, 0x08, 0x72, 0x65, 0x6c, 0x61, + 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x42, 0x30, 0xfa, 0x42, 0x2d, 0x72, + 0x2b, 0x28, 0x40, 0x32, 0x27, 0x5e, 0x28, 0x5c, 0x2e, 0x5c, 0x2e, 0x5c, 0x2e, 0x7c, 0x5b, 0x61, + 0x2d, 0x7a, 0x5d, 0x5b, 0x61, 0x2d, 0x7a, 0x30, 0x2d, 0x39, 0x5f, 0x5d, 0x7b, 0x31, 0x2c, 0x36, + 0x32, 0x7d, 0x5b, 0x61, 0x2d, 0x7a, 0x30, 0x2d, 0x39, 0x5d, 0x29, 0x24, 0x52, 0x08, 0x72, 0x65, + 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x22, 0x27, 0x0a, 0x06, 0x5a, 0x6f, 0x6f, 0x6b, 0x69, 0x65, + 0x12, 0x1d, 0x0a, 0x05, 0x74, 0x6f, 0x6b, 0x65, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x42, + 0x07, 0xfa, 0x42, 0x04, 0x72, 0x02, 0x20, 0x01, 0x52, 0x05, 0x74, 0x6f, 0x6b, 0x65, 0x6e, 0x22, + 0xda, 0x01, 0x0a, 0x13, 0x52, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x75, 0x70, 0x6c, + 0x65, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x12, 0x4e, 0x0a, 0x09, 0x6f, 0x70, 0x65, 0x72, 0x61, + 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x26, 0x2e, 0x63, 0x6f, 0x72, + 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x52, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x75, 0x70, + 0x6c, 0x65, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x2e, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, + 0x6f, 0x6e, 0x42, 0x08, 0xfa, 0x42, 0x05, 0x82, 0x01, 0x02, 0x10, 0x01, 0x52, 0x09, 0x6f, 0x70, + 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x36, 0x0a, 0x05, 0x74, 0x75, 0x70, 0x6c, 0x65, + 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x16, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x76, 0x31, + 0x2e, 0x52, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x75, 0x70, 0x6c, 0x65, 0x42, 0x08, + 0xfa, 0x42, 0x05, 0x8a, 0x01, 0x02, 0x10, 0x01, 0x52, 0x05, 0x74, 0x75, 0x70, 0x6c, 0x65, 0x22, + 0x3b, 0x0a, 0x09, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x0b, 0x0a, 0x07, + 0x55, 0x4e, 0x4b, 0x4e, 0x4f, 0x57, 0x4e, 0x10, 0x00, 0x12, 0x0a, 0x0a, 0x06, 0x43, 0x52, 0x45, + 0x41, 0x54, 0x45, 0x10, 0x01, 0x12, 0x09, 0x0a, 0x05, 0x54, 0x4f, 0x55, 0x43, 0x48, 0x10, 0x02, + 0x12, 0x0a, 0x0a, 0x06, 0x44, 0x45, 0x4c, 0x45, 0x54, 0x45, 0x10, 0x03, 0x22, 0xa9, 0x02, 0x0a, + 0x15, 0x52, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x75, 0x70, 0x6c, 0x65, 0x54, 0x72, + 0x65, 0x65, 0x4e, 0x6f, 0x64, 0x65, 0x12, 0x4b, 0x0a, 0x11, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6d, + 0x65, 0x64, 0x69, 0x61, 0x74, 0x65, 0x5f, 0x6e, 0x6f, 0x64, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x1c, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x65, 0x74, 0x4f, + 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x55, 0x73, 0x65, 0x72, 0x73, 0x65, 0x74, 0x48, + 0x00, 0x52, 0x10, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6d, 0x65, 0x64, 0x69, 0x61, 0x74, 0x65, 0x4e, + 0x6f, 0x64, 0x65, 0x12, 0x36, 0x0a, 0x09, 0x6c, 0x65, 0x61, 0x66, 0x5f, 0x6e, 0x6f, 0x64, 0x65, + 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x76, 0x31, + 0x2e, 0x44, 0x69, 0x72, 0x65, 0x63, 0x74, 0x53, 0x75, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x73, 0x48, + 0x00, 0x52, 0x08, 0x6c, 0x65, 0x61, 0x66, 0x4e, 0x6f, 0x64, 0x65, 0x12, 0x36, 0x0a, 0x08, 0x65, + 0x78, 0x70, 0x61, 0x6e, 0x64, 0x65, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, + 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x4f, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x41, 0x6e, + 0x64, 0x52, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x08, 0x65, 0x78, 0x70, 0x61, 0x6e, + 0x64, 0x65, 0x64, 0x12, 0x46, 0x0a, 0x11, 0x63, 0x61, 0x76, 0x65, 0x61, 0x74, 0x5f, 0x65, 0x78, + 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, + 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x43, 0x61, 0x76, 0x65, 0x61, 0x74, 0x45, + 0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x52, 0x10, 0x63, 0x61, 0x76, 0x65, 0x61, + 0x74, 0x45, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x42, 0x0b, 0x0a, 0x09, 0x6e, + 0x6f, 0x64, 0x65, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x22, 0xe2, 0x01, 0x0a, 0x13, 0x53, 0x65, 0x74, + 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x55, 0x73, 0x65, 0x72, 0x73, 0x65, 0x74, + 0x12, 0x44, 0x0a, 0x09, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x0e, 0x32, 0x26, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x65, + 0x74, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x55, 0x73, 0x65, 0x72, 0x73, 0x65, + 0x74, 0x2e, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x09, 0x6f, 0x70, 0x65, + 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x3f, 0x0a, 0x0b, 0x63, 0x68, 0x69, 0x6c, 0x64, 0x5f, + 0x6e, 0x6f, 0x64, 0x65, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x63, 0x6f, + 0x72, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x52, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x75, + 0x70, 0x6c, 0x65, 0x54, 0x72, 0x65, 0x65, 0x4e, 0x6f, 0x64, 0x65, 0x52, 0x0a, 0x63, 0x68, 0x69, + 0x6c, 0x64, 0x4e, 0x6f, 0x64, 0x65, 0x73, 0x22, 0x44, 0x0a, 0x09, 0x4f, 0x70, 0x65, 0x72, 0x61, + 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x0b, 0x0a, 0x07, 0x49, 0x4e, 0x56, 0x41, 0x4c, 0x49, 0x44, 0x10, + 0x00, 0x12, 0x09, 0x0a, 0x05, 0x55, 0x4e, 0x49, 0x4f, 0x4e, 0x10, 0x01, 0x12, 0x10, 0x0a, 0x0c, + 0x49, 0x4e, 0x54, 0x45, 0x52, 0x53, 0x45, 0x43, 0x54, 0x49, 0x4f, 0x4e, 0x10, 0x02, 0x12, 0x0d, + 0x0a, 0x09, 0x45, 0x58, 0x43, 0x4c, 0x55, 0x53, 0x49, 0x4f, 0x4e, 0x10, 0x03, 0x22, 0x8d, 0x01, + 0x0a, 0x0d, 0x44, 0x69, 0x72, 0x65, 0x63, 0x74, 0x53, 0x75, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x12, + 0x34, 0x0a, 0x07, 0x73, 0x75, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x1a, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x4f, 0x62, 0x6a, 0x65, 0x63, + 0x74, 0x41, 0x6e, 0x64, 0x52, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x07, 0x73, 0x75, + 0x62, 0x6a, 0x65, 0x63, 0x74, 0x12, 0x46, 0x0a, 0x11, 0x63, 0x61, 0x76, 0x65, 0x61, 0x74, 0x5f, + 0x65, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x19, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x43, 0x61, 0x76, 0x65, 0x61, + 0x74, 0x45, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x52, 0x10, 0x63, 0x61, 0x76, + 0x65, 0x61, 0x74, 0x45, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x22, 0x44, 0x0a, + 0x0e, 0x44, 0x69, 0x72, 0x65, 0x63, 0x74, 0x53, 0x75, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x73, 0x12, + 0x32, 0x0a, 0x08, 0x73, 0x75, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, + 0x0b, 0x32, 0x16, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x44, 0x69, 0x72, 0x65, + 0x63, 0x74, 0x53, 0x75, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x52, 0x08, 0x73, 0x75, 0x62, 0x6a, 0x65, + 0x63, 0x74, 0x73, 0x22, 0xb8, 0x01, 0x0a, 0x08, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, + 0x12, 0xab, 0x01, 0x0a, 0x10, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x5f, 0x6d, 0x65, + 0x73, 0x73, 0x61, 0x67, 0x65, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x14, 0x2e, 0x67, 0x6f, + 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x41, 0x6e, + 0x79, 0x42, 0x6a, 0xfa, 0x42, 0x67, 0x92, 0x01, 0x64, 0x08, 0x01, 0x22, 0x60, 0x8a, 0x01, 0x02, + 0x10, 0x01, 0xa2, 0x01, 0x58, 0x08, 0x01, 0x12, 0x26, 0x74, 0x79, 0x70, 0x65, 0x2e, 0x67, 0x6f, + 0x6f, 0x67, 0x6c, 0x65, 0x61, 0x70, 0x69, 0x73, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x69, 0x6d, 0x70, + 0x6c, 0x2e, 0x76, 0x31, 0x2e, 0x44, 0x6f, 0x63, 0x43, 0x6f, 0x6d, 0x6d, 0x65, 0x6e, 0x74, 0x12, + 0x2c, 0x74, 0x79, 0x70, 0x65, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x61, 0x70, 0x69, 0x73, + 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x69, 0x6d, 0x70, 0x6c, 0x2e, 0x76, 0x31, 0x2e, 0x52, 0x65, 0x6c, + 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x52, 0x0f, 0x6d, + 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0x93, + 0x02, 0x0a, 0x13, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x44, 0x65, 0x66, 0x69, + 0x6e, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x5c, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x09, 0x42, 0x48, 0xfa, 0x42, 0x45, 0x72, 0x43, 0x28, 0x80, 0x01, 0x32, 0x3e, + 0x5e, 0x28, 0x5b, 0x61, 0x2d, 0x7a, 0x5d, 0x5b, 0x61, 0x2d, 0x7a, 0x30, 0x2d, 0x39, 0x5f, 0x5d, + 0x7b, 0x31, 0x2c, 0x36, 0x32, 0x7d, 0x5b, 0x61, 0x2d, 0x7a, 0x30, 0x2d, 0x39, 0x5d, 0x2f, 0x29, + 0x2a, 0x5b, 0x61, 0x2d, 0x7a, 0x5d, 0x5b, 0x61, 0x2d, 0x7a, 0x30, 0x2d, 0x39, 0x5f, 0x5d, 0x7b, + 0x31, 0x2c, 0x36, 0x32, 0x7d, 0x5b, 0x61, 0x2d, 0x7a, 0x30, 0x2d, 0x39, 0x5d, 0x24, 0x52, 0x04, + 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x2d, 0x0a, 0x08, 0x72, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, + 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x11, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x76, 0x31, + 0x2e, 0x52, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x08, 0x72, 0x65, 0x6c, 0x61, 0x74, + 0x69, 0x6f, 0x6e, 0x12, 0x2d, 0x0a, 0x08, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x18, + 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x11, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x76, 0x31, 0x2e, + 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x52, 0x08, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, + 0x74, 0x61, 0x12, 0x40, 0x0a, 0x0f, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x70, 0x6f, 0x73, + 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x63, 0x6f, + 0x72, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x50, 0x6f, 0x73, 0x69, + 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0e, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x50, 0x6f, 0x73, 0x69, + 0x74, 0x69, 0x6f, 0x6e, 0x22, 0x9c, 0x03, 0x0a, 0x08, 0x52, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, + 0x6e, 0x12, 0x3b, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x42, + 0x27, 0xfa, 0x42, 0x24, 0x72, 0x22, 0x28, 0x40, 0x32, 0x1e, 0x5e, 0x5b, 0x61, 0x2d, 0x7a, 0x5d, 0x5b, 0x61, 0x2d, 0x7a, 0x30, 0x2d, 0x39, 0x5f, 0x5d, 0x7b, 0x31, 0x2c, 0x36, 0x32, 0x7d, 0x5b, - 0x61, 0x2d, 0x7a, 0x30, 0x2d, 0x39, 0x5d, 0x24, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x2d, - 0x0a, 0x08, 0x72, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, - 0x32, 0x11, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x52, 0x65, 0x6c, 0x61, 0x74, - 0x69, 0x6f, 0x6e, 0x52, 0x08, 0x72, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x2d, 0x0a, - 0x08, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, - 0x11, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, - 0x74, 0x61, 0x52, 0x08, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x12, 0x40, 0x0a, 0x0f, - 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, - 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x76, 0x31, 0x2e, - 0x53, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x50, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0e, - 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x50, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x22, 0x9c, - 0x03, 0x0a, 0x08, 0x52, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x3b, 0x0a, 0x04, 0x6e, - 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x42, 0x27, 0xfa, 0x42, 0x24, 0x72, 0x22, - 0x28, 0x40, 0x32, 0x1e, 0x5e, 0x5b, 0x61, 0x2d, 0x7a, 0x5d, 0x5b, 0x61, 0x2d, 0x7a, 0x30, 0x2d, - 0x39, 0x5f, 0x5d, 0x7b, 0x31, 0x2c, 0x36, 0x32, 0x7d, 0x5b, 0x61, 0x2d, 0x7a, 0x30, 0x2d, 0x39, - 0x5d, 0x24, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x40, 0x0a, 0x0f, 0x75, 0x73, 0x65, 0x72, - 0x73, 0x65, 0x74, 0x5f, 0x72, 0x65, 0x77, 0x72, 0x69, 0x74, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, - 0x0b, 0x32, 0x17, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x55, 0x73, 0x65, 0x72, - 0x73, 0x65, 0x74, 0x52, 0x65, 0x77, 0x72, 0x69, 0x74, 0x65, 0x52, 0x0e, 0x75, 0x73, 0x65, 0x72, - 0x73, 0x65, 0x74, 0x52, 0x65, 0x77, 0x72, 0x69, 0x74, 0x65, 0x12, 0x43, 0x0a, 0x10, 0x74, 0x79, - 0x70, 0x65, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x72, 0x6d, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x03, - 0x20, 0x01, 0x28, 0x0b, 0x32, 0x18, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x54, - 0x79, 0x70, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x72, 0x6d, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0f, - 0x74, 0x79, 0x70, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x72, 0x6d, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, - 0x2d, 0x0a, 0x08, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x18, 0x04, 0x20, 0x01, 0x28, - 0x0b, 0x32, 0x11, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x4d, 0x65, 0x74, 0x61, - 0x64, 0x61, 0x74, 0x61, 0x52, 0x08, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x12, 0x40, + 0x61, 0x2d, 0x7a, 0x30, 0x2d, 0x39, 0x5d, 0x24, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x40, + 0x0a, 0x0f, 0x75, 0x73, 0x65, 0x72, 0x73, 0x65, 0x74, 0x5f, 0x72, 0x65, 0x77, 0x72, 0x69, 0x74, + 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x76, + 0x31, 0x2e, 0x55, 0x73, 0x65, 0x72, 0x73, 0x65, 0x74, 0x52, 0x65, 0x77, 0x72, 0x69, 0x74, 0x65, + 0x52, 0x0e, 0x75, 0x73, 0x65, 0x72, 0x73, 0x65, 0x74, 0x52, 0x65, 0x77, 0x72, 0x69, 0x74, 0x65, + 0x12, 0x43, 0x0a, 0x10, 0x74, 0x79, 0x70, 0x65, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x72, 0x6d, 0x61, + 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x18, 0x2e, 0x63, 0x6f, 0x72, + 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x54, 0x79, 0x70, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x72, 0x6d, 0x61, + 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0f, 0x74, 0x79, 0x70, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x72, 0x6d, + 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x2d, 0x0a, 0x08, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, + 0x61, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x11, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x76, + 0x31, 0x2e, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x52, 0x08, 0x6d, 0x65, 0x74, 0x61, + 0x64, 0x61, 0x74, 0x61, 0x12, 0x40, 0x0a, 0x0f, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x70, + 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, + 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x50, 0x6f, + 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0e, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x50, 0x6f, + 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x2b, 0x0a, 0x11, 0x61, 0x6c, 0x69, 0x61, 0x73, 0x69, + 0x6e, 0x67, 0x5f, 0x72, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x06, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x10, 0x61, 0x6c, 0x69, 0x61, 0x73, 0x69, 0x6e, 0x67, 0x52, 0x65, 0x6c, 0x61, 0x74, + 0x69, 0x6f, 0x6e, 0x12, 0x2e, 0x0a, 0x13, 0x63, 0x61, 0x6e, 0x6f, 0x6e, 0x69, 0x63, 0x61, 0x6c, + 0x5f, 0x63, 0x61, 0x63, 0x68, 0x65, 0x5f, 0x6b, 0x65, 0x79, 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x11, 0x63, 0x61, 0x6e, 0x6f, 0x6e, 0x69, 0x63, 0x61, 0x6c, 0x43, 0x61, 0x63, 0x68, 0x65, + 0x4b, 0x65, 0x79, 0x22, 0xf4, 0x03, 0x0a, 0x11, 0x52, 0x65, 0x61, 0x63, 0x68, 0x61, 0x62, 0x69, + 0x6c, 0x69, 0x74, 0x79, 0x47, 0x72, 0x61, 0x70, 0x68, 0x12, 0x77, 0x0a, 0x1b, 0x65, 0x6e, 0x74, + 0x72, 0x79, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x73, 0x5f, 0x62, 0x79, 0x5f, 0x73, 0x75, 0x62, 0x6a, + 0x65, 0x63, 0x74, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x38, + 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x52, 0x65, 0x61, 0x63, 0x68, 0x61, 0x62, + 0x69, 0x6c, 0x69, 0x74, 0x79, 0x47, 0x72, 0x61, 0x70, 0x68, 0x2e, 0x45, 0x6e, 0x74, 0x72, 0x79, + 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x73, 0x42, 0x79, 0x53, 0x75, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x54, + 0x79, 0x70, 0x65, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x18, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x70, + 0x6f, 0x69, 0x6e, 0x74, 0x73, 0x42, 0x79, 0x53, 0x75, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x54, 0x79, + 0x70, 0x65, 0x12, 0x83, 0x01, 0x0a, 0x1f, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x70, 0x6f, 0x69, 0x6e, + 0x74, 0x73, 0x5f, 0x62, 0x79, 0x5f, 0x73, 0x75, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x5f, 0x72, 0x65, + 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x3c, 0x2e, 0x63, + 0x6f, 0x72, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x52, 0x65, 0x61, 0x63, 0x68, 0x61, 0x62, 0x69, 0x6c, + 0x69, 0x74, 0x79, 0x47, 0x72, 0x61, 0x70, 0x68, 0x2e, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x70, 0x6f, + 0x69, 0x6e, 0x74, 0x73, 0x42, 0x79, 0x53, 0x75, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x52, 0x65, 0x6c, + 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x1c, 0x65, 0x6e, 0x74, 0x72, + 0x79, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x73, 0x42, 0x79, 0x53, 0x75, 0x62, 0x6a, 0x65, 0x63, 0x74, + 0x52, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x1a, 0x6d, 0x0a, 0x1d, 0x45, 0x6e, 0x74, 0x72, + 0x79, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x73, 0x42, 0x79, 0x53, 0x75, 0x62, 0x6a, 0x65, 0x63, 0x74, + 0x54, 0x79, 0x70, 0x65, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x36, 0x0a, 0x05, 0x76, + 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x63, 0x6f, 0x72, + 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x52, 0x65, 0x61, 0x63, 0x68, 0x61, 0x62, 0x69, 0x6c, 0x69, 0x74, + 0x79, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x73, 0x52, 0x05, 0x76, 0x61, + 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x1a, 0x71, 0x0a, 0x21, 0x45, 0x6e, 0x74, 0x72, 0x79, + 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x73, 0x42, 0x79, 0x53, 0x75, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x52, + 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, + 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x36, + 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, + 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x52, 0x65, 0x61, 0x63, 0x68, 0x61, 0x62, 0x69, + 0x6c, 0x69, 0x74, 0x79, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x73, 0x52, + 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0xc6, 0x01, 0x0a, 0x17, 0x52, + 0x65, 0x61, 0x63, 0x68, 0x61, 0x62, 0x69, 0x6c, 0x69, 0x74, 0x79, 0x45, 0x6e, 0x74, 0x72, 0x79, + 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x73, 0x12, 0x41, 0x0a, 0x0b, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x70, + 0x6f, 0x69, 0x6e, 0x74, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1f, 0x2e, 0x63, 0x6f, + 0x72, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x52, 0x65, 0x61, 0x63, 0x68, 0x61, 0x62, 0x69, 0x6c, 0x69, + 0x74, 0x79, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x52, 0x0b, 0x65, 0x6e, + 0x74, 0x72, 0x79, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x73, 0x12, 0x21, 0x0a, 0x0c, 0x73, 0x75, 0x62, + 0x6a, 0x65, 0x63, 0x74, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x0b, 0x73, 0x75, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x54, 0x79, 0x70, 0x65, 0x12, 0x45, 0x0a, 0x10, + 0x73, 0x75, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x5f, 0x72, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, + 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x76, 0x31, + 0x2e, 0x52, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, + 0x63, 0x65, 0x52, 0x0f, 0x73, 0x75, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x52, 0x65, 0x6c, 0x61, 0x74, + 0x69, 0x6f, 0x6e, 0x22, 0xce, 0x04, 0x0a, 0x16, 0x52, 0x65, 0x61, 0x63, 0x68, 0x61, 0x62, 0x69, + 0x6c, 0x69, 0x74, 0x79, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x12, 0x4e, + 0x0a, 0x04, 0x6b, 0x69, 0x6e, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x3a, 0x2e, 0x63, + 0x6f, 0x72, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x52, 0x65, 0x61, 0x63, 0x68, 0x61, 0x62, 0x69, 0x6c, + 0x69, 0x74, 0x79, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x2e, 0x52, 0x65, + 0x61, 0x63, 0x68, 0x61, 0x62, 0x69, 0x6c, 0x69, 0x74, 0x79, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x70, + 0x6f, 0x69, 0x6e, 0x74, 0x4b, 0x69, 0x6e, 0x64, 0x52, 0x04, 0x6b, 0x69, 0x6e, 0x64, 0x12, 0x43, + 0x0a, 0x0f, 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 0x5f, 0x72, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, + 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x76, + 0x31, 0x2e, 0x52, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x66, 0x65, 0x72, 0x65, + 0x6e, 0x63, 0x65, 0x52, 0x0e, 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 0x52, 0x65, 0x6c, 0x61, 0x74, + 0x69, 0x6f, 0x6e, 0x12, 0x5b, 0x0a, 0x0d, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x5f, 0x73, 0x74, + 0x61, 0x74, 0x75, 0x73, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x36, 0x2e, 0x63, 0x6f, 0x72, + 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x52, 0x65, 0x61, 0x63, 0x68, 0x61, 0x62, 0x69, 0x6c, 0x69, 0x74, + 0x79, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x2e, 0x45, 0x6e, 0x74, 0x72, + 0x79, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x53, 0x74, 0x61, 0x74, + 0x75, 0x73, 0x52, 0x0c, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, + 0x12, 0x2b, 0x0a, 0x11, 0x74, 0x75, 0x70, 0x6c, 0x65, 0x73, 0x65, 0x74, 0x5f, 0x72, 0x65, 0x6c, + 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x10, 0x74, 0x75, 0x70, + 0x6c, 0x65, 0x73, 0x65, 0x74, 0x52, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x3a, 0x0a, + 0x19, 0x63, 0x6f, 0x6d, 0x70, 0x75, 0x74, 0x65, 0x64, 0x5f, 0x75, 0x73, 0x65, 0x72, 0x73, 0x65, + 0x74, 0x5f, 0x72, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x17, 0x63, 0x6f, 0x6d, 0x70, 0x75, 0x74, 0x65, 0x64, 0x55, 0x73, 0x65, 0x72, 0x73, 0x65, + 0x74, 0x52, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x22, 0x7a, 0x0a, 0x1a, 0x52, 0x65, 0x61, + 0x63, 0x68, 0x61, 0x62, 0x69, 0x6c, 0x69, 0x74, 0x79, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x70, 0x6f, + 0x69, 0x6e, 0x74, 0x4b, 0x69, 0x6e, 0x64, 0x12, 0x17, 0x0a, 0x13, 0x52, 0x45, 0x4c, 0x41, 0x54, + 0x49, 0x4f, 0x4e, 0x5f, 0x45, 0x4e, 0x54, 0x52, 0x59, 0x50, 0x4f, 0x49, 0x4e, 0x54, 0x10, 0x00, + 0x12, 0x1f, 0x0a, 0x1b, 0x43, 0x4f, 0x4d, 0x50, 0x55, 0x54, 0x45, 0x44, 0x5f, 0x55, 0x53, 0x45, + 0x52, 0x53, 0x45, 0x54, 0x5f, 0x45, 0x4e, 0x54, 0x52, 0x59, 0x50, 0x4f, 0x49, 0x4e, 0x54, 0x10, + 0x01, 0x12, 0x22, 0x0a, 0x1e, 0x54, 0x55, 0x50, 0x4c, 0x45, 0x53, 0x45, 0x54, 0x5f, 0x54, 0x4f, + 0x5f, 0x55, 0x53, 0x45, 0x52, 0x53, 0x45, 0x54, 0x5f, 0x45, 0x4e, 0x54, 0x52, 0x59, 0x50, 0x4f, + 0x49, 0x4e, 0x54, 0x10, 0x02, 0x22, 0x57, 0x0a, 0x16, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x70, 0x6f, + 0x69, 0x6e, 0x74, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, + 0x20, 0x0a, 0x1c, 0x52, 0x45, 0x41, 0x43, 0x48, 0x41, 0x42, 0x4c, 0x45, 0x5f, 0x43, 0x4f, 0x4e, + 0x44, 0x49, 0x54, 0x49, 0x4f, 0x4e, 0x41, 0x4c, 0x5f, 0x52, 0x45, 0x53, 0x55, 0x4c, 0x54, 0x10, + 0x00, 0x12, 0x1b, 0x0a, 0x17, 0x44, 0x49, 0x52, 0x45, 0x43, 0x54, 0x5f, 0x4f, 0x50, 0x45, 0x52, + 0x41, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x52, 0x45, 0x53, 0x55, 0x4c, 0x54, 0x10, 0x01, 0x4a, 0x04, + 0x08, 0x03, 0x10, 0x04, 0x22, 0x65, 0x0a, 0x0f, 0x54, 0x79, 0x70, 0x65, 0x49, 0x6e, 0x66, 0x6f, + 0x72, 0x6d, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x52, 0x0a, 0x18, 0x61, 0x6c, 0x6c, 0x6f, 0x77, + 0x65, 0x64, 0x5f, 0x64, 0x69, 0x72, 0x65, 0x63, 0x74, 0x5f, 0x72, 0x65, 0x6c, 0x61, 0x74, 0x69, + 0x6f, 0x6e, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x18, 0x2e, 0x63, 0x6f, 0x72, 0x65, + 0x2e, 0x76, 0x31, 0x2e, 0x41, 0x6c, 0x6c, 0x6f, 0x77, 0x65, 0x64, 0x52, 0x65, 0x6c, 0x61, 0x74, + 0x69, 0x6f, 0x6e, 0x52, 0x16, 0x61, 0x6c, 0x6c, 0x6f, 0x77, 0x65, 0x64, 0x44, 0x69, 0x72, 0x65, + 0x63, 0x74, 0x52, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x22, 0xca, 0x03, 0x0a, 0x0f, + 0x41, 0x6c, 0x6c, 0x6f, 0x77, 0x65, 0x64, 0x52, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, + 0x66, 0x0a, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x09, 0x42, 0x48, 0xfa, 0x42, 0x45, 0x72, 0x43, 0x28, 0x80, 0x01, 0x32, 0x3e, 0x5e, 0x28, + 0x5b, 0x61, 0x2d, 0x7a, 0x5d, 0x5b, 0x61, 0x2d, 0x7a, 0x30, 0x2d, 0x39, 0x5f, 0x5d, 0x7b, 0x31, + 0x2c, 0x36, 0x31, 0x7d, 0x5b, 0x61, 0x2d, 0x7a, 0x30, 0x2d, 0x39, 0x5d, 0x2f, 0x29, 0x2a, 0x5b, + 0x61, 0x2d, 0x7a, 0x5d, 0x5b, 0x61, 0x2d, 0x7a, 0x30, 0x2d, 0x39, 0x5f, 0x5d, 0x7b, 0x31, 0x2c, + 0x36, 0x32, 0x7d, 0x5b, 0x61, 0x2d, 0x7a, 0x30, 0x2d, 0x39, 0x5d, 0x24, 0x52, 0x09, 0x6e, 0x61, + 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x4e, 0x0a, 0x08, 0x72, 0x65, 0x6c, 0x61, 0x74, + 0x69, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x42, 0x30, 0xfa, 0x42, 0x2d, 0x72, 0x2b, + 0x28, 0x40, 0x32, 0x27, 0x5e, 0x28, 0x5c, 0x2e, 0x5c, 0x2e, 0x5c, 0x2e, 0x7c, 0x5b, 0x61, 0x2d, + 0x7a, 0x5d, 0x5b, 0x61, 0x2d, 0x7a, 0x30, 0x2d, 0x39, 0x5f, 0x5d, 0x7b, 0x31, 0x2c, 0x36, 0x32, + 0x7d, 0x5b, 0x61, 0x2d, 0x7a, 0x30, 0x2d, 0x39, 0x5d, 0x29, 0x24, 0x48, 0x00, 0x52, 0x08, 0x72, + 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x52, 0x0a, 0x0f, 0x70, 0x75, 0x62, 0x6c, 0x69, + 0x63, 0x5f, 0x77, 0x69, 0x6c, 0x64, 0x63, 0x61, 0x72, 0x64, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x27, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x41, 0x6c, 0x6c, 0x6f, 0x77, + 0x65, 0x64, 0x52, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x50, 0x75, 0x62, 0x6c, 0x69, + 0x63, 0x57, 0x69, 0x6c, 0x64, 0x63, 0x61, 0x72, 0x64, 0x48, 0x00, 0x52, 0x0e, 0x70, 0x75, 0x62, + 0x6c, 0x69, 0x63, 0x57, 0x69, 0x6c, 0x64, 0x63, 0x61, 0x72, 0x64, 0x12, 0x40, 0x0a, 0x0f, 0x73, + 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x05, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x53, + 0x6f, 0x75, 0x72, 0x63, 0x65, 0x50, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0e, 0x73, + 0x6f, 0x75, 0x72, 0x63, 0x65, 0x50, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x3f, 0x0a, + 0x0f, 0x72, 0x65, 0x71, 0x75, 0x69, 0x72, 0x65, 0x64, 0x5f, 0x63, 0x61, 0x76, 0x65, 0x61, 0x74, + 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x16, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x76, 0x31, + 0x2e, 0x41, 0x6c, 0x6c, 0x6f, 0x77, 0x65, 0x64, 0x43, 0x61, 0x76, 0x65, 0x61, 0x74, 0x52, 0x0e, + 0x72, 0x65, 0x71, 0x75, 0x69, 0x72, 0x65, 0x64, 0x43, 0x61, 0x76, 0x65, 0x61, 0x74, 0x1a, 0x10, + 0x0a, 0x0e, 0x50, 0x75, 0x62, 0x6c, 0x69, 0x63, 0x57, 0x69, 0x6c, 0x64, 0x63, 0x61, 0x72, 0x64, + 0x42, 0x16, 0x0a, 0x14, 0x72, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x6f, 0x72, 0x5f, + 0x77, 0x69, 0x6c, 0x64, 0x63, 0x61, 0x72, 0x64, 0x22, 0x30, 0x0a, 0x0d, 0x41, 0x6c, 0x6c, 0x6f, + 0x77, 0x65, 0x64, 0x43, 0x61, 0x76, 0x65, 0x61, 0x74, 0x12, 0x1f, 0x0a, 0x0b, 0x63, 0x61, 0x76, + 0x65, 0x61, 0x74, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, + 0x63, 0x61, 0x76, 0x65, 0x61, 0x74, 0x4e, 0x61, 0x6d, 0x65, 0x22, 0xad, 0x02, 0x0a, 0x0e, 0x55, + 0x73, 0x65, 0x72, 0x73, 0x65, 0x74, 0x52, 0x65, 0x77, 0x72, 0x69, 0x74, 0x65, 0x12, 0x37, 0x0a, + 0x05, 0x75, 0x6e, 0x69, 0x6f, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x63, + 0x6f, 0x72, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x65, 0x74, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, + 0x69, 0x6f, 0x6e, 0x42, 0x08, 0xfa, 0x42, 0x05, 0x8a, 0x01, 0x02, 0x10, 0x01, 0x48, 0x00, 0x52, + 0x05, 0x75, 0x6e, 0x69, 0x6f, 0x6e, 0x12, 0x45, 0x0a, 0x0c, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x73, + 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x63, + 0x6f, 0x72, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x65, 0x74, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, + 0x69, 0x6f, 0x6e, 0x42, 0x08, 0xfa, 0x42, 0x05, 0x8a, 0x01, 0x02, 0x10, 0x01, 0x48, 0x00, 0x52, + 0x0c, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x73, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x3f, 0x0a, + 0x09, 0x65, 0x78, 0x63, 0x6c, 0x75, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x15, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x65, 0x74, 0x4f, 0x70, + 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x42, 0x08, 0xfa, 0x42, 0x05, 0x8a, 0x01, 0x02, 0x10, + 0x01, 0x48, 0x00, 0x52, 0x09, 0x65, 0x78, 0x63, 0x6c, 0x75, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x40, 0x0a, 0x0f, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, - 0x6e, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x76, + 0x6e, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x50, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0e, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x50, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, - 0x12, 0x2b, 0x0a, 0x11, 0x61, 0x6c, 0x69, 0x61, 0x73, 0x69, 0x6e, 0x67, 0x5f, 0x72, 0x65, 0x6c, - 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x10, 0x61, 0x6c, 0x69, - 0x61, 0x73, 0x69, 0x6e, 0x67, 0x52, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x2e, 0x0a, - 0x13, 0x63, 0x61, 0x6e, 0x6f, 0x6e, 0x69, 0x63, 0x61, 0x6c, 0x5f, 0x63, 0x61, 0x63, 0x68, 0x65, - 0x5f, 0x6b, 0x65, 0x79, 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, 0x52, 0x11, 0x63, 0x61, 0x6e, 0x6f, - 0x6e, 0x69, 0x63, 0x61, 0x6c, 0x43, 0x61, 0x63, 0x68, 0x65, 0x4b, 0x65, 0x79, 0x22, 0xf4, 0x03, - 0x0a, 0x11, 0x52, 0x65, 0x61, 0x63, 0x68, 0x61, 0x62, 0x69, 0x6c, 0x69, 0x74, 0x79, 0x47, 0x72, - 0x61, 0x70, 0x68, 0x12, 0x77, 0x0a, 0x1b, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x70, 0x6f, 0x69, 0x6e, - 0x74, 0x73, 0x5f, 0x62, 0x79, 0x5f, 0x73, 0x75, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x5f, 0x74, 0x79, - 0x70, 0x65, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x38, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, - 0x76, 0x31, 0x2e, 0x52, 0x65, 0x61, 0x63, 0x68, 0x61, 0x62, 0x69, 0x6c, 0x69, 0x74, 0x79, 0x47, - 0x72, 0x61, 0x70, 0x68, 0x2e, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x73, - 0x42, 0x79, 0x53, 0x75, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x54, 0x79, 0x70, 0x65, 0x45, 0x6e, 0x74, - 0x72, 0x79, 0x52, 0x18, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x73, 0x42, - 0x79, 0x53, 0x75, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x54, 0x79, 0x70, 0x65, 0x12, 0x83, 0x01, 0x0a, - 0x1f, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x73, 0x5f, 0x62, 0x79, 0x5f, - 0x73, 0x75, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x5f, 0x72, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, - 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x3c, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x76, 0x31, - 0x2e, 0x52, 0x65, 0x61, 0x63, 0x68, 0x61, 0x62, 0x69, 0x6c, 0x69, 0x74, 0x79, 0x47, 0x72, 0x61, - 0x70, 0x68, 0x2e, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x73, 0x42, 0x79, - 0x53, 0x75, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x52, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x45, - 0x6e, 0x74, 0x72, 0x79, 0x52, 0x1c, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x70, 0x6f, 0x69, 0x6e, 0x74, - 0x73, 0x42, 0x79, 0x53, 0x75, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x52, 0x65, 0x6c, 0x61, 0x74, 0x69, - 0x6f, 0x6e, 0x1a, 0x6d, 0x0a, 0x1d, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x70, 0x6f, 0x69, 0x6e, 0x74, - 0x73, 0x42, 0x79, 0x53, 0x75, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x54, 0x79, 0x70, 0x65, 0x45, 0x6e, - 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, - 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x36, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, - 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x52, - 0x65, 0x61, 0x63, 0x68, 0x61, 0x62, 0x69, 0x6c, 0x69, 0x74, 0x79, 0x45, 0x6e, 0x74, 0x72, 0x79, - 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x73, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, - 0x01, 0x1a, 0x71, 0x0a, 0x21, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x73, - 0x42, 0x79, 0x53, 0x75, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x52, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, - 0x6e, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, - 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x36, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, - 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x76, - 0x31, 0x2e, 0x52, 0x65, 0x61, 0x63, 0x68, 0x61, 0x62, 0x69, 0x6c, 0x69, 0x74, 0x79, 0x45, 0x6e, - 0x74, 0x72, 0x79, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x73, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, - 0x3a, 0x02, 0x38, 0x01, 0x22, 0xc6, 0x01, 0x0a, 0x17, 0x52, 0x65, 0x61, 0x63, 0x68, 0x61, 0x62, - 0x69, 0x6c, 0x69, 0x74, 0x79, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x73, - 0x12, 0x41, 0x0a, 0x0b, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x73, 0x18, - 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1f, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x76, 0x31, 0x2e, - 0x52, 0x65, 0x61, 0x63, 0x68, 0x61, 0x62, 0x69, 0x6c, 0x69, 0x74, 0x79, 0x45, 0x6e, 0x74, 0x72, - 0x79, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x52, 0x0b, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x70, 0x6f, 0x69, - 0x6e, 0x74, 0x73, 0x12, 0x21, 0x0a, 0x0c, 0x73, 0x75, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x5f, 0x74, - 0x79, 0x70, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x73, 0x75, 0x62, 0x6a, 0x65, - 0x63, 0x74, 0x54, 0x79, 0x70, 0x65, 0x12, 0x45, 0x0a, 0x10, 0x73, 0x75, 0x62, 0x6a, 0x65, 0x63, - 0x74, 0x5f, 0x72, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, - 0x32, 0x1a, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x52, 0x65, 0x6c, 0x61, 0x74, - 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65, 0x52, 0x0f, 0x73, 0x75, - 0x62, 0x6a, 0x65, 0x63, 0x74, 0x52, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x22, 0xce, 0x04, - 0x0a, 0x16, 0x52, 0x65, 0x61, 0x63, 0x68, 0x61, 0x62, 0x69, 0x6c, 0x69, 0x74, 0x79, 0x45, 0x6e, - 0x74, 0x72, 0x79, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x12, 0x4e, 0x0a, 0x04, 0x6b, 0x69, 0x6e, 0x64, - 0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x3a, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x76, 0x31, - 0x2e, 0x52, 0x65, 0x61, 0x63, 0x68, 0x61, 0x62, 0x69, 0x6c, 0x69, 0x74, 0x79, 0x45, 0x6e, 0x74, - 0x72, 0x79, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x2e, 0x52, 0x65, 0x61, 0x63, 0x68, 0x61, 0x62, 0x69, - 0x6c, 0x69, 0x74, 0x79, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x4b, 0x69, - 0x6e, 0x64, 0x52, 0x04, 0x6b, 0x69, 0x6e, 0x64, 0x12, 0x43, 0x0a, 0x0f, 0x74, 0x61, 0x72, 0x67, - 0x65, 0x74, 0x5f, 0x72, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, - 0x0b, 0x32, 0x1a, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x52, 0x65, 0x6c, 0x61, - 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65, 0x52, 0x0e, 0x74, - 0x61, 0x72, 0x67, 0x65, 0x74, 0x52, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x5b, 0x0a, - 0x0d, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x5f, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x04, - 0x20, 0x01, 0x28, 0x0e, 0x32, 0x36, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x52, - 0x65, 0x61, 0x63, 0x68, 0x61, 0x62, 0x69, 0x6c, 0x69, 0x74, 0x79, 0x45, 0x6e, 0x74, 0x72, 0x79, - 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x2e, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x70, 0x6f, 0x69, 0x6e, 0x74, - 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x0c, 0x72, 0x65, - 0x73, 0x75, 0x6c, 0x74, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x2b, 0x0a, 0x11, 0x74, 0x75, - 0x70, 0x6c, 0x65, 0x73, 0x65, 0x74, 0x5f, 0x72, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, - 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x10, 0x74, 0x75, 0x70, 0x6c, 0x65, 0x73, 0x65, 0x74, 0x52, - 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x3a, 0x0a, 0x19, 0x63, 0x6f, 0x6d, 0x70, 0x75, - 0x74, 0x65, 0x64, 0x5f, 0x75, 0x73, 0x65, 0x72, 0x73, 0x65, 0x74, 0x5f, 0x72, 0x65, 0x6c, 0x61, - 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x17, 0x63, 0x6f, 0x6d, 0x70, - 0x75, 0x74, 0x65, 0x64, 0x55, 0x73, 0x65, 0x72, 0x73, 0x65, 0x74, 0x52, 0x65, 0x6c, 0x61, 0x74, - 0x69, 0x6f, 0x6e, 0x22, 0x7a, 0x0a, 0x1a, 0x52, 0x65, 0x61, 0x63, 0x68, 0x61, 0x62, 0x69, 0x6c, - 0x69, 0x74, 0x79, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x4b, 0x69, 0x6e, - 0x64, 0x12, 0x17, 0x0a, 0x13, 0x52, 0x45, 0x4c, 0x41, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x45, 0x4e, - 0x54, 0x52, 0x59, 0x50, 0x4f, 0x49, 0x4e, 0x54, 0x10, 0x00, 0x12, 0x1f, 0x0a, 0x1b, 0x43, 0x4f, - 0x4d, 0x50, 0x55, 0x54, 0x45, 0x44, 0x5f, 0x55, 0x53, 0x45, 0x52, 0x53, 0x45, 0x54, 0x5f, 0x45, - 0x4e, 0x54, 0x52, 0x59, 0x50, 0x4f, 0x49, 0x4e, 0x54, 0x10, 0x01, 0x12, 0x22, 0x0a, 0x1e, 0x54, - 0x55, 0x50, 0x4c, 0x45, 0x53, 0x45, 0x54, 0x5f, 0x54, 0x4f, 0x5f, 0x55, 0x53, 0x45, 0x52, 0x53, - 0x45, 0x54, 0x5f, 0x45, 0x4e, 0x54, 0x52, 0x59, 0x50, 0x4f, 0x49, 0x4e, 0x54, 0x10, 0x02, 0x22, - 0x57, 0x0a, 0x16, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x52, 0x65, 0x73, - 0x75, 0x6c, 0x74, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x20, 0x0a, 0x1c, 0x52, 0x45, 0x41, - 0x43, 0x48, 0x41, 0x42, 0x4c, 0x45, 0x5f, 0x43, 0x4f, 0x4e, 0x44, 0x49, 0x54, 0x49, 0x4f, 0x4e, - 0x41, 0x4c, 0x5f, 0x52, 0x45, 0x53, 0x55, 0x4c, 0x54, 0x10, 0x00, 0x12, 0x1b, 0x0a, 0x17, 0x44, - 0x49, 0x52, 0x45, 0x43, 0x54, 0x5f, 0x4f, 0x50, 0x45, 0x52, 0x41, 0x54, 0x49, 0x4f, 0x4e, 0x5f, - 0x52, 0x45, 0x53, 0x55, 0x4c, 0x54, 0x10, 0x01, 0x4a, 0x04, 0x08, 0x03, 0x10, 0x04, 0x22, 0x65, - 0x0a, 0x0f, 0x54, 0x79, 0x70, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x72, 0x6d, 0x61, 0x74, 0x69, 0x6f, - 0x6e, 0x12, 0x52, 0x0a, 0x18, 0x61, 0x6c, 0x6c, 0x6f, 0x77, 0x65, 0x64, 0x5f, 0x64, 0x69, 0x72, - 0x65, 0x63, 0x74, 0x5f, 0x72, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x01, 0x20, - 0x03, 0x28, 0x0b, 0x32, 0x18, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x41, 0x6c, - 0x6c, 0x6f, 0x77, 0x65, 0x64, 0x52, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x16, 0x61, - 0x6c, 0x6c, 0x6f, 0x77, 0x65, 0x64, 0x44, 0x69, 0x72, 0x65, 0x63, 0x74, 0x52, 0x65, 0x6c, 0x61, - 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x22, 0xca, 0x03, 0x0a, 0x0f, 0x41, 0x6c, 0x6c, 0x6f, 0x77, 0x65, - 0x64, 0x52, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x66, 0x0a, 0x09, 0x6e, 0x61, 0x6d, - 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x42, 0x48, 0xfa, 0x42, - 0x45, 0x72, 0x43, 0x28, 0x80, 0x01, 0x32, 0x3e, 0x5e, 0x28, 0x5b, 0x61, 0x2d, 0x7a, 0x5d, 0x5b, - 0x61, 0x2d, 0x7a, 0x30, 0x2d, 0x39, 0x5f, 0x5d, 0x7b, 0x31, 0x2c, 0x36, 0x31, 0x7d, 0x5b, 0x61, - 0x2d, 0x7a, 0x30, 0x2d, 0x39, 0x5d, 0x2f, 0x29, 0x2a, 0x5b, 0x61, 0x2d, 0x7a, 0x5d, 0x5b, 0x61, - 0x2d, 0x7a, 0x30, 0x2d, 0x39, 0x5f, 0x5d, 0x7b, 0x31, 0x2c, 0x36, 0x32, 0x7d, 0x5b, 0x61, 0x2d, - 0x7a, 0x30, 0x2d, 0x39, 0x5d, 0x24, 0x52, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, - 0x65, 0x12, 0x4e, 0x0a, 0x08, 0x72, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x03, 0x20, - 0x01, 0x28, 0x09, 0x42, 0x30, 0xfa, 0x42, 0x2d, 0x72, 0x2b, 0x28, 0x40, 0x32, 0x27, 0x5e, 0x28, - 0x5c, 0x2e, 0x5c, 0x2e, 0x5c, 0x2e, 0x7c, 0x5b, 0x61, 0x2d, 0x7a, 0x5d, 0x5b, 0x61, 0x2d, 0x7a, - 0x30, 0x2d, 0x39, 0x5f, 0x5d, 0x7b, 0x31, 0x2c, 0x36, 0x32, 0x7d, 0x5b, 0x61, 0x2d, 0x7a, 0x30, - 0x2d, 0x39, 0x5d, 0x29, 0x24, 0x48, 0x00, 0x52, 0x08, 0x72, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, - 0x6e, 0x12, 0x52, 0x0a, 0x0f, 0x70, 0x75, 0x62, 0x6c, 0x69, 0x63, 0x5f, 0x77, 0x69, 0x6c, 0x64, - 0x63, 0x61, 0x72, 0x64, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x27, 0x2e, 0x63, 0x6f, 0x72, - 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x41, 0x6c, 0x6c, 0x6f, 0x77, 0x65, 0x64, 0x52, 0x65, 0x6c, 0x61, - 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x50, 0x75, 0x62, 0x6c, 0x69, 0x63, 0x57, 0x69, 0x6c, 0x64, 0x63, - 0x61, 0x72, 0x64, 0x48, 0x00, 0x52, 0x0e, 0x70, 0x75, 0x62, 0x6c, 0x69, 0x63, 0x57, 0x69, 0x6c, - 0x64, 0x63, 0x61, 0x72, 0x64, 0x12, 0x40, 0x0a, 0x0f, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, - 0x70, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, - 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x50, - 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0e, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x50, - 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x3f, 0x0a, 0x0f, 0x72, 0x65, 0x71, 0x75, 0x69, - 0x72, 0x65, 0x64, 0x5f, 0x63, 0x61, 0x76, 0x65, 0x61, 0x74, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, - 0x32, 0x16, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x41, 0x6c, 0x6c, 0x6f, 0x77, - 0x65, 0x64, 0x43, 0x61, 0x76, 0x65, 0x61, 0x74, 0x52, 0x0e, 0x72, 0x65, 0x71, 0x75, 0x69, 0x72, - 0x65, 0x64, 0x43, 0x61, 0x76, 0x65, 0x61, 0x74, 0x1a, 0x10, 0x0a, 0x0e, 0x50, 0x75, 0x62, 0x6c, - 0x69, 0x63, 0x57, 0x69, 0x6c, 0x64, 0x63, 0x61, 0x72, 0x64, 0x42, 0x16, 0x0a, 0x14, 0x72, 0x65, - 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x6f, 0x72, 0x5f, 0x77, 0x69, 0x6c, 0x64, 0x63, 0x61, - 0x72, 0x64, 0x22, 0x30, 0x0a, 0x0d, 0x41, 0x6c, 0x6c, 0x6f, 0x77, 0x65, 0x64, 0x43, 0x61, 0x76, - 0x65, 0x61, 0x74, 0x12, 0x1f, 0x0a, 0x0b, 0x63, 0x61, 0x76, 0x65, 0x61, 0x74, 0x5f, 0x6e, 0x61, - 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x63, 0x61, 0x76, 0x65, 0x61, 0x74, - 0x4e, 0x61, 0x6d, 0x65, 0x22, 0xad, 0x02, 0x0a, 0x0e, 0x55, 0x73, 0x65, 0x72, 0x73, 0x65, 0x74, - 0x52, 0x65, 0x77, 0x72, 0x69, 0x74, 0x65, 0x12, 0x37, 0x0a, 0x05, 0x75, 0x6e, 0x69, 0x6f, 0x6e, - 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x76, 0x31, - 0x2e, 0x53, 0x65, 0x74, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x42, 0x08, 0xfa, - 0x42, 0x05, 0x8a, 0x01, 0x02, 0x10, 0x01, 0x48, 0x00, 0x52, 0x05, 0x75, 0x6e, 0x69, 0x6f, 0x6e, - 0x12, 0x45, 0x0a, 0x0c, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x73, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, - 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x76, 0x31, - 0x2e, 0x53, 0x65, 0x74, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x42, 0x08, 0xfa, - 0x42, 0x05, 0x8a, 0x01, 0x02, 0x10, 0x01, 0x48, 0x00, 0x52, 0x0c, 0x69, 0x6e, 0x74, 0x65, 0x72, - 0x73, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x3f, 0x0a, 0x09, 0x65, 0x78, 0x63, 0x6c, 0x75, - 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x63, 0x6f, 0x72, + 0x42, 0x18, 0x0a, 0x11, 0x72, 0x65, 0x77, 0x72, 0x69, 0x74, 0x65, 0x5f, 0x6f, 0x70, 0x65, 0x72, + 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x03, 0xf8, 0x42, 0x01, 0x22, 0xb2, 0x05, 0x0a, 0x0c, 0x53, + 0x65, 0x74, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x42, 0x0a, 0x05, 0x63, + 0x68, 0x69, 0x6c, 0x64, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x65, 0x74, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, - 0x6e, 0x42, 0x08, 0xfa, 0x42, 0x05, 0x8a, 0x01, 0x02, 0x10, 0x01, 0x48, 0x00, 0x52, 0x09, 0x65, - 0x78, 0x63, 0x6c, 0x75, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x40, 0x0a, 0x0f, 0x73, 0x6f, 0x75, 0x72, - 0x63, 0x65, 0x5f, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x04, 0x20, 0x01, 0x28, - 0x0b, 0x32, 0x17, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x6f, 0x75, 0x72, - 0x63, 0x65, 0x50, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0e, 0x73, 0x6f, 0x75, 0x72, - 0x63, 0x65, 0x50, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x42, 0x18, 0x0a, 0x11, 0x72, 0x65, - 0x77, 0x72, 0x69, 0x74, 0x65, 0x5f, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, - 0x03, 0xf8, 0x42, 0x01, 0x22, 0xb2, 0x05, 0x0a, 0x0c, 0x53, 0x65, 0x74, 0x4f, 0x70, 0x65, 0x72, - 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x42, 0x0a, 0x05, 0x63, 0x68, 0x69, 0x6c, 0x64, 0x18, 0x01, - 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x53, - 0x65, 0x74, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x43, 0x68, 0x69, 0x6c, - 0x64, 0x42, 0x0f, 0xfa, 0x42, 0x0c, 0x92, 0x01, 0x09, 0x08, 0x01, 0x22, 0x05, 0x8a, 0x01, 0x02, - 0x10, 0x01, 0x52, 0x05, 0x63, 0x68, 0x69, 0x6c, 0x64, 0x1a, 0xdd, 0x04, 0x0a, 0x05, 0x43, 0x68, - 0x69, 0x6c, 0x64, 0x12, 0x37, 0x0a, 0x05, 0x5f, 0x74, 0x68, 0x69, 0x73, 0x18, 0x01, 0x20, 0x01, - 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x65, 0x74, - 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x43, 0x68, 0x69, 0x6c, 0x64, 0x2e, - 0x54, 0x68, 0x69, 0x73, 0x48, 0x00, 0x52, 0x04, 0x54, 0x68, 0x69, 0x73, 0x12, 0x4f, 0x0a, 0x10, - 0x63, 0x6f, 0x6d, 0x70, 0x75, 0x74, 0x65, 0x64, 0x5f, 0x75, 0x73, 0x65, 0x72, 0x73, 0x65, 0x74, - 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x18, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x76, 0x31, - 0x2e, 0x43, 0x6f, 0x6d, 0x70, 0x75, 0x74, 0x65, 0x64, 0x55, 0x73, 0x65, 0x72, 0x73, 0x65, 0x74, - 0x42, 0x08, 0xfa, 0x42, 0x05, 0x8a, 0x01, 0x02, 0x10, 0x01, 0x48, 0x00, 0x52, 0x0f, 0x63, 0x6f, - 0x6d, 0x70, 0x75, 0x74, 0x65, 0x64, 0x55, 0x73, 0x65, 0x72, 0x73, 0x65, 0x74, 0x12, 0x4d, 0x0a, - 0x10, 0x74, 0x75, 0x70, 0x6c, 0x65, 0x5f, 0x74, 0x6f, 0x5f, 0x75, 0x73, 0x65, 0x72, 0x73, 0x65, - 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x76, - 0x31, 0x2e, 0x54, 0x75, 0x70, 0x6c, 0x65, 0x54, 0x6f, 0x55, 0x73, 0x65, 0x72, 0x73, 0x65, 0x74, - 0x42, 0x08, 0xfa, 0x42, 0x05, 0x8a, 0x01, 0x02, 0x10, 0x01, 0x48, 0x00, 0x52, 0x0e, 0x74, 0x75, - 0x70, 0x6c, 0x65, 0x54, 0x6f, 0x55, 0x73, 0x65, 0x72, 0x73, 0x65, 0x74, 0x12, 0x4c, 0x0a, 0x0f, - 0x75, 0x73, 0x65, 0x72, 0x73, 0x65, 0x74, 0x5f, 0x72, 0x65, 0x77, 0x72, 0x69, 0x74, 0x65, 0x18, - 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x76, 0x31, 0x2e, - 0x55, 0x73, 0x65, 0x72, 0x73, 0x65, 0x74, 0x52, 0x65, 0x77, 0x72, 0x69, 0x74, 0x65, 0x42, 0x08, - 0xfa, 0x42, 0x05, 0x8a, 0x01, 0x02, 0x10, 0x01, 0x48, 0x00, 0x52, 0x0e, 0x75, 0x73, 0x65, 0x72, - 0x73, 0x65, 0x74, 0x52, 0x65, 0x77, 0x72, 0x69, 0x74, 0x65, 0x12, 0x6c, 0x0a, 0x1b, 0x66, 0x75, - 0x6e, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x65, 0x64, 0x5f, 0x74, 0x75, 0x70, 0x6c, 0x65, 0x5f, 0x74, - 0x6f, 0x5f, 0x75, 0x73, 0x65, 0x72, 0x73, 0x65, 0x74, 0x18, 0x08, 0x20, 0x01, 0x28, 0x0b, 0x32, - 0x21, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x46, 0x75, 0x6e, 0x63, 0x74, 0x69, + 0x6e, 0x2e, 0x43, 0x68, 0x69, 0x6c, 0x64, 0x42, 0x0f, 0xfa, 0x42, 0x0c, 0x92, 0x01, 0x09, 0x08, + 0x01, 0x22, 0x05, 0x8a, 0x01, 0x02, 0x10, 0x01, 0x52, 0x05, 0x63, 0x68, 0x69, 0x6c, 0x64, 0x1a, + 0xdd, 0x04, 0x0a, 0x05, 0x43, 0x68, 0x69, 0x6c, 0x64, 0x12, 0x37, 0x0a, 0x05, 0x5f, 0x74, 0x68, + 0x69, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, + 0x76, 0x31, 0x2e, 0x53, 0x65, 0x74, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, + 0x43, 0x68, 0x69, 0x6c, 0x64, 0x2e, 0x54, 0x68, 0x69, 0x73, 0x48, 0x00, 0x52, 0x04, 0x54, 0x68, + 0x69, 0x73, 0x12, 0x4f, 0x0a, 0x10, 0x63, 0x6f, 0x6d, 0x70, 0x75, 0x74, 0x65, 0x64, 0x5f, 0x75, + 0x73, 0x65, 0x72, 0x73, 0x65, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x18, 0x2e, 0x63, + 0x6f, 0x72, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x43, 0x6f, 0x6d, 0x70, 0x75, 0x74, 0x65, 0x64, 0x55, + 0x73, 0x65, 0x72, 0x73, 0x65, 0x74, 0x42, 0x08, 0xfa, 0x42, 0x05, 0x8a, 0x01, 0x02, 0x10, 0x01, + 0x48, 0x00, 0x52, 0x0f, 0x63, 0x6f, 0x6d, 0x70, 0x75, 0x74, 0x65, 0x64, 0x55, 0x73, 0x65, 0x72, + 0x73, 0x65, 0x74, 0x12, 0x4d, 0x0a, 0x10, 0x74, 0x75, 0x70, 0x6c, 0x65, 0x5f, 0x74, 0x6f, 0x5f, + 0x75, 0x73, 0x65, 0x72, 0x73, 0x65, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, + 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x54, 0x75, 0x70, 0x6c, 0x65, 0x54, 0x6f, 0x55, + 0x73, 0x65, 0x72, 0x73, 0x65, 0x74, 0x42, 0x08, 0xfa, 0x42, 0x05, 0x8a, 0x01, 0x02, 0x10, 0x01, + 0x48, 0x00, 0x52, 0x0e, 0x74, 0x75, 0x70, 0x6c, 0x65, 0x54, 0x6f, 0x55, 0x73, 0x65, 0x72, 0x73, + 0x65, 0x74, 0x12, 0x4c, 0x0a, 0x0f, 0x75, 0x73, 0x65, 0x72, 0x73, 0x65, 0x74, 0x5f, 0x72, 0x65, + 0x77, 0x72, 0x69, 0x74, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x63, 0x6f, + 0x72, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x55, 0x73, 0x65, 0x72, 0x73, 0x65, 0x74, 0x52, 0x65, 0x77, + 0x72, 0x69, 0x74, 0x65, 0x42, 0x08, 0xfa, 0x42, 0x05, 0x8a, 0x01, 0x02, 0x10, 0x01, 0x48, 0x00, + 0x52, 0x0e, 0x75, 0x73, 0x65, 0x72, 0x73, 0x65, 0x74, 0x52, 0x65, 0x77, 0x72, 0x69, 0x74, 0x65, + 0x12, 0x6c, 0x0a, 0x1b, 0x66, 0x75, 0x6e, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x65, 0x64, 0x5f, 0x74, + 0x75, 0x70, 0x6c, 0x65, 0x5f, 0x74, 0x6f, 0x5f, 0x75, 0x73, 0x65, 0x72, 0x73, 0x65, 0x74, 0x18, + 0x08, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x76, 0x31, 0x2e, + 0x46, 0x75, 0x6e, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x65, 0x64, 0x54, 0x75, 0x70, 0x6c, 0x65, 0x54, + 0x6f, 0x55, 0x73, 0x65, 0x72, 0x73, 0x65, 0x74, 0x42, 0x08, 0xfa, 0x42, 0x05, 0x8a, 0x01, 0x02, + 0x10, 0x01, 0x48, 0x00, 0x52, 0x18, 0x66, 0x75, 0x6e, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x65, 0x64, + 0x54, 0x75, 0x70, 0x6c, 0x65, 0x54, 0x6f, 0x55, 0x73, 0x65, 0x72, 0x73, 0x65, 0x74, 0x12, 0x34, + 0x0a, 0x04, 0x5f, 0x6e, 0x69, 0x6c, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1f, 0x2e, 0x63, + 0x6f, 0x72, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x65, 0x74, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, + 0x69, 0x6f, 0x6e, 0x2e, 0x43, 0x68, 0x69, 0x6c, 0x64, 0x2e, 0x4e, 0x69, 0x6c, 0x48, 0x00, 0x52, + 0x03, 0x4e, 0x69, 0x6c, 0x12, 0x40, 0x0a, 0x0f, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x70, + 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, + 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x50, 0x6f, + 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0e, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x50, 0x6f, + 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x25, 0x0a, 0x0e, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, + 0x69, 0x6f, 0x6e, 0x5f, 0x70, 0x61, 0x74, 0x68, 0x18, 0x07, 0x20, 0x03, 0x28, 0x0d, 0x52, 0x0d, + 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x61, 0x74, 0x68, 0x1a, 0x06, 0x0a, + 0x04, 0x54, 0x68, 0x69, 0x73, 0x1a, 0x05, 0x0a, 0x03, 0x4e, 0x69, 0x6c, 0x42, 0x11, 0x0a, 0x0a, + 0x63, 0x68, 0x69, 0x6c, 0x64, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x12, 0x03, 0xf8, 0x42, 0x01, 0x22, + 0xba, 0x02, 0x0a, 0x0e, 0x54, 0x75, 0x70, 0x6c, 0x65, 0x54, 0x6f, 0x55, 0x73, 0x65, 0x72, 0x73, + 0x65, 0x74, 0x12, 0x46, 0x0a, 0x08, 0x74, 0x75, 0x70, 0x6c, 0x65, 0x73, 0x65, 0x74, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x54, + 0x75, 0x70, 0x6c, 0x65, 0x54, 0x6f, 0x55, 0x73, 0x65, 0x72, 0x73, 0x65, 0x74, 0x2e, 0x54, 0x75, + 0x70, 0x6c, 0x65, 0x73, 0x65, 0x74, 0x42, 0x08, 0xfa, 0x42, 0x05, 0x8a, 0x01, 0x02, 0x10, 0x01, + 0x52, 0x08, 0x74, 0x75, 0x70, 0x6c, 0x65, 0x73, 0x65, 0x74, 0x12, 0x4d, 0x0a, 0x10, 0x63, 0x6f, + 0x6d, 0x70, 0x75, 0x74, 0x65, 0x64, 0x5f, 0x75, 0x73, 0x65, 0x72, 0x73, 0x65, 0x74, 0x18, 0x02, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x18, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x43, + 0x6f, 0x6d, 0x70, 0x75, 0x74, 0x65, 0x64, 0x55, 0x73, 0x65, 0x72, 0x73, 0x65, 0x74, 0x42, 0x08, + 0xfa, 0x42, 0x05, 0x8a, 0x01, 0x02, 0x10, 0x01, 0x52, 0x0f, 0x63, 0x6f, 0x6d, 0x70, 0x75, 0x74, + 0x65, 0x64, 0x55, 0x73, 0x65, 0x72, 0x73, 0x65, 0x74, 0x12, 0x40, 0x0a, 0x0f, 0x73, 0x6f, 0x75, + 0x72, 0x63, 0x65, 0x5f, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x6f, 0x75, + 0x72, 0x63, 0x65, 0x50, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0e, 0x73, 0x6f, 0x75, + 0x72, 0x63, 0x65, 0x50, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x1a, 0x4f, 0x0a, 0x08, 0x54, + 0x75, 0x70, 0x6c, 0x65, 0x73, 0x65, 0x74, 0x12, 0x43, 0x0a, 0x08, 0x72, 0x65, 0x6c, 0x61, 0x74, + 0x69, 0x6f, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x42, 0x27, 0xfa, 0x42, 0x24, 0x72, 0x22, + 0x28, 0x40, 0x32, 0x1e, 0x5e, 0x5b, 0x61, 0x2d, 0x7a, 0x5d, 0x5b, 0x61, 0x2d, 0x7a, 0x30, 0x2d, + 0x39, 0x5f, 0x5d, 0x7b, 0x31, 0x2c, 0x36, 0x32, 0x7d, 0x5b, 0x61, 0x2d, 0x7a, 0x30, 0x2d, 0x39, + 0x5d, 0x24, 0x52, 0x08, 0x72, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x22, 0xec, 0x03, 0x0a, + 0x18, 0x46, 0x75, 0x6e, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x65, 0x64, 0x54, 0x75, 0x70, 0x6c, 0x65, + 0x54, 0x6f, 0x55, 0x73, 0x65, 0x72, 0x73, 0x65, 0x74, 0x12, 0x52, 0x0a, 0x08, 0x66, 0x75, 0x6e, + 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x2a, 0x2e, 0x63, 0x6f, + 0x72, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x46, 0x75, 0x6e, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x65, 0x64, + 0x54, 0x75, 0x70, 0x6c, 0x65, 0x54, 0x6f, 0x55, 0x73, 0x65, 0x72, 0x73, 0x65, 0x74, 0x2e, 0x46, + 0x75, 0x6e, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x42, 0x0a, 0xfa, 0x42, 0x07, 0x82, 0x01, 0x04, 0x10, + 0x01, 0x20, 0x00, 0x52, 0x08, 0x66, 0x75, 0x6e, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x50, 0x0a, + 0x08, 0x74, 0x75, 0x70, 0x6c, 0x65, 0x73, 0x65, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x2a, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x46, 0x75, 0x6e, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x65, 0x64, 0x54, 0x75, 0x70, 0x6c, 0x65, 0x54, 0x6f, 0x55, 0x73, 0x65, 0x72, 0x73, - 0x65, 0x74, 0x42, 0x08, 0xfa, 0x42, 0x05, 0x8a, 0x01, 0x02, 0x10, 0x01, 0x48, 0x00, 0x52, 0x18, - 0x66, 0x75, 0x6e, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x65, 0x64, 0x54, 0x75, 0x70, 0x6c, 0x65, 0x54, - 0x6f, 0x55, 0x73, 0x65, 0x72, 0x73, 0x65, 0x74, 0x12, 0x34, 0x0a, 0x04, 0x5f, 0x6e, 0x69, 0x6c, - 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1f, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x76, 0x31, - 0x2e, 0x53, 0x65, 0x74, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x43, 0x68, - 0x69, 0x6c, 0x64, 0x2e, 0x4e, 0x69, 0x6c, 0x48, 0x00, 0x52, 0x03, 0x4e, 0x69, 0x6c, 0x12, 0x40, + 0x65, 0x74, 0x2e, 0x54, 0x75, 0x70, 0x6c, 0x65, 0x73, 0x65, 0x74, 0x42, 0x08, 0xfa, 0x42, 0x05, + 0x8a, 0x01, 0x02, 0x10, 0x01, 0x52, 0x08, 0x74, 0x75, 0x70, 0x6c, 0x65, 0x73, 0x65, 0x74, 0x12, + 0x4d, 0x0a, 0x10, 0x63, 0x6f, 0x6d, 0x70, 0x75, 0x74, 0x65, 0x64, 0x5f, 0x75, 0x73, 0x65, 0x72, + 0x73, 0x65, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x18, 0x2e, 0x63, 0x6f, 0x72, 0x65, + 0x2e, 0x76, 0x31, 0x2e, 0x43, 0x6f, 0x6d, 0x70, 0x75, 0x74, 0x65, 0x64, 0x55, 0x73, 0x65, 0x72, + 0x73, 0x65, 0x74, 0x42, 0x08, 0xfa, 0x42, 0x05, 0x8a, 0x01, 0x02, 0x10, 0x01, 0x52, 0x0f, 0x63, + 0x6f, 0x6d, 0x70, 0x75, 0x74, 0x65, 0x64, 0x55, 0x73, 0x65, 0x72, 0x73, 0x65, 0x74, 0x12, 0x40, 0x0a, 0x0f, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, - 0x6e, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x76, + 0x6e, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x50, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0e, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x50, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, - 0x12, 0x25, 0x0a, 0x0e, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x70, 0x61, - 0x74, 0x68, 0x18, 0x07, 0x20, 0x03, 0x28, 0x0d, 0x52, 0x0d, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, - 0x69, 0x6f, 0x6e, 0x50, 0x61, 0x74, 0x68, 0x1a, 0x06, 0x0a, 0x04, 0x54, 0x68, 0x69, 0x73, 0x1a, - 0x05, 0x0a, 0x03, 0x4e, 0x69, 0x6c, 0x42, 0x11, 0x0a, 0x0a, 0x63, 0x68, 0x69, 0x6c, 0x64, 0x5f, - 0x74, 0x79, 0x70, 0x65, 0x12, 0x03, 0xf8, 0x42, 0x01, 0x22, 0xba, 0x02, 0x0a, 0x0e, 0x54, 0x75, - 0x70, 0x6c, 0x65, 0x54, 0x6f, 0x55, 0x73, 0x65, 0x72, 0x73, 0x65, 0x74, 0x12, 0x46, 0x0a, 0x08, - 0x74, 0x75, 0x70, 0x6c, 0x65, 0x73, 0x65, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, - 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x54, 0x75, 0x70, 0x6c, 0x65, 0x54, 0x6f, - 0x55, 0x73, 0x65, 0x72, 0x73, 0x65, 0x74, 0x2e, 0x54, 0x75, 0x70, 0x6c, 0x65, 0x73, 0x65, 0x74, - 0x42, 0x08, 0xfa, 0x42, 0x05, 0x8a, 0x01, 0x02, 0x10, 0x01, 0x52, 0x08, 0x74, 0x75, 0x70, 0x6c, - 0x65, 0x73, 0x65, 0x74, 0x12, 0x4d, 0x0a, 0x10, 0x63, 0x6f, 0x6d, 0x70, 0x75, 0x74, 0x65, 0x64, - 0x5f, 0x75, 0x73, 0x65, 0x72, 0x73, 0x65, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x18, - 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x43, 0x6f, 0x6d, 0x70, 0x75, 0x74, 0x65, - 0x64, 0x55, 0x73, 0x65, 0x72, 0x73, 0x65, 0x74, 0x42, 0x08, 0xfa, 0x42, 0x05, 0x8a, 0x01, 0x02, - 0x10, 0x01, 0x52, 0x0f, 0x63, 0x6f, 0x6d, 0x70, 0x75, 0x74, 0x65, 0x64, 0x55, 0x73, 0x65, 0x72, - 0x73, 0x65, 0x74, 0x12, 0x40, 0x0a, 0x0f, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x70, 0x6f, - 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x63, - 0x6f, 0x72, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x50, 0x6f, 0x73, - 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0e, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x50, 0x6f, 0x73, - 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x1a, 0x4f, 0x0a, 0x08, 0x54, 0x75, 0x70, 0x6c, 0x65, 0x73, 0x65, - 0x74, 0x12, 0x43, 0x0a, 0x08, 0x72, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x01, 0x20, - 0x01, 0x28, 0x09, 0x42, 0x27, 0xfa, 0x42, 0x24, 0x72, 0x22, 0x28, 0x40, 0x32, 0x1e, 0x5e, 0x5b, - 0x61, 0x2d, 0x7a, 0x5d, 0x5b, 0x61, 0x2d, 0x7a, 0x30, 0x2d, 0x39, 0x5f, 0x5d, 0x7b, 0x31, 0x2c, - 0x36, 0x32, 0x7d, 0x5b, 0x61, 0x2d, 0x7a, 0x30, 0x2d, 0x39, 0x5d, 0x24, 0x52, 0x08, 0x72, 0x65, - 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x22, 0xec, 0x03, 0x0a, 0x18, 0x46, 0x75, 0x6e, 0x63, 0x74, - 0x69, 0x6f, 0x6e, 0x65, 0x64, 0x54, 0x75, 0x70, 0x6c, 0x65, 0x54, 0x6f, 0x55, 0x73, 0x65, 0x72, - 0x73, 0x65, 0x74, 0x12, 0x52, 0x0a, 0x08, 0x66, 0x75, 0x6e, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x18, - 0x01, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x2a, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x76, 0x31, 0x2e, - 0x46, 0x75, 0x6e, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x65, 0x64, 0x54, 0x75, 0x70, 0x6c, 0x65, 0x54, - 0x6f, 0x55, 0x73, 0x65, 0x72, 0x73, 0x65, 0x74, 0x2e, 0x46, 0x75, 0x6e, 0x63, 0x74, 0x69, 0x6f, - 0x6e, 0x42, 0x0a, 0xfa, 0x42, 0x07, 0x82, 0x01, 0x04, 0x10, 0x01, 0x20, 0x00, 0x52, 0x08, 0x66, - 0x75, 0x6e, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x50, 0x0a, 0x08, 0x74, 0x75, 0x70, 0x6c, 0x65, - 0x73, 0x65, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2a, 0x2e, 0x63, 0x6f, 0x72, 0x65, - 0x2e, 0x76, 0x31, 0x2e, 0x46, 0x75, 0x6e, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x65, 0x64, 0x54, 0x75, - 0x70, 0x6c, 0x65, 0x54, 0x6f, 0x55, 0x73, 0x65, 0x72, 0x73, 0x65, 0x74, 0x2e, 0x54, 0x75, 0x70, - 0x6c, 0x65, 0x73, 0x65, 0x74, 0x42, 0x08, 0xfa, 0x42, 0x05, 0x8a, 0x01, 0x02, 0x10, 0x01, 0x52, - 0x08, 0x74, 0x75, 0x70, 0x6c, 0x65, 0x73, 0x65, 0x74, 0x12, 0x4d, 0x0a, 0x10, 0x63, 0x6f, 0x6d, - 0x70, 0x75, 0x74, 0x65, 0x64, 0x5f, 0x75, 0x73, 0x65, 0x72, 0x73, 0x65, 0x74, 0x18, 0x03, 0x20, - 0x01, 0x28, 0x0b, 0x32, 0x18, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x43, 0x6f, - 0x6d, 0x70, 0x75, 0x74, 0x65, 0x64, 0x55, 0x73, 0x65, 0x72, 0x73, 0x65, 0x74, 0x42, 0x08, 0xfa, - 0x42, 0x05, 0x8a, 0x01, 0x02, 0x10, 0x01, 0x52, 0x0f, 0x63, 0x6f, 0x6d, 0x70, 0x75, 0x74, 0x65, - 0x64, 0x55, 0x73, 0x65, 0x72, 0x73, 0x65, 0x74, 0x12, 0x40, 0x0a, 0x0f, 0x73, 0x6f, 0x75, 0x72, - 0x63, 0x65, 0x5f, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x04, 0x20, 0x01, 0x28, - 0x0b, 0x32, 0x17, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x6f, 0x75, 0x72, - 0x63, 0x65, 0x50, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0e, 0x73, 0x6f, 0x75, 0x72, - 0x63, 0x65, 0x50, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x1a, 0x4f, 0x0a, 0x08, 0x54, 0x75, - 0x70, 0x6c, 0x65, 0x73, 0x65, 0x74, 0x12, 0x43, 0x0a, 0x08, 0x72, 0x65, 0x6c, 0x61, 0x74, 0x69, - 0x6f, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x42, 0x27, 0xfa, 0x42, 0x24, 0x72, 0x22, 0x28, - 0x40, 0x32, 0x1e, 0x5e, 0x5b, 0x61, 0x2d, 0x7a, 0x5d, 0x5b, 0x61, 0x2d, 0x7a, 0x30, 0x2d, 0x39, - 0x5f, 0x5d, 0x7b, 0x31, 0x2c, 0x36, 0x32, 0x7d, 0x5b, 0x61, 0x2d, 0x7a, 0x30, 0x2d, 0x39, 0x5d, - 0x24, 0x52, 0x08, 0x72, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x22, 0x48, 0x0a, 0x08, 0x46, - 0x75, 0x6e, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x18, 0x0a, 0x14, 0x46, 0x55, 0x4e, 0x43, 0x54, - 0x49, 0x4f, 0x4e, 0x5f, 0x55, 0x4e, 0x53, 0x50, 0x45, 0x43, 0x49, 0x46, 0x49, 0x45, 0x44, 0x10, - 0x00, 0x12, 0x10, 0x0a, 0x0c, 0x46, 0x55, 0x4e, 0x43, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x41, 0x4e, - 0x59, 0x10, 0x01, 0x12, 0x10, 0x0a, 0x0c, 0x46, 0x55, 0x4e, 0x43, 0x54, 0x49, 0x4f, 0x4e, 0x5f, - 0x41, 0x4c, 0x4c, 0x10, 0x02, 0x22, 0x91, 0x02, 0x0a, 0x0f, 0x43, 0x6f, 0x6d, 0x70, 0x75, 0x74, - 0x65, 0x64, 0x55, 0x73, 0x65, 0x72, 0x73, 0x65, 0x74, 0x12, 0x41, 0x0a, 0x06, 0x6f, 0x62, 0x6a, - 0x65, 0x63, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x1f, 0x2e, 0x63, 0x6f, 0x72, 0x65, - 0x2e, 0x76, 0x31, 0x2e, 0x43, 0x6f, 0x6d, 0x70, 0x75, 0x74, 0x65, 0x64, 0x55, 0x73, 0x65, 0x72, - 0x73, 0x65, 0x74, 0x2e, 0x4f, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x42, 0x08, 0xfa, 0x42, 0x05, 0x82, - 0x01, 0x02, 0x10, 0x01, 0x52, 0x06, 0x6f, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x12, 0x43, 0x0a, 0x08, - 0x72, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x42, 0x27, + 0x1a, 0x4f, 0x0a, 0x08, 0x54, 0x75, 0x70, 0x6c, 0x65, 0x73, 0x65, 0x74, 0x12, 0x43, 0x0a, 0x08, + 0x72, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x42, 0x27, 0xfa, 0x42, 0x24, 0x72, 0x22, 0x28, 0x40, 0x32, 0x1e, 0x5e, 0x5b, 0x61, 0x2d, 0x7a, 0x5d, 0x5b, 0x61, 0x2d, 0x7a, 0x30, 0x2d, 0x39, 0x5f, 0x5d, 0x7b, 0x31, 0x2c, 0x36, 0x32, 0x7d, 0x5b, 0x61, 0x2d, 0x7a, 0x30, 0x2d, 0x39, 0x5d, 0x24, 0x52, 0x08, 0x72, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, - 0x6e, 0x12, 0x40, 0x0a, 0x0f, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x70, 0x6f, 0x73, 0x69, - 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x63, 0x6f, 0x72, - 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x50, 0x6f, 0x73, 0x69, 0x74, - 0x69, 0x6f, 0x6e, 0x52, 0x0e, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x50, 0x6f, 0x73, 0x69, 0x74, - 0x69, 0x6f, 0x6e, 0x22, 0x34, 0x0a, 0x06, 0x4f, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x12, 0x10, 0x0a, - 0x0c, 0x54, 0x55, 0x50, 0x4c, 0x45, 0x5f, 0x4f, 0x42, 0x4a, 0x45, 0x43, 0x54, 0x10, 0x00, 0x12, - 0x18, 0x0a, 0x14, 0x54, 0x55, 0x50, 0x4c, 0x45, 0x5f, 0x55, 0x53, 0x45, 0x52, 0x53, 0x45, 0x54, - 0x5f, 0x4f, 0x42, 0x4a, 0x45, 0x43, 0x54, 0x10, 0x01, 0x22, 0x8a, 0x01, 0x0a, 0x0e, 0x53, 0x6f, - 0x75, 0x72, 0x63, 0x65, 0x50, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x37, 0x0a, 0x18, - 0x7a, 0x65, 0x72, 0x6f, 0x5f, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x65, 0x64, 0x5f, 0x6c, 0x69, 0x6e, - 0x65, 0x5f, 0x6e, 0x75, 0x6d, 0x62, 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x04, 0x52, 0x15, - 0x7a, 0x65, 0x72, 0x6f, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x65, 0x64, 0x4c, 0x69, 0x6e, 0x65, 0x4e, - 0x75, 0x6d, 0x62, 0x65, 0x72, 0x12, 0x3f, 0x0a, 0x1c, 0x7a, 0x65, 0x72, 0x6f, 0x5f, 0x69, 0x6e, - 0x64, 0x65, 0x78, 0x65, 0x64, 0x5f, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x5f, 0x70, 0x6f, 0x73, - 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 0x52, 0x19, 0x7a, 0x65, 0x72, - 0x6f, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x65, 0x64, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x50, 0x6f, - 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x22, 0x9c, 0x01, 0x0a, 0x10, 0x43, 0x61, 0x76, 0x65, 0x61, - 0x74, 0x45, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x38, 0x0a, 0x09, 0x6f, - 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x18, - 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x43, 0x61, 0x76, 0x65, 0x61, 0x74, 0x4f, - 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x48, 0x00, 0x52, 0x09, 0x6f, 0x70, 0x65, 0x72, - 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x37, 0x0a, 0x06, 0x63, 0x61, 0x76, 0x65, 0x61, 0x74, 0x18, - 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x76, 0x31, 0x2e, - 0x43, 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74, 0x75, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x64, 0x43, 0x61, - 0x76, 0x65, 0x61, 0x74, 0x48, 0x00, 0x52, 0x06, 0x63, 0x61, 0x76, 0x65, 0x61, 0x74, 0x42, 0x15, - 0x0a, 0x13, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x6f, 0x72, 0x5f, 0x63, - 0x61, 0x76, 0x65, 0x61, 0x74, 0x22, 0xb0, 0x01, 0x0a, 0x0f, 0x43, 0x61, 0x76, 0x65, 0x61, 0x74, - 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x32, 0x0a, 0x02, 0x6f, 0x70, 0x18, - 0x01, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x22, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x76, 0x31, 0x2e, - 0x43, 0x61, 0x76, 0x65, 0x61, 0x74, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, - 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x02, 0x6f, 0x70, 0x12, 0x35, 0x0a, - 0x08, 0x63, 0x68, 0x69, 0x6c, 0x64, 0x72, 0x65, 0x6e, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, - 0x19, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x43, 0x61, 0x76, 0x65, 0x61, 0x74, - 0x45, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x52, 0x08, 0x63, 0x68, 0x69, 0x6c, - 0x64, 0x72, 0x65, 0x6e, 0x22, 0x32, 0x0a, 0x09, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, - 0x6e, 0x12, 0x0b, 0x0a, 0x07, 0x55, 0x4e, 0x4b, 0x4e, 0x4f, 0x57, 0x4e, 0x10, 0x00, 0x12, 0x06, - 0x0a, 0x02, 0x4f, 0x52, 0x10, 0x01, 0x12, 0x07, 0x0a, 0x03, 0x41, 0x4e, 0x44, 0x10, 0x02, 0x12, - 0x07, 0x0a, 0x03, 0x4e, 0x4f, 0x54, 0x10, 0x03, 0x22, 0xee, 0x03, 0x0a, 0x12, 0x52, 0x65, 0x6c, - 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x68, 0x69, 0x70, 0x46, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x12, - 0x70, 0x0a, 0x0d, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x74, 0x79, 0x70, 0x65, - 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x42, 0x4b, 0xfa, 0x42, 0x48, 0x72, 0x46, 0x28, 0x80, 0x01, - 0x32, 0x41, 0x5e, 0x28, 0x28, 0x5b, 0x61, 0x2d, 0x7a, 0x5d, 0x5b, 0x61, 0x2d, 0x7a, 0x30, 0x2d, + 0x6e, 0x22, 0x48, 0x0a, 0x08, 0x46, 0x75, 0x6e, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x18, 0x0a, + 0x14, 0x46, 0x55, 0x4e, 0x43, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x55, 0x4e, 0x53, 0x50, 0x45, 0x43, + 0x49, 0x46, 0x49, 0x45, 0x44, 0x10, 0x00, 0x12, 0x10, 0x0a, 0x0c, 0x46, 0x55, 0x4e, 0x43, 0x54, + 0x49, 0x4f, 0x4e, 0x5f, 0x41, 0x4e, 0x59, 0x10, 0x01, 0x12, 0x10, 0x0a, 0x0c, 0x46, 0x55, 0x4e, + 0x43, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x41, 0x4c, 0x4c, 0x10, 0x02, 0x22, 0x91, 0x02, 0x0a, 0x0f, + 0x43, 0x6f, 0x6d, 0x70, 0x75, 0x74, 0x65, 0x64, 0x55, 0x73, 0x65, 0x72, 0x73, 0x65, 0x74, 0x12, + 0x41, 0x0a, 0x06, 0x6f, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, 0x32, + 0x1f, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x43, 0x6f, 0x6d, 0x70, 0x75, 0x74, + 0x65, 0x64, 0x55, 0x73, 0x65, 0x72, 0x73, 0x65, 0x74, 0x2e, 0x4f, 0x62, 0x6a, 0x65, 0x63, 0x74, + 0x42, 0x08, 0xfa, 0x42, 0x05, 0x82, 0x01, 0x02, 0x10, 0x01, 0x52, 0x06, 0x6f, 0x62, 0x6a, 0x65, + 0x63, 0x74, 0x12, 0x43, 0x0a, 0x08, 0x72, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x02, + 0x20, 0x01, 0x28, 0x09, 0x42, 0x27, 0xfa, 0x42, 0x24, 0x72, 0x22, 0x28, 0x40, 0x32, 0x1e, 0x5e, + 0x5b, 0x61, 0x2d, 0x7a, 0x5d, 0x5b, 0x61, 0x2d, 0x7a, 0x30, 0x2d, 0x39, 0x5f, 0x5d, 0x7b, 0x31, + 0x2c, 0x36, 0x32, 0x7d, 0x5b, 0x61, 0x2d, 0x7a, 0x30, 0x2d, 0x39, 0x5d, 0x24, 0x52, 0x08, 0x72, + 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x40, 0x0a, 0x0f, 0x73, 0x6f, 0x75, 0x72, 0x63, + 0x65, 0x5f, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x17, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x6f, 0x75, 0x72, 0x63, + 0x65, 0x50, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0e, 0x73, 0x6f, 0x75, 0x72, 0x63, + 0x65, 0x50, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x22, 0x34, 0x0a, 0x06, 0x4f, 0x62, 0x6a, + 0x65, 0x63, 0x74, 0x12, 0x10, 0x0a, 0x0c, 0x54, 0x55, 0x50, 0x4c, 0x45, 0x5f, 0x4f, 0x42, 0x4a, + 0x45, 0x43, 0x54, 0x10, 0x00, 0x12, 0x18, 0x0a, 0x14, 0x54, 0x55, 0x50, 0x4c, 0x45, 0x5f, 0x55, + 0x53, 0x45, 0x52, 0x53, 0x45, 0x54, 0x5f, 0x4f, 0x42, 0x4a, 0x45, 0x43, 0x54, 0x10, 0x01, 0x22, + 0x8a, 0x01, 0x0a, 0x0e, 0x53, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x50, 0x6f, 0x73, 0x69, 0x74, 0x69, + 0x6f, 0x6e, 0x12, 0x37, 0x0a, 0x18, 0x7a, 0x65, 0x72, 0x6f, 0x5f, 0x69, 0x6e, 0x64, 0x65, 0x78, + 0x65, 0x64, 0x5f, 0x6c, 0x69, 0x6e, 0x65, 0x5f, 0x6e, 0x75, 0x6d, 0x62, 0x65, 0x72, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x04, 0x52, 0x15, 0x7a, 0x65, 0x72, 0x6f, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x65, + 0x64, 0x4c, 0x69, 0x6e, 0x65, 0x4e, 0x75, 0x6d, 0x62, 0x65, 0x72, 0x12, 0x3f, 0x0a, 0x1c, 0x7a, + 0x65, 0x72, 0x6f, 0x5f, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x65, 0x64, 0x5f, 0x63, 0x6f, 0x6c, 0x75, + 0x6d, 0x6e, 0x5f, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, + 0x04, 0x52, 0x19, 0x7a, 0x65, 0x72, 0x6f, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x65, 0x64, 0x43, 0x6f, + 0x6c, 0x75, 0x6d, 0x6e, 0x50, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x22, 0x9c, 0x01, 0x0a, + 0x10, 0x43, 0x61, 0x76, 0x65, 0x61, 0x74, 0x45, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, + 0x6e, 0x12, 0x38, 0x0a, 0x09, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x18, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x43, + 0x61, 0x76, 0x65, 0x61, 0x74, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x48, 0x00, + 0x52, 0x09, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x37, 0x0a, 0x06, 0x63, + 0x61, 0x76, 0x65, 0x61, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x63, 0x6f, + 0x72, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x43, 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74, 0x75, 0x61, 0x6c, + 0x69, 0x7a, 0x65, 0x64, 0x43, 0x61, 0x76, 0x65, 0x61, 0x74, 0x48, 0x00, 0x52, 0x06, 0x63, 0x61, + 0x76, 0x65, 0x61, 0x74, 0x42, 0x15, 0x0a, 0x13, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, + 0x6e, 0x5f, 0x6f, 0x72, 0x5f, 0x63, 0x61, 0x76, 0x65, 0x61, 0x74, 0x22, 0xb0, 0x01, 0x0a, 0x0f, + 0x43, 0x61, 0x76, 0x65, 0x61, 0x74, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, + 0x32, 0x0a, 0x02, 0x6f, 0x70, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x22, 0x2e, 0x63, 0x6f, + 0x72, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x43, 0x61, 0x76, 0x65, 0x61, 0x74, 0x4f, 0x70, 0x65, 0x72, + 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, + 0x02, 0x6f, 0x70, 0x12, 0x35, 0x0a, 0x08, 0x63, 0x68, 0x69, 0x6c, 0x64, 0x72, 0x65, 0x6e, 0x18, + 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x76, 0x31, 0x2e, + 0x43, 0x61, 0x76, 0x65, 0x61, 0x74, 0x45, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, + 0x52, 0x08, 0x63, 0x68, 0x69, 0x6c, 0x64, 0x72, 0x65, 0x6e, 0x22, 0x32, 0x0a, 0x09, 0x4f, 0x70, + 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x0b, 0x0a, 0x07, 0x55, 0x4e, 0x4b, 0x4e, 0x4f, + 0x57, 0x4e, 0x10, 0x00, 0x12, 0x06, 0x0a, 0x02, 0x4f, 0x52, 0x10, 0x01, 0x12, 0x07, 0x0a, 0x03, + 0x41, 0x4e, 0x44, 0x10, 0x02, 0x12, 0x07, 0x0a, 0x03, 0x4e, 0x4f, 0x54, 0x10, 0x03, 0x22, 0xee, + 0x03, 0x0a, 0x12, 0x52, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x68, 0x69, 0x70, 0x46, + 0x69, 0x6c, 0x74, 0x65, 0x72, 0x12, 0x70, 0x0a, 0x0d, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, + 0x65, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x42, 0x4b, 0xfa, 0x42, + 0x48, 0x72, 0x46, 0x28, 0x80, 0x01, 0x32, 0x41, 0x5e, 0x28, 0x28, 0x5b, 0x61, 0x2d, 0x7a, 0x5d, + 0x5b, 0x61, 0x2d, 0x7a, 0x30, 0x2d, 0x39, 0x5f, 0x5d, 0x7b, 0x31, 0x2c, 0x36, 0x31, 0x7d, 0x5b, + 0x61, 0x2d, 0x7a, 0x30, 0x2d, 0x39, 0x5d, 0x2f, 0x29, 0x2a, 0x5b, 0x61, 0x2d, 0x7a, 0x5d, 0x5b, + 0x61, 0x2d, 0x7a, 0x30, 0x2d, 0x39, 0x5f, 0x5d, 0x7b, 0x31, 0x2c, 0x36, 0x32, 0x7d, 0x5b, 0x61, + 0x2d, 0x7a, 0x30, 0x2d, 0x39, 0x5d, 0x29, 0x3f, 0x24, 0x52, 0x0c, 0x72, 0x65, 0x73, 0x6f, 0x75, + 0x72, 0x63, 0x65, 0x54, 0x79, 0x70, 0x65, 0x12, 0x57, 0x0a, 0x14, 0x6f, 0x70, 0x74, 0x69, 0x6f, + 0x6e, 0x61, 0x6c, 0x5f, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x69, 0x64, 0x18, + 0x02, 0x20, 0x01, 0x28, 0x09, 0x42, 0x25, 0xfa, 0x42, 0x22, 0x72, 0x20, 0x28, 0x80, 0x08, 0x32, + 0x1b, 0x5e, 0x28, 0x5b, 0x61, 0x2d, 0x7a, 0x41, 0x2d, 0x5a, 0x30, 0x2d, 0x39, 0x2f, 0x5f, 0x7c, + 0x5c, 0x2d, 0x3d, 0x2b, 0x5d, 0x7b, 0x31, 0x2c, 0x7d, 0x29, 0x3f, 0x24, 0x52, 0x12, 0x6f, 0x70, + 0x74, 0x69, 0x6f, 0x6e, 0x61, 0x6c, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x49, 0x64, + 0x12, 0x64, 0x0a, 0x1b, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x61, 0x6c, 0x5f, 0x72, 0x65, 0x73, + 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x69, 0x64, 0x5f, 0x70, 0x72, 0x65, 0x66, 0x69, 0x78, 0x18, + 0x05, 0x20, 0x01, 0x28, 0x09, 0x42, 0x25, 0xfa, 0x42, 0x22, 0x72, 0x20, 0x28, 0x80, 0x08, 0x32, + 0x1b, 0x5e, 0x28, 0x5b, 0x61, 0x2d, 0x7a, 0x41, 0x2d, 0x5a, 0x30, 0x2d, 0x39, 0x2f, 0x5f, 0x7c, + 0x5c, 0x2d, 0x3d, 0x2b, 0x5d, 0x7b, 0x31, 0x2c, 0x7d, 0x29, 0x3f, 0x24, 0x52, 0x18, 0x6f, 0x70, + 0x74, 0x69, 0x6f, 0x6e, 0x61, 0x6c, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x49, 0x64, + 0x50, 0x72, 0x65, 0x66, 0x69, 0x78, 0x12, 0x57, 0x0a, 0x11, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, + 0x61, 0x6c, 0x5f, 0x72, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, + 0x09, 0x42, 0x2a, 0xfa, 0x42, 0x27, 0x72, 0x25, 0x28, 0x40, 0x32, 0x21, 0x5e, 0x28, 0x5b, 0x61, + 0x2d, 0x7a, 0x5d, 0x5b, 0x61, 0x2d, 0x7a, 0x30, 0x2d, 0x39, 0x5f, 0x5d, 0x7b, 0x31, 0x2c, 0x36, + 0x32, 0x7d, 0x5b, 0x61, 0x2d, 0x7a, 0x30, 0x2d, 0x39, 0x5d, 0x29, 0x3f, 0x24, 0x52, 0x10, 0x6f, + 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x61, 0x6c, 0x52, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, + 0x4e, 0x0a, 0x17, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x61, 0x6c, 0x5f, 0x73, 0x75, 0x62, 0x6a, + 0x65, 0x63, 0x74, 0x5f, 0x66, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x16, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x75, 0x62, 0x6a, 0x65, + 0x63, 0x74, 0x46, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x52, 0x15, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, + 0x61, 0x6c, 0x53, 0x75, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x46, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x22, + 0x86, 0x03, 0x0a, 0x0d, 0x53, 0x75, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x46, 0x69, 0x6c, 0x74, 0x65, + 0x72, 0x12, 0x6b, 0x0a, 0x0c, 0x73, 0x75, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x5f, 0x74, 0x79, 0x70, + 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x42, 0x48, 0xfa, 0x42, 0x45, 0x72, 0x43, 0x28, 0x80, + 0x01, 0x32, 0x3e, 0x5e, 0x28, 0x5b, 0x61, 0x2d, 0x7a, 0x5d, 0x5b, 0x61, 0x2d, 0x7a, 0x30, 0x2d, 0x39, 0x5f, 0x5d, 0x7b, 0x31, 0x2c, 0x36, 0x31, 0x7d, 0x5b, 0x61, 0x2d, 0x7a, 0x30, 0x2d, 0x39, 0x5d, 0x2f, 0x29, 0x2a, 0x5b, 0x61, 0x2d, 0x7a, 0x5d, 0x5b, 0x61, 0x2d, 0x7a, 0x30, 0x2d, 0x39, 0x5f, 0x5d, 0x7b, 0x31, 0x2c, 0x36, 0x32, 0x7d, 0x5b, 0x61, 0x2d, 0x7a, 0x30, 0x2d, 0x39, 0x5d, - 0x29, 0x3f, 0x24, 0x52, 0x0c, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x54, 0x79, 0x70, - 0x65, 0x12, 0x57, 0x0a, 0x14, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x61, 0x6c, 0x5f, 0x72, 0x65, - 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x42, - 0x25, 0xfa, 0x42, 0x22, 0x72, 0x20, 0x28, 0x80, 0x08, 0x32, 0x1b, 0x5e, 0x28, 0x5b, 0x61, 0x2d, - 0x7a, 0x41, 0x2d, 0x5a, 0x30, 0x2d, 0x39, 0x2f, 0x5f, 0x7c, 0x5c, 0x2d, 0x3d, 0x2b, 0x5d, 0x7b, - 0x31, 0x2c, 0x7d, 0x29, 0x3f, 0x24, 0x52, 0x12, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x61, 0x6c, - 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x49, 0x64, 0x12, 0x64, 0x0a, 0x1b, 0x6f, 0x70, - 0x74, 0x69, 0x6f, 0x6e, 0x61, 0x6c, 0x5f, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, - 0x69, 0x64, 0x5f, 0x70, 0x72, 0x65, 0x66, 0x69, 0x78, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x42, - 0x25, 0xfa, 0x42, 0x22, 0x72, 0x20, 0x28, 0x80, 0x08, 0x32, 0x1b, 0x5e, 0x28, 0x5b, 0x61, 0x2d, - 0x7a, 0x41, 0x2d, 0x5a, 0x30, 0x2d, 0x39, 0x2f, 0x5f, 0x7c, 0x5c, 0x2d, 0x3d, 0x2b, 0x5d, 0x7b, - 0x31, 0x2c, 0x7d, 0x29, 0x3f, 0x24, 0x52, 0x18, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x61, 0x6c, - 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x49, 0x64, 0x50, 0x72, 0x65, 0x66, 0x69, 0x78, - 0x12, 0x57, 0x0a, 0x11, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x61, 0x6c, 0x5f, 0x72, 0x65, 0x6c, - 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x42, 0x2a, 0xfa, 0x42, 0x27, - 0x72, 0x25, 0x28, 0x40, 0x32, 0x21, 0x5e, 0x28, 0x5b, 0x61, 0x2d, 0x7a, 0x5d, 0x5b, 0x61, 0x2d, - 0x7a, 0x30, 0x2d, 0x39, 0x5f, 0x5d, 0x7b, 0x31, 0x2c, 0x36, 0x32, 0x7d, 0x5b, 0x61, 0x2d, 0x7a, - 0x30, 0x2d, 0x39, 0x5d, 0x29, 0x3f, 0x24, 0x52, 0x10, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x61, - 0x6c, 0x52, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x4e, 0x0a, 0x17, 0x6f, 0x70, 0x74, - 0x69, 0x6f, 0x6e, 0x61, 0x6c, 0x5f, 0x73, 0x75, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x5f, 0x66, 0x69, - 0x6c, 0x74, 0x65, 0x72, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x16, 0x2e, 0x63, 0x6f, 0x72, - 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x75, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x46, 0x69, 0x6c, 0x74, - 0x65, 0x72, 0x52, 0x15, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x61, 0x6c, 0x53, 0x75, 0x62, 0x6a, - 0x65, 0x63, 0x74, 0x46, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x22, 0x86, 0x03, 0x0a, 0x0d, 0x53, 0x75, - 0x62, 0x6a, 0x65, 0x63, 0x74, 0x46, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x12, 0x6b, 0x0a, 0x0c, 0x73, - 0x75, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, - 0x09, 0x42, 0x48, 0xfa, 0x42, 0x45, 0x72, 0x43, 0x28, 0x80, 0x01, 0x32, 0x3e, 0x5e, 0x28, 0x5b, + 0x24, 0x52, 0x0b, 0x73, 0x75, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x54, 0x79, 0x70, 0x65, 0x12, 0x5a, + 0x0a, 0x13, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x61, 0x6c, 0x5f, 0x73, 0x75, 0x62, 0x6a, 0x65, + 0x63, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x42, 0x2a, 0xfa, 0x42, 0x27, + 0x72, 0x25, 0x28, 0x80, 0x08, 0x32, 0x20, 0x5e, 0x28, 0x28, 0x5b, 0x61, 0x2d, 0x7a, 0x41, 0x2d, + 0x5a, 0x30, 0x2d, 0x39, 0x2f, 0x5f, 0x7c, 0x5c, 0x2d, 0x3d, 0x2b, 0x5d, 0x7b, 0x31, 0x2c, 0x7d, + 0x29, 0x7c, 0x5c, 0x2a, 0x29, 0x3f, 0x24, 0x52, 0x11, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x61, + 0x6c, 0x53, 0x75, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x49, 0x64, 0x12, 0x52, 0x0a, 0x11, 0x6f, 0x70, + 0x74, 0x69, 0x6f, 0x6e, 0x61, 0x6c, 0x5f, 0x72, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, + 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x25, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x76, 0x31, 0x2e, + 0x53, 0x75, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x46, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x2e, 0x52, 0x65, + 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x46, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x52, 0x10, 0x6f, 0x70, + 0x74, 0x69, 0x6f, 0x6e, 0x61, 0x6c, 0x52, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x1a, 0x58, + 0x0a, 0x0e, 0x52, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x46, 0x69, 0x6c, 0x74, 0x65, 0x72, + 0x12, 0x46, 0x0a, 0x08, 0x72, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x09, 0x42, 0x2a, 0xfa, 0x42, 0x27, 0x72, 0x25, 0x28, 0x40, 0x32, 0x21, 0x5e, 0x28, 0x5b, 0x61, 0x2d, 0x7a, 0x5d, 0x5b, 0x61, 0x2d, 0x7a, 0x30, 0x2d, 0x39, 0x5f, 0x5d, 0x7b, 0x31, 0x2c, - 0x36, 0x31, 0x7d, 0x5b, 0x61, 0x2d, 0x7a, 0x30, 0x2d, 0x39, 0x5d, 0x2f, 0x29, 0x2a, 0x5b, 0x61, - 0x2d, 0x7a, 0x5d, 0x5b, 0x61, 0x2d, 0x7a, 0x30, 0x2d, 0x39, 0x5f, 0x5d, 0x7b, 0x31, 0x2c, 0x36, - 0x32, 0x7d, 0x5b, 0x61, 0x2d, 0x7a, 0x30, 0x2d, 0x39, 0x5d, 0x24, 0x52, 0x0b, 0x73, 0x75, 0x62, - 0x6a, 0x65, 0x63, 0x74, 0x54, 0x79, 0x70, 0x65, 0x12, 0x5a, 0x0a, 0x13, 0x6f, 0x70, 0x74, 0x69, - 0x6f, 0x6e, 0x61, 0x6c, 0x5f, 0x73, 0x75, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x5f, 0x69, 0x64, 0x18, - 0x02, 0x20, 0x01, 0x28, 0x09, 0x42, 0x2a, 0xfa, 0x42, 0x27, 0x72, 0x25, 0x28, 0x80, 0x08, 0x32, - 0x20, 0x5e, 0x28, 0x28, 0x5b, 0x61, 0x2d, 0x7a, 0x41, 0x2d, 0x5a, 0x30, 0x2d, 0x39, 0x2f, 0x5f, - 0x7c, 0x5c, 0x2d, 0x3d, 0x2b, 0x5d, 0x7b, 0x31, 0x2c, 0x7d, 0x29, 0x7c, 0x5c, 0x2a, 0x29, 0x3f, - 0x24, 0x52, 0x11, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x61, 0x6c, 0x53, 0x75, 0x62, 0x6a, 0x65, - 0x63, 0x74, 0x49, 0x64, 0x12, 0x52, 0x0a, 0x11, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x61, 0x6c, - 0x5f, 0x72, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, - 0x25, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x75, 0x62, 0x6a, 0x65, 0x63, - 0x74, 0x46, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x2e, 0x52, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, - 0x46, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x52, 0x10, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x61, 0x6c, - 0x52, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x1a, 0x58, 0x0a, 0x0e, 0x52, 0x65, 0x6c, 0x61, - 0x74, 0x69, 0x6f, 0x6e, 0x46, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x12, 0x46, 0x0a, 0x08, 0x72, 0x65, - 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x42, 0x2a, 0xfa, 0x42, - 0x27, 0x72, 0x25, 0x28, 0x40, 0x32, 0x21, 0x5e, 0x28, 0x5b, 0x61, 0x2d, 0x7a, 0x5d, 0x5b, 0x61, - 0x2d, 0x7a, 0x30, 0x2d, 0x39, 0x5f, 0x5d, 0x7b, 0x31, 0x2c, 0x36, 0x32, 0x7d, 0x5b, 0x61, 0x2d, - 0x7a, 0x30, 0x2d, 0x39, 0x5d, 0x29, 0x3f, 0x24, 0x52, 0x08, 0x72, 0x65, 0x6c, 0x61, 0x74, 0x69, - 0x6f, 0x6e, 0x42, 0x8a, 0x01, 0x0a, 0x0b, 0x63, 0x6f, 0x6d, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, - 0x76, 0x31, 0x42, 0x09, 0x43, 0x6f, 0x72, 0x65, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x50, 0x01, 0x5a, - 0x33, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x61, 0x75, 0x74, 0x68, - 0x7a, 0x65, 0x64, 0x2f, 0x73, 0x70, 0x69, 0x63, 0x65, 0x64, 0x62, 0x2f, 0x70, 0x6b, 0x67, 0x2f, - 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x63, 0x6f, 0x72, 0x65, 0x2f, 0x76, 0x31, 0x3b, 0x63, 0x6f, - 0x72, 0x65, 0x76, 0x31, 0xa2, 0x02, 0x03, 0x43, 0x58, 0x58, 0xaa, 0x02, 0x07, 0x43, 0x6f, 0x72, - 0x65, 0x2e, 0x56, 0x31, 0xca, 0x02, 0x07, 0x43, 0x6f, 0x72, 0x65, 0x5c, 0x56, 0x31, 0xe2, 0x02, - 0x13, 0x43, 0x6f, 0x72, 0x65, 0x5c, 0x56, 0x31, 0x5c, 0x47, 0x50, 0x42, 0x4d, 0x65, 0x74, 0x61, - 0x64, 0x61, 0x74, 0x61, 0xea, 0x02, 0x08, 0x43, 0x6f, 0x72, 0x65, 0x3a, 0x3a, 0x56, 0x31, 0x62, - 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, + 0x36, 0x32, 0x7d, 0x5b, 0x61, 0x2d, 0x7a, 0x30, 0x2d, 0x39, 0x5d, 0x29, 0x3f, 0x24, 0x52, 0x08, + 0x72, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x42, 0x8a, 0x01, 0x0a, 0x0b, 0x63, 0x6f, 0x6d, + 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x76, 0x31, 0x42, 0x09, 0x43, 0x6f, 0x72, 0x65, 0x50, 0x72, + 0x6f, 0x74, 0x6f, 0x50, 0x01, 0x5a, 0x33, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, + 0x6d, 0x2f, 0x61, 0x75, 0x74, 0x68, 0x7a, 0x65, 0x64, 0x2f, 0x73, 0x70, 0x69, 0x63, 0x65, 0x64, + 0x62, 0x2f, 0x70, 0x6b, 0x67, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x63, 0x6f, 0x72, 0x65, + 0x2f, 0x76, 0x31, 0x3b, 0x63, 0x6f, 0x72, 0x65, 0x76, 0x31, 0xa2, 0x02, 0x03, 0x43, 0x58, 0x58, + 0xaa, 0x02, 0x07, 0x43, 0x6f, 0x72, 0x65, 0x2e, 0x56, 0x31, 0xca, 0x02, 0x07, 0x43, 0x6f, 0x72, + 0x65, 0x5c, 0x56, 0x31, 0xe2, 0x02, 0x13, 0x43, 0x6f, 0x72, 0x65, 0x5c, 0x56, 0x31, 0x5c, 0x47, + 0x50, 0x42, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0xea, 0x02, 0x08, 0x43, 0x6f, 0x72, + 0x65, 0x3a, 0x3a, 0x56, 0x31, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, } var ( @@ -3525,7 +3615,7 @@ func file_core_v1_core_proto_rawDescGZIP() []byte { } var file_core_v1_core_proto_enumTypes = make([]protoimpl.EnumInfo, 7) -var file_core_v1_core_proto_msgTypes = make([]protoimpl.MessageInfo, 41) +var file_core_v1_core_proto_msgTypes = make([]protoimpl.MessageInfo, 42) var file_core_v1_core_proto_goTypes = []any{ (RelationTupleUpdate_Operation)(0), // 0: core.v1.RelationTupleUpdate.Operation (SetOperationUserset_Operation)(0), // 1: core.v1.SetOperationUserset.Operation @@ -3535,123 +3625,127 @@ var file_core_v1_core_proto_goTypes = []any{ (ComputedUserset_Object)(0), // 5: core.v1.ComputedUserset.Object (CaveatOperation_Operation)(0), // 6: core.v1.CaveatOperation.Operation (*RelationTuple)(nil), // 7: core.v1.RelationTuple - (*ContextualizedCaveat)(nil), // 8: core.v1.ContextualizedCaveat - (*CaveatDefinition)(nil), // 9: core.v1.CaveatDefinition - (*CaveatTypeReference)(nil), // 10: core.v1.CaveatTypeReference - (*ObjectAndRelation)(nil), // 11: core.v1.ObjectAndRelation - (*RelationReference)(nil), // 12: core.v1.RelationReference - (*Zookie)(nil), // 13: core.v1.Zookie - (*RelationTupleUpdate)(nil), // 14: core.v1.RelationTupleUpdate - (*RelationTupleTreeNode)(nil), // 15: core.v1.RelationTupleTreeNode - (*SetOperationUserset)(nil), // 16: core.v1.SetOperationUserset - (*DirectSubject)(nil), // 17: core.v1.DirectSubject - (*DirectSubjects)(nil), // 18: core.v1.DirectSubjects - (*Metadata)(nil), // 19: core.v1.Metadata - (*NamespaceDefinition)(nil), // 20: core.v1.NamespaceDefinition - (*Relation)(nil), // 21: core.v1.Relation - (*ReachabilityGraph)(nil), // 22: core.v1.ReachabilityGraph - (*ReachabilityEntrypoints)(nil), // 23: core.v1.ReachabilityEntrypoints - (*ReachabilityEntrypoint)(nil), // 24: core.v1.ReachabilityEntrypoint - (*TypeInformation)(nil), // 25: core.v1.TypeInformation - (*AllowedRelation)(nil), // 26: core.v1.AllowedRelation - (*AllowedCaveat)(nil), // 27: core.v1.AllowedCaveat - (*UsersetRewrite)(nil), // 28: core.v1.UsersetRewrite - (*SetOperation)(nil), // 29: core.v1.SetOperation - (*TupleToUserset)(nil), // 30: core.v1.TupleToUserset - (*FunctionedTupleToUserset)(nil), // 31: core.v1.FunctionedTupleToUserset - (*ComputedUserset)(nil), // 32: core.v1.ComputedUserset - (*SourcePosition)(nil), // 33: core.v1.SourcePosition - (*CaveatExpression)(nil), // 34: core.v1.CaveatExpression - (*CaveatOperation)(nil), // 35: core.v1.CaveatOperation - (*RelationshipFilter)(nil), // 36: core.v1.RelationshipFilter - (*SubjectFilter)(nil), // 37: core.v1.SubjectFilter - nil, // 38: core.v1.CaveatDefinition.ParameterTypesEntry - nil, // 39: core.v1.ReachabilityGraph.EntrypointsBySubjectTypeEntry - nil, // 40: core.v1.ReachabilityGraph.EntrypointsBySubjectRelationEntry - (*AllowedRelation_PublicWildcard)(nil), // 41: core.v1.AllowedRelation.PublicWildcard - (*SetOperation_Child)(nil), // 42: core.v1.SetOperation.Child - (*SetOperation_Child_This)(nil), // 43: core.v1.SetOperation.Child.This - (*SetOperation_Child_Nil)(nil), // 44: core.v1.SetOperation.Child.Nil - (*TupleToUserset_Tupleset)(nil), // 45: core.v1.TupleToUserset.Tupleset - (*FunctionedTupleToUserset_Tupleset)(nil), // 46: core.v1.FunctionedTupleToUserset.Tupleset - (*SubjectFilter_RelationFilter)(nil), // 47: core.v1.SubjectFilter.RelationFilter - (*structpb.Struct)(nil), // 48: google.protobuf.Struct - (*anypb.Any)(nil), // 49: google.protobuf.Any + (*RelationshipIntegrity)(nil), // 8: core.v1.RelationshipIntegrity + (*ContextualizedCaveat)(nil), // 9: core.v1.ContextualizedCaveat + (*CaveatDefinition)(nil), // 10: core.v1.CaveatDefinition + (*CaveatTypeReference)(nil), // 11: core.v1.CaveatTypeReference + (*ObjectAndRelation)(nil), // 12: core.v1.ObjectAndRelation + (*RelationReference)(nil), // 13: core.v1.RelationReference + (*Zookie)(nil), // 14: core.v1.Zookie + (*RelationTupleUpdate)(nil), // 15: core.v1.RelationTupleUpdate + (*RelationTupleTreeNode)(nil), // 16: core.v1.RelationTupleTreeNode + (*SetOperationUserset)(nil), // 17: core.v1.SetOperationUserset + (*DirectSubject)(nil), // 18: core.v1.DirectSubject + (*DirectSubjects)(nil), // 19: core.v1.DirectSubjects + (*Metadata)(nil), // 20: core.v1.Metadata + (*NamespaceDefinition)(nil), // 21: core.v1.NamespaceDefinition + (*Relation)(nil), // 22: core.v1.Relation + (*ReachabilityGraph)(nil), // 23: core.v1.ReachabilityGraph + (*ReachabilityEntrypoints)(nil), // 24: core.v1.ReachabilityEntrypoints + (*ReachabilityEntrypoint)(nil), // 25: core.v1.ReachabilityEntrypoint + (*TypeInformation)(nil), // 26: core.v1.TypeInformation + (*AllowedRelation)(nil), // 27: core.v1.AllowedRelation + (*AllowedCaveat)(nil), // 28: core.v1.AllowedCaveat + (*UsersetRewrite)(nil), // 29: core.v1.UsersetRewrite + (*SetOperation)(nil), // 30: core.v1.SetOperation + (*TupleToUserset)(nil), // 31: core.v1.TupleToUserset + (*FunctionedTupleToUserset)(nil), // 32: core.v1.FunctionedTupleToUserset + (*ComputedUserset)(nil), // 33: core.v1.ComputedUserset + (*SourcePosition)(nil), // 34: core.v1.SourcePosition + (*CaveatExpression)(nil), // 35: core.v1.CaveatExpression + (*CaveatOperation)(nil), // 36: core.v1.CaveatOperation + (*RelationshipFilter)(nil), // 37: core.v1.RelationshipFilter + (*SubjectFilter)(nil), // 38: core.v1.SubjectFilter + nil, // 39: core.v1.CaveatDefinition.ParameterTypesEntry + nil, // 40: core.v1.ReachabilityGraph.EntrypointsBySubjectTypeEntry + nil, // 41: core.v1.ReachabilityGraph.EntrypointsBySubjectRelationEntry + (*AllowedRelation_PublicWildcard)(nil), // 42: core.v1.AllowedRelation.PublicWildcard + (*SetOperation_Child)(nil), // 43: core.v1.SetOperation.Child + (*SetOperation_Child_This)(nil), // 44: core.v1.SetOperation.Child.This + (*SetOperation_Child_Nil)(nil), // 45: core.v1.SetOperation.Child.Nil + (*TupleToUserset_Tupleset)(nil), // 46: core.v1.TupleToUserset.Tupleset + (*FunctionedTupleToUserset_Tupleset)(nil), // 47: core.v1.FunctionedTupleToUserset.Tupleset + (*SubjectFilter_RelationFilter)(nil), // 48: core.v1.SubjectFilter.RelationFilter + (*timestamppb.Timestamp)(nil), // 49: google.protobuf.Timestamp + (*structpb.Struct)(nil), // 50: google.protobuf.Struct + (*anypb.Any)(nil), // 51: google.protobuf.Any } var file_core_v1_core_proto_depIdxs = []int32{ - 11, // 0: core.v1.RelationTuple.resource_and_relation:type_name -> core.v1.ObjectAndRelation - 11, // 1: core.v1.RelationTuple.subject:type_name -> core.v1.ObjectAndRelation - 8, // 2: core.v1.RelationTuple.caveat:type_name -> core.v1.ContextualizedCaveat - 48, // 3: core.v1.ContextualizedCaveat.context:type_name -> google.protobuf.Struct - 38, // 4: core.v1.CaveatDefinition.parameter_types:type_name -> core.v1.CaveatDefinition.ParameterTypesEntry - 19, // 5: core.v1.CaveatDefinition.metadata:type_name -> core.v1.Metadata - 33, // 6: core.v1.CaveatDefinition.source_position:type_name -> core.v1.SourcePosition - 10, // 7: core.v1.CaveatTypeReference.child_types:type_name -> core.v1.CaveatTypeReference - 0, // 8: core.v1.RelationTupleUpdate.operation:type_name -> core.v1.RelationTupleUpdate.Operation - 7, // 9: core.v1.RelationTupleUpdate.tuple:type_name -> core.v1.RelationTuple - 16, // 10: core.v1.RelationTupleTreeNode.intermediate_node:type_name -> core.v1.SetOperationUserset - 18, // 11: core.v1.RelationTupleTreeNode.leaf_node:type_name -> core.v1.DirectSubjects - 11, // 12: core.v1.RelationTupleTreeNode.expanded:type_name -> core.v1.ObjectAndRelation - 34, // 13: core.v1.RelationTupleTreeNode.caveat_expression:type_name -> core.v1.CaveatExpression - 1, // 14: core.v1.SetOperationUserset.operation:type_name -> core.v1.SetOperationUserset.Operation - 15, // 15: core.v1.SetOperationUserset.child_nodes:type_name -> core.v1.RelationTupleTreeNode - 11, // 16: core.v1.DirectSubject.subject:type_name -> core.v1.ObjectAndRelation - 34, // 17: core.v1.DirectSubject.caveat_expression:type_name -> core.v1.CaveatExpression - 17, // 18: core.v1.DirectSubjects.subjects:type_name -> core.v1.DirectSubject - 49, // 19: core.v1.Metadata.metadata_message:type_name -> google.protobuf.Any - 21, // 20: core.v1.NamespaceDefinition.relation:type_name -> core.v1.Relation - 19, // 21: core.v1.NamespaceDefinition.metadata:type_name -> core.v1.Metadata - 33, // 22: core.v1.NamespaceDefinition.source_position:type_name -> core.v1.SourcePosition - 28, // 23: core.v1.Relation.userset_rewrite:type_name -> core.v1.UsersetRewrite - 25, // 24: core.v1.Relation.type_information:type_name -> core.v1.TypeInformation - 19, // 25: core.v1.Relation.metadata:type_name -> core.v1.Metadata - 33, // 26: core.v1.Relation.source_position:type_name -> core.v1.SourcePosition - 39, // 27: core.v1.ReachabilityGraph.entrypoints_by_subject_type:type_name -> core.v1.ReachabilityGraph.EntrypointsBySubjectTypeEntry - 40, // 28: core.v1.ReachabilityGraph.entrypoints_by_subject_relation:type_name -> core.v1.ReachabilityGraph.EntrypointsBySubjectRelationEntry - 24, // 29: core.v1.ReachabilityEntrypoints.entrypoints:type_name -> core.v1.ReachabilityEntrypoint - 12, // 30: core.v1.ReachabilityEntrypoints.subject_relation:type_name -> core.v1.RelationReference - 2, // 31: core.v1.ReachabilityEntrypoint.kind:type_name -> core.v1.ReachabilityEntrypoint.ReachabilityEntrypointKind - 12, // 32: core.v1.ReachabilityEntrypoint.target_relation:type_name -> core.v1.RelationReference - 3, // 33: core.v1.ReachabilityEntrypoint.result_status:type_name -> core.v1.ReachabilityEntrypoint.EntrypointResultStatus - 26, // 34: core.v1.TypeInformation.allowed_direct_relations:type_name -> core.v1.AllowedRelation - 41, // 35: core.v1.AllowedRelation.public_wildcard:type_name -> core.v1.AllowedRelation.PublicWildcard - 33, // 36: core.v1.AllowedRelation.source_position:type_name -> core.v1.SourcePosition - 27, // 37: core.v1.AllowedRelation.required_caveat:type_name -> core.v1.AllowedCaveat - 29, // 38: core.v1.UsersetRewrite.union:type_name -> core.v1.SetOperation - 29, // 39: core.v1.UsersetRewrite.intersection:type_name -> core.v1.SetOperation - 29, // 40: core.v1.UsersetRewrite.exclusion:type_name -> core.v1.SetOperation - 33, // 41: core.v1.UsersetRewrite.source_position:type_name -> core.v1.SourcePosition - 42, // 42: core.v1.SetOperation.child:type_name -> core.v1.SetOperation.Child - 45, // 43: core.v1.TupleToUserset.tupleset:type_name -> core.v1.TupleToUserset.Tupleset - 32, // 44: core.v1.TupleToUserset.computed_userset:type_name -> core.v1.ComputedUserset - 33, // 45: core.v1.TupleToUserset.source_position:type_name -> core.v1.SourcePosition - 4, // 46: core.v1.FunctionedTupleToUserset.function:type_name -> core.v1.FunctionedTupleToUserset.Function - 46, // 47: core.v1.FunctionedTupleToUserset.tupleset:type_name -> core.v1.FunctionedTupleToUserset.Tupleset - 32, // 48: core.v1.FunctionedTupleToUserset.computed_userset:type_name -> core.v1.ComputedUserset - 33, // 49: core.v1.FunctionedTupleToUserset.source_position:type_name -> core.v1.SourcePosition - 5, // 50: core.v1.ComputedUserset.object:type_name -> core.v1.ComputedUserset.Object - 33, // 51: core.v1.ComputedUserset.source_position:type_name -> core.v1.SourcePosition - 35, // 52: core.v1.CaveatExpression.operation:type_name -> core.v1.CaveatOperation - 8, // 53: core.v1.CaveatExpression.caveat:type_name -> core.v1.ContextualizedCaveat - 6, // 54: core.v1.CaveatOperation.op:type_name -> core.v1.CaveatOperation.Operation - 34, // 55: core.v1.CaveatOperation.children:type_name -> core.v1.CaveatExpression - 37, // 56: core.v1.RelationshipFilter.optional_subject_filter:type_name -> core.v1.SubjectFilter - 47, // 57: core.v1.SubjectFilter.optional_relation:type_name -> core.v1.SubjectFilter.RelationFilter - 10, // 58: core.v1.CaveatDefinition.ParameterTypesEntry.value:type_name -> core.v1.CaveatTypeReference - 23, // 59: core.v1.ReachabilityGraph.EntrypointsBySubjectTypeEntry.value:type_name -> core.v1.ReachabilityEntrypoints - 23, // 60: core.v1.ReachabilityGraph.EntrypointsBySubjectRelationEntry.value:type_name -> core.v1.ReachabilityEntrypoints - 43, // 61: core.v1.SetOperation.Child._this:type_name -> core.v1.SetOperation.Child.This - 32, // 62: core.v1.SetOperation.Child.computed_userset:type_name -> core.v1.ComputedUserset - 30, // 63: core.v1.SetOperation.Child.tuple_to_userset:type_name -> core.v1.TupleToUserset - 28, // 64: core.v1.SetOperation.Child.userset_rewrite:type_name -> core.v1.UsersetRewrite - 31, // 65: core.v1.SetOperation.Child.functioned_tuple_to_userset:type_name -> core.v1.FunctionedTupleToUserset - 44, // 66: core.v1.SetOperation.Child._nil:type_name -> core.v1.SetOperation.Child.Nil - 33, // 67: core.v1.SetOperation.Child.source_position:type_name -> core.v1.SourcePosition - 68, // [68:68] is the sub-list for method output_type - 68, // [68:68] is the sub-list for method input_type - 68, // [68:68] is the sub-list for extension type_name - 68, // [68:68] is the sub-list for extension extendee - 0, // [0:68] is the sub-list for field type_name + 12, // 0: core.v1.RelationTuple.resource_and_relation:type_name -> core.v1.ObjectAndRelation + 12, // 1: core.v1.RelationTuple.subject:type_name -> core.v1.ObjectAndRelation + 9, // 2: core.v1.RelationTuple.caveat:type_name -> core.v1.ContextualizedCaveat + 8, // 3: core.v1.RelationTuple.integrity:type_name -> core.v1.RelationshipIntegrity + 49, // 4: core.v1.RelationshipIntegrity.hashed_at:type_name -> google.protobuf.Timestamp + 50, // 5: core.v1.ContextualizedCaveat.context:type_name -> google.protobuf.Struct + 39, // 6: core.v1.CaveatDefinition.parameter_types:type_name -> core.v1.CaveatDefinition.ParameterTypesEntry + 20, // 7: core.v1.CaveatDefinition.metadata:type_name -> core.v1.Metadata + 34, // 8: core.v1.CaveatDefinition.source_position:type_name -> core.v1.SourcePosition + 11, // 9: core.v1.CaveatTypeReference.child_types:type_name -> core.v1.CaveatTypeReference + 0, // 10: core.v1.RelationTupleUpdate.operation:type_name -> core.v1.RelationTupleUpdate.Operation + 7, // 11: core.v1.RelationTupleUpdate.tuple:type_name -> core.v1.RelationTuple + 17, // 12: core.v1.RelationTupleTreeNode.intermediate_node:type_name -> core.v1.SetOperationUserset + 19, // 13: core.v1.RelationTupleTreeNode.leaf_node:type_name -> core.v1.DirectSubjects + 12, // 14: core.v1.RelationTupleTreeNode.expanded:type_name -> core.v1.ObjectAndRelation + 35, // 15: core.v1.RelationTupleTreeNode.caveat_expression:type_name -> core.v1.CaveatExpression + 1, // 16: core.v1.SetOperationUserset.operation:type_name -> core.v1.SetOperationUserset.Operation + 16, // 17: core.v1.SetOperationUserset.child_nodes:type_name -> core.v1.RelationTupleTreeNode + 12, // 18: core.v1.DirectSubject.subject:type_name -> core.v1.ObjectAndRelation + 35, // 19: core.v1.DirectSubject.caveat_expression:type_name -> core.v1.CaveatExpression + 18, // 20: core.v1.DirectSubjects.subjects:type_name -> core.v1.DirectSubject + 51, // 21: core.v1.Metadata.metadata_message:type_name -> google.protobuf.Any + 22, // 22: core.v1.NamespaceDefinition.relation:type_name -> core.v1.Relation + 20, // 23: core.v1.NamespaceDefinition.metadata:type_name -> core.v1.Metadata + 34, // 24: core.v1.NamespaceDefinition.source_position:type_name -> core.v1.SourcePosition + 29, // 25: core.v1.Relation.userset_rewrite:type_name -> core.v1.UsersetRewrite + 26, // 26: core.v1.Relation.type_information:type_name -> core.v1.TypeInformation + 20, // 27: core.v1.Relation.metadata:type_name -> core.v1.Metadata + 34, // 28: core.v1.Relation.source_position:type_name -> core.v1.SourcePosition + 40, // 29: core.v1.ReachabilityGraph.entrypoints_by_subject_type:type_name -> core.v1.ReachabilityGraph.EntrypointsBySubjectTypeEntry + 41, // 30: core.v1.ReachabilityGraph.entrypoints_by_subject_relation:type_name -> core.v1.ReachabilityGraph.EntrypointsBySubjectRelationEntry + 25, // 31: core.v1.ReachabilityEntrypoints.entrypoints:type_name -> core.v1.ReachabilityEntrypoint + 13, // 32: core.v1.ReachabilityEntrypoints.subject_relation:type_name -> core.v1.RelationReference + 2, // 33: core.v1.ReachabilityEntrypoint.kind:type_name -> core.v1.ReachabilityEntrypoint.ReachabilityEntrypointKind + 13, // 34: core.v1.ReachabilityEntrypoint.target_relation:type_name -> core.v1.RelationReference + 3, // 35: core.v1.ReachabilityEntrypoint.result_status:type_name -> core.v1.ReachabilityEntrypoint.EntrypointResultStatus + 27, // 36: core.v1.TypeInformation.allowed_direct_relations:type_name -> core.v1.AllowedRelation + 42, // 37: core.v1.AllowedRelation.public_wildcard:type_name -> core.v1.AllowedRelation.PublicWildcard + 34, // 38: core.v1.AllowedRelation.source_position:type_name -> core.v1.SourcePosition + 28, // 39: core.v1.AllowedRelation.required_caveat:type_name -> core.v1.AllowedCaveat + 30, // 40: core.v1.UsersetRewrite.union:type_name -> core.v1.SetOperation + 30, // 41: core.v1.UsersetRewrite.intersection:type_name -> core.v1.SetOperation + 30, // 42: core.v1.UsersetRewrite.exclusion:type_name -> core.v1.SetOperation + 34, // 43: core.v1.UsersetRewrite.source_position:type_name -> core.v1.SourcePosition + 43, // 44: core.v1.SetOperation.child:type_name -> core.v1.SetOperation.Child + 46, // 45: core.v1.TupleToUserset.tupleset:type_name -> core.v1.TupleToUserset.Tupleset + 33, // 46: core.v1.TupleToUserset.computed_userset:type_name -> core.v1.ComputedUserset + 34, // 47: core.v1.TupleToUserset.source_position:type_name -> core.v1.SourcePosition + 4, // 48: core.v1.FunctionedTupleToUserset.function:type_name -> core.v1.FunctionedTupleToUserset.Function + 47, // 49: core.v1.FunctionedTupleToUserset.tupleset:type_name -> core.v1.FunctionedTupleToUserset.Tupleset + 33, // 50: core.v1.FunctionedTupleToUserset.computed_userset:type_name -> core.v1.ComputedUserset + 34, // 51: core.v1.FunctionedTupleToUserset.source_position:type_name -> core.v1.SourcePosition + 5, // 52: core.v1.ComputedUserset.object:type_name -> core.v1.ComputedUserset.Object + 34, // 53: core.v1.ComputedUserset.source_position:type_name -> core.v1.SourcePosition + 36, // 54: core.v1.CaveatExpression.operation:type_name -> core.v1.CaveatOperation + 9, // 55: core.v1.CaveatExpression.caveat:type_name -> core.v1.ContextualizedCaveat + 6, // 56: core.v1.CaveatOperation.op:type_name -> core.v1.CaveatOperation.Operation + 35, // 57: core.v1.CaveatOperation.children:type_name -> core.v1.CaveatExpression + 38, // 58: core.v1.RelationshipFilter.optional_subject_filter:type_name -> core.v1.SubjectFilter + 48, // 59: core.v1.SubjectFilter.optional_relation:type_name -> core.v1.SubjectFilter.RelationFilter + 11, // 60: core.v1.CaveatDefinition.ParameterTypesEntry.value:type_name -> core.v1.CaveatTypeReference + 24, // 61: core.v1.ReachabilityGraph.EntrypointsBySubjectTypeEntry.value:type_name -> core.v1.ReachabilityEntrypoints + 24, // 62: core.v1.ReachabilityGraph.EntrypointsBySubjectRelationEntry.value:type_name -> core.v1.ReachabilityEntrypoints + 44, // 63: core.v1.SetOperation.Child._this:type_name -> core.v1.SetOperation.Child.This + 33, // 64: core.v1.SetOperation.Child.computed_userset:type_name -> core.v1.ComputedUserset + 31, // 65: core.v1.SetOperation.Child.tuple_to_userset:type_name -> core.v1.TupleToUserset + 29, // 66: core.v1.SetOperation.Child.userset_rewrite:type_name -> core.v1.UsersetRewrite + 32, // 67: core.v1.SetOperation.Child.functioned_tuple_to_userset:type_name -> core.v1.FunctionedTupleToUserset + 45, // 68: core.v1.SetOperation.Child._nil:type_name -> core.v1.SetOperation.Child.Nil + 34, // 69: core.v1.SetOperation.Child.source_position:type_name -> core.v1.SourcePosition + 70, // [70:70] is the sub-list for method output_type + 70, // [70:70] is the sub-list for method input_type + 70, // [70:70] is the sub-list for extension type_name + 70, // [70:70] is the sub-list for extension extendee + 0, // [0:70] is the sub-list for field type_name } func init() { file_core_v1_core_proto_init() } @@ -3673,7 +3767,7 @@ func file_core_v1_core_proto_init() { } } file_core_v1_core_proto_msgTypes[1].Exporter = func(v any, i int) any { - switch v := v.(*ContextualizedCaveat); i { + switch v := v.(*RelationshipIntegrity); i { case 0: return &v.state case 1: @@ -3685,7 +3779,7 @@ func file_core_v1_core_proto_init() { } } file_core_v1_core_proto_msgTypes[2].Exporter = func(v any, i int) any { - switch v := v.(*CaveatDefinition); i { + switch v := v.(*ContextualizedCaveat); i { case 0: return &v.state case 1: @@ -3697,7 +3791,7 @@ func file_core_v1_core_proto_init() { } } file_core_v1_core_proto_msgTypes[3].Exporter = func(v any, i int) any { - switch v := v.(*CaveatTypeReference); i { + switch v := v.(*CaveatDefinition); i { case 0: return &v.state case 1: @@ -3709,7 +3803,7 @@ func file_core_v1_core_proto_init() { } } file_core_v1_core_proto_msgTypes[4].Exporter = func(v any, i int) any { - switch v := v.(*ObjectAndRelation); i { + switch v := v.(*CaveatTypeReference); i { case 0: return &v.state case 1: @@ -3721,7 +3815,7 @@ func file_core_v1_core_proto_init() { } } file_core_v1_core_proto_msgTypes[5].Exporter = func(v any, i int) any { - switch v := v.(*RelationReference); i { + switch v := v.(*ObjectAndRelation); i { case 0: return &v.state case 1: @@ -3733,7 +3827,7 @@ func file_core_v1_core_proto_init() { } } file_core_v1_core_proto_msgTypes[6].Exporter = func(v any, i int) any { - switch v := v.(*Zookie); i { + switch v := v.(*RelationReference); i { case 0: return &v.state case 1: @@ -3745,7 +3839,7 @@ func file_core_v1_core_proto_init() { } } file_core_v1_core_proto_msgTypes[7].Exporter = func(v any, i int) any { - switch v := v.(*RelationTupleUpdate); i { + switch v := v.(*Zookie); i { case 0: return &v.state case 1: @@ -3757,7 +3851,7 @@ func file_core_v1_core_proto_init() { } } file_core_v1_core_proto_msgTypes[8].Exporter = func(v any, i int) any { - switch v := v.(*RelationTupleTreeNode); i { + switch v := v.(*RelationTupleUpdate); i { case 0: return &v.state case 1: @@ -3769,7 +3863,7 @@ func file_core_v1_core_proto_init() { } } file_core_v1_core_proto_msgTypes[9].Exporter = func(v any, i int) any { - switch v := v.(*SetOperationUserset); i { + switch v := v.(*RelationTupleTreeNode); i { case 0: return &v.state case 1: @@ -3781,7 +3875,7 @@ func file_core_v1_core_proto_init() { } } file_core_v1_core_proto_msgTypes[10].Exporter = func(v any, i int) any { - switch v := v.(*DirectSubject); i { + switch v := v.(*SetOperationUserset); i { case 0: return &v.state case 1: @@ -3793,7 +3887,7 @@ func file_core_v1_core_proto_init() { } } file_core_v1_core_proto_msgTypes[11].Exporter = func(v any, i int) any { - switch v := v.(*DirectSubjects); i { + switch v := v.(*DirectSubject); i { case 0: return &v.state case 1: @@ -3805,7 +3899,7 @@ func file_core_v1_core_proto_init() { } } file_core_v1_core_proto_msgTypes[12].Exporter = func(v any, i int) any { - switch v := v.(*Metadata); i { + switch v := v.(*DirectSubjects); i { case 0: return &v.state case 1: @@ -3817,7 +3911,7 @@ func file_core_v1_core_proto_init() { } } file_core_v1_core_proto_msgTypes[13].Exporter = func(v any, i int) any { - switch v := v.(*NamespaceDefinition); i { + switch v := v.(*Metadata); i { case 0: return &v.state case 1: @@ -3829,7 +3923,7 @@ func file_core_v1_core_proto_init() { } } file_core_v1_core_proto_msgTypes[14].Exporter = func(v any, i int) any { - switch v := v.(*Relation); i { + switch v := v.(*NamespaceDefinition); i { case 0: return &v.state case 1: @@ -3841,7 +3935,7 @@ func file_core_v1_core_proto_init() { } } file_core_v1_core_proto_msgTypes[15].Exporter = func(v any, i int) any { - switch v := v.(*ReachabilityGraph); i { + switch v := v.(*Relation); i { case 0: return &v.state case 1: @@ -3853,7 +3947,7 @@ func file_core_v1_core_proto_init() { } } file_core_v1_core_proto_msgTypes[16].Exporter = func(v any, i int) any { - switch v := v.(*ReachabilityEntrypoints); i { + switch v := v.(*ReachabilityGraph); i { case 0: return &v.state case 1: @@ -3865,7 +3959,7 @@ func file_core_v1_core_proto_init() { } } file_core_v1_core_proto_msgTypes[17].Exporter = func(v any, i int) any { - switch v := v.(*ReachabilityEntrypoint); i { + switch v := v.(*ReachabilityEntrypoints); i { case 0: return &v.state case 1: @@ -3877,7 +3971,7 @@ func file_core_v1_core_proto_init() { } } file_core_v1_core_proto_msgTypes[18].Exporter = func(v any, i int) any { - switch v := v.(*TypeInformation); i { + switch v := v.(*ReachabilityEntrypoint); i { case 0: return &v.state case 1: @@ -3889,7 +3983,7 @@ func file_core_v1_core_proto_init() { } } file_core_v1_core_proto_msgTypes[19].Exporter = func(v any, i int) any { - switch v := v.(*AllowedRelation); i { + switch v := v.(*TypeInformation); i { case 0: return &v.state case 1: @@ -3901,7 +3995,7 @@ func file_core_v1_core_proto_init() { } } file_core_v1_core_proto_msgTypes[20].Exporter = func(v any, i int) any { - switch v := v.(*AllowedCaveat); i { + switch v := v.(*AllowedRelation); i { case 0: return &v.state case 1: @@ -3913,7 +4007,7 @@ func file_core_v1_core_proto_init() { } } file_core_v1_core_proto_msgTypes[21].Exporter = func(v any, i int) any { - switch v := v.(*UsersetRewrite); i { + switch v := v.(*AllowedCaveat); i { case 0: return &v.state case 1: @@ -3925,7 +4019,7 @@ func file_core_v1_core_proto_init() { } } file_core_v1_core_proto_msgTypes[22].Exporter = func(v any, i int) any { - switch v := v.(*SetOperation); i { + switch v := v.(*UsersetRewrite); i { case 0: return &v.state case 1: @@ -3937,7 +4031,7 @@ func file_core_v1_core_proto_init() { } } file_core_v1_core_proto_msgTypes[23].Exporter = func(v any, i int) any { - switch v := v.(*TupleToUserset); i { + switch v := v.(*SetOperation); i { case 0: return &v.state case 1: @@ -3949,7 +4043,7 @@ func file_core_v1_core_proto_init() { } } file_core_v1_core_proto_msgTypes[24].Exporter = func(v any, i int) any { - switch v := v.(*FunctionedTupleToUserset); i { + switch v := v.(*TupleToUserset); i { case 0: return &v.state case 1: @@ -3961,7 +4055,7 @@ func file_core_v1_core_proto_init() { } } file_core_v1_core_proto_msgTypes[25].Exporter = func(v any, i int) any { - switch v := v.(*ComputedUserset); i { + switch v := v.(*FunctionedTupleToUserset); i { case 0: return &v.state case 1: @@ -3973,7 +4067,7 @@ func file_core_v1_core_proto_init() { } } file_core_v1_core_proto_msgTypes[26].Exporter = func(v any, i int) any { - switch v := v.(*SourcePosition); i { + switch v := v.(*ComputedUserset); i { case 0: return &v.state case 1: @@ -3985,7 +4079,7 @@ func file_core_v1_core_proto_init() { } } file_core_v1_core_proto_msgTypes[27].Exporter = func(v any, i int) any { - switch v := v.(*CaveatExpression); i { + switch v := v.(*SourcePosition); i { case 0: return &v.state case 1: @@ -3997,7 +4091,7 @@ func file_core_v1_core_proto_init() { } } file_core_v1_core_proto_msgTypes[28].Exporter = func(v any, i int) any { - switch v := v.(*CaveatOperation); i { + switch v := v.(*CaveatExpression); i { case 0: return &v.state case 1: @@ -4009,7 +4103,7 @@ func file_core_v1_core_proto_init() { } } file_core_v1_core_proto_msgTypes[29].Exporter = func(v any, i int) any { - switch v := v.(*RelationshipFilter); i { + switch v := v.(*CaveatOperation); i { case 0: return &v.state case 1: @@ -4021,6 +4115,18 @@ func file_core_v1_core_proto_init() { } } file_core_v1_core_proto_msgTypes[30].Exporter = func(v any, i int) any { + switch v := v.(*RelationshipFilter); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_core_v1_core_proto_msgTypes[31].Exporter = func(v any, i int) any { switch v := v.(*SubjectFilter); i { case 0: return &v.state @@ -4032,7 +4138,7 @@ func file_core_v1_core_proto_init() { return nil } } - file_core_v1_core_proto_msgTypes[34].Exporter = func(v any, i int) any { + file_core_v1_core_proto_msgTypes[35].Exporter = func(v any, i int) any { switch v := v.(*AllowedRelation_PublicWildcard); i { case 0: return &v.state @@ -4044,7 +4150,7 @@ func file_core_v1_core_proto_init() { return nil } } - file_core_v1_core_proto_msgTypes[35].Exporter = func(v any, i int) any { + file_core_v1_core_proto_msgTypes[36].Exporter = func(v any, i int) any { switch v := v.(*SetOperation_Child); i { case 0: return &v.state @@ -4056,7 +4162,7 @@ func file_core_v1_core_proto_init() { return nil } } - file_core_v1_core_proto_msgTypes[36].Exporter = func(v any, i int) any { + file_core_v1_core_proto_msgTypes[37].Exporter = func(v any, i int) any { switch v := v.(*SetOperation_Child_This); i { case 0: return &v.state @@ -4068,7 +4174,7 @@ func file_core_v1_core_proto_init() { return nil } } - file_core_v1_core_proto_msgTypes[37].Exporter = func(v any, i int) any { + file_core_v1_core_proto_msgTypes[38].Exporter = func(v any, i int) any { switch v := v.(*SetOperation_Child_Nil); i { case 0: return &v.state @@ -4080,7 +4186,7 @@ func file_core_v1_core_proto_init() { return nil } } - file_core_v1_core_proto_msgTypes[38].Exporter = func(v any, i int) any { + file_core_v1_core_proto_msgTypes[39].Exporter = func(v any, i int) any { switch v := v.(*TupleToUserset_Tupleset); i { case 0: return &v.state @@ -4092,7 +4198,7 @@ func file_core_v1_core_proto_init() { return nil } } - file_core_v1_core_proto_msgTypes[39].Exporter = func(v any, i int) any { + file_core_v1_core_proto_msgTypes[40].Exporter = func(v any, i int) any { switch v := v.(*FunctionedTupleToUserset_Tupleset); i { case 0: return &v.state @@ -4104,7 +4210,7 @@ func file_core_v1_core_proto_init() { return nil } } - file_core_v1_core_proto_msgTypes[40].Exporter = func(v any, i int) any { + file_core_v1_core_proto_msgTypes[41].Exporter = func(v any, i int) any { switch v := v.(*SubjectFilter_RelationFilter); i { case 0: return &v.state @@ -4117,24 +4223,24 @@ func file_core_v1_core_proto_init() { } } } - file_core_v1_core_proto_msgTypes[8].OneofWrappers = []any{ + file_core_v1_core_proto_msgTypes[9].OneofWrappers = []any{ (*RelationTupleTreeNode_IntermediateNode)(nil), (*RelationTupleTreeNode_LeafNode)(nil), } - file_core_v1_core_proto_msgTypes[19].OneofWrappers = []any{ + file_core_v1_core_proto_msgTypes[20].OneofWrappers = []any{ (*AllowedRelation_Relation)(nil), (*AllowedRelation_PublicWildcard_)(nil), } - file_core_v1_core_proto_msgTypes[21].OneofWrappers = []any{ + file_core_v1_core_proto_msgTypes[22].OneofWrappers = []any{ (*UsersetRewrite_Union)(nil), (*UsersetRewrite_Intersection)(nil), (*UsersetRewrite_Exclusion)(nil), } - file_core_v1_core_proto_msgTypes[27].OneofWrappers = []any{ + file_core_v1_core_proto_msgTypes[28].OneofWrappers = []any{ (*CaveatExpression_Operation)(nil), (*CaveatExpression_Caveat)(nil), } - file_core_v1_core_proto_msgTypes[35].OneofWrappers = []any{ + file_core_v1_core_proto_msgTypes[36].OneofWrappers = []any{ (*SetOperation_Child_XThis)(nil), (*SetOperation_Child_ComputedUserset)(nil), (*SetOperation_Child_TupleToUserset)(nil), @@ -4148,7 +4254,7 @@ func file_core_v1_core_proto_init() { GoPackagePath: reflect.TypeOf(x{}).PkgPath(), RawDescriptor: file_core_v1_core_proto_rawDesc, NumEnums: 7, - NumMessages: 41, + NumMessages: 42, NumExtensions: 0, NumServices: 0, }, diff --git a/pkg/proto/core/v1/core.pb.validate.go b/pkg/proto/core/v1/core.pb.validate.go index 1c5d9896ed..b88f4c55aa 100644 --- a/pkg/proto/core/v1/core.pb.validate.go +++ b/pkg/proto/core/v1/core.pb.validate.go @@ -166,6 +166,35 @@ func (m *RelationTuple) validate(all bool) error { } } + if all { + switch v := interface{}(m.GetIntegrity()).(type) { + case interface{ ValidateAll() error }: + if err := v.ValidateAll(); err != nil { + errors = append(errors, RelationTupleValidationError{ + field: "Integrity", + reason: "embedded message failed validation", + cause: err, + }) + } + case interface{ Validate() error }: + if err := v.Validate(); err != nil { + errors = append(errors, RelationTupleValidationError{ + field: "Integrity", + reason: "embedded message failed validation", + cause: err, + }) + } + } + } else if v, ok := interface{}(m.GetIntegrity()).(interface{ Validate() error }); ok { + if err := v.Validate(); err != nil { + return RelationTupleValidationError{ + field: "Integrity", + reason: "embedded message failed validation", + cause: err, + } + } + } + if len(errors) > 0 { return RelationTupleMultiError(errors) } @@ -244,6 +273,141 @@ var _ interface { ErrorName() string } = RelationTupleValidationError{} +// Validate checks the field values on RelationshipIntegrity with the rules +// defined in the proto definition for this message. If any rules are +// violated, the first error encountered is returned, or nil if there are no violations. +func (m *RelationshipIntegrity) Validate() error { + return m.validate(false) +} + +// ValidateAll checks the field values on RelationshipIntegrity with the rules +// defined in the proto definition for this message. If any rules are +// violated, the result is a list of violation errors wrapped in +// RelationshipIntegrityMultiError, or nil if none found. +func (m *RelationshipIntegrity) ValidateAll() error { + return m.validate(true) +} + +func (m *RelationshipIntegrity) validate(all bool) error { + if m == nil { + return nil + } + + var errors []error + + // no validation rules for KeyId + + // no validation rules for Hash + + if all { + switch v := interface{}(m.GetHashedAt()).(type) { + case interface{ ValidateAll() error }: + if err := v.ValidateAll(); err != nil { + errors = append(errors, RelationshipIntegrityValidationError{ + field: "HashedAt", + reason: "embedded message failed validation", + cause: err, + }) + } + case interface{ Validate() error }: + if err := v.Validate(); err != nil { + errors = append(errors, RelationshipIntegrityValidationError{ + field: "HashedAt", + reason: "embedded message failed validation", + cause: err, + }) + } + } + } else if v, ok := interface{}(m.GetHashedAt()).(interface{ Validate() error }); ok { + if err := v.Validate(); err != nil { + return RelationshipIntegrityValidationError{ + field: "HashedAt", + reason: "embedded message failed validation", + cause: err, + } + } + } + + if len(errors) > 0 { + return RelationshipIntegrityMultiError(errors) + } + + return nil +} + +// RelationshipIntegrityMultiError is an error wrapping multiple validation +// errors returned by RelationshipIntegrity.ValidateAll() if the designated +// constraints aren't met. +type RelationshipIntegrityMultiError []error + +// Error returns a concatenation of all the error messages it wraps. +func (m RelationshipIntegrityMultiError) Error() string { + var msgs []string + for _, err := range m { + msgs = append(msgs, err.Error()) + } + return strings.Join(msgs, "; ") +} + +// AllErrors returns a list of validation violation errors. +func (m RelationshipIntegrityMultiError) AllErrors() []error { return m } + +// RelationshipIntegrityValidationError is the validation error returned by +// RelationshipIntegrity.Validate if the designated constraints aren't met. +type RelationshipIntegrityValidationError struct { + field string + reason string + cause error + key bool +} + +// Field function returns field value. +func (e RelationshipIntegrityValidationError) Field() string { return e.field } + +// Reason function returns reason value. +func (e RelationshipIntegrityValidationError) Reason() string { return e.reason } + +// Cause function returns cause value. +func (e RelationshipIntegrityValidationError) Cause() error { return e.cause } + +// Key function returns key value. +func (e RelationshipIntegrityValidationError) Key() bool { return e.key } + +// ErrorName returns error name. +func (e RelationshipIntegrityValidationError) ErrorName() string { + return "RelationshipIntegrityValidationError" +} + +// Error satisfies the builtin error interface +func (e RelationshipIntegrityValidationError) Error() string { + cause := "" + if e.cause != nil { + cause = fmt.Sprintf(" | caused by: %v", e.cause) + } + + key := "" + if e.key { + key = "key for " + } + + return fmt.Sprintf( + "invalid %sRelationshipIntegrity.%s: %s%s", + key, + e.field, + e.reason, + cause) +} + +var _ error = RelationshipIntegrityValidationError{} + +var _ interface { + Field() string + Reason() string + Key() bool + Cause() error + ErrorName() string +} = RelationshipIntegrityValidationError{} + // Validate checks the field values on ContextualizedCaveat with the rules // defined in the proto definition for this message. If any rules are // violated, the first error encountered is returned, or nil if there are no violations. diff --git a/pkg/proto/core/v1/core_vtproto.pb.go b/pkg/proto/core/v1/core_vtproto.pb.go index cb4c57d1d6..e0edd59261 100644 --- a/pkg/proto/core/v1/core_vtproto.pb.go +++ b/pkg/proto/core/v1/core_vtproto.pb.go @@ -9,10 +9,12 @@ import ( protohelpers "github.com/planetscale/vtprotobuf/protohelpers" anypb1 "github.com/planetscale/vtprotobuf/types/known/anypb" structpb1 "github.com/planetscale/vtprotobuf/types/known/structpb" + timestamppb1 "github.com/planetscale/vtprotobuf/types/known/timestamppb" proto "google.golang.org/protobuf/proto" protoimpl "google.golang.org/protobuf/runtime/protoimpl" anypb "google.golang.org/protobuf/types/known/anypb" structpb "google.golang.org/protobuf/types/known/structpb" + timestamppb "google.golang.org/protobuf/types/known/timestamppb" io "io" ) @@ -31,6 +33,7 @@ func (m *RelationTuple) CloneVT() *RelationTuple { r.ResourceAndRelation = m.ResourceAndRelation.CloneVT() r.Subject = m.Subject.CloneVT() r.Caveat = m.Caveat.CloneVT() + r.Integrity = m.Integrity.CloneVT() if len(m.unknownFields) > 0 { r.unknownFields = make([]byte, len(m.unknownFields)) copy(r.unknownFields, m.unknownFields) @@ -42,6 +45,29 @@ func (m *RelationTuple) CloneMessageVT() proto.Message { return m.CloneVT() } +func (m *RelationshipIntegrity) CloneVT() *RelationshipIntegrity { + if m == nil { + return (*RelationshipIntegrity)(nil) + } + r := new(RelationshipIntegrity) + r.KeyId = m.KeyId + r.HashedAt = (*timestamppb.Timestamp)((*timestamppb1.Timestamp)(m.HashedAt).CloneVT()) + if rhs := m.Hash; rhs != nil { + tmpBytes := make([]byte, len(rhs)) + copy(tmpBytes, rhs) + r.Hash = tmpBytes + } + if len(m.unknownFields) > 0 { + r.unknownFields = make([]byte, len(m.unknownFields)) + copy(r.unknownFields, m.unknownFields) + } + return r +} + +func (m *RelationshipIntegrity) CloneMessageVT() proto.Message { + return m.CloneVT() +} + func (m *ContextualizedCaveat) CloneVT() *ContextualizedCaveat { if m == nil { return (*ContextualizedCaveat)(nil) @@ -969,6 +995,9 @@ func (this *RelationTuple) EqualVT(that *RelationTuple) bool { if !this.Caveat.EqualVT(that.Caveat) { return false } + if !this.Integrity.EqualVT(that.Integrity) { + return false + } return string(this.unknownFields) == string(that.unknownFields) } @@ -979,6 +1008,31 @@ func (this *RelationTuple) EqualMessageVT(thatMsg proto.Message) bool { } return this.EqualVT(that) } +func (this *RelationshipIntegrity) EqualVT(that *RelationshipIntegrity) bool { + if this == that { + return true + } else if this == nil || that == nil { + return false + } + if this.KeyId != that.KeyId { + return false + } + if string(this.Hash) != string(that.Hash) { + return false + } + if !(*timestamppb1.Timestamp)(this.HashedAt).EqualVT((*timestamppb1.Timestamp)(that.HashedAt)) { + return false + } + return string(this.unknownFields) == string(that.unknownFields) +} + +func (this *RelationshipIntegrity) EqualMessageVT(thatMsg proto.Message) bool { + that, ok := thatMsg.(*RelationshipIntegrity) + if !ok { + return false + } + return this.EqualVT(that) +} func (this *ContextualizedCaveat) EqualVT(that *ContextualizedCaveat) bool { if this == that { return true @@ -2451,6 +2505,16 @@ func (m *RelationTuple) MarshalToSizedBufferVT(dAtA []byte) (int, error) { i -= len(m.unknownFields) copy(dAtA[i:], m.unknownFields) } + if m.Integrity != nil { + size, err := m.Integrity.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x22 + } if m.Caveat != nil { size, err := m.Caveat.MarshalToSizedBufferVT(dAtA[:i]) if err != nil { @@ -2484,6 +2548,63 @@ func (m *RelationTuple) MarshalToSizedBufferVT(dAtA []byte) (int, error) { return len(dAtA) - i, nil } +func (m *RelationshipIntegrity) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *RelationshipIntegrity) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *RelationshipIntegrity) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.HashedAt != nil { + size, err := (*timestamppb1.Timestamp)(m.HashedAt).MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = protohelpers.EncodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1a + } + if len(m.Hash) > 0 { + i -= len(m.Hash) + copy(dAtA[i:], m.Hash) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.Hash))) + i-- + dAtA[i] = 0x12 + } + if len(m.KeyId) > 0 { + i -= len(m.KeyId) + copy(dAtA[i:], m.KeyId) + i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.KeyId))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + func (m *ContextualizedCaveat) MarshalVT() (dAtA []byte, err error) { if m == nil { return nil, nil @@ -4780,6 +4901,32 @@ func (m *RelationTuple) SizeVT() (n int) { l = m.Caveat.SizeVT() n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) } + if m.Integrity != nil { + l = m.Integrity.SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *RelationshipIntegrity) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.KeyId) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + l = len(m.Hash) + if l > 0 { + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } + if m.HashedAt != nil { + l = (*timestamppb1.Timestamp)(m.HashedAt).SizeVT() + n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) + } n += len(m.unknownFields) return n } @@ -5843,6 +5990,195 @@ func (m *RelationTuple) UnmarshalVT(dAtA []byte) error { return err } iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Integrity", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Integrity == nil { + m.Integrity = &RelationshipIntegrity{} + } + if err := m.Integrity.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := protohelpers.Skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return protohelpers.ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *RelationshipIntegrity) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: RelationshipIntegrity: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: RelationshipIntegrity: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field KeyId", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.KeyId = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Hash", wireType) + } + var byteLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + byteLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if byteLen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + byteLen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Hash = append(m.Hash[:0], dAtA[iNdEx:postIndex]...) + if m.Hash == nil { + m.Hash = []byte{} + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field HashedAt", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return protohelpers.ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return protohelpers.ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.HashedAt == nil { + m.HashedAt = ×tamppb.Timestamp{} + } + if err := (*timestamppb1.Timestamp)(m.HashedAt).UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex default: iNdEx = preIndex skippy, err := protohelpers.Skip(dAtA[iNdEx:]) diff --git a/pkg/tuple/tuple.go b/pkg/tuple/tuple.go index 96b94f40f0..ca0a1b1494 100644 --- a/pkg/tuple/tuple.go +++ b/pkg/tuple/tuple.go @@ -1,12 +1,14 @@ package tuple import ( + "bytes" "encoding/json" "fmt" "maps" "reflect" "regexp" "slices" + "sort" v1 "github.com/authzed/authzed-go/proto/authzed/api/v1" "github.com/jzelinskie/stringz" @@ -14,6 +16,7 @@ import ( "google.golang.org/protobuf/types/known/structpb" core "github.com/authzed/spicedb/pkg/proto/core/v1" + "github.com/authzed/spicedb/pkg/spiceerrors" ) const ( @@ -572,3 +575,95 @@ func WithCaveat(tpl *core.RelationTuple, caveatName string, contexts ...map[stri } return tpl, nil } + +// CanonicalBytes converts a tuple to a canonical set of bytes. If the tuple is nil or empty, returns nil. +// Can be used for hashing purposes. +func CanonicalBytes(tpl *core.RelationTuple) ([]byte, error) { + if tpl == nil { + return nil, nil + } + + var sb bytes.Buffer + sb.WriteString(tpl.ResourceAndRelation.Namespace) + sb.WriteString(":") + sb.WriteString(tpl.ResourceAndRelation.ObjectId) + sb.WriteString("#") + sb.WriteString(tpl.ResourceAndRelation.Relation) + sb.WriteString("@") + sb.WriteString(tpl.Subject.Namespace) + sb.WriteString(":") + sb.WriteString(tpl.Subject.ObjectId) + sb.WriteString("#") + sb.WriteString(tpl.Subject.Relation) + + if tpl.Caveat != nil && tpl.Caveat.CaveatName != "" { + sb.WriteString(" with ") + sb.WriteString(tpl.Caveat.CaveatName) + + if tpl.Caveat.Context != nil && len(tpl.Caveat.Context.Fields) > 0 { + sb.WriteString(":") + if err := writeCanonicalContext(&sb, tpl.Caveat.Context); err != nil { + return nil, err + } + } + } + + return sb.Bytes(), nil +} + +func writeCanonicalContext(sb *bytes.Buffer, context *structpb.Struct) error { + sb.WriteString("{") + for i, key := range sortedContextKeys(context.Fields) { + if i > 0 { + sb.WriteString(",") + } + sb.WriteString(key) + sb.WriteString(":") + if err := writeCanonicalContextValue(sb, context.Fields[key]); err != nil { + return err + } + } + sb.WriteString("}") + return nil +} + +func writeCanonicalContextValue(sb *bytes.Buffer, value *structpb.Value) error { + switch value.Kind.(type) { + case *structpb.Value_NullValue: + sb.WriteString("null") + case *structpb.Value_NumberValue: + sb.WriteString(fmt.Sprintf("%f", value.GetNumberValue())) + case *structpb.Value_StringValue: + sb.WriteString(value.GetStringValue()) + case *structpb.Value_BoolValue: + sb.WriteString(fmt.Sprintf("%t", value.GetBoolValue())) + case *structpb.Value_StructValue: + if err := writeCanonicalContext(sb, value.GetStructValue()); err != nil { + return err + } + case *structpb.Value_ListValue: + sb.WriteString("[") + for i, elem := range value.GetListValue().Values { + if i > 0 { + sb.WriteString(",") + } + if err := writeCanonicalContextValue(sb, elem); err != nil { + return err + } + } + sb.WriteString("]") + default: + return spiceerrors.MustBugf("unknown structpb.Value type: %T", value.Kind) + } + + return nil +} + +func sortedContextKeys(fields map[string]*structpb.Value) []string { + keys := make([]string, 0, len(fields)) + for key := range fields { + keys = append(keys, key) + } + sort.Strings(keys) + return keys +} diff --git a/pkg/tuple/tuple_test.go b/pkg/tuple/tuple_test.go index 26a81592e4..5e7ea7695b 100644 --- a/pkg/tuple/tuple_test.go +++ b/pkg/tuple/tuple_test.go @@ -4,6 +4,8 @@ import ( "strings" "testing" + b64 "encoding/base64" + v1 "github.com/authzed/authzed-go/proto/authzed/api/v1" "github.com/stretchr/testify/require" "google.golang.org/protobuf/types/known/structpb" @@ -69,10 +71,11 @@ func crel(resType, resID, relation, subType, subID, subRel, caveatName string, c var superLongID = strings.Repeat("f", 1024) var testCases = []struct { - input string - expectedOutput string - tupleFormat *core.RelationTuple - relFormat *v1.Relationship + input string + expectedOutput string + tupleFormat *core.RelationTuple + relFormat *v1.Relationship + stableCanonicalization string }{ { input: "testns:testobj#testrel@user:testusr", @@ -81,7 +84,8 @@ var testCases = []struct { ObjectAndRelation("testns", "testobj", "testrel"), ObjectAndRelation("user", "testusr", "..."), ), - relFormat: rel("testns", "testobj", "testrel", "user", "testusr", ""), + relFormat: rel("testns", "testobj", "testrel", "user", "testusr", ""), + stableCanonicalization: "dGVzdG5zOnRlc3RvYmojdGVzdHJlbEB1c2VyOnRlc3R1c3IjLi4u", }, { input: "testns:testobj#testrel@user:testusr#...", @@ -90,7 +94,8 @@ var testCases = []struct { ObjectAndRelation("testns", "testobj", "testrel"), ObjectAndRelation("user", "testusr", "..."), ), - relFormat: rel("testns", "testobj", "testrel", "user", "testusr", ""), + relFormat: rel("testns", "testobj", "testrel", "user", "testusr", ""), + stableCanonicalization: "dGVzdG5zOnRlc3RvYmojdGVzdHJlbEB1c2VyOnRlc3R1c3IjLi4u", }, { input: "tenant/testns:testobj#testrel@tenant/user:testusr", @@ -99,7 +104,8 @@ var testCases = []struct { ObjectAndRelation("tenant/testns", "testobj", "testrel"), ObjectAndRelation("tenant/user", "testusr", "..."), ), - relFormat: rel("tenant/testns", "testobj", "testrel", "tenant/user", "testusr", ""), + relFormat: rel("tenant/testns", "testobj", "testrel", "tenant/user", "testusr", ""), + stableCanonicalization: "dGVuYW50L3Rlc3Ruczp0ZXN0b2JqI3Rlc3RyZWxAdGVuYW50L3VzZXI6dGVzdHVzciMuLi4=", }, { input: "tenant/testns:testobj#testrel@tenant/user:testusr#...", @@ -108,7 +114,8 @@ var testCases = []struct { ObjectAndRelation("tenant/testns", "testobj", "testrel"), ObjectAndRelation("tenant/user", "testusr", "..."), ), - relFormat: rel("tenant/testns", "testobj", "testrel", "tenant/user", "testusr", ""), + relFormat: rel("tenant/testns", "testobj", "testrel", "tenant/user", "testusr", ""), + stableCanonicalization: "dGVuYW50L3Rlc3Ruczp0ZXN0b2JqI3Rlc3RyZWxAdGVuYW50L3VzZXI6dGVzdHVzciMuLi4=", }, { input: "tenant/testns:testobj#testrel@tenant/user:testusr#somerel", @@ -117,7 +124,8 @@ var testCases = []struct { ObjectAndRelation("tenant/testns", "testobj", "testrel"), ObjectAndRelation("tenant/user", "testusr", "somerel"), ), - relFormat: rel("tenant/testns", "testobj", "testrel", "tenant/user", "testusr", "somerel"), + relFormat: rel("tenant/testns", "testobj", "testrel", "tenant/user", "testusr", "somerel"), + stableCanonicalization: "dGVuYW50L3Rlc3Ruczp0ZXN0b2JqI3Rlc3RyZWxAdGVuYW50L3VzZXI6dGVzdHVzciNzb21lcmVs", }, { input: "org/division/team/testns:testobj#testrel@org/division/identity_team/user:testusr#somerel", @@ -126,7 +134,8 @@ var testCases = []struct { ObjectAndRelation("org/division/team/testns", "testobj", "testrel"), ObjectAndRelation("org/division/identity_team/user", "testusr", "somerel"), ), - relFormat: rel("org/division/team/testns", "testobj", "testrel", "org/division/identity_team/user", "testusr", "somerel"), + relFormat: rel("org/division/team/testns", "testobj", "testrel", "org/division/identity_team/user", "testusr", "somerel"), + stableCanonicalization: "b3JnL2RpdmlzaW9uL3RlYW0vdGVzdG5zOnRlc3RvYmojdGVzdHJlbEBvcmcvZGl2aXNpb24vaWRlbnRpdHlfdGVhbS91c2VyOnRlc3R1c3Ijc29tZXJlbA==", }, { input: "tenant/testns:testobj#testrel@tenant/user:testusr something", @@ -159,7 +168,8 @@ var testCases = []struct { ObjectAndRelation("foos", "bar", "bazzy"), ObjectAndRelation("groo", "grar", "..."), ), - relFormat: rel("foos", "bar", "bazzy", "groo", "grar", ""), + relFormat: rel("foos", "bar", "bazzy", "groo", "grar", ""), + stableCanonicalization: "Zm9vczpiYXIjYmF6enlAZ3JvbzpncmFyIy4uLg==", }, { input: "tenant/testns:testobj#testrel@tenant/user:*#...", @@ -168,7 +178,8 @@ var testCases = []struct { ObjectAndRelation("tenant/testns", "testobj", "testrel"), ObjectAndRelation("tenant/user", "*", "..."), ), - relFormat: rel("tenant/testns", "testobj", "testrel", "tenant/user", "*", ""), + relFormat: rel("tenant/testns", "testobj", "testrel", "tenant/user", "*", ""), + stableCanonicalization: "dGVuYW50L3Rlc3Ruczp0ZXN0b2JqI3Rlc3RyZWxAdGVuYW50L3VzZXI6KiMuLi4=", }, { input: "tenant/testns:testobj#testrel@tenant/user:authn|foo", @@ -177,7 +188,8 @@ var testCases = []struct { ObjectAndRelation("tenant/testns", "testobj", "testrel"), ObjectAndRelation("tenant/user", "authn|foo", "..."), ), - relFormat: rel("tenant/testns", "testobj", "testrel", "tenant/user", "authn|foo", ""), + relFormat: rel("tenant/testns", "testobj", "testrel", "tenant/user", "authn|foo", ""), + stableCanonicalization: "dGVuYW50L3Rlc3Ruczp0ZXN0b2JqI3Rlc3RyZWxAdGVuYW50L3VzZXI6YXV0aG58Zm9vIy4uLg==", }, { input: "document:foo#viewer@user:tom[somecaveat]", @@ -189,7 +201,8 @@ var testCases = []struct { ), "somecaveat", ), - relFormat: crel("document", "foo", "viewer", "user", "tom", "", "somecaveat", nil), + relFormat: crel("document", "foo", "viewer", "user", "tom", "", "somecaveat", nil), + stableCanonicalization: "ZG9jdW1lbnQ6Zm9vI3ZpZXdlckB1c2VyOnRvbSMuLi4gd2l0aCBzb21lY2F2ZWF0", }, { input: "document:foo#viewer@user:tom[tenant/somecaveat]", @@ -201,7 +214,8 @@ var testCases = []struct { ), "tenant/somecaveat", ), - relFormat: crel("document", "foo", "viewer", "user", "tom", "", "tenant/somecaveat", nil), + relFormat: crel("document", "foo", "viewer", "user", "tom", "", "tenant/somecaveat", nil), + stableCanonicalization: "ZG9jdW1lbnQ6Zm9vI3ZpZXdlckB1c2VyOnRvbSMuLi4gd2l0aCB0ZW5hbnQvc29tZWNhdmVhdA==", }, { input: "document:foo#viewer@user:tom[tenant/division/somecaveat]", @@ -213,7 +227,8 @@ var testCases = []struct { ), "tenant/division/somecaveat", ), - relFormat: crel("document", "foo", "viewer", "user", "tom", "", "tenant/division/somecaveat", nil), + relFormat: crel("document", "foo", "viewer", "user", "tom", "", "tenant/division/somecaveat", nil), + stableCanonicalization: "ZG9jdW1lbnQ6Zm9vI3ZpZXdlckB1c2VyOnRvbSMuLi4gd2l0aCB0ZW5hbnQvZGl2aXNpb24vc29tZWNhdmVhdA==", }, { input: "document:foo#viewer@user:tom[somecaveat", @@ -240,7 +255,8 @@ var testCases = []struct { "hi": "there", }, ), - relFormat: crel("document", "foo", "viewer", "user", "tom", "", "somecaveat", map[string]any{"hi": "there"}), + relFormat: crel("document", "foo", "viewer", "user", "tom", "", "somecaveat", map[string]any{"hi": "there"}), + stableCanonicalization: "ZG9jdW1lbnQ6Zm9vI3ZpZXdlckB1c2VyOnRvbSMuLi4gd2l0aCBzb21lY2F2ZWF0OntoaTp0aGVyZX0=", }, { input: `document:foo#viewer@user:tom[somecaveat:{"hi":{"yo": 123}}]`, @@ -262,6 +278,7 @@ var testCases = []struct { "yo": 123, }, }), + stableCanonicalization: "ZG9jdW1lbnQ6Zm9vI3ZpZXdlckB1c2VyOnRvbSMuLi4gd2l0aCBzb21lY2F2ZWF0OntoaTp7eW86MTIzLjAwMDAwMH19", }, { input: `document:foo#viewer@user:tom[somecaveat:{"hi":{"yo":{"hey":true}}}]`, @@ -287,6 +304,7 @@ var testCases = []struct { }, }, }), + stableCanonicalization: "ZG9jdW1lbnQ6Zm9vI3ZpZXdlckB1c2VyOnRvbSMuLi4gd2l0aCBzb21lY2F2ZWF0OntoaTp7eW86e2hleTp0cnVlfX19", }, { input: `document:foo#viewer@user:tom[somecaveat:{"hi":{"yo":{"hey":[1,2,3]}}}]`, @@ -312,6 +330,7 @@ var testCases = []struct { }, }, }), + stableCanonicalization: "ZG9jdW1lbnQ6Zm9vI3ZpZXdlckB1c2VyOnRvbSMuLi4gd2l0aCBzb21lY2F2ZWF0OntoaTp7eW86e2hleTpbMS4wMDAwMDAsMi4wMDAwMDAsMy4wMDAwMDBdfX19", }, { input: `document:foo#viewer@user:tom[somecaveat:{"hi":{"yo":"hey":true}}}]`, @@ -326,7 +345,8 @@ var testCases = []struct { ObjectAndRelation("testns", superLongID, "testrel"), ObjectAndRelation("user", "testusr", "..."), ), - relFormat: rel("testns", superLongID, "testrel", "user", "testusr", ""), + relFormat: rel("testns", superLongID, "testrel", "user", "testusr", ""), + stableCanonicalization: "dGVzdG5zOmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmYjdGVzdHJlbEB1c2VyOnRlc3R1c3IjLi4u", }, { input: "testns:foo#testrel@user:" + superLongID, @@ -335,7 +355,8 @@ var testCases = []struct { ObjectAndRelation("testns", "foo", "testrel"), ObjectAndRelation("user", superLongID, "..."), ), - relFormat: rel("testns", "foo", "testrel", "user", superLongID, ""), + relFormat: rel("testns", "foo", "testrel", "user", superLongID, ""), + stableCanonicalization: "dGVzdG5zOmZvbyN0ZXN0cmVsQHVzZXI6ZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZiMuLi4=", }, { input: "testns:foo#testrel@user:" + superLongID + "more", @@ -350,7 +371,8 @@ var testCases = []struct { ObjectAndRelation("testns", "-base64YWZzZGZh-ZHNmZHPwn5iK8J+YivC/fmIrwn5iK==", "testrel"), ObjectAndRelation("user", "-base65YWZzZGZh-ZHNmZHPwn5iK8J+YivC/fmIrwn5iK==", "..."), ), - relFormat: rel("testns", "-base64YWZzZGZh-ZHNmZHPwn5iK8J+YivC/fmIrwn5iK==", "testrel", "user", "-base65YWZzZGZh-ZHNmZHPwn5iK8J+YivC/fmIrwn5iK==", ""), + relFormat: rel("testns", "-base64YWZzZGZh-ZHNmZHPwn5iK8J+YivC/fmIrwn5iK==", "testrel", "user", "-base65YWZzZGZh-ZHNmZHPwn5iK8J+YivC/fmIrwn5iK==", ""), + stableCanonicalization: "dGVzdG5zOi1iYXNlNjRZV1p6WkdaaC1aSE5tWkhQd241aUs4SitZaXZDL2ZtSXJ3bjVpSz09I3Rlc3RyZWxAdXNlcjotYmFzZTY1WVdaelpHWmgtWkhObVpIUHduNWlLOEorWWl2Qy9mbUlyd241aUs9PSMuLi4=", }, { input: `document:foo#viewer@user:tom[somecaveat:{"hi":"a@example.com"}]`, @@ -368,9 +390,97 @@ var testCases = []struct { relFormat: crel("document", "foo", "viewer", "user", "tom", "", "somecaveat", map[string]any{ "hi": "a@example.com", }), + stableCanonicalization: "ZG9jdW1lbnQ6Zm9vI3ZpZXdlckB1c2VyOnRvbSMuLi4gd2l0aCBzb21lY2F2ZWF0OntoaTphQGV4YW1wbGUuY29tfQ==", + }, + { + input: `document:foo#viewer@user:tom[somecaveat:{"first":"a@example.com", "second": "b@example.com"}]`, + expectedOutput: `document:foo#viewer@user:tom[somecaveat:{"first":"a@example.com","second":"b@example.com"}]`, + tupleFormat: MustWithCaveat( + makeTuple( + ObjectAndRelation("document", "foo", "viewer"), + ObjectAndRelation("user", "tom", "..."), + ), + "somecaveat", + map[string]any{ + "first": "a@example.com", + "second": "b@example.com", + }, + ), + relFormat: crel("document", "foo", "viewer", "user", "tom", "", "somecaveat", map[string]any{ + "first": "a@example.com", + "second": "b@example.com", + }), + stableCanonicalization: "ZG9jdW1lbnQ6Zm9vI3ZpZXdlckB1c2VyOnRvbSMuLi4gd2l0aCBzb21lY2F2ZWF0OntmaXJzdDphQGV4YW1wbGUuY29tLHNlY29uZDpiQGV4YW1wbGUuY29tfQ==", + }, + { + input: `document:foo#viewer@user:tom[somecaveat:{"second": "b@example.com", "first":"a@example.com"}]`, + expectedOutput: `document:foo#viewer@user:tom[somecaveat:{"first":"a@example.com","second":"b@example.com"}]`, + tupleFormat: MustWithCaveat( + makeTuple( + ObjectAndRelation("document", "foo", "viewer"), + ObjectAndRelation("user", "tom", "..."), + ), + "somecaveat", + map[string]any{ + "first": "a@example.com", + "second": "b@example.com", + }, + ), + relFormat: crel("document", "foo", "viewer", "user", "tom", "", "somecaveat", map[string]any{ + "first": "a@example.com", + "second": "b@example.com", + }), + stableCanonicalization: "ZG9jdW1lbnQ6Zm9vI3ZpZXdlckB1c2VyOnRvbSMuLi4gd2l0aCBzb21lY2F2ZWF0OntmaXJzdDphQGV4YW1wbGUuY29tLHNlY29uZDpiQGV4YW1wbGUuY29tfQ==", }, } +func TestCanonicalBytes(t *testing.T) { + foundBytes := make(map[string]string) + + for _, tc := range testCases { + if tc.tupleFormat == nil { + continue + } + + tc := tc + + t.Run(tc.input, func(t *testing.T) { + // Ensure the serialization is stable. + serialized, err := CanonicalBytes(tc.tupleFormat) + require.NoError(t, err) + + encoded := b64.StdEncoding.EncodeToString(serialized) + require.Equal(t, tc.stableCanonicalization, encoded) + + // Ensure the serialization is unique. + existing, ok := foundBytes[string(serialized)] + if ok { + parsedInput := MustParse(tc.input) + parsedExisting := MustParse(existing) + + require.True(t, parsedInput.EqualVT(parsedExisting), "duplicate canonical bytes found. input: %s; found for input: %s", tc.input, existing) + } + foundBytes[string(serialized)] = tc.input + }) + } +} + +func BenchmarkMustCanonicalBytes(b *testing.B) { + for _, tc := range testCases { + tc := tc + if tc.tupleFormat == nil { + continue + } + + b.Run(tc.input, func(b *testing.B) { + for i := 0; i < b.N; i++ { + _, err := CanonicalBytes(tc.tupleFormat) + require.NoError(b, err) + } + }) + } +} + func TestSerialize(t *testing.T) { for _, tc := range testCases { tc := tc diff --git a/proto/internal/core/v1/core.proto b/proto/internal/core/v1/core.proto index 1b8a57ab59..40b40b642c 100644 --- a/proto/internal/core/v1/core.proto +++ b/proto/internal/core/v1/core.proto @@ -3,6 +3,7 @@ package core.v1; import "google/protobuf/any.proto"; import "google/protobuf/struct.proto"; +import "google/protobuf/timestamp.proto"; import "validate/validate.proto"; option go_package = "github.com/authzed/spicedb/pkg/proto/core/v1"; @@ -16,6 +17,20 @@ message RelationTuple { /** caveat is a reference to a the caveat that must be enforced over the tuple **/ ContextualizedCaveat caveat = 3 [(validate.rules).message.required = false]; + + /** integrity holds (optional) information about the integrity of the tuple */ + RelationshipIntegrity integrity = 4 [(validate.rules).message.required = false]; +} + +message RelationshipIntegrity { + /** key_id is the key ID used to hash the tuple */ + string key_id = 1; + + /** hash is the hash of the tuple */ + bytes hash = 2; + + /** hashed_at is the timestamp when the tuple was hashed */ + google.protobuf.Timestamp hashed_at = 3; } /** diff --git a/tools/analyzers/go.work.sum b/tools/analyzers/go.work.sum index 32ffeee248..edc76a819a 100644 --- a/tools/analyzers/go.work.sum +++ b/tools/analyzers/go.work.sum @@ -2142,6 +2142,7 @@ golang.org/x/sync v0.6.0/go.mod h1:Czt+wKu1gCyEFDUtn0jG5QVvpJ6rzVqr5aXyt9drQfk= golang.org/x/sys v0.13.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.18.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/sys v0.20.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= +golang.org/x/sys v0.21.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/telemetry v0.0.0-20240208230135-b75ee8823808 h1:+Kc94D8UVEVxJnLXp/+FMfqQARZtWHfVrcRtcG8aT3g= golang.org/x/telemetry v0.0.0-20240208230135-b75ee8823808/go.mod h1:KG1lNk5ZFNssSZLrpVb4sMXKMpGwGXOxSG3rnu2gZQQ= golang.org/x/telemetry v0.0.0-20240228155512-f48c80bd79b2 h1:IRJeR9r1pYWsHKTRe/IInb7lYvbBVIqOgsX/u0mbOWY=