Skip to content

Commit

Permalink
migration: add draining name migration
Browse files Browse the repository at this point in the history
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
Show file tree
Hide file tree
Showing 15 changed files with 387 additions and 52 deletions.
2 changes: 1 addition & 1 deletion docs/generated/settings/settings-for-tenants.txt
Original file line number Diff line number Diff line change
Expand Up @@ -167,4 +167,4 @@ trace.debug.enable boolean false if set, traces for recent requests can be seen
trace.jaeger.agent string the address of a Jaeger agent to receive traces using the Jaeger UDP Thrift protocol, as <host>:<port>. If no port is specified, 6381 will be used.
trace.opentelemetry.collector string address of an OpenTelemetry trace collector to receive traces using the otel gRPC protocol, as <host>:<port>. If no port is specified, 4317 will be used.
trace.zipkin.collector string the address of a Zipkin instance to receive traces, as <host>:<port>. If no port is specified, 9411 will be used.
version version 21.2-6 set the active cluster version in the format '<major>.<minor>'
version version 21.2-8 set the active cluster version in the format '<major>.<minor>'
2 changes: 1 addition & 1 deletion docs/generated/settings/settings.html
Original file line number Diff line number Diff line change
Expand Up @@ -172,6 +172,6 @@
<tr><td><code>trace.jaeger.agent</code></td><td>string</td><td><code></code></td><td>the address of a Jaeger agent to receive traces using the Jaeger UDP Thrift protocol, as <host>:<port>. If no port is specified, 6381 will be used.</td></tr>
<tr><td><code>trace.opentelemetry.collector</code></td><td>string</td><td><code></code></td><td>address of an OpenTelemetry trace collector to receive traces using the otel gRPC protocol, as <host>:<port>. If no port is specified, 4317 will be used.</td></tr>
<tr><td><code>trace.zipkin.collector</code></td><td>string</td><td><code></code></td><td>the address of a Zipkin instance to receive traces, as <host>:<port>. If no port is specified, 9411 will be used.</td></tr>
<tr><td><code>version</code></td><td>version</td><td><code>21.2-6</code></td><td>set the active cluster version in the format '<major>.<minor>'</td></tr>
<tr><td><code>version</code></td><td>version</td><td><code>21.2-8</code></td><td>set the active cluster version in the format '<major>.<minor>'</td></tr>
</tbody>
</table>
7 changes: 7 additions & 0 deletions pkg/clusterversion/cockroach_versions.go
Original file line number Diff line number Diff line change
Expand Up @@ -273,6 +273,9 @@ const (
// AvoidDrainingNames avoids using the draining_names field when renaming or
// dropping descriptors.
AvoidDrainingNames
// DrainingNamesMigration adds the migration which guarantees that no
// descriptors have draining names.
DrainingNamesMigration

// *************************************************
// Step (1): Add new versions here.
Expand Down Expand Up @@ -466,6 +469,10 @@ var versionsSingleton = keyedVersions{
Key: AvoidDrainingNames,
Version: roachpb.Version{Major: 21, Minor: 2, Internal: 6},
},
{
Key: DrainingNamesMigration,
Version: roachpb.Version{Major: 21, Minor: 2, Internal: 8},
},

// *************************************************
// Step (2): Add new versions here.
Expand Down
5 changes: 3 additions & 2 deletions pkg/clusterversion/key_string.go

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

2 changes: 2 additions & 0 deletions pkg/migration/migrations/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -6,6 +6,7 @@ go_library(
"alter_web_sessions_create_indexes.go",
"database_role_settings.go",
"delete_deprecated_namespace_tabledesc.go",
"ensure_no_draining_names.go",
"fix_descriptor_migration.go",
"join_tokens.go",
"migrations.go",
Expand Down Expand Up @@ -67,6 +68,7 @@ go_test(
srcs = [
"alter_web_sessions_create_indexes_test.go",
"delete_deprecated_namespace_tabledesc_external_test.go",
"ensure_no_draining_names_external_test.go",
"fix_descriptor_migration_external_test.go",
"helpers_test.go",
"main_test.go",
Expand Down
76 changes: 76 additions & 0 deletions pkg/migration/migrations/ensure_no_draining_names.go
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 pkg/migration/migrations/ensure_no_draining_names_external_test.go
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())
}
}
6 changes: 6 additions & 0 deletions pkg/migration/migrations/migrations.go
Original file line number Diff line number Diff line change
Expand Up @@ -124,6 +124,12 @@ var migrations = []migration.Migration{
NoPrecondition,
sqlStatsTablesMigration,
),
migration.NewTenantMigration(
"ensure that draining names are no longer in use",
toCV(clusterversion.DrainingNamesMigration),
NoPrecondition,
ensureNoDrainingNames,
),
}

func init() {
Expand Down
3 changes: 0 additions & 3 deletions pkg/sql/catalog/descriptor.go
Original file line number Diff line number Diff line change
Expand Up @@ -770,9 +770,6 @@ func FormatSafeDescriptorProperties(w *redact.StringBuilder, desc Descriptor) {
w.Printf(", OfflineReason: %q", redact.Safe(offlineReason))
}
}
if drainingNames := desc.GetDrainingNames(); len(drainingNames) > 0 {
w.Printf(", NumDrainingNames: %d", len(drainingNames))
}
}

// IsSystemDescriptor returns true iff the descriptor is a system or a reserved
Expand Down
19 changes: 0 additions & 19 deletions pkg/sql/catalog/descriptor_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -55,25 +55,6 @@ func TestFormatSafeDescriptorProperties(t *testing.T) {
}).BuildCreatedMutable(),
exp: "ID: 12, Version: 1, IsUncommitted: true, ModificationTime: \"0,0\", State: PUBLIC",
},
{
desc: func() catalog.Descriptor {
desc := tabledesc.NewBuilder(&descpb.TableDescriptor{
ID: 27,
Version: 2,
ParentID: 12,
UnexposedParentSchemaID: 51,
State: descpb.DescriptorState_PUBLIC,
}).BuildExistingMutableTable()
desc.MaybeIncrementVersion()
//lint:ignore SA1019 deprecated method call is OK
desc.AddDrainingName(descpb.NameInfo{
ParentID: 12,
ParentSchemaID: 51,
})
return desc.ImmutableCopy()
}(),
exp: "ID: 27, Version: 3, IsUncommitted: true, ModificationTime: \"0,0\", ParentID: 12, ParentSchemaID: 51, State: PUBLIC, NumDrainingNames: 1",
},
} {
t.Run("", func(t *testing.T) {
var buf redact.StringBuilder
Expand Down
4 changes: 0 additions & 4 deletions pkg/sql/catalog/descs/collection_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -245,10 +245,6 @@ func TestAddUncommittedDescriptorAndMutableResolution(t *testing.T) {
require.Same(t, immByName, immByID)

mut.Name = "new_name"
//lint:ignore SA1019 deprecated method call is OK
mut.SetDrainingNames([]descpb.NameInfo{{
Name: "db",
}})

// Don't write the descriptor, just write the namespace entry.
// This will mean that resolution still is based on the old name.
Expand Down
5 changes: 5 additions & 0 deletions pkg/sql/catalog/descs/kv_descriptors.go
Original file line number Diff line number Diff line change
Expand Up @@ -193,6 +193,11 @@ func (kd *kvDescriptors) getByName(
// Immediately after a RENAME an old name still points to the descriptor
// during the drain phase for the name. Do not return a descriptor during
// draining.
//
// TODO(postamar): remove this after 22.1 is release.
// At that point, draining names will no longer have to be supported.
// We can then consider making the descriptor collection aware of
// uncommitted namespace operations.
return nil, nil
}
return desc, nil
Expand Down
2 changes: 2 additions & 0 deletions pkg/sql/catalog/descs/uncommitted_descriptors.go
Original file line number Diff line number Diff line change
Expand Up @@ -97,6 +97,8 @@ type uncommittedDescriptors struct {
// as all of the known draining names. The idea is that if we find that
// a name is not in the above map but is in the set, then we can avoid
// doing a lookup.
//
// TODO(postamar): better uncommitted namespace changes handling after 22.1.
descNames nstree.Set
}

Expand Down
Loading

0 comments on commit 3daa0db

Please sign in to comment.