From d8c6d8a6a21cb8124e7666d7fd0dbd2ad6ab69a6 Mon Sep 17 00:00:00 2001 From: Lucy Zhang Date: Tue, 8 Sep 2020 22:11:20 -0400 Subject: [PATCH] backupccl: add all new restored descriptors in offline state This change updates the RESTORE job to add all new descriptors prior to restoring data in the OFFLINE state, and update their state to PUBLIC after data has been restored. This makes all descriptors behave like tables. Release justification: Low risk, high benefit changes to existing functionality (for database descriptors; other changes apply only to new functionality) Release note (sql change): Databases being restored will now be in the offline state, invisible to users, until the data has been restored. This is the same as the existing behavior for restored tables. (This change is also applied to enums and user-defined schemas being restored, which is a change relative to only the 20.2 alpha releases.) --- pkg/ccl/backupccl/backup_test.go | 271 +++++++++++++ pkg/ccl/backupccl/restore_job.go | 154 ++++++-- pkg/jobs/jobspb/jobs.pb.go | 630 ++++++++++++++++--------------- pkg/jobs/jobspb/jobs.proto | 5 +- 4 files changed, 707 insertions(+), 353 deletions(-) diff --git a/pkg/ccl/backupccl/backup_test.go b/pkg/ccl/backupccl/backup_test.go index 1ba7a5b94ec0..8ed88e0b6200 100644 --- a/pkg/ccl/backupccl/backup_test.go +++ b/pkg/ccl/backupccl/backup_test.go @@ -61,6 +61,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/testutils/skip" "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" "github.com/cockroachdb/cockroach/pkg/testutils/testcluster" + "github.com/cockroachdb/cockroach/pkg/util/ctxgroup" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/protoutil" @@ -6037,6 +6038,276 @@ func TestBackupDoesNotHangOnIntent(t *testing.T) { require.Error(t, tx.Commit()) } +// TestRestoreResetsDescriptorVersions tests that new descriptors created while +// restoring have their versions reset. Descriptors end up at version 2 after +// the job is finished, since they are updated once at the end of the job to +// make them public. +func TestRestoreResetsDescriptorVersions(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + _, tc, sqlDB, _, cleanupFn := BackupRestoreTestSetup(t, singleNode, 0, InitNone) + defer cleanupFn() + kvDB := tc.Server(0).DB() + + // Create some descriptors and do some schema changes to bump their versions. + sqlDB.Exec(t, ` +CREATE DATABASE d_old; +ALTER DATABASE d_old RENAME TO d; + +USE d; + +CREATE SCHEMA sc_old; +ALTER SCHEMA sc_old RENAME TO sc; + +CREATE TABLE sc.tb (x INT); +ALTER TABLE sc.tb ADD COLUMN a INT; + +CREATE TYPE sc.typ AS ENUM ('hello'); +ALTER TYPE sc.typ ADD VALUE 'hi'; +`) + // Back up the database. + sqlDB.Exec(t, `BACKUP DATABASE d TO 'nodelocal://0/test/'`) + + // Drop the database and restore into it. + sqlDB.Exec(t, `DROP DATABASE d`) + sqlDB.Exec(t, `RESTORE DATABASE d FROM 'nodelocal://0/test/'`) + + dbDesc := catalogkv.TestingGetDatabaseDescriptor(kvDB, keys.SystemSQLCodec, "d") + require.EqualValues(t, 2, dbDesc.Version) + + schemaDesc := catalogkv.TestingGetSchemaDescriptor(kvDB, keys.SystemSQLCodec, dbDesc.GetID(), "sc") + require.EqualValues(t, 2, schemaDesc.Version) + + tableDesc := catalogkv.TestingGetTableDescriptorFromSchema(kvDB, keys.SystemSQLCodec, "d", "sc", "tb") + require.EqualValues(t, 2, tableDesc.Version) + + typeDesc := catalogkv.TestingGetTypeDescriptorFromSchema(kvDB, keys.SystemSQLCodec, "d", "sc", "typ") + require.EqualValues(t, 2, typeDesc.Version) +} + +func TestOfflineDescriptorsDuringRestore(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + // Protects beforePublishingNotification, continueNotification. + var mu syncutil.Mutex + // Closed when the restore job has reached the point right before publishing + // descriptors. + beforePublishingNotification := make(chan struct{}) + // Closed when we're ready to resume with the restore. + continueNotification := make(chan struct{}) + initBackfillNotification := func() (chan struct{}, chan struct{}) { + mu.Lock() + defer mu.Unlock() + beforePublishingNotification = make(chan struct{}) + continueNotification = make(chan struct{}) + return beforePublishingNotification, continueNotification + } + notifyBackfill := func(ctx context.Context) { + mu.Lock() + defer mu.Unlock() + if beforePublishingNotification != nil { + close(beforePublishingNotification) + beforePublishingNotification = nil + } + select { + case <-continueNotification: + return + case <-ctx.Done(): + return + } + } + + t.Run("restore-database", func(t *testing.T) { + ctx, tc, sqlDB, _, cleanupFn := BackupRestoreTestSetup(t, singleNode, 0, InitNone) + defer cleanupFn() + ctx, cancel := context.WithCancel(ctx) + defer cancel() + kvDB := tc.Server(0).DB() + + for _, server := range tc.Servers { + registry := server.JobRegistry().(*jobs.Registry) + registry.TestingResumerCreationKnobs = map[jobspb.Type]func(raw jobs.Resumer) jobs.Resumer{ + jobspb.TypeRestore: func(raw jobs.Resumer) jobs.Resumer { + r := raw.(*restoreResumer) + r.testingKnobs.beforePublishingDescriptors = func() error { + notifyBackfill(ctx) + return nil + } + return r + }, + } + } + + sqlDB.Exec(t, ` +CREATE DATABASE d; +USE d; +CREATE SCHEMA sc; +CREATE TABLE sc.tb (x INT); +CREATE TYPE sc.typ AS ENUM ('hello'); +`) + + // Back up the database. + sqlDB.Exec(t, `BACKUP DATABASE d TO 'nodelocal://0/test/'`) + + // Drop the database and restore into it. + sqlDB.Exec(t, `DROP DATABASE d`) + + beforePublishingNotif, continueNotif := initBackfillNotification() + g := ctxgroup.WithContext(ctx) + g.GoCtx(func(ctx context.Context) error { + if _, err := sqlDB.DB.ExecContext(ctx, `RESTORE DATABASE d FROM 'nodelocal://0/test/'`); err != nil { + t.Fatal(err) + } + return nil + }) + + <-beforePublishingNotif + + // Verify that the descriptors are offline. + + dbDesc := catalogkv.TestingGetDatabaseDescriptor(kvDB, keys.SystemSQLCodec, "d") + require.Equal(t, descpb.DescriptorState_OFFLINE, dbDesc.State) + + schemaDesc := catalogkv.TestingGetSchemaDescriptor(kvDB, keys.SystemSQLCodec, dbDesc.GetID(), "sc") + require.Equal(t, descpb.DescriptorState_OFFLINE, schemaDesc.State) + + tableDesc := catalogkv.TestingGetTableDescriptorFromSchema(kvDB, keys.SystemSQLCodec, "d", "sc", "tb") + require.Equal(t, descpb.DescriptorState_OFFLINE, tableDesc.State) + + typeDesc := catalogkv.TestingGetTypeDescriptorFromSchema(kvDB, keys.SystemSQLCodec, "d", "sc", "typ") + require.Equal(t, descpb.DescriptorState_OFFLINE, typeDesc.State) + + // Verify that the descriptors are not visible. + // TODO (lucy): Arguably there should be a SQL test where we manually create + // the offline descriptors. This part doesn't have much to do with RESTORE + // per se. + + // Sometimes name resolution doesn't result in an "offline" error because + // the lookups are performed in planner.LookupObject(), which sets the + // Required flag to false so that callers can decide what to do with a + // negative result, but also means that we never generate the error in the + // first place. Right now we just settle for having some error reported, even + // if it's not the ideal error. + + sqlDB.CheckQueryResults(t, `SHOW DATABASES`, [][]string{{"data"}, {"defaultdb"}, {"postgres"}, {"system"}}) + + sqlDB.ExpectErr(t, `database "d" is offline: restoring`, `USE d`) + + sqlDB.ExpectErr(t, `target database or schema does not exist`, `SHOW TABLES FROM d`) + sqlDB.ExpectErr(t, `target database or schema does not exist`, `SHOW TABLES FROM d.sc`) + + sqlDB.ExpectErr(t, `relation "d.sc.tb" does not exist`, `SELECT * FROM d.sc.tb`) + sqlDB.ExpectErr(t, `relation "d.sc.tb" does not exist`, `ALTER TABLE d.sc.tb ADD COLUMN b INT`) + + sqlDB.ExpectErr(t, `type "d.sc.typ" does not exist`, `ALTER TYPE d.sc.typ RENAME TO typ2`) + + sqlDB.ExpectErr(t, `cannot create "d.sc.other" because the target database or schema does not exist`, `CREATE TABLE d.sc.other()`) + + close(continueNotif) + require.NoError(t, g.Wait()) + }) + + t.Run("restore-into-existing-database", func(t *testing.T) { + ctx, tc, sqlDB, _, cleanupFn := BackupRestoreTestSetup(t, singleNode, 0, InitNone) + defer cleanupFn() + ctx, cancel := context.WithCancel(ctx) + defer cancel() + kvDB := tc.Server(0).DB() + + for _, server := range tc.Servers { + registry := server.JobRegistry().(*jobs.Registry) + registry.TestingResumerCreationKnobs = map[jobspb.Type]func(raw jobs.Resumer) jobs.Resumer{ + jobspb.TypeRestore: func(raw jobs.Resumer) jobs.Resumer { + r := raw.(*restoreResumer) + r.testingKnobs.beforePublishingDescriptors = func() error { + notifyBackfill(ctx) + return nil + } + return r + }, + } + } + + sqlDB.Exec(t, ` +CREATE DATABASE d; +USE d; +CREATE SCHEMA sc; +CREATE TABLE sc.tb (x INT); +CREATE TYPE sc.typ AS ENUM ('hello'); +`) + + // Back up the database. + sqlDB.Exec(t, `BACKUP DATABASE d TO 'nodelocal://0/test/'`) + + // Drop the database. + sqlDB.Exec(t, `DROP DATABASE d`) + + // Create a new database and restore into it. + sqlDB.Exec(t, `CREATE DATABASE newdb`) + + beforePublishingNotif, continueNotif := initBackfillNotification() + g := ctxgroup.WithContext(ctx) + g.GoCtx(func(ctx context.Context) error { + if _, err := sqlDB.DB.ExecContext(ctx, `RESTORE d.* FROM 'nodelocal://0/test/' WITH into_db='newdb'`); err != nil { + t.Fatal(err) + } + return nil + }) + + <-beforePublishingNotif + + // Verify that the descriptors are offline. + + dbDesc := catalogkv.TestingGetDatabaseDescriptor(kvDB, keys.SystemSQLCodec, "newdb") + schemaDesc := catalogkv.TestingGetSchemaDescriptor(kvDB, keys.SystemSQLCodec, dbDesc.GetID(), "sc") + require.Equal(t, descpb.DescriptorState_OFFLINE, schemaDesc.State) + + tableDesc := catalogkv.TestingGetTableDescriptorFromSchema(kvDB, keys.SystemSQLCodec, "newdb", "sc", "tb") + require.Equal(t, descpb.DescriptorState_OFFLINE, tableDesc.State) + + typeDesc := catalogkv.TestingGetTypeDescriptorFromSchema(kvDB, keys.SystemSQLCodec, "newdb", "sc", "typ") + require.Equal(t, descpb.DescriptorState_OFFLINE, typeDesc.State) + + // Verify that the descriptors are not visible. + // TODO (lucy): Arguably there should be a SQL test where we manually create + // the offline descriptors. This part doesn't have much to do with RESTORE + // per se. + + // Sometimes name resolution doesn't result in an "offline" error because + // the lookups are performed in planner.LookupObject(), which sets the + // Required flag to false so that callers can decide what to do with a + // negative result, but also means that we never generate the error in the + // first place. Right now we just settle for having some error reported, even + // if it's not the ideal error. + + sqlDB.Exec(t, `USE newdb`) + + sqlDB.CheckQueryResults(t, `SHOW TABLES`, [][]string{}) + sqlDB.CheckQueryResults(t, `SHOW TYPES`, [][]string{}) + sqlDB.CheckQueryResults(t, `SHOW SCHEMAS`, [][]string{ + {"crdb_internal"}, {"information_schema"}, {"pg_catalog"}, {"pg_extension"}, {"public"}, + }) + + sqlDB.ExpectErr(t, `target database or schema does not exist`, `SHOW TABLES FROM newdb.sc`) + + sqlDB.ExpectErr(t, `schema "sc" is offline: restoring`, `SELECT * FROM newdb.sc.tb`) + sqlDB.ExpectErr(t, `schema "sc" is offline: restoring`, `SELECT * FROM sc.tb`) + sqlDB.ExpectErr(t, `schema "sc" is offline: restoring`, `ALTER TABLE newdb.sc.tb ADD COLUMN b INT`) + sqlDB.ExpectErr(t, `schema "sc" is offline: restoring`, `ALTER TABLE sc.tb ADD COLUMN b INT`) + + sqlDB.ExpectErr(t, `schema "sc" is offline: restoring`, `ALTER TYPE newdb.sc.typ RENAME TO typ2`) + sqlDB.ExpectErr(t, `schema "sc" is offline: restoring`, `ALTER TYPE sc.typ RENAME TO typ2`) + + sqlDB.ExpectErr(t, `schema "sc" is offline: restoring`, `ALTER SCHEMA sc RENAME TO sc2`) + + close(continueNotif) + require.NoError(t, g.Wait()) + + }) +} + // TestManifestBitFlip tests that we can detect a corrupt manifest when a bit // was flipped on disk for both an unencrypted and an encrypted manifest. func TestManifestBitFlip(t *testing.T) { diff --git a/pkg/ccl/backupccl/restore_job.go b/pkg/ccl/backupccl/restore_job.go index 14c000407fb6..f36883ef3648 100644 --- a/pkg/ccl/backupccl/restore_job.go +++ b/pkg/ccl/backupccl/restore_job.go @@ -690,6 +690,9 @@ type restoreResumer struct { execCfg *sql.ExecutorConfig testingKnobs struct { + // beforePublishingDescriptors is called right before publishing + // descriptors, after any data has been restored. + beforePublishingDescriptors func() error // duringSystemTableRestoration is called once for every system table we // restore. It is used to simulate any errors that we may face at this point // of the restore. @@ -898,10 +901,20 @@ func createImportingDescriptors( return nil, nil, nil, err } - for _, desc := range tableDescs { - desc.Version++ - desc.State = descpb.DescriptorState_OFFLINE - desc.OfflineReason = "restoring" + // Set the new descriptors' states to offline. + for _, desc := range mutableTables { + desc.SetOffline("restoring") + } + for _, desc := range typesToWrite { + desc.SetOffline("restoring") + } + for _, desc := range schemasToWrite { + desc.SetOffline("restoring") + } + if r.settings.Version.IsActive(ctx, clusterversion.VersionLeasedDatabaseDescriptors) { + for _, desc := range mutableDatabases { + desc.SetOffline("restoring") + } } // Collect all types after they have had their ID's rewritten. @@ -927,7 +940,7 @@ func createImportingDescriptors( return err } } - // Write the new TableDescriptors which are set in the OFFLINE state. + // Write the new descriptors which are set in the OFFLINE state. if err := WriteDescriptors(ctx, txn, databases, writtenSchemas, tables, writtenTypes, details.DescriptorCoverage, r.settings, nil /* extra */); err != nil { return errors.Wrapf(err, "restoring %d TableDescriptors from %d databases", len(tables), len(databases)) } @@ -1110,6 +1123,13 @@ func (r *restoreResumer) Resume( // Since we have already created any new databases that we needed, // we can return without importing any data. log.Warning(ctx, "nothing to restore") + // The database was created in the offline state and needs to be made + // public. + // TODO (lucy): Ideally we'd just create the database in the public state in + // the first place, as a special case. + if err := r.publishDescriptors(ctx); err != nil { + return err + } return nil } @@ -1217,12 +1237,17 @@ func insertStats( return nil } -// publishDescriptors updates the RESTORED tables status from OFFLINE to PUBLIC. +// publishDescriptors updates the RESTORED descriptors' status from OFFLINE to PUBLIC. func (r *restoreResumer) publishDescriptors(ctx context.Context) error { details := r.job.Details().(jobspb.RestoreDetails) - if details.TablesPublished { + if details.DescriptorsPublished { return nil } + if fn := r.testingKnobs.beforePublishingDescriptors; fn != nil { + if err := fn(); err != nil { + return err + } + } log.Event(ctx, "making tables live") newDescriptorChangeJobs := make([]*jobs.StartableJob, 0) @@ -1235,42 +1260,36 @@ func (r *restoreResumer) publishDescriptors(ctx context.Context) error { return err } } - // Write the new TableDescriptors and flip state over to public so they can be + + // Write the new descriptors and flip state over to public so they can be // accessed. - b := txn.NewBatch() + allMutDescs := make([]catalog.MutableDescriptor, 0, + len(details.TableDescs)+len(details.TypeDescs)+len(details.SchemaDescs)+len(details.DatabaseDescs)) + // Create slices of raw descriptors for the restore job details. newTables := make([]*descpb.TableDescriptor, 0, len(details.TableDescs)) + newTypes := make([]*descpb.TypeDescriptor, 0, len(details.TypeDescs)) + newSchemas := make([]*descpb.SchemaDescriptor, 0, len(details.SchemaDescs)) + newDBs := make([]*descpb.DatabaseDescriptor, 0, len(details.DatabaseDescs)) + for _, tbl := range details.TableDescs { - existingTable := tabledesc.NewImmutable(*tbl) - newTableDesc := tabledesc.NewExistingMutable(*tbl) - newTableDesc.Version++ - newTableDesc.State = descpb.DescriptorState_PUBLIC - newTableDesc.OfflineReason = "" + mutTable := tabledesc.NewExistingMutable(*tbl) + allMutDescs = append(allMutDescs, mutTable) + newTables = append(newTables, mutTable.TableDesc()) // Convert any mutations that were in progress on the table descriptor // when the backup was taken, and convert them to schema change jobs. newJobs, err := createSchemaChangeJobsFromMutations(ctx, - r.execCfg.JobRegistry, r.execCfg.Codec, txn, r.job.Payload().Username, newTableDesc) + r.execCfg.JobRegistry, r.execCfg.Codec, txn, r.job.Payload().Username, mutTable) if err != nil { return err } newDescriptorChangeJobs = append(newDescriptorChangeJobs, newJobs...) - if err := VerifyDescriptorVersion(ctx, txn, r.execCfg.Codec, existingTable); err != nil { - return errors.Wrap(err, "validating table descriptor has not changed") - } - b.Put( - catalogkeys.MakeDescMetadataKey(keys.SystemSQLCodec, newTableDesc.ID), - newTableDesc.DescriptorProto(), - ) - newTables = append(newTables, newTableDesc.TableDesc()) - } - - if err := txn.Run(ctx, b); err != nil { - return errors.Wrap(err, "publishing tables") } - // For all of the newly created types, make type schema change jobs for any // type descriptors that were backed up in the middle of a type schema change. for _, typDesc := range details.TypeDescs { typ := typedesc.NewExistingMutable(*typDesc) + allMutDescs = append(allMutDescs, typ) + newTypes = append(newTypes, typ.TypeDesc()) if typ.HasPendingSchemaChanges() { typJob, err := createTypeChangeJobFromDesc(ctx, r.execCfg.JobRegistry, r.execCfg.Codec, txn, r.job.Payload().Username, typ) if err != nil { @@ -1279,6 +1298,52 @@ func (r *restoreResumer) publishDescriptors(ctx context.Context) error { newDescriptorChangeJobs = append(newDescriptorChangeJobs, typJob) } } + for _, sc := range details.SchemaDescs { + mutSchema := schemadesc.NewMutableExisting(*sc) + allMutDescs = append(allMutDescs, mutSchema) + newSchemas = append(newSchemas, mutSchema.SchemaDesc()) + } + for _, db := range details.DatabaseDescs { + // Jobs started before 20.2 upgrade finalization don't put databases in + // an offline state. + // TODO (lucy): Should we make this more explicit with a format version + // field in the details? + if db.GetState() != descpb.DescriptorState_OFFLINE { + newDBs = append(newDBs, db) + } else { + mutDB := dbdesc.NewExistingMutable(*db) + allMutDescs = append(allMutDescs, dbdesc.NewExistingMutable(*db)) + newDBs = append(newDBs, mutDB.DatabaseDesc()) + } + } + + b := txn.NewBatch() + for _, desc := range allMutDescs { + // Verify that the version of the descriptor stored in the job details is + // the same as what's actually on disk. If not, then the descriptor was + // somehow changed from under us and it's unsafe to proceed. + if err := VerifyDescriptorVersion(ctx, txn, keys.SystemSQLCodec, desc); err != nil { + return errors.Wrap(err, "publishing descriptors") + } + + desc.SetPublic() + desc.MaybeIncrementVersion() + if err := catalogkv.WriteDescToBatch( + ctx, + false, /* kvTrace */ + r.settings, + b, + keys.SystemSQLCodec, + desc.GetID(), + desc, + ); err != nil { + return err + } + } + + if err := txn.Run(ctx, b); err != nil { + return errors.Wrap(err, "publishing descriptors") + } for _, tenant := range details.Tenants { if err := sql.ActivateTenant(ctx, r.execCfg, txn, tenant.ID); err != nil { @@ -1287,8 +1352,11 @@ func (r *restoreResumer) publishDescriptors(ctx context.Context) error { } // Update and persist the state of the job. - details.TablesPublished = true + details.DescriptorsPublished = true details.TableDescs = newTables + details.TypeDescs = newTypes + details.SchemaDescs = newSchemas + details.DatabaseDescs = newDBs if err := r.job.WithTxn(txn).SetDetails(ctx, details); err != nil { for _, newJob := range newDescriptorChangeJobs { if cleanupErr := newJob.CleanupOnRollback(ctx); cleanupErr != nil { @@ -1346,6 +1414,8 @@ func (r *restoreResumer) OnFailOrCancel(ctx context.Context, phs interface{}) er } // dropDescriptors implements the OnFailOrCancel logic. +// TODO (lucy): If the descriptors have already been published, we need to queue +// drop jobs for all the descriptors. func (r *restoreResumer) dropDescriptors( ctx context.Context, jr *jobs.Registry, txn *kv.Txn, ) error { @@ -1379,21 +1449,28 @@ func (r *restoreResumer) dropDescriptors( tablesToGC := make([]descpb.ID, 0, len(details.TableDescs)) for i := range mutableTables { tableToDrop := mutableTables[i] + if err := VerifyDescriptorVersion(ctx, txn, keys.SystemSQLCodec, tableToDrop); err != nil { + return errors.Wrap(err, "deleting descriptors") + } + tablesToGC = append(tablesToGC, tableToDrop.ID) - prev := tableToDrop.ImmutableCopy().(catalog.TableDescriptor) - tableToDrop.Version++ + tableToDrop.MaybeIncrementVersion() tableToDrop.State = descpb.DescriptorState_DROP err := catalogkv.RemovePublicTableNamespaceEntry(ctx, txn, keys.SystemSQLCodec, tableToDrop.ParentID, tableToDrop.Name) if err != nil { return errors.Wrap(err, "dropping tables caused by restore fail/cancel from public namespace") } - if err := VerifyDescriptorVersion(ctx, txn, r.execCfg.Codec, prev); err != nil { - return errors.Wrap(err, "dropping tables caused by restore fail/cancel") + if err := catalogkv.WriteDescToBatch( + ctx, + false, /* kvTrace */ + r.settings, + b, + keys.SystemSQLCodec, + tableToDrop.GetID(), + tableToDrop, + ); err != nil { + return err } - b.Put( - catalogkeys.MakeDescMetadataKey(keys.SystemSQLCodec, tableToDrop.ID), - tableToDrop.DescriptorProto(), - ) } // Drop the type descriptors that this restore created. @@ -1417,6 +1494,9 @@ func (r *restoreResumer) dropDescriptors( // Drop any schema descriptors that this restore created. Also collect the // descriptors so we can update their parent databases later. + // TODO (lucy): It's possible for tables and types to be added under the + // schema after it becomes public. To account for this, we need to refrain + // from dropping the schema if it has children, like we do for databases. dbsWithDeletedSchemas := make(map[descpb.ID][]*descpb.SchemaDescriptor) for i := range details.SchemaDescs { sc := details.SchemaDescs[i] diff --git a/pkg/jobs/jobspb/jobs.pb.go b/pkg/jobs/jobspb/jobs.pb.go index 58a3506a38f8..037f3b3a37d9 100644 --- a/pkg/jobs/jobspb/jobs.pb.go +++ b/pkg/jobs/jobspb/jobs.pb.go @@ -54,7 +54,7 @@ func (x EncryptionMode) String() string { return proto.EnumName(EncryptionMode_name, int32(x)) } func (EncryptionMode) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_jobs_4a7d6385c7acb10d, []int{0} + return fileDescriptor_jobs_105068742e01a54c, []int{0} } type Status int32 @@ -83,7 +83,7 @@ func (x Status) String() string { return proto.EnumName(Status_name, int32(x)) } func (Status) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_jobs_4a7d6385c7acb10d, []int{1} + return fileDescriptor_jobs_105068742e01a54c, []int{1} } type Type int32 @@ -129,7 +129,7 @@ var Type_value = map[string]int32{ } func (Type) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_jobs_4a7d6385c7acb10d, []int{2} + return fileDescriptor_jobs_105068742e01a54c, []int{2} } type EncryptionInfo_Scheme int32 @@ -149,7 +149,7 @@ func (x EncryptionInfo_Scheme) String() string { return proto.EnumName(EncryptionInfo_Scheme_name, int32(x)) } func (EncryptionInfo_Scheme) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_jobs_4a7d6385c7acb10d, []int{2, 0} + return fileDescriptor_jobs_105068742e01a54c, []int{2, 0} } type SchemaChangeGCProgress_Status int32 @@ -179,7 +179,7 @@ func (x SchemaChangeGCProgress_Status) String() string { return proto.EnumName(SchemaChangeGCProgress_Status_name, int32(x)) } func (SchemaChangeGCProgress_Status) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_jobs_4a7d6385c7acb10d, []int{16, 0} + return fileDescriptor_jobs_105068742e01a54c, []int{16, 0} } type Lease struct { @@ -193,7 +193,7 @@ func (m *Lease) Reset() { *m = Lease{} } func (m *Lease) String() string { return proto.CompactTextString(m) } func (*Lease) ProtoMessage() {} func (*Lease) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_4a7d6385c7acb10d, []int{0} + return fileDescriptor_jobs_105068742e01a54c, []int{0} } func (m *Lease) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -234,7 +234,7 @@ func (m *BackupEncryptionOptions) Reset() { *m = BackupEncryptionOptions func (m *BackupEncryptionOptions) String() string { return proto.CompactTextString(m) } func (*BackupEncryptionOptions) ProtoMessage() {} func (*BackupEncryptionOptions) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_4a7d6385c7acb10d, []int{1} + return fileDescriptor_jobs_105068742e01a54c, []int{1} } func (m *BackupEncryptionOptions) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -268,7 +268,7 @@ func (m *BackupEncryptionOptions_KMSInfo) Reset() { *m = BackupEncryptio func (m *BackupEncryptionOptions_KMSInfo) String() string { return proto.CompactTextString(m) } func (*BackupEncryptionOptions_KMSInfo) ProtoMessage() {} func (*BackupEncryptionOptions_KMSInfo) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_4a7d6385c7acb10d, []int{1, 0} + return fileDescriptor_jobs_105068742e01a54c, []int{1, 0} } func (m *BackupEncryptionOptions_KMSInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -308,7 +308,7 @@ func (m *EncryptionInfo) Reset() { *m = EncryptionInfo{} } func (m *EncryptionInfo) String() string { return proto.CompactTextString(m) } func (*EncryptionInfo) ProtoMessage() {} func (*EncryptionInfo) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_4a7d6385c7acb10d, []int{2} + return fileDescriptor_jobs_105068742e01a54c, []int{2} } func (m *EncryptionInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -363,7 +363,7 @@ func (m *BackupDetails) Reset() { *m = BackupDetails{} } func (m *BackupDetails) String() string { return proto.CompactTextString(m) } func (*BackupDetails) ProtoMessage() {} func (*BackupDetails) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_4a7d6385c7acb10d, []int{3} + return fileDescriptor_jobs_105068742e01a54c, []int{3} } func (m *BackupDetails) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -395,7 +395,7 @@ func (m *BackupProgress) Reset() { *m = BackupProgress{} } func (m *BackupProgress) String() string { return proto.CompactTextString(m) } func (*BackupProgress) ProtoMessage() {} func (*BackupProgress) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_4a7d6385c7acb10d, []int{4} + return fileDescriptor_jobs_105068742e01a54c, []int{4} } func (m *BackupProgress) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -444,18 +444,20 @@ type RestoreDetails struct { OverrideDB string `protobuf:"bytes,6,opt,name=override_db,json=overrideDb,proto3" json:"override_db,omitempty"` // The restore job has several atomic stages. For now, we keep track of which // stages have completed via these flags. - PrepareCompleted bool `protobuf:"varint,8,opt,name=prepare_completed,json=prepareCompleted,proto3" json:"prepare_completed,omitempty"` - StatsInserted bool `protobuf:"varint,9,opt,name=stats_inserted,json=statsInserted,proto3" json:"stats_inserted,omitempty"` - TablesPublished bool `protobuf:"varint,10,opt,name=tables_published,json=tablesPublished,proto3" json:"tables_published,omitempty"` - DescriptorCoverage github_com_cockroachdb_cockroach_pkg_sql_sem_tree.DescriptorCoverage `protobuf:"varint,11,opt,name=descriptor_coverage,json=descriptorCoverage,proto3,casttype=github.com/cockroachdb/cockroach/pkg/sql/sem/tree.DescriptorCoverage" json:"descriptor_coverage,omitempty"` - Encryption *BackupEncryptionOptions `protobuf:"bytes,12,opt,name=encryption,proto3" json:"encryption,omitempty"` + PrepareCompleted bool `protobuf:"varint,8,opt,name=prepare_completed,json=prepareCompleted,proto3" json:"prepare_completed,omitempty"` + StatsInserted bool `protobuf:"varint,9,opt,name=stats_inserted,json=statsInserted,proto3" json:"stats_inserted,omitempty"` + // DescriptorsPublished indicates whether or not the descriptors written in + // the job have been transactionally updated after the data was restored. + DescriptorsPublished bool `protobuf:"varint,10,opt,name=descriptors_published,json=descriptorsPublished,proto3" json:"descriptors_published,omitempty"` + DescriptorCoverage github_com_cockroachdb_cockroach_pkg_sql_sem_tree.DescriptorCoverage `protobuf:"varint,11,opt,name=descriptor_coverage,json=descriptorCoverage,proto3,casttype=github.com/cockroachdb/cockroach/pkg/sql/sem/tree.DescriptorCoverage" json:"descriptor_coverage,omitempty"` + Encryption *BackupEncryptionOptions `protobuf:"bytes,12,opt,name=encryption,proto3" json:"encryption,omitempty"` } func (m *RestoreDetails) Reset() { *m = RestoreDetails{} } func (m *RestoreDetails) String() string { return proto.CompactTextString(m) } func (*RestoreDetails) ProtoMessage() {} func (*RestoreDetails) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_4a7d6385c7acb10d, []int{5} + return fileDescriptor_jobs_105068742e01a54c, []int{5} } func (m *RestoreDetails) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -492,7 +494,7 @@ func (m *RestoreDetails_DescriptorRewrite) Reset() { *m = RestoreDetails func (m *RestoreDetails_DescriptorRewrite) String() string { return proto.CompactTextString(m) } func (*RestoreDetails_DescriptorRewrite) ProtoMessage() {} func (*RestoreDetails_DescriptorRewrite) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_4a7d6385c7acb10d, []int{5, 0} + return fileDescriptor_jobs_105068742e01a54c, []int{5, 0} } func (m *RestoreDetails_DescriptorRewrite) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -525,7 +527,7 @@ func (m *RestoreDetails_BackupLocalityInfo) Reset() { *m = RestoreDetail func (m *RestoreDetails_BackupLocalityInfo) String() string { return proto.CompactTextString(m) } func (*RestoreDetails_BackupLocalityInfo) ProtoMessage() {} func (*RestoreDetails_BackupLocalityInfo) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_4a7d6385c7acb10d, []int{5, 1} + return fileDescriptor_jobs_105068742e01a54c, []int{5, 1} } func (m *RestoreDetails_BackupLocalityInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -559,7 +561,7 @@ func (m *RestoreDetails_Tenant) Reset() { *m = RestoreDetails_Tenant{} } func (m *RestoreDetails_Tenant) String() string { return proto.CompactTextString(m) } func (*RestoreDetails_Tenant) ProtoMessage() {} func (*RestoreDetails_Tenant) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_4a7d6385c7acb10d, []int{5, 3} + return fileDescriptor_jobs_105068742e01a54c, []int{5, 3} } func (m *RestoreDetails_Tenant) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -592,7 +594,7 @@ func (m *RestoreProgress) Reset() { *m = RestoreProgress{} } func (m *RestoreProgress) String() string { return proto.CompactTextString(m) } func (*RestoreProgress) ProtoMessage() {} func (*RestoreProgress) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_4a7d6385c7acb10d, []int{6} + return fileDescriptor_jobs_105068742e01a54c, []int{6} } func (m *RestoreProgress) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -651,7 +653,7 @@ func (m *ImportDetails) Reset() { *m = ImportDetails{} } func (m *ImportDetails) String() string { return proto.CompactTextString(m) } func (*ImportDetails) ProtoMessage() {} func (*ImportDetails) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_4a7d6385c7acb10d, []int{7} + return fileDescriptor_jobs_105068742e01a54c, []int{7} } func (m *ImportDetails) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -689,7 +691,7 @@ func (m *ImportDetails_Table) Reset() { *m = ImportDetails_Table{} } func (m *ImportDetails_Table) String() string { return proto.CompactTextString(m) } func (*ImportDetails_Table) ProtoMessage() {} func (*ImportDetails_Table) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_4a7d6385c7acb10d, []int{7, 0} + return fileDescriptor_jobs_105068742e01a54c, []int{7, 0} } func (m *ImportDetails_Table) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -732,7 +734,7 @@ func (m *ImportProgress) Reset() { *m = ImportProgress{} } func (m *ImportProgress) String() string { return proto.CompactTextString(m) } func (*ImportProgress) ProtoMessage() {} func (*ImportProgress) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_4a7d6385c7acb10d, []int{8} + return fileDescriptor_jobs_105068742e01a54c, []int{8} } func (m *ImportProgress) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -766,7 +768,7 @@ func (m *TypeSchemaChangeDetails) Reset() { *m = TypeSchemaChangeDetails func (m *TypeSchemaChangeDetails) String() string { return proto.CompactTextString(m) } func (*TypeSchemaChangeDetails) ProtoMessage() {} func (*TypeSchemaChangeDetails) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_4a7d6385c7acb10d, []int{9} + return fileDescriptor_jobs_105068742e01a54c, []int{9} } func (m *TypeSchemaChangeDetails) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -799,7 +801,7 @@ func (m *TypeSchemaChangeProgress) Reset() { *m = TypeSchemaChangeProgre func (m *TypeSchemaChangeProgress) String() string { return proto.CompactTextString(m) } func (*TypeSchemaChangeProgress) ProtoMessage() {} func (*TypeSchemaChangeProgress) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_4a7d6385c7acb10d, []int{10} + return fileDescriptor_jobs_105068742e01a54c, []int{10} } func (m *TypeSchemaChangeProgress) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -832,7 +834,7 @@ func (m *ResumeSpanList) Reset() { *m = ResumeSpanList{} } func (m *ResumeSpanList) String() string { return proto.CompactTextString(m) } func (*ResumeSpanList) ProtoMessage() {} func (*ResumeSpanList) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_4a7d6385c7acb10d, []int{11} + return fileDescriptor_jobs_105068742e01a54c, []int{11} } func (m *ResumeSpanList) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -867,7 +869,7 @@ func (m *DroppedTableDetails) Reset() { *m = DroppedTableDetails{} } func (m *DroppedTableDetails) String() string { return proto.CompactTextString(m) } func (*DroppedTableDetails) ProtoMessage() {} func (*DroppedTableDetails) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_4a7d6385c7acb10d, []int{12} + return fileDescriptor_jobs_105068742e01a54c, []int{12} } func (m *DroppedTableDetails) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -925,7 +927,7 @@ func (m *SchemaChangeGCDetails) Reset() { *m = SchemaChangeGCDetails{} } func (m *SchemaChangeGCDetails) String() string { return proto.CompactTextString(m) } func (*SchemaChangeGCDetails) ProtoMessage() {} func (*SchemaChangeGCDetails) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_4a7d6385c7acb10d, []int{13} + return fileDescriptor_jobs_105068742e01a54c, []int{13} } func (m *SchemaChangeGCDetails) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -959,7 +961,7 @@ func (m *SchemaChangeGCDetails_DroppedIndex) Reset() { *m = SchemaChange func (m *SchemaChangeGCDetails_DroppedIndex) String() string { return proto.CompactTextString(m) } func (*SchemaChangeGCDetails_DroppedIndex) ProtoMessage() {} func (*SchemaChangeGCDetails_DroppedIndex) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_4a7d6385c7acb10d, []int{13, 0} + return fileDescriptor_jobs_105068742e01a54c, []int{13, 0} } func (m *SchemaChangeGCDetails_DroppedIndex) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -993,7 +995,7 @@ func (m *SchemaChangeGCDetails_DroppedID) Reset() { *m = SchemaChangeGCD func (m *SchemaChangeGCDetails_DroppedID) String() string { return proto.CompactTextString(m) } func (*SchemaChangeGCDetails_DroppedID) ProtoMessage() {} func (*SchemaChangeGCDetails_DroppedID) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_4a7d6385c7acb10d, []int{13, 1} + return fileDescriptor_jobs_105068742e01a54c, []int{13, 1} } func (m *SchemaChangeGCDetails_DroppedID) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1057,7 +1059,7 @@ func (m *SchemaChangeDetails) Reset() { *m = SchemaChangeDetails{} } func (m *SchemaChangeDetails) String() string { return proto.CompactTextString(m) } func (*SchemaChangeDetails) ProtoMessage() {} func (*SchemaChangeDetails) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_4a7d6385c7acb10d, []int{14} + return fileDescriptor_jobs_105068742e01a54c, []int{14} } func (m *SchemaChangeDetails) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1089,7 +1091,7 @@ func (m *SchemaChangeProgress) Reset() { *m = SchemaChangeProgress{} } func (m *SchemaChangeProgress) String() string { return proto.CompactTextString(m) } func (*SchemaChangeProgress) ProtoMessage() {} func (*SchemaChangeProgress) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_4a7d6385c7acb10d, []int{15} + return fileDescriptor_jobs_105068742e01a54c, []int{15} } func (m *SchemaChangeProgress) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1125,7 +1127,7 @@ func (m *SchemaChangeGCProgress) Reset() { *m = SchemaChangeGCProgress{} func (m *SchemaChangeGCProgress) String() string { return proto.CompactTextString(m) } func (*SchemaChangeGCProgress) ProtoMessage() {} func (*SchemaChangeGCProgress) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_4a7d6385c7acb10d, []int{16} + return fileDescriptor_jobs_105068742e01a54c, []int{16} } func (m *SchemaChangeGCProgress) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1159,7 +1161,7 @@ func (m *SchemaChangeGCProgress_IndexProgress) Reset() { *m = SchemaChan func (m *SchemaChangeGCProgress_IndexProgress) String() string { return proto.CompactTextString(m) } func (*SchemaChangeGCProgress_IndexProgress) ProtoMessage() {} func (*SchemaChangeGCProgress_IndexProgress) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_4a7d6385c7acb10d, []int{16, 0} + return fileDescriptor_jobs_105068742e01a54c, []int{16, 0} } func (m *SchemaChangeGCProgress_IndexProgress) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1193,7 +1195,7 @@ func (m *SchemaChangeGCProgress_TableProgress) Reset() { *m = SchemaChan func (m *SchemaChangeGCProgress_TableProgress) String() string { return proto.CompactTextString(m) } func (*SchemaChangeGCProgress_TableProgress) ProtoMessage() {} func (*SchemaChangeGCProgress_TableProgress) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_4a7d6385c7acb10d, []int{16, 1} + return fileDescriptor_jobs_105068742e01a54c, []int{16, 1} } func (m *SchemaChangeGCProgress_TableProgress) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1226,7 +1228,7 @@ func (m *ChangefeedTarget) Reset() { *m = ChangefeedTarget{} } func (m *ChangefeedTarget) String() string { return proto.CompactTextString(m) } func (*ChangefeedTarget) ProtoMessage() {} func (*ChangefeedTarget) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_4a7d6385c7acb10d, []int{17} + return fileDescriptor_jobs_105068742e01a54c, []int{17} } func (m *ChangefeedTarget) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1279,7 +1281,7 @@ func (m *ChangefeedDetails) Reset() { *m = ChangefeedDetails{} } func (m *ChangefeedDetails) String() string { return proto.CompactTextString(m) } func (*ChangefeedDetails) ProtoMessage() {} func (*ChangefeedDetails) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_4a7d6385c7acb10d, []int{18} + return fileDescriptor_jobs_105068742e01a54c, []int{18} } func (m *ChangefeedDetails) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1314,7 +1316,7 @@ func (m *ResolvedSpan) Reset() { *m = ResolvedSpan{} } func (m *ResolvedSpan) String() string { return proto.CompactTextString(m) } func (*ResolvedSpan) ProtoMessage() {} func (*ResolvedSpan) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_4a7d6385c7acb10d, []int{19} + return fileDescriptor_jobs_105068742e01a54c, []int{19} } func (m *ResolvedSpan) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1357,7 +1359,7 @@ func (m *ChangefeedProgress) Reset() { *m = ChangefeedProgress{} } func (m *ChangefeedProgress) String() string { return proto.CompactTextString(m) } func (*ChangefeedProgress) ProtoMessage() {} func (*ChangefeedProgress) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_4a7d6385c7acb10d, []int{20} + return fileDescriptor_jobs_105068742e01a54c, []int{20} } func (m *ChangefeedProgress) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1401,7 +1403,7 @@ func (m *CreateStatsDetails) Reset() { *m = CreateStatsDetails{} } func (m *CreateStatsDetails) String() string { return proto.CompactTextString(m) } func (*CreateStatsDetails) ProtoMessage() {} func (*CreateStatsDetails) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_4a7d6385c7acb10d, []int{21} + return fileDescriptor_jobs_105068742e01a54c, []int{21} } func (m *CreateStatsDetails) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1442,7 +1444,7 @@ func (m *CreateStatsDetails_ColStat) Reset() { *m = CreateStatsDetails_C func (m *CreateStatsDetails_ColStat) String() string { return proto.CompactTextString(m) } func (*CreateStatsDetails_ColStat) ProtoMessage() {} func (*CreateStatsDetails_ColStat) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_4a7d6385c7acb10d, []int{21, 0} + return fileDescriptor_jobs_105068742e01a54c, []int{21, 0} } func (m *CreateStatsDetails_ColStat) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1474,7 +1476,7 @@ func (m *CreateStatsProgress) Reset() { *m = CreateStatsProgress{} } func (m *CreateStatsProgress) String() string { return proto.CompactTextString(m) } func (*CreateStatsProgress) ProtoMessage() {} func (*CreateStatsProgress) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_4a7d6385c7acb10d, []int{22} + return fileDescriptor_jobs_105068742e01a54c, []int{22} } func (m *CreateStatsProgress) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1541,7 +1543,7 @@ func (m *Payload) Reset() { *m = Payload{} } func (m *Payload) String() string { return proto.CompactTextString(m) } func (*Payload) ProtoMessage() {} func (*Payload) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_4a7d6385c7acb10d, []int{23} + return fileDescriptor_jobs_105068742e01a54c, []int{23} } func (m *Payload) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1880,7 +1882,7 @@ func (m *Progress) Reset() { *m = Progress{} } func (m *Progress) String() string { return proto.CompactTextString(m) } func (*Progress) ProtoMessage() {} func (*Progress) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_4a7d6385c7acb10d, []int{24} + return fileDescriptor_jobs_105068742e01a54c, []int{24} } func (m *Progress) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2286,7 +2288,7 @@ func (m *Job) Reset() { *m = Job{} } func (m *Job) String() string { return proto.CompactTextString(m) } func (*Job) ProtoMessage() {} func (*Job) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_4a7d6385c7acb10d, []int{25} + return fileDescriptor_jobs_105068742e01a54c, []int{25} } func (m *Job) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2875,10 +2877,10 @@ func (m *RestoreDetails) MarshalTo(dAtA []byte) (int, error) { } i++ } - if m.TablesPublished { + if m.DescriptorsPublished { dAtA[i] = 0x50 i++ - if m.TablesPublished { + if m.DescriptorsPublished { dAtA[i] = 1 } else { dAtA[i] = 0 @@ -4839,7 +4841,7 @@ func (m *RestoreDetails) Size() (n int) { if m.StatsInserted { n += 2 } - if m.TablesPublished { + if m.DescriptorsPublished { n += 2 } if m.DescriptorCoverage != 0 { @@ -7118,7 +7120,7 @@ func (m *RestoreDetails) Unmarshal(dAtA []byte) error { m.StatsInserted = bool(v != 0) case 10: if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field TablesPublished", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field DescriptorsPublished", wireType) } var v int for shift := uint(0); ; shift += 7 { @@ -7135,7 +7137,7 @@ func (m *RestoreDetails) Unmarshal(dAtA []byte) error { break } } - m.TablesPublished = bool(v != 0) + m.DescriptorsPublished = bool(v != 0) case 11: if wireType != 0 { return fmt.Errorf("proto: wrong wireType = %d for field DescriptorCoverage", wireType) @@ -12494,267 +12496,267 @@ var ( ErrIntOverflowJobs = fmt.Errorf("proto: integer overflow") ) -func init() { proto.RegisterFile("jobs/jobspb/jobs.proto", fileDescriptor_jobs_4a7d6385c7acb10d) } - -var fileDescriptor_jobs_4a7d6385c7acb10d = []byte{ - // 4131 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xc4, 0x5b, 0xcd, 0x6f, 0x23, 0x47, - 0x76, 0x57, 0x93, 0x2d, 0xb2, 0xf9, 0x28, 0x52, 0xcd, 0x92, 0x66, 0x86, 0xcb, 0xd8, 0xa2, 0x42, - 0xdb, 0xeb, 0x99, 0xb1, 0x4d, 0x79, 0xe5, 0x5d, 0xaf, 0x77, 0xb2, 0x1e, 0xaf, 0xf8, 0x21, 0x89, - 0xd4, 0xe8, 0xc3, 0x4d, 0x69, 0xbc, 0xf6, 0xc2, 0xe9, 0x34, 0xd9, 0x25, 0xaa, 0x23, 0xb2, 0x9b, - 0xd3, 0xd5, 0x9c, 0x19, 0x2d, 0x82, 0x24, 0xd8, 0x20, 0xc1, 0x62, 0x4e, 0x09, 0xb0, 0xc9, 0x21, - 0xc9, 0x04, 0x01, 0x12, 0x23, 0x39, 0x04, 0x08, 0x10, 0x04, 0x49, 0xfe, 0x04, 0x1f, 0x72, 0xd8, - 0xe3, 0x22, 0x07, 0x6e, 0x42, 0x5f, 0x72, 0x48, 0x80, 0x45, 0x72, 0x9b, 0x53, 0x50, 0x1f, 0xdd, - 0x6c, 0x52, 0x1a, 0x89, 0x1a, 0xd9, 0x9b, 0x8b, 0xa6, 0xeb, 0x55, 0xd5, 0xaf, 0xbe, 0xde, 0xfb, - 0xbd, 0x57, 0xaf, 0x38, 0x70, 0xfd, 0x37, 0x9d, 0x26, 0x59, 0xa1, 0x7f, 0x7a, 0x4d, 0xf6, 0x4f, - 0xb1, 0xe7, 0x3a, 0x9e, 0x83, 0xbe, 0xd6, 0x72, 0x5a, 0xc7, 0xae, 0x63, 0xb4, 0x8e, 0x8a, 0xe4, - 0x41, 0xa7, 0xc8, 0x6a, 0x78, 0xab, 0xdc, 0x35, 0xec, 0xba, 0x8e, 0x4b, 0xdb, 0xf3, 0x0f, 0xde, - 0x23, 0xb7, 0xd8, 0x76, 0xda, 0x0e, 0xfb, 0x5c, 0xa1, 0x5f, 0x42, 0x8a, 0x18, 0x46, 0xaf, 0xb9, - 0x62, 0x1a, 0x9e, 0x21, 0x64, 0x59, 0x5f, 0x66, 0x39, 0x6f, 0x1d, 0x3a, 0x6e, 0xd7, 0xf0, 0x7c, - 0x8c, 0x57, 0xc8, 0x83, 0xce, 0x4a, 0xcb, 0xf0, 0x8c, 0x8e, 0xd3, 0x5e, 0x31, 0x31, 0x69, 0xf5, - 0x9a, 0x2b, 0xc4, 0x73, 0xfb, 0x2d, 0xaf, 0xef, 0x62, 0xd3, 0xef, 0xde, 0xf7, 0xac, 0xce, 0xca, - 0x51, 0xa7, 0xb5, 0xe2, 0x59, 0x5d, 0x4c, 0x3c, 0xa3, 0xdb, 0xe3, 0x35, 0x85, 0xdf, 0x81, 0xd9, - 0x7b, 0xd8, 0x20, 0x18, 0x7d, 0x02, 0x71, 0xdb, 0x31, 0xb1, 0x6e, 0x99, 0x59, 0x69, 0x59, 0xba, - 0x99, 0x2a, 0xad, 0x0d, 0x07, 0xf9, 0xd8, 0x8e, 0x63, 0xe2, 0x5a, 0xe5, 0xd9, 0x20, 0xff, 0x4e, - 0xdb, 0xf2, 0x8e, 0xfa, 0xcd, 0x62, 0xcb, 0xe9, 0xae, 0x04, 0xeb, 0x34, 0x9b, 0xa3, 0xef, 0x95, - 0xde, 0x71, 0x7b, 0x45, 0xcc, 0xb2, 0xc8, 0xbb, 0x69, 0x31, 0x8a, 0x58, 0x33, 0xd1, 0x22, 0xcc, - 0xe2, 0x9e, 0xd3, 0x3a, 0xca, 0x46, 0x96, 0xa5, 0x9b, 0x51, 0x8d, 0x17, 0xee, 0xc8, 0xff, 0xf9, - 0x97, 0x79, 0xa9, 0xf0, 0x59, 0x04, 0x6e, 0x94, 0x8c, 0xd6, 0x71, 0xbf, 0x57, 0xb5, 0x5b, 0xee, - 0x49, 0xcf, 0xb3, 0x1c, 0x7b, 0x97, 0xfd, 0x25, 0x48, 0x85, 0xe8, 0x31, 0x3e, 0x61, 0xf3, 0x99, - 0xd3, 0xe8, 0x27, 0x7a, 0x1f, 0xe4, 0xae, 0x63, 0x62, 0x06, 0x94, 0x5e, 0xbd, 0x55, 0x7c, 0xee, - 0x96, 0x17, 0x47, 0x68, 0xdb, 0x8e, 0x89, 0x35, 0xd6, 0x0d, 0x35, 0x41, 0x39, 0xee, 0x12, 0xdd, - 0xb2, 0x0f, 0x9d, 0x6c, 0x74, 0x59, 0xba, 0x99, 0x5c, 0xbd, 0x73, 0x0e, 0xc4, 0x73, 0xa6, 0x55, - 0xdc, 0xda, 0x6e, 0xd4, 0xec, 0x43, 0xa7, 0x94, 0x1c, 0x0e, 0xf2, 0x71, 0x51, 0xd0, 0xe2, 0xc7, - 0x5d, 0x42, 0x3f, 0x72, 0xbb, 0xe0, 0xcb, 0xe8, 0xfc, 0xfb, 0xae, 0xc5, 0xe6, 0x9f, 0xd0, 0xe8, - 0x27, 0x7a, 0x13, 0x10, 0xe6, 0x78, 0xd8, 0xd4, 0xe9, 0xf9, 0xea, 0x74, 0x81, 0x11, 0xb6, 0x40, - 0x35, 0xa8, 0xa9, 0x18, 0x9e, 0xb1, 0x85, 0x4f, 0xf8, 0x0e, 0x89, 0x7d, 0xfa, 0xdd, 0x28, 0xa4, - 0x47, 0x53, 0x61, 0xf0, 0x9b, 0x10, 0x23, 0xad, 0x23, 0xdc, 0xc5, 0x6c, 0x84, 0xf4, 0xea, 0xdb, - 0x53, 0x6d, 0x07, 0xed, 0x5a, 0x6c, 0xb0, 0x7e, 0x9a, 0xe8, 0x8f, 0x10, 0xc8, 0xc4, 0xe8, 0x78, - 0x62, 0x22, 0xec, 0x1b, 0xfd, 0x99, 0x04, 0xcb, 0x93, 0x33, 0x2a, 0x9d, 0x6c, 0x6d, 0x37, 0xb6, - 0x0d, 0xe2, 0x61, 0x77, 0x0b, 0x9f, 0xd4, 0x2a, 0xd9, 0xe8, 0x72, 0xf4, 0x66, 0x72, 0x75, 0x77, - 0xfa, 0x81, 0xab, 0x17, 0x20, 0x56, 0x6d, 0xcf, 0x3d, 0xd1, 0x2e, 0x1c, 0x38, 0xd7, 0x80, 0xd7, - 0xa6, 0x82, 0x0a, 0xeb, 0x50, 0x82, 0xeb, 0xd0, 0x22, 0xcc, 0x3e, 0x34, 0x3a, 0x7d, 0x2c, 0x56, - 0xcb, 0x0b, 0x77, 0x22, 0xef, 0x49, 0x85, 0x1b, 0x10, 0xe3, 0x1b, 0x83, 0x52, 0x90, 0x58, 0xab, - 0x36, 0x56, 0xbf, 0xf5, 0xee, 0x46, 0x79, 0x5b, 0x9d, 0x11, 0x47, 0xf0, 0x37, 0x31, 0x48, 0x71, - 0x9d, 0xa8, 0x60, 0xcf, 0xb0, 0x3a, 0x04, 0x95, 0x00, 0x88, 0x67, 0xb8, 0x9e, 0x4e, 0xcd, 0x8a, - 0x8d, 0x91, 0x5c, 0x7d, 0x39, 0xb4, 0x19, 0xd4, 0xec, 0x8a, 0x47, 0x9d, 0x56, 0x71, 0xdf, 0x37, - 0xbb, 0x92, 0xfc, 0xf9, 0x20, 0x3f, 0xa3, 0x25, 0x58, 0x37, 0x2a, 0x45, 0x77, 0x41, 0xc1, 0xb6, - 0xc9, 0x11, 0x22, 0xd3, 0x23, 0xc4, 0xb1, 0x6d, 0xb2, 0xfe, 0x5f, 0xe3, 0x4a, 0x46, 0xd5, 0x39, - 0x51, 0x8a, 0x0f, 0x07, 0xf9, 0xe8, 0x81, 0x56, 0xe3, 0xda, 0xf6, 0x3a, 0xcc, 0x37, 0xd9, 0x7c, - 0xf5, 0xae, 0x61, 0x5b, 0x87, 0x98, 0x78, 0x59, 0x99, 0xad, 0x39, 0xcd, 0xc5, 0xdb, 0x42, 0x8a, - 0x7e, 0x4f, 0x82, 0x85, 0xbe, 0x6b, 0x11, 0xbd, 0x79, 0xa2, 0x77, 0x9c, 0x96, 0xd1, 0xb1, 0xbc, - 0x13, 0xfd, 0xf8, 0x61, 0x76, 0x96, 0x1d, 0xef, 0xdd, 0x0b, 0x6d, 0x44, 0xec, 0x47, 0xf1, 0xc0, - 0xb5, 0x48, 0xe9, 0xe4, 0x9e, 0x40, 0xd8, 0x7a, 0xc8, 0x8e, 0xa0, 0xb4, 0x38, 0x1c, 0xe4, 0xd5, - 0x03, 0xad, 0x16, 0xae, 0xba, 0xaf, 0xa9, 0xfd, 0x89, 0xc6, 0xc8, 0x08, 0x8c, 0xc3, 0x72, 0x6c, - 0xdd, 0xe1, 0xd6, 0x96, 0x8d, 0xb1, 0x3d, 0x59, 0xbd, 0xbc, 0x9d, 0x6a, 0x19, 0x7c, 0x8a, 0x51, - 0xfe, 0x48, 0x82, 0x1c, 0x25, 0x3e, 0xdc, 0xa2, 0x06, 0x18, 0x90, 0xa1, 0xee, 0xe2, 0x96, 0xe3, - 0x9a, 0xd9, 0x38, 0xdd, 0x9d, 0x52, 0xe3, 0xdf, 0xa6, 0xe5, 0x3b, 0x46, 0xab, 0xfd, 0xbe, 0x65, - 0x16, 0x0f, 0x0e, 0x6a, 0x95, 0xe1, 0x20, 0x9f, 0xdd, 0xf3, 0xc1, 0x83, 0xf3, 0xd2, 0x18, 0xb4, - 0x96, 0xed, 0x3d, 0xa7, 0x06, 0xbd, 0x07, 0xe9, 0x96, 0xd3, 0xe9, 0xe0, 0x16, 0x5b, 0xf6, 0x81, - 0x56, 0xcb, 0x2a, 0xec, 0x2c, 0x33, 0xc3, 0x41, 0x3e, 0x55, 0x0e, 0x6a, 0xe8, 0xa9, 0xa6, 0x5a, - 0xe1, 0x22, 0xd2, 0x60, 0x3e, 0xb4, 0x61, 0x8c, 0xd5, 0x12, 0x6c, 0xb7, 0x6e, 0x4d, 0x6d, 0x90, - 0x5a, 0x1a, 0x8f, 0x95, 0x73, 0x65, 0xb8, 0x76, 0xe6, 0x29, 0x5e, 0x64, 0x48, 0x89, 0xb0, 0x21, - 0xa9, 0x90, 0xe6, 0x87, 0xb2, 0xe7, 0x3a, 0x6d, 0x17, 0x13, 0x52, 0xf8, 0xef, 0x34, 0xa4, 0x35, - 0x4c, 0x3c, 0xc7, 0xc5, 0xbe, 0xf1, 0xfc, 0x83, 0x04, 0x0b, 0xd4, 0x61, 0xb9, 0x56, 0xcf, 0x73, - 0x5c, 0xdd, 0xc5, 0x8f, 0x5c, 0xcb, 0xc3, 0x24, 0x1b, 0x61, 0x4a, 0xb7, 0x76, 0xce, 0x12, 0xc6, - 0x81, 0x8a, 0x95, 0x00, 0x44, 0x13, 0x18, 0x5c, 0xef, 0xee, 0xfe, 0xe8, 0xe7, 0xf9, 0x3b, 0x53, - 0x9d, 0xe3, 0x69, 0x1f, 0x5a, 0xac, 0x55, 0x34, 0x64, 0x9e, 0x02, 0x46, 0x2f, 0x81, 0x4c, 0xf5, - 0x96, 0x11, 0x5f, 0xa2, 0xa4, 0x0c, 0x07, 0x79, 0x99, 0x6a, 0xb6, 0xc6, 0xa4, 0x63, 0xb6, 0x2c, - 0xbf, 0x80, 0x2d, 0x6f, 0x40, 0xd2, 0x33, 0x9a, 0x1d, 0xac, 0xd3, 0x91, 0x89, 0x30, 0xbf, 0xaf, - 0x4f, 0xec, 0x04, 0x79, 0xd0, 0x69, 0x1a, 0x04, 0x17, 0xf7, 0x69, 0xcb, 0xd0, 0xda, 0xc1, 0xf3, - 0x05, 0x04, 0xad, 0x40, 0xd2, 0x79, 0x88, 0x5d, 0xd7, 0x32, 0xb1, 0x6e, 0x36, 0x99, 0x0d, 0x25, - 0x4a, 0xe9, 0xe1, 0x20, 0x0f, 0xbb, 0x42, 0x5c, 0x29, 0x69, 0xe0, 0x37, 0xa9, 0x34, 0x91, 0x07, - 0x8b, 0x82, 0x2a, 0x02, 0xfb, 0x67, 0xfa, 0x14, 0x67, 0x53, 0xf8, 0xee, 0xf4, 0x87, 0xc1, 0xcf, - 0xdd, 0x57, 0x1e, 0xe6, 0x27, 0xf9, 0x22, 0x51, 0xf3, 0x54, 0x0d, 0x7a, 0x03, 0x32, 0x3d, 0x17, - 0xf7, 0x0c, 0x17, 0xeb, 0x2d, 0xa7, 0xdb, 0xeb, 0x60, 0x0f, 0x9b, 0x4c, 0xfb, 0x15, 0x4d, 0x15, - 0x15, 0x65, 0x5f, 0x8e, 0x5e, 0x83, 0x34, 0xf1, 0x0c, 0x8f, 0xba, 0x6f, 0x82, 0x5d, 0xda, 0x32, - 0xc1, 0x5a, 0xa6, 0x98, 0xb4, 0x26, 0x84, 0xe8, 0x16, 0xa8, 0x6c, 0x23, 0x88, 0xde, 0xeb, 0x37, - 0x3b, 0x16, 0x39, 0xc2, 0x66, 0x16, 0x58, 0xc3, 0x79, 0x2e, 0xdf, 0xf3, 0xc5, 0xe8, 0x64, 0x4c, - 0x01, 0x5b, 0x74, 0x3b, 0x8c, 0x36, 0xce, 0x26, 0x97, 0xa5, 0x9b, 0xb3, 0xa5, 0xcd, 0x67, 0x83, - 0x7c, 0x65, 0x6a, 0xed, 0x21, 0xb8, 0xbb, 0xe2, 0xb9, 0x18, 0x87, 0x94, 0xb1, 0x2c, 0xf0, 0xc2, - 0x7a, 0xe4, 0xcb, 0x90, 0x06, 0x30, 0x32, 0xbc, 0xec, 0xdc, 0x0b, 0x73, 0x5c, 0x08, 0x05, 0xed, - 0x41, 0xdc, 0xc3, 0xb6, 0x61, 0x7b, 0x24, 0x9b, 0x62, 0xc7, 0xf6, 0xf6, 0xf4, 0xc7, 0xb6, 0xcf, - 0x3a, 0xfa, 0xfa, 0x28, 0x60, 0x50, 0x05, 0xc0, 0x3b, 0xe9, 0xf9, 0xea, 0x98, 0x66, 0xa0, 0xaf, - 0x3d, 0x4f, 0x1d, 0x4f, 0x7a, 0x61, 0x6d, 0x4c, 0x78, 0xa2, 0x4c, 0x50, 0x1d, 0xe6, 0x58, 0x9c, - 0x61, 0x08, 0x9c, 0x79, 0x86, 0xf3, 0xfa, 0x73, 0x70, 0x98, 0x07, 0x36, 0x42, 0x48, 0x49, 0x12, - 0x48, 0x08, 0xda, 0x83, 0x34, 0x0d, 0x9b, 0x68, 0x4b, 0x81, 0xa6, 0x32, 0xb4, 0x5b, 0xcf, 0x41, - 0xab, 0x88, 0xc6, 0x21, 0xbc, 0x94, 0x19, 0x92, 0x91, 0xdc, 0xff, 0x4a, 0x90, 0x39, 0xc5, 0x20, - 0x68, 0x1f, 0x22, 0x41, 0x24, 0x4c, 0x89, 0x3d, 0xc2, 0xa2, 0xe0, 0xab, 0xb0, 0x49, 0xc4, 0x32, - 0x51, 0x1b, 0x12, 0x54, 0xa7, 0x6d, 0x8f, 0x86, 0xd9, 0x11, 0x06, 0x5e, 0x1f, 0x0e, 0xf2, 0xca, - 0x1e, 0x13, 0x5e, 0x79, 0x08, 0x85, 0x83, 0xd7, 0x4c, 0x94, 0x87, 0xa4, 0xe7, 0xe8, 0xf8, 0xb1, - 0x45, 0x3c, 0xcb, 0x6e, 0xb3, 0xe0, 0x40, 0xd1, 0xc0, 0x73, 0xaa, 0x42, 0x92, 0xfb, 0xf3, 0x08, - 0xa0, 0xd3, 0xa6, 0x8a, 0xfe, 0x45, 0x82, 0x97, 0xfc, 0x40, 0xc0, 0x71, 0xad, 0xb6, 0x65, 0x1b, - 0x9d, 0xb1, 0x88, 0x40, 0x62, 0xbb, 0xfd, 0xc9, 0x55, 0xf8, 0x40, 0x44, 0x09, 0xbb, 0x02, 0x7e, - 0x32, 0x5a, 0x78, 0x89, 0xba, 0x51, 0x1e, 0x2d, 0x9c, 0x6a, 0x72, 0x5f, 0xcb, 0xf6, 0x9f, 0xd3, - 0x39, 0xb7, 0x05, 0x2f, 0x9f, 0x0b, 0x7c, 0x19, 0x07, 0x96, 0xfb, 0x91, 0x04, 0x37, 0x9e, 0xe3, - 0x56, 0xc2, 0x38, 0x29, 0x8e, 0xf3, 0x61, 0x18, 0x27, 0xb9, 0xfa, 0x6b, 0x57, 0x70, 0x5d, 0xe1, - 0x49, 0x7c, 0x13, 0x62, 0xdc, 0x2c, 0xd1, 0xf5, 0x40, 0x1b, 0xe5, 0x52, 0x8c, 0x6b, 0x23, 0xd3, - 0x27, 0x04, 0x32, 0x63, 0x69, 0x11, 0xb7, 0xd3, 0xef, 0xba, 0xac, 0x48, 0x6a, 0xa4, 0xf0, 0x36, - 0xcc, 0x8b, 0xa1, 0x7c, 0x17, 0x8c, 0x5e, 0x06, 0x38, 0xb2, 0xda, 0x47, 0xfa, 0x23, 0xc3, 0xc3, - 0xae, 0xb8, 0x54, 0x25, 0xa8, 0xe4, 0x23, 0x2a, 0x28, 0x7c, 0xa6, 0x40, 0xaa, 0xd6, 0xed, 0x39, - 0xae, 0xe7, 0x3b, 0xe8, 0x7b, 0x10, 0xe3, 0x8c, 0x29, 0x4e, 0xbd, 0x78, 0xce, 0xba, 0xc6, 0x7a, - 0x72, 0xd7, 0x24, 0xc8, 0x44, 0x60, 0x04, 0x9e, 0x33, 0x72, 0xa6, 0xe7, 0x7c, 0x1f, 0x62, 0xfc, - 0x5e, 0x2b, 0xee, 0x65, 0xf9, 0xd0, 0x58, 0xfe, 0xad, 0xb2, 0xb6, 0xbb, 0x6e, 0x75, 0xf0, 0x3a, - 0x6b, 0xe6, 0x83, 0xf3, 0x4e, 0xe8, 0xeb, 0xa0, 0x10, 0xe2, 0xe9, 0xc4, 0xfa, 0x21, 0x77, 0xbc, - 0x51, 0x7e, 0x39, 0x6b, 0x34, 0xf6, 0x1b, 0xd6, 0x0f, 0xb1, 0x16, 0x27, 0xc4, 0xa3, 0x1f, 0x28, - 0x07, 0xca, 0x23, 0xa3, 0xd3, 0x61, 0x0e, 0x7a, 0x96, 0x5d, 0x46, 0x83, 0xf2, 0xb8, 0x71, 0xc6, - 0xbe, 0x5a, 0xe3, 0x14, 0xbe, 0xb6, 0x67, 0x78, 0x47, 0x2c, 0xe8, 0x4c, 0x68, 0xc0, 0x45, 0x7b, - 0x86, 0x77, 0x84, 0xb2, 0x10, 0x27, 0x06, 0x75, 0x7b, 0x24, 0xab, 0x2c, 0x47, 0x6f, 0xce, 0x69, - 0x7e, 0x11, 0x2d, 0x01, 0x73, 0xda, 0xbc, 0xc8, 0xfc, 0x5f, 0x54, 0x0b, 0x49, 0xd8, 0x3e, 0x1c, - 0x5b, 0x3d, 0xfd, 0xf0, 0x98, 0x70, 0xa7, 0x27, 0xf6, 0xe1, 0xd8, 0xea, 0xad, 0x6f, 0x11, 0x2d, - 0x4e, 0x2b, 0xd7, 0x8f, 0x09, 0xbd, 0x19, 0x58, 0x76, 0x1b, 0x13, 0x4f, 0x37, 0x2d, 0x17, 0xb7, - 0xbc, 0xce, 0x09, 0xf3, 0x7a, 0x8a, 0x96, 0xe6, 0xe2, 0x8a, 0x90, 0x52, 0x6f, 0x3a, 0xe9, 0xa1, - 0x99, 0xb7, 0x52, 0xb4, 0xf9, 0x09, 0x07, 0x7d, 0xa6, 0xe3, 0x4d, 0x9d, 0xed, 0x78, 0x2f, 0x08, - 0xc3, 0xaf, 0xff, 0x3f, 0x84, 0xe1, 0xb9, 0xcf, 0x22, 0x30, 0xcb, 0xf4, 0x16, 0xdd, 0x01, 0x99, - 0x1e, 0x9b, 0xb8, 0xcf, 0x4d, 0x1b, 0x7e, 0xb1, 0x3e, 0xd4, 0x22, 0x6d, 0xa3, 0x8b, 0xb3, 0x88, - 0x1d, 0x2a, 0xfb, 0x46, 0x37, 0x20, 0x4e, 0xf0, 0x03, 0xfd, 0xa1, 0xd1, 0xc9, 0x2e, 0xb0, 0x13, - 0x8b, 0x11, 0xfc, 0xe0, 0xbe, 0xd1, 0x41, 0xd7, 0x20, 0x66, 0x11, 0xdd, 0xc6, 0x8f, 0xb2, 0x8b, - 0x6c, 0x9f, 0x66, 0x2d, 0xb2, 0x83, 0x1f, 0x31, 0xf2, 0x36, 0xdc, 0x36, 0xf6, 0xf4, 0x96, 0xd3, - 0x21, 0xd9, 0x6b, 0xd4, 0x60, 0x68, 0x74, 0x47, 0x45, 0x65, 0xa7, 0x43, 0xd0, 0xaf, 0x40, 0xe2, - 0x91, 0x41, 0x74, 0xdc, 0xed, 0x79, 0x27, 0x6c, 0xb3, 0x14, 0xaa, 0xc6, 0xa4, 0x4a, 0xcb, 0x75, - 0x59, 0x89, 0xa8, 0xd1, 0xba, 0xac, 0x44, 0x55, 0xb9, 0x2e, 0x2b, 0xb2, 0x3a, 0x5b, 0x97, 0x95, - 0x59, 0x35, 0x56, 0x97, 0x95, 0x98, 0x1a, 0xaf, 0xcb, 0x4a, 0x5c, 0x55, 0xea, 0xb2, 0xa2, 0xa8, - 0x89, 0xba, 0xac, 0x24, 0x54, 0xa8, 0xcb, 0x0a, 0xa8, 0xc9, 0xba, 0xac, 0x24, 0xd5, 0xb9, 0xba, - 0xac, 0xcc, 0xa9, 0xa9, 0xba, 0xac, 0xa4, 0xd4, 0x74, 0x5d, 0x56, 0xd2, 0xea, 0x7c, 0x5d, 0x56, - 0xe6, 0x55, 0xb5, 0x2e, 0x2b, 0xaa, 0x9a, 0xa9, 0xcb, 0x4a, 0x46, 0x45, 0x85, 0xff, 0x92, 0x20, - 0xcd, 0xad, 0x3d, 0x60, 0x96, 0x37, 0x20, 0xc3, 0xf4, 0xcf, 0xb2, 0xdb, 0x7a, 0x4f, 0x08, 0x19, - 0x67, 0x44, 0x34, 0xd5, 0xaf, 0x08, 0x1a, 0xbf, 0x02, 0x29, 0x17, 0x1b, 0xe6, 0xa8, 0x61, 0x84, - 0x35, 0x9c, 0xa3, 0xc2, 0xa0, 0xd1, 0x6b, 0x90, 0x66, 0x74, 0x38, 0x6a, 0x15, 0x65, 0xad, 0x52, - 0x4c, 0x1a, 0x34, 0x2b, 0x41, 0x8a, 0xf4, 0x0c, 0x7b, 0xd4, 0x4a, 0x66, 0x44, 0x75, 0xe3, 0x0c, - 0xf2, 0x68, 0xf4, 0x0c, 0x5b, 0x90, 0xc6, 0x1c, 0xed, 0x13, 0xa6, 0x45, 0x17, 0x93, 0x7e, 0x17, - 0xeb, 0x3d, 0x87, 0x87, 0xdc, 0x51, 0x2d, 0xc1, 0x25, 0x7b, 0x0e, 0x29, 0xfc, 0x16, 0xdc, 0xa0, - 0x91, 0x0d, 0x8f, 0x4a, 0xca, 0x47, 0x86, 0xdd, 0x0e, 0x2e, 0x30, 0x06, 0xc4, 0x59, 0x74, 0x14, - 0x04, 0x0a, 0x9b, 0xc3, 0x41, 0x3e, 0x46, 0x5b, 0x5f, 0x99, 0x2c, 0x62, 0x14, 0xb8, 0x66, 0x16, - 0x72, 0x90, 0x9d, 0x1c, 0x3d, 0xb8, 0x52, 0x69, 0xec, 0x46, 0xd5, 0xef, 0x62, 0xba, 0xb4, 0x7b, - 0x16, 0xf1, 0xd0, 0xf7, 0x60, 0x4e, 0x2c, 0x85, 0xae, 0xd0, 0xa7, 0xed, 0x0b, 0x76, 0x23, 0xe9, - 0x06, 0x20, 0xa4, 0xf0, 0x8f, 0x12, 0x2c, 0x54, 0x5c, 0xa7, 0xd7, 0xc3, 0xa6, 0xd0, 0x6f, 0xbe, - 0x54, 0x5f, 0xad, 0xa5, 0x90, 0x5a, 0xef, 0x40, 0xa4, 0x56, 0x11, 0x51, 0xcc, 0xdd, 0xab, 0x06, - 0x47, 0xb5, 0x0a, 0xfa, 0x0e, 0xc4, 0x68, 0x24, 0xdf, 0x27, 0xcc, 0x05, 0xa4, 0x57, 0x7f, 0xf5, - 0x1c, 0x77, 0xd3, 0x60, 0x0d, 0x35, 0xd1, 0xa1, 0xf0, 0x07, 0x31, 0xb8, 0x16, 0xde, 0xa3, 0x8d, - 0xb2, 0x3f, 0xf1, 0x4f, 0x21, 0x6e, 0xd9, 0x26, 0x7e, 0x1c, 0x38, 0xb1, 0xf7, 0xcf, 0x43, 0x3d, - 0x0b, 0xa2, 0x28, 0xf6, 0xa3, 0x46, 0x61, 0xfc, 0x00, 0x59, 0x60, 0xa2, 0xef, 0x07, 0x2e, 0x92, - 0xdf, 0x5a, 0xef, 0xbc, 0x30, 0x7a, 0x65, 0xc2, 0x5d, 0x8e, 0x79, 0xa3, 0x28, 0x73, 0x69, 0x5f, - 0x8d, 0x37, 0x6a, 0x40, 0xc6, 0xb2, 0x3d, 0xec, 0x76, 0xb0, 0xf1, 0x90, 0x92, 0x31, 0x1d, 0x5e, - 0x5c, 0x5e, 0xa7, 0xa5, 0x3e, 0x35, 0x04, 0xc0, 0x29, 0xf4, 0x53, 0x58, 0x08, 0x83, 0xfa, 0x47, - 0x70, 0xfe, 0x85, 0x96, 0xed, 0xf0, 0x08, 0xd6, 0xbf, 0x37, 0x86, 0x80, 0x6a, 0x1c, 0x27, 0xf7, - 0x27, 0x12, 0xcc, 0x85, 0x8f, 0x05, 0x59, 0xa0, 0xb0, 0x31, 0x7c, 0x5b, 0x8c, 0x96, 0x76, 0xa8, - 0xdf, 0x63, 0x95, 0x6c, 0xaf, 0x3e, 0x78, 0xe1, 0xbd, 0xe2, 0x10, 0xe2, 0xc8, 0x6b, 0x26, 0x25, - 0x5f, 0xd3, 0x75, 0x7a, 0xa3, 0x84, 0x5d, 0x54, 0x53, 0xa8, 0x80, 0xfa, 0x93, 0xdc, 0x6f, 0x43, - 0x22, 0x38, 0xd0, 0xd0, 0x1d, 0x22, 0xfa, 0x25, 0xde, 0x21, 0xce, 0x1b, 0xbf, 0xf0, 0xf3, 0x18, - 0x2c, 0x9c, 0x45, 0x55, 0x1f, 0x83, 0x1a, 0x62, 0x06, 0xbd, 0x63, 0x11, 0x4f, 0x68, 0xec, 0xad, - 0xf3, 0x83, 0xd5, 0x10, 0xbd, 0x88, 0xf3, 0x48, 0xbb, 0xe3, 0xa4, 0xf3, 0x03, 0x48, 0x9b, 0x7c, - 0xc9, 0xba, 0x30, 0x85, 0xe8, 0x85, 0xd1, 0xe2, 0x19, 0x14, 0x23, 0xd0, 0x53, 0x66, 0xa8, 0x8a, - 0xb0, 0xc4, 0xa4, 0x8f, 0x1e, 0xdc, 0xfb, 0x2c, 0x93, 0xe9, 0x67, 0xaa, 0xd4, 0x18, 0x0e, 0xf2, - 0x19, 0x81, 0xe5, 0x5f, 0xf4, 0xae, 0xbc, 0xc7, 0x19, 0x73, 0x02, 0xd0, 0xa4, 0x44, 0x4f, 0xeb, - 0xe9, 0xc0, 0xb3, 0x23, 0xa2, 0xa7, 0x9a, 0x7a, 0x75, 0xa2, 0xa7, 0x9f, 0x35, 0x13, 0xfd, 0xbe, - 0x04, 0x19, 0x9e, 0xfa, 0xe9, 0xf6, 0x3d, 0x83, 0xe7, 0xf3, 0xfc, 0x28, 0xf4, 0xe3, 0xe1, 0x20, - 0x3f, 0xcf, 0x36, 0x64, 0x5b, 0xd4, 0xb1, 0x61, 0x4b, 0x2f, 0x3a, 0xec, 0x08, 0x45, 0x44, 0x66, - 0x81, 0xc0, 0x44, 0x5b, 0x90, 0xe6, 0x21, 0xb5, 0x4e, 0x83, 0x4a, 0xcb, 0xb1, 0x59, 0x78, 0x9a, - 0x2a, 0xbd, 0xfa, 0x6c, 0x90, 0x5f, 0x3e, 0x43, 0xb3, 0x78, 0x34, 0x7e, 0x9f, 0xb7, 0xd5, 0x52, - 0x87, 0xe1, 0x22, 0x6a, 0x41, 0x2a, 0x50, 0x8d, 0x93, 0x9e, 0x88, 0x66, 0xaf, 0xee, 0x2c, 0xe6, - 0x7c, 0x1d, 0xa1, 0x98, 0xa8, 0x0d, 0xf3, 0xfe, 0x20, 0x3c, 0x51, 0x40, 0xb2, 0x89, 0x2f, 0x65, - 0x18, 0x5f, 0xad, 0xf9, 0xaa, 0x89, 0xb8, 0x58, 0x5d, 0x87, 0xc5, 0x33, 0xbd, 0xf1, 0x9f, 0xce, - 0xc2, 0xf5, 0x71, 0x86, 0x0f, 0x22, 0x0c, 0x7d, 0xd2, 0x07, 0x7d, 0x30, 0xb5, 0x97, 0xf0, 0x31, - 0x38, 0x0d, 0xf9, 0xa5, 0x49, 0x2f, 0xf4, 0xe9, 0x84, 0x17, 0x7a, 0x01, 0x7c, 0xa6, 0x5e, 0x13, - 0xf8, 0x02, 0x34, 0xf7, 0xaf, 0x12, 0xa4, 0xc6, 0xc6, 0xff, 0x65, 0xd2, 0xed, 0x5e, 0x10, 0x15, - 0xf0, 0x37, 0xbf, 0xf7, 0x2e, 0xbf, 0xb6, 0xf1, 0x60, 0x21, 0xf7, 0xcf, 0x12, 0xa4, 0xc6, 0x96, - 0xfb, 0x15, 0x11, 0xf5, 0x97, 0x3e, 0xf3, 0xc2, 0xb7, 0x21, 0xc6, 0x25, 0x08, 0x41, 0xfa, 0xa3, - 0xb5, 0xda, 0x7e, 0x6d, 0x67, 0x43, 0x5f, 0xdf, 0xd5, 0xf4, 0x8d, 0xb2, 0x3a, 0x83, 0xe6, 0x40, - 0xa9, 0x54, 0xef, 0x55, 0xa9, 0x50, 0x95, 0x50, 0x12, 0xe2, 0xac, 0x54, 0xad, 0xa8, 0x91, 0x42, - 0x09, 0x54, 0x8e, 0x7d, 0x88, 0x29, 0xb5, 0xd2, 0x8b, 0x04, 0x2a, 0xc2, 0x02, 0x85, 0xc5, 0x5d, - 0x1a, 0x63, 0x50, 0x67, 0xa2, 0x87, 0x22, 0xbc, 0x4c, 0x50, 0x45, 0xdd, 0xca, 0x8e, 0xd1, 0xc5, - 0x85, 0x7f, 0x92, 0x21, 0x33, 0x02, 0xf1, 0x1d, 0x0b, 0xbd, 0x70, 0x5a, 0xf6, 0xb1, 0x3e, 0x7a, - 0x82, 0xe2, 0x17, 0x4e, 0xcb, 0x3e, 0x3e, 0xd0, 0x6a, 0x5a, 0x9c, 0x56, 0x1e, 0xb8, 0x16, 0xaa, - 0x83, 0xec, 0xf4, 0x3c, 0x3f, 0x40, 0x7f, 0xf7, 0x9c, 0xad, 0x38, 0x35, 0x46, 0x71, 0xb7, 0xe7, - 0xf1, 0xd4, 0x8b, 0xc6, 0x30, 0xd0, 0xdf, 0x49, 0x10, 0xe7, 0xb7, 0x21, 0x92, 0x8d, 0x31, 0xbc, - 0xef, 0x5c, 0x0a, 0x8f, 0x6f, 0x80, 0x78, 0x24, 0xf8, 0x88, 0xaa, 0xfa, 0xb3, 0x41, 0x3e, 0x33, - 0xb9, 0x41, 0xe4, 0x8a, 0xaf, 0x07, 0xfe, 0x14, 0x51, 0x9d, 0xe7, 0xad, 0x47, 0x1b, 0xcd, 0x28, - 0x75, 0xca, 0xa7, 0x81, 0xd4, 0xd8, 0x41, 0xe4, 0xda, 0x30, 0x17, 0x9e, 0xfd, 0x19, 0xb9, 0xa8, - 0xb5, 0xf1, 0x5c, 0xd4, 0x1b, 0x53, 0xed, 0x0c, 0xc7, 0x0c, 0xe7, 0x9e, 0xbe, 0x0d, 0x89, 0x60, - 0xdb, 0x2f, 0x93, 0x39, 0xe3, 0x2c, 0x19, 0x5c, 0x42, 0x67, 0xd5, 0x58, 0xe1, 0xef, 0x25, 0x98, - 0xd3, 0x30, 0x71, 0x3a, 0x0f, 0xb1, 0x49, 0xa3, 0x06, 0xf4, 0x0d, 0x90, 0x69, 0x14, 0x22, 0xee, - 0xd7, 0x17, 0x5c, 0x4f, 0x58, 0x53, 0xb4, 0x06, 0x89, 0x20, 0x4b, 0x70, 0x99, 0x57, 0xd2, 0x51, - 0x2f, 0x74, 0x0b, 0xd4, 0xa6, 0xd3, 0xb7, 0x4d, 0xc3, 0x3d, 0xd1, 0x5d, 0x6c, 0xb4, 0x8e, 0xb0, - 0x29, 0xf2, 0xa2, 0xf3, 0xbe, 0x5c, 0xe3, 0xe2, 0xc2, 0x8f, 0x23, 0x80, 0x46, 0x9b, 0x13, 0xa2, - 0x09, 0x1a, 0xfb, 0xb0, 0x75, 0x88, 0x0b, 0x56, 0xe4, 0xcc, 0x4c, 0xf6, 0x44, 0x08, 0x15, 0x2c, - 0xdc, 0x3f, 0x52, 0x37, 0x24, 0x23, 0xe8, 0x8f, 0xcf, 0xcf, 0x85, 0x44, 0x59, 0x2e, 0x84, 0x69, - 0xe9, 0x2f, 0x35, 0x1f, 0x22, 0xdc, 0xdd, 0xff, 0xc8, 0x80, 0xca, 0x2e, 0x36, 0x3c, 0x4c, 0x99, - 0x87, 0x9c, 0x77, 0x1f, 0x2c, 0xc1, 0x2c, 0xbf, 0x3c, 0x44, 0x2e, 0x73, 0x79, 0x10, 0x9b, 0xc2, - 0xbb, 0xa2, 0x5f, 0x87, 0xb9, 0x96, 0xd3, 0xe9, 0x77, 0x6d, 0x9d, 0x3d, 0xea, 0x88, 0x50, 0xf2, - 0x5b, 0xe7, 0x29, 0xf1, 0xa9, 0xc9, 0x15, 0xcb, 0x4e, 0x87, 0x96, 0xfd, 0xfb, 0x2d, 0x07, 0x64, - 0x2d, 0xd0, 0x4b, 0x90, 0x08, 0x0c, 0x8a, 0x05, 0x91, 0x09, 0x6d, 0x24, 0x40, 0xab, 0x30, 0x6b, - 0x10, 0xdd, 0x39, 0x64, 0x51, 0xde, 0x45, 0x1a, 0xa6, 0xc9, 0x06, 0xd9, 0x3d, 0x44, 0xef, 0x40, - 0xea, 0xf0, 0x01, 0x8f, 0x7c, 0x39, 0x81, 0xf2, 0xb7, 0xb6, 0xf9, 0xe1, 0x20, 0x9f, 0x5c, 0xff, - 0x90, 0x2d, 0x96, 0xd2, 0xa7, 0x96, 0x3c, 0x7c, 0x10, 0x14, 0xd0, 0x6d, 0xc8, 0x74, 0x8d, 0xc7, - 0xfa, 0xa1, 0x6b, 0xb4, 0x44, 0xa8, 0xd7, 0xe1, 0xac, 0x20, 0x69, 0xf3, 0x5d, 0xe3, 0xf1, 0xba, - 0x90, 0xd7, 0xcc, 0x0e, 0xce, 0xfd, 0x42, 0x82, 0xb8, 0x58, 0x11, 0xea, 0x01, 0x88, 0xed, 0xb1, - 0x4c, 0x1e, 0x4c, 0xa4, 0x4a, 0x1f, 0x0e, 0x07, 0xf9, 0x44, 0x99, 0x49, 0x6b, 0x15, 0xf2, 0x6c, - 0x90, 0xff, 0xde, 0x8b, 0x92, 0x96, 0x0f, 0xa2, 0x25, 0xf8, 0x20, 0x35, 0x93, 0x65, 0x6b, 0x8e, - 0x0c, 0xa2, 0x1f, 0x59, 0xc4, 0x73, 0xda, 0xae, 0xd1, 0x65, 0x87, 0xab, 0x68, 0x73, 0x47, 0x06, - 0xd9, 0xf4, 0x65, 0x28, 0x47, 0xc3, 0x81, 0x87, 0xfc, 0x4d, 0x8e, 0x9b, 0x54, 0x50, 0x46, 0xab, - 0x70, 0x2d, 0xe8, 0xac, 0xd3, 0x45, 0x37, 0xfb, 0xad, 0x63, 0xcc, 0x3c, 0x01, 0xe5, 0xac, 0x85, - 0xa0, 0x72, 0xdb, 0x78, 0x5c, 0xe2, 0x55, 0x85, 0x6b, 0xb0, 0x10, 0x3a, 0xd6, 0x20, 0xc4, 0xfa, - 0x0b, 0x80, 0xf8, 0x9e, 0x71, 0xd2, 0x71, 0x0c, 0x13, 0x2d, 0x43, 0xd2, 0x7f, 0x55, 0xa3, 0x41, - 0x2a, 0xd7, 0xc3, 0xb0, 0x88, 0x4e, 0xaa, 0x4f, 0xb0, 0xcb, 0xce, 0x84, 0xb3, 0x54, 0x50, 0x16, - 0x4f, 0x89, 0x74, 0x7e, 0x7a, 0xd7, 0x6a, 0xb9, 0x0e, 0x4f, 0x39, 0x44, 0x19, 0xdb, 0x52, 0xe9, - 0x36, 0x13, 0xa2, 0xd7, 0x61, 0xfe, 0xd0, 0xb2, 0x59, 0xca, 0xd2, 0x6f, 0xc7, 0x92, 0xcb, 0x5a, - 0xda, 0x17, 0x8b, 0x86, 0x0f, 0x21, 0x1d, 0x7a, 0x48, 0xa4, 0x67, 0x13, 0x63, 0x67, 0xb3, 0x3b, - 0x1c, 0xe4, 0x53, 0x23, 0x5d, 0xe7, 0xe7, 0x73, 0x15, 0xa7, 0x92, 0x1a, 0x0d, 0x43, 0x4f, 0x67, - 0x11, 0x66, 0xd9, 0x0f, 0xca, 0xf8, 0x2f, 0x06, 0x34, 0x5e, 0x40, 0xef, 0xc2, 0x6c, 0x07, 0x1b, - 0x04, 0x8b, 0x1f, 0x03, 0x2c, 0x9f, 0x63, 0x3d, 0xec, 0xb7, 0x5f, 0x1a, 0x6f, 0x8e, 0x4a, 0x10, - 0xe3, 0x49, 0x68, 0x96, 0x3a, 0x4e, 0xae, 0xde, 0x9c, 0xf6, 0x77, 0x1f, 0x9b, 0x33, 0x9a, 0xe8, - 0x89, 0xaa, 0x10, 0x77, 0xf9, 0xbb, 0x03, 0x4b, 0x28, 0x5f, 0x78, 0xbf, 0x0c, 0x3d, 0x86, 0x6c, - 0xce, 0x68, 0x7e, 0x5f, 0xb4, 0xef, 0x3f, 0x19, 0x72, 0x1a, 0x16, 0x0f, 0xa4, 0xc5, 0x29, 0x23, - 0xa8, 0x11, 0xe0, 0x18, 0x0a, 0x5d, 0xa0, 0xc5, 0x32, 0x97, 0x2c, 0x2f, 0x7d, 0xfe, 0x02, 0xc7, - 0x1e, 0x34, 0xe8, 0x02, 0x79, 0x4f, 0xb4, 0x03, 0xd0, 0x0a, 0x5c, 0x43, 0x36, 0xcd, 0x70, 0xde, - 0xbc, 0x4c, 0xf8, 0xb1, 0x39, 0xa3, 0x85, 0x10, 0xd0, 0x87, 0x90, 0x6c, 0x8d, 0x74, 0x3d, 0x3b, - 0xcf, 0x00, 0xdf, 0xba, 0x14, 0xe1, 0x6d, 0x52, 0x92, 0x1b, 0x49, 0xc7, 0x49, 0x4e, 0x9d, 0x24, - 0xb9, 0x2a, 0xa4, 0x44, 0x2a, 0x80, 0xff, 0x16, 0x31, 0x9b, 0x61, 0x1c, 0x1b, 0xd6, 0x12, 0xff, - 0xd7, 0x8a, 0xc5, 0xaa, 0xdd, 0x72, 0x4c, 0x6c, 0x56, 0x69, 0x59, 0x13, 0xb9, 0x45, 0x56, 0x20, - 0x68, 0x03, 0xd2, 0xad, 0x0e, 0x36, 0xec, 0x7e, 0xcf, 0xc7, 0x41, 0x53, 0xe2, 0xa4, 0x44, 0x3f, - 0x01, 0xb4, 0x03, 0xe8, 0x90, 0x3d, 0x33, 0x86, 0x67, 0xc5, 0x12, 0xe5, 0xd3, 0x80, 0xa9, 0xac, - 0xaf, 0x36, 0x9a, 0x19, 0x7a, 0x15, 0x52, 0xb6, 0x63, 0xb7, 0x0c, 0xbb, 0x85, 0x3b, 0xcc, 0x1d, - 0xf1, 0xdc, 0xfa, 0xb8, 0x10, 0x7d, 0x02, 0x69, 0x32, 0x16, 0x73, 0x67, 0xaf, 0xb1, 0x11, 0xdf, - 0xbe, 0x6c, 0x02, 0x6f, 0x73, 0x46, 0x9b, 0x40, 0x42, 0xbf, 0x01, 0xaa, 0x37, 0x91, 0xb4, 0x65, - 0x59, 0xfa, 0xf3, 0x5f, 0xf8, 0x9f, 0x93, 0x65, 0xde, 0x9c, 0xd1, 0x4e, 0xa1, 0x95, 0x12, 0x10, - 0x37, 0x79, 0x75, 0x90, 0xdc, 0x8f, 0xab, 0x4a, 0xe1, 0x17, 0x31, 0x50, 0x82, 0x68, 0x65, 0x05, - 0x50, 0xe0, 0x5f, 0x46, 0x3f, 0xae, 0xa0, 0x44, 0x19, 0xd9, 0x9c, 0xd1, 0x32, 0x7e, 0xdd, 0xe8, - 0xf7, 0x15, 0xaf, 0xc3, 0x7c, 0xd7, 0x31, 0xad, 0x43, 0x6b, 0xc4, 0x76, 0x3c, 0xbd, 0x94, 0xf6, - 0xc5, 0x82, 0xed, 0xee, 0x8e, 0x3d, 0x24, 0x46, 0xa7, 0xf0, 0x95, 0x9b, 0x33, 0xa1, 0x97, 0x46, - 0xca, 0xbe, 0x6e, 0xdf, 0xb6, 0x2d, 0xbb, 0xad, 0x8b, 0x0b, 0x12, 0xf7, 0xc4, 0x29, 0x21, 0x15, - 0x77, 0x9c, 0xf2, 0x04, 0x1d, 0xdd, 0xba, 0x90, 0x8e, 0xfc, 0xb5, 0x6f, 0x4a, 0x01, 0x1f, 0xad, - 0x4f, 0xf2, 0xd1, 0xed, 0x8b, 0xf9, 0x28, 0x04, 0x13, 0x10, 0xd2, 0xc1, 0x99, 0x84, 0xb4, 0x32, - 0xa5, 0xb6, 0x84, 0x10, 0xc7, 0x19, 0xa9, 0x3c, 0xc1, 0x48, 0xb7, 0x2e, 0x64, 0xa4, 0xf0, 0x1a, - 0x05, 0x25, 0xed, 0x9e, 0x41, 0x49, 0x6f, 0x4d, 0x45, 0x49, 0x21, 0xb0, 0x30, 0x27, 0x69, 0x67, - 0x71, 0x52, 0x71, 0x3a, 0x4e, 0x0a, 0x41, 0x8e, 0x91, 0xd2, 0x0f, 0x4e, 0x19, 0x9c, 0xca, 0x60, - 0xbf, 0x71, 0xe9, 0x5b, 0xf1, 0xa6, 0x74, 0xca, 0xe2, 0x8c, 0x33, 0x2c, 0x2e, 0xc3, 0xe0, 0xdf, - 0xb9, 0x84, 0xc5, 0x85, 0x06, 0x38, 0x6d, 0x72, 0x00, 0x8a, 0xff, 0xca, 0x14, 0x32, 0xbf, 0xc2, - 0x4f, 0x24, 0x88, 0xd6, 0x9d, 0x26, 0x4a, 0x8f, 0x52, 0x08, 0xec, 0xf2, 0xff, 0xc1, 0xa8, 0xb9, - 0x88, 0x87, 0x5f, 0x39, 0x67, 0x26, 0xfe, 0xc8, 0x5a, 0xd0, 0x09, 0x7d, 0x17, 0xe2, 0x3d, 0x1e, - 0xeb, 0x08, 0x0b, 0x2b, 0x9c, 0xd7, 0x9f, 0xb7, 0xd4, 0xfc, 0x2e, 0xb7, 0x6f, 0x85, 0x7f, 0x2c, - 0xbc, 0xed, 0x98, 0x18, 0xa5, 0x01, 0xf6, 0x0c, 0x42, 0x7a, 0x47, 0xae, 0x41, 0xb0, 0x3a, 0x83, - 0xe2, 0x10, 0xdd, 0xda, 0x6e, 0xa8, 0xd2, 0xed, 0xef, 0x87, 0x93, 0x0a, 0x15, 0x6d, 0xad, 0xb6, - 0x53, 0xdb, 0xd9, 0xd0, 0x77, 0xd6, 0xb6, 0xab, 0x0d, 0x75, 0x06, 0x65, 0x61, 0xf1, 0xa3, 0xb5, - 0xda, 0xbe, 0xc8, 0x32, 0xe8, 0xb5, 0x9d, 0xfd, 0xaa, 0x76, 0x7f, 0xed, 0x9e, 0x2a, 0xa1, 0xeb, - 0x80, 0xb4, 0xdd, 0xf2, 0x56, 0xa3, 0x52, 0xd2, 0xcb, 0xbb, 0xdb, 0x7b, 0x6b, 0xe5, 0xfd, 0xda, - 0xee, 0x8e, 0x1a, 0x41, 0x0a, 0xc8, 0x95, 0xdd, 0x9d, 0xaa, 0x0a, 0xb7, 0x7f, 0x12, 0x05, 0x99, - 0xee, 0x31, 0x7a, 0x15, 0x92, 0x07, 0x3b, 0x8d, 0xbd, 0x6a, 0xb9, 0xb6, 0x5e, 0xab, 0x56, 0xd4, - 0x99, 0xdc, 0xc2, 0x93, 0xa7, 0xcb, 0xf3, 0xb4, 0xea, 0xc0, 0x26, 0x3d, 0xdc, 0x62, 0xe4, 0x82, - 0x72, 0x10, 0x2b, 0xad, 0x95, 0xb7, 0x0e, 0xf6, 0x54, 0x29, 0x97, 0x7e, 0xf2, 0x74, 0x19, 0x68, - 0x03, 0x6e, 0xd8, 0xe8, 0x25, 0x88, 0x6b, 0xd5, 0xc6, 0xfe, 0xae, 0x56, 0x55, 0x23, 0xb9, 0xf9, - 0x27, 0x4f, 0x97, 0x93, 0xb4, 0x52, 0xd8, 0x2b, 0x7a, 0x1d, 0x52, 0x8d, 0xf2, 0x66, 0x75, 0x7b, - 0x4d, 0x2f, 0x6f, 0xae, 0xed, 0x6c, 0x54, 0xd5, 0x68, 0x6e, 0xf1, 0xc9, 0xd3, 0x65, 0x75, 0xf2, - 0x80, 0xe9, 0x10, 0xb5, 0xed, 0xbd, 0x5d, 0x6d, 0x5f, 0x95, 0x47, 0x43, 0x70, 0xbb, 0x42, 0x05, - 0x00, 0xde, 0x7b, 0xbd, 0x5a, 0xad, 0xa8, 0xb3, 0x39, 0xf4, 0xe4, 0xe9, 0x72, 0x9a, 0xd6, 0x8f, - 0xcc, 0x05, 0xbd, 0x06, 0x73, 0x65, 0xad, 0xba, 0xb6, 0x5f, 0xd5, 0x1b, 0xfb, 0x6b, 0xfb, 0x0d, - 0x35, 0x36, 0x5a, 0x49, 0xc8, 0x04, 0x50, 0x11, 0x32, 0x6b, 0x07, 0xfb, 0xbb, 0xfa, 0x58, 0xdb, - 0x78, 0xee, 0xc6, 0x93, 0xa7, 0xcb, 0x0b, 0xb4, 0xed, 0x5a, 0xdf, 0x73, 0xc2, 0xed, 0xdf, 0x04, - 0x75, 0x6c, 0xfe, 0xfa, 0x46, 0x59, 0x55, 0x72, 0xd7, 0x9f, 0x3c, 0x5d, 0x46, 0x93, 0x4b, 0xd8, - 0x28, 0xa3, 0x6f, 0xc2, 0xf5, 0xfd, 0x8f, 0xf7, 0xaa, 0x95, 0x6a, 0xa3, 0xac, 0x8f, 0x2f, 0x3b, - 0x91, 0xcb, 0x3e, 0x79, 0xba, 0xbc, 0x48, 0xfb, 0x4c, 0xf6, 0xcb, 0x29, 0x3f, 0xfe, 0xab, 0xa5, - 0x99, 0xbf, 0xfd, 0xeb, 0xa5, 0x99, 0xd2, 0xcd, 0xcf, 0xff, 0x63, 0x69, 0xe6, 0xf3, 0xe1, 0x92, - 0xf4, 0xd3, 0xe1, 0x92, 0xf4, 0xb3, 0xe1, 0x92, 0xf4, 0xef, 0xc3, 0x25, 0xe9, 0x0f, 0xbf, 0x58, - 0x9a, 0xf9, 0xe9, 0x17, 0x4b, 0x33, 0x3f, 0xfb, 0x62, 0x69, 0xe6, 0x93, 0x18, 0x57, 0xab, 0x66, - 0x8c, 0xfd, 0x1f, 0x81, 0x77, 0xfe, 0x2f, 0x00, 0x00, 0xff, 0xff, 0x8d, 0x9c, 0xf9, 0xa1, 0xf2, - 0x30, 0x00, 0x00, +func init() { proto.RegisterFile("jobs/jobspb/jobs.proto", fileDescriptor_jobs_105068742e01a54c) } + +var fileDescriptor_jobs_105068742e01a54c = []byte{ + // 4141 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xc4, 0x5b, 0xcd, 0x6f, 0x1b, 0x49, + 0x76, 0x57, 0x93, 0x2d, 0xb2, 0xf9, 0x28, 0x52, 0xad, 0x92, 0x6c, 0x73, 0x19, 0x8f, 0xa8, 0x70, + 0xc6, 0xeb, 0x8f, 0x99, 0xa1, 0x66, 0xe5, 0xdd, 0xd9, 0x59, 0x67, 0xc7, 0xb3, 0xe2, 0x87, 0x24, + 0x52, 0xd6, 0xc7, 0x34, 0x25, 0xcf, 0xce, 0x2c, 0x26, 0x9d, 0x26, 0xbb, 0x44, 0x75, 0x44, 0x76, + 0xd3, 0x5d, 0x4d, 0xdb, 0x5a, 0x04, 0x49, 0xb0, 0x41, 0x82, 0x85, 0x4f, 0x09, 0xb0, 0xc9, 0x21, + 0x89, 0x83, 0x00, 0xc9, 0x20, 0x39, 0x04, 0x08, 0x10, 0x04, 0x49, 0xfe, 0x84, 0x39, 0xe4, 0xb0, + 0xc7, 0x45, 0x0e, 0xdc, 0x84, 0x73, 0xc9, 0x21, 0x87, 0x45, 0x72, 0x08, 0xe0, 0x53, 0x50, 0x1f, + 0xdd, 0x6c, 0x52, 0xb2, 0x44, 0x59, 0x33, 0x9b, 0x8b, 0xdc, 0xf5, 0xaa, 0xea, 0x57, 0x5f, 0xef, + 0xfd, 0xde, 0xab, 0x57, 0x34, 0x5c, 0xfd, 0x4d, 0xa7, 0x41, 0x96, 0xe9, 0x9f, 0x6e, 0x83, 0xfd, + 0x53, 0xe8, 0xba, 0x8e, 0xe7, 0xa0, 0xaf, 0x35, 0x9d, 0xe6, 0x91, 0xeb, 0x18, 0xcd, 0xc3, 0x02, + 0x79, 0xd4, 0x2e, 0xb0, 0x1a, 0xde, 0x2a, 0x7b, 0x05, 0xbb, 0xae, 0xe3, 0xd2, 0xf6, 0xfc, 0x83, + 0xf7, 0xc8, 0x2e, 0xb4, 0x9c, 0x96, 0xc3, 0x3e, 0x97, 0xe9, 0x97, 0x90, 0x22, 0x86, 0xd1, 0x6d, + 0x2c, 0x9b, 0x86, 0x67, 0x08, 0x59, 0xc6, 0x97, 0x59, 0xce, 0xdb, 0x07, 0x8e, 0xdb, 0x31, 0x3c, + 0x1f, 0xe3, 0x75, 0xf2, 0xa8, 0xbd, 0xdc, 0x34, 0x3c, 0xa3, 0xed, 0xb4, 0x96, 0x4d, 0x4c, 0x9a, + 0xdd, 0xc6, 0x32, 0xf1, 0xdc, 0x5e, 0xd3, 0xeb, 0xb9, 0xd8, 0xf4, 0xbb, 0xf7, 0x3c, 0xab, 0xbd, + 0x7c, 0xd8, 0x6e, 0x2e, 0x7b, 0x56, 0x07, 0x13, 0xcf, 0xe8, 0x74, 0x79, 0x4d, 0xfe, 0x77, 0x60, + 0xfa, 0x01, 0x36, 0x08, 0x46, 0x9f, 0x40, 0xdc, 0x76, 0x4c, 0xac, 0x5b, 0x66, 0x46, 0x5a, 0x92, + 0x6e, 0xa5, 0x8a, 0xab, 0x83, 0x7e, 0x2e, 0xb6, 0xed, 0x98, 0xb8, 0x5a, 0x7e, 0xd1, 0xcf, 0xdd, + 0x6d, 0x59, 0xde, 0x61, 0xaf, 0x51, 0x68, 0x3a, 0x9d, 0xe5, 0x60, 0x9d, 0x66, 0x63, 0xf8, 0xbd, + 0xdc, 0x3d, 0x6a, 0x2d, 0x8b, 0x59, 0x16, 0x78, 0x37, 0x2d, 0x46, 0x11, 0xab, 0x26, 0x5a, 0x80, + 0x69, 0xdc, 0x75, 0x9a, 0x87, 0x99, 0xc8, 0x92, 0x74, 0x2b, 0xaa, 0xf1, 0xc2, 0x3d, 0xf9, 0x3f, + 0xff, 0x32, 0x27, 0xe5, 0x3f, 0x8b, 0xc0, 0xb5, 0xa2, 0xd1, 0x3c, 0xea, 0x75, 0x2b, 0x76, 0xd3, + 0x3d, 0xee, 0x7a, 0x96, 0x63, 0xef, 0xb0, 0xbf, 0x04, 0xa9, 0x10, 0x3d, 0xc2, 0xc7, 0x6c, 0x3e, + 0x33, 0x1a, 0xfd, 0x44, 0xef, 0x83, 0xdc, 0x71, 0x4c, 0xcc, 0x80, 0xd2, 0x2b, 0xb7, 0x0b, 0x2f, + 0xdd, 0xf2, 0xc2, 0x10, 0x6d, 0xcb, 0x31, 0xb1, 0xc6, 0xba, 0xa1, 0x06, 0x28, 0x47, 0x1d, 0xa2, + 0x5b, 0xf6, 0x81, 0x93, 0x89, 0x2e, 0x49, 0xb7, 0x92, 0x2b, 0xf7, 0xce, 0x80, 0x78, 0xc9, 0xb4, + 0x0a, 0x9b, 0x5b, 0xf5, 0xaa, 0x7d, 0xe0, 0x14, 0x93, 0x83, 0x7e, 0x2e, 0x2e, 0x0a, 0x5a, 0xfc, + 0xa8, 0x43, 0xe8, 0x47, 0x76, 0x07, 0x7c, 0x19, 0x9d, 0x7f, 0xcf, 0xb5, 0xd8, 0xfc, 0x13, 0x1a, + 0xfd, 0x44, 0x6f, 0x01, 0xc2, 0x1c, 0x0f, 0x9b, 0x3a, 0x3d, 0x5f, 0x9d, 0x2e, 0x30, 0xc2, 0x16, + 0xa8, 0x06, 0x35, 0x65, 0xc3, 0x33, 0x36, 0xf1, 0x31, 0xdf, 0x21, 0xb1, 0x4f, 0xbf, 0x1b, 0x85, + 0xf4, 0x70, 0x2a, 0x0c, 0x7e, 0x03, 0x62, 0xa4, 0x79, 0x88, 0x3b, 0x98, 0x8d, 0x90, 0x5e, 0x79, + 0x67, 0xa2, 0xed, 0xa0, 0x5d, 0x0b, 0x75, 0xd6, 0x4f, 0x13, 0xfd, 0x11, 0x02, 0x99, 0x18, 0x6d, + 0x4f, 0x4c, 0x84, 0x7d, 0xa3, 0x3f, 0x93, 0x60, 0x69, 0x7c, 0x46, 0xc5, 0xe3, 0xcd, 0xad, 0xfa, + 0x96, 0x41, 0x3c, 0xec, 0x6e, 0xe2, 0xe3, 0x6a, 0x39, 0x13, 0x5d, 0x8a, 0xde, 0x4a, 0xae, 0xec, + 0x4c, 0x3e, 0x70, 0xe5, 0x1c, 0xc4, 0x8a, 0xed, 0xb9, 0xc7, 0xda, 0xb9, 0x03, 0x67, 0xeb, 0x70, + 0x63, 0x22, 0xa8, 0xb0, 0x0e, 0x25, 0xb8, 0x0e, 0x2d, 0xc0, 0xf4, 0x63, 0xa3, 0xdd, 0xc3, 0x62, + 0xb5, 0xbc, 0x70, 0x2f, 0xf2, 0x9e, 0x94, 0xbf, 0x06, 0x31, 0xbe, 0x31, 0x28, 0x05, 0x89, 0xd5, + 0x4a, 0x7d, 0xe5, 0x5b, 0xef, 0xae, 0x97, 0xb6, 0xd4, 0x29, 0x71, 0x04, 0x7f, 0x13, 0x83, 0x14, + 0xd7, 0x89, 0x32, 0xf6, 0x0c, 0xab, 0x4d, 0x50, 0x11, 0x80, 0x78, 0x86, 0xeb, 0xe9, 0xd4, 0xac, + 0xd8, 0x18, 0xc9, 0x95, 0xd7, 0x42, 0x9b, 0x41, 0xcd, 0xae, 0x70, 0xd8, 0x6e, 0x16, 0xf6, 0x7c, + 0xb3, 0x2b, 0xca, 0x9f, 0xf7, 0x73, 0x53, 0x5a, 0x82, 0x75, 0xa3, 0x52, 0x74, 0x1f, 0x14, 0x6c, + 0x9b, 0x1c, 0x21, 0x32, 0x39, 0x42, 0x1c, 0xdb, 0x26, 0xeb, 0xff, 0x35, 0xae, 0x64, 0x54, 0x9d, + 0x13, 0xc5, 0xf8, 0xa0, 0x9f, 0x8b, 0xee, 0x6b, 0x55, 0xae, 0x6d, 0x37, 0x61, 0xb6, 0xc1, 0xe6, + 0xab, 0x77, 0x0c, 0xdb, 0x3a, 0xc0, 0xc4, 0xcb, 0xc8, 0x6c, 0xcd, 0x69, 0x2e, 0xde, 0x12, 0x52, + 0xf4, 0x7b, 0x12, 0xcc, 0xf7, 0x5c, 0x8b, 0xe8, 0x8d, 0x63, 0xbd, 0xed, 0x34, 0x8d, 0xb6, 0xe5, + 0x1d, 0xeb, 0x47, 0x8f, 0x33, 0xd3, 0xec, 0x78, 0xef, 0x9f, 0x6b, 0x23, 0x62, 0x3f, 0x0a, 0xfb, + 0xae, 0x45, 0x8a, 0xc7, 0x0f, 0x04, 0xc2, 0xe6, 0x63, 0x76, 0x04, 0xc5, 0x85, 0x41, 0x3f, 0xa7, + 0xee, 0x6b, 0xd5, 0x70, 0xd5, 0x43, 0x4d, 0xed, 0x8d, 0x35, 0x46, 0x46, 0x60, 0x1c, 0x96, 0x63, + 0xeb, 0x0e, 0xb7, 0xb6, 0x4c, 0x8c, 0xed, 0xc9, 0xca, 0xc5, 0xed, 0x54, 0x9b, 0xc3, 0x27, 0x18, + 0xe5, 0x8f, 0x24, 0xc8, 0x52, 0xe2, 0xc3, 0x4d, 0x6a, 0x80, 0x01, 0x19, 0xea, 0x2e, 0x6e, 0x3a, + 0xae, 0x99, 0x89, 0xd3, 0xdd, 0x29, 0xd6, 0xff, 0x6d, 0x52, 0xbe, 0x63, 0xb4, 0xda, 0xeb, 0x59, + 0x66, 0x61, 0x7f, 0xbf, 0x5a, 0x1e, 0xf4, 0x73, 0x99, 0x5d, 0x1f, 0x3c, 0x38, 0x2f, 0x8d, 0x41, + 0x6b, 0x99, 0xee, 0x4b, 0x6a, 0xd0, 0x7b, 0x90, 0x6e, 0x3a, 0xed, 0x36, 0x6e, 0xb2, 0x65, 0xef, + 0x6b, 0xd5, 0x8c, 0xc2, 0xce, 0x72, 0x6e, 0xd0, 0xcf, 0xa5, 0x4a, 0x41, 0x0d, 0x3d, 0xd5, 0x54, + 0x33, 0x5c, 0x44, 0x1a, 0xcc, 0x86, 0x36, 0x8c, 0xb1, 0x5a, 0x82, 0xed, 0xd6, 0xed, 0x89, 0x0d, + 0x52, 0x4b, 0xe3, 0x91, 0x72, 0xb6, 0x04, 0x57, 0x4e, 0x3d, 0xc5, 0xf3, 0x0c, 0x29, 0x11, 0x36, + 0x24, 0x15, 0xd2, 0xfc, 0x50, 0x76, 0x5d, 0xa7, 0xe5, 0x62, 0x42, 0xf2, 0xff, 0x9b, 0x86, 0xb4, + 0x86, 0x89, 0xe7, 0xb8, 0xd8, 0x37, 0x9e, 0x7f, 0x90, 0x60, 0x9e, 0x3a, 0x2c, 0xd7, 0xea, 0x7a, + 0x8e, 0xab, 0xbb, 0xf8, 0x89, 0x6b, 0x79, 0x98, 0x64, 0x22, 0x4c, 0xe9, 0x56, 0xcf, 0x58, 0xc2, + 0x28, 0x50, 0xa1, 0x1c, 0x80, 0x68, 0x02, 0x83, 0xeb, 0xdd, 0xfd, 0x1f, 0xfd, 0x3c, 0x77, 0x6f, + 0xa2, 0x73, 0x3c, 0xe9, 0x43, 0x0b, 0xd5, 0xb2, 0x86, 0xcc, 0x13, 0xc0, 0xe8, 0x3a, 0xc8, 0x54, + 0x6f, 0x19, 0xf1, 0x25, 0x8a, 0xca, 0xa0, 0x9f, 0x93, 0xa9, 0x66, 0x6b, 0x4c, 0x3a, 0x62, 0xcb, + 0xf2, 0x2b, 0xd8, 0xf2, 0x3a, 0x24, 0x3d, 0xa3, 0xd1, 0xc6, 0x3a, 0x1d, 0x99, 0x08, 0xf3, 0xfb, + 0xfa, 0xd8, 0x4e, 0x90, 0x47, 0xed, 0x86, 0x41, 0x70, 0x61, 0x8f, 0xb6, 0x0c, 0xad, 0x1d, 0x3c, + 0x5f, 0x40, 0xd0, 0x32, 0x24, 0x9d, 0xc7, 0xd8, 0x75, 0x2d, 0x13, 0xeb, 0x66, 0x83, 0xd9, 0x50, + 0xa2, 0x98, 0x1e, 0xf4, 0x73, 0xb0, 0x23, 0xc4, 0xe5, 0xa2, 0x06, 0x7e, 0x93, 0x72, 0x03, 0x79, + 0xb0, 0x20, 0xa8, 0x22, 0xb0, 0x7f, 0xa6, 0x4f, 0x71, 0x36, 0x85, 0xef, 0x4e, 0x7e, 0x18, 0xfc, + 0xdc, 0x7d, 0xe5, 0x61, 0x7e, 0x92, 0x2f, 0x12, 0x35, 0x4e, 0xd4, 0xa0, 0x37, 0x61, 0xae, 0xeb, + 0xe2, 0xae, 0xe1, 0x62, 0xbd, 0xe9, 0x74, 0xba, 0x6d, 0xec, 0x61, 0x93, 0x69, 0xbf, 0xa2, 0xa9, + 0xa2, 0xa2, 0xe4, 0xcb, 0xd1, 0x0d, 0x48, 0x13, 0xcf, 0xf0, 0xa8, 0xfb, 0x26, 0xd8, 0xa5, 0x2d, + 0x13, 0xac, 0x65, 0x8a, 0x49, 0xab, 0x42, 0x88, 0xee, 0xc2, 0x95, 0xe1, 0xb9, 0x11, 0xbd, 0xdb, + 0x6b, 0xb4, 0x2d, 0x72, 0x88, 0xcd, 0x0c, 0xb0, 0xd6, 0x0b, 0xa1, 0xca, 0x5d, 0xbf, 0x0e, 0x1d, + 0x8f, 0xa8, 0x62, 0x93, 0x6e, 0x8c, 0xd1, 0xc2, 0x99, 0xe4, 0x92, 0x74, 0x6b, 0xba, 0xb8, 0xf1, + 0xa2, 0x9f, 0x2b, 0x4f, 0xac, 0x47, 0x04, 0x77, 0x96, 0x3d, 0x17, 0xe3, 0x90, 0x5a, 0x96, 0x04, + 0x5e, 0x58, 0xa3, 0x7c, 0x19, 0xd2, 0x00, 0x86, 0x26, 0x98, 0x99, 0x79, 0x65, 0xb6, 0x0b, 0xa1, + 0xa0, 0x5d, 0x88, 0x7b, 0xd8, 0x36, 0x6c, 0x8f, 0x64, 0x52, 0xec, 0x00, 0xdf, 0x99, 0xfc, 0x00, + 0xf7, 0x58, 0x47, 0x5f, 0x33, 0x05, 0x0c, 0x2a, 0x03, 0x78, 0xc7, 0x5d, 0x5f, 0x31, 0xd3, 0x0c, + 0xf4, 0xc6, 0xcb, 0x14, 0xf3, 0xb8, 0x1b, 0xd6, 0xcb, 0x84, 0x27, 0xca, 0x04, 0xd5, 0x60, 0x86, + 0x45, 0x1c, 0x86, 0xc0, 0x99, 0x65, 0x38, 0x37, 0x5f, 0x82, 0xc3, 0x7c, 0xb1, 0x11, 0x42, 0x4a, + 0x92, 0x40, 0x42, 0xd0, 0x2e, 0xa4, 0x69, 0x00, 0x45, 0x5b, 0x0a, 0x34, 0x95, 0xa1, 0xdd, 0x7e, + 0x09, 0x5a, 0x59, 0x34, 0x0e, 0xe1, 0xa5, 0xcc, 0x90, 0x8c, 0x64, 0xff, 0x47, 0x82, 0xb9, 0x13, + 0x5c, 0x82, 0xf6, 0x20, 0x12, 0xc4, 0xc4, 0x94, 0xe2, 0x23, 0x2c, 0x1e, 0xbe, 0x0c, 0xaf, 0x44, + 0x2c, 0x13, 0xb5, 0x20, 0x41, 0xb5, 0xdb, 0xf6, 0x68, 0xc0, 0x1d, 0x61, 0xe0, 0xb5, 0x41, 0x3f, + 0xa7, 0xec, 0x32, 0xe1, 0xa5, 0x87, 0x50, 0x38, 0x78, 0xd5, 0x44, 0x39, 0x48, 0x7a, 0x8e, 0x8e, + 0x9f, 0x5a, 0xc4, 0xb3, 0xec, 0x16, 0x0b, 0x13, 0x14, 0x0d, 0x3c, 0xa7, 0x22, 0x24, 0xd9, 0x3f, + 0x8f, 0x00, 0x3a, 0x69, 0xb4, 0xe8, 0x5f, 0x24, 0xb8, 0xee, 0x87, 0x04, 0x8e, 0x6b, 0xb5, 0x2c, + 0xdb, 0x68, 0x8f, 0xc4, 0x06, 0x12, 0xdb, 0xed, 0x4f, 0x2e, 0xc3, 0x0c, 0x22, 0x5e, 0xd8, 0x11, + 0xf0, 0xe3, 0x71, 0xc3, 0x75, 0xea, 0x50, 0x79, 0xdc, 0x70, 0xa2, 0xc9, 0x43, 0x2d, 0xd3, 0x7b, + 0x49, 0xe7, 0xec, 0x26, 0xbc, 0x76, 0x26, 0xf0, 0x45, 0x5c, 0x59, 0xf6, 0x47, 0x12, 0x5c, 0x7b, + 0x89, 0x83, 0x09, 0xe3, 0xa4, 0x38, 0xce, 0x87, 0x61, 0x9c, 0xe4, 0xca, 0xaf, 0x5d, 0xc2, 0x89, + 0x85, 0x27, 0xf1, 0x4d, 0x88, 0x71, 0xb3, 0x44, 0x57, 0x03, 0x6d, 0x94, 0x8b, 0x31, 0xae, 0x8d, + 0x4c, 0x9f, 0x10, 0xc8, 0x8c, 0xaf, 0x45, 0x04, 0x4f, 0xbf, 0x6b, 0xb2, 0x22, 0xa9, 0x91, 0xfc, + 0x3b, 0x30, 0x2b, 0x86, 0xf2, 0x9d, 0x31, 0x7a, 0x0d, 0xe0, 0xd0, 0x6a, 0x1d, 0xea, 0x4f, 0x0c, + 0x0f, 0xbb, 0xe2, 0x7a, 0x95, 0xa0, 0x92, 0x8f, 0xa8, 0x20, 0xff, 0x99, 0x02, 0xa9, 0x6a, 0xa7, + 0xeb, 0xb8, 0x9e, 0xef, 0xaa, 0x1f, 0x40, 0x8c, 0x39, 0x17, 0x22, 0x4e, 0xbd, 0x70, 0xc6, 0xba, + 0x46, 0x7a, 0x72, 0x27, 0x25, 0xc8, 0x44, 0x60, 0x04, 0x3e, 0x34, 0x72, 0xaa, 0x0f, 0x7d, 0x1f, + 0x62, 0xfc, 0x86, 0x2b, 0x6e, 0x68, 0xb9, 0xd0, 0x58, 0xfe, 0xfd, 0xb2, 0xba, 0xb3, 0x66, 0xb5, + 0xf1, 0x1a, 0x6b, 0xe6, 0x83, 0xf3, 0x4e, 0xe8, 0xeb, 0xa0, 0x10, 0xe2, 0xe9, 0xc4, 0xfa, 0x21, + 0x77, 0xc1, 0x51, 0x7e, 0x4d, 0xab, 0xd7, 0xf7, 0xea, 0xd6, 0x0f, 0xb1, 0x16, 0x27, 0xc4, 0xa3, + 0x1f, 0x28, 0x0b, 0xca, 0x13, 0xa3, 0xdd, 0x66, 0xae, 0x7a, 0x9a, 0x5d, 0x4b, 0x83, 0xf2, 0xa8, + 0x71, 0xc6, 0xbe, 0x5a, 0xe3, 0x14, 0x5e, 0xb7, 0x6b, 0x78, 0x87, 0x2c, 0xfc, 0x4c, 0x68, 0xc0, + 0x45, 0xbb, 0x86, 0x77, 0x88, 0x32, 0x10, 0x27, 0x06, 0x75, 0x80, 0x24, 0xa3, 0x2c, 0x45, 0x6f, + 0xcd, 0x68, 0x7e, 0x11, 0x2d, 0x02, 0x73, 0xdf, 0xbc, 0xc8, 0x3c, 0x61, 0x54, 0x0b, 0x49, 0xd8, + 0x3e, 0x1c, 0x59, 0x5d, 0xfd, 0xe0, 0x88, 0x70, 0xcf, 0x27, 0xf6, 0xe1, 0xc8, 0xea, 0xae, 0x6d, + 0x12, 0x2d, 0x4e, 0x2b, 0xd7, 0x8e, 0x08, 0xbd, 0x23, 0x58, 0x76, 0x0b, 0x13, 0x4f, 0x37, 0x2d, + 0x17, 0x37, 0xbd, 0xf6, 0x31, 0xf3, 0x7a, 0x8a, 0x96, 0xe6, 0xe2, 0xb2, 0x90, 0xa2, 0xdb, 0xa0, + 0x8e, 0xfb, 0x6a, 0xe6, 0xad, 0x14, 0x6d, 0x76, 0xcc, 0x55, 0xd3, 0xa6, 0xfc, 0xa8, 0x43, 0xde, + 0x37, 0xc5, 0x9b, 0x72, 0xf9, 0xd0, 0xf1, 0x9e, 0x13, 0x90, 0x5f, 0xfd, 0x7f, 0x08, 0xc8, 0xb3, + 0x9f, 0x45, 0x60, 0x9a, 0xe9, 0x2d, 0xba, 0x07, 0x32, 0x3d, 0x36, 0x71, 0xb3, 0x9b, 0x34, 0x10, + 0x63, 0x7d, 0xa8, 0x45, 0xda, 0x46, 0x07, 0x67, 0x10, 0x3b, 0x54, 0xf6, 0x8d, 0xae, 0x41, 0x9c, + 0xe0, 0x47, 0xfa, 0x63, 0xa3, 0x9d, 0x99, 0x67, 0x27, 0x16, 0x23, 0xf8, 0xd1, 0x43, 0xa3, 0x8d, + 0xae, 0x40, 0xcc, 0x22, 0xba, 0x8d, 0x9f, 0x64, 0x16, 0xd8, 0x3e, 0x4d, 0x5b, 0x64, 0x1b, 0x3f, + 0x61, 0xe4, 0x6d, 0xb8, 0x2d, 0xec, 0xe9, 0x4d, 0xa7, 0x4d, 0x32, 0x57, 0xa8, 0xc1, 0xd0, 0x38, + 0x8f, 0x8a, 0x4a, 0x4e, 0x9b, 0xa0, 0x5f, 0x81, 0xc4, 0x13, 0x83, 0xe8, 0xb8, 0xd3, 0xf5, 0x8e, + 0xd9, 0x66, 0x29, 0x54, 0x8d, 0x49, 0x85, 0x96, 0x6b, 0xb2, 0x12, 0x51, 0xa3, 0x35, 0x59, 0x89, + 0xaa, 0x72, 0x4d, 0x56, 0x64, 0x75, 0xba, 0x26, 0x2b, 0xd3, 0x6a, 0xac, 0x26, 0x2b, 0x31, 0x35, + 0x5e, 0x93, 0x95, 0xb8, 0xaa, 0xd4, 0x64, 0x45, 0x51, 0x13, 0x35, 0x59, 0x49, 0xa8, 0x50, 0x93, + 0x15, 0x50, 0x93, 0x35, 0x59, 0x49, 0xaa, 0x33, 0x35, 0x59, 0x99, 0x51, 0x53, 0x35, 0x59, 0x49, + 0xa9, 0xe9, 0x9a, 0xac, 0xa4, 0xd5, 0xd9, 0x9a, 0xac, 0xcc, 0xaa, 0x6a, 0x4d, 0x56, 0x54, 0x75, + 0xae, 0x26, 0x2b, 0x73, 0x2a, 0xca, 0xff, 0x97, 0x04, 0x69, 0x6e, 0xed, 0x01, 0xb3, 0xbc, 0x09, + 0x73, 0x4c, 0xff, 0x2c, 0xbb, 0xa5, 0x77, 0x85, 0x90, 0x71, 0x46, 0x44, 0x53, 0xfd, 0x8a, 0xa0, + 0xf1, 0xeb, 0x90, 0x72, 0xb1, 0x61, 0x0e, 0x1b, 0x46, 0x58, 0xc3, 0x19, 0x2a, 0x0c, 0x1a, 0xdd, + 0x80, 0x34, 0xa3, 0xc3, 0x61, 0xab, 0x28, 0x6b, 0x95, 0x62, 0xd2, 0xa0, 0x59, 0x11, 0x52, 0xa4, + 0x6b, 0xd8, 0xc3, 0x56, 0x32, 0x23, 0xaa, 0x6b, 0xa7, 0x90, 0x47, 0xbd, 0x6b, 0xd8, 0x82, 0x34, + 0x66, 0x68, 0x9f, 0x30, 0x2d, 0xba, 0x98, 0xf4, 0x3a, 0x58, 0xef, 0x3a, 0x3c, 0xf8, 0x8e, 0x6a, + 0x09, 0x2e, 0xd9, 0x75, 0x48, 0xfe, 0xb7, 0xe0, 0x1a, 0x8d, 0x6c, 0x78, 0x54, 0x52, 0x3a, 0x34, + 0xec, 0x56, 0x70, 0x95, 0x31, 0x20, 0xce, 0xa2, 0xa3, 0x20, 0x50, 0xd8, 0x18, 0xf4, 0x73, 0x31, + 0xda, 0xfa, 0xd2, 0x64, 0x11, 0xa3, 0xc0, 0x55, 0x33, 0x9f, 0x85, 0xcc, 0xf8, 0xe8, 0xc1, 0xe5, + 0x4a, 0x63, 0x77, 0xab, 0x5e, 0x07, 0xd3, 0xa5, 0x3d, 0xb0, 0x88, 0x87, 0xbe, 0x07, 0x33, 0x62, + 0x29, 0x74, 0x85, 0x3e, 0x6d, 0x9f, 0xb3, 0x1b, 0x49, 0x37, 0x00, 0x21, 0xf9, 0x7f, 0x94, 0x60, + 0xbe, 0xec, 0x3a, 0xdd, 0x2e, 0x36, 0x85, 0x7e, 0xf3, 0xa5, 0xfa, 0x6a, 0x2d, 0x85, 0xd4, 0x7a, + 0x1b, 0x22, 0xd5, 0xb2, 0x88, 0x62, 0xee, 0x5f, 0x36, 0x38, 0xaa, 0x96, 0xd1, 0x77, 0x20, 0x46, + 0x63, 0xfa, 0x1e, 0x61, 0x2e, 0x20, 0xbd, 0xf2, 0xab, 0x67, 0xb8, 0x9b, 0x3a, 0x6b, 0xa8, 0x89, + 0x0e, 0xf9, 0x3f, 0x88, 0xc1, 0x95, 0xf0, 0x1e, 0xad, 0x97, 0xfc, 0x89, 0x7f, 0x0a, 0x71, 0xcb, + 0x36, 0xf1, 0xd3, 0xc0, 0x89, 0xbd, 0x7f, 0x16, 0xea, 0x69, 0x10, 0x05, 0xb1, 0x1f, 0x55, 0x0a, + 0xe3, 0x07, 0xc8, 0x02, 0x13, 0x7d, 0x3f, 0x70, 0x91, 0xfc, 0xfe, 0x7a, 0xef, 0x95, 0xd1, 0xcb, + 0x63, 0xee, 0x72, 0xc4, 0x1b, 0x45, 0x99, 0x4b, 0xfb, 0x6a, 0xbc, 0x51, 0x1d, 0xe6, 0x2c, 0xdb, + 0xc3, 0x6e, 0x1b, 0x1b, 0x8f, 0x29, 0x19, 0xd3, 0xe1, 0xc5, 0x35, 0x76, 0x52, 0xea, 0x53, 0x43, + 0x00, 0x9c, 0x42, 0x3f, 0x85, 0xf9, 0x30, 0xa8, 0x7f, 0x04, 0x67, 0x5f, 0x6d, 0xd9, 0x0e, 0x0f, + 0x61, 0xfd, 0x1b, 0x64, 0x08, 0xa8, 0xca, 0x71, 0xb2, 0x7f, 0x22, 0xc1, 0x4c, 0xf8, 0x58, 0x90, + 0x05, 0x0a, 0x1b, 0xc3, 0xb7, 0xc5, 0x68, 0x71, 0x9b, 0xfa, 0x3d, 0x56, 0xc9, 0xf6, 0xea, 0x83, + 0x57, 0xde, 0x2b, 0x0e, 0x21, 0x8e, 0xbc, 0x6a, 0x52, 0xf2, 0x35, 0x5d, 0xa7, 0x3b, 0x4c, 0xdd, + 0x45, 0x35, 0x85, 0x0a, 0xa8, 0x3f, 0xc9, 0xfe, 0x36, 0x24, 0x82, 0x03, 0x0d, 0xdd, 0x21, 0xa2, + 0x5f, 0xe2, 0x1d, 0xe2, 0xac, 0xf1, 0xf3, 0x3f, 0x8f, 0xc1, 0xfc, 0x69, 0x54, 0xf5, 0x31, 0xa8, + 0x21, 0x66, 0xd0, 0xdb, 0x16, 0xf1, 0x84, 0xc6, 0xde, 0x3e, 0x3b, 0x58, 0x0d, 0xd1, 0x8b, 0x38, + 0x8f, 0xb4, 0x3b, 0x4a, 0x3a, 0x3f, 0x80, 0xb4, 0xc9, 0x97, 0xac, 0x0b, 0x53, 0x88, 0x9e, 0x1b, + 0x2d, 0x9e, 0x42, 0x31, 0x02, 0x3d, 0x65, 0x86, 0xaa, 0x08, 0x4b, 0x51, 0xfa, 0xe8, 0xc1, 0xbd, + 0xcf, 0x32, 0x99, 0x7e, 0xa6, 0x8a, 0xf5, 0x41, 0x3f, 0x37, 0x27, 0xb0, 0xfc, 0x8b, 0xde, 0xa5, + 0xf7, 0x78, 0xce, 0x1c, 0x03, 0x34, 0x29, 0xd1, 0xd3, 0x7a, 0x3a, 0xf0, 0xf4, 0x90, 0xe8, 0xa9, + 0xa6, 0x5e, 0x9e, 0xe8, 0xe9, 0x67, 0xd5, 0x44, 0xbf, 0x2f, 0xc1, 0x1c, 0x4f, 0x02, 0x75, 0x7a, + 0x9e, 0xc1, 0x33, 0x7b, 0x7e, 0x14, 0xfa, 0xf1, 0xa0, 0x9f, 0x9b, 0x65, 0x1b, 0xb2, 0x25, 0xea, + 0xd8, 0xb0, 0xc5, 0x57, 0x1d, 0x76, 0x88, 0x22, 0x22, 0xb3, 0x40, 0x60, 0xa2, 0x4d, 0x48, 0xf3, + 0x90, 0x5a, 0xa7, 0x41, 0xa5, 0xe5, 0xd8, 0x2c, 0x3c, 0x4d, 0x15, 0xdf, 0x78, 0xd1, 0xcf, 0x2d, + 0x9d, 0xa2, 0x59, 0x3c, 0x1a, 0x7f, 0xc8, 0xdb, 0x6a, 0xa9, 0x83, 0x70, 0x11, 0x35, 0x21, 0x15, + 0xa8, 0xc6, 0x71, 0x57, 0x44, 0xb3, 0x97, 0x77, 0x16, 0x33, 0xbe, 0x8e, 0x50, 0x4c, 0xd4, 0x82, + 0x59, 0x7f, 0x10, 0x9e, 0x28, 0x20, 0x99, 0xc4, 0x97, 0x32, 0x8c, 0xaf, 0xd6, 0x7c, 0xd5, 0x44, + 0x5c, 0xac, 0xae, 0xc2, 0xc2, 0xa9, 0xde, 0xf8, 0x4f, 0xa7, 0xe1, 0xea, 0x28, 0xc3, 0x07, 0x11, + 0x86, 0x3e, 0xee, 0x83, 0x3e, 0x98, 0xd8, 0x4b, 0xf8, 0x18, 0x9c, 0x86, 0xfc, 0xd2, 0xb8, 0x17, + 0xfa, 0x74, 0xcc, 0x0b, 0xbd, 0x02, 0x3e, 0x53, 0xaf, 0x31, 0x7c, 0x01, 0x9a, 0xfd, 0x57, 0x09, + 0x52, 0x23, 0xe3, 0xff, 0x32, 0xe9, 0x76, 0x37, 0x88, 0x0a, 0xf8, 0xeb, 0xdf, 0x7b, 0x17, 0x5f, + 0xdb, 0x68, 0xb0, 0x90, 0xfd, 0x67, 0x09, 0x52, 0x23, 0xcb, 0xfd, 0x8a, 0x88, 0xfa, 0x4b, 0x9f, + 0x79, 0xfe, 0xdb, 0x10, 0xe3, 0x12, 0x84, 0x20, 0xfd, 0xd1, 0x6a, 0x75, 0xaf, 0xba, 0xbd, 0xae, + 0xaf, 0xed, 0x68, 0xfa, 0x7a, 0x49, 0x9d, 0x42, 0x33, 0xa0, 0x94, 0x2b, 0x0f, 0x2a, 0x54, 0xa8, + 0x4a, 0x28, 0x09, 0x71, 0x56, 0xaa, 0x94, 0xd5, 0x48, 0xbe, 0x08, 0x2a, 0xc7, 0x3e, 0xc0, 0x94, + 0x5a, 0xe9, 0x45, 0x02, 0x15, 0x60, 0x9e, 0xc2, 0xe2, 0x0e, 0x8d, 0x31, 0xa8, 0x33, 0xd1, 0x43, + 0x11, 0xde, 0x5c, 0x50, 0x45, 0xdd, 0xca, 0xb6, 0xd1, 0xc1, 0xf9, 0x7f, 0x92, 0x61, 0x6e, 0x08, + 0xe2, 0x3b, 0x16, 0x7a, 0xe1, 0xb4, 0xec, 0x23, 0x7d, 0xf8, 0x18, 0xc5, 0x2f, 0x9c, 0x96, 0x7d, + 0xb4, 0xaf, 0x55, 0xb5, 0x38, 0xad, 0xdc, 0x77, 0x2d, 0x54, 0x03, 0xd9, 0xe9, 0x7a, 0x7e, 0x80, + 0xfe, 0xee, 0x19, 0x5b, 0x71, 0x62, 0x8c, 0xc2, 0x4e, 0xd7, 0xe3, 0xa9, 0x17, 0x8d, 0x61, 0xa0, + 0xbf, 0x93, 0x20, 0xce, 0x6f, 0x43, 0x24, 0x13, 0x63, 0x78, 0xdf, 0xb9, 0x10, 0x1e, 0xdf, 0x00, + 0xf1, 0x5c, 0xf0, 0x11, 0x55, 0xf5, 0x17, 0xfd, 0xdc, 0xdc, 0xf8, 0x06, 0x91, 0x4b, 0xbe, 0x23, + 0xf8, 0x53, 0x44, 0x35, 0x9e, 0xc1, 0x1e, 0x6e, 0x34, 0xa3, 0xd4, 0x09, 0x1f, 0x09, 0x52, 0x23, + 0x07, 0x91, 0x6d, 0xc1, 0x4c, 0x78, 0xf6, 0xa7, 0xe4, 0xa2, 0x56, 0x47, 0x73, 0x51, 0x6f, 0x4e, + 0xb4, 0x33, 0x1c, 0x33, 0x9c, 0x7b, 0xfa, 0x36, 0x24, 0x82, 0x6d, 0xbf, 0x48, 0xe6, 0x8c, 0xb3, + 0x64, 0x70, 0x09, 0x9d, 0x56, 0x63, 0xf9, 0xbf, 0x97, 0x60, 0x46, 0xc3, 0xc4, 0x69, 0x3f, 0xc6, + 0x26, 0x8d, 0x1a, 0xd0, 0x37, 0x40, 0xa6, 0x51, 0x88, 0xb8, 0x5f, 0x9f, 0x73, 0x3d, 0x61, 0x4d, + 0xd1, 0x2a, 0x24, 0x82, 0x2c, 0xc1, 0x45, 0xde, 0x4b, 0x87, 0xbd, 0xd0, 0x6d, 0x50, 0x1b, 0x4e, + 0xcf, 0x36, 0x0d, 0xf7, 0x58, 0x77, 0xb1, 0xd1, 0x3c, 0xc4, 0xa6, 0xc8, 0x8b, 0xce, 0xfa, 0x72, + 0x8d, 0x8b, 0xf3, 0x3f, 0x8e, 0x00, 0x1a, 0x6e, 0x4e, 0x88, 0x26, 0x68, 0xec, 0xc3, 0xd6, 0x21, + 0x2e, 0x58, 0x91, 0x53, 0x33, 0xd9, 0x63, 0x21, 0x54, 0xb0, 0x70, 0xff, 0x48, 0xdd, 0x90, 0x8c, + 0xa0, 0x3f, 0x3e, 0x3b, 0x17, 0x12, 0x65, 0xb9, 0x10, 0xa6, 0xa5, 0xbf, 0xd4, 0x7c, 0x88, 0x70, + 0x77, 0xff, 0x2d, 0x03, 0x2a, 0xb9, 0xd8, 0xf0, 0x30, 0x65, 0x1e, 0x72, 0xd6, 0x7d, 0xb0, 0x08, + 0xd3, 0xfc, 0xf2, 0x10, 0xb9, 0xc8, 0xe5, 0x41, 0x6c, 0x0a, 0xef, 0x8a, 0x7e, 0x1d, 0x66, 0x9a, + 0x4e, 0xbb, 0xd7, 0xb1, 0x75, 0xf6, 0xbc, 0x23, 0x42, 0xc9, 0x6f, 0x9d, 0xa5, 0xc4, 0x27, 0x26, + 0x57, 0x28, 0x39, 0x6d, 0x5a, 0xf6, 0xef, 0xb7, 0x1c, 0x90, 0xb5, 0x40, 0xd7, 0x21, 0x11, 0x18, + 0x14, 0x0b, 0x22, 0x13, 0xda, 0x50, 0x80, 0x56, 0x60, 0xda, 0x20, 0xba, 0x73, 0xc0, 0xa2, 0xbc, + 0xf3, 0x34, 0x4c, 0x93, 0x0d, 0xb2, 0x73, 0x80, 0xee, 0x42, 0xea, 0xe0, 0x11, 0x8f, 0x7c, 0x39, + 0x81, 0xf2, 0x57, 0xb7, 0xd9, 0x41, 0x3f, 0x97, 0x5c, 0xfb, 0x90, 0x2d, 0x96, 0xd2, 0xa7, 0x96, + 0x3c, 0x78, 0x14, 0x14, 0xd0, 0x1d, 0x98, 0xeb, 0x18, 0x4f, 0xf5, 0x03, 0xd7, 0x68, 0x8a, 0x50, + 0xaf, 0xcd, 0x59, 0x41, 0xd2, 0x66, 0x3b, 0xc6, 0xd3, 0x35, 0x21, 0xaf, 0x9a, 0x6d, 0x9c, 0xfd, + 0x85, 0x04, 0x71, 0xb1, 0x22, 0xd4, 0x05, 0x10, 0xdb, 0x63, 0x99, 0x3c, 0x98, 0x48, 0x15, 0x3f, + 0x1c, 0xf4, 0x73, 0x89, 0x12, 0x93, 0x56, 0xcb, 0xe4, 0x45, 0x3f, 0xf7, 0xbd, 0x57, 0x25, 0x2d, + 0x1f, 0x44, 0x4b, 0xf0, 0x41, 0xaa, 0x26, 0xcb, 0xd6, 0x1c, 0x1a, 0x44, 0x3f, 0xb4, 0x88, 0xe7, + 0xb4, 0x5c, 0xa3, 0xc3, 0x0e, 0x57, 0xd1, 0x66, 0x0e, 0x0d, 0xb2, 0xe1, 0xcb, 0x50, 0x96, 0x86, + 0x03, 0x8f, 0xf9, 0xeb, 0x1c, 0x37, 0xa9, 0xa0, 0x8c, 0x56, 0xe0, 0x4a, 0xd0, 0x59, 0xa7, 0x8b, + 0x6e, 0xf4, 0x9a, 0x47, 0x98, 0x79, 0x02, 0xca, 0x59, 0xf3, 0x41, 0xe5, 0x96, 0xf1, 0xb4, 0xc8, + 0xab, 0xf2, 0x57, 0x60, 0x3e, 0x74, 0xac, 0x41, 0x88, 0xf5, 0x17, 0x00, 0xf1, 0x5d, 0xe3, 0xb8, + 0xed, 0x18, 0x26, 0x5a, 0x82, 0xa4, 0xff, 0xaa, 0x46, 0x83, 0x54, 0xae, 0x87, 0x61, 0x11, 0x9d, + 0x54, 0x8f, 0x60, 0x97, 0x9d, 0x09, 0x67, 0xa9, 0xa0, 0x2c, 0x1e, 0x15, 0xe9, 0xfc, 0xf4, 0x8e, + 0xd5, 0x74, 0x1d, 0x9e, 0x72, 0x88, 0x32, 0xb6, 0xa5, 0xd2, 0x2d, 0x26, 0x44, 0x37, 0x61, 0xf6, + 0xc0, 0xb2, 0x59, 0xca, 0xd2, 0x6f, 0xc7, 0x92, 0xcb, 0x5a, 0xda, 0x17, 0x8b, 0x86, 0x8f, 0x21, + 0x1d, 0x7a, 0x48, 0xa4, 0x67, 0x13, 0x63, 0x67, 0xb3, 0x33, 0xe8, 0xe7, 0x52, 0x43, 0x5d, 0xe7, + 0xe7, 0x73, 0x19, 0xa7, 0x92, 0x1a, 0x0e, 0x43, 0x4f, 0x67, 0x01, 0xa6, 0xd9, 0x4f, 0xcb, 0xf8, + 0x6f, 0x07, 0x34, 0x5e, 0x40, 0xef, 0xc2, 0x74, 0x1b, 0x1b, 0x04, 0x8b, 0x9f, 0x05, 0x2c, 0x9d, + 0x61, 0x3d, 0xec, 0x57, 0x60, 0x1a, 0x6f, 0x8e, 0x8a, 0x10, 0xe3, 0x49, 0x68, 0x96, 0x3a, 0x4e, + 0xae, 0xdc, 0x9a, 0xf4, 0x17, 0x20, 0x1b, 0x53, 0x9a, 0xe8, 0x89, 0x2a, 0x10, 0x77, 0xf9, 0xbb, + 0x03, 0x4b, 0x28, 0x9f, 0x7b, 0xbf, 0x0c, 0x3d, 0x86, 0x6c, 0x4c, 0x69, 0x7e, 0x5f, 0xb4, 0xe7, + 0x3f, 0x19, 0x72, 0x1a, 0x16, 0x0f, 0xa4, 0x85, 0x09, 0x23, 0xa8, 0x21, 0xe0, 0x08, 0x0a, 0x5d, + 0xa0, 0xc5, 0x32, 0x97, 0x2c, 0x2f, 0x7d, 0xf6, 0x02, 0x47, 0x1e, 0x34, 0xe8, 0x02, 0x79, 0x4f, + 0xb4, 0x0d, 0xd0, 0x0c, 0x5c, 0x43, 0x26, 0xcd, 0x70, 0xde, 0xba, 0x48, 0xf8, 0xb1, 0x31, 0xa5, + 0x85, 0x10, 0xd0, 0x87, 0x90, 0x6c, 0x0e, 0x75, 0x3d, 0x33, 0xcb, 0x00, 0xdf, 0xbe, 0x10, 0xe1, + 0x6d, 0x50, 0x92, 0x1b, 0x4a, 0x47, 0x49, 0x4e, 0x1d, 0x27, 0xb9, 0x0a, 0xa4, 0x44, 0x2a, 0x80, + 0xff, 0x2a, 0x31, 0x33, 0xc7, 0x38, 0x36, 0xac, 0x25, 0xfe, 0xef, 0x16, 0x0b, 0x15, 0xbb, 0xe9, + 0x98, 0xd8, 0xac, 0xd0, 0xb2, 0x26, 0x72, 0x8b, 0xac, 0x40, 0xd0, 0x3a, 0xa4, 0x9b, 0x6d, 0x6c, + 0xd8, 0xbd, 0xae, 0x8f, 0x83, 0x26, 0xc4, 0x49, 0x89, 0x7e, 0x02, 0x68, 0x1b, 0xd0, 0x01, 0x7b, + 0x66, 0x0c, 0xcf, 0x8a, 0x25, 0xca, 0x27, 0x01, 0x53, 0x59, 0x5f, 0x6d, 0x38, 0x33, 0xf4, 0x06, + 0xa4, 0x6c, 0xc7, 0x6e, 0x1a, 0x76, 0x13, 0xb7, 0x99, 0x3b, 0xe2, 0xb9, 0xf5, 0x51, 0x21, 0xfa, + 0x04, 0xd2, 0x64, 0x24, 0xe6, 0xce, 0x5c, 0x61, 0x23, 0xbe, 0x73, 0xd1, 0x04, 0xde, 0xc6, 0x94, + 0x36, 0x86, 0x84, 0x7e, 0x03, 0x54, 0x6f, 0x2c, 0x69, 0xcb, 0xb2, 0xf4, 0x67, 0xbf, 0xf0, 0xbf, + 0x24, 0xcb, 0xbc, 0x31, 0xa5, 0x9d, 0x40, 0x2b, 0x26, 0x20, 0x6e, 0xf2, 0xea, 0x20, 0xb9, 0x1f, + 0x57, 0x95, 0xfc, 0x2f, 0x62, 0xa0, 0x04, 0xd1, 0xca, 0x32, 0xa0, 0xc0, 0xbf, 0x0c, 0x7f, 0x66, + 0x41, 0x89, 0x32, 0xb2, 0x31, 0xa5, 0xcd, 0xf9, 0x75, 0xc3, 0x5f, 0x5a, 0xdc, 0x84, 0xd9, 0x8e, + 0x63, 0x5a, 0x07, 0xd6, 0x90, 0xed, 0x78, 0x7a, 0x29, 0xed, 0x8b, 0x05, 0xdb, 0xdd, 0x1f, 0x79, + 0x48, 0x8c, 0x4e, 0xe0, 0x2b, 0x37, 0xa6, 0x42, 0x2f, 0x8d, 0x94, 0x7d, 0xdd, 0x9e, 0x6d, 0x5b, + 0x76, 0x4b, 0x17, 0x17, 0x24, 0xee, 0x89, 0x53, 0x42, 0x2a, 0xee, 0x38, 0xa5, 0x31, 0x3a, 0xba, + 0x7d, 0x2e, 0x1d, 0xf9, 0x6b, 0xdf, 0x90, 0x02, 0x3e, 0x5a, 0x1b, 0xe7, 0xa3, 0x3b, 0xe7, 0xf3, + 0x51, 0x08, 0x26, 0x20, 0xa4, 0xfd, 0x53, 0x09, 0x69, 0x79, 0x42, 0x6d, 0x09, 0x21, 0x8e, 0x32, + 0x52, 0x69, 0x8c, 0x91, 0x6e, 0x9f, 0xcb, 0x48, 0xe1, 0x35, 0x0a, 0x4a, 0xda, 0x39, 0x85, 0x92, + 0xde, 0x9e, 0x88, 0x92, 0x42, 0x60, 0x61, 0x4e, 0xd2, 0x4e, 0xe3, 0xa4, 0xc2, 0x64, 0x9c, 0x14, + 0x82, 0x1c, 0x21, 0xa5, 0x1f, 0x9c, 0x30, 0x38, 0x95, 0xc1, 0x7e, 0xe3, 0xc2, 0xb7, 0xe2, 0x0d, + 0xe9, 0x84, 0xc5, 0x19, 0xa7, 0x58, 0xdc, 0x1c, 0x83, 0xbf, 0x7b, 0x01, 0x8b, 0x0b, 0x0d, 0x70, + 0xd2, 0xe4, 0x00, 0x14, 0xff, 0x95, 0x29, 0x64, 0x7e, 0xf9, 0x9f, 0x48, 0x10, 0xad, 0x39, 0x0d, + 0x94, 0x1e, 0xa6, 0x10, 0xd8, 0xe5, 0xff, 0x83, 0x61, 0x73, 0x11, 0x0f, 0xbf, 0x7e, 0xc6, 0x4c, + 0xfc, 0x91, 0xb5, 0xa0, 0x13, 0xfa, 0x2e, 0xc4, 0xbb, 0x3c, 0xd6, 0x11, 0x16, 0x96, 0x3f, 0xab, + 0x3f, 0x6f, 0xa9, 0xf9, 0x5d, 0xee, 0xdc, 0x0e, 0xff, 0x6c, 0x78, 0xcb, 0x31, 0x31, 0x4a, 0x03, + 0xec, 0x1a, 0x84, 0x74, 0x0f, 0x5d, 0x83, 0x60, 0x75, 0x0a, 0xc5, 0x21, 0xba, 0xb9, 0x55, 0x57, + 0xa5, 0x3b, 0xdf, 0x0f, 0x27, 0x15, 0xca, 0xda, 0x6a, 0x75, 0xbb, 0xba, 0xbd, 0xae, 0x6f, 0xaf, + 0x6e, 0x55, 0xea, 0xea, 0x14, 0xca, 0xc0, 0xc2, 0x47, 0xab, 0xd5, 0x3d, 0x91, 0x65, 0xd0, 0xab, + 0xdb, 0x7b, 0x15, 0xed, 0xe1, 0xea, 0x03, 0x55, 0x42, 0x57, 0x01, 0x69, 0x3b, 0xa5, 0xcd, 0x7a, + 0xb9, 0xa8, 0x97, 0x76, 0xb6, 0x76, 0x57, 0x4b, 0x7b, 0xd5, 0x9d, 0x6d, 0x35, 0x82, 0x14, 0x90, + 0xcb, 0x3b, 0xdb, 0x15, 0x15, 0xee, 0xfc, 0x24, 0x0a, 0x32, 0xdd, 0x63, 0xf4, 0x06, 0x24, 0xf7, + 0xb7, 0xeb, 0xbb, 0x95, 0x52, 0x75, 0xad, 0x5a, 0x29, 0xab, 0x53, 0xd9, 0xf9, 0x67, 0xcf, 0x97, + 0x66, 0x69, 0xd5, 0xbe, 0x4d, 0xba, 0xb8, 0xc9, 0xc8, 0x05, 0x65, 0x21, 0x56, 0x5c, 0x2d, 0x6d, + 0xee, 0xef, 0xaa, 0x52, 0x36, 0xfd, 0xec, 0xf9, 0x12, 0xd0, 0x06, 0xdc, 0xb0, 0xd1, 0x75, 0x88, + 0x6b, 0x95, 0xfa, 0xde, 0x8e, 0x56, 0x51, 0x23, 0xd9, 0xd9, 0x67, 0xcf, 0x97, 0x92, 0xb4, 0x52, + 0xd8, 0x2b, 0xba, 0x09, 0xa9, 0x7a, 0x69, 0xa3, 0xb2, 0xb5, 0xaa, 0x97, 0x36, 0x56, 0xb7, 0xd7, + 0x2b, 0x6a, 0x34, 0xbb, 0xf0, 0xec, 0xf9, 0x92, 0x3a, 0x7e, 0xc0, 0x74, 0x88, 0xea, 0xd6, 0xee, + 0x8e, 0xb6, 0xa7, 0xca, 0xc3, 0x21, 0xb8, 0x5d, 0xa1, 0x3c, 0x00, 0xef, 0xbd, 0x56, 0xa9, 0x94, + 0xd5, 0xe9, 0x2c, 0x7a, 0xf6, 0x7c, 0x29, 0x4d, 0xeb, 0x87, 0xe6, 0x82, 0x6e, 0xc0, 0x4c, 0x49, + 0xab, 0xac, 0xee, 0x55, 0xf4, 0xfa, 0xde, 0xea, 0x5e, 0x5d, 0x8d, 0x0d, 0x57, 0x12, 0x32, 0x01, + 0x54, 0x80, 0xb9, 0xd5, 0xfd, 0xbd, 0x1d, 0x7d, 0xa4, 0x6d, 0x3c, 0x7b, 0xed, 0xd9, 0xf3, 0xa5, + 0x79, 0xda, 0x76, 0xb5, 0xe7, 0x39, 0xe1, 0xf6, 0x6f, 0x81, 0x3a, 0x32, 0x7f, 0x7d, 0xbd, 0xa4, + 0x2a, 0xd9, 0xab, 0xcf, 0x9e, 0x2f, 0xa1, 0xf1, 0x25, 0xac, 0x97, 0xd0, 0x37, 0xe1, 0xea, 0xde, + 0xc7, 0xbb, 0x95, 0x72, 0xa5, 0x5e, 0xd2, 0x47, 0x97, 0x9d, 0xc8, 0x66, 0x9e, 0x3d, 0x5f, 0x5a, + 0xa0, 0x7d, 0xc6, 0xfb, 0x65, 0x95, 0x1f, 0xff, 0xd5, 0xe2, 0xd4, 0xdf, 0xfe, 0xf5, 0xe2, 0x54, + 0xf1, 0xd6, 0xe7, 0xff, 0xb1, 0x38, 0xf5, 0xf9, 0x60, 0x51, 0xfa, 0xe9, 0x60, 0x51, 0xfa, 0xd9, + 0x60, 0x51, 0xfa, 0xf7, 0xc1, 0xa2, 0xf4, 0x87, 0x5f, 0x2c, 0x4e, 0xfd, 0xf4, 0x8b, 0xc5, 0xa9, + 0x9f, 0x7d, 0xb1, 0x38, 0xf5, 0x49, 0x8c, 0xab, 0x55, 0x23, 0xc6, 0xfe, 0xb7, 0xc0, 0xdd, 0xff, + 0x0b, 0x00, 0x00, 0xff, 0xff, 0xab, 0x44, 0xa7, 0xca, 0xfc, 0x30, 0x00, 0x00, } diff --git a/pkg/jobs/jobspb/jobs.proto b/pkg/jobs/jobspb/jobs.proto index 786a890e595e..6fa5789c5fc8 100644 --- a/pkg/jobs/jobspb/jobs.proto +++ b/pkg/jobs/jobspb/jobs.proto @@ -164,8 +164,9 @@ message RestoreDetails { // stages have completed via these flags. bool prepare_completed = 8; bool stats_inserted = 9; - bool tables_published = 10; - + // DescriptorsPublished indicates whether or not the descriptors written in + // the job have been transactionally updated after the data was restored. + bool descriptors_published = 10; int32 descriptor_coverage = 11 [ (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/sql/sem/tree.DescriptorCoverage" ];