Skip to content

Commit

Permalink
gcjob,partitionccl: use descs.Collection to fix bug in partitioning b…
Browse files Browse the repository at this point in the history
…y enums

Prior to this commit we would use the low-level KV API to retrieve the table
descriptor for use in rewriting the zone configs. This was problematic
because we may need the hydrated types to generate the subzone spans. There
is absolutely no reason to be using the lower-level API.

Release note (bug fix): Fixed a bug introduced in alphas which could cause
panics when dropping indexes on tables partitioned by user-defined types.
  • Loading branch information
ajwerner committed Mar 25, 2021
1 parent cbebc6e commit 2300690
Show file tree
Hide file tree
Showing 4 changed files with 95 additions and 5 deletions.
1 change: 1 addition & 0 deletions pkg/ccl/partitionccl/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -45,6 +45,7 @@ go_test(
"//pkg/ccl/utilccl",
"//pkg/config",
"//pkg/config/zonepb",
"//pkg/jobs",
"//pkg/jobs/jobspb",
"//pkg/keys",
"//pkg/kv/kvserver",
Expand Down
67 changes: 67 additions & 0 deletions pkg/ccl/partitionccl/drop_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -15,6 +15,7 @@ import (

"github.com/cockroachdb/cockroach/pkg/base"
"github.com/cockroachdb/cockroach/pkg/config/zonepb"
"github.com/cockroachdb/cockroach/pkg/jobs"
"github.com/cockroachdb/cockroach/pkg/jobs/jobspb"
"github.com/cockroachdb/cockroach/pkg/keys"
"github.com/cockroachdb/cockroach/pkg/sql"
Expand All @@ -24,6 +25,7 @@ import (
"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/testutils/testcluster"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/protoutil"
Expand Down Expand Up @@ -145,3 +147,68 @@ func TestDropIndexWithZoneConfigCCL(t *testing.T) {
return nil
})
}

// TestDropIndexPartitionedByUserDefinedTypeCCL is a regression test to ensure
// that dropping an index partitioned by a user-defined types is safe.
func TestDropIndexPartitionedByUserDefinedTypeCCL(t *testing.T) {
defer leaktest.AfterTest(t)()

waitForJobDone := func(t *testing.T, tdb *sqlutils.SQLRunner, description string) {
t.Helper()
var id int
tdb.QueryRow(t, `
SELECT job_id
FROM crdb_internal.jobs
WHERE description LIKE $1
`, description).Scan(&id)
testutils.SucceedsSoon(t, func() error {
var status string
tdb.QueryRow(t,
`SELECT status FROM [SHOW JOB $1]`,
id,
).Scan(&status)
if status != string(jobs.StatusSucceeded) {
return errors.Errorf("expected %q, got %q", jobs.StatusSucceeded, status)
}
return nil
})
}

// This is a regression test for a bug which was caused by not using hydrated
// descriptors in the index gc job to re-write zone config subzone spans.
// This test ensures that subzone spans can be re-written by creating a
// table partitioned by user-defined types and then dropping an index and
// ensuring that the drop job for the index completes successfully.
t.Run("drop index, type-partitioned table", func(t *testing.T) {
// Sketch of the test:
//
// * Set up a partitioned table partitioned by an enum.
// * Create an index.
// * Set a short GC TTL on the index.
// * Drop the index.
// * Wait for the index to be cleaned up, which would have crashed before the
// this fix

defer log.Scope(t).Close(t)
ctx := context.Background()
tc := testcluster.StartTestCluster(t, 1, base.TestClusterArgs{})
defer tc.Stopper().Stop(ctx)

tdb := sqlutils.MakeSQLRunner(tc.ServerConn(0))

tdb.Exec(t, `
CREATE TYPE typ AS ENUM ('a', 'b', 'c');
CREATE TABLE t (e typ PRIMARY KEY) PARTITION BY LIST (e) (
PARTITION a VALUES IN ('a'),
PARTITION b VALUES IN ('b'),
PARTITION c VALUES IN ('c')
);
CREATE INDEX idx ON t (e);
ALTER PARTITION a OF TABLE t CONFIGURE ZONE USING range_min_bytes = 123456, range_max_bytes = 654321;
ALTER INDEX t@idx CONFIGURE ZONE USING gc.ttlseconds = 1;
DROP INDEX t@idx;
`)

waitForJobDone(t, tdb, "GC for DROP INDEX%idx")
})
}
1 change: 1 addition & 0 deletions pkg/sql/gcjob/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -33,6 +33,7 @@ go_library(
"//pkg/sql/catalog/tabledesc",
"//pkg/sql/pgwire/pgcode",
"//pkg/sql/pgwire/pgerror",
"//pkg/sql/sem/tree",
"//pkg/util/log",
"//pkg/util/timeutil",
"@com_github_cockroachdb_errors//:errors",
Expand Down
31 changes: 26 additions & 5 deletions pkg/sql/gcjob/index_garbage_collection.go
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,8 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/catalog"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkv"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descs"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/errors"
)
Expand Down Expand Up @@ -65,17 +67,36 @@ func gcIndexes(

// All the data chunks have been removed. Now also removed the
// zone configs for the dropped indexes, if any.
if err := execCfg.DB.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error {
freshParentTableDesc, err := catalogkv.MustGetTableDescByID(ctx, txn, execCfg.Codec, parentID)
removeIndexZoneConfigs := func(
ctx context.Context, txn *kv.Txn, descriptors *descs.Collection,
) error {
freshParentTableDesc, err := descriptors.GetMutableTableByID(
ctx, txn, parentID, tree.ObjectLookupFlags{
CommonLookupFlags: tree.CommonLookupFlags{
AvoidCached: true,
Required: true,
IncludeDropped: true,
IncludeOffline: true,
},
})
if err != nil {
return err
}
return sql.RemoveIndexZoneConfigs(ctx, txn, execCfg, freshParentTableDesc, []descpb.IndexDescriptor{indexDesc})
}); err != nil {
toRemove := []descpb.IndexDescriptor{indexDesc}
return sql.RemoveIndexZoneConfigs(
ctx, txn, execCfg, freshParentTableDesc, toRemove,
)
}
lm, ie, db := execCfg.LeaseManager, execCfg.InternalExecutor, execCfg.DB
if err := descs.Txn(
ctx, execCfg.Settings, lm, ie, db, removeIndexZoneConfigs,
); err != nil {
return errors.Wrapf(err, "removing index %d zone configs", indexDesc.ID)
}

if err := completeDroppedIndex(ctx, execCfg, parentTable, index.IndexID, progress); err != nil {
if err := completeDroppedIndex(
ctx, execCfg, parentTable, index.IndexID, progress,
); err != nil {
return err
}
}
Expand Down

0 comments on commit 2300690

Please sign in to comment.