-
Notifications
You must be signed in to change notification settings - Fork 3.8k
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
migration: add draining name migration
This commit complements the preceding commit by adding a migration which waits for all descriptors to no longer have any draining names. Since the previous cluster version prevents the addition of new draining names, this new cluster version guarantees that the draining names mechanism is no longer in use. All code pertaining to draining names can therefore be deleted after the upcoming release, which is 22.1. Release note: None
- Loading branch information
Marius Posta
committed
Nov 4, 2021
1 parent
782ae3e
commit 3daa0db
Showing
15 changed files
with
387 additions
and
52 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.
Oops, something went wrong.
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,76 @@ | ||
// Copyright 2021 The Cockroach Authors. | ||
// | ||
// Use of this software is governed by the Business Source License | ||
// included in the file licenses/BSL.txt. | ||
// | ||
// As of the Change Date specified in that file, in accordance with | ||
// the Business Source License, use of this software will be governed | ||
// by the Apache License, Version 2.0, included in the file | ||
// licenses/APL.txt. | ||
|
||
package migrations | ||
|
||
import ( | ||
"context" | ||
"time" | ||
|
||
"github.com/cockroachdb/cockroach/pkg/clusterversion" | ||
"github.com/cockroachdb/cockroach/pkg/jobs" | ||
"github.com/cockroachdb/cockroach/pkg/migration" | ||
"github.com/cockroachdb/cockroach/pkg/security" | ||
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" | ||
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree" | ||
"github.com/cockroachdb/cockroach/pkg/sql/sessiondata" | ||
"github.com/cockroachdb/cockroach/pkg/util/log" | ||
"github.com/cockroachdb/cockroach/pkg/util/retry" | ||
) | ||
|
||
const query = ` | ||
WITH | ||
cte1 | ||
AS ( | ||
SELECT | ||
crdb_internal.pb_to_json('cockroach.sql.sqlbase.Descriptor', descriptor, false) AS d | ||
FROM | ||
system.descriptor | ||
ORDER BY | ||
id ASC | ||
), | ||
cte2 AS (SELECT COALESCE(d->'table', d->'database', d->'schema', d->'type') AS d FROM cte1) | ||
SELECT | ||
(d->'id')::INT8 AS id, d->>'name' AS name | ||
FROM | ||
cte2 | ||
WHERE | ||
COALESCE(json_array_length(d->'drainingNames'), 0) > 0 | ||
LIMIT 1; | ||
` | ||
|
||
// ensureNoDrainingNames waits until every descriptor has no draining names. | ||
func ensureNoDrainingNames( | ||
ctx context.Context, _ clusterversion.ClusterVersion, d migration.TenantDeps, _ *jobs.Job, | ||
) error { | ||
retryOpts := retry.Options{ | ||
MaxBackoff: 10 * time.Second, | ||
} | ||
for r := retry.StartWithCtx(ctx, retryOpts); r.Next(); { | ||
rows, err := d.InternalExecutor.QueryBufferedEx( | ||
ctx, | ||
"ensure-no-draining-names", | ||
nil, /* txn */ | ||
sessiondata.InternalExecutorOverride{User: security.RootUserName()}, | ||
query, | ||
) | ||
if err != nil { | ||
return err | ||
} | ||
if len(rows) == 0 { | ||
return nil | ||
} | ||
datums := rows[0] | ||
id := descpb.ID(*datums[0].(*tree.DInt)) | ||
name := string(*datums[1].(*tree.DString)) | ||
log.Infof(ctx, "descriptor with ID %d and name %q still has draining names", id, name) | ||
} | ||
panic("unreachable") | ||
} |
134 changes: 134 additions & 0 deletions
134
pkg/migration/migrations/ensure_no_draining_names_external_test.go
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,134 @@ | ||
// Copyright 2021 The Cockroach Authors. | ||
// | ||
// Use of this software is governed by the Business Source License | ||
// included in the file licenses/BSL.txt. | ||
// | ||
// As of the Change Date specified in that file, in accordance with | ||
// the Business Source License, use of this software will be governed | ||
// by the Apache License, Version 2.0, included in the file | ||
// licenses/APL.txt. | ||
|
||
package migrations_test | ||
|
||
import ( | ||
"context" | ||
"testing" | ||
|
||
"github.com/cockroachdb/cockroach/pkg/base" | ||
"github.com/cockroachdb/cockroach/pkg/clusterversion" | ||
"github.com/cockroachdb/cockroach/pkg/jobs" | ||
"github.com/cockroachdb/cockroach/pkg/keys" | ||
"github.com/cockroachdb/cockroach/pkg/server" | ||
"github.com/cockroachdb/cockroach/pkg/sql" | ||
"github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkv" | ||
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" | ||
"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/stretchr/testify/require" | ||
) | ||
|
||
// TestEnsureNoDrainingNames tests that the draining names migration performs | ||
// as expected. | ||
func TestEnsureNoDrainingNames(t *testing.T) { | ||
defer leaktest.AfterTest(t)() | ||
defer log.Scope(t).Close(t) | ||
|
||
renameBlocked := make(chan struct{}) | ||
renameUnblocked := make(chan struct{}) | ||
clusterArgs := base.TestClusterArgs{ | ||
ServerArgs: base.TestServerArgs{ | ||
Knobs: base.TestingKnobs{ | ||
Server: &server.TestingKnobs{ | ||
DisableAutomaticVersionUpgrade: 1, | ||
BinaryVersionOverride: clusterversion.ByKey( | ||
clusterversion.AvoidDrainingNames - 1), | ||
}, | ||
JobsTestingKnobs: jobs.NewTestingKnobsWithShortIntervals(), | ||
SQLSchemaChanger: &sql.SchemaChangerTestingKnobs{ | ||
OldNamesDrainedNotification: func() { | ||
renameBlocked <- struct{}{} | ||
<-renameUnblocked | ||
}, | ||
}, | ||
}, | ||
}, | ||
} | ||
|
||
c := keys.SystemSQLCodec | ||
ctx := context.Background() | ||
tc := testcluster.StartTestCluster(t, 1, clusterArgs) | ||
s := tc.Server(0) | ||
defer tc.Stopper().Stop(ctx) | ||
sqlDB := tc.ServerConn(0) | ||
tdb := sqlutils.MakeSQLRunner(sqlDB) | ||
|
||
// Create a database and a schema, rename the schema. | ||
tdb.Exec(t, "CREATE DATABASE t") | ||
tdb.Exec(t, "CREATE SCHEMA t.foo") | ||
|
||
// Concurrently, rename the table. | ||
threadDone := make(chan error) | ||
go func() { | ||
_, err := sqlDB.Exec(`ALTER SCHEMA t.foo RENAME TO bar`) | ||
threadDone <- err | ||
}() | ||
defer func() { | ||
close(renameBlocked) | ||
close(renameUnblocked) | ||
// Block until the thread doing the rename has finished, so the test can | ||
// clean up. It needs to wait for the transaction to release its lease. | ||
if err := <-threadDone; err != nil { | ||
t.Fatal(err) | ||
} | ||
}() | ||
<-renameBlocked | ||
|
||
// Check that the draining name persists in the descriptor and in the db's | ||
// schema mapping. | ||
{ | ||
db := catalogkv.TestingGetDatabaseDescriptor(s.DB(), c, "t") | ||
_ = db.ForEachSchemaInfo(func(id descpb.ID, name string, isDropped bool) error { | ||
switch name { | ||
case "foo": | ||
require.True(t, isDropped) | ||
case "bar": | ||
require.False(t, isDropped) | ||
} | ||
return nil | ||
}) | ||
foo := catalogkv.TestingGetSchemaDescriptor(s.DB(), c, db.GetID(), "foo") | ||
require.NotEmpty(t, foo.GetDrainingNames()) | ||
} | ||
|
||
// Reuse the old schema name. | ||
// This should fail in this pre-AvoidDrainingNames cluster version. | ||
tdb.ExpectErr(t, `schema "foo" already exists`, "CREATE SCHEMA t.foo") | ||
|
||
renameUnblocked <- struct{}{} | ||
|
||
// Migrate to the new cluster version. | ||
tdb.Exec(t, `SET CLUSTER SETTING version = $1`, | ||
clusterversion.ByKey(clusterversion.DrainingNamesMigration).String()) | ||
|
||
tdb.CheckQueryResultsRetry(t, "SHOW CLUSTER SETTING version", | ||
[][]string{{clusterversion.ByKey(clusterversion.DrainingNamesMigration).String()}}) | ||
|
||
tdb.Exec(t, "CREATE SCHEMA t.foo") | ||
|
||
// Check that there are no draining names and that the database schema mapping | ||
// is correct. | ||
{ | ||
db := catalogkv.TestingGetDatabaseDescriptor(s.DB(), c, "t") | ||
_ = db.ForEachSchemaInfo(func(id descpb.ID, name string, isDropped bool) error { | ||
require.False(t, isDropped) | ||
require.True(t, name == "foo" || name == "bar") | ||
return nil | ||
}) | ||
foo := catalogkv.TestingGetSchemaDescriptor(s.DB(), c, db.GetID(), "foo") | ||
require.Empty(t, foo.GetDrainingNames()) | ||
bar := catalogkv.TestingGetSchemaDescriptor(s.DB(), c, db.GetID(), "bar") | ||
require.Empty(t, bar.GetDrainingNames()) | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.