From 98a13289c21efb3ff55bb56dfbdae988a99bac05 Mon Sep 17 00:00:00 2001 From: Erik Trinh Date: Thu, 20 Sep 2018 15:04:03 -0400 Subject: [PATCH] sql: asynchronously drop non-interleaved indexes This change drops non-interleaved indexes asynchronously by performing the deletion of data using an asynchronous schema changer. This is in preparation to eventually remove index data using `ClearRange` after the GC TTL period has passed. The initial schema changer runs through the state machine but does not perform the deletion of index data. Instead the mutation is moved to a separate list and has a timestamp attached. The created asynchronous schema changer uses the timestamp and index's configured GC TTL value to determine when it should begin execution and actually truncate the index. When the async schema changer deletes the index data two things occur: the job is marked as succeeded and the index zone config is removed. The job can immediately be marked as succeeded because currently a separate job is created for each index that is dropped. Interleaved indexes are unaffected and have their data deleted immediately. Related to #20696 Release note: none --- pkg/ccl/partitionccl/drop_test.go | 83 ++- pkg/sql/backfill.go | 36 +- pkg/sql/crdb_internal.go | 3 + pkg/sql/drop_table.go | 22 +- pkg/sql/drop_test.go | 73 ++- pkg/sql/lease_test.go | 31 +- .../logictest/testdata/logic_test/alter_table | 20 +- .../logictest/testdata/logic_test/show_trace | 6 +- pkg/sql/schema_changer.go | 243 ++++++- pkg/sql/schema_changer_test.go | 160 +++-- pkg/sql/sqlbase/structured.go | 20 +- pkg/sql/sqlbase/structured.pb.go | 618 ++++++++++++------ pkg/sql/sqlbase/structured.proto | 25 + pkg/sql/tablewriter_delete.go | 2 +- pkg/sql/truncate.go | 1 + 15 files changed, 988 insertions(+), 355 deletions(-) diff --git a/pkg/ccl/partitionccl/drop_test.go b/pkg/ccl/partitionccl/drop_test.go index 802f3032945c..94a674276b08 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,62 @@ 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) + } + } 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 51d1f8e776b1..8496e027bbd4 100644 --- a/pkg/sql/backfill.go +++ b/pkg/sql/backfill.go @@ -114,9 +114,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 { @@ -136,7 +133,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 } @@ -158,9 +155,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) @@ -170,18 +166,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 } } @@ -222,7 +209,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 { @@ -270,7 +256,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 @@ -278,6 +269,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 5df4fb8d9919..27a67fc5fc28 100644 --- a/pkg/sql/crdb_internal.go +++ b/pkg/sql/crdb_internal.go @@ -1748,6 +1748,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..e0ff1a2b6d47 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 · 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 1b6087d76e41..77ea72fb9753 100644 --- a/pkg/sql/schema_changer.go +++ b/pkg/sql/schema_changer.go @@ -78,6 +78,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 @@ -92,6 +98,8 @@ type SchemaChanger struct { // table.DropTime. dropTime int64 + dropIndexTimes []droppedIndex + readAsOf hlc.Timestamp testingKnobs *SchemaChangerTestingKnobs distSQLPlanner *DistSQLPlanner @@ -320,6 +328,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, @@ -506,6 +518,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 { @@ -661,6 +757,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 { @@ -848,7 +948,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 { @@ -857,6 +963,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 } @@ -879,8 +998,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 @@ -986,7 +1113,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 } @@ -1181,7 +1308,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 @@ -1199,7 +1326,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 { @@ -1309,7 +1436,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 @@ -1428,24 +1556,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) } } } @@ -1490,14 +1649,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) @@ -1525,13 +1713,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 2513b2ade688..454ebd095a66 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,21 @@ 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 { + read := atomic.LoadInt32(&attempts) + if 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 +1552,11 @@ 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) } // There is still some garbage index data that needs to be purged. All the @@ -1581,8 +1580,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 +1724,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 +1777,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 +1884,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,12 +2118,10 @@ 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 + // Dropping the index happens in a separate mutation job from the drop column + // and completes successfully. The drop column part of the change errors + // during backfilling the second chunk but cannot rollback the drop index. + const expectedAttempts = 2 const maxValue = (expectedAttempts/2+1)*chunkSize + 1 params.Knobs = base.TestingKnobs{ SQLSchemaChanger: &sql.SchemaChangerTestingKnobs{ @@ -2170,12 +2173,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 +2353,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 { @@ -3345,8 +3352,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{ @@ -3387,8 +3402,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 @@ -3405,15 +3420,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 @@ -3434,13 +3451,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 { @@ -3458,9 +3482,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 650e1b48aab4..7feba002ec39 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 @@ -1930,6 +1940,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) } @@ -1973,10 +1988,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()