Skip to content

Commit

Permalink
Merge #62421
Browse files Browse the repository at this point in the history
62421: gcjob,partitionccl: fixes for gcjob and zone configs on enum-partitioned tables r=ajwerner a=ajwerner

See individual commits. This ends up being two problems, neither of which are solved the way that was originally discussed in #62029. The hydration issue is easily side-stepped (as it should be). The dropped table issue as well. 

Fixes #62029.

Co-authored-by: Andrew Werner <[email protected]>
  • Loading branch information
craig[bot] and ajwerner committed Mar 26, 2021
2 parents 4962590 + 3f58646 commit fd5d337
Show file tree
Hide file tree
Showing 5 changed files with 157 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
120 changes: 120 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,121 @@ 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")
})

// This is a regression test for a hazardous scenario whereby a drop index gc
// job may attempt to rewrite subzone spans for a dropped table which used types
// which no longer exist.
t.Run("drop table and type", func(t *testing.T) {

// Sketch of the test:
//
// * Set up a partitioned table and index which are partitioned by an enum.
// * Set a short GC TTL on the index.
// * Drop the index.
// * Drop the table.
// * Drop the type.
// * Wait for the index to be cleaned up, which would have crashed before the
// this fix.
// * Set a short GC TTL on everything.
// * Wait for the table to be cleaned up.
//

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)
PARTITION BY LIST (e)
(
PARTITION ai VALUES IN ('a'),
PARTITION bi VALUES IN ('b'),
PARTITION ci VALUES IN ('c')
);
ALTER PARTITION ai OF INDEX t@idx CONFIGURE ZONE USING range_min_bytes = 123456, range_max_bytes = 654321;
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;
DROP TABLE t;
DROP TYPE typ;
`)

waitForJobDone(t, tdb, "GC for DROP INDEX%idx")
tdb.Exec(t, `ALTER RANGE default CONFIGURE ZONE USING gc.ttlseconds = 1`)
waitForJobDone(t, tdb, "GC for DROP TABLE%t")
})
}
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
9 changes: 9 additions & 0 deletions pkg/sql/partition_utils.go
Original file line number Diff line number Diff line change
Expand Up @@ -86,6 +86,15 @@ func GenerateSubzoneSpans(
}
}

// We already completely avoid creating subzone spans for dropped indexes.
// Whether this was intentional is a different story, but it turns out to be
// pretty sane. Dropped elements may refer to dropped types and we aren't
// necessarily in a position to deal with those dropped types. Add a special
// case to avoid generating any subzone spans in the face of being dropped.
if tableDesc.Dropped() {
return nil, nil
}

a := &rowenc.DatumAlloc{}

subzoneIndexByIndexID := make(map[descpb.IndexID]int32)
Expand Down

0 comments on commit fd5d337

Please sign in to comment.