diff --git a/cloud/kubernetes/multiregion/cockroachdb-statefulset-secure.yaml b/cloud/kubernetes/multiregion/cockroachdb-statefulset-secure.yaml index 4d944be4bced..651e7936969b 100644 --- a/cloud/kubernetes/multiregion/cockroachdb-statefulset-secure.yaml +++ b/cloud/kubernetes/multiregion/cockroachdb-statefulset-secure.yaml @@ -192,7 +192,7 @@ spec: mountPath: /cockroach/cockroach-certs env: - name: COCKROACH_CHANNEL - value: kubernetes-secure + value: kubernetes-multiregion command: - "/bin/bash" - "-ecx" diff --git a/cloud/kubernetes/performance/cockroachdb-daemonset-insecure.yaml b/cloud/kubernetes/performance/cockroachdb-daemonset-insecure.yaml index 3ca2c92115a6..43d5497c6a74 100644 --- a/cloud/kubernetes/performance/cockroachdb-daemonset-insecure.yaml +++ b/cloud/kubernetes/performance/cockroachdb-daemonset-insecure.yaml @@ -122,6 +122,9 @@ spec: volumeMounts: - name: datadir mountPath: /cockroach/cockroach-data + env: + - name: COCKROACH_CHANNEL + value: kubernetes-insecure command: - "/bin/bash" - "-ecx" diff --git a/cloud/kubernetes/performance/cockroachdb-daemonset-secure.yaml b/cloud/kubernetes/performance/cockroachdb-daemonset-secure.yaml index 4b394670a035..723d8f2b6991 100644 --- a/cloud/kubernetes/performance/cockroachdb-daemonset-secure.yaml +++ b/cloud/kubernetes/performance/cockroachdb-daemonset-secure.yaml @@ -240,6 +240,9 @@ spec: mountPath: /cockroach/cockroach-data - name: certs mountPath: /cockroach/cockroach-certs + env: + - name: COCKROACH_CHANNEL + value: kubernetes-secure command: - "/bin/bash" - "-ecx" diff --git a/pkg/ccl/changefeedccl/avro_test.go b/pkg/ccl/changefeedccl/avro_test.go index 88f3ceabfba3..af449a809e07 100644 --- a/pkg/ccl/changefeedccl/avro_test.go +++ b/pkg/ccl/changefeedccl/avro_test.go @@ -221,7 +221,10 @@ func TestAvroSchema(t *testing.T) { datum = tree.MakeDTimestamp(t, time.Microsecond) case sqlbase.ColumnType_DECIMAL: // TODO(dan): Make RandDatum respect Precision and Width instead. - typ.Precision = rng.Int31n(10) + 1 + // TODO(dan): The precision is really meant to be in [1,10], but it + // sure looks like there's an off by one error in the avro library + // that makes this test flake if it picks precision of 1. + typ.Precision = rng.Int31n(10) + 2 typ.Width = rng.Int31n(typ.Precision + 1) coeff := rng.Int63n(int64(math.Pow10(int(typ.Precision)))) datum = &tree.DDecimal{Decimal: *apd.New(coeff, -typ.Width)} diff --git a/pkg/ccl/changefeedccl/changefeed_test.go b/pkg/ccl/changefeedccl/changefeed_test.go index 040432d4aaa2..3cb1c418ec17 100644 --- a/pkg/ccl/changefeedccl/changefeed_test.go +++ b/pkg/ccl/changefeedccl/changefeed_test.go @@ -24,12 +24,9 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/ccl/utilccl" "github.com/cockroachdb/cockroach/pkg/jobs" - "github.com/cockroachdb/cockroach/pkg/keys" - "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/security" "github.com/cockroachdb/cockroach/pkg/server" "github.com/cockroachdb/cockroach/pkg/sql/distsqlrun" - "github.com/cockroachdb/cockroach/pkg/storage" "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" @@ -890,9 +887,6 @@ func TestChangefeedDataTTL(t *testing.T) { // versions. sqlDB.Exec(t, `CREATE TABLE foo (a INT PRIMARY KEY, b STRING)`) - // Set a one second GC time; all historical rows subject to GC ASAP. - sqlDB.Exec(t, `ALTER TABLE foo CONFIGURE ZONE USING gc.ttlseconds = $1`, 1) - counter := 0 upsertRow := func() { counter++ @@ -916,32 +910,87 @@ func TestChangefeedDataTTL(t *testing.T) { upsertRow() upsertRow() - // TODO(mrtracy): Even though the GC TTL on the table is set to 1 second, - // this does not work at 1 or even 2 seconds. Investigate why this is the - // case. - time.Sleep(3 * time.Second) - // Force a GC of the table. This should cause both older versions of the // table to be deleted, with the middle version being lost to the changefeed. - tblID, err := sqlutils.QueryTableID(sqlDB.DB, "d", "foo") - if err != nil { - t.Fatal(err) + forceTableGC(t, f.Server(), sqlDB, "d", "foo") + + // Resume our changefeed normally. + atomic.StoreInt32(&shouldWait, 0) + resume <- struct{}{} + + // Verify that the third call to Next() returns an error (the first is the + // initial row, the second is the first change. The third should detect the + // GC interval mismatch). + _, _, _, _, _, _ = dataExpiredRows.Next(t) + _, _, _, _, _, _ = dataExpiredRows.Next(t) + _, _, _, _, _, _ = dataExpiredRows.Next(t) + if err := dataExpiredRows.Err(); !testutils.IsError(err, `must be after replica GC threshold`) { + t.Errorf(`expected "must be after replica GC threshold" error got: %+v`, err) } + } - tablePrefix := keys.MakeTablePrefix(tblID) - tableStartKey := roachpb.RKey(tablePrefix) - tableSpan := roachpb.RSpan{ - Key: tableStartKey, - EndKey: tableStartKey.PrefixEnd(), + t.Run("sinkless", enterpriseTest(testFn)) + t.Run("enterprise", enterpriseTest(testFn)) +} + +// TestChangefeedSchemaTTL ensures that changefeeds fail with an error in the case +// where the feed has fallen behind the GC TTL of the table's schema. +func TestChangefeedSchemaTTL(t *testing.T) { + defer leaktest.AfterTest(t)() + + testFn := func(t *testing.T, db *gosql.DB, f testfeedFactory) { + // Set a very simple channel-based, wait-and-resume function as the + // BeforeEmitRow hook. + var shouldWait int32 + wait := make(chan struct{}) + resume := make(chan struct{}) + knobs := f.Server().(*server.TestServer).Cfg.TestingKnobs. + DistSQL.(*distsqlrun.TestingKnobs). + Changefeed.(*TestingKnobs) + knobs.BeforeEmitRow = func() error { + if atomic.LoadInt32(&shouldWait) == 0 { + return nil + } + wait <- struct{}{} + <-resume + return nil } - ts := f.Server().(*server.TestServer) - if err := ts.GetStores().(*storage.Stores).VisitStores(func(st *storage.Store) error { - return st.ManuallyEnqueueSpan(context.Background(), "gc", tableSpan, true /* skipShouldQueue */) - }); err != nil { - t.Fatal(err) + sqlDB := sqlutils.MakeSQLRunner(db) + + // Create the data table; it will only contain a single row with multiple + // versions. + sqlDB.Exec(t, `CREATE TABLE foo (a INT PRIMARY KEY, b STRING)`) + + counter := 0 + upsertRow := func() { + counter++ + sqlDB.Exec(t, `UPSERT INTO foo (a, b) VALUES (1, $1)`, fmt.Sprintf("version %d", counter)) } + // Create the initial version of the row and the changefeed itself. The initial + // version is necessary to prevent CREATE CHANGEFEED itself from hanging. + upsertRow() + dataExpiredRows := f.Feed(t, "CREATE CHANGEFEED FOR TABLE foo") + defer dataExpiredRows.Close(t) + + // Set up our emit trap and update the row, which will allow us to "pause" the + // changefeed in order to force a GC. + atomic.StoreInt32(&shouldWait, 1) + upsertRow() + <-wait + + // Upsert two additional versions. One of these will be deleted by the GC + // process before changefeed polling is resumed. + waitForSchemaChange(t, sqlDB, "ALTER TABLE foo ADD COLUMN c STRING") + upsertRow() + waitForSchemaChange(t, sqlDB, "ALTER TABLE foo ADD COLUMN d STRING") + upsertRow() + + // Force a GC of the table. This should cause both older versions of the + // table to be deleted, with the middle version being lost to the changefeed. + forceTableGC(t, f.Server(), sqlDB, "system", "descriptor") + // Resume our changefeed normally. atomic.StoreInt32(&shouldWait, 0) resume <- struct{}{} @@ -952,13 +1001,12 @@ func TestChangefeedDataTTL(t *testing.T) { _, _, _, _, _, _ = dataExpiredRows.Next(t) _, _, _, _, _, _ = dataExpiredRows.Next(t) _, _, _, _, _, _ = dataExpiredRows.Next(t) - if err := dataExpiredRows.Err(); !testutils.IsError(err, `must be after replica GC threshold`) { - t.Errorf(`expected "must be after replica GC threshold" error got: %+v`, err) + if err := dataExpiredRows.Err(); !testutils.IsError(err, `GC threshold`) { + t.Errorf(`expected "GC threshold" error got: %+v`, err) } } - // Due to the minimum 3 second run time (due to needing to wait that long for - // rows to be properly GCed), only run an enterprise test. + t.Run("sinkless", enterpriseTest(testFn)) t.Run("enterprise", enterpriseTest(testFn)) } diff --git a/pkg/ccl/changefeedccl/helpers_test.go b/pkg/ccl/changefeedccl/helpers_test.go index b96fc5e887ca..de701989b6f2 100644 --- a/pkg/ccl/changefeedccl/helpers_test.go +++ b/pkg/ccl/changefeedccl/helpers_test.go @@ -23,9 +23,12 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/internal/client" + "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/sql/distsqlrun" "github.com/cockroachdb/cockroach/pkg/sql/parser" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/apd" "github.com/cockroachdb/cockroach/pkg/security" @@ -525,6 +528,25 @@ func (c *tableFeed) Close(t testing.TB) { c.urlCleanup() } +func waitForSchemaChange( + t testing.TB, sqlDB *sqlutils.SQLRunner, stmt string, arguments ...interface{}, +) { + sqlDB.Exec(t, stmt, arguments...) + row := sqlDB.QueryRow(t, "SELECT job_id FROM [SHOW JOBS] ORDER BY created DESC LIMIT 1") + var jobID string + row.Scan(&jobID) + + testutils.SucceedsSoon(t, func() error { + row := sqlDB.QueryRow(t, "SELECT status FROM [SHOW JOBS] WHERE job_id = $1", jobID) + var status string + row.Scan(&status) + if status != "succeeded" { + return fmt.Errorf("Job %s had status %s, wanted 'succeeded'", jobID, status) + } + return nil + }) +} + func assertPayloads(t testing.TB, f testfeed, expected []string) { t.Helper() @@ -674,3 +696,28 @@ func enterpriseTest(testFn func(*testing.T, *gosql.DB, testfeedFactory)) func(*t testFn(t, db, f) } } + +func forceTableGC( + t testing.TB, + tsi serverutils.TestServerInterface, + sqlDB *sqlutils.SQLRunner, + database, table string, +) { + t.Helper() + tblID, err := sqlutils.QueryTableID(sqlDB.DB, database, table) + if err != nil { + t.Fatal(err) + } + + tblKey := roachpb.Key(keys.MakeTablePrefix(tblID)) + gcr := roachpb.GCRequest{ + RequestHeader: roachpb.RequestHeader{ + Key: tblKey, + EndKey: tblKey.PrefixEnd(), + }, + Threshold: tsi.Clock().Now(), + } + if _, err := client.SendWrapped(context.Background(), tsi.DistSender(), &gcr); err != nil { + t.Fatal(err) + } +} diff --git a/pkg/ccl/partitionccl/drop_test.go b/pkg/ccl/partitionccl/drop_test.go index 802f3032945c..49b81e46f3c9 100644 --- a/pkg/ccl/partitionccl/drop_test.go +++ b/pkg/ccl/partitionccl/drop_test.go @@ -12,19 +12,46 @@ import ( "context" "testing" + "github.com/pkg/errors" + + "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/config" + "github.com/cockroachdb/cockroach/pkg/sql" "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" "github.com/cockroachdb/cockroach/pkg/sql/tests" + "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" "github.com/cockroachdb/cockroach/pkg/util/leaktest" + "github.com/cockroachdb/cockroach/pkg/util/protoutil" ) +func subzoneExists(cfg *config.ZoneConfig, index uint32, partition string) bool { + for _, s := range cfg.Subzones { + if s.IndexID == index && s.PartitionName == partition { + return true + } + } + return false +} + func TestDropIndexWithZoneConfigCCL(t *testing.T) { defer leaktest.AfterTest(t)() const numRows = 100 + asyncNotification := make(chan struct{}) + params, _ := tests.CreateTestServerParams() + params.Knobs = base.TestingKnobs{ + SQLSchemaChanger: &sql.SchemaChangerTestingKnobs{ + AsyncExecNotification: func() error { + <-asyncNotification + return nil + }, + AsyncExecQuickly: true, + }, + } s, sqlDBRaw, kvDB := serverutils.StartServer(t, params) sqlDB := sqlutils.MakeSQLRunner(sqlDBRaw) defer s.Stopper().Stop(context.Background()) @@ -47,30 +74,69 @@ func TestDropIndexWithZoneConfigCCL(t *testing.T) { // Set zone configs on the primary index, secondary index, and one partition // of the secondary index. + ttlYaml := "gc: {ttlseconds: 1}" sqlutils.SetZoneConfig(t, sqlDB, "INDEX t.kv@primary", "") - sqlutils.SetZoneConfig(t, sqlDB, "INDEX t.kv@i", "") - sqlutils.SetZoneConfig(t, sqlDB, "PARTITION p2 OF TABLE t.kv", "") - for _, target := range []string{"t.kv@primary", "t.kv@i", "t.kv.p2"} { - if exists := sqlutils.ZoneConfigExists(t, sqlDB, target); !exists { - t.Fatalf(`zone config for %s does not exist`, target) - } - } + sqlutils.SetZoneConfig(t, sqlDB, "INDEX t.kv@i", ttlYaml) + sqlutils.SetZoneConfig(t, sqlDB, "PARTITION p2 OF TABLE t.kv", ttlYaml) // Drop the index and verify that the zone config for the secondary index and // its partition are removed but the zone config for the primary index // remains. sqlDB.Exec(t, `DROP INDEX t.kv@i`) - tests.CheckKeyCount(t, kvDB, indexSpan, 0) + // All zone configs should still exist. + var buf []byte + cfg := &config.ZoneConfig{} + sqlDB.QueryRow(t, "SELECT config FROM system.zones WHERE id = $1", tableDesc.ID).Scan(&buf) + if err := protoutil.Unmarshal(buf, cfg); err != nil { + t.Fatal(err) + } + + subzones := []struct { + index uint32 + partition string + }{ + {1, ""}, + {3, ""}, + {3, "p2"}, + } + for _, target := range subzones { + if exists := subzoneExists(cfg, target.index, target.partition); !exists { + t.Fatalf(`zone config for %v does not exist`, target) + } + } + // Dropped indexes waiting for GC shouldn't have their zone configs be visible + // using SHOW ZONE CONFIGURATIONS ..., but still need to exist in system.zones. + for _, target := range []string{"t.kv@i", "t.kv.p2"} { + if exists := sqlutils.ZoneConfigExists(t, sqlDB, target); exists { + t.Fatalf(`zone config for %s still exists`, target) + } + } tableDesc = sqlbase.GetTableDescriptor(kvDB, "t", "kv") if _, _, err := tableDesc.FindIndexByName("i"); err == nil { t.Fatalf("table descriptor still contains index after index is dropped") } - if exists := sqlutils.ZoneConfigExists(t, sqlDB, "t.kv@primary"); !exists { + close(asyncNotification) + + // Wait for index drop to complete so zone configs are updated. + testutils.SucceedsSoon(t, func() error { + if kvs, err := kvDB.Scan(context.TODO(), indexSpan.Key, indexSpan.EndKey, 0); err != nil { + return err + } else if l := 0; len(kvs) != l { + return errors.Errorf("expected %d key value pairs, but got %d", l, len(kvs)) + } + return nil + }) + + sqlDB.QueryRow(t, "SELECT config FROM system.zones WHERE id = $1", tableDesc.ID).Scan(&buf) + if err := protoutil.Unmarshal(buf, cfg); err != nil { + t.Fatal(err) + } + if exists := subzoneExists(cfg, 1, ""); !exists { t.Fatal("zone config for primary index removed after dropping secondary index") } - for _, target := range []string{"t.kv@i", "t.kv.p2"} { - if exists := sqlutils.ZoneConfigExists(t, sqlDB, target); exists { - t.Fatalf(`zone config for %s still exists`, target) + for _, target := range subzones[1:] { + if exists := subzoneExists(cfg, target.index, target.partition); exists { + t.Fatalf(`zone config for %v still exists`, target) } } } diff --git a/pkg/sql/backfill.go b/pkg/sql/backfill.go index da786b1ae210..8d2ef303dcb7 100644 --- a/pkg/sql/backfill.go +++ b/pkg/sql/backfill.go @@ -113,9 +113,6 @@ func (sc *SchemaChanger) runBackfill( // mutations. Collect the elements that are part of the mutation. var droppedIndexDescs []sqlbase.IndexDescriptor var addedIndexDescs []sqlbase.IndexDescriptor - // Indexes within the Mutations slice for checkpointing. - mutationSentinel := -1 - var droppedIndexMutationIdx int var tableDesc *sqlbase.TableDescriptor if err := sc.db.Txn(ctx, func(ctx context.Context, txn *client.Txn) error { @@ -135,7 +132,7 @@ func (sc *SchemaChanger) runBackfill( tableDesc.Name, tableDesc.Version, sc.mutationID) needColumnBackfill := false - for i, m := range tableDesc.Mutations { + for _, m := range tableDesc.Mutations { if m.MutationID != sc.mutationID { break } @@ -157,9 +154,8 @@ func (sc *SchemaChanger) runBackfill( case *sqlbase.DescriptorMutation_Column: needColumnBackfill = true case *sqlbase.DescriptorMutation_Index: - droppedIndexDescs = append(droppedIndexDescs, *t.Index) - if droppedIndexMutationIdx == mutationSentinel { - droppedIndexMutationIdx = i + if !sc.canClearRangeForDrop(t.Index) { + droppedIndexDescs = append(droppedIndexDescs, *t.Index) } default: return errors.Errorf("unsupported mutation: %+v", m) @@ -169,18 +165,9 @@ func (sc *SchemaChanger) runBackfill( // First drop indexes, then add/drop columns, and only then add indexes. - // Drop indexes. + // Drop indexes not to be removed by `ClearRange`. if len(droppedIndexDescs) > 0 { - if err := sc.truncateIndexes( - ctx, lease, version, droppedIndexDescs, droppedIndexMutationIdx, - ); err != nil { - return err - } - - // Remove index zone configs. - if err := sc.db.Txn(ctx, func(ctx context.Context, txn *client.Txn) error { - return removeIndexZoneConfigs(ctx, txn, sc.execCfg, tableDesc.ID, droppedIndexDescs) - }); err != nil { + if err := sc.truncateIndexes(ctx, lease, version, droppedIndexDescs); err != nil { return err } } @@ -221,7 +208,6 @@ func (sc *SchemaChanger) truncateIndexes( lease *sqlbase.TableDescriptor_SchemaChangeLease, version sqlbase.DescriptorVersion, dropped []sqlbase.IndexDescriptor, - mutationIdx int, ) error { chunkSize := sc.getChunkSize(indexTruncateChunkSize) if sc.testingKnobs.BackfillChunkSize > 0 { @@ -269,7 +255,12 @@ func (sc *SchemaChanger) truncateIndexes( return err } resume, err = td.deleteIndex( - ctx, &desc, resumeAt, chunkSize, noAutoCommit, false, /* traceKV */ + ctx, + &desc, + resumeAt, + chunkSize, + noAutoCommit, + false, /* traceKV */ ) done = resume.Key == nil return err @@ -277,6 +268,11 @@ func (sc *SchemaChanger) truncateIndexes( return err } } + if err := sc.db.Txn(ctx, func(ctx context.Context, txn *client.Txn) error { + return removeIndexZoneConfigs(ctx, txn, sc.execCfg, sc.tableID, dropped) + }); err != nil { + return err + } } return nil } diff --git a/pkg/sql/crdb_internal.go b/pkg/sql/crdb_internal.go index d4c03dbe5091..7afd2321be43 100644 --- a/pkg/sql/crdb_internal.go +++ b/pkg/sql/crdb_internal.go @@ -1752,6 +1752,9 @@ CREATE TABLE crdb_internal.zones ( for _, s := range subzones { index, err := table.FindIndexByID(sqlbase.IndexID(s.IndexID)) if err != nil { + if err == sqlbase.ErrIndexGCMutationsList { + continue + } return err } if zoneSpecifier != nil { diff --git a/pkg/sql/drop_table.go b/pkg/sql/drop_table.go index c2d3e8508ce6..9554fec4d58a 100644 --- a/pkg/sql/drop_table.go +++ b/pkg/sql/drop_table.go @@ -361,6 +361,7 @@ func (p *planner) initiateDropTable( } // Mark all jobs scheduled for schema changes as successful. + jobIDs := make(map[int64]struct{}) var id sqlbase.MutationID for _, m := range tableDesc.Mutations { if id != m.MutationID { @@ -369,13 +370,20 @@ func (p *planner) initiateDropTable( if err != nil { return err } - job, err := p.ExecCfg().JobRegistry.LoadJobWithTxn(ctx, jobID, p.txn) - if err != nil { - return err - } - if err := job.WithTxn(p.txn).Succeeded(ctx, jobs.NoopFn); err != nil { - return errors.Wrapf(err, "failed to mark job %d as as successful", jobID) - } + jobIDs[jobID] = struct{}{} + } + } + for _, gcm := range tableDesc.GCMutations { + jobIDs[gcm.JobID] = struct{}{} + } + for jobID := range jobIDs { + job, err := p.ExecCfg().JobRegistry.LoadJobWithTxn(ctx, jobID, p.txn) + if err != nil { + return err + } + + if err := job.WithTxn(p.txn).Succeeded(ctx, jobs.NoopFn); err != nil { + return errors.Wrapf(err, "failed to mark job %d as as successful", jobID) } } diff --git a/pkg/sql/drop_test.go b/pkg/sql/drop_test.go index dd0e29577904..5b3414b63336 100644 --- a/pkg/sql/drop_test.go +++ b/pkg/sql/drop_test.go @@ -96,7 +96,17 @@ func addImmediateGCZoneConfig(sqlDB *gosql.DB, id sqlbase.ID) (config.ZoneConfig if err != nil { return cfg, err } - _, err = sqlDB.Exec(`INSERT INTO system.zones VALUES ($1, $2)`, id, buf) + _, err = sqlDB.Exec(`UPSERT INTO system.zones VALUES ($1, $2)`, id, buf) + return cfg, err +} + +func addDefaultZoneConfig(sqlDB *gosql.DB, id sqlbase.ID) (config.ZoneConfig, error) { + cfg := config.DefaultZoneConfig() + buf, err := protoutil.Marshal(&cfg) + if err != nil { + return cfg, err + } + _, err = sqlDB.Exec(`UPSERT INTO system.zones VALUES ($1, $2)`, id, buf) return cfg, err } @@ -434,33 +444,77 @@ func TestDropIndex(t *testing.T) { params.Knobs = base.TestingKnobs{ SQLSchemaChanger: &sql.SchemaChangerTestingKnobs{ BackfillChunkSize: chunkSize, + AsyncExecQuickly: true, }, } s, sqlDB, kvDB := serverutils.StartServer(t, params) defer s.Stopper().Stop(context.TODO()) - numRows := 2*chunkSize + 1 if err := tests.CreateKVTable(sqlDB, "kv", numRows); err != nil { t.Fatal(err) } tableDesc := sqlbase.GetTableDescriptor(kvDB, "t", "kv") - + tests.CheckKeyCount(t, kvDB, tableDesc.TableSpan(), 3*numRows) idx, _, err := tableDesc.FindIndexByName("foo") if err != nil { t.Fatal(err) } indexSpan := tableDesc.IndexSpan(idx.ID) - tests.CheckKeyCount(t, kvDB, indexSpan, numRows) if _, err := sqlDB.Exec(`DROP INDEX t.kv@foo`); err != nil { t.Fatal(err) } - tests.CheckKeyCount(t, kvDB, indexSpan, 0) tableDesc = sqlbase.GetTableDescriptor(kvDB, "t", "kv") if _, _, err := tableDesc.FindIndexByName("foo"); err == nil { t.Fatalf("table descriptor still contains index after index is dropped") } + // Index data hasn't been deleted. + tests.CheckKeyCount(t, kvDB, indexSpan, numRows) + tests.CheckKeyCount(t, kvDB, tableDesc.TableSpan(), 3*numRows) + + sqlRun := sqlutils.MakeSQLRunner(sqlDB) + if err := jobutils.VerifyRunningSystemJob(t, sqlRun, 1, jobspb.TypeSchemaChange, jobs.RunningStatusWaitingGC, jobs.Record{ + Username: security.RootUser, + Description: `DROP INDEX t.public.kv@foo`, + DescriptorIDs: sqlbase.IDs{ + tableDesc.ID, + }, + }); err != nil { + t.Fatal(err) + } + + if _, err := sqlDB.Exec(`CREATE INDEX foo on t.kv (v);`); err != nil { + t.Fatal(err) + } + + tableDesc = sqlbase.GetTableDescriptor(kvDB, "t", "kv") + newIdx, _, err := tableDesc.FindIndexByName("foo") + if err != nil { + t.Fatal(err) + } + newIdxSpan := tableDesc.IndexSpan(newIdx.ID) + tests.CheckKeyCount(t, kvDB, newIdxSpan, numRows) + tests.CheckKeyCount(t, kvDB, tableDesc.TableSpan(), 4*numRows) + + // Add a zone config for the table. + if _, err := addImmediateGCZoneConfig(sqlDB, tableDesc.ID); err != nil { + t.Fatal(err) + } + + testutils.SucceedsSoon(t, func() error { + return jobutils.VerifySystemJob(t, sqlRun, 1, jobspb.TypeSchemaChange, jobs.StatusSucceeded, jobs.Record{ + Username: security.RootUser, + Description: `DROP INDEX t.public.kv@foo`, + DescriptorIDs: sqlbase.IDs{ + tableDesc.ID, + }, + }) + }) + + tests.CheckKeyCount(t, kvDB, newIdxSpan, numRows) + tests.CheckKeyCount(t, kvDB, indexSpan, 0) + tests.CheckKeyCount(t, kvDB, tableDesc.TableSpan(), 3*numRows) } func TestDropIndexWithZoneConfigOSS(t *testing.T) { @@ -531,7 +585,7 @@ func TestDropIndexWithZoneConfigOSS(t *testing.T) { if exists := sqlutils.ZoneConfigExists(t, sqlDB, "t.kv@foo"); exists { t.Fatal("zone config for index still exists after dropping index") } - tests.CheckKeyCount(t, kvDB, indexSpan, 0) + tests.CheckKeyCount(t, kvDB, indexSpan, numRows) tableDesc = sqlbase.GetTableDescriptor(kvDB, "t", "kv") if _, _, err := tableDesc.FindIndexByName("foo"); err == nil { t.Fatalf("table descriptor still contains index after index is dropped") @@ -544,6 +598,7 @@ func TestDropIndexInterleaved(t *testing.T) { params, _ := tests.CreateTestServerParams() params.Knobs = base.TestingKnobs{ SQLSchemaChanger: &sql.SchemaChangerTestingKnobs{ + AsyncExecQuickly: true, BackfillChunkSize: chunkSize, }, } @@ -597,14 +652,10 @@ func TestDropTable(t *testing.T) { } // Add a zone config for the table. - cfg := config.DefaultZoneConfig() - buf, err := protoutil.Marshal(&cfg) + cfg, err := addDefaultZoneConfig(sqlDB, tableDesc.ID) if err != nil { t.Fatal(err) } - if _, err := sqlDB.Exec(`INSERT INTO system.zones VALUES ($1, $2)`, tableDesc.ID, buf); err != nil { - t.Fatal(err) - } if err := zoneExists(sqlDB, &cfg, tableDesc.ID); err != nil { t.Fatal(err) diff --git a/pkg/sql/lease_test.go b/pkg/sql/lease_test.go index d9e183b26cff..fc592e03436e 100644 --- a/pkg/sql/lease_test.go +++ b/pkg/sql/lease_test.go @@ -29,14 +29,19 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/config" "github.com/cockroachdb/cockroach/pkg/internal/client" + "github.com/cockroachdb/cockroach/pkg/jobs" + "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/security" "github.com/cockroachdb/cockroach/pkg/server" "github.com/cockroachdb/cockroach/pkg/sql" "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" "github.com/cockroachdb/cockroach/pkg/sql/tests" "github.com/cockroachdb/cockroach/pkg/testutils" + "github.com/cockroachdb/cockroach/pkg/testutils/jobutils" "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" + "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" @@ -855,7 +860,12 @@ CREATE TABLE t.foo (v INT); func TestTxnObeysTableModificationTime(t *testing.T) { defer leaktest.AfterTest(t)() params, _ := tests.CreateTestServerParams() - s, sqlDB, _ := serverutils.StartServer(t, params) + params.Knobs = base.TestingKnobs{ + SQLSchemaChanger: &sql.SchemaChangerTestingKnobs{ + AsyncExecQuickly: true, + }, + } + s, sqlDB, kvDB := serverutils.StartServer(t, params) defer s.Stopper().Stop(context.TODO()) if _, err := sqlDB.Exec(` @@ -865,6 +875,10 @@ INSERT INTO t.kv VALUES ('a', 'b'); `); err != nil { t.Fatal(err) } + tableDesc := sqlbase.GetTableDescriptor(kvDB, "t", "kv") + if _, err := addImmediateGCZoneConfig(sqlDB, tableDesc.ID); err != nil { + t.Fatal(err) + } // A read-write transaction that uses the old version of the descriptor. txReadWrite, err := sqlDB.Begin() @@ -959,11 +973,11 @@ INSERT INTO t.kv VALUES ('a', 'b'); // Test the deadline exceeded error with a CREATE/DROP INDEX. schemaChanges := []struct{ sql string }{ - {`CREATE INDEX foo on t.kv (v)`}, - {`DROP INDEX t.kv@foo`}, + {`CREATE INDEX foo ON t.public.kv (v)`}, + {`DROP INDEX t.public.kv@foo`}, } - for _, change := range schemaChanges { + for i, change := range schemaChanges { txWrite, err := sqlDB.Begin() if err != nil { @@ -978,6 +992,15 @@ INSERT INTO t.kv VALUES ('a', 'b'); if _, err := sqlDB.Exec(change.sql); err != nil { t.Fatal(err) } + testutils.SucceedsSoon(t, func() error { + return jobutils.VerifySystemJob(t, sqlutils.MakeSQLRunner(sqlDB), i+1, jobspb.TypeSchemaChange, jobs.StatusSucceeded, jobs.Record{ + Username: security.RootUser, + Description: change.sql, + DescriptorIDs: sqlbase.IDs{ + tableDesc.ID, + }, + }) + }) // This INSERT will cause the transaction to be pushed transparently, // which will be detected when we attempt to Commit() below only because diff --git a/pkg/sql/logictest/testdata/logic_test/alter_table b/pkg/sql/logictest/testdata/logic_test/alter_table index daa64730178f..38006ce80ad9 100644 --- a/pkg/sql/logictest/testdata/logic_test/alter_table +++ b/pkg/sql/logictest/testdata/logic_test/alter_table @@ -68,14 +68,14 @@ statement error duplicate key value \(c\)=\(1\) violates unique constraint "bar" ALTER TABLE t ADD CONSTRAINT bar UNIQUE (c) # Test that rollback was successful -query TTTTRT -SELECT job_type, regexp_replace(description, 'JOB \d+', 'JOB ...'), user_name, status, fraction_completed::decimal(10,2), error +query TTTTTRT +SELECT job_type, regexp_replace(description, 'JOB \d+', 'JOB ...'), user_name, status, running_status, fraction_completed::decimal(10,2), error FROM crdb_internal.jobs ORDER BY created DESC LIMIT 2 ---- -SCHEMA CHANGE ROLL BACK JOB ...: ALTER TABLE test.public.t ADD CONSTRAINT bar UNIQUE (c) root succeeded 1.00 · -SCHEMA CHANGE ALTER TABLE test.public.t ADD CONSTRAINT bar UNIQUE (c) root failed 0.00 duplicate key value (c)=(1) violates unique constraint "bar" +SCHEMA CHANGE ROLL BACK JOB ...: ALTER TABLE test.public.t ADD CONSTRAINT bar UNIQUE (c) root running waiting for GC TTL 0.00 · +SCHEMA CHANGE ALTER TABLE test.public.t ADD CONSTRAINT bar UNIQUE (c) root failed NULL 0.00 duplicate key value (c)=(1) violates unique constraint "bar" query IIII colnames,rowsort SELECT * FROM t @@ -178,13 +178,13 @@ ALTER TABLE t DROP CONSTRAINT foo statement ok DROP INDEX foo CASCADE -query TTTTRT -SELECT job_type, description, user_name, status, fraction_completed, error +query TTTTTRT +SELECT job_type, description, user_name, status, running_status, fraction_completed, error FROM crdb_internal.jobs ORDER BY created DESC LIMIT 1 ---- -SCHEMA CHANGE DROP INDEX test.public.t@foo CASCADE root succeeded 1 · +SCHEMA CHANGE DROP INDEX test.public.t@foo CASCADE root running waiting for GC TTL 0 · query TTBITTBB colnames SHOW INDEXES FROM t @@ -261,13 +261,13 @@ INSERT INTO t (a, d, x, y, z) VALUES (33, 34, DECIMAL '2.0', DECIMAL '2.1', DECI statement ok DROP INDEX t@t_f_idx -query TTTTRT -SELECT job_type, description, user_name, status, fraction_completed, error +query TTTTTRT +SELECT job_type, description, user_name, status, running_status, fraction_completed, error FROM crdb_internal.jobs ORDER BY created DESC LIMIT 1 ---- -SCHEMA CHANGE DROP INDEX test.public.t@t_f_idx root succeeded 1 · +SCHEMA CHANGE DROP INDEX test.public.t@t_f_idx root running waiting for GC TTL 0 · statement ok ALTER TABLE t DROP COLUMN f diff --git a/pkg/sql/logictest/testdata/logic_test/show_trace b/pkg/sql/logictest/testdata/logic_test/show_trace index 1d87de7b181b..15c1af768ab7 100644 --- a/pkg/sql/logictest/testdata/logic_test/show_trace +++ b/pkg/sql/logictest/testdata/logic_test/show_trace @@ -411,7 +411,7 @@ statement ok SET tracing = on,kv,results; DROP TABLE t.kv; SET tracing = off query TT -SELECT operation, regexp_replace(regexp_replace(regexp_replace(message, 'drop_job_id:[1-9]\d*', 'drop_job_id:...'), 'wall_time:\d+', 'wall_time:...'), 'drop_time:\d+', 'drop_time:...') as message +SELECT operation, regexp_replace(regexp_replace(regexp_replace(message, 'job_id:[1-9]\d*', 'job_id:...', 'g'), 'wall_time:\d+', 'wall_time:...'), 'drop_time:\d+', 'drop_time:...', 'g') as message FROM [SHOW KV TRACE FOR SESSION] WHERE message NOT LIKE '%Z/%' AND tag NOT LIKE '%intExec=%' @@ -438,12 +438,12 @@ dist sender send querying next range at /Table/3/1/53/2/1 dist sender send r1: sending batch 1 Get to (n1,s1):1 dist sender send querying next range at /Table/5/1/0/2/1 dist sender send r1: sending batch 1 Get to (n1,s1):1 -sql txn Put /Table/3/1/54/2/1 -> table: columns: nullable:false hidden:false > columns: nullable:true hidden:false > next_column_id:3 families: next_family_id:1 primary_index: interleave:<> partitioning: type:FORWARD > next_index_id:3 privileges: users: > next_mutation_id:3 format_version:3 state:DROP draining_names: view_query:"" drop_time:... replacement_of: > audit_mode:DISABLED drop_job_id:... > +sql txn Put /Table/3/1/54/2/1 -> table: columns: nullable:false hidden:false > columns: nullable:true hidden:false > next_column_id:3 families: next_family_id:1 primary_index: interleave:<> partitioning: type:FORWARD > next_index_id:3 privileges: users: > next_mutation_id:3 format_version:3 state:DROP draining_names: view_query:"" drop_time:... replacement_of: > audit_mode:DISABLED drop_job_id:... gc_mutations: > dist sender send querying next range at /Table/3/1/54/2/1 dist sender send r1: sending batch 1 Put to (n1,s1):1 sql txn rows affected: 0 dist sender send querying next range at /Table/SystemConfigSpan/Start -dist sender send r1: sending batch 1 EndTxn, 10 QueryIntent to (n1,s1):1 +dist sender send r1: sending batch 1 EndTxn, 14 QueryIntent to (n1,s1):1 # Check that session tracing does not inhibit the fast path for inserts & # friends (the path resulting in 1PC transactions). diff --git a/pkg/sql/schema_changer.go b/pkg/sql/schema_changer.go index 246d89c35fba..86e6adf08901 100644 --- a/pkg/sql/schema_changer.go +++ b/pkg/sql/schema_changer.go @@ -79,6 +79,12 @@ var schemaChangeLeaseRenewFraction = settings.RegisterFloatSetting( // attempting to become the job coordinator. const asyncSchemaChangeDelay = 1 * time.Minute +type droppedIndex struct { + indexID sqlbase.IndexID + dropTime int64 + deadline int64 +} + // SchemaChanger is used to change the schema on a table. type SchemaChanger struct { tableID sqlbase.ID @@ -93,6 +99,8 @@ type SchemaChanger struct { // table.DropTime. dropTime int64 + dropIndexTimes []droppedIndex + testingKnobs *SchemaChangerTestingKnobs distSQLPlanner *DistSQLPlanner jobRegistry *jobs.Registry @@ -327,6 +335,10 @@ func (sc *SchemaChanger) ExtendLease( return nil } +func (sc *SchemaChanger) canClearRangeForDrop(index *sqlbase.IndexDescriptor) bool { + return sc.execCfg.Settings.Version.IsActive(cluster.VersionClearRange) && !index.IsInterleaved() +} + // DropTableDesc removes a descriptor from the KV database. func (sc *SchemaChanger) DropTableDesc( ctx context.Context, tableDesc *sqlbase.TableDescriptor, traceKV bool, @@ -513,6 +525,90 @@ func (sc *SchemaChanger) maybeAddDrop( return false, nil } +func (sc *SchemaChanger) maybeGCMutations( + ctx context.Context, + inSession bool, + lease *sqlbase.TableDescriptor_SchemaChangeLease, + table *sqlbase.TableDescriptor, +) error { + if inSession || len(table.GCMutations) == 0 || len(sc.dropIndexTimes) == 0 { + return nil + } + + // Don't perform GC work if there are non-GC mutations waiting. + if len(table.Mutations) > 0 { + return nil + } + + if err := sc.ExtendLease(ctx, lease); err != nil { + return err + } + + // Find dropped index with earliest GC deadline. + dropped := sc.dropIndexTimes[0] + for i := 1; i < len(sc.dropIndexTimes); i++ { + if other := sc.dropIndexTimes[i]; other.deadline < dropped.deadline { + dropped = other + } + } + + var mutation sqlbase.TableDescriptor_GCDescriptorMutation + found := false + for _, gcm := range table.GCMutations { + if gcm.IndexID == sc.dropIndexTimes[0].indexID { + found = true + mutation = gcm + break + } + } + if !found { + return errors.Errorf("no GC mutation for index %d", sc.dropIndexTimes[0].indexID) + } + + // Check if the deadline for GC'd dropped index expired because + // a change other than the drop could have scheduled the changer + // for this table. + timeRemaining := timeutil.Since(timeutil.Unix(0, dropped.deadline)) + if timeRemaining < 0 { + // Return nil to allow other any mutations to make progress. + return nil + } + + if err := sc.truncateIndexes(ctx, lease, table.Version, []sqlbase.IndexDescriptor{{ID: mutation.IndexID}}); err != nil { + return err + } + + _, err := sc.leaseMgr.Publish( + ctx, + table.ID, + func(tbl *sqlbase.TableDescriptor) error { + found := false + for i := 0; i < len(tbl.GCMutations); i++ { + if other := tbl.GCMutations[i]; other.IndexID == mutation.IndexID { + tbl.GCMutations = append(tbl.GCMutations[:i], tbl.GCMutations[i+1:]...) + found = true + break + } + } + + if !found { + return errDidntUpdateDescriptor + } + + return nil + }, + func(txn *client.Txn) error { + job, err := sc.jobRegistry.LoadJobWithTxn(ctx, mutation.JobID, txn) + if err != nil { + return err + } + return job.WithTxn(txn).Succeeded(ctx, jobs.NoopFn) + }, + ) + + return err +} + func (sc *SchemaChanger) updateDropTableJob( ctx context.Context, txn *client.Txn, jobID int64, tableID sqlbase.ID, status jobspb.Status, ) error { @@ -668,6 +764,10 @@ func (sc *SchemaChanger) exec( } } + if err := sc.maybeGCMutations(ctx, inSession, &lease, tableDesc); err != nil { + return err + } + if drop, err := sc.maybeAddDrop(ctx, inSession, &lease, tableDesc, evalCtx); err != nil { return err } else if drop { @@ -855,7 +955,13 @@ func (sc *SchemaChanger) waitToUpdateLeases(ctx context.Context, tableID sqlbase // Returns the updated of the descriptor. func (sc *SchemaChanger) done(ctx context.Context) (*sqlbase.Descriptor, error) { isRollback := false + jobSucceeded := true + now := timeutil.Now().UnixNano() return sc.leaseMgr.Publish(ctx, sc.tableID, func(desc *sqlbase.TableDescriptor) error { + // Reset vars here because update function can be called multiple times in a retry. + isRollback = false + jobSucceeded = true + i := 0 for _, mutation := range desc.Mutations { if mutation.MutationID != sc.mutationID { @@ -864,6 +970,19 @@ func (sc *SchemaChanger) done(ctx context.Context) (*sqlbase.Descriptor, error) break } isRollback = mutation.Rollback + if indexDesc := mutation.GetIndex(); mutation.Direction == sqlbase.DescriptorMutation_DROP && + indexDesc != nil { + if sc.canClearRangeForDrop(indexDesc) { + jobSucceeded = false + desc.GCMutations = append( + desc.GCMutations, + sqlbase.TableDescriptor_GCDescriptorMutation{ + IndexID: indexDesc.ID, + DropTime: now, + JobID: *sc.job.ID(), + }) + } + } if err := desc.MakeMutationComplete(mutation); err != nil { return err } @@ -886,8 +1005,16 @@ func (sc *SchemaChanger) done(ctx context.Context) (*sqlbase.Descriptor, error) } return nil }, func(txn *client.Txn) error { - if err := sc.job.WithTxn(txn).Succeeded(ctx, jobs.NoopFn); err != nil { - return errors.Wrapf(err, "failed to mark job %d as as successful", *sc.job.ID()) + if jobSucceeded { + if err := sc.job.WithTxn(txn).Succeeded(ctx, jobs.NoopFn); err != nil { + return errors.Wrapf(err, "failed to mark job %d as successful", *sc.job.ID()) + } + } else { + if err := sc.job.WithTxn(txn).RunningStatus(ctx, func(ctx context.Context, details jobspb.Details) (jobs.RunningStatus, error) { + return jobs.RunningStatusWaitingGC, nil + }); err != nil { + return errors.Wrapf(err, "failed to update running status of job %d", *sc.job.ID()) + } } schemaChangeEventType := EventLogFinishSchemaChange @@ -993,7 +1120,7 @@ func (sc *SchemaChanger) reverseMutations(ctx context.Context, causingError erro } log.Warningf(ctx, "reverse schema change mutation: %+v", mutation) - desc.Mutations[i], columns = reverseMutation(mutation, false /*notStarted*/, columns) + desc.Mutations[i], columns = sc.reverseMutation(mutation, false /*notStarted*/, columns) desc.Mutations[i].Rollback = true } @@ -1188,7 +1315,7 @@ func (sc *SchemaChanger) deleteIndexMutationsWithReversedColumns( // Reverse mutation. Update columns to reflect additional // columns that have been purged. This mutation doesn't need // a rollback because it was not started. - mutation, columns = reverseMutation(mutation, true /*notStarted*/, columns) + mutation, columns = sc.reverseMutation(mutation, true /*notStarted*/, columns) // Mark as complete because this mutation needs no backfill. if err := desc.MakeMutationComplete(mutation); err != nil { return nil, err @@ -1206,7 +1333,7 @@ func (sc *SchemaChanger) deleteIndexMutationsWithReversedColumns( // Reverse a mutation. Returns the updated mutation and updated columns. // notStarted is set to true only if the schema change state machine // was not started for the mutation. -func reverseMutation( +func (sc *SchemaChanger) reverseMutation( mutation sqlbase.DescriptorMutation, notStarted bool, columns map[string]struct{}, ) (sqlbase.DescriptorMutation, map[string]struct{}) { switch mutation.Direction { @@ -1316,7 +1443,8 @@ type SchemaChangeManager struct { testingKnobs *SchemaChangerTestingKnobs // Create a schema changer for every outstanding schema change seen. schemaChangers map[sqlbase.ID]SchemaChanger - // Create a schema changer for every dropped table that needs to be GC-ed. + // Create a schema changer for every table that is dropped or has + // dropped indexes that needs to be GC-ed. forGC map[sqlbase.ID]SchemaChanger distSQLPlanner *DistSQLPlanner ieFactory sqlutil.SessionBoundInternalExecutorFactory @@ -1435,24 +1563,55 @@ func (s *SchemaChangeManager) Start(stopper *stop.Stopper) { // GC TTL for a table has indeed changed it is modified // and enqueued with the new TTL timeout. for id, sc := range s.forGC { + zoneCfg, placeholder, _, err := ZoneConfigHook(cfg, uint32(id)) + if err != nil { + log.Errorf(ctx, "no zone config for desc: %d", id) + return + } + if placeholder == nil { + placeholder = zoneCfg + } + + newExecTime := sc.execAfter if sc.dropTime > 0 { - zoneCfg, _, _, err := ZoneConfigHook(cfg, uint32(id)) - if err != nil { - log.Errorf(ctx, "no zone config for desc: %d", id) - return - } deadline := sc.dropTime + int64(zoneCfg.GC.TTLSeconds)*time.Second.Nanoseconds() + int64(delay) - if ea := timeutil.Unix(0, deadline); ea != sc.execAfter { - resetTimer = true - sc.execAfter = ea - // Safe to modify map inplace while iterating over it. - s.forGC[id] = sc - if log.V(2) { - log.Infof(ctx, - "re-queue up pending drop table GC; table: %d", id) + newExecTime = timeutil.Unix(0, deadline) + } + if len(sc.dropIndexTimes) > 0 { + var earliestIndexExec time.Time + + for i := 0; i < len(sc.dropIndexTimes); i++ { + droppedIdx := &sc.dropIndexTimes[i] + + ttlSeconds := zoneCfg.GC.TTLSeconds + if subzone := placeholder.GetSubzone(uint32(droppedIdx.indexID), ""); subzone != nil { + ttlSeconds = subzone.Config.GC.TTLSeconds } + + deadline := droppedIdx.dropTime + + int64(ttlSeconds)*time.Second.Nanoseconds() + + int64(delay) + execTime := timeutil.Unix(0, deadline) + droppedIdx.deadline = deadline + + if earliestIndexExec.IsZero() || execTime.Before(earliestIndexExec) { + earliestIndexExec = execTime + } + } + if newExecTime.After(earliestIndexExec) { + newExecTime = earliestIndexExec + } + } + if newExecTime != sc.execAfter { + resetTimer = true + sc.execAfter = newExecTime + // Safe to modify map inplace while iterating over it. + s.forGC[id] = sc + if log.V(2) { + log.Infof(ctx, + "re-queue up pending drop table GC; table: %d", id) } } } @@ -1497,14 +1656,43 @@ func (s *SchemaChangeManager) Start(stopper *stop.Stopper) { schemaChanger.execAfter = execAfter schemaChanger.dropTime = 0 + var minDeadline int64 + if len(table.GCMutations) > 0 { + zoneCfg, placeholder, _, err := ZoneConfigHook(cfg, uint32(table.ID)) + if err != nil { + log.Errorf(ctx, "no zone config for desc: %d", table.ID) + return + } + if placeholder == nil { + placeholder = zoneCfg + } + + for _, m := range table.GCMutations { + ttlSeconds := zoneCfg.GC.TTLSeconds + if subzone := placeholder.GetSubzone(uint32(m.IndexID), ""); subzone != nil { + ttlSeconds = subzone.Config.GC.TTLSeconds + } + + deadline := m.DropTime + + int64(ttlSeconds)*time.Second.Nanoseconds() + + int64(delay) + dropped := droppedIndex{m.IndexID, m.DropTime, deadline} + if minDeadline == 0 || deadline < minDeadline { + minDeadline = deadline + } + schemaChanger.dropIndexTimes = append(schemaChanger.dropIndexTimes, dropped) + } + } + // Keep track of outstanding schema changes. // If all schema change commands always set UpVersion, why // check for the presence of mutations? // A schema change execution might fail soon after // unsetting UpVersion, and we still want to process // outstanding mutations. - if table.UpVersion || table.Adding() || - table.HasDrainingNames() || len(table.Mutations) > 0 { + pendingChanges := table.UpVersion || table.Adding() || + table.HasDrainingNames() || len(table.Mutations) > 0 + if pendingChanges { if log.V(2) { log.Infof(ctx, "%s: queue up pending schema change; table: %d, version: %d", kv.Key, table.ID, table.Version) @@ -1532,13 +1720,20 @@ func (s *SchemaChangeManager) Start(stopper *stop.Stopper) { deadline := table.DropTime + int64(zoneCfg.GC.TTLSeconds)*time.Second.Nanoseconds() + int64(delay) - schemaChanger.execAfter = timeutil.Unix(0, deadline) + if minDeadline == 0 || deadline < minDeadline { + minDeadline = deadline + } } + } + + if minDeadline != 0 { + schemaChanger.execAfter = timeutil.Unix(0, minDeadline) + } + if table.Dropped() || (!pendingChanges && len(table.GCMutations) > 0) { s.forGC[table.ID] = schemaChanger // Remove from schema change map if present because - // this table has been dropped and is only waiting - // to be GC-ed. + // this table has been dropped or only has other GC waiting mutations. delete(s.schemaChangers, table.ID) } diff --git a/pkg/sql/schema_changer_test.go b/pkg/sql/schema_changer_test.go index 627879a89d37..7133ae2443e3 100644 --- a/pkg/sql/schema_changer_test.go +++ b/pkg/sql/schema_changer_test.go @@ -467,8 +467,8 @@ func runSchemaChangeWithOperations( execCfg *sql.ExecutorConfig, ) { tableDesc := sqlbase.GetTableDescriptor(kvDB, "t", "test") - // Run the schema change in a separate goroutine. + var wg sync.WaitGroup wg.Add(1) go func() { @@ -599,8 +599,7 @@ func TestRaceWithBackfill(t *testing.T) { backfillNotification = nil } } - // Disable asynchronous schema change execution to allow synchronous path - // to trigger start of backfill notification. + // Disable asynchronous schema change execution. params.Knobs = base.TestingKnobs{ SQLSchemaChanger: &sql.SchemaChangerTestingKnobs{ AsyncExecNotification: asyncSchemaChangerDisabled, @@ -695,18 +694,6 @@ CREATE UNIQUE INDEX vidx ON t.test (v); initBackfillNotification(), &execCfg) - // Drop index. - runSchemaChangeWithOperations( - t, - sqlDB, - kvDB, - jobRegistry, - "DROP INDEX t.test@vidx CASCADE", - maxValue, - 2, - initBackfillNotification(), - &execCfg) - // Verify that the index foo over v is consistent, and that column x has // been backfilled properly. rows, err := sqlDB.Query(`SELECT v, x from t.test@foo`) @@ -882,11 +869,10 @@ func TestBackfillErrors(t *testing.T) { const numNodes, chunkSize, maxValue = 5, 100, 4000 params, _ := tests.CreateTestServerParams() - // Disable asynchronous schema change execution. params.Knobs = base.TestingKnobs{ SQLSchemaChanger: &sql.SchemaChangerTestingKnobs{ - AsyncExecNotification: asyncSchemaChangerDisabled, - BackfillChunkSize: chunkSize, + AsyncExecQuickly: true, + BackfillChunkSize: chunkSize, }, } @@ -906,6 +892,12 @@ CREATE TABLE t.test (k INT PRIMARY KEY, v INT); t.Fatal(err) } + tableDesc := sqlbase.GetTableDescriptor(kvDB, "t", "test") + // Add a zone config for the table. + if _, err := addImmediateGCZoneConfig(sqlDB, tableDesc.ID); err != nil { + t.Fatal(err) + } + // Bulk insert. if err := bulkInsertIntoTable(sqlDB, maxValue); err != nil { t.Fatal(err) @@ -927,7 +919,6 @@ CREATE TABLE t.test (k INT PRIMARY KEY, v INT); } // Split the table into multiple ranges. - tableDesc := sqlbase.GetTableDescriptor(kvDB, "t", "test") // SplitTable moves the right range, so we split things back to front // in order to move less data. for i := numNodes - 1; i > 0; i-- { @@ -946,9 +937,9 @@ CREATE UNIQUE INDEX vidx ON t.test (v); t.Fatalf("got err=%s", err) } - if err := checkTableKeyCount(ctx, kvDB, 1, maxValue); err != nil { - t.Fatal(err) - } + testutils.SucceedsSoon(t, func() error { + return checkTableKeyCount(ctx, kvDB, 1, maxValue) + }) if _, err := sqlDB.Exec(` ALTER TABLE t.test ADD COLUMN p DECIMAL NOT NULL DEFAULT (DECIMAL '1-3'); @@ -988,12 +979,10 @@ func TestAbortSchemaChangeBackfill(t *testing.T) { retriedBackfill := int64(0) var retriedSpan roachpb.Span - // Disable asynchronous schema change execution to allow synchronous path - // to trigger start of backfill notification. params.Knobs = base.TestingKnobs{ SQLSchemaChanger: &sql.SchemaChangerTestingKnobs{ - AsyncExecNotification: asyncSchemaChangerDisabled, - BackfillChunkSize: maxValue, + AsyncExecQuickly: true, + BackfillChunkSize: maxValue, }, DistSQL: &distsqlrun.TestingKnobs{ RunBeforeBackfillChunk: func(sp roachpb.Span) error { @@ -1039,6 +1028,11 @@ CREATE TABLE t.test (k INT PRIMARY KEY, v INT); `); err != nil { t.Fatal(err) } + tableDesc := sqlbase.GetTableDescriptor(kvDB, "t", "test") + // Add a zone config for the table. + if _, err := addImmediateGCZoneConfig(sqlDB, tableDesc.ID); err != nil { + t.Fatal(err) + } // Bulk insert enough rows to exceed the chunk size. inserts := make([]string, maxValue+1) @@ -1357,7 +1351,7 @@ CREATE TABLE t.test (k INT PRIMARY KEY, v INT); currChunk = 0 seenSpan = roachpb.Span{} - dropIndexSchemaChange(t, sqlDB, kvDB, maxValue, 1) + dropIndexSchemaChange(t, sqlDB, kvDB, maxValue, 2) } // Test schema changes are retried and complete properly when the table @@ -1469,14 +1463,13 @@ func TestSchemaChangePurgeFailure(t *testing.T) { defer leaktest.AfterTest(t)() params, _ := tests.CreateTestServerParams() const chunkSize = 200 - // Disable the async schema changer. - var enableAsyncSchemaChanges uint32 - attempts := 0 + var enableAsyncSchemaChanges uint32 = 1 + var attempts int32 // attempt 1: write the first chunk of the index. // attempt 2: write the second chunk and hit a unique constraint // violation; purge the schema change. // attempt 3: return an error while purging the schema change. - expectedAttempts := 3 + var expectedAttempts int32 = 3 params.Knobs = base.TestingKnobs{ SQLSchemaChanger: &sql.SchemaChangerTestingKnobs{ AsyncExecNotification: func() error { @@ -1492,10 +1485,11 @@ func TestSchemaChangePurgeFailure(t *testing.T) { }, DistSQL: &distsqlrun.TestingKnobs{ RunBeforeBackfillChunk: func(sp roachpb.Span) error { - attempts++ // Return a deadline exceeded error during the third attempt // which attempts to clean up the schema change. - if attempts == expectedAttempts { + if atomic.AddInt32(&attempts, 1) == expectedAttempts { + // Disable the async schema changer for assertions. + atomic.StoreUint32(&enableAsyncSchemaChanges, 0) return context.DeadlineExceeded } return nil @@ -1535,11 +1529,20 @@ CREATE TABLE t.test (k INT PRIMARY KEY, v INT); ); !testutils.IsError(err, "violates unique constraint") { t.Fatal(err) } + + tableDesc := sqlbase.GetTableDescriptor(kvDB, "t", "test") + if _, err := addImmediateGCZoneConfig(sqlDB, tableDesc.ID); err != nil { + t.Fatal(err) + } + // The deadline exceeded error in the schema change purge results in no // retry attempts of the purge. - if attempts != expectedAttempts { - t.Fatalf("%d retries, despite allowing only (schema change + reverse) = %d", attempts, expectedAttempts) - } + testutils.SucceedsSoon(t, func() error { + if read := atomic.LoadInt32(&attempts); read != expectedAttempts { + return errors.Errorf("%d retries, despite allowing only (schema change + reverse) = %d", read, expectedAttempts) + } + return nil + }) // The index doesn't exist if _, err := sqlDB.Query( @@ -1548,16 +1551,13 @@ CREATE TABLE t.test (k INT PRIMARY KEY, v INT); t.Fatal(err) } - // Read table descriptor. - tableDesc := sqlbase.GetTableDescriptor(kvDB, "t", "test") + tableDesc = sqlbase.GetTableDescriptor(kvDB, "t", "test") - // There is still a mutation hanging off of it. - if e := 1; len(tableDesc.Mutations) != e { - t.Fatalf("the table has %d instead of %d mutations", len(tableDesc.Mutations), e) - } - // The mutation is for a DROP. - if tableDesc.Mutations[0].Direction != sqlbase.DescriptorMutation_DROP { - t.Fatalf("the table has mutation %v instead of a DROP", tableDesc.Mutations[0]) + // There is still a DROP INDEX mutation waiting for GC. + if e := 1; len(tableDesc.GCMutations) != e { + t.Fatalf("the table has %d instead of %d GC mutations", len(tableDesc.GCMutations), e) + } else if m := tableDesc.GCMutations[0]; m.IndexID != 2 && m.DropTime == 0 && m.JobID == 0 { + t.Fatalf("unexpected GC mutation %v", m) } // There is still some garbage index data that needs to be purged. All the @@ -1581,8 +1581,8 @@ CREATE TABLE t.test (k INT PRIMARY KEY, v INT); testutils.SucceedsSoon(t, func() error { tableDesc = sqlbase.GetTableDescriptor(kvDB, "t", "test") - if len(tableDesc.Mutations) > 0 { - return errors.Errorf("%d mutations remaining", len(tableDesc.Mutations)) + if len(tableDesc.GCMutations) > 0 { + return errors.Errorf("%d GC mutations remaining", len(tableDesc.GCMutations)) } return nil }) @@ -1725,6 +1725,12 @@ CREATE TABLE t.test (k INT PRIMARY KEY, v INT); t.Fatal(err) } + tableDesc := sqlbase.GetTableDescriptor(kvDB, "t", "test") + // Add a zone config for the table. + if _, err := addImmediateGCZoneConfig(sqlDB, tableDesc.ID); err != nil { + t.Fatal(err) + } + testCases := []struct { sql string status jobs.Status @@ -1772,7 +1778,7 @@ CREATE TABLE t.test (k INT PRIMARY KEY, v INT); } } - tableDesc := sqlbase.GetTableDescriptor(kvDB, "t", "test") + tableDesc = sqlbase.GetTableDescriptor(kvDB, "t", "test") if e := 13; e != len(tableDesc.Mutations) { t.Fatalf("e = %d, v = %d", e, len(tableDesc.Mutations)) } @@ -1879,7 +1885,7 @@ CREATE TABLE t.test (k INT PRIMARY KEY, v INT); } // Check that the index on c gets purged. - if _, err = sqlDB.Query(`SELECT * from t.test@foo`); err == nil { + if _, err := sqlDB.Query(`SELECT * from t.test@foo`); err == nil { t.Fatal("SELECT over index 'foo' works") } @@ -2113,13 +2119,17 @@ func TestSchemaUniqueColumnDropFailure(t *testing.T) { const chunkSize = 200 attempts := 0 // DROP UNIQUE COLUMN is executed in two steps: drop index and drop column. - // Chunked backfill attempts: - // attempt 1-5: drop index - // attempt 6-7: drop part of the column before hitting a schema - // change error. - // purge the schema change. - const expectedAttempts = 7 - const maxValue = (expectedAttempts/2+1)*chunkSize + 1 + // Dropping the index happens in a separate mutation job from the drop column + // which does not perform backfilling (like adding indexes and add/drop + // columns) and completes successfully. However, note that the testing knob + // hooks are still run as if they were backfill attempts. The index truncation + // happens as an asynchronous change after the index descriptor is removed, + // and will be run after the GC TTL is passed and there are no pending + // synchronous mutations. Therefore, the first two backfill attempts are from + // the column drop. This part of the change errors during backfilling the + // second chunk. + const expectedColumnBackfillAttempts = 2 + const maxValue = (expectedColumnBackfillAttempts/2+1)*chunkSize + 1 params.Knobs = base.TestingKnobs{ SQLSchemaChanger: &sql.SchemaChangerTestingKnobs{ AsyncExecNotification: asyncSchemaChangerDisabled, @@ -2133,7 +2143,7 @@ func TestSchemaUniqueColumnDropFailure(t *testing.T) { attempts++ // Return a deadline exceeded error while dropping // the column after the index has been dropped. - if attempts == expectedAttempts { + if attempts == expectedColumnBackfillAttempts { return errors.New("permanent failure") } return nil @@ -2170,12 +2180,16 @@ CREATE TABLE t.test (k INT PRIMARY KEY, v INT UNIQUE DEFAULT 23 CREATE FAMILY F3 } // The index is not regenerated. - if err := checkTableKeyCount(context.TODO(), kvDB, 1, maxValue); err != nil { + tableDesc := sqlbase.GetTableDescriptor(kvDB, "t", "test") + if len(tableDesc.Indexes) > 0 { + t.Fatalf("indexes %+v", tableDesc.Indexes) + } + // Index data not wiped yet (waiting for GC TTL). + if err := checkTableKeyCount(context.TODO(), kvDB, 2, maxValue); err != nil { t.Fatal(err) } // Column v still exists with the default value. - tableDesc := sqlbase.GetTableDescriptor(kvDB, "t", "test") if e := 2; e != len(tableDesc.Columns) { t.Fatalf("e = %d, v = %d, columns = %+v", e, len(tableDesc.Columns), tableDesc.Columns) } else if tableDesc.Columns[0].Name != "k" || tableDesc.Columns[1].Name != "v" { @@ -2346,7 +2360,7 @@ func TestBackfillCompletesOnChunkBoundary(t *testing.T) { {sql: "ALTER TABLE t.test ADD COLUMN x DECIMAL DEFAULT (DECIMAL '1.4')", numKeysPerRow: 2}, {sql: "ALTER TABLE t.test DROP pi", numKeysPerRow: 2}, {sql: "CREATE UNIQUE INDEX foo ON t.test (v)", numKeysPerRow: 3}, - {sql: "DROP INDEX t.test@vidx CASCADE", numKeysPerRow: 2}, + {sql: "DROP INDEX t.test@vidx CASCADE", numKeysPerRow: 3}, } for _, tc := range testCases { @@ -3362,8 +3376,16 @@ func TestCancelSchemaChange(t *testing.T) { var db *gosql.DB params, _ := tests.CreateTestServerParams() doCancel := false + var enableAsyncSchemaChanges uint32 params.Knobs = base.TestingKnobs{ SQLSchemaChanger: &sql.SchemaChangerTestingKnobs{ + AsyncExecNotification: func() error { + if enable := atomic.LoadUint32(&enableAsyncSchemaChanges); enable == 0 { + return errors.New("async schema changes are disabled") + } + return nil + }, + AsyncExecQuickly: true, BackfillChunkSize: 10, }, DistSQL: &distsqlrun.TestingKnobs{ @@ -3404,8 +3426,8 @@ func TestCancelSchemaChange(t *testing.T) { t.Fatal(err) } - // Split the table into multiple ranges. tableDesc := sqlbase.GetTableDescriptor(kvDB, "t", "test") + // Split the table into multiple ranges. // SplitTable moves the right range, so we split things back to front // in order to move less data. const numSplits = numNodes * 2 @@ -3422,15 +3444,17 @@ func TestCancelSchemaChange(t *testing.T) { sql string // Set to true if this schema change is to be canceled. cancel bool + // Set to true if the rollback returns in a running, waiting status. + isGC bool }{ {`ALTER TABLE t.public.test ADD COLUMN x DECIMAL DEFAULT 1.4::DECIMAL CREATE FAMILY f2`, - true}, + true, false}, {`CREATE INDEX foo ON t.public.test (v)`, - true}, + true, true}, {`ALTER TABLE t.public.test ADD COLUMN x DECIMAL DEFAULT 1.2::DECIMAL CREATE FAMILY f3`, - false}, + false, false}, {`CREATE INDEX foo ON t.public.test (v)`, - false}, + false, true}, } idx := 0 @@ -3451,13 +3475,20 @@ func TestCancelSchemaChange(t *testing.T) { } jobID := jobutils.GetJobID(t, sqlDB, idx) idx++ - if err := jobutils.VerifySystemJob(t, sqlDB, idx, jobspb.TypeSchemaChange, jobs.StatusSucceeded, jobs.Record{ + jobRecord := jobs.Record{ Username: security.RootUser, Description: fmt.Sprintf("ROLL BACK JOB %d: %s", jobID, tc.sql), DescriptorIDs: sqlbase.IDs{ tableDesc.ID, }, - }); err != nil { + } + var err error + if tc.isGC { + err = jobutils.VerifyRunningSystemJob(t, sqlDB, idx, jobspb.TypeSchemaChange, jobs.RunningStatusWaitingGC, jobRecord) + } else { + err = jobutils.VerifySystemJob(t, sqlDB, idx, jobspb.TypeSchemaChange, jobs.StatusSucceeded, jobRecord) + } + if err != nil { t.Fatal(err) } } else { @@ -3475,9 +3506,13 @@ func TestCancelSchemaChange(t *testing.T) { idx++ } - if err := checkTableKeyCount(ctx, kvDB, 3, maxValue); err != nil { + atomic.StoreUint32(&enableAsyncSchemaChanges, 1) + if _, err := addImmediateGCZoneConfig(db, tableDesc.ID); err != nil { t.Fatal(err) } + testutils.SucceedsSoon(t, func() error { + return checkTableKeyCount(ctx, kvDB, 3, maxValue) + }) // Verify that the index foo over v is consistent, and that column x has // been backfilled properly. diff --git a/pkg/sql/sqlbase/structured.go b/pkg/sql/sqlbase/structured.go index 49947c66737e..2495268e80f6 100644 --- a/pkg/sql/sqlbase/structured.go +++ b/pkg/sql/sqlbase/structured.go @@ -141,6 +141,11 @@ func (dir IndexDescriptor_Direction) ToEncodingDirection() (encoding.Direction, // descriptor could not be found with the given id. var ErrDescriptorNotFound = errors.New("descriptor not found") +// ErrIndexGCMutationsList is returned by FindIndexByID to signal that the +// index with the given ID does not have a descriptor and is in the garbage +// collected mutations list. +var ErrIndexGCMutationsList = errors.New("index in GC mutations list") + // GetDatabaseDescFromID retrieves the database descriptor for the database // ID passed in using an existing txn. Returns an error if the descriptor // doesn't exist or if it exists and is not a database. @@ -343,6 +348,11 @@ func (desc *IndexDescriptor) SQLString(tableName *tree.TableName) string { return f.CloseAndGetString() } +// IsInterleaved returns whether the index is interleaved or not. +func (desc *IndexDescriptor) IsInterleaved() bool { + return len(desc.Interleave.Ancestors) > 0 || len(desc.InterleavedBy) > 0 +} + // SetID implements the DescriptorProto interface. func (desc *TableDescriptor) SetID(id ID) { desc.ID = id @@ -1944,6 +1954,11 @@ func (desc *TableDescriptor) FindIndexByID(id IndexID) (*IndexDescriptor, error) return idx, nil } } + for _, m := range desc.GCMutations { + if m.IndexID == id { + return nil, ErrIndexGCMutationsList + } + } return nil, fmt.Errorf("index-id \"%d\" does not exist", id) } @@ -1987,10 +2002,7 @@ func (desc *TableDescriptor) GetIndexMutationCapabilities(id IndexID) (bool, boo // another table's data. func (desc *TableDescriptor) IsInterleaved() bool { for _, index := range desc.AllNonDropIndexes() { - if len(index.Interleave.Ancestors) > 0 { - return true - } - if len(index.InterleavedBy) > 0 { + if index.IsInterleaved() { return true } } diff --git a/pkg/sql/sqlbase/structured.pb.go b/pkg/sql/sqlbase/structured.pb.go index 933f1fe63f08..ab16e2213481 100644 --- a/pkg/sql/sqlbase/structured.pb.go +++ b/pkg/sql/sqlbase/structured.pb.go @@ -1077,6 +1077,17 @@ type TableDescriptor struct { // The job id for a drop job is the id in the system.jobs table of the // dropping of this table. DropJobID int64 `protobuf:"varint,32,opt,name=drop_job_id,json=dropJobId" json:"drop_job_id"` + // The schema elements that have been dropped and whose underlying + // data needs to be gc-ed. These schema elements have already transitioned + // through the drop state machine when they were in the above mutations + // list, and can be safely deleted. The names for these schema elements + // can be reused. This list is separate because mutations can + // lie in this list for a long time (gc deadline) and should not block + // the execution of other schema changes on the table. + // + // TODO(vivekmenezes): This is currently only used by the non-interleaved drop + // index case. Also use for dropped interleaved indexes and columns. + GCMutations []TableDescriptor_GCDescriptorMutation `protobuf:"bytes,33,rep,name=gc_mutations,json=gcMutations" json:"gc_mutations"` } func (m *TableDescriptor) Reset() { *m = TableDescriptor{} } @@ -1294,6 +1305,13 @@ func (m *TableDescriptor) GetDropJobID() int64 { return 0 } +func (m *TableDescriptor) GetGCMutations() []TableDescriptor_GCDescriptorMutation { + if m != nil { + return m.GCMutations + } + return nil +} + // The schema update lease. A single goroutine across a cockroach cluster // can own it, and will execute pending schema changes for this table. // Since the execution of a pending schema change is through transactions, @@ -1473,6 +1491,21 @@ func (*TableDescriptor_Replacement) Descriptor() ([]byte, []int) { return fileDescriptorStructured, []int{8, 6} } +type TableDescriptor_GCDescriptorMutation struct { + IndexID IndexID `protobuf:"varint,1,opt,name=index_id,json=indexId,casttype=IndexID" json:"index_id"` + DropTime int64 `protobuf:"varint,2,opt,name=drop_time,json=dropTime" json:"drop_time"` + // The job id for a mutation job is the id in the system.jobs table of the + // schema change job executing the mutation referenced by mutation_id. + JobID int64 `protobuf:"varint,3,opt,name=job_id,json=jobId" json:"job_id"` +} + +func (m *TableDescriptor_GCDescriptorMutation) Reset() { *m = TableDescriptor_GCDescriptorMutation{} } +func (m *TableDescriptor_GCDescriptorMutation) String() string { return proto.CompactTextString(m) } +func (*TableDescriptor_GCDescriptorMutation) ProtoMessage() {} +func (*TableDescriptor_GCDescriptorMutation) Descriptor() ([]byte, []int) { + return fileDescriptorStructured, []int{8, 7} +} + // DatabaseDescriptor represents a namespace (aka database) and is stored // in a structured metadata key. The DatabaseDescriptor has a globally-unique // ID shared with the TableDescriptor ID. @@ -1653,6 +1686,7 @@ func init() { proto.RegisterType((*TableDescriptor_MutationJob)(nil), "cockroach.sql.sqlbase.TableDescriptor.MutationJob") proto.RegisterType((*TableDescriptor_SequenceOpts)(nil), "cockroach.sql.sqlbase.TableDescriptor.SequenceOpts") proto.RegisterType((*TableDescriptor_Replacement)(nil), "cockroach.sql.sqlbase.TableDescriptor.Replacement") + proto.RegisterType((*TableDescriptor_GCDescriptorMutation)(nil), "cockroach.sql.sqlbase.TableDescriptor.GCDescriptorMutation") proto.RegisterType((*DatabaseDescriptor)(nil), "cockroach.sql.sqlbase.DatabaseDescriptor") proto.RegisterType((*Descriptor)(nil), "cockroach.sql.sqlbase.Descriptor") proto.RegisterEnum("cockroach.sql.sqlbase.ConstraintValidity", ConstraintValidity_name, ConstraintValidity_value) @@ -2625,6 +2659,20 @@ func (m *TableDescriptor) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintStructured(dAtA, i, uint64(m.DropJobID)) + if len(m.GCMutations) > 0 { + for _, msg := range m.GCMutations { + dAtA[i] = 0x8a + i++ + dAtA[i] = 0x2 + i++ + i = encodeVarintStructured(dAtA, i, uint64(msg.Size())) + n, err := msg.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n + } + } return i, nil } @@ -2835,6 +2883,33 @@ func (m *TableDescriptor_Replacement) MarshalTo(dAtA []byte) (int, error) { return i, nil } +func (m *TableDescriptor_GCDescriptorMutation) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *TableDescriptor_GCDescriptorMutation) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + dAtA[i] = 0x8 + i++ + i = encodeVarintStructured(dAtA, i, uint64(m.IndexID)) + dAtA[i] = 0x10 + i++ + i = encodeVarintStructured(dAtA, i, uint64(m.DropTime)) + dAtA[i] = 0x18 + i++ + i = encodeVarintStructured(dAtA, i, uint64(m.JobID)) + return i, nil +} + func (m *DatabaseDescriptor) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) @@ -3287,6 +3362,12 @@ func (m *TableDescriptor) Size() (n int) { n += 2 + l + sovStructured(uint64(l)) n += 2 + sovStructured(uint64(m.AuditMode)) n += 2 + sovStructured(uint64(m.DropJobID)) + if len(m.GCMutations) > 0 { + for _, e := range m.GCMutations { + l = e.Size() + n += 2 + l + sovStructured(uint64(l)) + } + } return n } @@ -3364,6 +3445,15 @@ func (m *TableDescriptor_Replacement) Size() (n int) { return n } +func (m *TableDescriptor_GCDescriptorMutation) Size() (n int) { + var l int + _ = l + n += 1 + sovStructured(uint64(m.IndexID)) + n += 1 + sovStructured(uint64(m.DropTime)) + n += 1 + sovStructured(uint64(m.JobID)) + return n +} + func (m *DatabaseDescriptor) Size() (n int) { var l int _ = l @@ -6687,6 +6777,37 @@ func (m *TableDescriptor) Unmarshal(dAtA []byte) error { break } } + case 33: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field GCMutations", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowStructured + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthStructured + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.GCMutations = append(m.GCMutations, TableDescriptor_GCDescriptorMutation{}) + if err := m.GCMutations[len(m.GCMutations)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipStructured(dAtA[iNdEx:]) @@ -7566,6 +7687,113 @@ func (m *TableDescriptor_Replacement) Unmarshal(dAtA []byte) error { } return nil } +func (m *TableDescriptor_GCDescriptorMutation) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowStructured + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: GCDescriptorMutation: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: GCDescriptorMutation: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field IndexID", wireType) + } + m.IndexID = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowStructured + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.IndexID |= (IndexID(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + case 2: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field DropTime", wireType) + } + m.DropTime = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowStructured + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.DropTime |= (int64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + case 3: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field JobID", wireType) + } + m.JobID = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowStructured + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.JobID |= (int64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + default: + iNdEx = preIndex + skippy, err := skipStructured(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthStructured + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} func (m *DatabaseDescriptor) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 @@ -7919,197 +8147,201 @@ var ( func init() { proto.RegisterFile("sql/sqlbase/structured.proto", fileDescriptorStructured) } var fileDescriptorStructured = []byte{ - // 3072 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xa4, 0x59, 0x4b, 0x73, 0xe3, 0xc6, - 0xf1, 0x17, 0x48, 0x90, 0x04, 0x9a, 0x2f, 0x68, 0xf6, 0x61, 0x2e, 0xbd, 0x96, 0xb4, 0xb4, 0xd7, - 0x7f, 0xf9, 0x45, 0xad, 0xb5, 0xfe, 0x27, 0x2e, 0x27, 0xe5, 0x0a, 0x1f, 0xd0, 0x2e, 0xb4, 0x14, - 0xa9, 0x05, 0x29, 0xad, 0xd7, 0xe5, 0x84, 0x05, 0x11, 0x23, 0x09, 0x5e, 0x10, 0xe0, 0x02, 0xa0, - 0x2c, 0x7d, 0x03, 0x1f, 0x53, 0xb9, 0xe4, 0xe6, 0xb8, 0x7c, 0x4a, 0x55, 0xae, 0x39, 0xe4, 0x23, - 0x6c, 0x6e, 0x29, 0x9f, 0x72, 0x89, 0x2a, 0x51, 0x2a, 0x55, 0xf9, 0x04, 0x39, 0xb8, 0x2a, 0x55, - 0xa9, 0x19, 0xcc, 0x80, 0xa0, 0x5e, 0xa6, 0x76, 0x6f, 0x60, 0x4f, 0xf7, 0x6f, 0xa6, 0x7b, 0xba, - 0x7f, 0xd3, 0x33, 0x84, 0xdb, 0xfe, 0x73, 0x7b, 0xc5, 0x7f, 0x6e, 0xef, 0x18, 0x3e, 0x5e, 0xf1, - 0x03, 0x6f, 0x3c, 0x08, 0xc6, 0x1e, 0x36, 0xab, 0x23, 0xcf, 0x0d, 0x5c, 0x74, 0x63, 0xe0, 0x0e, - 0x9e, 0x79, 0xae, 0x31, 0xd8, 0xaf, 0xfa, 0xcf, 0xed, 0x2a, 0xd3, 0x2b, 0x97, 0xc6, 0x81, 0x65, - 0xaf, 0xec, 0xdb, 0x83, 0x95, 0xc0, 0x1a, 0x62, 0x3f, 0x30, 0x86, 0xa3, 0xd0, 0xa0, 0xfc, 0x7a, - 0x1c, 0x6e, 0xe4, 0x59, 0x07, 0x96, 0x8d, 0xf7, 0x30, 0x1b, 0xbc, 0xbe, 0xe7, 0xee, 0xb9, 0xf4, - 0x73, 0x85, 0x7c, 0x85, 0xd2, 0xca, 0xf7, 0x19, 0x80, 0x86, 0x6b, 0x8f, 0x87, 0x4e, 0xef, 0x68, - 0x84, 0xd1, 0x53, 0xc8, 0xfb, 0x78, 0x68, 0x38, 0x81, 0x35, 0xe8, 0x07, 0x47, 0x23, 0x5c, 0x12, - 0x96, 0x84, 0xe5, 0xc2, 0x6a, 0xb5, 0x7a, 0xee, 0x52, 0xaa, 0x13, 0xcb, 0x6a, 0x97, 0x99, 0x91, - 0x1f, 0x75, 0xf1, 0xc5, 0xf1, 0xe2, 0x9c, 0x9e, 0xf3, 0x63, 0x32, 0x54, 0x86, 0xd4, 0x57, 0x96, - 0x19, 0xec, 0x97, 0x12, 0x4b, 0xc2, 0x72, 0x8a, 0xa9, 0x84, 0x22, 0x54, 0x01, 0x79, 0xe4, 0xe1, - 0x81, 0xe5, 0x5b, 0xae, 0x53, 0x4a, 0xc6, 0xc6, 0x27, 0x62, 0xf4, 0x0e, 0x28, 0x86, 0xe7, 0x19, - 0x47, 0x7d, 0xd3, 0x1a, 0x62, 0x87, 0x88, 0xfc, 0x92, 0xb8, 0x94, 0x5c, 0x4e, 0xe9, 0x45, 0x2a, - 0x6f, 0x46, 0x62, 0x74, 0x13, 0xd2, 0xb6, 0x3b, 0x30, 0x6c, 0x5c, 0x4a, 0x2d, 0x09, 0xcb, 0xb2, - 0xce, 0x7e, 0xa1, 0x6d, 0xc8, 0x1d, 0x58, 0xbe, 0xb5, 0x63, 0xe3, 0xd0, 0xb9, 0x34, 0x75, 0xee, - 0x83, 0x1f, 0x77, 0x6e, 0x3b, 0xb4, 0x8a, 0xf9, 0x96, 0x3d, 0x98, 0x88, 0xd0, 0x16, 0x14, 0xc2, - 0xa5, 0x0d, 0x5c, 0x27, 0xc0, 0x4e, 0xe0, 0x97, 0x32, 0x2f, 0x13, 0x36, 0x3d, 0x4f, 0x51, 0x1a, - 0x0c, 0x04, 0xb5, 0xa1, 0x10, 0x8c, 0x47, 0x36, 0x9e, 0xc0, 0x4a, 0x4b, 0xc9, 0xe5, 0xec, 0xea, - 0x9d, 0x1f, 0x85, 0x65, 0x8b, 0xcc, 0x53, 0xf3, 0x08, 0xef, 0x0e, 0xe4, 0x42, 0x3c, 0xdb, 0xd8, - 0xc1, 0xb6, 0x5f, 0x92, 0x97, 0x92, 0xcb, 0xb2, 0x9e, 0xa5, 0xb2, 0x16, 0x15, 0x55, 0x7e, 0x9f, - 0x80, 0x5c, 0x7c, 0x49, 0x48, 0x02, 0xb1, 0xde, 0xe9, 0xb4, 0x94, 0x39, 0x94, 0x81, 0xa4, 0xd6, - 0xee, 0x29, 0x02, 0x92, 0x21, 0xb5, 0xd6, 0xea, 0xd4, 0x7a, 0x4a, 0x02, 0x65, 0x21, 0xd3, 0x54, - 0x1b, 0xda, 0x46, 0xad, 0xa5, 0x24, 0x89, 0x6a, 0xb3, 0xd6, 0x53, 0x15, 0x11, 0xe5, 0x41, 0xee, - 0x69, 0x1b, 0x6a, 0xb7, 0x57, 0xdb, 0xd8, 0x54, 0x52, 0x28, 0x07, 0x92, 0xd6, 0xee, 0xa9, 0xfa, - 0x76, 0xad, 0xa5, 0xa4, 0x11, 0x40, 0xba, 0xdb, 0xd3, 0xb5, 0xf6, 0x03, 0x25, 0x43, 0xa0, 0xea, - 0x4f, 0x7b, 0x6a, 0x57, 0x91, 0x50, 0x11, 0xb2, 0x91, 0x4d, 0xef, 0x73, 0x45, 0x46, 0x08, 0x0a, - 0x8d, 0x4e, 0xab, 0x55, 0xeb, 0xa9, 0x4d, 0xa6, 0x0f, 0x64, 0x8a, 0x76, 0x6d, 0x43, 0x55, 0xb2, - 0x64, 0x35, 0x1d, 0xad, 0xa9, 0xe4, 0xa8, 0x68, 0xab, 0xd5, 0x52, 0xf2, 0xe4, 0x6b, 0x6b, 0x4b, - 0x6b, 0x2a, 0x05, 0x02, 0x5b, 0xd3, 0xf5, 0xda, 0x53, 0xa5, 0x48, 0x84, 0x5a, 0x5b, 0xed, 0x29, - 0x0a, 0xf9, 0x22, 0x13, 0x28, 0xf3, 0x64, 0x78, 0xbd, 0xdb, 0x69, 0xd7, 0x15, 0x44, 0x3e, 0x7b, - 0x5b, 0x9b, 0x2d, 0x55, 0xb9, 0x4e, 0x10, 0xeb, 0x5a, 0x4f, 0xb9, 0x81, 0x8a, 0x00, 0x5a, 0xbb, - 0xb7, 0xba, 0xad, 0x36, 0x7a, 0x1d, 0x5d, 0x79, 0x21, 0xa0, 0x02, 0xc8, 0x1d, 0xad, 0xc9, 0x7e, - 0xff, 0x59, 0xa8, 0x88, 0xd2, 0x35, 0xe5, 0x5a, 0xe5, 0x37, 0x02, 0x64, 0x63, 0x79, 0x41, 0x17, - 0xd2, 0x69, 0xab, 0xca, 0x1c, 0x89, 0x0a, 0xf1, 0xf7, 0x81, 0xaa, 0x2b, 0x02, 0x71, 0xbe, 0xbb, - 0x51, 0x6b, 0xb5, 0x48, 0xec, 0x12, 0xc4, 0xf9, 0xba, 0xf6, 0x80, 0x7c, 0xd3, 0x78, 0xe9, 0x6a, - 0xad, 0xa5, 0xa4, 0xd0, 0x75, 0x50, 0x9a, 0x9d, 0xad, 0x7a, 0x4b, 0xed, 0x6f, 0xea, 0x6a, 0x43, - 0xeb, 0x6a, 0x9d, 0xb6, 0x92, 0x26, 0x30, 0xdb, 0x35, 0xbd, 0xf1, 0xb0, 0xa6, 0x2b, 0x19, 0xa2, - 0x4c, 0xbf, 0x24, 0xb2, 0xe4, 0xc7, 0xf4, 0x53, 0x26, 0x68, 0xdb, 0x35, 0x9d, 0xac, 0x1a, 0x2a, - 0xa2, 0x24, 0x2a, 0xe2, 0x27, 0xe2, 0xbf, 0xbf, 0x5d, 0x14, 0x2a, 0xff, 0x49, 0xc2, 0xb5, 0x35, - 0xd7, 0xc3, 0xd6, 0x9e, 0xf3, 0x08, 0x1f, 0xe9, 0x78, 0x17, 0x7b, 0xd8, 0x19, 0x60, 0xb4, 0x04, - 0xa9, 0xc0, 0xd8, 0xb1, 0xc3, 0xaa, 0xce, 0xd7, 0x81, 0x24, 0xc9, 0x0f, 0xc7, 0x8b, 0x09, 0xad, - 0xa9, 0x87, 0x03, 0xe8, 0x2e, 0xa4, 0x2c, 0xc7, 0xc4, 0x87, 0xb4, 0x48, 0xf3, 0xf5, 0x22, 0xd3, - 0xc8, 0x68, 0x44, 0x48, 0xd4, 0xe8, 0x28, 0x2a, 0x81, 0xe8, 0x18, 0x43, 0x4c, 0x4b, 0x55, 0x66, - 0xc9, 0x46, 0x25, 0xe8, 0x11, 0x48, 0x07, 0x86, 0x6d, 0x99, 0x56, 0x70, 0x54, 0x12, 0x69, 0x11, - 0xbc, 0x73, 0x61, 0xb6, 0x3a, 0x7e, 0xe0, 0x19, 0x96, 0x13, 0x6c, 0x33, 0x03, 0x06, 0x14, 0x01, - 0xa0, 0x7b, 0x30, 0xef, 0xef, 0x1b, 0x1e, 0x36, 0xfb, 0x23, 0x0f, 0xef, 0x5a, 0x87, 0x7d, 0x1b, - 0x3b, 0xb4, 0xa4, 0x39, 0x3d, 0x14, 0xc3, 0xe1, 0x4d, 0x3a, 0xda, 0xc2, 0x0e, 0xea, 0x81, 0xec, - 0x3a, 0x7d, 0x13, 0xdb, 0x38, 0xe0, 0xe5, 0xfd, 0xe1, 0x05, 0xf3, 0x9f, 0x13, 0xa0, 0x6a, 0x6d, - 0x10, 0x58, 0xae, 0xc3, 0xd7, 0xe1, 0x3a, 0x4d, 0x0a, 0xc4, 0x50, 0xc7, 0x23, 0xd3, 0x08, 0x30, - 0x2b, 0xed, 0x57, 0x41, 0xdd, 0xa2, 0x40, 0x95, 0xc7, 0x90, 0x0e, 0x47, 0x48, 0xbd, 0xb4, 0x3b, - 0xfd, 0x5a, 0xa3, 0x47, 0x36, 0x7e, 0x8e, 0xa4, 0x8c, 0xae, 0x92, 0x9c, 0x6f, 0xf4, 0x58, 0x02, - 0xa9, 0xbd, 0x3e, 0x4d, 0xf2, 0x04, 0x29, 0x13, 0xf2, 0xab, 0xa9, 0xae, 0xd5, 0xb6, 0x5a, 0x24, - 0x8b, 0xb2, 0x90, 0x69, 0xd4, 0xba, 0x8d, 0x5a, 0x53, 0x55, 0xc4, 0xca, 0xdf, 0x12, 0xa0, 0x84, - 0x2c, 0xd0, 0xc4, 0xfe, 0xc0, 0xb3, 0x46, 0x81, 0xeb, 0x45, 0x9b, 0x25, 0x9c, 0xd9, 0xac, 0xb7, - 0x21, 0x61, 0x99, 0x6c, 0xab, 0x6f, 0x12, 0xf9, 0x09, 0x4d, 0x86, 0x1f, 0x8e, 0x17, 0xa5, 0x10, - 0x45, 0x6b, 0xea, 0x09, 0xcb, 0x44, 0x3f, 0x03, 0x91, 0xf2, 0x25, 0xd9, 0xee, 0x2b, 0xd0, 0x0f, - 0x35, 0x42, 0x4b, 0x20, 0x39, 0x63, 0xdb, 0xa6, 0x79, 0x47, 0x32, 0x42, 0xe2, 0x81, 0xe0, 0x52, - 0xc2, 0x4b, 0x26, 0xde, 0x35, 0xc6, 0x76, 0xd0, 0xc7, 0x87, 0x23, 0x8f, 0x91, 0x76, 0x96, 0xc9, - 0xd4, 0xc3, 0x91, 0x87, 0x6e, 0x43, 0x7a, 0xdf, 0x32, 0x4d, 0xec, 0xd0, 0x4d, 0xe5, 0x10, 0x4c, - 0x86, 0x56, 0x61, 0x7e, 0xec, 0x63, 0xbf, 0xef, 0xe3, 0xe7, 0x63, 0x12, 0xf1, 0xbe, 0x65, 0xfa, - 0x25, 0x58, 0x4a, 0x2e, 0xe7, 0xeb, 0x69, 0x96, 0xdf, 0x45, 0xa2, 0xd0, 0x65, 0xe3, 0x9a, 0x49, - 0xc9, 0x70, 0xe0, 0x0e, 0x47, 0xe3, 0x00, 0x87, 0x93, 0x66, 0xc3, 0x49, 0x99, 0x8c, 0x4c, 0xba, - 0x2e, 0x4a, 0x92, 0x22, 0xaf, 0x8b, 0x92, 0xac, 0xc0, 0xba, 0x28, 0x65, 0x14, 0xa9, 0xf2, 0x75, - 0x02, 0x6e, 0x86, 0x6e, 0xae, 0x19, 0x43, 0xcb, 0x3e, 0x7a, 0xd5, 0x28, 0x87, 0x28, 0x2c, 0xca, - 0x74, 0x45, 0x04, 0xbb, 0x4f, 0xcc, 0xfc, 0x52, 0x32, 0xa4, 0xe7, 0x50, 0xd6, 0x26, 0x22, 0xf4, - 0x31, 0x00, 0x53, 0x21, 0x1e, 0x8a, 0xd4, 0xc3, 0x5b, 0x27, 0xc7, 0x8b, 0x32, 0xdf, 0x2e, 0x7f, - 0x6a, 0xef, 0xe4, 0x50, 0x99, 0xb8, 0xdb, 0x81, 0x79, 0x1e, 0xe3, 0x08, 0x81, 0x06, 0x3a, 0x5f, - 0x7f, 0x93, 0xad, 0xa9, 0xd8, 0x0c, 0x15, 0xb8, 0xf9, 0x14, 0x54, 0xd1, 0x9c, 0x1a, 0x34, 0x2b, - 0x7f, 0x48, 0xc0, 0x75, 0xcd, 0x09, 0xb0, 0x67, 0x63, 0xe3, 0x00, 0xc7, 0x02, 0xf1, 0x19, 0xc8, - 0x86, 0x33, 0xc0, 0x7e, 0xe0, 0x7a, 0x7e, 0x49, 0xa0, 0x07, 0xd6, 0x47, 0x17, 0x64, 0xcc, 0x79, - 0xf6, 0xd5, 0x1a, 0x33, 0xe6, 0x1d, 0x40, 0x04, 0x56, 0xfe, 0x93, 0x00, 0x12, 0x1f, 0x45, 0xf7, - 0x40, 0xa2, 0x94, 0x45, 0xfc, 0x08, 0xe9, 0xec, 0x06, 0xf3, 0x23, 0xd3, 0x23, 0x72, 0xba, 0x7e, - 0xb2, 0xf3, 0x19, 0xaa, 0xa6, 0x99, 0xe8, 0xff, 0x41, 0xa2, 0xec, 0xd5, 0x8f, 0x76, 0xa3, 0xcc, - 0x2d, 0x18, 0xbd, 0xc5, 0x99, 0x2e, 0x43, 0x75, 0x35, 0x13, 0x35, 0xce, 0x23, 0xa1, 0x24, 0xb5, - 0x7f, 0x8d, 0x47, 0xae, 0x3b, 0x4d, 0x43, 0x67, 0x78, 0xa9, 0xf2, 0xaf, 0x24, 0xdc, 0xdc, 0x34, - 0xbc, 0xc0, 0x22, 0xf5, 0x6e, 0x39, 0x7b, 0xb1, 0x78, 0xdd, 0x85, 0xac, 0x33, 0x1e, 0xb2, 0x5d, - 0xf1, 0x99, 0x2f, 0xa1, 0xef, 0xe0, 0x8c, 0x87, 0x61, 0xc0, 0x7d, 0xd4, 0x02, 0xd1, 0xb6, 0xfc, - 0xa0, 0x94, 0xa0, 0x11, 0x5d, 0xbd, 0x20, 0xa2, 0xe7, 0xcf, 0x51, 0x6d, 0x59, 0x7e, 0xc0, 0x73, - 0x92, 0xa0, 0xa0, 0x0e, 0xa4, 0x3c, 0xc3, 0xd9, 0xc3, 0x34, 0xc9, 0xb2, 0xab, 0xf7, 0xaf, 0x06, - 0xa7, 0x13, 0x53, 0xde, 0xc1, 0x51, 0x9c, 0xf2, 0x6f, 0x05, 0x10, 0xc9, 0x2c, 0x97, 0xd4, 0xc1, - 0x4d, 0x48, 0x1f, 0x18, 0xf6, 0x18, 0xfb, 0xd4, 0x87, 0x9c, 0xce, 0x7e, 0xa1, 0x5f, 0x42, 0xd1, - 0x1f, 0xef, 0x8c, 0x62, 0x53, 0x31, 0xa2, 0xf9, 0xe0, 0x4a, 0xab, 0x8a, 0x8e, 0x84, 0x69, 0xac, - 0xf2, 0x33, 0x48, 0xd1, 0xf5, 0x5e, 0xb2, 0x32, 0xd2, 0x18, 0xb9, 0x7d, 0x7c, 0x38, 0xb0, 0xc7, - 0xbe, 0x75, 0x80, 0x69, 0x76, 0xe4, 0xf4, 0x6c, 0xe0, 0xaa, 0x5c, 0x84, 0xee, 0x42, 0x61, 0xd7, - 0x73, 0x87, 0x7d, 0xcb, 0xe1, 0x4a, 0x49, 0xaa, 0x94, 0x27, 0x52, 0x8d, 0x0b, 0x2b, 0xff, 0x95, - 0xa0, 0x48, 0x33, 0x68, 0x26, 0x66, 0xb8, 0x1b, 0x63, 0x86, 0x1b, 0x53, 0xcc, 0x10, 0xa5, 0x21, - 0x21, 0x86, 0xdb, 0x90, 0x1e, 0x3b, 0xd6, 0xf3, 0x71, 0x38, 0x67, 0x44, 0x7e, 0xa1, 0xec, 0x0c, - 0x6d, 0x88, 0x67, 0x69, 0xe3, 0x7d, 0x40, 0xa4, 0x66, 0x70, 0x7f, 0x4a, 0x31, 0x45, 0x15, 0x15, - 0x3a, 0xd2, 0xb8, 0x90, 0x64, 0xd2, 0x57, 0x20, 0x99, 0x87, 0xa0, 0xe0, 0xc3, 0xc0, 0x33, 0xfa, - 0x31, 0xfb, 0x0c, 0xb5, 0x5f, 0x38, 0x39, 0x5e, 0x2c, 0xa8, 0x64, 0xec, 0x7c, 0x90, 0x02, 0x8e, - 0x8d, 0x99, 0x24, 0x27, 0xe6, 0x19, 0x86, 0x69, 0x79, 0x98, 0x9e, 0x92, 0x61, 0xf7, 0x5b, 0x58, - 0xbd, 0x77, 0x21, 0x99, 0x4c, 0x85, 0xbd, 0xda, 0xe4, 0x86, 0xba, 0x12, 0x42, 0x45, 0x02, 0x1f, - 0x3d, 0x86, 0xec, 0x6e, 0x78, 0x50, 0xf7, 0x9f, 0xe1, 0xa3, 0x92, 0x4c, 0xd3, 0xed, 0xdd, 0xd9, - 0x8f, 0x74, 0x5e, 0x9f, 0xbb, 0xd1, 0x10, 0xda, 0x82, 0xbc, 0xc7, 0x87, 0xcd, 0xfe, 0xce, 0x11, - 0x3d, 0x7f, 0x5e, 0x06, 0x34, 0x37, 0x81, 0xa9, 0x1f, 0xa1, 0xc7, 0x00, 0x56, 0xc4, 0x92, 0xf4, - 0x90, 0xca, 0xae, 0xbe, 0x77, 0x05, 0x3a, 0xe5, 0x2b, 0x9d, 0x80, 0xa0, 0x27, 0x50, 0x98, 0xfc, - 0xa2, 0x4b, 0xcd, 0xbd, 0xe4, 0x52, 0xf3, 0x31, 0x9c, 0xfa, 0x11, 0xea, 0xc1, 0x75, 0x72, 0x7c, - 0xba, 0xbe, 0x15, 0xe0, 0x78, 0x0a, 0xe4, 0x69, 0x0a, 0x54, 0x4e, 0x8e, 0x17, 0x51, 0x83, 0x8f, - 0x9f, 0x9f, 0x06, 0x68, 0x70, 0x6a, 0x3c, 0x4c, 0xaa, 0xa9, 0xe4, 0x25, 0x88, 0x85, 0x49, 0x52, - 0x75, 0x27, 0xe9, 0x7b, 0x26, 0xa9, 0x62, 0xa9, 0x4d, 0x90, 0x9e, 0x40, 0x6e, 0x8a, 0x65, 0x8a, - 0x2f, 0xcf, 0x32, 0x53, 0x40, 0x48, 0x65, 0xfd, 0x91, 0x42, 0x5b, 0xc3, 0xf7, 0x66, 0x4c, 0xd0, - 0xd3, 0x9d, 0x52, 0x65, 0x01, 0xe4, 0x28, 0x47, 0xc9, 0x75, 0xa4, 0xd6, 0x6d, 0x28, 0x73, 0xf4, - 0x5a, 0xa5, 0x76, 0x1b, 0x8a, 0x50, 0xb9, 0x03, 0x22, 0xbd, 0x69, 0x64, 0x21, 0xb3, 0xd6, 0xd1, - 0x9f, 0xd4, 0xf4, 0x66, 0xd8, 0x2c, 0x6a, 0xed, 0x6d, 0x55, 0xef, 0xa9, 0x4d, 0x45, 0xa8, 0x7c, - 0x27, 0x02, 0x9a, 0x4c, 0xb1, 0x31, 0x0e, 0x0c, 0x0a, 0x56, 0x83, 0x74, 0x18, 0x3d, 0x4a, 0x42, - 0xd9, 0xd5, 0xff, 0xbb, 0xb4, 0x85, 0x9b, 0x00, 0x3c, 0x9c, 0xd3, 0x99, 0x21, 0xfa, 0x34, 0x7e, - 0x33, 0xc8, 0xae, 0xbe, 0x3d, 0x9b, 0x93, 0x0f, 0xe7, 0xf8, 0x95, 0xe1, 0x11, 0xa4, 0xfc, 0x80, - 0xf4, 0xcf, 0x49, 0x1a, 0xa4, 0x95, 0x0b, 0xec, 0xcf, 0x2e, 0xbe, 0xda, 0x25, 0x66, 0xfc, 0xb4, - 0xa1, 0x18, 0xe8, 0x09, 0xc8, 0x11, 0x2f, 0xb0, 0x6b, 0xc6, 0xfd, 0xd9, 0x01, 0xa3, 0x20, 0xf3, - 0x16, 0x23, 0xc2, 0x42, 0x35, 0xc8, 0x0e, 0x99, 0xda, 0xa4, 0x41, 0x5a, 0x62, 0xd4, 0x0c, 0x1c, - 0x81, 0x52, 0x74, 0xec, 0x97, 0x0e, 0xdc, 0x48, 0x33, 0x49, 0xbf, 0xeb, 0xb9, 0xb6, 0xbd, 0x63, - 0x0c, 0x9e, 0xd1, 0xbb, 0x42, 0xd4, 0xef, 0x72, 0x69, 0xe5, 0x17, 0x90, 0xa2, 0x3e, 0x91, 0x8d, - 0xdc, 0x6a, 0x3f, 0x6a, 0x77, 0x9e, 0x90, 0xae, 0xbf, 0x08, 0xd9, 0xa6, 0xda, 0x52, 0x7b, 0x6a, - 0xbf, 0xd3, 0x6e, 0x3d, 0x55, 0x04, 0x74, 0x0b, 0x6e, 0x30, 0x41, 0xad, 0xdd, 0xec, 0x3f, 0xd1, - 0x35, 0x3e, 0x94, 0xa8, 0x2c, 0xc7, 0x33, 0x65, 0x72, 0xf1, 0x24, 0x39, 0xd3, 0x6c, 0x2a, 0x02, - 0xcd, 0x19, 0xbd, 0xb3, 0xa9, 0x24, 0xea, 0x39, 0x00, 0x33, 0x8a, 0xc0, 0xba, 0x28, 0xa5, 0x95, - 0x4c, 0xe5, 0x77, 0xb7, 0xa0, 0x48, 0x7b, 0xa4, 0x99, 0x0e, 0xa9, 0x25, 0x7a, 0x48, 0x85, 0x0d, - 0x8f, 0x32, 0x75, 0x48, 0x25, 0xd8, 0xf9, 0x74, 0x1f, 0xe4, 0x91, 0xe1, 0x61, 0x27, 0x20, 0x21, - 0x13, 0xa7, 0xfa, 0x5c, 0x69, 0x93, 0x0e, 0x44, 0xea, 0x52, 0xa8, 0xa8, 0x11, 0xa3, 0xcc, 0x01, - 0xf6, 0xe8, 0x83, 0x4f, 0x18, 0xe5, 0x5b, 0xec, 0xae, 0x39, 0x3f, 0x59, 0xd5, 0x76, 0xa8, 0xa0, - 0x73, 0x4d, 0xf4, 0x26, 0xc0, 0x78, 0xd4, 0xe7, 0x76, 0xf1, 0xab, 0x80, 0x3c, 0x1e, 0x31, 0x6d, - 0xb4, 0x09, 0xf3, 0x43, 0xd7, 0xb4, 0x76, 0xad, 0x41, 0xb8, 0x8f, 0x81, 0x35, 0x0c, 0x6f, 0x6d, - 0xd9, 0xd5, 0x37, 0x62, 0x49, 0x32, 0x0e, 0x2c, 0xbb, 0xba, 0x6f, 0x0f, 0xaa, 0x3d, 0xfe, 0x8a, - 0xc6, 0xa0, 0x94, 0xb8, 0x35, 0x19, 0x44, 0x0f, 0x20, 0xc3, 0xdb, 0xb3, 0xf0, 0x05, 0x66, 0xd6, - 0xfa, 0x61, 0x88, 0xdc, 0x1a, 0xad, 0x41, 0xc1, 0xc1, 0x87, 0xf1, 0x16, 0x5c, 0x9e, 0xca, 0xb0, - 0x5c, 0x1b, 0x1f, 0x9e, 0xdf, 0x7f, 0xe7, 0x9c, 0xc9, 0x88, 0x89, 0x1e, 0x43, 0x7e, 0xe4, 0x59, - 0x43, 0xc3, 0x3b, 0xea, 0x87, 0x45, 0x09, 0x57, 0x29, 0xca, 0x88, 0xc3, 0x42, 0x08, 0x3a, 0x8a, - 0xd6, 0x20, 0xec, 0x78, 0xb1, 0x5f, 0xca, 0x52, 0x1f, 0xaf, 0x06, 0xc6, 0x8d, 0x51, 0x1d, 0xf2, - 0xd4, 0xc5, 0xa8, 0xd5, 0xce, 0x51, 0x0f, 0x17, 0x98, 0x87, 0x59, 0xe2, 0xe1, 0x39, 0xed, 0x76, - 0xd6, 0x89, 0xe4, 0x26, 0x5a, 0x07, 0x88, 0x5e, 0x2f, 0xc9, 0xf1, 0x71, 0xd9, 0xe9, 0xbc, 0xc9, - 0x15, 0x27, 0x4b, 0xd2, 0x63, 0xd6, 0x68, 0x03, 0x64, 0x5e, 0x9c, 0xe1, 0xb9, 0x91, 0xbd, 0xf0, - 0x45, 0xe2, 0x2c, 0x55, 0xf0, 0xe4, 0x8a, 0x10, 0x50, 0x1b, 0x52, 0x36, 0x36, 0x7c, 0xcc, 0x0e, - 0x8f, 0x8f, 0x2f, 0x80, 0x3a, 0x55, 0x5e, 0xd5, 0xee, 0x60, 0x1f, 0x0f, 0x8d, 0xc6, 0x3e, 0x69, - 0x44, 0x5b, 0xc4, 0x5e, 0x0f, 0x61, 0x50, 0x1b, 0x14, 0x1a, 0xae, 0x38, 0xeb, 0x28, 0x34, 0x62, - 0x6f, 0xb1, 0x88, 0x15, 0x48, 0xc4, 0x2e, 0x64, 0x1e, 0x9a, 0x4f, 0x1b, 0x13, 0xf6, 0xf9, 0x39, - 0x14, 0x76, 0x5d, 0x6f, 0x68, 0x04, 0x51, 0x95, 0xcc, 0x4f, 0xda, 0xcb, 0x1f, 0x8e, 0x17, 0xf3, - 0x6b, 0x74, 0x94, 0x57, 0x56, 0x7e, 0x37, 0xfe, 0x13, 0x3d, 0xe4, 0x24, 0x7d, 0x8d, 0x72, 0xea, - 0xfb, 0xb3, 0x7a, 0x77, 0x96, 0xa1, 0xdb, 0x90, 0x1e, 0xec, 0xe3, 0xc1, 0x33, 0xbf, 0x74, 0x9d, - 0xc6, 0xfc, 0x27, 0x33, 0x42, 0x35, 0x88, 0xd1, 0xe4, 0x69, 0x48, 0x67, 0x28, 0xe8, 0x0b, 0x28, - 0x98, 0x44, 0x62, 0x39, 0x7b, 0xac, 0x7d, 0xbd, 0x41, 0x71, 0x57, 0x66, 0xc4, 0x25, 0xad, 0xad, - 0xe6, 0xec, 0xba, 0xbc, 0x73, 0xe1, 0x60, 0x61, 0xcb, 0xdb, 0x01, 0x69, 0x97, 0x5c, 0xc5, 0x2d, - 0xec, 0x97, 0x6e, 0x52, 0xdc, 0xcb, 0x1f, 0x85, 0x4f, 0xdf, 0xfe, 0x39, 0xc5, 0x73, 0x90, 0xa8, - 0xd0, 0xa9, 0xe0, 0x88, 0x6c, 0xea, 0x6b, 0x67, 0x0b, 0x9d, 0xdf, 0xfe, 0xa7, 0x5e, 0x02, 0x68, - 0xa1, 0xb3, 0x5f, 0x26, 0x21, 0xbc, 0x03, 0x0b, 0x7f, 0xd5, 0x7f, 0x3e, 0xc6, 0xde, 0x51, 0xa9, - 0x14, 0x23, 0x67, 0x99, 0xc8, 0x1f, 0x13, 0x31, 0xfa, 0x10, 0x64, 0x13, 0x8f, 0xb0, 0x63, 0xfa, - 0x1d, 0xa7, 0x74, 0x8b, 0xb6, 0x46, 0xd7, 0x48, 0xbf, 0xde, 0xe4, 0x42, 0x46, 0xbe, 0x13, 0x2d, - 0xf4, 0x25, 0xe4, 0xc2, 0x1f, 0xd8, 0xec, 0x38, 0xf5, 0xa3, 0x52, 0x99, 0x3a, 0x7d, 0x6f, 0xc6, - 0x60, 0x4e, 0xfa, 0xc0, 0xeb, 0xdc, 0x9f, 0x66, 0x0c, 0x4d, 0x9f, 0xc2, 0x46, 0x5f, 0x40, 0x8e, - 0x67, 0xf7, 0xba, 0xbb, 0xe3, 0x97, 0x5e, 0xbf, 0xf4, 0x06, 0x7b, 0x7a, 0xae, 0x8d, 0x89, 0x29, - 0xe7, 0xad, 0x38, 0x1a, 0xfa, 0x0c, 0xf2, 0xd1, 0xb3, 0x8f, 0x3b, 0x0a, 0xfc, 0xd2, 0x6d, 0x5a, - 0x98, 0xf7, 0x67, 0x4d, 0x5d, 0x66, 0xdb, 0x19, 0x05, 0xbe, 0x9e, 0xf3, 0x63, 0xbf, 0xd0, 0x1d, - 0x90, 0x4d, 0xcf, 0x1d, 0x85, 0xe7, 0xc7, 0x1b, 0x4b, 0xc2, 0x72, 0x92, 0x6f, 0x33, 0x11, 0xd3, - 0x83, 0xa1, 0x0f, 0x05, 0x0f, 0x8f, 0x6c, 0x63, 0x80, 0x87, 0xe4, 0xf8, 0x73, 0x77, 0x4b, 0x0b, - 0x74, 0xf6, 0xd5, 0x99, 0x03, 0x19, 0x19, 0xf3, 0xc4, 0x8c, 0xe1, 0x75, 0x76, 0xd1, 0x16, 0x80, - 0x31, 0x36, 0xad, 0xa0, 0x3f, 0x74, 0x4d, 0x5c, 0x5a, 0xa4, 0x55, 0x39, 0xeb, 0x2e, 0xd5, 0x88, - 0xe1, 0x86, 0x6b, 0xe2, 0xe8, 0x25, 0x85, 0x0b, 0xd0, 0x87, 0x90, 0xa5, 0xae, 0x7d, 0xe9, 0xee, - 0x90, 0xdc, 0x5c, 0xa2, 0xce, 0xcd, 0xb3, 0xbd, 0x94, 0x9b, 0x9e, 0x3b, 0x5a, 0x77, 0x77, 0x68, - 0xc6, 0xb0, 0x4f, 0xb3, 0xfc, 0x9d, 0x00, 0xf3, 0x67, 0x58, 0x0c, 0xfd, 0x0a, 0x32, 0x8e, 0x6b, - 0xc6, 0x1e, 0x61, 0x54, 0x06, 0x92, 0x6e, 0xbb, 0x66, 0xf8, 0x06, 0x73, 0x7f, 0xcf, 0x0a, 0xf6, - 0xc7, 0x3b, 0xd5, 0x81, 0x3b, 0x5c, 0x89, 0x16, 0x6e, 0xee, 0x4c, 0xbe, 0x57, 0x46, 0xcf, 0xf6, - 0x56, 0xe8, 0xd7, 0x68, 0xa7, 0x1a, 0x9a, 0xe9, 0x69, 0x82, 0xaa, 0x99, 0xe8, 0x03, 0x28, 0xe2, - 0xc3, 0x91, 0xe5, 0xc5, 0x4e, 0xf2, 0x44, 0x6c, 0x27, 0x0a, 0x93, 0x41, 0xb2, 0x1f, 0xe5, 0xef, - 0x05, 0x28, 0x9e, 0x62, 0x10, 0xd2, 0xd9, 0xd0, 0x07, 0xbe, 0xa9, 0xce, 0x86, 0x48, 0xa2, 0x9e, - 0x27, 0x71, 0xe9, 0x2b, 0x76, 0xf2, 0x55, 0x5f, 0xb1, 0xa7, 0xef, 0xd3, 0xa9, 0xd9, 0xef, 0xd3, - 0xeb, 0xa2, 0x24, 0x2a, 0xa9, 0xf2, 0x53, 0x90, 0x38, 0x7b, 0x4d, 0xb7, 0x5a, 0xc2, 0x8c, 0xad, - 0xd6, 0x85, 0x7e, 0x96, 0xbf, 0x11, 0x40, 0x8e, 0xff, 0x3d, 0x90, 0x88, 0x50, 0xcf, 0xef, 0xf4, - 0x5e, 0xf2, 0x09, 0x6d, 0x3a, 0x02, 0xc9, 0xd9, 0x23, 0x50, 0x3e, 0x80, 0x6c, 0x8c, 0x00, 0x4e, - 0xb7, 0xe7, 0xc2, 0x4b, 0xb4, 0xe7, 0x6f, 0x41, 0x9a, 0x65, 0x7d, 0x98, 0x48, 0x79, 0x66, 0x9d, - 0x0a, 0x33, 0x3e, 0xf5, 0x25, 0xcd, 0xf6, 0x3f, 0x0a, 0x90, 0x8b, 0x53, 0x03, 0xaa, 0x80, 0x6c, - 0x39, 0x03, 0x8f, 0xd6, 0x25, 0x9d, 0x97, 0xa7, 0xe0, 0x44, 0x4c, 0x08, 0x63, 0x68, 0x39, 0x7d, - 0xfa, 0xac, 0x35, 0x95, 0xa6, 0xd2, 0xd0, 0x72, 0xb6, 0x89, 0x94, 0xaa, 0x18, 0x87, 0x4c, 0x25, - 0x39, 0xa5, 0x62, 0x1c, 0x86, 0x2a, 0x65, 0x7a, 0x06, 0x7b, 0x01, 0xed, 0xa4, 0x93, 0xb1, 0x53, - 0xd5, 0x0b, 0xd0, 0x02, 0x64, 0x0e, 0x2c, 0x2f, 0x18, 0x1b, 0x36, 0x6d, 0x9a, 0x79, 0xf3, 0xcb, - 0x85, 0xe5, 0x7d, 0xc8, 0xc6, 0x28, 0x65, 0x86, 0x0d, 0xfd, 0x29, 0x88, 0x51, 0x51, 0xcd, 0xd8, - 0x1e, 0x53, 0x83, 0xca, 0xdb, 0xfc, 0x0e, 0x03, 0x90, 0xde, 0xdc, 0xaa, 0xb7, 0xb4, 0xc6, 0xb9, - 0xf7, 0x0f, 0x72, 0x53, 0x89, 0x78, 0x88, 0xdc, 0x55, 0x9b, 0x5a, 0xb7, 0x56, 0x6f, 0xa9, 0xe4, - 0xe6, 0x9a, 0x07, 0x59, 0x57, 0x6b, 0x4d, 0x7a, 0xb1, 0x51, 0x84, 0x4f, 0xc4, 0xaf, 0xbf, 0x5d, - 0x14, 0xd6, 0x45, 0x09, 0x29, 0xd7, 0x2a, 0xdf, 0x09, 0x80, 0x9a, 0x46, 0x60, 0x90, 0x12, 0xbb, - 0xc2, 0x25, 0x25, 0x71, 0x89, 0xa7, 0xd3, 0x3d, 0x65, 0xf2, 0x55, 0x7a, 0xca, 0x70, 0xa9, 0x95, - 0x6f, 0x04, 0x80, 0xd8, 0xe2, 0x3e, 0x8d, 0xff, 0xb9, 0x76, 0x71, 0xfb, 0x7c, 0x8a, 0xa5, 0xc9, - 0x05, 0x39, 0xfc, 0xeb, 0xed, 0x01, 0x48, 0x26, 0x73, 0x99, 0x6d, 0xc7, 0x85, 0x7d, 0xea, 0x99, - 0xc8, 0x3c, 0x24, 0x87, 0x12, 0x93, 0xd6, 0x33, 0x90, 0x1a, 0x3b, 0x96, 0xeb, 0xbc, 0xfb, 0x11, - 0xa0, 0xb3, 0xf4, 0x44, 0xc2, 0x4e, 0xbf, 0x8d, 0x00, 0x9b, 0xe1, 0xb5, 0x73, 0xcb, 0x39, 0x88, - 0x04, 0x42, 0xfd, 0xce, 0x8b, 0x7f, 0x2c, 0xcc, 0xbd, 0x38, 0x59, 0x10, 0xfe, 0x72, 0xb2, 0x20, - 0xfc, 0xf5, 0x64, 0x41, 0xf8, 0xfb, 0xc9, 0x82, 0xf0, 0xeb, 0x7f, 0x2e, 0xcc, 0x7d, 0x9e, 0x61, - 0x0b, 0xf8, 0x5f, 0x00, 0x00, 0x00, 0xff, 0xff, 0xab, 0x04, 0xf2, 0xa5, 0xb7, 0x20, 0x00, 0x00, + // 3135 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xa4, 0x59, 0xdd, 0x6f, 0xdc, 0xc6, + 0xb5, 0x17, 0xf7, 0x93, 0x3c, 0xdc, 0x0f, 0x6a, 0x2c, 0x3b, 0xeb, 0x8d, 0x23, 0xc9, 0x9b, 0x38, + 0x57, 0xf9, 0x5a, 0x39, 0x72, 0xee, 0xbd, 0x41, 0x72, 0x11, 0xdc, 0xfd, 0xa0, 0x6c, 0xca, 0xab, + 0x5d, 0x99, 0xbb, 0x92, 0xe3, 0x20, 0xed, 0x82, 0x5a, 0x8e, 0x24, 0xc6, 0x5c, 0x72, 0x4d, 0x72, + 0x15, 0xe9, 0x3f, 0xc8, 0x63, 0xd1, 0x87, 0xf6, 0x2d, 0x08, 0xf2, 0x54, 0xa0, 0xe8, 0x5b, 0x1f, + 0xfa, 0x27, 0xb8, 0x6f, 0x45, 0x9e, 0xfa, 0x52, 0xa3, 0x55, 0x51, 0xa0, 0x7f, 0x41, 0x1f, 0x02, + 0x14, 0x28, 0x66, 0x38, 0xc3, 0xe5, 0xea, 0x2b, 0x2b, 0xfb, 0x8d, 0x3c, 0x73, 0xce, 0x8f, 0x73, + 0xce, 0x9c, 0xf3, 0x9b, 0x33, 0x43, 0xb8, 0xe5, 0x3f, 0xb3, 0x57, 0xfd, 0x67, 0xf6, 0xae, 0xe1, + 0xe3, 0x55, 0x3f, 0xf0, 0xc6, 0x83, 0x60, 0xec, 0x61, 0xb3, 0x3a, 0xf2, 0xdc, 0xc0, 0x45, 0xd7, + 0x07, 0xee, 0xe0, 0xa9, 0xe7, 0x1a, 0x83, 0x83, 0xaa, 0xff, 0xcc, 0xae, 0x32, 0xbd, 0x72, 0x69, + 0x1c, 0x58, 0xf6, 0xea, 0x81, 0x3d, 0x58, 0x0d, 0xac, 0x21, 0xf6, 0x03, 0x63, 0x38, 0x0a, 0x0d, + 0xca, 0xaf, 0xc7, 0xe1, 0x46, 0x9e, 0x75, 0x68, 0xd9, 0x78, 0x1f, 0xb3, 0xc1, 0x85, 0x7d, 0x77, + 0xdf, 0xa5, 0x8f, 0xab, 0xe4, 0x29, 0x94, 0x56, 0x7e, 0xc8, 0x02, 0x34, 0x5c, 0x7b, 0x3c, 0x74, + 0x7a, 0xc7, 0x23, 0x8c, 0x9e, 0x40, 0xde, 0xc7, 0x43, 0xc3, 0x09, 0xac, 0x41, 0x3f, 0x38, 0x1e, + 0xe1, 0x92, 0xb0, 0x2c, 0xac, 0x14, 0xd6, 0xaa, 0xd5, 0x73, 0xa7, 0x52, 0x9d, 0x58, 0x56, 0xbb, + 0xcc, 0x8c, 0xbc, 0xd4, 0x53, 0xcf, 0x5f, 0x2c, 0xcd, 0xe9, 0x39, 0x3f, 0x26, 0x43, 0x65, 0x48, + 0x7f, 0x6d, 0x99, 0xc1, 0x41, 0x29, 0xb1, 0x2c, 0xac, 0xa4, 0x99, 0x4a, 0x28, 0x42, 0x15, 0x90, + 0x46, 0x1e, 0x1e, 0x58, 0xbe, 0xe5, 0x3a, 0xa5, 0x64, 0x6c, 0x7c, 0x22, 0x46, 0xef, 0x80, 0x62, + 0x78, 0x9e, 0x71, 0xdc, 0x37, 0xad, 0x21, 0x76, 0x88, 0xc8, 0x2f, 0xa5, 0x96, 0x93, 0x2b, 0x69, + 0xbd, 0x48, 0xe5, 0xcd, 0x48, 0x8c, 0x6e, 0x40, 0xc6, 0x76, 0x07, 0x86, 0x8d, 0x4b, 0xe9, 0x65, + 0x61, 0x45, 0xd2, 0xd9, 0x1b, 0xda, 0x81, 0xdc, 0xa1, 0xe5, 0x5b, 0xbb, 0x36, 0x0e, 0x9d, 0xcb, + 0x50, 0xe7, 0x3e, 0xf8, 0x69, 0xe7, 0x76, 0x42, 0xab, 0x98, 0x6f, 0xf2, 0xe1, 0x44, 0x84, 0xb6, + 0xa1, 0x10, 0x4e, 0x6d, 0xe0, 0x3a, 0x01, 0x76, 0x02, 0xbf, 0x94, 0x7d, 0x99, 0xb0, 0xe9, 0x79, + 0x8a, 0xd2, 0x60, 0x20, 0xa8, 0x0d, 0x85, 0x60, 0x3c, 0xb2, 0xf1, 0x04, 0x56, 0x5c, 0x4e, 0xae, + 0xc8, 0x6b, 0xb7, 0x7f, 0x12, 0x96, 0x4d, 0x32, 0x4f, 0xcd, 0x23, 0xbc, 0xdb, 0x90, 0x0b, 0xf1, + 0x6c, 0x63, 0x17, 0xdb, 0x7e, 0x49, 0x5a, 0x4e, 0xae, 0x48, 0xba, 0x4c, 0x65, 0x2d, 0x2a, 0xaa, + 0xfc, 0x26, 0x01, 0xb9, 0xf8, 0x94, 0x90, 0x08, 0xa9, 0x7a, 0xa7, 0xd3, 0x52, 0xe6, 0x50, 0x16, + 0x92, 0x5a, 0xbb, 0xa7, 0x08, 0x48, 0x82, 0xf4, 0x7a, 0xab, 0x53, 0xeb, 0x29, 0x09, 0x24, 0x43, + 0xb6, 0xa9, 0x36, 0xb4, 0xcd, 0x5a, 0x4b, 0x49, 0x12, 0xd5, 0x66, 0xad, 0xa7, 0x2a, 0x29, 0x94, + 0x07, 0xa9, 0xa7, 0x6d, 0xaa, 0xdd, 0x5e, 0x6d, 0x73, 0x4b, 0x49, 0xa3, 0x1c, 0x88, 0x5a, 0xbb, + 0xa7, 0xea, 0x3b, 0xb5, 0x96, 0x92, 0x41, 0x00, 0x99, 0x6e, 0x4f, 0xd7, 0xda, 0xf7, 0x95, 0x2c, + 0x81, 0xaa, 0x3f, 0xe9, 0xa9, 0x5d, 0x45, 0x44, 0x45, 0x90, 0x23, 0x9b, 0xde, 0x17, 0x8a, 0x84, + 0x10, 0x14, 0x1a, 0x9d, 0x56, 0xab, 0xd6, 0x53, 0x9b, 0x4c, 0x1f, 0xc8, 0x27, 0xda, 0xb5, 0x4d, + 0x55, 0x91, 0xc9, 0x6c, 0x3a, 0x5a, 0x53, 0xc9, 0x51, 0xd1, 0x76, 0xab, 0xa5, 0xe4, 0xc9, 0xd3, + 0xf6, 0xb6, 0xd6, 0x54, 0x0a, 0x04, 0xb6, 0xa6, 0xeb, 0xb5, 0x27, 0x4a, 0x91, 0x08, 0xb5, 0xb6, + 0xda, 0x53, 0x14, 0xf2, 0x44, 0x3e, 0xa0, 0xcc, 0x93, 0xe1, 0x8d, 0x6e, 0xa7, 0x5d, 0x57, 0x10, + 0x79, 0xec, 0x6d, 0x6f, 0xb5, 0x54, 0x65, 0x81, 0x20, 0xd6, 0xb5, 0x9e, 0x72, 0x1d, 0x15, 0x01, + 0xb4, 0x76, 0x6f, 0x6d, 0x47, 0x6d, 0xf4, 0x3a, 0xba, 0xf2, 0x5c, 0x40, 0x05, 0x90, 0x3a, 0x5a, + 0x93, 0xbd, 0xff, 0x51, 0xa8, 0xa4, 0xc4, 0x6b, 0xca, 0xb5, 0xca, 0x2f, 0x05, 0x90, 0x63, 0x79, + 0x41, 0x27, 0xd2, 0x69, 0xab, 0xca, 0x1c, 0x89, 0x0a, 0xf1, 0xf7, 0xbe, 0xaa, 0x2b, 0x02, 0x71, + 0xbe, 0xbb, 0x59, 0x6b, 0xb5, 0x48, 0xec, 0x12, 0xc4, 0xf9, 0xba, 0x76, 0x9f, 0x3c, 0xd3, 0x78, + 0xe9, 0x6a, 0xad, 0xa5, 0xa4, 0xd1, 0x02, 0x28, 0xcd, 0xce, 0x76, 0xbd, 0xa5, 0xf6, 0xb7, 0x74, + 0xb5, 0xa1, 0x75, 0xb5, 0x4e, 0x5b, 0xc9, 0x10, 0x98, 0x9d, 0x9a, 0xde, 0x78, 0x50, 0xd3, 0x95, + 0x2c, 0x51, 0xa6, 0x4f, 0x22, 0x99, 0xf2, 0x23, 0xfa, 0x28, 0x11, 0xb4, 0x9d, 0x9a, 0x4e, 0x66, + 0x0d, 0x95, 0x94, 0x98, 0x52, 0x52, 0x9f, 0xa4, 0xfe, 0xf9, 0xdd, 0x92, 0x50, 0xf9, 0x57, 0x12, + 0xae, 0xad, 0xbb, 0x1e, 0xb6, 0xf6, 0x9d, 0x87, 0xf8, 0x58, 0xc7, 0x7b, 0xd8, 0xc3, 0xce, 0x00, + 0xa3, 0x65, 0x48, 0x07, 0xc6, 0xae, 0x1d, 0x56, 0x75, 0xbe, 0x0e, 0x24, 0x49, 0x7e, 0x7c, 0xb1, + 0x94, 0xd0, 0x9a, 0x7a, 0x38, 0x80, 0xee, 0x40, 0xda, 0x72, 0x4c, 0x7c, 0x44, 0x8b, 0x34, 0x5f, + 0x2f, 0x32, 0x8d, 0xac, 0x46, 0x84, 0x44, 0x8d, 0x8e, 0xa2, 0x12, 0xa4, 0x1c, 0x63, 0x88, 0x69, + 0xa9, 0x4a, 0x2c, 0xd9, 0xa8, 0x04, 0x3d, 0x04, 0xf1, 0xd0, 0xb0, 0x2d, 0xd3, 0x0a, 0x8e, 0x4b, + 0x29, 0x5a, 0x04, 0xef, 0x5c, 0x98, 0xad, 0x8e, 0x1f, 0x78, 0x86, 0xe5, 0x04, 0x3b, 0xcc, 0x80, + 0x01, 0x45, 0x00, 0xe8, 0x2e, 0xcc, 0xfb, 0x07, 0x86, 0x87, 0xcd, 0xfe, 0xc8, 0xc3, 0x7b, 0xd6, + 0x51, 0xdf, 0xc6, 0x0e, 0x2d, 0x69, 0x4e, 0x0f, 0xc5, 0x70, 0x78, 0x8b, 0x8e, 0xb6, 0xb0, 0x83, + 0x7a, 0x20, 0xb9, 0x4e, 0xdf, 0xc4, 0x36, 0x0e, 0x78, 0x79, 0x7f, 0x78, 0xc1, 0xf7, 0xcf, 0x09, + 0x50, 0xb5, 0x36, 0x08, 0x2c, 0xd7, 0xe1, 0xf3, 0x70, 0x9d, 0x26, 0x05, 0x62, 0xa8, 0xe3, 0x91, + 0x69, 0x04, 0x98, 0x95, 0xf6, 0xab, 0xa0, 0x6e, 0x53, 0xa0, 0xca, 0x23, 0xc8, 0x84, 0x23, 0xa4, + 0x5e, 0xda, 0x9d, 0x7e, 0xad, 0xd1, 0x23, 0x0b, 0x3f, 0x47, 0x52, 0x46, 0x57, 0x49, 0xce, 0x37, + 0x7a, 0x2c, 0x81, 0xd4, 0x5e, 0x9f, 0x26, 0x79, 0x82, 0x94, 0x09, 0x79, 0x6b, 0xaa, 0xeb, 0xb5, + 0xed, 0x16, 0xc9, 0x22, 0x19, 0xb2, 0x8d, 0x5a, 0xb7, 0x51, 0x6b, 0xaa, 0x4a, 0xaa, 0xf2, 0x97, + 0x04, 0x28, 0x21, 0x0b, 0x34, 0xb1, 0x3f, 0xf0, 0xac, 0x51, 0xe0, 0x7a, 0xd1, 0x62, 0x09, 0x67, + 0x16, 0xeb, 0x6d, 0x48, 0x58, 0x26, 0x5b, 0xea, 0x1b, 0x44, 0x7e, 0x42, 0x93, 0xe1, 0xc7, 0x17, + 0x4b, 0x62, 0x88, 0xa2, 0x35, 0xf5, 0x84, 0x65, 0xa2, 0x4f, 0x21, 0x45, 0xf9, 0x92, 0x2c, 0xf7, + 0x15, 0xe8, 0x87, 0x1a, 0xa1, 0x65, 0x10, 0x9d, 0xb1, 0x6d, 0xd3, 0xbc, 0x23, 0x19, 0x21, 0xf2, + 0x40, 0x70, 0x29, 0xe1, 0x25, 0x13, 0xef, 0x19, 0x63, 0x3b, 0xe8, 0xe3, 0xa3, 0x91, 0xc7, 0x48, + 0x5b, 0x66, 0x32, 0xf5, 0x68, 0xe4, 0xa1, 0x5b, 0x90, 0x39, 0xb0, 0x4c, 0x13, 0x3b, 0x74, 0x51, + 0x39, 0x04, 0x93, 0xa1, 0x35, 0x98, 0x1f, 0xfb, 0xd8, 0xef, 0xfb, 0xf8, 0xd9, 0x98, 0x44, 0xbc, + 0x6f, 0x99, 0x7e, 0x09, 0x96, 0x93, 0x2b, 0xf9, 0x7a, 0x86, 0xe5, 0x77, 0x91, 0x28, 0x74, 0xd9, + 0xb8, 0x66, 0x52, 0x32, 0x1c, 0xb8, 0xc3, 0xd1, 0x38, 0xc0, 0xe1, 0x47, 0xe5, 0xf0, 0xa3, 0x4c, + 0x46, 0x3e, 0xba, 0x91, 0x12, 0x45, 0x45, 0xda, 0x48, 0x89, 0x92, 0x02, 0x1b, 0x29, 0x31, 0xab, + 0x88, 0x95, 0x6f, 0x12, 0x70, 0x23, 0x74, 0x73, 0xdd, 0x18, 0x5a, 0xf6, 0xf1, 0xab, 0x46, 0x39, + 0x44, 0x61, 0x51, 0xa6, 0x33, 0x22, 0xd8, 0x7d, 0x62, 0xe6, 0x97, 0x92, 0x21, 0x3d, 0x87, 0xb2, + 0x36, 0x11, 0xa1, 0x8f, 0x01, 0x98, 0x0a, 0xf1, 0x30, 0x45, 0x3d, 0xbc, 0x79, 0xf2, 0x62, 0x49, + 0xe2, 0xcb, 0xe5, 0x4f, 0xad, 0x9d, 0x14, 0x2a, 0x13, 0x77, 0x3b, 0x30, 0xcf, 0x63, 0x1c, 0x21, + 0xd0, 0x40, 0xe7, 0xeb, 0x6f, 0xb2, 0x39, 0x15, 0x9b, 0xa1, 0x02, 0x37, 0x9f, 0x82, 0x2a, 0x9a, + 0x53, 0x83, 0x66, 0xe5, 0xb7, 0x09, 0x58, 0xd0, 0x9c, 0x00, 0x7b, 0x36, 0x36, 0x0e, 0x71, 0x2c, + 0x10, 0x9f, 0x83, 0x64, 0x38, 0x03, 0xec, 0x07, 0xae, 0xe7, 0x97, 0x04, 0xba, 0x61, 0x7d, 0x74, + 0x41, 0xc6, 0x9c, 0x67, 0x5f, 0xad, 0x31, 0x63, 0xde, 0x01, 0x44, 0x60, 0xe5, 0x3f, 0x08, 0x20, + 0xf2, 0x51, 0x74, 0x17, 0x44, 0x4a, 0x59, 0xc4, 0x8f, 0x90, 0xce, 0xae, 0x33, 0x3f, 0xb2, 0x3d, + 0x22, 0xa7, 0xf3, 0x27, 0x2b, 0x9f, 0xa5, 0x6a, 0x9a, 0x89, 0xfe, 0x1b, 0x44, 0xca, 0x5e, 0xfd, + 0x68, 0x35, 0xca, 0xdc, 0x82, 0xd1, 0x5b, 0x9c, 0xe9, 0xb2, 0x54, 0x57, 0x33, 0x51, 0xe3, 0x3c, + 0x12, 0x4a, 0x52, 0xfb, 0xd7, 0x78, 0xe4, 0xba, 0xd3, 0x34, 0x74, 0x86, 0x97, 0x2a, 0xff, 0x48, + 0xc2, 0x8d, 0x2d, 0xc3, 0x0b, 0x2c, 0x52, 0xef, 0x96, 0xb3, 0x1f, 0x8b, 0xd7, 0x1d, 0x90, 0x9d, + 0xf1, 0x90, 0xad, 0x8a, 0xcf, 0x7c, 0x09, 0x7d, 0x07, 0x67, 0x3c, 0x0c, 0x03, 0xee, 0xa3, 0x16, + 0xa4, 0x6c, 0xcb, 0x0f, 0x4a, 0x09, 0x1a, 0xd1, 0xb5, 0x0b, 0x22, 0x7a, 0xfe, 0x37, 0xaa, 0x2d, + 0xcb, 0x0f, 0x78, 0x4e, 0x12, 0x14, 0xd4, 0x81, 0xb4, 0x67, 0x38, 0xfb, 0x98, 0x26, 0x99, 0xbc, + 0x76, 0xef, 0x6a, 0x70, 0x3a, 0x31, 0xe5, 0x1d, 0x1c, 0xc5, 0x29, 0xff, 0x5a, 0x80, 0x14, 0xf9, + 0xca, 0x25, 0x75, 0x70, 0x03, 0x32, 0x87, 0x86, 0x3d, 0xc6, 0x3e, 0xf5, 0x21, 0xa7, 0xb3, 0x37, + 0xf4, 0x33, 0x28, 0xfa, 0xe3, 0xdd, 0x51, 0xec, 0x53, 0x8c, 0x68, 0x3e, 0xb8, 0xd2, 0xac, 0xa2, + 0x2d, 0x61, 0x1a, 0xab, 0xfc, 0x14, 0xd2, 0x74, 0xbe, 0x97, 0xcc, 0x8c, 0x34, 0x46, 0x6e, 0x1f, + 0x1f, 0x0d, 0xec, 0xb1, 0x6f, 0x1d, 0x62, 0x9a, 0x1d, 0x39, 0x5d, 0x0e, 0x5c, 0x95, 0x8b, 0xd0, + 0x1d, 0x28, 0xec, 0x79, 0xee, 0xb0, 0x6f, 0x39, 0x5c, 0x29, 0x49, 0x95, 0xf2, 0x44, 0xaa, 0x71, + 0x61, 0xe5, 0xdf, 0x22, 0x14, 0x69, 0x06, 0xcd, 0xc4, 0x0c, 0x77, 0x62, 0xcc, 0x70, 0x7d, 0x8a, + 0x19, 0xa2, 0x34, 0x24, 0xc4, 0x70, 0x0b, 0x32, 0x63, 0xc7, 0x7a, 0x36, 0x0e, 0xbf, 0x19, 0x91, + 0x5f, 0x28, 0x3b, 0x43, 0x1b, 0xa9, 0xb3, 0xb4, 0xf1, 0x3e, 0x20, 0x52, 0x33, 0xb8, 0x3f, 0xa5, + 0x98, 0xa6, 0x8a, 0x0a, 0x1d, 0x69, 0x5c, 0x48, 0x32, 0x99, 0x2b, 0x90, 0xcc, 0x03, 0x50, 0xf0, + 0x51, 0xe0, 0x19, 0xfd, 0x98, 0x7d, 0x96, 0xda, 0x2f, 0x9e, 0xbc, 0x58, 0x2a, 0xa8, 0x64, 0xec, + 0x7c, 0x90, 0x02, 0x8e, 0x8d, 0x99, 0x24, 0x27, 0xe6, 0x19, 0x86, 0x69, 0x79, 0x98, 0xee, 0x92, + 0x61, 0xf7, 0x5b, 0x58, 0xbb, 0x7b, 0x21, 0x99, 0x4c, 0x85, 0xbd, 0xda, 0xe4, 0x86, 0xba, 0x12, + 0x42, 0x45, 0x02, 0x1f, 0x3d, 0x02, 0x79, 0x2f, 0xdc, 0xa8, 0xfb, 0x4f, 0xf1, 0x71, 0x49, 0xa2, + 0xe9, 0xf6, 0xee, 0xec, 0x5b, 0x3a, 0xaf, 0xcf, 0xbd, 0x68, 0x08, 0x6d, 0x43, 0xde, 0xe3, 0xc3, + 0x66, 0x7f, 0xf7, 0x98, 0xee, 0x3f, 0x2f, 0x03, 0x9a, 0x9b, 0xc0, 0xd4, 0x8f, 0xd1, 0x23, 0x00, + 0x2b, 0x62, 0x49, 0xba, 0x49, 0xc9, 0x6b, 0xef, 0x5d, 0x81, 0x4e, 0xf9, 0x4c, 0x27, 0x20, 0xe8, + 0x31, 0x14, 0x26, 0x6f, 0x74, 0xaa, 0xb9, 0x97, 0x9c, 0x6a, 0x3e, 0x86, 0x53, 0x3f, 0x46, 0x3d, + 0x58, 0x20, 0xdb, 0xa7, 0xeb, 0x5b, 0x01, 0x8e, 0xa7, 0x40, 0x9e, 0xa6, 0x40, 0xe5, 0xe4, 0xc5, + 0x12, 0x6a, 0xf0, 0xf1, 0xf3, 0xd3, 0x00, 0x0d, 0x4e, 0x8d, 0x87, 0x49, 0x35, 0x95, 0xbc, 0x04, + 0xb1, 0x30, 0x49, 0xaa, 0xee, 0x24, 0x7d, 0xcf, 0x24, 0x55, 0x2c, 0xb5, 0x09, 0xd2, 0x63, 0xc8, + 0x4d, 0xb1, 0x4c, 0xf1, 0xe5, 0x59, 0x66, 0x0a, 0x08, 0xa9, 0xac, 0x3f, 0x52, 0x68, 0x6b, 0xf8, + 0xde, 0x8c, 0x09, 0x7a, 0xba, 0x53, 0xaa, 0x2c, 0x82, 0x14, 0xe5, 0x28, 0x39, 0x8e, 0xd4, 0xba, + 0x0d, 0x65, 0x8e, 0x1e, 0xab, 0xd4, 0x6e, 0x43, 0x11, 0x2a, 0xb7, 0x21, 0x45, 0x4f, 0x1a, 0x32, + 0x64, 0xd7, 0x3b, 0xfa, 0xe3, 0x9a, 0xde, 0x0c, 0x9b, 0x45, 0xad, 0xbd, 0xa3, 0xea, 0x3d, 0xb5, + 0xa9, 0x08, 0x95, 0xef, 0x53, 0x80, 0x26, 0x9f, 0xd8, 0x1c, 0x07, 0x06, 0x05, 0xab, 0x41, 0x26, + 0x8c, 0x1e, 0x25, 0x21, 0x79, 0xed, 0xbf, 0x2e, 0x6d, 0xe1, 0x26, 0x00, 0x0f, 0xe6, 0x74, 0x66, + 0x88, 0x3e, 0x8b, 0x9f, 0x0c, 0xe4, 0xb5, 0xb7, 0x67, 0x73, 0xf2, 0xc1, 0x1c, 0x3f, 0x32, 0x3c, + 0x84, 0xb4, 0x1f, 0x90, 0xfe, 0x39, 0x49, 0x83, 0xb4, 0x7a, 0x81, 0xfd, 0xd9, 0xc9, 0x57, 0xbb, + 0xc4, 0x8c, 0xef, 0x36, 0x14, 0x03, 0x3d, 0x06, 0x29, 0xe2, 0x05, 0x76, 0xcc, 0xb8, 0x37, 0x3b, + 0x60, 0x14, 0x64, 0xde, 0x62, 0x44, 0x58, 0xa8, 0x06, 0xf2, 0x90, 0xa9, 0x4d, 0x1a, 0xa4, 0x65, + 0x46, 0xcd, 0xc0, 0x11, 0x28, 0x45, 0xc7, 0xde, 0x74, 0xe0, 0x46, 0x9a, 0x49, 0xfa, 0x5d, 0xcf, + 0xb5, 0xed, 0x5d, 0x63, 0xf0, 0x94, 0x9e, 0x15, 0xa2, 0x7e, 0x97, 0x4b, 0x2b, 0xff, 0x0f, 0x69, + 0xea, 0x13, 0x59, 0xc8, 0xed, 0xf6, 0xc3, 0x76, 0xe7, 0x31, 0xe9, 0xfa, 0x8b, 0x20, 0x37, 0xd5, + 0x96, 0xda, 0x53, 0xfb, 0x9d, 0x76, 0xeb, 0x89, 0x22, 0xa0, 0x9b, 0x70, 0x9d, 0x09, 0x6a, 0xed, + 0x66, 0xff, 0xb1, 0xae, 0xf1, 0xa1, 0x44, 0x65, 0x25, 0x9e, 0x29, 0x93, 0x83, 0x27, 0xc9, 0x99, + 0x66, 0x53, 0x11, 0x68, 0xce, 0xe8, 0x9d, 0x2d, 0x25, 0x51, 0xcf, 0x01, 0x98, 0x51, 0x04, 0x36, + 0x52, 0x62, 0x46, 0xc9, 0x56, 0x7e, 0xf7, 0x3a, 0x14, 0x69, 0x8f, 0x34, 0xd3, 0x26, 0xb5, 0x4c, + 0x37, 0xa9, 0xb0, 0xe1, 0x51, 0xa6, 0x36, 0xa9, 0x04, 0xdb, 0x9f, 0xee, 0x81, 0x34, 0x32, 0x3c, + 0xec, 0x04, 0x24, 0x64, 0xa9, 0xa9, 0x3e, 0x57, 0xdc, 0xa2, 0x03, 0x91, 0xba, 0x18, 0x2a, 0x6a, + 0xc4, 0x28, 0x7b, 0x88, 0x3d, 0x7a, 0xe1, 0x13, 0x46, 0xf9, 0x26, 0x3b, 0x6b, 0xce, 0x4f, 0x66, + 0xb5, 0x13, 0x2a, 0xe8, 0x5c, 0x13, 0xbd, 0x09, 0x30, 0x1e, 0xf5, 0xb9, 0x5d, 0xfc, 0x28, 0x20, + 0x8d, 0x47, 0x4c, 0x1b, 0x6d, 0xc1, 0xfc, 0xd0, 0x35, 0xad, 0x3d, 0x6b, 0x10, 0xae, 0x63, 0x60, + 0x0d, 0xc3, 0x53, 0x9b, 0xbc, 0xf6, 0x46, 0x2c, 0x49, 0xc6, 0x81, 0x65, 0x57, 0x0f, 0xec, 0x41, + 0xb5, 0xc7, 0x6f, 0xd1, 0x18, 0x94, 0x12, 0xb7, 0x26, 0x83, 0xe8, 0x3e, 0x64, 0x79, 0x7b, 0x16, + 0xde, 0xc0, 0xcc, 0x5a, 0x3f, 0x0c, 0x91, 0x5b, 0xa3, 0x75, 0x28, 0x38, 0xf8, 0x28, 0xde, 0x82, + 0x4b, 0x53, 0x19, 0x96, 0x6b, 0xe3, 0xa3, 0xf3, 0xfb, 0xef, 0x9c, 0x33, 0x19, 0x31, 0xd1, 0x23, + 0xc8, 0x8f, 0x3c, 0x6b, 0x68, 0x78, 0xc7, 0xfd, 0xb0, 0x28, 0xe1, 0x2a, 0x45, 0x19, 0x71, 0x58, + 0x08, 0x41, 0x47, 0xd1, 0x3a, 0x84, 0x1d, 0x2f, 0xf6, 0x4b, 0x32, 0xf5, 0xf1, 0x6a, 0x60, 0xdc, + 0x18, 0xd5, 0x21, 0x4f, 0x5d, 0x8c, 0x5a, 0xed, 0x1c, 0xf5, 0x70, 0x91, 0x79, 0x28, 0x13, 0x0f, + 0xcf, 0x69, 0xb7, 0x65, 0x27, 0x92, 0x9b, 0x68, 0x03, 0x20, 0xba, 0xbd, 0x24, 0xdb, 0xc7, 0x65, + 0xbb, 0xf3, 0x16, 0x57, 0x9c, 0x4c, 0x49, 0x8f, 0x59, 0xa3, 0x4d, 0x90, 0x78, 0x71, 0x86, 0xfb, + 0x86, 0x7c, 0xe1, 0x8d, 0xc4, 0x59, 0xaa, 0xe0, 0xc9, 0x15, 0x21, 0xa0, 0x36, 0xa4, 0x6d, 0x6c, + 0xf8, 0x98, 0x6d, 0x1e, 0x1f, 0x5f, 0x00, 0x75, 0xaa, 0xbc, 0xaa, 0xdd, 0xc1, 0x01, 0x1e, 0x1a, + 0x8d, 0x03, 0xd2, 0x88, 0xb6, 0x88, 0xbd, 0x1e, 0xc2, 0xa0, 0x36, 0x28, 0x34, 0x5c, 0x71, 0xd6, + 0x51, 0x68, 0xc4, 0xde, 0x62, 0x11, 0x2b, 0x90, 0x88, 0x5d, 0xc8, 0x3c, 0x34, 0x9f, 0x36, 0x27, + 0xec, 0xf3, 0x7f, 0x50, 0xd8, 0x73, 0xbd, 0xa1, 0x11, 0x44, 0x55, 0x32, 0x3f, 0x69, 0x2f, 0x7f, + 0x7c, 0xb1, 0x94, 0x5f, 0xa7, 0xa3, 0xbc, 0xb2, 0xf2, 0x7b, 0xf1, 0x57, 0xf4, 0x80, 0x93, 0xf4, + 0x35, 0xca, 0xa9, 0xef, 0xcf, 0xea, 0xdd, 0x59, 0x86, 0x6e, 0x43, 0x66, 0x70, 0x80, 0x07, 0x4f, + 0xfd, 0xd2, 0x02, 0x8d, 0xf9, 0xff, 0xcc, 0x08, 0xd5, 0x20, 0x46, 0x93, 0xab, 0x21, 0x9d, 0xa1, + 0xa0, 0x2f, 0xa1, 0x60, 0x12, 0x89, 0xe5, 0xec, 0xb3, 0xf6, 0xf5, 0x3a, 0xc5, 0x5d, 0x9d, 0x11, + 0x97, 0xb4, 0xb6, 0x9a, 0xb3, 0xe7, 0xf2, 0xce, 0x85, 0x83, 0x85, 0x2d, 0x6f, 0x07, 0xc4, 0x3d, + 0x72, 0x14, 0xb7, 0xb0, 0x5f, 0xba, 0x41, 0x71, 0x2f, 0xbf, 0x14, 0x3e, 0x7d, 0xfa, 0xe7, 0x14, + 0xcf, 0x41, 0xa2, 0x42, 0xa7, 0x82, 0x63, 0xb2, 0xa8, 0xaf, 0x9d, 0x2d, 0x74, 0x7e, 0xfa, 0x9f, + 0xba, 0x09, 0xa0, 0x85, 0xce, 0xde, 0x4c, 0x42, 0x78, 0x87, 0x16, 0xfe, 0xba, 0xff, 0x6c, 0x8c, + 0xbd, 0xe3, 0x52, 0x29, 0x46, 0xce, 0x12, 0x91, 0x3f, 0x22, 0x62, 0xf4, 0x21, 0x48, 0x26, 0x1e, + 0x61, 0xc7, 0xf4, 0x3b, 0x4e, 0xe9, 0x26, 0x6d, 0x8d, 0xae, 0x91, 0x7e, 0xbd, 0xc9, 0x85, 0x8c, + 0x7c, 0x27, 0x5a, 0xe8, 0x2b, 0xc8, 0x85, 0x2f, 0xd8, 0xec, 0x38, 0xf5, 0xe3, 0x52, 0x99, 0x3a, + 0x7d, 0x77, 0xc6, 0x60, 0x4e, 0xfa, 0xc0, 0x05, 0xee, 0x4f, 0x33, 0x86, 0xa6, 0x4f, 0x61, 0xa3, + 0x2f, 0x21, 0xc7, 0xb3, 0x7b, 0xc3, 0xdd, 0xf5, 0x4b, 0xaf, 0x5f, 0x7a, 0x82, 0x3d, 0xfd, 0xad, + 0xcd, 0x89, 0x29, 0xe7, 0xad, 0x38, 0x1a, 0xfa, 0x1c, 0xf2, 0xd1, 0xb5, 0x8f, 0x3b, 0x0a, 0xfc, + 0xd2, 0x2d, 0x5a, 0x98, 0xf7, 0x66, 0x4d, 0x5d, 0x66, 0xdb, 0x19, 0x05, 0xbe, 0x9e, 0xf3, 0x63, + 0x6f, 0xe8, 0x36, 0x48, 0xa6, 0xe7, 0x8e, 0xc2, 0xfd, 0xe3, 0x8d, 0x65, 0x61, 0x25, 0xc9, 0x97, + 0x99, 0x88, 0xe9, 0xc6, 0xd0, 0x87, 0x82, 0x87, 0x47, 0xb6, 0x31, 0xc0, 0x43, 0xb2, 0xfd, 0xb9, + 0x7b, 0xa5, 0x45, 0xfa, 0xf5, 0xb5, 0x99, 0x03, 0x19, 0x19, 0xf3, 0xc4, 0x8c, 0xe1, 0x75, 0xf6, + 0xd0, 0x36, 0x80, 0x31, 0x36, 0xad, 0xa0, 0x3f, 0x74, 0x4d, 0x5c, 0x5a, 0xa2, 0x55, 0x39, 0xeb, + 0x2a, 0xd5, 0x88, 0xe1, 0xa6, 0x6b, 0xe2, 0xe8, 0x26, 0x85, 0x0b, 0xd0, 0x87, 0x20, 0x53, 0xd7, + 0xbe, 0x72, 0x77, 0x49, 0x6e, 0x2e, 0x53, 0xe7, 0xe6, 0xd9, 0x5a, 0x4a, 0x4d, 0xcf, 0x1d, 0x6d, + 0xb8, 0xbb, 0x34, 0x63, 0xd8, 0xa3, 0x89, 0x7c, 0xc8, 0xed, 0x0f, 0xfa, 0x13, 0x2a, 0xbd, 0x4d, + 0x57, 0xf1, 0xd3, 0x19, 0xe7, 0x72, 0xbf, 0x71, 0x0e, 0xb9, 0x5e, 0xe3, 0x7b, 0xc2, 0xfd, 0x06, + 0x97, 0xf9, 0xba, 0xbc, 0x3f, 0x88, 0x5e, 0xca, 0xdf, 0x0b, 0x30, 0x7f, 0x86, 0x3a, 0xd1, 0xcf, + 0x21, 0xeb, 0xb8, 0x66, 0xec, 0xe6, 0x47, 0x65, 0x40, 0x99, 0xb6, 0x6b, 0x86, 0x17, 0x3f, 0xf7, + 0xf6, 0xad, 0xe0, 0x60, 0xbc, 0x5b, 0x1d, 0xb8, 0xc3, 0xd5, 0x68, 0x86, 0xe6, 0xee, 0xe4, 0x79, + 0x75, 0xf4, 0x74, 0x7f, 0x95, 0x3e, 0x8d, 0x76, 0xab, 0xa1, 0x99, 0x9e, 0x21, 0xa8, 0x9a, 0x89, + 0x3e, 0x80, 0x22, 0x3e, 0x1a, 0x59, 0x5e, 0xac, 0x7d, 0x48, 0xc4, 0x96, 0xbf, 0x30, 0x19, 0x24, + 0x49, 0x50, 0xfe, 0x41, 0x80, 0xe2, 0x29, 0xda, 0x22, 0xed, 0x14, 0xbd, 0x55, 0x9c, 0x6a, 0xa7, + 0x88, 0x24, 0x6a, 0xb4, 0x12, 0x97, 0x5e, 0x9d, 0x27, 0x5f, 0xf5, 0xea, 0x7c, 0xfa, 0x10, 0x9f, + 0x9e, 0xfd, 0x10, 0xbf, 0x91, 0x12, 0x53, 0x4a, 0xba, 0xfc, 0x04, 0x44, 0x4e, 0x99, 0xd3, 0xfd, + 0x9d, 0x30, 0x63, 0x7f, 0x77, 0xa1, 0x9f, 0xe5, 0x6f, 0x05, 0x90, 0xe2, 0xff, 0x24, 0x12, 0x11, + 0xea, 0xf9, 0xed, 0xe5, 0x4b, 0xde, 0xdb, 0x4d, 0x47, 0x20, 0x39, 0x7b, 0x04, 0xca, 0x87, 0x20, + 0xc7, 0x58, 0xe7, 0xf4, 0x99, 0x40, 0x78, 0x89, 0x33, 0xc1, 0x5b, 0x90, 0x61, 0xa5, 0x16, 0x26, + 0x52, 0x9e, 0x59, 0xa7, 0xc3, 0x32, 0x4b, 0x7f, 0x45, 0x4a, 0xac, 0xfc, 0x7b, 0x01, 0x72, 0x71, + 0x3e, 0x42, 0x15, 0x90, 0x2c, 0x67, 0xe0, 0x51, 0x32, 0xa0, 0xdf, 0xe5, 0x29, 0x38, 0x11, 0x13, + 0x96, 0x1a, 0x5a, 0x4e, 0x9f, 0xde, 0xa5, 0x4d, 0xa5, 0xa9, 0x38, 0xb4, 0x9c, 0x1d, 0x22, 0xa5, + 0x2a, 0xc6, 0x11, 0x53, 0x49, 0x4e, 0xa9, 0x18, 0x47, 0xa1, 0x4a, 0x99, 0x6e, 0xfc, 0x5e, 0x40, + 0xdb, 0xf7, 0x64, 0x6c, 0x2b, 0xf7, 0x02, 0xb4, 0x08, 0xd9, 0x43, 0xcb, 0x0b, 0xc6, 0x86, 0x4d, + 0x3b, 0x75, 0xde, 0x71, 0x73, 0x61, 0xf9, 0x00, 0xe4, 0x18, 0x8f, 0xcd, 0xb0, 0xa0, 0xff, 0x0b, + 0xa9, 0xa8, 0xa8, 0x66, 0xec, 0xc9, 0xa9, 0x41, 0xf9, 0x57, 0x02, 0x2c, 0x9c, 0xc7, 0x24, 0x53, + 0x29, 0x12, 0xc6, 0x69, 0xa6, 0x14, 0x99, 0x62, 0xf8, 0xc4, 0xb9, 0x0c, 0x3f, 0x59, 0xb9, 0xe4, + 0xc5, 0x2b, 0x57, 0x79, 0x9b, 0x9f, 0xe8, 0x00, 0x32, 0x5b, 0xdb, 0xf5, 0x96, 0xd6, 0x38, 0xf7, + 0x34, 0x46, 0xce, 0x6d, 0x11, 0x2b, 0x93, 0x93, 0x7b, 0x53, 0xeb, 0xd6, 0xea, 0x2d, 0x95, 0x9c, + 0xe3, 0xf3, 0x20, 0xe9, 0x6a, 0xad, 0x49, 0x8f, 0x79, 0x8a, 0xf0, 0x49, 0xea, 0x9b, 0xef, 0x96, + 0x84, 0x8d, 0x94, 0x88, 0x94, 0x6b, 0x95, 0xef, 0x05, 0x40, 0x4d, 0x23, 0x30, 0x48, 0xed, 0x5f, + 0xe1, 0xc8, 0x96, 0xb8, 0x64, 0x09, 0xa6, 0x3b, 0xec, 0xe4, 0xab, 0x74, 0xd8, 0xe1, 0x54, 0x2b, + 0xdf, 0x0a, 0x00, 0xb1, 0xc9, 0x7d, 0x16, 0xff, 0xd5, 0x78, 0xf1, 0x61, 0xe2, 0xd4, 0x3e, 0xf1, + 0x60, 0x8e, 0xff, 0x88, 0xbc, 0x0f, 0xa2, 0xc9, 0x5c, 0x66, 0x79, 0x72, 0x61, 0xd7, 0x7e, 0x26, + 0x32, 0x0f, 0xc8, 0x02, 0x32, 0x69, 0x3d, 0x0b, 0xe9, 0xb1, 0x63, 0xb9, 0xce, 0xbb, 0x1f, 0x01, + 0x3a, 0xcb, 0x9b, 0x24, 0xec, 0xf4, 0xd9, 0x08, 0xb0, 0x19, 0x1e, 0xc2, 0xb7, 0x9d, 0xc3, 0x48, + 0x20, 0xd4, 0x6f, 0x3f, 0xff, 0xdb, 0xe2, 0xdc, 0xf3, 0x93, 0x45, 0xe1, 0x4f, 0x27, 0x8b, 0xc2, + 0x9f, 0x4f, 0x16, 0x85, 0xbf, 0x9e, 0x2c, 0x0a, 0xbf, 0xf8, 0xfb, 0xe2, 0xdc, 0x17, 0x59, 0x36, + 0x81, 0xff, 0x04, 0x00, 0x00, 0xff, 0xff, 0x2a, 0xda, 0xda, 0xf4, 0xc5, 0x21, 0x00, 0x00, } diff --git a/pkg/sql/sqlbase/structured.proto b/pkg/sql/sqlbase/structured.proto index 45b3f1809440..3f7b0ee70113 100644 --- a/pkg/sql/sqlbase/structured.proto +++ b/pkg/sql/sqlbase/structured.proto @@ -829,6 +829,31 @@ message TableDescriptor { // The job id for a drop job is the id in the system.jobs table of the // dropping of this table. optional int64 drop_job_id = 32 [(gogoproto.nullable) = false, (gogoproto.customname) = "DropJobID"]; + + message GCDescriptorMutation { + optional int64 index_id = 1 [(gogoproto.nullable) = false, (gogoproto.customname) = "IndexID", + (gogoproto.casttype) = "IndexID"]; + + optional int64 drop_time = 2 [(gogoproto.nullable) = false]; + + // The job id for a mutation job is the id in the system.jobs table of the + // schema change job executing the mutation referenced by mutation_id. + optional int64 job_id = 3 [(gogoproto.nullable) = false, + (gogoproto.customname) = "JobID"]; + } + + // The schema elements that have been dropped and whose underlying + // data needs to be gc-ed. These schema elements have already transitioned + // through the drop state machine when they were in the above mutations + // list, and can be safely deleted. The names for these schema elements + // can be reused. This list is separate because mutations can + // lie in this list for a long time (gc deadline) and should not block + // the execution of other schema changes on the table. + // + // TODO(vivekmenezes): This is currently only used by the non-interleaved drop + // index case. Also use for dropped interleaved indexes and columns. + repeated GCDescriptorMutation gc_mutations = 33 [(gogoproto.nullable) = false, + (gogoproto.customname) = "GCMutations"]; } // DatabaseDescriptor represents a namespace (aka database) and is stored diff --git a/pkg/sql/tablewriter_delete.go b/pkg/sql/tablewriter_delete.go index fcbac1e9d2dd..3e37af2ba3a4 100644 --- a/pkg/sql/tablewriter_delete.go +++ b/pkg/sql/tablewriter_delete.go @@ -259,7 +259,7 @@ func (td *tableDeleter) deleteIndex( autoCommit autoCommitOpt, traceKV bool, ) (roachpb.Span, error) { - if len(idx.Interleave.Ancestors) > 0 || len(idx.InterleavedBy) > 0 { + if idx.IsInterleaved() { if log.V(2) { log.Info(ctx, "delete forced to scan: table is interleaved") } diff --git a/pkg/sql/truncate.go b/pkg/sql/truncate.go index 3cb6de6bfa32..cdc9f765722d 100644 --- a/pkg/sql/truncate.go +++ b/pkg/sql/truncate.go @@ -262,6 +262,7 @@ func (p *planner) truncateTable( } } newTableDesc.Mutations = nil + newTableDesc.GCMutations = nil tKey := tableKey{parentID: newTableDesc.ParentID, name: newTableDesc.Name} key := tKey.Key() diff --git a/pkg/storage/store.go b/pkg/storage/store.go index ce0dfe5e27c9..77cec15910c9 100644 --- a/pkg/storage/store.go +++ b/pkg/storage/store.go @@ -4603,28 +4603,6 @@ func (s *Store) ManuallyEnqueue( return collect(), "", nil } -// ManuallyEnqueueSpan runs all replicas in the supplied span through the named -// queue. This is currently intended for use in internal tests which have access -// to the store directly. -func (s *Store) ManuallyEnqueueSpan( - ctx context.Context, queueName string, span roachpb.RSpan, skipShouldQueue bool, -) error { - var outerErr error - newStoreReplicaVisitor(s).Visit(func(repl *Replica) bool { - desc := repl.Desc() - if bytes.Compare(span.Key, desc.EndKey) >= 0 || bytes.Compare(desc.StartKey, span.EndKey) >= 0 { - return true // continue - } - - if _, _, err := s.ManuallyEnqueue(ctx, queueName, repl, skipShouldQueue); err != nil { - outerErr = err - return false - } - return true - }) - return outerErr -} - // WriteClusterVersion writes the given cluster version to the store-local cluster version key. func WriteClusterVersion( ctx context.Context, writer engine.ReadWriter, cv cluster.ClusterVersion,