diff --git a/pkg/sql/drop_table.go b/pkg/sql/drop_table.go index 0c3f3133ea8f..486fe45087ac 100644 --- a/pkg/sql/drop_table.go +++ b/pkg/sql/drop_table.go @@ -15,6 +15,8 @@ import ( "fmt" "strings" + "github.com/cockroachdb/cockroach/pkg/jobs" + "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/security" "github.com/cockroachdb/cockroach/pkg/server/telemetry" @@ -26,6 +28,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" "github.com/cockroachdb/cockroach/pkg/sql/sqltelemetry" "github.com/cockroachdb/cockroach/pkg/util/errorutil/unimplemented" + "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/log/eventpb" "github.com/cockroachdb/cockroach/pkg/util/timeutil" "github.com/cockroachdb/errors" @@ -442,6 +445,12 @@ func (p *planner) initiateDropTable( // subsequent schema changes in the transaction (ie. this drop table statement) do not get a cache hit // and do not try to update succeeded jobs, which would raise an error. Instead, this drop table // statement will create a new job to drop the table. + // + // Note that we still wait for jobs removed from the cache to finish running + // after the transaction, since they're not removed from the jobsCollection. + // Also, changes made here do not affect schema change jobs created in this + // transaction with no mutation ID; they remain in the cache, and will be + // updated when writing the job record to drop the table. jobIDs := make(map[int64]struct{}) var id descpb.MutationID for _, m := range tableDesc.Mutations { @@ -455,9 +464,38 @@ func (p *planner) initiateDropTable( } } for jobID := range jobIDs { - if err := p.ExecCfg().JobRegistry.Succeeded(ctx, p.txn, jobID); err != nil { - return errors.Wrapf(err, - "failed to mark job %d as as successful", errors.Safe(jobID)) + // Mark jobs as succeeded when possible, but be defensive about jobs that + // are already in a terminal state or nonexistent. This could happen for + // schema change jobs that couldn't be successfully reverted and ended up in + // a failed state. Such jobs could have already been GCed from the jobs + // table by the time this code runs. + mutationJob, err := p.execCfg.JobRegistry.LoadJobWithTxn(ctx, jobID, p.txn) + if err != nil { + if jobs.HasJobNotFoundError(err) { + log.Warningf(ctx, "mutation job %d not found", jobID) + continue + } + return err + } + if err := mutationJob.WithTxn(p.txn).Update( + ctx, func(txn *kv.Txn, md jobs.JobMetadata, ju *jobs.JobUpdater) error { + status := md.Status + switch status { + case jobs.StatusSucceeded, jobs.StatusCanceled, jobs.StatusFailed: + log.Warningf(ctx, "mutation job %d in unexpected state %s", jobID, status) + return nil + case jobs.StatusRunning, jobs.StatusPending: + status = jobs.StatusSucceeded + default: + // We shouldn't mark jobs as succeeded if they're not in a state where + // they're eligible to ever succeed, so mark them as failed. + status = jobs.StatusFailed + } + log.Infof(ctx, "marking mutation job %d for dropped table as %s", jobID, status) + ju.UpdateStatus(status) + return nil + }); err != nil { + return errors.Wrap(err, "updating mutation job for dropped table") } delete(p.ExtendedEvalContext().SchemaChangeJobCache, tableDesc.ID) } diff --git a/pkg/sql/opt/exec/execbuilder/testdata/fk b/pkg/sql/opt/exec/execbuilder/testdata/fk index d02dca60d793..f9d99b60f4e5 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/fk +++ b/pkg/sql/opt/exec/execbuilder/testdata/fk @@ -1520,3 +1520,91 @@ vectorized: true estimated row count: 10 table: small_parent@primary spans: FULL SCAN + +# Test that partial indexes with IS NOT NULL predicates are used for performing +# FK checks. +subtest partial_index + +statement ok +CREATE TABLE partial_parent ( + id INT PRIMARY KEY +) + +statement ok +CREATE TABLE partial_child ( + id INT PRIMARY KEY, + parent_id INT, + CONSTRAINT fk FOREIGN KEY (parent_id) REFERENCES partial_parent(id), + INDEX partial_idx (parent_id) WHERE parent_id IS NOT NULL +) + +query T +EXPLAIN DELETE FROM partial_parent WHERE id = 1 +---- +distribution: local +vectorized: true +· +• root +│ +├── • delete +│ │ from: partial_parent +│ │ +│ └── • buffer +│ │ label: buffer 1 +│ │ +│ └── • scan +│ missing stats +│ table: partial_parent@primary +│ spans: [/1 - /1] +│ +└── • constraint-check + │ + └── • error if rows + │ + └── • lookup join (semi) + │ table: partial_child@partial_idx (partial index) + │ equality: (id) = (parent_id) + │ + └── • scan buffer + label: buffer 1 + +query T +EXPLAIN UPDATE partial_parent SET id = 2 WHERE id = 1 +---- +distribution: local +vectorized: true +· +• root +│ +├── • update +│ │ table: partial_parent +│ │ set: id +│ │ +│ └── • buffer +│ │ label: buffer 1 +│ │ +│ └── • render +│ │ +│ └── • scan +│ missing stats +│ table: partial_parent@primary +│ spans: [/1 - /1] +│ locking strength: for update +│ +└── • constraint-check + │ + └── • error if rows + │ + └── • lookup join (semi) + │ table: partial_child@partial_idx (partial index) + │ equality: (id) = (parent_id) + │ + └── • except + │ + ├── • scan buffer + │ label: buffer 1 + │ + └── • scan buffer + label: buffer 1 + +subtest end diff --git a/pkg/sql/opt/exec/execbuilder/testdata/partial_index b/pkg/sql/opt/exec/execbuilder/testdata/partial_index new file mode 100644 index 000000000000..6567fc547c38 --- /dev/null +++ b/pkg/sql/opt/exec/execbuilder/testdata/partial_index @@ -0,0 +1,125 @@ +# LogicTest: local + +statement ok +CREATE TABLE t ( + a INT PRIMARY KEY, + b INT, + c STRING, + FAMILY (a, b, c), + INDEX b_partial (b) WHERE b > 10 +) + +statement ok +CREATE TABLE inv ( + a INT PRIMARY KEY, + b JSON, + c STRING, + INVERTED INDEX i (b) WHERE c IN ('foo', 'bar'), + FAMILY (a, b, c) +) + +# --------------------------------------------------------- +# EXPLAIN +# --------------------------------------------------------- + +# EXPLAIN output shows the partial index label on scans and joins on partial +# indexes. +query T +EXPLAIN SELECT b FROM t WHERE b > 10 +---- +distribution: local +vectorized: true +· +• scan + missing stats + table: t@b_partial (partial index) + spans: FULL SCAN + +query T +EXPLAIN SELECT t1.a FROM t t1 INNER LOOKUP JOIN t t2 ON t1.a = t2.b AND t2.b > 10 +---- +distribution: local +vectorized: true +· +• lookup join +│ table: t@b_partial (partial index) +│ equality: (a) = (b) +│ +└── • scan + missing stats + table: t@primary + spans: [/11 - ] + +query T +EXPLAIN SELECT a FROM inv@i WHERE b @> '{"x": "y"}' AND c IN ('foo', 'bar') +---- +distribution: local +vectorized: true +· +• scan + missing stats + table: inv@i (partial index) + spans: 1 span + +query T +EXPLAIN SELECT a FROM inv@i WHERE b @> '{"x": "y"}' AND c = 'foo' +---- +distribution: local +vectorized: true +· +• filter +│ filter: c = 'foo' +│ +└── • index join + │ table: inv@primary + │ + └── • scan + missing stats + table: inv@i (partial index) + spans: 1 span + +query T +EXPLAIN SELECT * FROM inv@i WHERE b @> '{"x": "y"}' AND c IN ('foo', 'bar') +---- +distribution: local +vectorized: true +· +• index join +│ table: inv@primary +│ +└── • scan + missing stats + table: inv@i (partial index) + spans: 1 span + +# --------------------------------------------------------- +# JOIN +# --------------------------------------------------------- + +statement ok +CREATE TABLE a (a INT PRIMARY KEY); + +statement ok +CREATE TABLE b (b INT, INDEX (b) WHERE b IS NOT NULL) + +# The partial index can be used because the ON condition implicitly implies the +# partial index predicate, b IS NOT NULL. +query T +EXPLAIN SELECT * FROM a JOIN b ON a = b +---- +distribution: local +vectorized: true +· +• merge join +│ equality: (a) = (b) +│ left cols are key +│ +├── • scan +│ missing stats +│ table: a@primary +│ spans: FULL SCAN +│ +└── • scan + missing stats + table: b@b_b_idx (partial index) + spans: FULL SCAN diff --git a/pkg/sql/opt/exec/execbuilder/testdata/partial_index_nonmetamorphic b/pkg/sql/opt/exec/execbuilder/testdata/partial_index_nonmetamorphic index 83bb297e4512..e45575fce18a 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/partial_index_nonmetamorphic +++ b/pkg/sql/opt/exec/execbuilder/testdata/partial_index_nonmetamorphic @@ -882,64 +882,6 @@ UPSERT INTO inv VALUES (6, '{"x": "y", "num": 8}', 'baz') Scan /Table/55/1/6{-/#} CPut /Table/55/1/6/0 -> /TUPLE/ -# --------------------------------------------------------- -# EXPLAIN -# --------------------------------------------------------- - -# EXPLAIN output shows the partial index label on scans over partial indexes. -query T -EXPLAIN SELECT b FROM t WHERE b > 10 ----- -distribution: local -vectorized: true -· -• scan - missing stats - table: t@b_partial (partial index) - spans: FULL SCAN - -query T -EXPLAIN SELECT a FROM inv@i WHERE b @> '{"x": "y"}' AND c IN ('foo', 'bar') ----- -distribution: local -vectorized: true -· -• scan - missing stats - table: inv@i (partial index) - spans: 1 span - -query T -EXPLAIN SELECT a FROM inv@i WHERE b @> '{"x": "y"}' AND c = 'foo' ----- -distribution: local -vectorized: true -· -• filter -│ filter: c = 'foo' -│ -└── • index join - │ table: inv@primary - │ - └── • scan - missing stats - table: inv@i (partial index) - spans: 1 span - -query T -EXPLAIN SELECT * FROM inv@i WHERE b @> '{"x": "y"}' AND c IN ('foo', 'bar') ----- -distribution: local -vectorized: true -· -• index join -│ table: inv@primary -│ -└── • scan - missing stats - table: inv@i (partial index) - spans: 1 span - # Regression test for #57085. Cascading DELETEs should not issue DEL operations # for partial indexes of a child table when the deleted row was not in the # partial index. diff --git a/pkg/sql/opt/norm/reject_nulls_funcs.go b/pkg/sql/opt/norm/reject_nulls_funcs.go index d0bdb22e5652..8ce83632baf1 100644 --- a/pkg/sql/opt/norm/reject_nulls_funcs.go +++ b/pkg/sql/opt/norm/reject_nulls_funcs.go @@ -162,6 +162,9 @@ func DeriveRejectNullCols(in memo.RelExpr) opt.ColSet { case opt.ProjectOp: relProps.Rule.RejectNullCols.UnionWith(deriveProjectRejectNullCols(in)) + + case opt.ScanOp: + relProps.Rule.RejectNullCols.UnionWith(deriveScanRejectNullCols(in)) } return relProps.Rule.RejectNullCols @@ -317,3 +320,43 @@ func deriveProjectRejectNullCols(in memo.RelExpr) opt.ColSet { } return (rejectNullCols.Union(projectionsRejectCols)).Intersection(in.Relational().OutputCols) } + +// deriveScanRejectNullCols returns the set of Scan columns which are eligible +// for null rejection. Scan columns can be null-rejected only when there are +// partial indexes that have explicit "column IS NOT NULL" expressions. Creating +// null-rejecting filters is useful in this case because the filters may imply a +// partial index predicate expression, allowing a scan over the index. +func deriveScanRejectNullCols(in memo.RelExpr) opt.ColSet { + md := in.Memo().Metadata() + scan := in.(*memo.ScanExpr) + + var rejectNullCols opt.ColSet + for _, pred := range md.TableMeta(scan.Table).PartialIndexPredicates { + predFilters := *pred.(*memo.FiltersExpr) + rejectNullCols.UnionWith(isNotNullCols(predFilters)) + } + + return rejectNullCols +} + +// isNotNullCols returns the set of columns with explicit, top-level IS NOT NULL +// filter conditions in the given filters. Note that And and Or expressions are +// not traversed. +func isNotNullCols(filters memo.FiltersExpr) opt.ColSet { + var notNullCols opt.ColSet + for i := range filters { + c := filters[i].Condition + isNot, ok := c.(*memo.IsNotExpr) + if !ok { + continue + } + col, ok := isNot.Left.(*memo.VariableExpr) + if !ok { + continue + } + if isNot.Right == memo.NullSingleton { + notNullCols.Add(col.Col) + } + } + return notNullCols +} diff --git a/pkg/sql/opt/norm/rules/reject_nulls.opt b/pkg/sql/opt/norm/rules/reject_nulls.opt index a0c35d7d4a12..d92744a41eaf 100644 --- a/pkg/sql/opt/norm/rules/reject_nulls.opt +++ b/pkg/sql/opt/norm/rules/reject_nulls.opt @@ -179,7 +179,8 @@ # RejectNullsUnderJoinRight mirrors RejectNullsUnderJoinLeft. [RejectNullsUnderJoinRight, Normalize, LowPriority] -(InnerJoin | InnerJoinApply | LeftJoin | LeftJoinApply +(InnerJoin | InnerJoinApply | LeftJoin | LeftJoinApply | SemiJoin + | AntiJoin $left:* $right:* & ^(ColsAreEmpty $rejectCols:(RejectNullCols $right)) diff --git a/pkg/sql/opt/norm/testdata/rules/reject_nulls b/pkg/sql/opt/norm/testdata/rules/reject_nulls index 3c7c9ebfd059..ce84584af1d2 100644 --- a/pkg/sql/opt/norm/testdata/rules/reject_nulls +++ b/pkg/sql/opt/norm/testdata/rules/reject_nulls @@ -1166,6 +1166,120 @@ left-join-apply └── filters └── u:1 = y:6 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] +exec-ddl +CREATE TABLE ab ( + a INT, + b INT, + c INT, + INDEX b_idx (b) WHERE b IS NOT NULL, + INDEX c_idx (c) WHERE c > 0 +) +---- + +# Reject nulls for a scan with an IS NOT NULL partial index predicate expression +# on the right side of a semi-join. +norm expect=RejectNullsUnderJoinRight +SELECT * FROM ab t1 WHERE EXISTS (SELECT * FROM ab t2 WHERE t1.a = t2.b) +---- +semi-join (hash) + ├── columns: a:1 b:2 c:3 + ├── scan ab [as=t1] + │ ├── columns: t1.a:1 t1.b:2 t1.c:3 + │ └── partial index predicates + │ ├── b_idx: filters + │ │ └── t1.b:2 IS NOT NULL [outer=(2), constraints=(/2: (/NULL - ]; tight)] + │ └── c_idx: filters + │ └── t1.c:3 > 0 [outer=(3), constraints=(/3: [/1 - ]; tight)] + ├── select + │ ├── columns: t2.b:7!null + │ ├── scan ab [as=t2] + │ │ ├── columns: t2.b:7 + │ │ └── partial index predicates + │ │ ├── b_idx: filters + │ │ │ └── t2.b:7 IS NOT NULL [outer=(7), constraints=(/7: (/NULL - ]; tight)] + │ │ └── c_idx: filters + │ │ └── t2.c:8 > 0 [outer=(8), constraints=(/8: [/1 - ]; tight)] + │ └── filters + │ └── t2.b:7 IS NOT NULL [outer=(7), constraints=(/7: (/NULL - ]; tight)] + └── filters + └── t1.a:1 = t2.b:7 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] + +# Fully optimizing the query shows that a partial index scan is generated +# because the null-reject filters imply the partial index predicate. +opt expect=(RejectNullsUnderJoinRight,GeneratePartialIndexScans) +SELECT * FROM ab t1 WHERE EXISTS (SELECT * FROM ab t2 WHERE t1.a = t2.b) +---- +project + ├── columns: a:1 b:2 c:3 + └── inner-join (hash) + ├── columns: t1.a:1!null t1.b:2 t1.c:3 t2.b:7!null + ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-more) + ├── fd: (1)==(7), (7)==(1) + ├── scan ab [as=t1] + │ ├── columns: t1.a:1 t1.b:2 t1.c:3 + │ └── partial index predicates + │ ├── b_idx: filters + │ │ └── t1.b:2 IS NOT NULL [outer=(2), constraints=(/2: (/NULL - ]; tight)] + │ └── c_idx: filters + │ └── t1.c:3 > 0 [outer=(3), constraints=(/3: [/1 - ]; tight)] + ├── distinct-on + │ ├── columns: t2.b:7!null + │ ├── grouping columns: t2.b:7!null + │ ├── internal-ordering: +7 + │ ├── key: (7) + │ └── scan ab@b_idx,partial [as=t2] + │ ├── columns: t2.b:7!null + │ └── ordering: +7 + └── filters + └── t1.a:1 = t2.b:7 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] + +# Reject nulls for a scan with an IS NOT NULL partial index predicate expression +# on the right side of a anti-join. +norm expect=RejectNullsUnderJoinRight +SELECT * FROM ab t1 WHERE NOT EXISTS (SELECT * FROM ab t2 WHERE t1.a = t2.b) +---- +anti-join (hash) + ├── columns: a:1 b:2 c:3 + ├── scan ab [as=t1] + │ ├── columns: t1.a:1 t1.b:2 t1.c:3 + │ └── partial index predicates + │ ├── b_idx: filters + │ │ └── t1.b:2 IS NOT NULL [outer=(2), constraints=(/2: (/NULL - ]; tight)] + │ └── c_idx: filters + │ └── t1.c:3 > 0 [outer=(3), constraints=(/3: [/1 - ]; tight)] + ├── select + │ ├── columns: t2.b:7!null + │ ├── scan ab [as=t2] + │ │ ├── columns: t2.b:7 + │ │ └── partial index predicates + │ │ ├── b_idx: filters + │ │ │ └── t2.b:7 IS NOT NULL [outer=(7), constraints=(/7: (/NULL - ]; tight)] + │ │ └── c_idx: filters + │ │ └── t2.c:8 > 0 [outer=(8), constraints=(/8: [/1 - ]; tight)] + │ └── filters + │ └── t2.b:7 IS NOT NULL [outer=(7), constraints=(/7: (/NULL - ]; tight)] + └── filters + └── t1.a:1 = t2.b:7 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] + +# Fully optimizing the query shows that a partial index scan is generated +# because the null-reject filters imply the partial index predicate. +opt expect=(RejectNullsUnderJoinRight,GeneratePartialIndexScans) +SELECT * FROM ab t1 WHERE NOT EXISTS (SELECT * FROM ab t2 WHERE t1.a = t2.b) +---- +anti-join (hash) + ├── columns: a:1 b:2 c:3 + ├── scan ab [as=t1] + │ ├── columns: t1.a:1 t1.b:2 t1.c:3 + │ └── partial index predicates + │ ├── b_idx: filters + │ │ └── t1.b:2 IS NOT NULL [outer=(2), constraints=(/2: (/NULL - ]; tight)] + │ └── c_idx: filters + │ └── t1.c:3 > 0 [outer=(3), constraints=(/3: [/1 - ]; tight)] + ├── scan ab@b_idx,partial [as=t2] + │ └── columns: t2.b:7!null + └── filters + └── t1.a:1 = t2.b:7 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] + # ---------------------------------------------------------- # RejectNullsProject # ---------------------------------------------------------- diff --git a/pkg/sql/schema_changer_test.go b/pkg/sql/schema_changer_test.go index 36bb0e988493..affeb9f7960f 100644 --- a/pkg/sql/schema_changer_test.go +++ b/pkg/sql/schema_changer_test.go @@ -6114,6 +6114,67 @@ SELECT value }) } +// TestDropTableWhileSchemaChangeReverting tests that schema changes in the +// reverting state end up as failed when the table is dropped. +func TestDropTableWhileSchemaChangeReverting(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + defer setTestJobsAdoptInterval()() + ctx := context.Background() + + // Closed when we enter the RunBeforeOnFailOrCancel knob, at which point the + // job is in the reverting state. + beforeOnFailOrCancelNotification := make(chan struct{}) + // Closed when we're ready to continue with the schema change (rollback). + continueNotification := make(chan struct{}) + params, _ := tests.CreateTestServerParams() + params.Knobs = base.TestingKnobs{ + SQLSchemaChanger: &sql.SchemaChangerTestingKnobs{ + RunBeforeOnFailOrCancel: func(_ int64) error { + close(beforeOnFailOrCancelNotification) + <-continueNotification + // Return a retry error, so that we can be sure to test the path where + // the job is marked as failed by the DROP TABLE instead of running to + // completion and ending up in the failed state on its own. + return jobs.NewRetryJobError("injected retry error") + }, + }, + } + s, sqlDB, _ := serverutils.StartServer(t, params) + defer s.Stopper().Stop(ctx) + + _, err := sqlDB.Exec(` +CREATE DATABASE t; +CREATE TABLE t.test (k INT PRIMARY KEY, v INT8); +INSERT INTO t.test VALUES (1, 2), (2, 2); +`) + require.NoError(t, err) + + g := ctxgroup.WithContext(ctx) + g.GoCtx(func(ctx context.Context) error { + // Try to create a unique index which won't be valid and will need a rollback. + _, err := sqlDB.Exec(`CREATE UNIQUE INDEX i ON t.test(v);`) + assert.Regexp(t, "violates unique constraint", err) + return nil + }) + + <-beforeOnFailOrCancelNotification + + _, err = sqlDB.Exec(`DROP TABLE t.test;`) + require.NoError(t, err) + + close(continueNotification) + require.NoError(t, g.Wait()) + + var status jobs.Status + var jobError string + require.NoError(t, sqlDB.QueryRow(` +SELECT status, error FROM crdb_internal.jobs WHERE description LIKE '%CREATE UNIQUE INDEX%' +`).Scan(&status, &jobError)) + require.Equal(t, jobs.StatusFailed, status) + require.Regexp(t, "violates unique constraint", jobError) +} + // TestPermanentErrorDuringRollback tests that a permanent error while rolling // back a schema change causes the job to fail, and that the appropriate error // is displayed in the jobs table. @@ -6123,7 +6184,7 @@ func TestPermanentErrorDuringRollback(t *testing.T) { defer setTestJobsAdoptInterval()() ctx := context.Background() - runTest := func(params base.TestServerArgs) { + runTest := func(t *testing.T, params base.TestServerArgs, gcJobRecord bool) { s, sqlDB, _ := serverutils.StartServer(t, params) defer s.Stopper().Stop(ctx) @@ -6140,10 +6201,20 @@ CREATE UNIQUE INDEX i ON t.test(v); `) require.Regexp(t, "violates unique constraint", err.Error()) + var jobID int64 var jobErr string - row := sqlDB.QueryRow("SELECT error FROM [SHOW JOBS] WHERE job_type = 'SCHEMA CHANGE'") - require.NoError(t, row.Scan(&jobErr)) + row := sqlDB.QueryRow("SELECT job_id, error FROM [SHOW JOBS] WHERE job_type = 'SCHEMA CHANGE'") + require.NoError(t, row.Scan(&jobID, &jobErr)) require.Regexp(t, "cannot be reverted, manual cleanup may be required: permanent error", jobErr) + + if gcJobRecord { + _, err := sqlDB.Exec(`DELETE FROM system.jobs WHERE id = $1`, jobID) + require.NoError(t, err) + } + + // Test that dropping the table is still possible. + _, err = sqlDB.Exec(`DROP TABLE t.test`) + require.NoError(t, err) } t.Run("error-before-backfill", func(t *testing.T) { @@ -6168,7 +6239,9 @@ CREATE UNIQUE INDEX i ON t.test(v); }, }, } - runTest(params) + // Don't GC the job record after the schema change, so we can test dropping + // the table with a failed mutation job. + runTest(t, params, false /* gcJobRecord */) }) t.Run("error-before-reversing-mutations", func(t *testing.T) { @@ -6193,7 +6266,9 @@ CREATE UNIQUE INDEX i ON t.test(v); }, }, } - runTest(params) + // GC the job record after the schema change, so we can test dropping the + // table with a nonexistent mutation job. + runTest(t, params, true /* gcJobRecord */) }) }