diff --git a/.github/CODEOWNERS b/.github/CODEOWNERS
index d9c9172ca928..a3906418fa07 100644
--- a/.github/CODEOWNERS
+++ b/.github/CODEOWNERS
@@ -346,7 +346,6 @@
/pkg/security/clientsecopts/ @cockroachdb/unowned @cockroachdb/server-prs @cockroachdb/sql-experience @cockroachdb/prodsec
/pkg/settings/ @cockroachdb/unowned
/pkg/spanconfig/ @cockroachdb/kv-prs
-/pkg/startupmigrations/ @cockroachdb/unowned @cockroachdb/sql-schema
/pkg/repstream/ @cockroachdb/disaster-recovery
/pkg/testutils/ @cockroachdb/test-eng-noreview
/pkg/testutils/reduce/ @cockroachdb/sql-queries
diff --git a/docs/generated/sql/functions.md b/docs/generated/sql/functions.md
index d6484ab8081c..0cbdefa404ad 100644
--- a/docs/generated/sql/functions.md
+++ b/docs/generated/sql/functions.md
@@ -3044,8 +3044,6 @@ may increase either contention or retry errors, or both.
crdb_internal.create_session_revival_token() → bytes | Generate a token that can be used to create a new session for the current user.
diff --git a/pkg/BUILD.bazel b/pkg/BUILD.bazel
index d9b30903fef3..387686d27c57 100644
--- a/pkg/BUILD.bazel
+++ b/pkg/BUILD.bazel
@@ -507,8 +507,6 @@ ALL_TESTS = [
"//pkg/sql/types:types_test",
"//pkg/sql:sql_disallowed_imports_test",
"//pkg/sql:sql_test",
- "//pkg/startupmigrations/leasemanager:leasemanager_test",
- "//pkg/startupmigrations:startupmigrations_test",
"//pkg/storage/enginepb:enginepb_test",
"//pkg/storage/fs:fs_test",
"//pkg/storage/metamorphic:metamorphic_test",
@@ -1829,10 +1827,6 @@ GO_TARGETS = [
"//pkg/sql/vtable:vtable",
"//pkg/sql:sql",
"//pkg/sql:sql_test",
- "//pkg/startupmigrations/leasemanager:leasemanager",
- "//pkg/startupmigrations/leasemanager:leasemanager_test",
- "//pkg/startupmigrations:startupmigrations",
- "//pkg/startupmigrations:startupmigrations_test",
"//pkg/storage/enginepb:enginepb",
"//pkg/storage/enginepb:enginepb_test",
"//pkg/storage/fs:fs",
@@ -1919,7 +1913,9 @@ GO_TARGETS = [
"//pkg/ts:ts_test",
"//pkg/ui:ui",
"//pkg/ui:ui_test",
+ "//pkg/upgrade/migrationstable:migrationstable",
"//pkg/upgrade/nodelivenesstest:nodelivenesstest",
+ "//pkg/upgrade/upgradebase:upgradebase",
"//pkg/upgrade/upgradecluster:upgradecluster",
"//pkg/upgrade/upgradecluster:upgradecluster_test",
"//pkg/upgrade/upgradejob:upgrade_job",
@@ -2909,8 +2905,6 @@ GET_X_DATA_TARGETS = [
"//pkg/sql/ttl/ttlschedule:get_x_data",
"//pkg/sql/types:get_x_data",
"//pkg/sql/vtable:get_x_data",
- "//pkg/startupmigrations:get_x_data",
- "//pkg/startupmigrations/leasemanager:get_x_data",
"//pkg/storage:get_x_data",
"//pkg/storage/enginepb:get_x_data",
"//pkg/storage/fs:get_x_data",
@@ -2968,7 +2962,9 @@ GET_X_DATA_TARGETS = [
"//pkg/ts/tspb:get_x_data",
"//pkg/ui:get_x_data",
"//pkg/upgrade:get_x_data",
+ "//pkg/upgrade/migrationstable:get_x_data",
"//pkg/upgrade/nodelivenesstest:get_x_data",
+ "//pkg/upgrade/upgradebase:get_x_data",
"//pkg/upgrade/upgradecluster:get_x_data",
"//pkg/upgrade/upgradejob:get_x_data",
"//pkg/upgrade/upgrademanager:get_x_data",
diff --git a/pkg/base/license.go b/pkg/base/license.go
index 0d3c82b88a9d..28881d933ea9 100644
--- a/pkg/base/license.go
+++ b/pkg/base/license.go
@@ -32,6 +32,13 @@ var CheckEnterpriseEnabled = func(_ *cluster.Settings, _ uuid.UUID, feature stri
return errEnterpriseNotEnabled // nb: this is squarely in the hot path on OSS builds
}
+// CCLDistributionAndEnterpriseEnabled is a simpler version of
+// CheckEnterpriseEnabled which doesn't take in feature-related info and doesn't
+// return an error with a nice message.
+var CCLDistributionAndEnterpriseEnabled = func(st *cluster.Settings, clusterID uuid.UUID) bool {
+ return CheckEnterpriseEnabled(st, clusterID, "" /* feature */) == nil
+}
+
var licenseTTLMetadata = metric.Metadata{
// This metric name isn't namespaced for backwards
// compatibility. The prior version of this metric was manually
diff --git a/pkg/ccl/backupccl/backup_test.go b/pkg/ccl/backupccl/backup_test.go
index 28a06ac74f14..619185243c37 100644
--- a/pkg/ccl/backupccl/backup_test.go
+++ b/pkg/ccl/backupccl/backup_test.go
@@ -727,7 +727,7 @@ func TestBackupAndRestoreJobDescription(t *testing.T) {
asOf1 := strings.TrimPrefix(matches[1], "/full")
sqlDB.CheckQueryResults(
- t, "SELECT description FROM [SHOW JOBS] WHERE status != 'failed'",
+ t, "SELECT description FROM [SHOW JOBS] WHERE job_type != 'MIGRATION' AND status != 'failed'",
[][]string{
{fmt.Sprintf("BACKUP TO ('%s', '%s', '%s')", backups[0].(string), backups[1].(string),
backups[2].(string))},
@@ -5608,7 +5608,7 @@ func TestBackupRestoreShowJob(t *testing.T) {
// TODO (lucy): Update this if/when we decide to change how these jobs queued by
// the startup migration are handled.
sqlDB.CheckQueryResults(
- t, "SELECT description FROM [SHOW JOBS] WHERE description != 'updating privileges' ORDER BY description",
+ t, "SELECT description FROM [SHOW JOBS] WHERE job_type != 'MIGRATION' AND description != 'updating privileges' ORDER BY description",
[][]string{
{"BACKUP DATABASE data TO 'nodelocal://0/foo' WITH revision_history = true"},
{"RESTORE TABLE data.bank FROM 'nodelocal://0/foo' WITH into_db = 'data 2', skip_missing_foreign_keys"},
diff --git a/pkg/ccl/changefeedccl/changefeed_processors.go b/pkg/ccl/changefeedccl/changefeed_processors.go
index 315677b898ae..b8d902850025 100644
--- a/pkg/ccl/changefeedccl/changefeed_processors.go
+++ b/pkg/ccl/changefeedccl/changefeed_processors.go
@@ -608,6 +608,7 @@ func (ca *changeAggregator) emitResolved(batch jobspb.ResolvedSpans) error {
rowenc.EncDatum{Datum: tree.DNull}, // key
rowenc.EncDatum{Datum: tree.DNull}, // value
})
+ ca.metrics.ResolvedMessages.Inc(1)
ca.recentKVCount = 0
return nil
diff --git a/pkg/ccl/kvccl/kvtenantccl/BUILD.bazel b/pkg/ccl/kvccl/kvtenantccl/BUILD.bazel
index a76d53840aed..fdf7b53d3535 100644
--- a/pkg/ccl/kvccl/kvtenantccl/BUILD.bazel
+++ b/pkg/ccl/kvccl/kvtenantccl/BUILD.bazel
@@ -82,7 +82,7 @@ go_test(
"//pkg/testutils/sqlutils",
"//pkg/testutils/testcluster",
"//pkg/upgrade",
- "//pkg/upgrade/upgrades",
+ "//pkg/upgrade/upgradebase",
"//pkg/util",
"//pkg/util/grpcutil",
"//pkg/util/hlc",
diff --git a/pkg/ccl/kvccl/kvtenantccl/tenant_upgrade_test.go b/pkg/ccl/kvccl/kvtenantccl/tenant_upgrade_test.go
index 2062e8dcdce9..1b27a1ae8516 100644
--- a/pkg/ccl/kvccl/kvtenantccl/tenant_upgrade_test.go
+++ b/pkg/ccl/kvccl/kvtenantccl/tenant_upgrade_test.go
@@ -26,7 +26,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/testutils/sqlutils"
"github.com/cockroachdb/cockroach/pkg/testutils/testcluster"
"github.com/cockroachdb/cockroach/pkg/upgrade"
- "github.com/cockroachdb/cockroach/pkg/upgrade/upgrades"
+ "github.com/cockroachdb/cockroach/pkg/upgrade/upgradebase"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/stop"
@@ -251,16 +251,16 @@ func TestTenantUpgradeFailure(t *testing.T) {
TenantID: roachpb.MustMakeTenantID(id),
TestingKnobs: base.TestingKnobs{
JobsTestingKnobs: jobs.NewTestingKnobsWithShortIntervals(),
- UpgradeManager: &upgrade.TestingKnobs{
+ UpgradeManager: &upgradebase.TestingKnobs{
ListBetweenOverride: func(from, to roachpb.Version) []roachpb.Version {
return []roachpb.Version{v1, v2}
},
- RegistryOverride: func(v roachpb.Version) (upgrade.Upgrade, bool) {
+ RegistryOverride: func(v roachpb.Version) (upgradebase.Upgrade, bool) {
switch v {
case v1:
return upgrade.NewTenantUpgrade("testing",
v1,
- upgrades.NoPrecondition,
+ upgrade.NoPrecondition,
func(
ctx context.Context, version clusterversion.ClusterVersion, deps upgrade.TenantDeps,
) error {
@@ -269,7 +269,7 @@ func TestTenantUpgradeFailure(t *testing.T) {
case v2:
return upgrade.NewTenantUpgrade("testing next",
v2,
- upgrades.NoPrecondition,
+ upgrade.NoPrecondition,
func(
ctx context.Context, version clusterversion.ClusterVersion, deps upgrade.TenantDeps,
) error {
diff --git a/pkg/ccl/streamingccl/streamingest/BUILD.bazel b/pkg/ccl/streamingccl/streamingest/BUILD.bazel
index 05c03fdad4df..061efda48fa6 100644
--- a/pkg/ccl/streamingccl/streamingest/BUILD.bazel
+++ b/pkg/ccl/streamingccl/streamingest/BUILD.bazel
@@ -118,6 +118,7 @@ go_test(
"//pkg/testutils/sqlutils",
"//pkg/testutils/storageutils",
"//pkg/testutils/testcluster",
+ "//pkg/upgrade/upgradebase",
"//pkg/util/hlc",
"//pkg/util/json",
"//pkg/util/leaktest",
diff --git a/pkg/ccl/streamingccl/streamingest/stream_ingestion_frontier_processor_test.go b/pkg/ccl/streamingccl/streamingest/stream_ingestion_frontier_processor_test.go
index 25748d5ae3f5..0c24d4524ca6 100644
--- a/pkg/ccl/streamingccl/streamingest/stream_ingestion_frontier_processor_test.go
+++ b/pkg/ccl/streamingccl/streamingest/stream_ingestion_frontier_processor_test.go
@@ -31,6 +31,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/testutils/distsqlutils"
"github.com/cockroachdb/cockroach/pkg/testutils/skip"
"github.com/cockroachdb/cockroach/pkg/testutils/testcluster"
+ "github.com/cockroachdb/cockroach/pkg/upgrade/upgradebase"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
"github.com/cockroachdb/cockroach/pkg/util/limit"
@@ -53,6 +54,10 @@ func TestStreamIngestionFrontierProcessor(t *testing.T) {
// be adopted as the processors are being manually executed in the test.
DisableAdoptions: true,
},
+ // DisableAdoptions needs this.
+ UpgradeManager: &upgradebase.TestingKnobs{
+ DontUseJobs: true,
+ },
},
},
})
diff --git a/pkg/cli/BUILD.bazel b/pkg/cli/BUILD.bazel
index f43dde40b92c..01270a147ebc 100644
--- a/pkg/cli/BUILD.bazel
+++ b/pkg/cli/BUILD.bazel
@@ -176,13 +176,13 @@ go_library(
"//pkg/sql/sem/eval",
"//pkg/sql/sem/tree",
"//pkg/sql/sqlstats",
- "//pkg/startupmigrations",
"//pkg/storage",
"//pkg/storage/enginepb",
"//pkg/storage/fs",
"//pkg/testutils/serverutils",
"//pkg/ts",
"//pkg/ts/tspb",
+ "//pkg/upgrade/upgrades",
"//pkg/util",
"//pkg/util/cgroups",
"//pkg/util/contextutil",
diff --git a/pkg/cli/gen.go b/pkg/cli/gen.go
index 448d72f22ccf..46fec5a06724 100644
--- a/pkg/cli/gen.go
+++ b/pkg/cli/gen.go
@@ -25,7 +25,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/cli/clisqlexec"
"github.com/cockroachdb/cockroach/pkg/settings"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
- "github.com/cockroachdb/cockroach/pkg/startupmigrations"
+ "github.com/cockroachdb/cockroach/pkg/upgrade/upgrades"
"github.com/cockroachdb/errors/oserror"
"github.com/spf13/cobra"
"github.com/spf13/cobra/doc"
@@ -239,7 +239,7 @@ Output the list of cluster settings known to this binary.
defaultVal = sm.SettingsListDefault()
} else {
defaultVal = setting.String(&s.SV)
- if override, ok := startupmigrations.SettingsDefaultOverrides[name]; ok {
+ if override, ok := upgrades.SettingsDefaultOverrides[name]; ok {
defaultVal = override
}
}
diff --git a/pkg/cli/testdata/doctor/test_examine_cluster b/pkg/cli/testdata/doctor/test_examine_cluster
index afeb09eee874..03188c63c49d 100644
--- a/pkg/cli/testdata/doctor/test_examine_cluster
+++ b/pkg/cli/testdata/doctor/test_examine_cluster
@@ -3,5 +3,5 @@ debug doctor examine cluster
debug doctor examine cluster
Examining 48 descriptors and 47 namespace entries...
ParentID 100, ParentSchemaID 101: relation "foo" (105): expected matching namespace entry, found none
-Examining 4 jobs...
+Examining 12 jobs...
ERROR: validation failed
diff --git a/pkg/clusterversion/cockroach_versions.go b/pkg/clusterversion/cockroach_versions.go
index 76e7f1fb6b5f..e163cc2fbd55 100644
--- a/pkg/clusterversion/cockroach_versions.go
+++ b/pkg/clusterversion/cockroach_versions.go
@@ -161,6 +161,19 @@ type Key int
const (
invalidVersionKey Key = iota - 1 // want first named one to start at zero
+ // VPrimordial versions are used by upgrades below BinaryMinSupportedVersion,
+ // for whom the exact version they were associated with no longer matters.
+
+ VPrimordial1
+ VPrimordial2
+ VPrimordial3
+ VPrimordial4
+ VPrimordial5
+ VPrimordial6
+ VPrimordial7
+ VPrimordial8
+ VPrimordialMax
+
// V22_1 is CockroachDB v22.1. It's used for all v22.1.x patch releases.
V22_1
@@ -360,6 +373,42 @@ const TODOPreV22_1 = V22_1
// large number to every major if building from master, so as to ensure that
// master builds cannot be upgraded to release-branch builds.
var rawVersionsSingleton = keyedVersions{
+ {
+ Key: VPrimordial1,
+ Version: roachpb.Version{Major: 0, Minor: 0, Internal: 2},
+ },
+ {
+ Key: VPrimordial2,
+ Version: roachpb.Version{Major: 0, Minor: 0, Internal: 4},
+ },
+ {
+ Key: VPrimordial3,
+ Version: roachpb.Version{Major: 0, Minor: 0, Internal: 6},
+ },
+ {
+ Key: VPrimordial4,
+ Version: roachpb.Version{Major: 0, Minor: 0, Internal: 8},
+ },
+ {
+ Key: VPrimordial5,
+ Version: roachpb.Version{Major: 0, Minor: 0, Internal: 10},
+ },
+ {
+ Key: VPrimordial6,
+ Version: roachpb.Version{Major: 0, Minor: 0, Internal: 12},
+ },
+ {
+ Key: VPrimordial7,
+ Version: roachpb.Version{Major: 0, Minor: 0, Internal: 14},
+ },
+ {
+ Key: VPrimordial8,
+ Version: roachpb.Version{Major: 0, Minor: 0, Internal: 16},
+ },
+ {
+ Key: VPrimordialMax,
+ Version: roachpb.Version{Major: 0, Minor: 0, Internal: 424242},
+ },
{
Key: V22_1,
Version: roachpb.Version{Major: 22, Minor: 1},
@@ -585,8 +634,16 @@ var versionsSingleton = func() keyedVersions {
// then on to 1000004, etc.
skipFirst := envutil.EnvOrDefaultBool("COCKROACH_UPGRADE_TO_DEV_VERSION", false)
const devOffset = 1000000
+ first := true
for i := range rawVersionsSingleton {
- if i == 0 && skipFirst {
+ // VPrimordial versions are not offset; they don't matter for the logic
+ // offsetting is used for.
+ if rawVersionsSingleton[i].Major == rawVersionsSingleton.MustByKey(VPrimordialMax).Major {
+ continue
+ }
+
+ if skipFirst && first {
+ first = false
continue
}
rawVersionsSingleton[i].Major += devOffset
diff --git a/pkg/clusterversion/keyed_versions.go b/pkg/clusterversion/keyed_versions.go
index 83c0a1fda22d..7b06ea7b040c 100644
--- a/pkg/clusterversion/keyed_versions.go
+++ b/pkg/clusterversion/keyed_versions.go
@@ -134,3 +134,7 @@ func (kv keyedVersions) Validate() error {
return nil
}
+
+func (kv keyedVersions) LastVersion() roachpb.Version {
+ return kv[len(kv)-1].Version
+}
diff --git a/pkg/cmd/roachtest/tests/cdc.go b/pkg/cmd/roachtest/tests/cdc.go
index 51b7e6e19de8..0a37a0e3fbed 100644
--- a/pkg/cmd/roachtest/tests/cdc.go
+++ b/pkg/cmd/roachtest/tests/cdc.go
@@ -29,6 +29,7 @@ import (
"sort"
"strconv"
"strings"
+ "sync"
"sync/atomic"
"time"
@@ -43,6 +44,8 @@ import (
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/test"
"github.com/cockroachdb/cockroach/pkg/jobs/jobspb"
"github.com/cockroachdb/cockroach/pkg/roachprod/install"
+ "github.com/cockroachdb/cockroach/pkg/roachprod/logger"
+ "github.com/cockroachdb/cockroach/pkg/roachprod/prometheus"
"github.com/cockroachdb/cockroach/pkg/testutils/sqlutils"
"github.com/cockroachdb/cockroach/pkg/util/protoutil"
"github.com/cockroachdb/cockroach/pkg/util/retry"
@@ -50,85 +53,79 @@ import (
"github.com/cockroachdb/errors"
)
-type workloadType string
-
-const (
- tpccWorkloadType workloadType = "tpcc"
- ledgerWorkloadType workloadType = "ledger"
-)
-
// kafkaCreateTopicRetryDuration is the retry duration we use while
// trying to create a Kafka topic after setting it up on a
// node. Without retrying, a `kafka controller not available` error is
// seen with a 1-5% probability
var kafkaCreateTopicRetryDuration = 1 * time.Minute
-type sinkType int32
+type sinkType string
const (
- cloudStorageSink sinkType = iota + 1
- webhookSink
- pubsubSink
+ cloudStorageSink sinkType = "cloudstorage"
+ webhookSink sinkType = "webhook"
+ pubsubSink sinkType = "pubsub"
+ kafkaSink sinkType = "kafka"
+ nullSink sinkType = "null"
)
-type cdcTestArgs struct {
- workloadType workloadType
- tpccWarehouseCount int
- workloadDuration string
- initialScan bool
- kafkaChaos bool
- crdbChaos bool
- whichSink sinkType
- sinkURI string
- assumeRole string
-
- // preStartStatements are executed after the workload is initialized but before the
- // changefeed is created.
- preStartStatements []string
-
- targetInitialScanLatency time.Duration
- targetSteadyLatency time.Duration
- targetTxnPerSecond float64
-}
-
-func cdcBasicTest(ctx context.Context, t test.Test, c cluster.Cluster, args cdcTestArgs) {
- crdbNodes := c.Range(1, c.Spec().NodeCount-1)
- workloadNode := c.Node(c.Spec().NodeCount)
- kafkaNode := c.Node(c.Spec().NodeCount)
- c.Put(ctx, t.Cockroach(), "./cockroach")
- c.Put(ctx, t.DeprecatedWorkload(), "./workload", workloadNode)
- c.Start(ctx, t.L(), option.DefaultStartOpts(), install.MakeClusterSettings(), crdbNodes)
+type cdcTester struct {
+ ctx context.Context
+ t test.Test
+ mon cluster.Monitor
+ cluster cluster.Cluster
+ crdbNodes option.NodeListOption
+ workloadNode option.NodeListOption
+ logger *logger.Logger
- db := c.Conn(ctx, t.L(), 1)
- defer stopFeeds(db)
- kafka := kafkaManager{
- t: t,
- c: c,
- nodes: kafkaNode,
+ // sinkType -> sinkURI
+ sinkCache map[sinkType]string
+
+ workloadWg *sync.WaitGroup
+ doneCh chan struct{}
+}
+
+func (ct *cdcTester) startCRDBChaos() {
+ chaosStopper := make(chan time.Time)
+ ct.mon.Go(func(ctx context.Context) error {
+ select {
+ case <-ct.doneCh:
+ case <-ctx.Done():
+ }
+ chaosStopper <- timeutil.Now()
+ return nil
+ })
+ ch := Chaos{
+ Timer: Periodic{Period: 2 * time.Minute, DownTime: 20 * time.Second},
+ Target: ct.crdbNodes.RandNode,
+ Stopper: chaosStopper,
}
+ ct.mon.Go(ch.Runner(ct.cluster, ct.t, ct.mon))
+}
+func (ct *cdcTester) setupSink(args feedArgs) string {
var sinkURI string
- if args.sinkURI != "" {
- sinkURI = args.sinkURI
- } else if args.whichSink == cloudStorageSink {
+ switch args.sinkType {
+ case nullSink:
+ sinkURI = "null://"
+ case cloudStorageSink:
ts := timeutil.Now().Format(`20060102150405`)
// cockroach-tmp is a multi-region bucket with a TTL to clean up old
// data.
sinkURI = `experimental-gs://cockroach-tmp/roachtest/` + ts + "?AUTH=implicit"
- } else if args.whichSink == webhookSink {
- // setup a sample cert for use by the mock sink
+ case webhookSink:
cert, certEncoded, err := cdctest.NewCACertBase64Encoded()
if err != nil {
- t.Fatal(err)
+ ct.t.Fatal(err)
}
sinkDest, err := cdctest.StartMockWebhookSink(cert)
if err != nil {
- t.Fatal(err)
+ ct.t.Fatal(err)
}
sinkDestHost, err := url.Parse(sinkDest.URL())
if err != nil {
- t.Fatal(err)
+ ct.t.Fatal(err)
}
params := sinkDestHost.Query()
@@ -136,176 +133,367 @@ func cdcBasicTest(ctx context.Context, t test.Test, c cluster.Cluster, args cdcT
sinkDestHost.RawQuery = params.Encode()
sinkURI = fmt.Sprintf("webhook-%s", sinkDestHost.String())
- } else if args.whichSink == pubsubSink {
+ case pubsubSink:
sinkURI = changefeedccl.GcpScheme + `://cockroach-ephemeral` + "?AUTH=implicit&topic_name=pubsubSink-roachtest®ion=us-east1"
- } else {
- t.Status("installing kafka")
- kafka.install(ctx)
- kafka.start(ctx)
- sinkURI = kafka.sinkURL(ctx)
+ case kafkaSink:
+ kafkaNode := ct.cluster.Node(ct.cluster.Spec().NodeCount)
+ kafka := kafkaManager{
+ t: ct.t,
+ c: ct.cluster,
+ nodes: kafkaNode,
+ }
+ kafka.install(ct.ctx)
+ kafka.start(ct.ctx)
+
+ if args.kafkaChaos {
+ ct.mon.Go(func(ctx context.Context) error {
+ period, downTime := 2*time.Minute, 20*time.Second
+ return kafka.chaosLoop(ctx, period, downTime, ct.doneCh)
+ })
+ }
+
+ sinkURI = kafka.sinkURL(ct.ctx)
+ default:
+ ct.t.Fatalf("unknown sink provided: %s", args.sinkType)
}
if args.assumeRole != "" {
sinkURI = sinkURI + "&ASSUME_ROLE=" + args.assumeRole
}
- m := c.NewMonitor(ctx, crdbNodes)
- workloadCompleteCh := make(chan struct{}, 1)
+ return sinkURI
+}
- workloadStart := timeutil.Now()
- if args.workloadType == tpccWorkloadType {
- t.Status("installing TPCC")
- tpcc := tpccWorkload{
- sqlNodes: crdbNodes,
- workloadNodes: workloadNode,
- tpccWarehouseCount: args.tpccWarehouseCount,
- // TolerateErrors if crdbChaos is true; otherwise, the workload will fail
- // if it attempts to use the node which was brought down by chaos.
- tolerateErrors: args.crdbChaos,
- }
+type tpccArgs struct {
+ warehouses int
+ duration string
+ tolerateErrors bool
+}
+
+func (ct *cdcTester) runTPCCWorkload(args tpccArgs) {
+ tpcc := tpccWorkload{
+ sqlNodes: ct.crdbNodes,
+ workloadNodes: ct.workloadNode,
+ tpccWarehouseCount: args.warehouses,
+ // TolerateErrors if crdbChaos is true; otherwise, the workload will fail
+ // if it attempts to use the node which was brought down by chaos.
+ tolerateErrors: args.tolerateErrors,
+ }
+
+ if !ct.t.SkipInit() {
+ ct.t.Status("installing TPCC workload")
+ tpcc.install(ct.ctx, ct.cluster)
+ } else {
+ ct.t.Status("skipping TPCC installation")
+ }
- tpcc.install(ctx, c)
+ if args.duration != "" {
// TODO(dan,ajwerner): sleeping momentarily before running the workload
// mitigates errors like "error in newOrder: missing stock row" from tpcc.
time.Sleep(2 * time.Second)
- t.Status("initiating workload")
- m.Go(func(ctx context.Context) error {
- defer func() { close(workloadCompleteCh) }()
- tpcc.run(ctx, c, args.workloadDuration)
+ ct.t.Status("initiating TPCC workload")
+ ct.mon.Go(func(ctx context.Context) error {
+ ct.workloadWg.Add(1)
+ defer ct.workloadWg.Done()
+ tpcc.run(ctx, ct.cluster, args.duration)
return nil
})
} else {
- t.Status("installing Ledger Workload")
- lw := ledgerWorkload{
- sqlNodes: crdbNodes,
- workloadNodes: workloadNode,
+ ct.t.Status("skipping TPCC run")
+ }
+}
+
+type ledgerArgs struct {
+ duration string
+}
+
+func (ct *cdcTester) runLedgerWorkload(args ledgerArgs) {
+ lw := ledgerWorkload{
+ sqlNodes: ct.crdbNodes,
+ workloadNodes: ct.workloadNode,
+ }
+ if !ct.t.SkipInit() {
+ ct.t.Status("installing Ledger workload")
+ lw.install(ct.ctx, ct.cluster)
+ } else {
+ ct.t.Status("skipping Ledger installation")
+ }
+
+ ct.t.Status("initiating Ledger workload")
+ ct.mon.Go(func(ctx context.Context) error {
+ ct.workloadWg.Add(1)
+ defer ct.workloadWg.Done()
+ lw.run(ctx, ct.cluster, args.duration)
+ return nil
+ })
+}
+
+func (ct *cdcTester) DB() *gosql.DB {
+ return ct.cluster.Conn(ct.ctx, ct.t.L(), 1)
+}
+
+func (ct *cdcTester) Close() {
+ ct.t.Status("cdcTester closing")
+ close(ct.doneCh)
+ ct.mon.Wait()
+
+ _, _ = ct.DB().Exec(`CANCEL ALL CHANGEFEED JOBS;`)
+
+ if !ct.t.IsDebug() {
+ if err := ct.cluster.StopGrafana(ct.ctx, ct.logger, ct.t.ArtifactsDir()); err != nil {
+ ct.t.Errorf("error shutting down prometheus/grafana: %s", err)
+ }
+ }
+
+ ct.logger.Close()
+ ct.t.Status("cdcTester closed")
+}
+
+type changefeedJob struct {
+ ctx context.Context
+ sinkURI string
+ jobID int
+ targets []string
+ opts map[string]string
+ logger *logger.Logger
+ db *gosql.DB
+ tolerateErrors bool
+}
+
+func (cj *changefeedJob) Label() string {
+ if label, ok := cj.opts["metrics_label"]; ok {
+ return label
+ }
+ return "default"
+}
+
+var allTpccTargets []string = []string{
+ `tpcc.warehouse`,
+ `tpcc.district`,
+ `tpcc.customer`,
+ `tpcc.history`,
+ `tpcc.order`,
+ `tpcc.new_order`,
+ `tpcc.item`,
+ `tpcc.stock`,
+ `tpcc.order_line`,
+}
+
+var allLedgerTargets []string = []string{
+ `ledger.customer`,
+ `ledger.transaction`,
+ `ledger.entry`,
+ `ledger.session`,
+}
+
+type feedArgs struct {
+ sinkType sinkType
+ targets []string
+ opts map[string]string
+ kafkaChaos bool
+ assumeRole string
+ tolerateErrors bool
+}
+
+// TODO: Maybe move away from feedArgs since its only 3 things
+func (ct *cdcTester) newChangefeed(args feedArgs) changefeedJob {
+ ct.t.Status(fmt.Sprintf("initiating %s sink", args.sinkType))
+ sinkURI := ct.setupSink(args)
+ ct.t.Status(fmt.Sprintf("using sinkURI %s", sinkURI))
+
+ targetsStr := strings.Join(args.targets, ", ")
+
+ feedOptions := make(map[string]string)
+ feedOptions["min_checkpoint_frequency"] = "'10s'"
+ if args.sinkType == cloudStorageSink || args.sinkType == webhookSink {
+ // Webhook and cloudstorage don't have a concept of keys and therefore
+ // require envelope=wrapped
+ feedOptions["envelope"] = "wrapped"
+
+ feedOptions["resolved"] = "'10s'"
+
+ } else {
+ feedOptions["resolved"] = ""
+ }
+
+ for option, value := range args.opts {
+ feedOptions[option] = value
+ if option == "initial_scan_only" || (option == "initial_scan" && value == "'only'") {
+ delete(feedOptions, "resolved")
}
- lw.install(ctx, c)
+ }
- t.Status("initiating workload")
- m.Go(func(ctx context.Context) error {
- defer func() { close(workloadCompleteCh) }()
- lw.run(ctx, c, args.workloadDuration)
- return nil
- })
+ ct.t.Status(fmt.Sprintf(
+ "creating %s changefeed into targets %s with options (%+v)",
+ args.sinkType, args.targets, feedOptions,
+ ))
+ db := ct.DB()
+ jobID, err := newChangefeedCreator(db, targetsStr, sinkURI).
+ With(feedOptions).Create()
+ if err != nil {
+ ct.t.Fatalf("failed to create changefeed: %s", err.Error())
}
- changefeedLogger, err := t.L().ChildLogger("changefeed")
+ cj := changefeedJob{
+ ctx: ct.ctx,
+ sinkURI: sinkURI,
+ jobID: jobID,
+ targets: args.targets,
+ opts: feedOptions,
+ db: db,
+ tolerateErrors: args.tolerateErrors,
+ }
+
+ ct.t.Status(fmt.Sprintf("created changefeed %s with jobID %d", cj.Label(), jobID))
+
+ return cj
+}
+
+func (ct *cdcTester) runFeedLatencyVerifier(cj changefeedJob, targets latencyTargets) {
+ info, err := getChangefeedInfo(ct.DB(), cj.jobID)
if err != nil {
- t.Fatal(err)
+ ct.t.Fatalf("failed to get changefeed info: %s", err.Error())
}
- defer changefeedLogger.Close()
+
verifier := makeLatencyVerifier(
- "changefeed",
- args.targetInitialScanLatency,
- args.targetSteadyLatency,
- changefeedLogger,
+ fmt.Sprintf("changefeed[%s]", cj.Label()),
+ targets.initialScanLatency,
+ targets.steadyLatency,
+ ct.logger,
func(db *gosql.DB, jobID int) (jobInfo, error) { return getChangefeedInfo(db, jobID) },
- t.Status,
- args.crdbChaos,
+ ct.t.Status,
+ cj.tolerateErrors,
)
- defer verifier.maybeLogLatencyHist()
+ verifier.statementTime = info.statementTime
- m.Go(func(ctx context.Context) error {
- // Some of the tests have a tight enough bound on targetSteadyLatency
- // that the default for kv.closed_timestamp.target_duration means the
- // changefeed is never considered sufficiently caught up. We could
- // instead make targetSteadyLatency less aggressive, but it'd be nice to
- // keep it where it is.
- //
- // TODO(ssd): As of 797819b35f5 this is actually increasing rather than decreasing
- // the closed_timestamp.target_duration. We can probably remove this. However,
- // as of 2021-04-20, we want to understand why this test has started failing more often
- // before changing this.
- if _, err := db.Exec(
- `SET CLUSTER SETTING kv.closed_timestamp.target_duration='10s'`,
- ); err != nil {
- t.Fatal(err)
- }
-
- // With a target_duration of 10s, we won't see slow span logs from changefeeds untils we are > 100s
- // behind, which is well above the 60s targetSteadyLatency we have in some tests.
- if _, err := db.Exec(
- `SET CLUSTER SETTING changefeed.slow_span_log_threshold='30s'`,
- ); err != nil {
- // We don't hard fail here because, not all versions support this setting
- t.L().Printf("failed to set cluster setting: %s", err)
- }
-
- for _, stmt := range args.preStartStatements {
- _, err := db.ExecContext(ctx, stmt)
- if err != nil {
- t.Fatalf("failed pre-start statement %q: %s", stmt, err.Error())
- }
+ ct.mon.Go(func(ctx context.Context) error {
+ err := verifier.pollLatency(ctx, ct.DB(), cj.jobID, time.Second, ct.doneCh)
+ if err != nil {
+ return err
}
- var targets string
- if args.workloadType == tpccWorkloadType {
- targets = `tpcc.warehouse, tpcc.district, tpcc.customer, tpcc.history,
- tpcc.order, tpcc.new_order, tpcc.item, tpcc.stock,
- tpcc.order_line`
- } else {
- targets = `ledger.customer, ledger.transaction, ledger.entry, ledger.session`
- }
+ verifier.assertValid(ct.t)
+ verifier.maybeLogLatencyHist()
+ return nil
+ })
+}
- var options []cdcOption
- if args.whichSink == cloudStorageSink || args.whichSink == webhookSink {
- options = []cdcOption{
- {"resolved", "'10s'"},
- {"envelope", "wrapped"},
- {"min_checkpoint_frequency", "'10s'"},
- }
- } else {
- options = []cdcOption{{"resolved", ""}, {"min_checkpoint_frequency", "'10s'"}}
- }
- if !args.initialScan {
- options = append(options, cdcOption{"cursor", "'-1s'"})
+func (cj *changefeedJob) runFeedPoller(
+ ctx context.Context,
+ pollInterval time.Duration,
+ stopper chan struct{},
+ onInfo func(info *changefeedInfo),
+) error {
+ for {
+ select {
+ case <-ctx.Done():
+ return ctx.Err()
+ case <-stopper:
+ return nil
+ case <-time.After(pollInterval):
}
- jobID, err := newChangefeedCreator(db, targets, sinkURI).With(options...).Create()
+ info, err := getChangefeedInfo(cj.db, cj.jobID)
if err != nil {
+ if cj.tolerateErrors {
+ cj.logger.Printf("error getting changefeed info: %s", err)
+ continue
+ }
return err
}
- info, err := getChangefeedInfo(db, jobID)
- if err != nil {
- return err
+ onInfo(info)
+ }
+}
+
+func (ct *cdcTester) waitForWorkload() {
+ ct.workloadWg.Wait()
+}
+
+func (cj *changefeedJob) waitForCompletion() {
+ completionCh := make(chan struct{})
+ err := cj.runFeedPoller(cj.ctx, time.Second, completionCh, func(info *changefeedInfo) {
+ if info.status == "succeeded" || info.status == "failed" {
+ close(completionCh)
}
- verifier.statementTime = info.statementTime
- changefeedLogger.Printf("started changefeed at (%d) %s\n",
- verifier.statementTime.UnixNano(), verifier.statementTime)
- t.Status("watching changefeed")
- return verifier.pollLatency(ctx, db, jobID, time.Second, workloadCompleteCh)
})
+ if err != nil {
+ cj.logger.Printf("completion poller error: %s", err)
+ }
+}
- if args.kafkaChaos {
- m.Go(func(ctx context.Context) error {
- period, downTime := 2*time.Minute, 20*time.Second
- return kafka.chaosLoop(ctx, period, downTime, workloadCompleteCh)
- })
+func newCDCTester(ctx context.Context, t test.Test, c cluster.Cluster) cdcTester {
+ tester := cdcTester{
+ ctx: ctx,
+ t: t,
+ cluster: c,
+ crdbNodes: c.Range(1, c.Spec().NodeCount-1),
+ workloadNode: c.Node(c.Spec().NodeCount),
+ doneCh: make(chan struct{}),
+ sinkCache: make(map[sinkType]string),
+ workloadWg: &sync.WaitGroup{},
}
+ tester.mon = c.NewMonitor(ctx, tester.crdbNodes)
- if args.crdbChaos {
- chaosDuration, err := time.ParseDuration(args.workloadDuration)
- if err != nil {
- t.Fatal(err)
- }
- ch := Chaos{
- Timer: Periodic{Period: 2 * time.Minute, DownTime: 20 * time.Second},
- Target: crdbNodes.RandNode,
- Stopper: time.After(chaosDuration),
- }
- m.Go(ch.Runner(c, t, m))
+ changefeedLogger, err := t.L().ChildLogger("changefeed")
+ if err != nil {
+ t.Fatal(err)
}
- m.Wait()
+ tester.logger = changefeedLogger
- verifier.assertValid(t)
- workloadEnd := timeutil.Now()
- if args.targetTxnPerSecond > 0.0 {
- verifyTxnPerSecond(
- ctx, c, t, crdbNodes.RandNode(), workloadStart, workloadEnd, args.targetTxnPerSecond, 0.05,
- )
+ c.Put(ctx, t.Cockroach(), "./cockroach")
+
+ settings := install.MakeClusterSettings()
+ settings.Env = append(settings.Env, "COCKROACH_EXPERIMENTAL_ENABLE_PER_CHANGEFEED_METRICS=true")
+ c.Start(ctx, t.L(), option.DefaultStartOpts(), settings, tester.crdbNodes)
+
+ c.Put(ctx, t.DeprecatedWorkload(), "./workload", tester.workloadNode)
+
+ db := tester.DB()
+ // With a target_duration of 10s, we won't see slow span logs from changefeeds untils we are > 100s
+ // behind, which is well above the 60s targetSteadyLatency we have in some tests.
+ if _, err := db.Exec(
+ `SET CLUSTER SETTING changefeed.slow_span_log_threshold='30s'`,
+ ); err != nil {
+ // We don't hard fail here because, not all versions support this setting
+ t.L().Printf("failed to set cluster setting: %s", err)
+ }
+ if _, err := db.Exec("SET CLUSTER SETTING kv.rangefeed.enabled = true"); err != nil {
+ t.L().Printf("failed to set cluster setting: %s", err)
}
+ if _, err := db.Exec("SET CLUSTER SETTING server.child_metrics.enabled = true"); err != nil {
+ t.L().Printf("failed to set cluster setting: %s", err)
+ }
+
+ if !t.SkipInit() {
+ tester.startGrafana()
+ }
+
+ return tester
+}
+
+func (ct *cdcTester) startGrafana() {
+ // Setup the prometheus instance on the workload node
+ cfg := (&prometheus.Config{}).
+ WithPrometheusNode(ct.workloadNode.InstallNodes()[0]).
+ WithCluster(ct.crdbNodes.InstallNodes()).
+ WithNodeExporter(ct.crdbNodes.InstallNodes()).
+ WithGrafanaDashboard("https://go.crdb.dev/p/changefeed-roachtest-grafana-dashboard")
+ cfg.Grafana.Enabled = true
+ err := ct.cluster.StartGrafana(ct.ctx, ct.t.L(), cfg)
+ if err != nil {
+ ct.t.Errorf("error starting prometheus/grafana: %s", err)
+ }
+ nodeURLs, err := ct.cluster.ExternalIP(ct.ctx, ct.t.L(), ct.workloadNode)
+ if err != nil {
+ ct.t.Errorf("error getting grafana node external ip: %s", err)
+ }
+ ct.t.Status(fmt.Sprintf("started grafana at http://%s:3000/d/928XNlN4k/basic?from=now-15m&to=now", nodeURLs[0]))
+}
+
+type latencyTargets struct {
+ initialScanLatency time.Duration
+ steadyLatency time.Duration
}
func runCDCBank(ctx context.Context, t test.Test, c cluster.Cluster) {
@@ -333,16 +521,18 @@ func runCDCBank(ctx context.Context, t test.Test, c cluster.Cluster) {
db := c.Conn(ctx, t.L(), 1)
defer stopFeeds(db)
- options := []cdcOption{
- {"updated", ""},
- {"resolved", ""},
+ options := map[string]string{
+ "updated": "",
+ "resolved": "",
// we need to set a min_checkpoint_frequency here because if we
// use the default 30s duration, the test will likely not be able
// to finish within 30 minutes
- {"min_checkpoint_frequency", "'10s'"},
- {"diff", ""},
+ "min_checkpoint_frequency": "'10s'",
+ "diff": "",
}
- _, err := newChangefeedCreator(db, "bank.bank", kafka.sinkURL(ctx)).With(options...).Create()
+ _, err := newChangefeedCreator(db, "bank.bank", kafka.sinkURL(ctx)).
+ With(options).
+ Create()
if err != nil {
t.Fatal(err)
}
@@ -485,19 +675,16 @@ func runCDCSchemaRegistry(ctx context.Context, t test.Test, c cluster.Cluster) {
t.Fatal(err)
}
- // NB: the WITH diff option was not supported until v20.1.
- options := []cdcOption{
- {"updated", ""},
- {"resolved", ""},
- {"format", "experimental_avro"},
- {"confluent_schema_registry", "$2"},
- }
- withDiff := t.IsBuildVersion("v20.1.0")
- if withDiff {
- options = append(options, cdcOption{"diff", ""})
+ options := map[string]string{
+ "updated": "",
+ "resolved": "",
+ "format": "experimental_avro",
+ "confluent_schema_registry": "$2",
+ "diff": "",
}
+
_, err := newChangefeedCreator(db, "foo", kafka.sinkURL(ctx)).
- With(options...).
+ With(options).
Args(kafka.schemaRegistryURL(ctx)).
Create()
if err != nil {
@@ -560,27 +747,14 @@ func runCDCSchemaRegistry(ctx context.Context, t test.Test, c cluster.Cluster) {
}
sort.Strings(updated)
- var expected []string
- if withDiff {
- expected = []string{
- `{"before":null,"after":{"foo":{"a":{"long":1}}},"updated":{"string":""}}`,
- `{"before":null,"after":{"foo":{"a":{"long":2},"b":{"string":"2"}}},"updated":{"string":""}}`,
- `{"before":null,"after":{"foo":{"a":{"long":3},"b":{"string":"3"},"c":{"long":3}}},"updated":{"string":""}}`,
- `{"before":null,"after":{"foo":{"a":{"long":4},"c":{"long":4}}},"updated":{"string":""}}`,
- `{"before":{"foo_before":{"a":{"long":1},"b":null,"c":null}},"after":{"foo":{"a":{"long":1},"c":null}},"updated":{"string":""}}`,
- `{"before":{"foo_before":{"a":{"long":2},"b":{"string":"2"},"c":null}},"after":{"foo":{"a":{"long":2},"c":null}},"updated":{"string":""}}`,
- `{"before":{"foo_before":{"a":{"long":3},"b":{"string":"3"},"c":{"long":3}}},"after":{"foo":{"a":{"long":3},"c":{"long":3}}},"updated":{"string":""}}`,
- }
- } else {
- expected = []string{
- `{"updated":{"string":""},"after":{"foo":{"a":{"long":1},"c":null}}}`,
- `{"updated":{"string":""},"after":{"foo":{"a":{"long":1}}}}`,
- `{"updated":{"string":""},"after":{"foo":{"a":{"long":2},"b":{"string":"2"}}}}`,
- `{"updated":{"string":""},"after":{"foo":{"a":{"long":2},"c":null}}}`,
- `{"updated":{"string":""},"after":{"foo":{"a":{"long":3},"b":{"string":"3"},"c":{"long":3}}}}`,
- `{"updated":{"string":""},"after":{"foo":{"a":{"long":3},"c":{"long":3}}}}`,
- `{"updated":{"string":""},"after":{"foo":{"a":{"long":4},"c":{"long":4}}}}`,
- }
+ expected := []string{
+ `{"before":null,"after":{"foo":{"a":{"long":1}}},"updated":{"string":""}}`,
+ `{"before":null,"after":{"foo":{"a":{"long":2},"b":{"string":"2"}}},"updated":{"string":""}}`,
+ `{"before":null,"after":{"foo":{"a":{"long":3},"b":{"string":"3"},"c":{"long":3}}},"updated":{"string":""}}`,
+ `{"before":null,"after":{"foo":{"a":{"long":4},"c":{"long":4}}},"updated":{"string":""}}`,
+ `{"before":{"foo_before":{"a":{"long":1},"b":null,"c":null}},"after":{"foo":{"a":{"long":1},"c":null}},"updated":{"string":""}}`,
+ `{"before":{"foo_before":{"a":{"long":2},"b":{"string":"2"},"c":null}},"after":{"foo":{"a":{"long":2},"c":null}},"updated":{"string":""}}`,
+ `{"before":{"foo_before":{"a":{"long":3},"b":{"string":"3"},"c":{"long":3}}},"after":{"foo":{"a":{"long":3},"c":{"long":3}}},"updated":{"string":""}}`,
}
if strings.Join(expected, "\n") != strings.Join(updated, "\n") {
t.Fatalf("expected\n%s\n\ngot\n%s\n\n",
@@ -661,19 +835,49 @@ func runCDCKafkaAuth(ctx context.Context, t test.Test, c cluster.Cluster) {
}
func registerCDC(r registry.Registry) {
+ r.Add(registry.TestSpec{
+ Name: "cdc/initial-scan-only",
+ Owner: registry.OwnerCDC,
+ Cluster: r.MakeClusterSpec(4, spec.CPU(16)),
+ RequiresLicense: true,
+ Run: func(ctx context.Context, t test.Test, c cluster.Cluster) {
+ ct := newCDCTester(ctx, t, c)
+ defer ct.Close()
+
+ ct.runTPCCWorkload(tpccArgs{warehouses: 100})
+
+ feed := ct.newChangefeed(feedArgs{
+ sinkType: kafkaSink,
+ targets: allTpccTargets,
+ opts: map[string]string{"initial_scan": "'only'"},
+ })
+ ct.runFeedLatencyVerifier(feed, latencyTargets{
+ initialScanLatency: 30 * time.Minute,
+ })
+ feed.waitForCompletion()
+ },
+ })
r.Add(registry.TestSpec{
Name: "cdc/tpcc-1000",
Owner: registry.OwnerCDC,
Cluster: r.MakeClusterSpec(4, spec.CPU(16)),
RequiresLicense: true,
Run: func(ctx context.Context, t test.Test, c cluster.Cluster) {
- cdcBasicTest(ctx, t, c, cdcTestArgs{
- workloadType: tpccWorkloadType,
- tpccWarehouseCount: 1000,
- workloadDuration: "120m",
- targetInitialScanLatency: 3 * time.Minute,
- targetSteadyLatency: 10 * time.Minute,
+ ct := newCDCTester(ctx, t, c)
+ defer ct.Close()
+
+ ct.runTPCCWorkload(tpccArgs{warehouses: 1000, duration: "120m"})
+
+ feed := ct.newChangefeed(feedArgs{
+ sinkType: kafkaSink,
+ targets: allTpccTargets,
+ opts: map[string]string{"initial_scan": "'no'"},
+ })
+ ct.runFeedLatencyVerifier(feed, latencyTargets{
+ initialScanLatency: 3 * time.Minute,
+ steadyLatency: 10 * time.Minute,
})
+ ct.waitForWorkload()
},
})
r.Add(registry.TestSpec{
@@ -683,14 +887,21 @@ func registerCDC(r registry.Registry) {
Tags: []string{"manual"},
RequiresLicense: true,
Run: func(ctx context.Context, t test.Test, c cluster.Cluster) {
- cdcBasicTest(ctx, t, c, cdcTestArgs{
- workloadType: tpccWorkloadType,
- tpccWarehouseCount: 1000,
- workloadDuration: "120m",
- targetInitialScanLatency: 3 * time.Minute,
- targetSteadyLatency: 10 * time.Minute,
- sinkURI: "null://",
+ ct := newCDCTester(ctx, t, c)
+ defer ct.Close()
+
+ ct.runTPCCWorkload(tpccArgs{warehouses: 1000, duration: "120m"})
+
+ feed := ct.newChangefeed(feedArgs{
+ sinkType: nullSink,
+ targets: allTpccTargets,
+ opts: map[string]string{"initial_scan": "'no'"},
+ })
+ ct.runFeedLatencyVerifier(feed, latencyTargets{
+ initialScanLatency: 3 * time.Minute,
+ steadyLatency: 10 * time.Minute,
})
+ ct.waitForWorkload()
},
})
r.Add(registry.TestSpec{
@@ -699,14 +910,17 @@ func registerCDC(r registry.Registry) {
Cluster: r.MakeClusterSpec(4, spec.CPU(16)),
RequiresLicense: true,
Run: func(ctx context.Context, t test.Test, c cluster.Cluster) {
- cdcBasicTest(ctx, t, c, cdcTestArgs{
- workloadType: tpccWorkloadType,
- tpccWarehouseCount: 100,
- workloadDuration: "30m",
- initialScan: true,
- targetInitialScanLatency: 30 * time.Minute,
- targetSteadyLatency: time.Minute,
+ ct := newCDCTester(ctx, t, c)
+ defer ct.Close()
+
+ ct.runTPCCWorkload(tpccArgs{warehouses: 100, duration: "30m"})
+
+ feed := ct.newChangefeed(feedArgs{sinkType: nullSink, targets: allTpccTargets})
+ ct.runFeedLatencyVerifier(feed, latencyTargets{
+ initialScanLatency: 30 * time.Minute,
+ steadyLatency: time.Minute,
})
+ ct.waitForWorkload()
},
})
r.Add(registry.TestSpec{
@@ -715,14 +929,22 @@ func registerCDC(r registry.Registry) {
Cluster: r.MakeClusterSpec(4, spec.CPU(16)),
RequiresLicense: true,
Run: func(ctx context.Context, t test.Test, c cluster.Cluster) {
- cdcBasicTest(ctx, t, c, cdcTestArgs{
- workloadType: tpccWorkloadType,
- tpccWarehouseCount: 100,
- workloadDuration: "30m",
- kafkaChaos: true,
- targetInitialScanLatency: 3 * time.Minute,
- targetSteadyLatency: 5 * time.Minute,
+ ct := newCDCTester(ctx, t, c)
+ defer ct.Close()
+
+ ct.runTPCCWorkload(tpccArgs{warehouses: 100, duration: "30m"})
+
+ feed := ct.newChangefeed(feedArgs{
+ sinkType: kafkaSink,
+ targets: allTpccTargets,
+ kafkaChaos: true,
+ opts: map[string]string{"initial_scan": "'no'"},
+ })
+ ct.runFeedLatencyVerifier(feed, latencyTargets{
+ initialScanLatency: 3 * time.Minute,
+ steadyLatency: 5 * time.Minute,
})
+ ct.waitForWorkload()
},
})
r.Add(registry.TestSpec{
@@ -731,16 +953,24 @@ func registerCDC(r registry.Registry) {
Cluster: r.MakeClusterSpec(4, spec.CPU(16)),
RequiresLicense: true,
Run: func(ctx context.Context, t test.Test, c cluster.Cluster) {
- cdcBasicTest(ctx, t, c, cdcTestArgs{
- workloadType: tpccWorkloadType,
- tpccWarehouseCount: 100,
- workloadDuration: "30m",
- crdbChaos: true,
- targetInitialScanLatency: 3 * time.Minute,
- // TODO(aayush): It should be okay to drop this as low as 2 to 3 minutes. See
- // #36879 for some discussion.
- targetSteadyLatency: 5 * time.Minute,
+ ct := newCDCTester(ctx, t, c)
+ defer ct.Close()
+
+ ct.startCRDBChaos()
+
+ ct.runTPCCWorkload(tpccArgs{warehouses: 100, duration: "30m", tolerateErrors: true})
+
+ feed := ct.newChangefeed(feedArgs{
+ sinkType: kafkaSink,
+ targets: allTpccTargets,
+ opts: map[string]string{"initial_scan": "'no'"},
+ tolerateErrors: true,
+ })
+ ct.runFeedLatencyVerifier(feed, latencyTargets{
+ initialScanLatency: 3 * time.Minute,
+ steadyLatency: 5 * time.Minute,
})
+ ct.waitForWorkload()
},
})
r.Add(registry.TestSpec{
@@ -752,19 +982,29 @@ func registerCDC(r registry.Registry) {
Cluster: r.MakeClusterSpec(4, spec.CPU(16)),
RequiresLicense: true,
Run: func(ctx context.Context, t test.Test, c cluster.Cluster) {
- cdcBasicTest(ctx, t, c, cdcTestArgs{
- workloadType: ledgerWorkloadType,
- // TODO(ssd): Range splits cause changefeed latencies to balloon
- // because of catchup-scan performance. Reducing the test time and
- // bumping the range_max_bytes avoids the split until we can improve
- // catchup scan performance.
- workloadDuration: "28m",
- initialScan: true,
- targetInitialScanLatency: 10 * time.Minute,
- targetSteadyLatency: time.Minute,
- targetTxnPerSecond: 575,
- preStartStatements: []string{"ALTER DATABASE ledger CONFIGURE ZONE USING range_max_bytes = 805306368, range_min_bytes = 134217728"},
+ ct := newCDCTester(ctx, t, c)
+ defer ct.Close()
+
+ workloadStart := timeutil.Now()
+ ct.runLedgerWorkload(ledgerArgs{duration: "28m"})
+
+ alterStmt := "ALTER DATABASE ledger CONFIGURE ZONE USING range_max_bytes = 805306368, range_min_bytes = 134217728"
+ _, err := ct.DB().ExecContext(ctx, alterStmt)
+ if err != nil {
+ t.Fatalf("failed statement %q: %s", alterStmt, err.Error())
+ }
+
+ feed := ct.newChangefeed(feedArgs{sinkType: kafkaSink, targets: allLedgerTargets})
+ ct.runFeedLatencyVerifier(feed, latencyTargets{
+ initialScanLatency: 10 * time.Minute,
+ steadyLatency: time.Minute,
})
+ ct.waitForWorkload()
+
+ workloadEnd := timeutil.Now()
+ verifyTxnPerSecond(
+ ctx, c, t, ct.crdbNodes.RandNode(), workloadStart, workloadEnd, 575, 0.05,
+ )
},
})
r.Add(registry.TestSpec{
@@ -773,19 +1013,24 @@ func registerCDC(r registry.Registry) {
Cluster: r.MakeClusterSpec(4, spec.CPU(16)),
RequiresLicense: true,
Run: func(ctx context.Context, t test.Test, c cluster.Cluster) {
- cdcBasicTest(ctx, t, c, cdcTestArgs{
- workloadType: tpccWorkloadType,
- // Sending data to Google Cloud Storage is a bit slower than sending to
- // Kafka on an adjacent machine, so use half the data of the
- // initial-scan test. Consider adding a test that writes to nodelocal,
- // which should be much faster, with a larger warehouse count.
- tpccWarehouseCount: 50,
- workloadDuration: "30m",
- initialScan: true,
- whichSink: cloudStorageSink,
- targetInitialScanLatency: 30 * time.Minute,
- targetSteadyLatency: time.Minute,
+ ct := newCDCTester(ctx, t, c)
+ defer ct.Close()
+
+ // Sending data to Google Cloud Storage is a bit slower than sending to
+ // Kafka on an adjacent machine, so use half the data of the
+ // initial-scan test. Consider adding a test that writes to nodelocal,
+ // which should be much faster, with a larger warehouse count.
+ ct.runTPCCWorkload(tpccArgs{warehouses: 50, duration: "30m"})
+
+ feed := ct.newChangefeed(feedArgs{
+ sinkType: cloudStorageSink,
+ targets: allTpccTargets,
+ })
+ ct.runFeedLatencyVerifier(feed, latencyTargets{
+ initialScanLatency: 30 * time.Minute,
+ steadyLatency: time.Minute,
})
+ ct.waitForWorkload()
},
})
r.Add(registry.TestSpec{
@@ -794,15 +1039,20 @@ func registerCDC(r registry.Registry) {
Cluster: r.MakeClusterSpec(4, spec.CPU(16)),
RequiresLicense: true,
Run: func(ctx context.Context, t test.Test, c cluster.Cluster) {
- cdcBasicTest(ctx, t, c, cdcTestArgs{
- workloadType: tpccWorkloadType,
- tpccWarehouseCount: 1,
- workloadDuration: "30m",
- initialScan: true,
- whichSink: pubsubSink,
- targetInitialScanLatency: 30 * time.Minute,
- targetSteadyLatency: time.Minute,
+ ct := newCDCTester(ctx, t, c)
+ defer ct.Close()
+
+ ct.runTPCCWorkload(tpccArgs{warehouses: 1, duration: "30m"})
+
+ feed := ct.newChangefeed(feedArgs{
+ sinkType: pubsubSink,
+ targets: allTpccTargets,
})
+ ct.runFeedLatencyVerifier(feed, latencyTargets{
+ initialScanLatency: 30 * time.Minute,
+ steadyLatency: time.Minute,
+ })
+ ct.waitForWorkload()
},
})
@@ -820,16 +1070,21 @@ func registerCDC(r registry.Registry) {
Cluster: r.MakeClusterSpec(4, spec.CPU(16)),
RequiresLicense: true,
Run: func(ctx context.Context, t test.Test, c cluster.Cluster) {
- cdcBasicTest(ctx, t, c, cdcTestArgs{
- workloadType: tpccWorkloadType,
- tpccWarehouseCount: 1,
- workloadDuration: "30m",
- initialScan: true,
- whichSink: pubsubSink,
- assumeRole: "cdc-roachtest-intermediate@cockroach-ephemeral.iam.gserviceaccount.com,cdc-roachtest@cockroach-ephemeral.iam.gserviceaccount.com",
- targetInitialScanLatency: 30 * time.Minute,
- targetSteadyLatency: time.Minute,
+ ct := newCDCTester(ctx, t, c)
+ defer ct.Close()
+
+ ct.runTPCCWorkload(tpccArgs{warehouses: 1, duration: "30m"})
+
+ feed := ct.newChangefeed(feedArgs{
+ sinkType: pubsubSink,
+ assumeRole: "cdc-roachtest-intermediate@cockroach-ephemeral.iam.gserviceaccount.com,cdc-roachtest@cockroach-ephemeral.iam.gserviceaccount.com",
+ targets: allTpccTargets,
+ })
+ ct.runFeedLatencyVerifier(feed, latencyTargets{
+ initialScanLatency: 30 * time.Minute,
+ steadyLatency: time.Minute,
})
+ ct.waitForWorkload()
},
})
@@ -847,15 +1102,21 @@ func registerCDC(r registry.Registry) {
Cluster: r.MakeClusterSpec(4, spec.CPU(16)),
RequiresLicense: true,
Run: func(ctx context.Context, t test.Test, c cluster.Cluster) {
- cdcBasicTest(ctx, t, c, cdcTestArgs{
- tpccWarehouseCount: 50,
- workloadDuration: "30m",
- initialScan: true,
- whichSink: cloudStorageSink,
- assumeRole: "cdc-roachtest-intermediate@cockroach-ephemeral.iam.gserviceaccount.com,cdc-roachtest@cockroach-ephemeral.iam.gserviceaccount.com",
- targetInitialScanLatency: 30 * time.Minute,
- targetSteadyLatency: time.Minute,
+ ct := newCDCTester(ctx, t, c)
+ defer ct.Close()
+
+ ct.runTPCCWorkload(tpccArgs{warehouses: 50, duration: "30m"})
+
+ feed := ct.newChangefeed(feedArgs{
+ sinkType: cloudStorageSink,
+ assumeRole: "cdc-roachtest-intermediate@cockroach-ephemeral.iam.gserviceaccount.com,cdc-roachtest@cockroach-ephemeral.iam.gserviceaccount.com",
+ targets: allTpccTargets,
+ })
+ ct.runFeedLatencyVerifier(feed, latencyTargets{
+ initialScanLatency: 30 * time.Minute,
+ steadyLatency: time.Minute,
})
+ ct.waitForWorkload()
},
})
@@ -863,23 +1124,28 @@ func registerCDC(r registry.Registry) {
// currently fails with "initial scan did not complete" because sink
// URI is set as localhost, need to expose it to the other nodes via IP
/*
- r.Add(testSpec{
- Name: "cdc/webhook-sink",
- Owner: OwnerCDC,
- Cluster: r.MakeClusterSpec(4, spec.CPU(16)),
- RequiresLicense: true,
- Run: func(ctx context.Context, t *test, c Cluster) {
- cdcBasicTest(ctx, t, c, cdcTestArgs{
- workloadType: tpccWorkloadType,
- tpccWarehouseCount: 100,
- workloadDuration: "30m",
- initialScan: true,
- whichSink: webhookSink,
- targetInitialScanLatency: 30 * time.Minute,
- targetSteadyLatency: time.Minute,
+ r.Add(registry.TestSpec{
+ Name: "cdc/webhook-sink",
+ Owner: `cdc`,
+ Cluster: r.MakeClusterSpec(4, spec.CPU(16)),
+ RequiresLicense: true,
+ Run: func(ctx context.Context, t test.Test, c cluster.Cluster) {
+ ct := newCDCTester(ctx, t, c)
+ defer ct.Close()
+
+ ct.runTPCCWorkload(tpccArgs{warehouses: 100, duration: "30m"})
+
+ feed := ct.newChangefeed(feedArgs{
+ sinkType: webhookSink,
+ targets: allTpccTargets,
+ })
+ ct.runFeedLatencyVerifier(feed, latencyTargets{
+ initialScanLatency: 30 * time.Minute,
+ steadyLatency: time.Minute,
+ })
+ ct.waitForWorkload()
+ },
})
- },
- })
*/
r.Add(registry.TestSpec{
Name: "cdc/kafka-auth",
@@ -1629,18 +1895,13 @@ func (lw *ledgerWorkload) run(ctx context.Context, c cluster.Cluster, workloadDu
))
}
-type cdcOption struct {
- option string
- value string
-}
-
// changefeedCreator wraps the process of creating a changefeed with
// different options and sinks
type changefeedCreator struct {
db *gosql.DB
targets string
sinkURL string
- options []cdcOption
+ options map[string]string
extraArgs []interface{}
}
@@ -1649,14 +1910,17 @@ func newChangefeedCreator(db *gosql.DB, targets, sinkURL string) *changefeedCrea
db: db,
targets: targets,
sinkURL: sinkURL,
+ options: make(map[string]string),
}
}
// With adds options to the changefeed being created. If a non-zero
// `value` is passed in one of the options, the option will be passed
// as {option}={value}.
-func (cfc *changefeedCreator) With(options ...cdcOption) *changefeedCreator {
- cfc.options = append(cfc.options, options...)
+func (cfc *changefeedCreator) With(opts map[string]string) *changefeedCreator {
+ for option, value := range opts {
+ cfc.options[option] = value
+ }
return cfc
}
@@ -1681,10 +1945,9 @@ func (cfc *changefeedCreator) Create() (int, error) {
stmt := fmt.Sprintf("CREATE CHANGEFEED FOR %s INTO $1", cfc.targets)
var options []string
- for _, opt := range cfc.options {
- option := opt.option
- if opt.value != "" {
- option += fmt.Sprintf("=%s", opt.value)
+ for option, value := range cfc.options {
+ if value != "" {
+ option += fmt.Sprintf("=%s", value)
}
options = append(options, option)
}
@@ -1707,11 +1970,13 @@ type changefeedInfo struct {
errMsg string
statementTime time.Time
highwaterTime time.Time
+ finishedTime time.Time
}
-func (c *changefeedInfo) GetHighWater() time.Time { return c.highwaterTime }
-func (c *changefeedInfo) GetStatus() string { return c.status }
-func (c *changefeedInfo) GetError() string { return c.status }
+func (c *changefeedInfo) GetHighWater() time.Time { return c.highwaterTime }
+func (c *changefeedInfo) GetFinishedTime() time.Time { return c.finishedTime }
+func (c *changefeedInfo) GetStatus() string { return c.status }
+func (c *changefeedInfo) GetError() string { return c.status }
var _ jobInfo = (*changefeedInfo)(nil)
@@ -1742,6 +2007,7 @@ func getChangefeedInfo(db *gosql.DB, jobID int) (*changefeedInfo, error) {
errMsg: payload.Error,
statementTime: payload.GetChangefeed().StatementTime.GoTime(),
highwaterTime: highwaterTime,
+ finishedTime: time.UnixMicro(payload.FinishedMicros),
}, nil
}
diff --git a/pkg/cmd/roachtest/tests/cluster_to_cluster.go b/pkg/cmd/roachtest/tests/cluster_to_cluster.go
index 89e086ab8848..8cb89d71123c 100644
--- a/pkg/cmd/roachtest/tests/cluster_to_cluster.go
+++ b/pkg/cmd/roachtest/tests/cluster_to_cluster.go
@@ -223,11 +223,13 @@ type streamIngesitonJobInfo struct {
status string
errMsg string
highwaterTime time.Time
+ finishedTime time.Time
}
-func (c *streamIngesitonJobInfo) GetHighWater() time.Time { return c.highwaterTime }
-func (c *streamIngesitonJobInfo) GetStatus() string { return c.status }
-func (c *streamIngesitonJobInfo) GetError() string { return c.status }
+func (c *streamIngesitonJobInfo) GetHighWater() time.Time { return c.highwaterTime }
+func (c *streamIngesitonJobInfo) GetFinishedTime() time.Time { return c.finishedTime }
+func (c *streamIngesitonJobInfo) GetStatus() string { return c.status }
+func (c *streamIngesitonJobInfo) GetError() string { return c.status }
var _ jobInfo = (*streamIngesitonJobInfo)(nil)
@@ -257,6 +259,7 @@ func getStreamIngestionJobInfo(db *gosql.DB, jobID int) (jobInfo, error) {
status: status,
errMsg: payload.Error,
highwaterTime: highwaterTime,
+ finishedTime: time.UnixMicro(payload.FinishedMicros),
}, nil
}
diff --git a/pkg/cmd/roachtest/tests/latency_verifier.go b/pkg/cmd/roachtest/tests/latency_verifier.go
index f14453b4c803..e79bbde3bb77 100644
--- a/pkg/cmd/roachtest/tests/latency_verifier.go
+++ b/pkg/cmd/roachtest/tests/latency_verifier.go
@@ -26,6 +26,7 @@ import (
type jobInfo interface {
GetHighWater() time.Time
+ GetFinishedTime() time.Time
GetStatus() string
GetError() string
}
@@ -139,11 +140,14 @@ func (lv *latencyVerifier) pollLatency(
return err
}
status := info.GetStatus()
- if status != "running" {
+ if status == "succeeded" {
+ lv.noteHighwater(info.GetFinishedTime())
+ } else if status == "running" {
+ lv.noteHighwater(info.GetHighWater())
+ } else {
lv.logger.Printf("unexpected status: %s, error: %s", status, info.GetError())
return errors.Errorf("unexpected status: %s", status)
}
- lv.noteHighwater(info.GetHighWater())
}
}
@@ -155,10 +159,10 @@ func (lv *latencyVerifier) assertValid(t test.Test) {
t.Fatalf("initial scan latency was more than target: %s vs %s",
lv.initialScanLatency, lv.targetInitialScanLatency)
}
- if !lv.latencyBecameSteady {
+ if lv.targetSteadyLatency != 0 && !lv.latencyBecameSteady {
t.Fatalf("latency never dropped to acceptable steady level: %s", lv.targetSteadyLatency)
}
- if lv.maxSeenSteadyLatency > lv.targetSteadyLatency {
+ if lv.targetSteadyLatency != 0 && lv.maxSeenSteadyLatency > lv.targetSteadyLatency {
t.Fatalf("max latency was more than allowed: %s vs %s",
lv.maxSeenSteadyLatency, lv.targetSteadyLatency)
}
diff --git a/pkg/cmd/roachtest/tests/mixed_version_cdc.go b/pkg/cmd/roachtest/tests/mixed_version_cdc.go
index 5a0fcec2bebd..3e1386cb5c3a 100644
--- a/pkg/cmd/roachtest/tests/mixed_version_cdc.go
+++ b/pkg/cmd/roachtest/tests/mixed_version_cdc.go
@@ -323,12 +323,12 @@ func (cmvt *cdcMixedVersionTester) createChangeFeed(node int) versionStep {
t.Status("creating changefeed")
db := u.conn(ctx, t, node)
- options := []cdcOption{
- {"updated", ""},
- {"resolved", fmt.Sprintf("'%s'", resolvedInterval)},
+ options := map[string]string{
+ "updated": "",
+ "resolved": fmt.Sprintf("'%s'", resolvedInterval),
}
_, err := newChangefeedCreator(db, fmt.Sprintf("%s.%s", targetDB, targetTable), cmvt.kafka.sinkURL(ctx)).
- With(options...).
+ With(options).
Create()
if err != nil {
t.Fatal(err)
diff --git a/pkg/config/system_test.go b/pkg/config/system_test.go
index 82f0e18c6a72..dce068d505de 100644
--- a/pkg/config/system_test.go
+++ b/pkg/config/system_test.go
@@ -317,12 +317,6 @@ func TestComputeSplitKeySystemRanges(t *testing.T) {
{roachpb.RKey(keys.NodeLivenessPrefix), roachpb.RKey(keys.NodeLivenessKeyMax), nil},
{roachpb.RKey(keys.NodeLivenessPrefix), roachpb.RKeyMax, keys.NodeLivenessKeyMax},
{roachpb.RKey(keys.NodeLivenessKeyMax), roachpb.RKeyMax, keys.TimeseriesPrefix},
- {roachpb.RKey(keys.StartupMigrationPrefix), roachpb.RKey(keys.NodeLivenessPrefix), nil},
- {roachpb.RKey(keys.StartupMigrationPrefix), roachpb.RKey(keys.NodeLivenessKeyMax), nil},
- {roachpb.RKey(keys.StartupMigrationPrefix), roachpb.RKey(keys.StoreIDGenerator), nil},
- {roachpb.RKey(keys.StartupMigrationPrefix), roachpb.RKey(keys.TimeseriesPrefix), nil},
- {roachpb.RKey(keys.StartupMigrationPrefix), roachpb.RKey(keys.TimeseriesPrefix.Next()), keys.TimeseriesPrefix},
- {roachpb.RKey(keys.StartupMigrationPrefix), roachpb.RKeyMax, keys.TimeseriesPrefix},
{roachpb.RKey(keys.TimeseriesPrefix), roachpb.RKey(keys.TimeseriesPrefix.Next()), nil},
{roachpb.RKey(keys.TimeseriesPrefix), roachpb.RKey(keys.TimeseriesPrefix.PrefixEnd()), nil},
{roachpb.RKey(keys.TimeseriesPrefix), roachpb.RKeyMax, keys.TimeseriesPrefix.PrefixEnd()},
@@ -559,7 +553,6 @@ func TestGetZoneConfigForKey(t *testing.T) {
{roachpb.RKey(keys.MetaMax), keys.SystemRangesID},
{roachpb.RKey(keys.SystemPrefix), keys.SystemRangesID},
{roachpb.RKey(keys.SystemPrefix.Next()), keys.SystemRangesID},
- {roachpb.RKey(keys.StartupMigrationLease), keys.SystemRangesID},
{roachpb.RKey(keys.NodeLivenessPrefix), keys.LivenessRangesID},
{roachpb.RKey(keys.LegacyDescIDGenerator), keys.SystemRangesID},
{roachpb.RKey(keys.NodeIDGenerator), keys.SystemRangesID},
diff --git a/pkg/gen/protobuf.bzl b/pkg/gen/protobuf.bzl
index ddf47b046add..85236f5c65d8 100644
--- a/pkg/gen/protobuf.bzl
+++ b/pkg/gen/protobuf.bzl
@@ -62,7 +62,6 @@ PROTOBUF_SRCS = [
"//pkg/sql/sqlstats/persistedsqlstats:persistedsqlstats_go_proto",
"//pkg/sql/stats:stats_go_proto",
"//pkg/sql/types:types_go_proto",
- "//pkg/startupmigrations/leasemanager:leasemanager_go_proto",
"//pkg/storage/enginepb:enginepb_go_proto",
"//pkg/testutils/grpcutils:grpcutils_go_proto",
"//pkg/ts/catalog:catalog_go_proto",
diff --git a/pkg/jobs/BUILD.bazel b/pkg/jobs/BUILD.bazel
index 84bfeb902f95..de1954b5b779 100644
--- a/pkg/jobs/BUILD.bazel
+++ b/pkg/jobs/BUILD.bazel
@@ -131,6 +131,7 @@ go_test(
"//pkg/testutils/skip",
"//pkg/testutils/sqlutils",
"//pkg/testutils/testcluster",
+ "//pkg/upgrade/upgradebase",
"//pkg/util/ctxgroup",
"//pkg/util/hlc",
"//pkg/util/leaktest",
diff --git a/pkg/jobs/jobs_test.go b/pkg/jobs/jobs_test.go
index 09babfe144f5..5fae3dbb89f3 100644
--- a/pkg/jobs/jobs_test.go
+++ b/pkg/jobs/jobs_test.go
@@ -49,6 +49,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/upgrade/upgradebase"
"github.com/cockroachdb/cockroach/pkg/util/ctxgroup"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
@@ -227,6 +228,9 @@ func (rts *registryTestSuite) setUp(t *testing.T) {
args.Knobs.SpanConfig = &spanconfig.TestingKnobs{
ManagerDisableJobCreation: true,
}
+ args.Knobs.UpgradeManager = &upgradebase.TestingKnobs{
+ DontUseJobs: true,
+ }
if rts.traceRealSpan {
baseDir, dirCleanupFn := testutils.TempDir(t)
diff --git a/pkg/jobs/registry_test.go b/pkg/jobs/registry_test.go
index d8aad354bd0b..33889b0e7eb1 100644
--- a/pkg/jobs/registry_test.go
+++ b/pkg/jobs/registry_test.go
@@ -38,6 +38,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/testutils/serverutils"
"github.com/cockroachdb/cockroach/pkg/testutils/skip"
"github.com/cockroachdb/cockroach/pkg/testutils/sqlutils"
+ "github.com/cockroachdb/cockroach/pkg/upgrade/upgradebase"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
"github.com/cockroachdb/cockroach/pkg/util/log"
@@ -119,6 +120,10 @@ func TestRegistryGC(t *testing.T) {
// test itself.
ManagerDisableJobCreation: true,
},
+ UpgradeManager: &upgradebase.TestingKnobs{
+ // This test wants to look at job records.
+ DontUseJobs: true,
+ },
},
})
defer s.Stopper().Stop(ctx)
@@ -260,6 +265,10 @@ func TestRegistryGCPagination(t *testing.T) {
// test itself.
ManagerDisableJobCreation: true,
},
+ UpgradeManager: &upgradebase.TestingKnobs{
+ // This test wants to count job records.
+ DontUseJobs: true,
+ },
},
})
db := sqlutils.MakeSQLRunner(sqlDB)
@@ -306,6 +315,10 @@ func TestBatchJobsCreation(t *testing.T) {
JobsTestingKnobs: &TestingKnobs{
DisableAdoptions: true,
},
+ // DisableAdoptions needs this.
+ UpgradeManager: &upgradebase.TestingKnobs{
+ DontUseJobs: true,
+ },
},
}
@@ -474,6 +487,9 @@ func TestRetriesWithExponentialBackoff(t *testing.T) {
// test itself.
ManagerDisableJobCreation: true,
},
+ UpgradeManager: &upgradebase.TestingKnobs{
+ DontUseJobs: true,
+ },
},
}
var sqlDB *gosql.DB
@@ -776,7 +792,13 @@ func TestExponentialBackoffSettings(t *testing.T) {
retryMaxDelaySetting.Override(ctx, &cs.SV, time.Hour)
args := base.TestServerArgs{
Settings: cs,
- Knobs: base.TestingKnobs{JobsTestingKnobs: &TestingKnobs{BeforeUpdate: intercept}},
+ Knobs: base.TestingKnobs{
+ JobsTestingKnobs: &TestingKnobs{BeforeUpdate: intercept},
+ // This test wants to intercept the jobs that get created.
+ UpgradeManager: &upgradebase.TestingKnobs{
+ DontUseJobs: true,
+ },
+ },
}
s, sdb, kvDB := serverutils.StartServer(t, args)
defer s.Stopper().Stop(ctx)
@@ -1124,6 +1146,10 @@ func TestDisablingJobAdoptionClearsClaimSessionID(t *testing.T) {
Cancel: &intervalOverride,
},
},
+ // DisableAdoptions needs this.
+ UpgradeManager: &upgradebase.TestingKnobs{
+ DontUseJobs: true,
+ },
},
})
ctx := context.Background()
diff --git a/pkg/jobs/testing_knobs.go b/pkg/jobs/testing_knobs.go
index 6e28862b8142..1c85dbc55afa 100644
--- a/pkg/jobs/testing_knobs.go
+++ b/pkg/jobs/testing_knobs.go
@@ -70,6 +70,10 @@ type TestingKnobs struct {
TimeSource *hlc.Clock
// DisableAdoptions disables job adoptions.
+ //
+ // When setting this, you probably want to set UpgradeManager.DontUseJobs too,
+ // otherwise a test server will fail to bootstrap. The TestServer code
+ // validates that these knobs are used in tandem.
DisableAdoptions bool
// DisableRegistryLifecycleManagement
diff --git a/pkg/keys/constants.go b/pkg/keys/constants.go
index 9f40cd00d4af..8f5178d5487a 100644
--- a/pkg/keys/constants.go
+++ b/pkg/keys/constants.go
@@ -299,12 +299,6 @@ var (
// StatusNodePrefix stores all status info for nodes.
StatusNodePrefix = roachpb.Key(makeKey(StatusPrefix, roachpb.RKey("node-")))
//
- // StartupMigrationPrefix specifies the key prefix to store all migration details.
- StartupMigrationPrefix = roachpb.Key(makeKey(SystemPrefix, roachpb.RKey("system-version/")))
- // StartupMigrationLease is the key that nodes must take a lease on in order to run
- // system migrations on the cluster.
- StartupMigrationLease = roachpb.Key(makeKey(StartupMigrationPrefix, roachpb.RKey("lease")))
- //
// TimeseriesPrefix is the key prefix for all timeseries data.
TimeseriesPrefix = roachpb.Key(makeKey(SystemPrefix, roachpb.RKey("tsd")))
// TimeseriesKeyMax is the maximum value for any timeseries data.
diff --git a/pkg/keys/doc.go b/pkg/keys/doc.go
index 79c95c52f08a..da93161c6b9b 100644
--- a/pkg/keys/doc.go
+++ b/pkg/keys/doc.go
@@ -242,16 +242,16 @@ var _ = [...]interface{}{
// 2. System keys: This is where we store global, system data which is
// replicated across the cluster.
SystemPrefix,
- NodeLivenessPrefix, // "\x00liveness-"
- BootstrapVersionKey, // "bootstrap-version"
- LegacyDescIDGenerator, // "desc-idgen"
- NodeIDGenerator, // "node-idgen"
- RangeIDGenerator, // "range-idgen"
- StatusPrefix, // "status-"
- StatusNodePrefix, // "status-node-"
- StoreIDGenerator, // "store-idgen"
- StartupMigrationPrefix, // "system-version/"
- StartupMigrationLease, // "system-version/lease"
+ NodeLivenessPrefix, // "\x00liveness-"
+ BootstrapVersionKey, // "bootstrap-version"
+ LegacyDescIDGenerator, // "desc-idgen"
+ NodeIDGenerator, // "node-idgen"
+ RangeIDGenerator, // "range-idgen"
+ StatusPrefix, // "status-"
+ StatusNodePrefix, // "status-node-"
+ StoreIDGenerator, // "store-idgen"
+ // StartupMigrationPrefix, // "system-version/" - removed in 23.1
+ // StartupMigrationLease, // "system-version/lease" - removed in 23.1
TimeseriesPrefix, // "tsd"
SystemSpanConfigPrefix, // "xffsys-scfg"
SystemMax,
diff --git a/pkg/keys/sql.go b/pkg/keys/sql.go
index 83ec9a9ceab9..90e9336fa530 100644
--- a/pkg/keys/sql.go
+++ b/pkg/keys/sql.go
@@ -163,18 +163,6 @@ func (e sqlEncoder) SequenceKey(tableID uint32) roachpb.Key {
return k
}
-// StartupMigrationKeyPrefix returns the key prefix to store all startup
-// migration details.
-func (e sqlEncoder) StartupMigrationKeyPrefix() roachpb.Key {
- return append(e.TenantPrefix(), StartupMigrationPrefix...)
-}
-
-// StartupMigrationLeaseKey returns the key that nodes must take a lease on in
-// order to run startup migration upgrades on the cluster.
-func (e sqlEncoder) StartupMigrationLeaseKey() roachpb.Key {
- return append(e.TenantPrefix(), StartupMigrationLease...)
-}
-
// unexpected to avoid colliding with sqlEncoder.tenantPrefix.
func (d sqlDecoder) tenantPrefix() roachpb.Key {
return *d.buf
diff --git a/pkg/server/BUILD.bazel b/pkg/server/BUILD.bazel
index 143b1364e43d..5c0e4f4374bc 100644
--- a/pkg/server/BUILD.bazel
+++ b/pkg/server/BUILD.bazel
@@ -225,7 +225,6 @@ go_library(
"//pkg/sql/ttl/ttljob",
"//pkg/sql/ttl/ttlschedule",
"//pkg/sql/types",
- "//pkg/startupmigrations",
"//pkg/storage",
"//pkg/storage/enginepb",
"//pkg/storage/fs",
@@ -235,6 +234,7 @@ go_library(
"//pkg/ts/catalog",
"//pkg/ui",
"//pkg/upgrade",
+ "//pkg/upgrade/upgradebase",
"//pkg/upgrade/upgradecluster",
"//pkg/upgrade/upgrademanager",
"//pkg/util",
@@ -447,7 +447,6 @@ go_test(
"//pkg/sql/sqlstats",
"//pkg/sql/sqlstats/persistedsqlstats",
"//pkg/sql/tests",
- "//pkg/startupmigrations",
"//pkg/storage",
"//pkg/storage/enginepb",
"//pkg/testutils",
@@ -460,7 +459,7 @@ go_test(
"//pkg/ts/tspb",
"//pkg/ui",
"//pkg/upgrade",
- "//pkg/upgrade/upgrades",
+ "//pkg/upgrade/upgradebase",
"//pkg/util",
"//pkg/util/admission",
"//pkg/util/encoding",
diff --git a/pkg/server/migration_test.go b/pkg/server/migration_test.go
index 5d0efa6df526..19587ec3ce28 100644
--- a/pkg/server/migration_test.go
+++ b/pkg/server/migration_test.go
@@ -22,10 +22,10 @@ import (
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/server/serverpb"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
- "github.com/cockroachdb/cockroach/pkg/startupmigrations"
"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/upgrade/upgradebase"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
"github.com/stretchr/testify/require"
)
@@ -254,8 +254,8 @@ func TestUpgradeHappensAfterMigrations(t *testing.T) {
Server: &TestingKnobs{
BinaryVersionOverride: clusterversion.TestingBinaryMinSupportedVersion,
},
- StartupMigrationManager: &startupmigrations.MigrationManagerTestingKnobs{
- AfterEnsureMigrations: func() {
+ UpgradeManager: &upgradebase.TestingKnobs{
+ AfterRunPermanentUpgrades: func() {
// Try to encourage other goroutines to run.
const N = 100
for i := 0; i < N; i++ {
diff --git a/pkg/server/server.go b/pkg/server/server.go
index 06580d4a62d3..55ee5203e108 100644
--- a/pkg/server/server.go
+++ b/pkg/server/server.go
@@ -1715,8 +1715,9 @@ func (s *Server) PreStart(ctx context.Context) error {
s.ctSender.Run(workersCtx, state.nodeID)
// Attempt to upgrade cluster version now that the sql server has been
- // started. At this point we know that all startupmigrations have successfully
- // been run so it is safe to upgrade to the binary's current version.
+ // started. At this point we know that all startupmigrations and permanent
+ // upgrades have successfully been run so it is safe to upgrade to the
+ // binary's current version.
//
// NB: We run this under the startup ctx (not workersCtx) so as to ensure
// all the upgrade steps are traced, for use during troubleshooting.
diff --git a/pkg/server/server_sql.go b/pkg/server/server_sql.go
index dadc6cd20b15..7c0aae14c9f5 100644
--- a/pkg/server/server_sql.go
+++ b/pkg/server/server_sql.go
@@ -98,10 +98,10 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/sqlstats"
"github.com/cockroachdb/cockroach/pkg/sql/stats"
"github.com/cockroachdb/cockroach/pkg/sql/stmtdiagnostics"
- "github.com/cockroachdb/cockroach/pkg/startupmigrations"
"github.com/cockroachdb/cockroach/pkg/storage"
"github.com/cockroachdb/cockroach/pkg/storage/fs"
"github.com/cockroachdb/cockroach/pkg/upgrade"
+ "github.com/cockroachdb/cockroach/pkg/upgrade/upgradebase"
"github.com/cockroachdb/cockroach/pkg/upgrade/upgradecluster"
"github.com/cockroachdb/cockroach/pkg/upgrade/upgrademanager"
"github.com/cockroachdb/cockroach/pkg/util"
@@ -194,6 +194,10 @@ type SQLServer struct {
// Server is closed. Every InternalExecutor created via the factory
// uses this memory monitor.
internalExecutorFactoryMemMonitor *mon.BytesMonitor
+
+ // upgradeManager deals with cluster version upgrades on bootstrap and on
+ // `set cluster setting version = `.
+ upgradeManager *upgrademanager.Manager
}
// sqlServerOptionalKVArgs are the arguments supplied to newSQLServer which are
@@ -1075,6 +1079,7 @@ func newSQLServer(ctx context.Context, cfg sqlServerArgs) (*SQLServer, error) {
)
execCfg.StmtDiagnosticsRecorder = stmtDiagnosticsRegistry
+ var upgradeMgr *upgrademanager.Manager
{
// We only need to attach a version upgrade hook if we're the system
// tenant. Regular tenants are disallowed from changing cluster
@@ -1089,26 +1094,28 @@ func newSQLServer(ctx context.Context, cfg sqlServerArgs) (*SQLServer, error) {
DB: cfg.db,
})
systemDeps = upgrade.SystemDeps{
- Cluster: c,
- DB: cfg.db,
- DistSender: cfg.distSender,
- Stopper: cfg.stopper,
+ Cluster: c,
+ DB: cfg.db,
+ InternalExecutor: cfg.circularInternalExecutor,
+ DistSender: cfg.distSender,
+ Stopper: cfg.stopper,
}
} else {
c = upgradecluster.NewTenantCluster(cfg.db)
systemDeps = upgrade.SystemDeps{
- Cluster: c,
- DB: cfg.db,
+ Cluster: c,
+ DB: cfg.db,
+ InternalExecutor: cfg.circularInternalExecutor,
}
}
- knobs, _ := cfg.TestingKnobs.UpgradeManager.(*upgrade.TestingKnobs)
- migrationMgr := upgrademanager.NewManager(
+ knobs, _ := cfg.TestingKnobs.UpgradeManager.(*upgradebase.TestingKnobs)
+ upgradeMgr = upgrademanager.NewManager(
systemDeps, leaseMgr, cfg.circularInternalExecutor, cfg.internalExecutorFactory, jobRegistry, codec,
- cfg.Settings, knobs,
+ cfg.Settings, clusterIDForSQL.Get(), knobs,
)
- execCfg.UpgradeJobDeps = migrationMgr
- execCfg.VersionUpgradeHook = migrationMgr.Migrate
+ execCfg.UpgradeJobDeps = upgradeMgr
+ execCfg.VersionUpgradeHook = upgradeMgr.Migrate
execCfg.UpgradeTestingKnobs = knobs
}
@@ -1252,6 +1259,7 @@ func newSQLServer(ctx context.Context, cfg sqlServerArgs) (*SQLServer, error) {
isMeta1Leaseholder: cfg.isMeta1Leaseholder,
cfg: cfg.BaseConfig,
internalExecutorFactoryMemMonitor: ieFactoryMonitor,
+ upgradeManager: upgradeMgr,
}, nil
}
@@ -1351,15 +1359,6 @@ func (s *SQLServer) preStart(
return err
}
- // Before serving SQL requests, we have to make sure the database is
- // in an acceptable form for this version of the software.
- // We have to do this after actually starting up the server to be able to
- // seamlessly use the kv client against other nodes in the cluster.
- var mmKnobs startupmigrations.MigrationManagerTestingKnobs
- if migrationManagerTestingKnobs := knobs.StartupMigrationManager; migrationManagerTestingKnobs != nil {
- mmKnobs = *migrationManagerTestingKnobs.(*startupmigrations.MigrationManagerTestingKnobs)
- }
-
s.leaseMgr.RefreshLeases(ctx, stopper, s.execCfg.DB)
s.leaseMgr.PeriodicallyRefreshSomeLeases(ctx)
@@ -1380,17 +1379,6 @@ func (s *SQLServer) preStart(
DistSQLMode: sessiondatapb.DistSQLOff,
},
})
- startupMigrationsMgr := startupmigrations.NewManager(
- stopper,
- s.execCfg.DB,
- s.execCfg.Codec,
- &migrationsExecutor,
- s.execCfg.Clock,
- mmKnobs,
- s.execCfg.NodeInfo.NodeID.SQLInstanceID().String(),
- s.execCfg.Settings,
- s.jobRegistry,
- )
if err := s.jobRegistry.Start(ctx, stopper); err != nil {
return err
@@ -1435,9 +1423,22 @@ func (s *SQLServer) preStart(
return errors.Wrap(err, "initializing settings")
}
- // Run startup upgrades (note: these depend on jobs subsystem running).
- if err := startupMigrationsMgr.EnsureMigrations(ctx, bootstrapVersion); err != nil {
- return errors.Wrap(err, "ensuring SQL migrations")
+ // Run all the "permanent" upgrades that haven't already run in this cluster,
+ // until the currently active version. Upgrades for higher versions, if any,
+ // will be run in response to `SET CLUSTER SETTING version = `, just like
+ // non-permanent upgrade.
+ //
+ // NOTE: We're going to run the permanent upgrades up to the active version.
+ // For mixed kv/sql nodes, I think we could use bootstrapVersion here instead.
+ // If the active version has diverged from bootstrap version, then all
+ // upgrades in between the two must have run when the cluster version
+ // advanced. But for sql-only servers the bootstrap version is not
+ // well-defined, so we use the active version.
+ if err := s.upgradeManager.RunPermanentUpgrades(
+ ctx,
+ s.cfg.Settings.Version.ActiveVersion(ctx).Version, /* upToVersion */
+ ); err != nil {
+ return err
}
log.Infof(ctx, "done ensuring all necessary startup migrations have run")
diff --git a/pkg/server/testserver.go b/pkg/server/testserver.go
index 6185f2fe0c35..22a281651c43 100644
--- a/pkg/server/testserver.go
+++ b/pkg/server/testserver.go
@@ -28,6 +28,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/config"
"github.com/cockroachdb/cockroach/pkg/config/zonepb"
"github.com/cockroachdb/cockroach/pkg/gossip"
+ "github.com/cockroachdb/cockroach/pkg/jobs"
"github.com/cockroachdb/cockroach/pkg/keys"
"github.com/cockroachdb/cockroach/pkg/kv"
"github.com/cockroachdb/cockroach/pkg/kv/kvclient/kvcoord"
@@ -51,6 +52,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/testutils/serverutils"
"github.com/cockroachdb/cockroach/pkg/testutils/skip"
"github.com/cockroachdb/cockroach/pkg/ts"
+ "github.com/cockroachdb/cockroach/pkg/upgrade/upgradebase"
"github.com/cockroachdb/cockroach/pkg/util"
"github.com/cockroachdb/cockroach/pkg/util/admission"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
@@ -1533,6 +1535,14 @@ var TestServerFactory = testServerFactoryImpl{}
// New is part of TestServerFactory interface.
func (testServerFactoryImpl) New(params base.TestServerArgs) (interface{}, error) {
+ if params.Knobs.JobsTestingKnobs != nil {
+ if params.Knobs.JobsTestingKnobs.(*jobs.TestingKnobs).DisableAdoptions {
+ if params.Knobs.UpgradeManager == nil || !params.Knobs.UpgradeManager.(*upgradebase.TestingKnobs).DontUseJobs {
+ return nil, errors.AssertionFailedf("DontUseJobs needs to be set when DisableAdoptions is set")
+ }
+ }
+ }
+
cfg := makeTestConfigFromParams(params)
ts := &TestServer{Cfg: &cfg, params: params}
diff --git a/pkg/server/version_cluster_test.go b/pkg/server/version_cluster_test.go
index e53d10e21dcc..5d5e94d2bb00 100644
--- a/pkg/server/version_cluster_test.go
+++ b/pkg/server/version_cluster_test.go
@@ -28,7 +28,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/testutils/sqlutils"
"github.com/cockroachdb/cockroach/pkg/testutils/testcluster"
"github.com/cockroachdb/cockroach/pkg/upgrade"
- "github.com/cockroachdb/cockroach/pkg/upgrade/upgrades"
+ "github.com/cockroachdb/cockroach/pkg/upgrade/upgradebase"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/protoutil"
@@ -418,17 +418,17 @@ func TestClusterVersionMixedVersionTooOld(t *testing.T) {
},
// Inject an upgrade which would run to upgrade the cluster.
// We'll validate that we never create a job for this upgrade.
- UpgradeManager: &upgrade.TestingKnobs{
+ UpgradeManager: &upgradebase.TestingKnobs{
ListBetweenOverride: func(from, to roachpb.Version) []roachpb.Version {
return []roachpb.Version{to}
},
- RegistryOverride: func(cv roachpb.Version) (upgrade.Upgrade, bool) {
+ RegistryOverride: func(cv roachpb.Version) (upgradebase.Upgrade, bool) {
if !cv.Equal(v1) {
return nil, false
}
return upgrade.NewTenantUpgrade("testing",
v1,
- upgrades.NoPrecondition,
+ upgrade.NoPrecondition,
func(
ctx context.Context, version clusterversion.ClusterVersion, deps upgrade.TenantDeps,
) error {
diff --git a/pkg/sql/BUILD.bazel b/pkg/sql/BUILD.bazel
index 1e6b026792a7..be209bb0b72b 100644
--- a/pkg/sql/BUILD.bazel
+++ b/pkg/sql/BUILD.bazel
@@ -463,6 +463,7 @@ go_library(
"//pkg/storage/enginepb",
"//pkg/testutils/serverutils",
"//pkg/upgrade",
+ "//pkg/upgrade/upgradebase",
"//pkg/util",
"//pkg/util/admission",
"//pkg/util/admission/admissionpb",
@@ -758,6 +759,7 @@ go_test(
"//pkg/testutils/skip",
"//pkg/testutils/sqlutils",
"//pkg/testutils/testcluster",
+ "//pkg/upgrade/upgradebase",
"//pkg/util",
"//pkg/util/admission",
"//pkg/util/bitarray",
diff --git a/pkg/sql/catalog/lease/lease_internal_test.go b/pkg/sql/catalog/lease/lease_internal_test.go
index aaa68a39a114..15c206edd2ae 100644
--- a/pkg/sql/catalog/lease/lease_internal_test.go
+++ b/pkg/sql/catalog/lease/lease_internal_test.go
@@ -833,6 +833,8 @@ CREATE TABLE t.test (k CHAR PRIMARY KEY, v CHAR);
func TestLeaseAcquireAndReleaseConcurrently(t *testing.T) {
defer leaktest.AfterTest(t)()
+ // TODO(andrei): the startupmigrations pkg is gone and so are migrations
+ // requiring backfill. The test crashes when run, though; it rotted.
skip.WithIssue(t, 51798, "fails in the presence of migrations requiring backfill, "+
"but cannot import startupmigrations")
diff --git a/pkg/sql/crdb_internal_test.go b/pkg/sql/crdb_internal_test.go
index 8cb8b1ba8d69..376960c68ebf 100644
--- a/pkg/sql/crdb_internal_test.go
+++ b/pkg/sql/crdb_internal_test.go
@@ -46,6 +46,7 @@ import (
"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/upgrade/upgradebase"
"github.com/cockroachdb/cockroach/pkg/util/ctxgroup"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
"github.com/cockroachdb/cockroach/pkg/util/log"
@@ -798,6 +799,10 @@ func TestInternalJobsTableRetryColumns(t *testing.T) {
JobsTestingKnobs: &jobs.TestingKnobs{
DisableAdoptions: true,
},
+ // DisableAdoptions needs this.
+ UpgradeManager: &upgradebase.TestingKnobs{
+ DontUseJobs: true,
+ },
},
})
ctx := context.Background()
diff --git a/pkg/sql/exec_util.go b/pkg/sql/exec_util.go
index af4bd70c2649..55f19d0320b3 100644
--- a/pkg/sql/exec_util.go
+++ b/pkg/sql/exec_util.go
@@ -96,6 +96,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/types"
"github.com/cockroachdb/cockroach/pkg/testutils/serverutils"
"github.com/cockroachdb/cockroach/pkg/upgrade"
+ "github.com/cockroachdb/cockroach/pkg/upgrade/upgradebase"
"github.com/cockroachdb/cockroach/pkg/util/bitarray"
"github.com/cockroachdb/cockroach/pkg/util/duration"
"github.com/cockroachdb/cockroach/pkg/util/errorutil/unimplemented"
@@ -1195,7 +1196,7 @@ type ExecutorConfig struct {
InternalRowMetrics *rowinfra.Metrics
TestingKnobs ExecutorTestingKnobs
- UpgradeTestingKnobs *upgrade.TestingKnobs
+ UpgradeTestingKnobs *upgradebase.TestingKnobs
PGWireTestingKnobs *PGWireTestingKnobs
SchemaChangerTestingKnobs *SchemaChangerTestingKnobs
DeclarativeSchemaChangerTestingKnobs *scexec.TestingKnobs
diff --git a/pkg/sql/importer/import_stmt_test.go b/pkg/sql/importer/import_stmt_test.go
index 4b8f5b2d0c37..eb1aaade1bc0 100644
--- a/pkg/sql/importer/import_stmt_test.go
+++ b/pkg/sql/importer/import_stmt_test.go
@@ -6209,7 +6209,7 @@ func TestImportPgDumpSchemas(t *testing.T) {
// There should be two jobs, the import and a job updating the parent
// database descriptor.
- sqlDB.CheckQueryResults(t, `SELECT job_type, status FROM [SHOW JOBS] ORDER BY job_type`,
+ sqlDB.CheckQueryResults(t, `SELECT job_type, status FROM [SHOW JOBS] WHERE job_type != 'MIGRATION' ORDER BY job_type`,
[][]string{{"IMPORT", "succeeded"}, {"SCHEMA CHANGE", "succeeded"}})
// Attempt to rename one of the imported schema's so as to verify that
diff --git a/pkg/sql/logictest/BUILD.bazel b/pkg/sql/logictest/BUILD.bazel
index 680265262c9b..6940456c0122 100644
--- a/pkg/sql/logictest/BUILD.bazel
+++ b/pkg/sql/logictest/BUILD.bazel
@@ -58,7 +58,7 @@ go_library(
"//pkg/testutils/serverutils",
"//pkg/testutils/skip",
"//pkg/testutils/sqlutils",
- "//pkg/upgrade",
+ "//pkg/upgrade/upgradebase",
"//pkg/util",
"//pkg/util/envutil",
"//pkg/util/log",
diff --git a/pkg/sql/logictest/logic.go b/pkg/sql/logictest/logic.go
index bcc8158a8c36..ba54b97e4792 100644
--- a/pkg/sql/logictest/logic.go
+++ b/pkg/sql/logictest/logic.go
@@ -70,7 +70,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/testutils/serverutils"
"github.com/cockroachdb/cockroach/pkg/testutils/skip"
"github.com/cockroachdb/cockroach/pkg/testutils/sqlutils"
- "github.com/cockroachdb/cockroach/pkg/upgrade"
+ "github.com/cockroachdb/cockroach/pkg/upgrade/upgradebase"
"github.com/cockroachdb/cockroach/pkg/util"
"github.com/cockroachdb/cockroach/pkg/util/envutil"
"github.com/cockroachdb/cockroach/pkg/util/log"
@@ -1344,9 +1344,9 @@ func (t *logicTest) newCluster(
// If we're injecting fake versions, hook up logic to simulate the end
// version existing.
if len(clusterversion.ListBetween(cfg.BootstrapVersion, cfg.BinaryVersion)) == 0 {
- mm, ok := nodeParams.Knobs.UpgradeManager.(*upgrade.TestingKnobs)
+ mm, ok := nodeParams.Knobs.UpgradeManager.(*upgradebase.TestingKnobs)
if !ok {
- mm = &upgrade.TestingKnobs{}
+ mm = &upgradebase.TestingKnobs{}
nodeParams.Knobs.UpgradeManager = mm
}
mm.ListBetweenOverride = func(
diff --git a/pkg/sql/logictest/testdata/logic_test/drop_database b/pkg/sql/logictest/testdata/logic_test/drop_database
index 60a4f145322c..e74b91df69a7 100644
--- a/pkg/sql/logictest/testdata/logic_test/drop_database
+++ b/pkg/sql/logictest/testdata/logic_test/drop_database
@@ -42,7 +42,7 @@ test root NULL NULL {} NULL
skipif config local-legacy-schema-changer
query TT
-SELECT job_type, status FROM [SHOW JOBS] WHERE user_name = 'root'
+SELECT job_type, status FROM [SHOW JOBS] WHERE user_name = 'root' and job_type != 'MIGRATION'
----
SCHEMA CHANGE succeeded
SCHEMA CHANGE succeeded
diff --git a/pkg/sql/sem/builtins/builtins.go b/pkg/sql/sem/builtins/builtins.go
index e1aeacf4a0f1..2eba5e4f292f 100644
--- a/pkg/sql/sem/builtins/builtins.go
+++ b/pkg/sql/sem/builtins/builtins.go
@@ -5999,33 +5999,6 @@ value if you rely on the HLC for accuracy.`,
Volatility: volatility.Stable,
},
),
- "crdb_internal.completed_migrations": makeBuiltin(
- tree.FunctionProperties{
- Category: builtinconstants.CategorySystemInfo,
- },
- tree.Overload{
- Types: tree.ArgTypes{},
- ReturnType: tree.FixedReturnType(types.StringArray),
- Fn: func(ctx context.Context, evalCtx *eval.Context, args tree.Datums) (tree.Datum, error) {
- prefix := evalCtx.Codec.StartupMigrationKeyPrefix()
- keyvals, err := evalCtx.Txn.Scan(ctx, prefix, prefix.PrefixEnd(), 0 /* maxRows */)
- if err != nil {
- return nil, errors.Wrapf(err, "failed to get list of completed migrations")
- }
- ret := &tree.DArray{ParamTyp: types.String, Array: make(tree.Datums, 0, len(keyvals))}
- for _, keyval := range keyvals {
- key := keyval.Key
- if len(key) > len(keys.StartupMigrationPrefix) {
- key = key[len(keys.StartupMigrationPrefix):]
- }
- ret.Array = append(ret.Array, tree.NewDString(string(key)))
- }
- return ret, nil
- },
- Info: "This function is used only by CockroachDB's developers for testing purposes.",
- Volatility: volatility.Volatile,
- },
- ),
"crdb_internal.unsafe_upsert_descriptor": makeBuiltin(
tree.FunctionProperties{
Category: builtinconstants.CategorySystemRepair,
diff --git a/pkg/sql/sem/builtins/fixed_oids.go b/pkg/sql/sem/builtins/fixed_oids.go
index 4e5789e0f445..97417a84e220 100644
--- a/pkg/sql/sem/builtins/fixed_oids.go
+++ b/pkg/sql/sem/builtins/fixed_oids.go
@@ -378,7 +378,6 @@ var builtinOidsBySignature = map[string]oid.Oid{
`crdb_internal.compact_engine_span(node_id: int, store_id: int, start_key: bytes, end_key: bytes) -> bool`: 1356,
`crdb_internal.complete_replication_stream(stream_id: int, successful_ingestion: bool) -> int`: 1552,
`crdb_internal.complete_stream_ingestion_job(job_id: int, cutover_ts: timestamptz) -> int`: 1545,
- `crdb_internal.completed_migrations() -> string[]`: 1344,
`crdb_internal.create_join_token() -> string`: 1303,
`crdb_internal.create_regclass(oid: oid, name: string) -> regclass`: 2022,
`crdb_internal.create_regnamespace(oid: oid, name: string) -> regnamespace`: 2024,
diff --git a/pkg/startupmigrations/BUILD.bazel b/pkg/startupmigrations/BUILD.bazel
deleted file mode 100644
index e96a4868e11e..000000000000
--- a/pkg/startupmigrations/BUILD.bazel
+++ /dev/null
@@ -1,68 +0,0 @@
-load("//build/bazelutil/unused_checker:unused.bzl", "get_x_data")
-load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test")
-
-go_library(
- name = "startupmigrations",
- srcs = [
- "doc.go",
- "migrations.go",
- ],
- importpath = "github.com/cockroachdb/cockroach/pkg/startupmigrations",
- visibility = ["//visibility:public"],
- deps = [
- "//pkg/base",
- "//pkg/clusterversion",
- "//pkg/jobs",
- "//pkg/keys",
- "//pkg/kv",
- "//pkg/roachpb",
- "//pkg/security/username",
- "//pkg/settings/cluster",
- "//pkg/sql",
- "//pkg/sql/catalog/catalogkeys",
- "//pkg/sql/sem/tree",
- "//pkg/sql/sessiondata",
- "//pkg/startupmigrations/leasemanager",
- "//pkg/util/hlc",
- "//pkg/util/log",
- "//pkg/util/protoutil",
- "//pkg/util/retry",
- "//pkg/util/stop",
- "//pkg/util/timeutil",
- "@com_github_cockroachdb_errors//:errors",
- ],
-)
-
-go_test(
- name = "startupmigrations_test",
- srcs = [
- "main_test.go",
- "migrations_test.go",
- ],
- args = ["-test.timeout=295s"],
- embed = [":startupmigrations"],
- deps = [
- "//pkg/base",
- "//pkg/cli/exit",
- "//pkg/keys",
- "//pkg/kv",
- "//pkg/roachpb",
- "//pkg/security/securityassets",
- "//pkg/security/securitytest",
- "//pkg/security/username",
- "//pkg/server",
- "//pkg/sql",
- "//pkg/startupmigrations/leasemanager",
- "//pkg/testutils",
- "//pkg/testutils/serverutils",
- "//pkg/testutils/sqlutils",
- "//pkg/util/leaktest",
- "//pkg/util/log",
- "//pkg/util/randutil",
- "//pkg/util/stop",
- "@com_github_cockroachdb_errors//:errors",
- "@com_github_stretchr_testify//require",
- ],
-)
-
-get_x_data(name = "get_x_data")
diff --git a/pkg/startupmigrations/doc.go b/pkg/startupmigrations/doc.go
deleted file mode 100644
index c49108b6221b..000000000000
--- a/pkg/startupmigrations/doc.go
+++ /dev/null
@@ -1,38 +0,0 @@
-// Copyright 2020 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 startupmigrations provides a toolkit for running migrations upon
-// startup.
-//
-// These migrations may be associated with a clusterversion which includes the
-// relevant change in bootstrap or they may be permanent. Migrations
-// associated with bootstrap state are said to be "baked in" to a certain
-// version and thus can be deleted in the subsequent major release.
-//
-// # Differences from upgrades package
-//
-// This package overlaps in functionality with the migration subsystem. The
-// major differences are that the "long running" migrations in pkg/migration
-// run only after all instances have been upgraded to run the new code as
-// opposed to startup migrations which run upon the first instance to run
-// the new code. Another difference is that startup migrations run before
-// the instance will serve any traffic whereas the long-running migrations
-// run only after startup.
-//
-// A key differentiator between the two migration frameworks is the
-// possibility of "permanent" migrations. The long-running migrations
-// are always anchored to a version and thus are always assumed to be
-// "baked in" at that version. This is not the case for startupmigrations.
-// An advantage to "permanent" migrations over baking some data into the
-// cluster at bootstrap is that it may be easier to write given that you
-// can execute sql and exercise code on a running cluster.
-//
-// In a future world, it may make sense to unify these two frameworks.
-package startupmigrations
diff --git a/pkg/startupmigrations/leasemanager/BUILD.bazel b/pkg/startupmigrations/leasemanager/BUILD.bazel
deleted file mode 100644
index 8c15a5a85a04..000000000000
--- a/pkg/startupmigrations/leasemanager/BUILD.bazel
+++ /dev/null
@@ -1,68 +0,0 @@
-load("//build/bazelutil/unused_checker:unused.bzl", "get_x_data")
-load("@rules_proto//proto:defs.bzl", "proto_library")
-load("@io_bazel_rules_go//proto:def.bzl", "go_proto_library")
-load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test")
-
-go_library(
- name = "leasemanager",
- srcs = ["lease.go"],
- embed = [":leasemanager_go_proto"],
- importpath = "github.com/cockroachdb/cockroach/pkg/startupmigrations/leasemanager",
- visibility = ["//visibility:public"],
- deps = [
- "//pkg/kv",
- "//pkg/roachpb",
- "//pkg/util/hlc",
- "//pkg/util/uuid",
- "@com_github_cockroachdb_errors//:errors",
- ],
-)
-
-go_test(
- name = "leasemanager_test",
- size = "small",
- srcs = [
- "lease_test.go",
- "main_test.go",
- ],
- args = ["-test.timeout=55s"],
- deps = [
- ":leasemanager",
- "//pkg/base",
- "//pkg/roachpb",
- "//pkg/security/securityassets",
- "//pkg/security/securitytest",
- "//pkg/server",
- "//pkg/testutils",
- "//pkg/testutils/serverutils",
- "//pkg/util/hlc",
- "//pkg/util/leaktest",
- "//pkg/util/timeutil",
- "@com_github_cockroachdb_errors//:errors",
- ],
-)
-
-proto_library(
- name = "leasemanager_proto",
- srcs = ["lease.proto"],
- strip_import_prefix = "/pkg",
- visibility = ["//visibility:public"],
- deps = [
- "//pkg/util/hlc:hlc_proto",
- "@com_github_gogo_protobuf//gogoproto:gogo_proto",
- ],
-)
-
-go_proto_library(
- name = "leasemanager_go_proto",
- compilers = ["//pkg/cmd/protoc-gen-gogoroach:protoc-gen-gogoroach_compiler"],
- importpath = "github.com/cockroachdb/cockroach/pkg/startupmigrations/leasemanager",
- proto = ":leasemanager_proto",
- visibility = ["//visibility:public"],
- deps = [
- "//pkg/util/hlc",
- "@com_github_gogo_protobuf//gogoproto",
- ],
-)
-
-get_x_data(name = "get_x_data")
diff --git a/pkg/startupmigrations/leasemanager/lease.go b/pkg/startupmigrations/leasemanager/lease.go
deleted file mode 100644
index 07b81759cd99..000000000000
--- a/pkg/startupmigrations/leasemanager/lease.go
+++ /dev/null
@@ -1,187 +0,0 @@
-// Copyright 2016 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 leasemanager provides functionality for acquiring and managing leases
-// via the kv api for use during startupmigrations.
-package leasemanager
-
-import (
- "context"
- "fmt"
- "time"
-
- "github.com/cockroachdb/cockroach/pkg/kv"
- "github.com/cockroachdb/cockroach/pkg/roachpb"
- "github.com/cockroachdb/cockroach/pkg/util/hlc"
- "github.com/cockroachdb/cockroach/pkg/util/uuid"
- "github.com/cockroachdb/errors"
-)
-
-// DefaultLeaseDuration is the duration a lease will be acquired for if no
-// duration was specified in a LeaseManager's options.
-// Exported for testing purposes.
-const DefaultLeaseDuration = 1 * time.Minute
-
-// LeaseNotAvailableError indicates that the lease the caller attempted to
-// acquire is currently held by a different client.
-type LeaseNotAvailableError struct {
- key roachpb.Key
- expiration hlc.Timestamp
-}
-
-func (e *LeaseNotAvailableError) Error() string {
- return fmt.Sprintf("lease %q is not available until at least %s", e.key, e.expiration)
-}
-
-// LeaseManager provides functionality for acquiring and managing leases
-// via the kv api.
-type LeaseManager struct {
- db *kv.DB
- clock *hlc.Clock
- clientID string
- leaseDuration time.Duration
-}
-
-// Lease contains the state of a lease on a particular key.
-type Lease struct {
- key roachpb.Key
- val struct {
- sem chan struct{}
- lease *LeaseVal
- leaseRaw []byte
- }
-}
-
-// Options are used to configure a new LeaseManager.
-type Options struct {
- // ClientID must be unique to this LeaseManager instance.
- ClientID string
- LeaseDuration time.Duration
-}
-
-// New allocates a new LeaseManager.
-func New(db *kv.DB, clock *hlc.Clock, options Options) *LeaseManager {
- if options.ClientID == "" {
- options.ClientID = uuid.MakeV4().String()
- }
- if options.LeaseDuration <= 0 {
- options.LeaseDuration = DefaultLeaseDuration
- }
- return &LeaseManager{
- db: db,
- clock: clock,
- clientID: options.ClientID,
- leaseDuration: options.LeaseDuration,
- }
-}
-
-// AcquireLease attempts to grab a lease on the provided key. Returns a non-nil
-// lease object if it was successful, or an error if it failed to acquire the
-// lease for any reason.
-//
-// NB: Acquiring a non-expired lease is allowed if this LeaseManager's clientID
-// matches the lease owner's ID. This behavior allows a process to re-grab
-// leases without having to wait if it restarts and uses the same ID.
-func (m *LeaseManager) AcquireLease(ctx context.Context, key roachpb.Key) (*Lease, error) {
- lease := &Lease{
- key: key,
- }
- lease.val.sem = make(chan struct{}, 1)
- if err := m.db.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error {
- var val LeaseVal
- err := txn.GetProto(ctx, key, &val)
- if err != nil {
- return err
- }
- if !m.leaseAvailable(&val) {
- return &LeaseNotAvailableError{key: key, expiration: val.Expiration}
- }
- lease.val.lease = &LeaseVal{
- Owner: m.clientID,
- Expiration: m.clock.Now().Add(m.leaseDuration.Nanoseconds(), 0),
- }
- var leaseRaw roachpb.Value
- if err := leaseRaw.SetProto(lease.val.lease); err != nil {
- return err
- }
- if err := txn.Put(ctx, key, &leaseRaw); err != nil {
- return err
- }
- lease.val.leaseRaw = leaseRaw.TagAndDataBytes()
- return nil
- }); err != nil {
- return nil, err
- }
- return lease, nil
-}
-
-func (m *LeaseManager) leaseAvailable(val *LeaseVal) bool {
- return val.Owner == m.clientID || m.timeRemaining(val) <= 0
-}
-
-// TimeRemaining returns the amount of time left on the given lease.
-func (m *LeaseManager) TimeRemaining(l *Lease) time.Duration {
- l.val.sem <- struct{}{}
- defer func() { <-l.val.sem }()
- return m.timeRemaining(l.val.lease)
-}
-
-func (m *LeaseManager) timeRemaining(val *LeaseVal) time.Duration {
- maxOffset := m.clock.MaxOffset()
- return val.Expiration.GoTime().Sub(m.clock.Now().GoTime()) - maxOffset
-}
-
-// ExtendLease attempts to push the expiration time of the lease farther out
-// into the future.
-func (m *LeaseManager) ExtendLease(ctx context.Context, l *Lease) error {
- select {
- case <-ctx.Done():
- return ctx.Err()
- case l.val.sem <- struct{}{}:
- }
- defer func() { <-l.val.sem }()
-
- if m.timeRemaining(l.val.lease) < 0 {
- return errors.Errorf("can't extend lease that expired at time %s", l.val.lease.Expiration)
- }
-
- newVal := &LeaseVal{
- Owner: m.clientID,
- Expiration: m.clock.Now().Add(m.leaseDuration.Nanoseconds(), 0),
- }
- var newRaw roachpb.Value
- if err := newRaw.SetProto(newVal); err != nil {
- return err
- }
- if err := m.db.CPut(ctx, l.key, &newRaw, l.val.leaseRaw); err != nil {
- if errors.HasType(err, (*roachpb.ConditionFailedError)(nil)) {
- // Something is wrong - immediately expire the local lease state.
- l.val.lease.Expiration = hlc.Timestamp{}
- return errors.Wrapf(err, "local lease state %v out of sync with DB state", l.val.lease)
- }
- return err
- }
- l.val.lease = newVal
- l.val.leaseRaw = newRaw.TagAndDataBytes()
- return nil
-}
-
-// ReleaseLease attempts to release the given lease so that another process can
-// grab it.
-func (m *LeaseManager) ReleaseLease(ctx context.Context, l *Lease) error {
- select {
- case <-ctx.Done():
- return ctx.Err()
- case l.val.sem <- struct{}{}:
- }
- defer func() { <-l.val.sem }()
-
- return m.db.CPut(ctx, l.key, nil /* value - delete the lease */, l.val.leaseRaw)
-}
diff --git a/pkg/startupmigrations/leasemanager/lease.proto b/pkg/startupmigrations/leasemanager/lease.proto
deleted file mode 100644
index 333416e5012c..000000000000
--- a/pkg/startupmigrations/leasemanager/lease.proto
+++ /dev/null
@@ -1,24 +0,0 @@
-// Copyright 2016 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.
-
-syntax = "proto2";
-package cockroach.client;
-option go_package = "leasemanager";
-
-import "util/hlc/timestamp.proto";
-import "gogoproto/gogo.proto";
-
-message LeaseVal {
- // An opaque string that should be unique per client to identify which client
- // owns the lease.
- optional string owner = 1 [(gogoproto.nullable) = false];
- // The expiration time of the lease.
- optional util.hlc.Timestamp expiration = 2 [(gogoproto.nullable) = false];
-}
diff --git a/pkg/startupmigrations/leasemanager/lease_test.go b/pkg/startupmigrations/leasemanager/lease_test.go
deleted file mode 100644
index 87496aaffa00..000000000000
--- a/pkg/startupmigrations/leasemanager/lease_test.go
+++ /dev/null
@@ -1,179 +0,0 @@
-// Copyright 2016 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 leasemanager_test
-
-import (
- "context"
- "testing"
- "time"
-
- "github.com/cockroachdb/cockroach/pkg/base"
- "github.com/cockroachdb/cockroach/pkg/roachpb"
- "github.com/cockroachdb/cockroach/pkg/startupmigrations/leasemanager"
- "github.com/cockroachdb/cockroach/pkg/testutils"
- "github.com/cockroachdb/cockroach/pkg/testutils/serverutils"
- "github.com/cockroachdb/cockroach/pkg/util/hlc"
- "github.com/cockroachdb/cockroach/pkg/util/leaktest"
- "github.com/cockroachdb/cockroach/pkg/util/timeutil"
- "github.com/cockroachdb/errors"
-)
-
-const (
- clientID1 = "1"
- clientID2 = "2"
-)
-
-var (
- leaseKey = roachpb.Key("/SystemVersion/lease")
-)
-
-func TestAcquireAndRelease(t *testing.T) {
- defer leaktest.AfterTest(t)()
- ctx := context.Background()
- s, _, db := serverutils.StartServer(t, base.TestServerArgs{})
- defer s.Stopper().Stop(ctx)
-
- clock := hlc.NewClock(timeutil.NewManualTime(timeutil.Unix(0, 123)), time.Nanosecond /* maxOffset */)
- lm := leasemanager.New(db, clock, leasemanager.Options{ClientID: clientID1})
-
- l, err := lm.AcquireLease(ctx, leaseKey)
- if err != nil {
- t.Fatal(err)
- }
- if err := lm.ReleaseLease(ctx, l); err != nil {
- t.Fatal(err)
- }
- if err := lm.ReleaseLease(ctx, l); !testutils.IsError(err, "unexpected value") {
- t.Fatal(err)
- }
-
- l, err = lm.AcquireLease(ctx, leaseKey)
- if err != nil {
- t.Fatal(err)
- }
- if err := lm.ReleaseLease(ctx, l); err != nil {
- t.Fatal(err)
- }
-}
-
-func TestReacquireLease(t *testing.T) {
- defer leaktest.AfterTest(t)()
-
- ctx := context.Background()
- s, _, db := serverutils.StartServer(t, base.TestServerArgs{})
- defer s.Stopper().Stop(ctx)
-
- clock := hlc.NewClock(timeutil.NewManualTime(timeutil.Unix(0, 123)), time.Nanosecond /* maxOffset */)
- lm := leasemanager.New(db, clock, leasemanager.Options{ClientID: clientID1})
-
- if _, err := lm.AcquireLease(ctx, leaseKey); err != nil {
- t.Fatal(err)
- }
-
- // We allow re-acquiring the same lease as long as the client ID is
- // the same to allow a client to reacquire its own leases rather than
- // having to wait them out if it crashes and restarts.
- l, err := lm.AcquireLease(ctx, leaseKey)
- if err != nil {
- t.Fatal(err)
- }
- if err := lm.ReleaseLease(ctx, l); err != nil {
- t.Fatal(err)
- }
-}
-
-func TestExtendLease(t *testing.T) {
- defer leaktest.AfterTest(t)()
-
- ctx := context.Background()
- s, _, db := serverutils.StartServer(t, base.TestServerArgs{})
- defer s.Stopper().Stop(ctx)
-
- manual := timeutil.NewManualTime(timeutil.Unix(0, 123))
- clock := hlc.NewClock(manual, time.Nanosecond /* maxOffset */)
- lm := leasemanager.New(db, clock, leasemanager.Options{ClientID: clientID1})
-
- l, err := lm.AcquireLease(ctx, leaseKey)
- if err != nil {
- t.Fatal(err)
- }
-
- manual.Advance(time.Second)
- timeRemainingBefore := lm.TimeRemaining(l)
- if err := lm.ExtendLease(ctx, l); err != nil {
- t.Fatal(err)
- }
- timeRemainingAfter := lm.TimeRemaining(l)
- if !(timeRemainingAfter > timeRemainingBefore) {
- t.Errorf("expected time remaining after renewal (%s) to be greater than before renewal (%s)",
- timeRemainingAfter, timeRemainingBefore)
- }
-
- manual.Advance(leasemanager.DefaultLeaseDuration + 1)
- if tr := lm.TimeRemaining(l); tr >= 0 {
- t.Errorf("expected negative time remaining on lease, got %s", tr)
- }
- if err := lm.ExtendLease(ctx, l); !testutils.IsError(err, "can't extend lease that expired") {
- t.Fatalf("didn't get expected error when renewing lease %+v: %v", l, err)
- }
-
- if err := lm.ReleaseLease(ctx, l); err != nil {
- t.Fatal(err)
- }
-}
-
-func TestLeasesMultipleClients(t *testing.T) {
- defer leaktest.AfterTest(t)()
-
- ctx := context.Background()
- s, _, db := serverutils.StartServer(t, base.TestServerArgs{})
- defer s.Stopper().Stop(ctx)
-
- manual1 := timeutil.NewManualTime(timeutil.Unix(0, 123))
- clock1 := hlc.NewClock(manual1, time.Nanosecond /* maxOffset */)
- manual2 := timeutil.NewManualTime(timeutil.Unix(0, 123))
- clock2 := hlc.NewClock(manual2, time.Nanosecond /* maxOffset */)
- lm1 := leasemanager.New(db, clock1, leasemanager.Options{ClientID: clientID1})
- lm2 := leasemanager.New(db, clock2, leasemanager.Options{ClientID: clientID2})
-
- l1, err := lm1.AcquireLease(ctx, leaseKey)
- if err != nil {
- t.Fatal(err)
- }
- _, err = lm2.AcquireLease(ctx, leaseKey)
- if !testutils.IsError(err, "is not available until") {
- t.Fatalf("didn't get expected error trying to acquire already held lease: %v", err)
- }
- if !errors.HasType(err, (*leasemanager.LeaseNotAvailableError)(nil)) {
- t.Fatalf("expected LeaseNotAvailableError, got %v", err)
- }
-
- // Ensure a lease can be "stolen" after it's expired.
- manual2.Advance(leasemanager.DefaultLeaseDuration + 1)
- l2, err := lm2.AcquireLease(ctx, leaseKey)
- if err != nil {
- t.Fatal(err)
- }
-
- // lm1's clock indicates that its lease should still be valid, but it doesn't
- // own it anymore.
- manual1.Advance(leasemanager.DefaultLeaseDuration / 2)
- if err := lm1.ExtendLease(ctx, l1); !testutils.IsError(err, "out of sync with DB state") {
- t.Fatalf("didn't get expected error trying to extend expired lease: %v", err)
- }
- if err := lm1.ReleaseLease(ctx, l1); !testutils.IsError(err, "unexpected value") {
- t.Fatalf("didn't get expected error trying to release stolen lease: %v", err)
- }
-
- if err := lm2.ReleaseLease(ctx, l2); err != nil {
- t.Fatal(err)
- }
-}
diff --git a/pkg/startupmigrations/leasemanager/main_test.go b/pkg/startupmigrations/leasemanager/main_test.go
deleted file mode 100644
index 006d7b7361b5..000000000000
--- a/pkg/startupmigrations/leasemanager/main_test.go
+++ /dev/null
@@ -1,32 +0,0 @@
-// Copyright 2015 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 leasemanager_test
-
-import (
- "os"
- "testing"
-
- "github.com/cockroachdb/cockroach/pkg/security/securityassets"
- "github.com/cockroachdb/cockroach/pkg/security/securitytest"
- "github.com/cockroachdb/cockroach/pkg/server"
- "github.com/cockroachdb/cockroach/pkg/testutils/serverutils"
-)
-
-func init() {
- securityassets.SetLoader(securitytest.EmbeddedAssets)
-}
-
-func TestMain(m *testing.M) {
- serverutils.InitTestServerFactory(server.TestServerFactory)
- os.Exit(m.Run())
-}
-
-//go:generate ../../util/leaktest/add-leaktest.sh *_test.go
diff --git a/pkg/startupmigrations/main_test.go b/pkg/startupmigrations/main_test.go
deleted file mode 100644
index 0e480bf2cdca..000000000000
--- a/pkg/startupmigrations/main_test.go
+++ /dev/null
@@ -1,30 +0,0 @@
-// Copyright 2017 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 startupmigrations_test
-
-import (
- "os"
- "testing"
-
- "github.com/cockroachdb/cockroach/pkg/security/securityassets"
- "github.com/cockroachdb/cockroach/pkg/security/securitytest"
- "github.com/cockroachdb/cockroach/pkg/server"
- "github.com/cockroachdb/cockroach/pkg/testutils/serverutils"
-)
-
-func TestMain(m *testing.M) {
- securityassets.SetLoader(securitytest.EmbeddedAssets)
- serverutils.InitTestServerFactory(server.TestServerFactory)
-
- os.Exit(m.Run())
-}
-
-//go:generate ../util/leaktest/add-leaktest.sh *_test.go
diff --git a/pkg/startupmigrations/migrations.go b/pkg/startupmigrations/migrations.go
deleted file mode 100644
index 1a4e48eb7a35..000000000000
--- a/pkg/startupmigrations/migrations.go
+++ /dev/null
@@ -1,869 +0,0 @@
-// Copyright 2016 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 startupmigrations
-
-import (
- "context"
- "fmt"
- "time"
-
- "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/kv"
- "github.com/cockroachdb/cockroach/pkg/roachpb"
- "github.com/cockroachdb/cockroach/pkg/security/username"
- "github.com/cockroachdb/cockroach/pkg/settings/cluster"
- "github.com/cockroachdb/cockroach/pkg/sql"
- "github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkeys"
- "github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
- "github.com/cockroachdb/cockroach/pkg/sql/sessiondata"
- "github.com/cockroachdb/cockroach/pkg/startupmigrations/leasemanager"
- "github.com/cockroachdb/cockroach/pkg/util/hlc"
- "github.com/cockroachdb/cockroach/pkg/util/log"
- "github.com/cockroachdb/cockroach/pkg/util/protoutil"
- "github.com/cockroachdb/cockroach/pkg/util/retry"
- "github.com/cockroachdb/cockroach/pkg/util/stop"
- "github.com/cockroachdb/cockroach/pkg/util/timeutil"
- "github.com/cockroachdb/errors"
-)
-
-var (
- leaseDuration = time.Minute
- leaseRefreshInterval = leaseDuration / 5
-)
-
-// MigrationManagerTestingKnobs contains testing knobs.
-type MigrationManagerTestingKnobs struct {
- // AfterEnsureMigrations is called after each call to EnsureMigrations.
- AfterEnsureMigrations func()
-}
-
-// ModuleTestingKnobs is part of the base.ModuleTestingKnobs interface.
-func (*MigrationManagerTestingKnobs) ModuleTestingKnobs() {}
-
-var _ base.ModuleTestingKnobs = &MigrationManagerTestingKnobs{}
-
-// backwardCompatibleMigrations is a hard-coded list of migrations to be run on
-// startup. They will always be run from top-to-bottom, and because they are
-// assumed to be backward-compatible, they will be run regardless of what other
-// node versions are currently running within the cluster.
-// Migrations must be idempotent: a migration may run successfully but not be
-// recorded as completed, causing a second run.
-//
-// Attention: If a migration is creating new tables, it should also be added to
-// the metadata schema written by bootstrap (see addSystemDatabaseToSchema())
-// and it should have the includedInBootstrap field set (see comments on that
-// field too).
-var backwardCompatibleMigrations = []migrationDescriptor{
- {
- // Introduced in v1.0. Baked into v2.0.
- name: "default UniqueID to uuid_v4 in system.eventlog",
- },
- {
- // Introduced in v1.0. Baked into v2.0.
- name: "create system.jobs table",
- },
- {
- // Introduced in v1.0. Baked into v2.0.
- name: "create system.settings table",
- },
- {
- // Introduced in v1.0. Permanent migration.
- name: "enable diagnostics reporting",
- workFn: optInToDiagnosticsStatReporting,
- },
- {
- // Introduced in v1.1. Baked into v2.0.
- name: "establish conservative dependencies for views #17280 #17269 #17306",
- },
- {
- // Introduced in v1.1. Baked into v2.0.
- name: "create system.sessions table",
- },
- {
- // Introduced in v1.1. Permanent migration.
- name: "populate initial version cluster setting table entry",
- workFn: populateVersionSetting,
- clusterWide: true,
- },
- {
- // Introduced in v2.0. Baked into v2.1.
- name: "create system.table_statistics table",
- },
- {
- // Introduced in v2.0. Permanent migration.
- name: "add root user",
- workFn: addRootUser,
- },
- {
- // Introduced in v2.0. Baked into v2.1.
- name: "create system.locations table",
- },
- {
- // Introduced in v2.0. Baked into v2.1.
- name: "add default .meta and .liveness zone configs",
- },
- {
- // Introduced in v2.0. Baked into v2.1.
- name: "create system.role_members table",
- },
- {
- // Introduced in v2.0. Permanent migration.
- name: "add system.users isRole column and create admin role",
- workFn: addAdminRole,
- },
- {
- // Introduced in v2.0, replaced by "ensure admin role privileges in all descriptors"
- name: "grant superuser privileges on all objects to the admin role",
- },
- {
- // Introduced in v2.0. Permanent migration.
- name: "make root a member of the admin role",
- workFn: addRootToAdminRole,
- },
- {
- // Introduced in v2.0. Baked into v2.1.
- name: "upgrade table descs to interleaved format version",
- },
- {
- // Introduced in v2.0 alphas then folded into `retiredSettings`.
- name: "remove cluster setting `kv.gc.batch_size`",
- },
- {
- // Introduced in v2.0 alphas then folded into `retiredSettings`.
- name: "remove cluster setting `kv.transaction.max_intents`",
- },
- {
- // Introduced in v2.0. Baked into v2.1.
- name: "add default system.jobs zone config",
- },
- {
- // Introduced in v2.0. Permanent migration.
- name: "initialize cluster.secret",
- workFn: initializeClusterSecret,
- },
- {
- // Introduced in v2.0. Repeated in v2.1 below.
- name: "ensure admin role privileges in all descriptors",
- },
- {
- // Introduced in v2.1, repeat of 2.0 migration to catch mixed-version issues.
- // TODO(mberhault): bake into v19.1.
- name: "repeat: ensure admin role privileges in all descriptors",
- },
- {
- // Introduced in v2.1.
- // TODO(mberhault): bake into v19.1.
- name: "disallow public user or role name",
- workFn: disallowPublicUserOrRole,
- },
- {
- // Introduced in v2.1.
- // TODO(knz): bake this migration into v19.1.
- name: "create default databases",
- workFn: createDefaultDbs,
- },
- {
- // Introduced in v2.1. Baked into 20.1.
- name: "add progress to system.jobs",
- },
- {
- // Introduced in v19.1.
- name: "create system.comment table",
- },
- {
- // This migration has been introduced some time before 19.2.
- name: "create system.replication_constraint_stats table",
- },
- {
- // This migration has been introduced some time before 19.2.
- name: "create system.replication_critical_localities table",
- },
- {
- // This migration has been introduced some time before 19.2.
- name: "create system.reports_meta table",
- },
- {
- // This migration has been introduced some time before 19.2.
- name: "create system.replication_stats table",
- },
- {
- // Introduced in v19.1.
- // TODO(knz): bake this migration into v19.2.
- name: "propagate the ts purge interval to the new setting names",
- workFn: retireOldTsPurgeIntervalSettings,
- clusterWide: true,
- },
- {
- // Introduced in v19.2.
- name: "update system.locations with default location data",
- workFn: updateSystemLocationData,
- },
- {
- // Introduced in v19.2, baked into v20.1.
- name: "change reports fields from timestamp to timestamptz",
- },
- {
- // Introduced in v20.1, baked into v20.2.
- name: "create system.protected_ts_meta table",
- },
- {
- // Introduced in v20.1, baked into v20.2.
- name: "create system.protected_ts_records table",
- },
- {
- // Introduced in v20.1, baked into v21.2.
- name: "create new system.namespace table v2",
- },
- {
- // Introduced in v20.10. Replaced in v20.1.1 and v20.2 by the
- // StartSystemNamespaceMigration post-finalization-style migration.
- name: "migrate system.namespace_deprecated entries into system.namespace",
- // workFn: migrateSystemNamespace,
- },
- {
- // Introduced in v20.1, baked into v20.2.
- name: "create system.role_options table",
- },
- {
- // Introduced in v20.1, baked into v20.2.
- name: "create statement_diagnostics_requests, statement_diagnostics and " +
- "system.statement_bundle_chunks tables",
- },
- {
- // Introduced in v20.1. Baked into v20.2.
- name: "add CREATEROLE privilege to admin/root",
- },
- {
- // Introduced in v20.2. Baked into v21.1.
- name: "add created_by columns to system.jobs",
- },
- {
- // Introduced in v20.2. Baked into v21.1.
- name: "create new system.scheduled_jobs table",
- },
- {
- // Introduced in v20.2. Baked into v21.1.
- name: "add new sqlliveness table and claim columns to system.jobs",
- },
- {
- // Introduced in v20.2. Baked into v21.1.
- name: "create new system.tenants table",
- // This migration does not have a dedicated cluster version key but was
- // added just before 20.1.5. With the upcoming 21.2 release, all 20.2 and
- // 21.1 version keys are deprecated and we are certainly not adding any new
- // ones in those ranges. Until these deprecated version keys are all deleted
- // we tie this migration to the last 20.2 version key.
- includedInBootstrap: roachpb.Version{Major: 20, Minor: 2},
- },
- {
- // Introduced in v20.2. Baked into v21.1.
- name: "alter scheduled jobs",
- },
- {
- // Introduced in v20.2.
- name: "add CREATELOGIN privilege to roles with CREATEROLE",
- workFn: extendCreateRoleWithCreateLogin,
- },
- {
- // Introduced in v20.2.
- name: "mark non-terminal schema change jobs with a pre-20.1 format version as failed",
- },
-}
-
-// migrationDescriptor describes a single migration hook that's used to modify
-// some part of the cluster state when the CockroachDB version is upgraded.
-// See docs/RFCs/cluster_upgrade_tool.md for details.
-type migrationDescriptor struct {
- // name must be unique amongst all hard-coded migrations.
- // ATTENTION: A migration's name can never be changed. It is included in a key
- // marking a migration as completed.
- name string
- // workFn must be idempotent so that we can safely re-run it if a node failed
- // while running it. nil if the migration has been "backed in" and is no
- // longer to be performed at cluster startup.
- workFn func(context.Context, runner) error
- // includedInBootstrap is set for migrations that need to be performed for
- // updating old clusters, but are also covered by the MetadataSchema that gets
- // created by hand for a new cluster when it bootstraps itself. This kind of
- // duplication between a migration and the MetadataSchema is useful for
- // migrations that create system descriptor - for new clusters (particularly
- // for tests) we want to create these tables by hand so that a corresponding
- // range is created at bootstrap time. Otherwise, we'd have the split queue
- // asynchronously creating some ranges which is annoying for tests.
- //
- // Generally when setting this field you'll want to introduce a new cluster
- // version.
- includedInBootstrap roachpb.Version
- // clusterWide migrations are only run by the system tenant. All other
- // migrations are run by each individual tenant. clusterWide migrations
- // typically have to do with cluster settings, which is a cluster-wide
- // concept.
- clusterWide bool
-}
-
-func init() {
- // Ensure that all migrations have unique names.
- names := make(map[string]struct{}, len(backwardCompatibleMigrations))
- for _, migration := range backwardCompatibleMigrations {
- name := migration.name
- if _, ok := names[name]; ok {
- log.Fatalf(context.Background(), "duplicate sql migration %q", name)
- }
- names[name] = struct{}{}
- }
-}
-
-type runner struct {
- db DB
- codec keys.SQLCodec
- sqlExecutor *sql.InternalExecutor
- settings *cluster.Settings
-}
-
-func (r runner) execAsRoot(ctx context.Context, opName, stmt string, qargs ...interface{}) error {
- _, err := r.sqlExecutor.ExecEx(ctx, opName, nil, /* txn */
- sessiondata.InternalExecutorOverride{
- User: username.RootUserName(),
- },
- stmt, qargs...)
- return err
-}
-
-func (r runner) execAsRootWithRetry(
- ctx context.Context, opName string, stmt string, qargs ...interface{},
-) error {
- // Retry a limited number of times because returning an error and letting
- // the node kill itself is better than holding the migration lease for an
- // arbitrarily long time.
- var err error
- for retry := retry.Start(retry.Options{MaxRetries: 5}); retry.Next(); {
- err := r.execAsRoot(ctx, opName, stmt, qargs...)
- if err == nil {
- break
- }
- log.Warningf(ctx, "failed to run %s: %v", stmt, err)
- }
- return err
-}
-
-// leaseManager is defined just to allow us to use a fake client.LeaseManager
-// when testing this package.
-type leaseManager interface {
- AcquireLease(ctx context.Context, key roachpb.Key) (*leasemanager.Lease, error)
- ExtendLease(ctx context.Context, l *leasemanager.Lease) error
- ReleaseLease(ctx context.Context, l *leasemanager.Lease) error
- TimeRemaining(l *leasemanager.Lease) time.Duration
-}
-
-// DB is defined just to allow us to use a fake client.DB when testing this
-// package.
-type DB interface {
- Scan(ctx context.Context, begin, end interface{}, maxRows int64) ([]kv.KeyValue, error)
- Get(ctx context.Context, key interface{}) (kv.KeyValue, error)
- Put(ctx context.Context, key, value interface{}) error
- Txn(ctx context.Context, retryable func(ctx context.Context, txn *kv.Txn) error) error
-
- // ReadCommittedScan is like Scan but may return an inconsistent and stale
- // snapshot.
- ReadCommittedScan(ctx context.Context, begin, end interface{}, maxRows int64) ([]kv.KeyValue, error)
-}
-
-// Manager encapsulates the necessary functionality for handling migrations
-// of data in the cluster.
-type Manager struct {
- stopper *stop.Stopper
- leaseManager leaseManager
- db DB
- codec keys.SQLCodec
- sqlExecutor *sql.InternalExecutor
- testingKnobs MigrationManagerTestingKnobs
- settings *cluster.Settings
- jobRegistry *jobs.Registry
-}
-
-// NewManager initializes and returns a new Manager object.
-func NewManager(
- stopper *stop.Stopper,
- db *kv.DB,
- codec keys.SQLCodec,
- executor *sql.InternalExecutor,
- clock *hlc.Clock,
- testingKnobs MigrationManagerTestingKnobs,
- clientID string,
- settings *cluster.Settings,
- registry *jobs.Registry,
-) *Manager {
- opts := leasemanager.Options{
- ClientID: clientID,
- LeaseDuration: leaseDuration,
- }
- return &Manager{
- stopper: stopper,
- leaseManager: leasemanager.New(db, clock, opts),
- db: dbAdapter{DB: db},
- codec: codec,
- sqlExecutor: executor,
- testingKnobs: testingKnobs,
- settings: settings,
- jobRegistry: registry,
- }
-}
-
-// dbAdapter augments the kv.DB with a ReadCommittedScan method as required
-// by the DB interface.
-type dbAdapter struct {
- *kv.DB
-}
-
-func (d dbAdapter) ReadCommittedScan(
- ctx context.Context, begin, end interface{}, maxRows int64,
-) ([]kv.KeyValue, error) {
- var b kv.Batch
- b.Header.ReadConsistency = roachpb.INCONSISTENT
- b.Scan(begin, end)
- if err := d.Run(ctx, &b); err != nil {
- return nil, err
- }
- return b.Results[0].Rows, nil
-}
-
-// EnsureMigrations should be run during node startup to ensure that all
-// required migrations have been run (and running all those that are definitely
-// safe to run).
-func (m *Manager) EnsureMigrations(ctx context.Context, bootstrapVersion roachpb.Version) error {
- if m.testingKnobs.AfterEnsureMigrations != nil {
- defer m.testingKnobs.AfterEnsureMigrations()
- }
- // First, check whether there are any migrations that need to be run.
- // We do the check potentially twice, once with a readCommittedScan which
- // might read stale values, but can be performed locally, and then, if
- // there are migrations to run, again with a consistent scan.
- if allComplete, err := m.checkIfAllMigrationsAreComplete(
- ctx, bootstrapVersion, m.db.ReadCommittedScan,
- ); err != nil || allComplete {
- return err
- }
- if allComplete, err := m.checkIfAllMigrationsAreComplete(
- ctx, bootstrapVersion, m.db.Scan,
- ); err != nil || allComplete {
- return err
- }
-
- // If there are any, grab the migration lease to ensure that only one
- // node is ever doing migrations at a time.
- // Note that we shouldn't ever let client.LeaseNotAvailableErrors cause us
- // to stop trying, because if we return an error the server will be shut down,
- // and this server being down may prevent the leaseholder from finishing.
- var lease *leasemanager.Lease
- if log.V(1) {
- log.Info(ctx, "trying to acquire lease")
- }
- var err error
- for r := retry.StartWithCtx(ctx, base.DefaultRetryOptions()); r.Next(); {
- lease, err = m.leaseManager.AcquireLease(ctx, m.codec.StartupMigrationLeaseKey())
- if err == nil {
- break
- }
- log.Infof(ctx, "failed attempt to acquire migration lease: %s", err)
- }
- if err != nil {
- return errors.Wrapf(err, "failed to acquire lease for running necessary migrations")
- }
-
- // Ensure that we hold the lease throughout the migration process and release
- // it when we're done.
- done := make(chan interface{}, 1)
- defer func() {
- done <- nil
- if log.V(1) {
- log.Info(ctx, "trying to release the lease")
- }
- if err := m.leaseManager.ReleaseLease(ctx, lease); err != nil {
- log.Errorf(ctx, "failed to release migration lease: %s", err)
- }
- }()
- if err := m.stopper.RunAsyncTask(ctx, "migrations.Manager: lease watcher",
- func(ctx context.Context) {
- select {
- case <-done:
- return
- case <-time.After(leaseRefreshInterval):
- if err := m.leaseManager.ExtendLease(ctx, lease); err != nil {
- log.Warningf(ctx, "unable to extend ownership of expiration lease: %s", err)
- }
- if m.leaseManager.TimeRemaining(lease) < leaseRefreshInterval {
- // Do one last final check of whether we're done - it's possible that
- // ReleaseLease can sneak in and execute ahead of ExtendLease even if
- // the ExtendLease started first (making for an unexpected value error),
- // and doing this final check can avoid unintended shutdowns.
- select {
- case <-done:
- return
- default:
- // Note that we may be able to do better than this by influencing the
- // deadline of migrations' transactions based on the lease expiration
- // time, but simply kill the process for now for the sake of simplicity.
- log.Fatal(ctx, "not enough time left on migration lease, terminating for safety")
- }
- }
- }
- }); err != nil {
- return err
- }
-
- // Re-get the list of migrations in case any of them were completed between
- // our initial check and our grabbing of the lease.
- completedMigrations, err := getCompletedMigrations(ctx, m.db.Scan, m.codec)
- if err != nil {
- return err
- }
-
- startTime := timeutil.Now().String()
- r := runner{
- db: m.db,
- codec: m.codec,
- sqlExecutor: m.sqlExecutor,
- settings: m.settings,
- }
- for _, migration := range backwardCompatibleMigrations {
- if !m.shouldRunMigration(migration, bootstrapVersion) {
- continue
- }
-
- key := migrationKey(m.codec, migration)
- if _, ok := completedMigrations[string(key)]; ok {
- continue
- }
-
- if log.V(1) {
- log.Infof(ctx, "running migration %q", migration.name)
- }
- if err := migration.workFn(ctx, r); err != nil {
- return errors.Wrapf(err, "failed to run migration %q", migration.name)
- }
-
- log.VEventf(ctx, 1, "persisting record of completing migration %s", migration.name)
- if err := m.db.Put(ctx, key, startTime); err != nil {
- return errors.Wrapf(err, "failed to persist record of completing migration %q",
- migration.name)
- }
- }
-
- return nil
-}
-
-func (m *Manager) checkIfAllMigrationsAreComplete(
- ctx context.Context, bootstrapVersion roachpb.Version, scan scanFunc,
-) (completedAll bool, _ error) {
- completedMigrations, err := getCompletedMigrations(ctx, scan, m.codec)
- if err != nil {
- return false, err
- }
- allMigrationsCompleted := true
- for _, migration := range backwardCompatibleMigrations {
- if !m.shouldRunMigration(migration, bootstrapVersion) {
- continue
- }
- key := migrationKey(m.codec, migration)
- if _, ok := completedMigrations[string(key)]; !ok {
- allMigrationsCompleted = false
- }
- }
- return allMigrationsCompleted, nil
-}
-
-func (m *Manager) shouldRunMigration(
- migration migrationDescriptor, bootstrapVersion roachpb.Version,
-) bool {
- if migration.workFn == nil {
- // The migration has been baked in.
- return false
- }
- minVersion := migration.includedInBootstrap
- if minVersion != (roachpb.Version{}) && !bootstrapVersion.Less(minVersion) {
- // The migration is unnecessary.
- return false
- }
- if migration.clusterWide && !m.codec.ForSystemTenant() {
- // The migration is a cluster-wide migration and we are not the
- // system tenant.
- return false
- }
- return true
-}
-
-type scanFunc = func(_ context.Context, from, to interface{}, maxRows int64) ([]kv.KeyValue, error)
-
-func getCompletedMigrations(
- ctx context.Context, scan scanFunc, codec keys.SQLCodec,
-) (map[string]struct{}, error) {
- if log.V(1) {
- log.Info(ctx, "trying to get the list of completed migrations")
- }
- prefix := codec.StartupMigrationKeyPrefix()
- keyvals, err := scan(ctx, prefix, prefix.PrefixEnd(), 0 /* maxRows */)
- if err != nil {
- return nil, errors.Wrapf(err, "failed to get list of completed migrations")
- }
- completedMigrations := make(map[string]struct{})
- for _, keyval := range keyvals {
- completedMigrations[string(keyval.Key)] = struct{}{}
- }
- return completedMigrations, nil
-}
-
-func migrationKey(codec keys.SQLCodec, migration migrationDescriptor) roachpb.Key {
- return append(codec.StartupMigrationKeyPrefix(), roachpb.RKey(migration.name)...)
-}
-
-func extendCreateRoleWithCreateLogin(ctx context.Context, r runner) error {
- // Add the CREATELOGIN option to roles that already have CREATEROLE.
- const upsertCreateRoleStmt = `
- UPSERT INTO system.role_options (username, option, value)
- SELECT username, 'CREATELOGIN', NULL
- FROM system.role_options
- WHERE option = 'CREATEROLE'
- `
- return r.execAsRootWithRetry(ctx,
- "add CREATELOGIN where a role already has CREATEROLE",
- upsertCreateRoleStmt)
-}
-
-// SettingsDefaultOverrides documents the effect of several migrations that add
-// an explicit value for a setting, effectively changing the "default value"
-// from what was defined in code.
-var SettingsDefaultOverrides = map[string]string{
- "diagnostics.reporting.enabled": "true",
- "cluster.secret": "",
-}
-
-func optInToDiagnosticsStatReporting(ctx context.Context, r runner) error {
- // We're opting-out of the automatic opt-in. See discussion in updates.go.
- if cluster.TelemetryOptOut() {
- return nil
- }
- return r.execAsRootWithRetry(ctx, "optInToDiagnosticsStatReporting",
- `SET CLUSTER SETTING diagnostics.reporting.enabled = true`)
-}
-
-func initializeClusterSecret(ctx context.Context, r runner) error {
- return r.execAsRootWithRetry(
- ctx, "initializeClusterSecret",
- `SET CLUSTER SETTING cluster.secret = gen_random_uuid()::STRING`,
- )
-}
-
-func populateVersionSetting(ctx context.Context, r runner) error {
- if !r.codec.ForSystemTenant() {
- log.Fatalf(ctx, "populateVersionSetting can only run for the system tenant")
- }
-
- var v roachpb.Version
- if err := r.db.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error {
- return txn.GetProto(ctx, keys.BootstrapVersionKey, &v)
- }); err != nil {
- return err
- }
- if v == (roachpb.Version{}) {
- // The cluster was bootstrapped at v1.0 (or even earlier), so just use
- // the TestingBinaryMinSupportedVersion of the binary.
- v = clusterversion.TestingBinaryMinSupportedVersion
- }
-
- b, err := protoutil.Marshal(&clusterversion.ClusterVersion{Version: v})
- if err != nil {
- return errors.Wrap(err, "while marshaling version")
- }
-
- // Add a ON CONFLICT DO NOTHING to avoid changing an existing version.
- // Again, this can happen if the migration doesn't run to completion
- // (overwriting also seems reasonable, but what for).
- // We don't allow users to perform version changes until we have run
- // the insert below.
- if err := r.execAsRoot(
- ctx,
- "insert-setting",
- fmt.Sprintf(`INSERT INTO system.settings (name, value, "lastUpdated", "valueType") VALUES ('version', x'%x', now(), 'm') ON CONFLICT(name) DO NOTHING`, b),
- ); err != nil {
- return err
- }
-
- // Add the host cluster version override for all tenants. This override is
- // used by secondary tenants to observe the host cluster version number and
- // ensure that secondary tenants don't upgrade to a version beyond the host
- // cluster version. As mentioned above, don't retry on conflict.
- tenantID := tree.NewDInt(0) // Tenant ID 0 indicates that we're overriding the value for all tenants.
- if err := r.execAsRoot(
- ctx,
- "insert-setting",
- fmt.Sprintf(`INSERT INTO system.tenant_settings (tenant_id, name, value, "last_updated", "value_type") VALUES (%d, 'version', x'%x', now(), 'm') ON CONFLICT(tenant_id, name) DO NOTHING`, tenantID, b),
- ); err != nil {
- return err
- }
-
- return nil
-}
-
-func addRootUser(ctx context.Context, r runner) error {
- // Upsert the root user into the table. We intentionally override any existing entry.
- const upsertRootStmt = `
- UPSERT INTO system.users (username, "hashedPassword", "isRole", "user_id") VALUES ($1, '', false, 1)
- `
- return r.execAsRootWithRetry(ctx, "addRootUser", upsertRootStmt, username.RootUser)
-}
-
-func addAdminRole(ctx context.Context, r runner) error {
- // Upsert the admin role into the table. We intentionally override any existing entry.
- const upsertAdminStmt = `
- UPSERT INTO system.users (username, "hashedPassword", "isRole", "user_id") VALUES ($1, '', true, 2)
- `
- return r.execAsRootWithRetry(ctx, "addAdminRole", upsertAdminStmt, username.AdminRole)
-}
-
-func addRootToAdminRole(ctx context.Context, r runner) error {
- // Upsert the role membership into the table. We intentionally override any existing entry.
- const upsertAdminStmt = `
- UPSERT INTO system.role_members ("role", "member", "isAdmin") VALUES ($1, $2, true)
- `
- return r.execAsRootWithRetry(
- ctx, "addRootToAdminRole", upsertAdminStmt, username.AdminRole, username.RootUser)
-}
-
-func disallowPublicUserOrRole(ctx context.Context, r runner) error {
- // Check whether a user or role named "public" exists.
- const selectPublicStmt = `
- SELECT username, "isRole" from system.users WHERE username = $1
- `
-
- for retry := retry.Start(retry.Options{MaxRetries: 5}); retry.Next(); {
- row, err := r.sqlExecutor.QueryRowEx(
- ctx, "disallowPublicUserOrRole", nil, /* txn */
- sessiondata.InternalExecutorOverride{
- User: username.RootUserName(),
- },
- selectPublicStmt, username.PublicRole,
- )
- if err != nil {
- continue
- }
- if row == nil {
- // No such user.
- return nil
- }
-
- isRole, ok := tree.AsDBool(row[1])
- if !ok {
- log.Fatalf(ctx, "expected 'isRole' column of system.users to be of type bool, got %v", row)
- }
-
- if isRole {
- return fmt.Errorf(`found a role named %s which is now a reserved name. Please drop the role `+
- `(DROP ROLE %s) using a previous version of CockroachDB and try again`,
- username.PublicRole, username.PublicRole)
- }
- return fmt.Errorf(`found a user named %s which is now a reserved name. Please drop the role `+
- `(DROP USER %s) using a previous version of CockroachDB and try again`,
- username.PublicRole, username.PublicRole)
- }
- return nil
-}
-
-func createDefaultDbs(ctx context.Context, r runner) error {
- // Create the default databases. These are plain databases with
- // default permissions. Nothing special happens if they exist
- // already.
- const createDbStmt = `CREATE DATABASE IF NOT EXISTS "%s"`
-
- var err error
- for retry := retry.Start(retry.Options{MaxRetries: 5}); retry.Next(); {
- for _, dbName := range []string{catalogkeys.DefaultDatabaseName, catalogkeys.PgDatabaseName} {
- stmt := fmt.Sprintf(createDbStmt, dbName)
- err = r.execAsRoot(ctx, "create-default-DB", stmt)
- if err != nil {
- log.Warningf(ctx, "failed attempt to add database %q: %s", dbName, err)
- break
- }
- }
- if err == nil {
- break
- }
- }
- return err
-}
-
-func retireOldTsPurgeIntervalSettings(ctx context.Context, r runner) error {
- // We are going to deprecate `timeseries.storage.10s_resolution_ttl`
- // into `timeseries.storage.resolution_10s.ttl` if the latter is not
- // defined.
- //
- // Ditto for the `30m` resolution.
-
- // Copy 'timeseries.storage.10s_resolution_ttl' into
- // 'timeseries.storage.resolution_10s.ttl' if the former is defined
- // and the latter is not defined yet.
- //
- // We rely on the SELECT returning no row if the original setting
- // was not defined, and INSERT ON CONFLICT DO NOTHING to ignore the
- // insert if the new name was already set.
- if err := r.execAsRoot(ctx, "copy-setting", `
-INSERT INTO system.settings (name, value, "lastUpdated", "valueType")
- SELECT 'timeseries.storage.resolution_10s.ttl', value, "lastUpdated", "valueType"
- FROM system.settings WHERE name = 'timeseries.storage.10s_resolution_ttl'
-ON CONFLICT (name) DO NOTHING`,
- ); err != nil {
- return err
- }
-
- // Ditto 30m.
- if err := r.execAsRoot(ctx, "copy-setting", `
-INSERT INTO system.settings (name, value, "lastUpdated", "valueType")
- SELECT 'timeseries.storage.resolution_30m.ttl', value, "lastUpdated", "valueType"
- FROM system.settings WHERE name = 'timeseries.storage.30m_resolution_ttl'
-ON CONFLICT (name) DO NOTHING`,
- ); err != nil {
- return err
- }
-
- return nil
-}
-
-func updateSystemLocationData(ctx context.Context, r runner) error {
- // See if the system.locations table already has data in it.
- // If so, we don't want to do anything.
- row, err := r.sqlExecutor.QueryRowEx(ctx, "update-system-locations",
- nil, /* txn */
- sessiondata.InternalExecutorOverride{User: username.RootUserName()},
- `SELECT count(*) FROM system.locations`)
- if err != nil {
- return err
- }
- if row == nil {
- return errors.New("failed to update system locations")
- }
- count := int(tree.MustBeDInt(row[0]))
- if count != 0 {
- return nil
- }
-
- for _, loc := range roachpb.DefaultLocationInformation {
- stmt := `UPSERT INTO system.locations VALUES ($1, $2, $3, $4)`
- tier := loc.Locality.Tiers[0]
- if err := r.execAsRoot(ctx, "update-system-locations",
- stmt, tier.Key, tier.Value, loc.Latitude, loc.Longitude,
- ); err != nil {
- return err
- }
- }
- return nil
-}
diff --git a/pkg/startupmigrations/migrations_test.go b/pkg/startupmigrations/migrations_test.go
deleted file mode 100644
index 7e7b02a8fe4d..000000000000
--- a/pkg/startupmigrations/migrations_test.go
+++ /dev/null
@@ -1,714 +0,0 @@
-// Copyright 2016 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 startupmigrations
-
-import (
- "bytes"
- "context"
- "fmt"
- "math/rand"
- "strings"
- "testing"
- "time"
-
- "github.com/cockroachdb/cockroach/pkg/base"
- "github.com/cockroachdb/cockroach/pkg/cli/exit"
- "github.com/cockroachdb/cockroach/pkg/keys"
- "github.com/cockroachdb/cockroach/pkg/kv"
- "github.com/cockroachdb/cockroach/pkg/roachpb"
- "github.com/cockroachdb/cockroach/pkg/security/username"
- "github.com/cockroachdb/cockroach/pkg/sql"
- "github.com/cockroachdb/cockroach/pkg/startupmigrations/leasemanager"
- "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/util/leaktest"
- "github.com/cockroachdb/cockroach/pkg/util/log"
- "github.com/cockroachdb/cockroach/pkg/util/randutil"
- "github.com/cockroachdb/cockroach/pkg/util/stop"
- "github.com/cockroachdb/errors"
- "github.com/stretchr/testify/require"
-)
-
-var (
- noopMigration1 = migrationDescriptor{
- name: "noop 1",
- workFn: func(_ context.Context, _ runner) error { return nil },
- }
- noopMigration2 = migrationDescriptor{
- name: "noop 2",
- workFn: func(_ context.Context, _ runner) error { return nil },
- }
- errorMigration = migrationDescriptor{
- name: "error",
- workFn: func(_ context.Context, _ runner) error { return errors.New("errorMigration") },
- }
- panicMigration = migrationDescriptor{
- name: "panic",
- workFn: func(_ context.Context, _ runner) error { panic("panicMigration") },
- }
-)
-
-type fakeLeaseManager struct {
- extendErr error
- releaseErr error
- leaseTimeRemaining time.Duration
-}
-
-func (f *fakeLeaseManager) AcquireLease(
- ctx context.Context, key roachpb.Key,
-) (*leasemanager.Lease, error) {
- return &leasemanager.Lease{}, nil
-}
-
-func (f *fakeLeaseManager) ExtendLease(ctx context.Context, l *leasemanager.Lease) error {
- return f.extendErr
-}
-
-func (f *fakeLeaseManager) ReleaseLease(ctx context.Context, l *leasemanager.Lease) error {
- return f.releaseErr
-}
-
-func (f *fakeLeaseManager) TimeRemaining(l *leasemanager.Lease) time.Duration {
- // Default to a reasonable amount of time left if the field wasn't set.
- if f.leaseTimeRemaining == 0 {
- return leaseRefreshInterval * 2
- }
- return f.leaseTimeRemaining
-}
-
-type fakeDB struct {
- codec keys.SQLCodec
- rand *rand.Rand
- kvs map[string][]byte
- scanErr error
- putErr error
-}
-
-func newFakeDB(codec keys.SQLCodec) *fakeDB {
- r, _ := randutil.NewTestRand()
- return &fakeDB{
- codec: codec,
- rand: r,
- kvs: make(map[string][]byte),
- }
-}
-
-// ReadCommittedScan never returns any data.
-func (f *fakeDB) ReadCommittedScan(
- ctx context.Context, begin, end interface{}, maxRows int64,
-) ([]kv.KeyValue, error) {
- // Sometimes return the data, sometimes return nothing.
- if f.rand.Float64() < .9 {
- return f.Scan(ctx, begin, end, maxRows)
- }
- return nil, nil
-}
-
-func (f *fakeDB) Scan(
- ctx context.Context, begin, end interface{}, maxRows int64,
-) ([]kv.KeyValue, error) {
- if f.scanErr != nil {
- return nil, f.scanErr
- }
- min := f.codec.StartupMigrationKeyPrefix()
- max := min.PrefixEnd()
- if !bytes.Equal(begin.(roachpb.Key), min) {
- return nil, errors.Errorf("expected begin key %q, got %q", min, begin)
- }
- if !bytes.Equal(end.(roachpb.Key), max) {
- return nil, errors.Errorf("expected end key %q, got %q", max, end)
- }
- var results []kv.KeyValue
- for k, v := range f.kvs {
- results = append(results, kv.KeyValue{
- Key: []byte(k),
- Value: &roachpb.Value{RawBytes: v},
- })
- }
- return results, nil
-}
-
-func (f *fakeDB) Get(ctx context.Context, key interface{}) (kv.KeyValue, error) {
- return kv.KeyValue{}, errors.New("unimplemented")
-}
-
-func (f *fakeDB) Put(ctx context.Context, key, value interface{}) error {
- if f.putErr != nil {
- return f.putErr
- }
- if f.kvs != nil {
- f.kvs[string(key.(roachpb.Key))] = []byte(value.(string))
- }
- return nil
-}
-
-func (f *fakeDB) Txn(context.Context, func(context.Context, *kv.Txn) error) error {
- return errors.New("unimplemented")
-}
-
-func TestEnsureMigrations(t *testing.T) {
- defer leaktest.AfterTest(t)()
- codec := keys.SystemSQLCodec
- db := newFakeDB(codec)
- mgr := Manager{
- stopper: stop.NewStopper(),
- leaseManager: &fakeLeaseManager{},
- db: db,
- codec: codec,
- }
- defer mgr.stopper.Stop(context.Background())
-
- fnGotCalled := false
- fnGotCalledDescriptor := migrationDescriptor{
- name: "got-called-verifier",
- workFn: func(context.Context, runner) error {
- fnGotCalled = true
- return nil
- },
- }
- testCases := []struct {
- preCompleted []migrationDescriptor
- migrations []migrationDescriptor
- expectedErr string
- }{
- {
- nil,
- nil,
- "",
- },
- {
- nil,
- []migrationDescriptor{noopMigration1},
- "",
- },
- {
- []migrationDescriptor{noopMigration1},
- []migrationDescriptor{noopMigration1},
- "",
- },
- {
- []migrationDescriptor{},
- []migrationDescriptor{noopMigration1, noopMigration2},
- "",
- },
- {
- []migrationDescriptor{noopMigration1},
- []migrationDescriptor{noopMigration1, noopMigration2},
- "",
- },
- {
- []migrationDescriptor{noopMigration1, noopMigration2, panicMigration},
- []migrationDescriptor{noopMigration1, noopMigration2, panicMigration},
- "",
- },
- {
- []migrationDescriptor{noopMigration1, noopMigration2},
- []migrationDescriptor{noopMigration1, noopMigration2, fnGotCalledDescriptor},
- "",
- },
- {
- []migrationDescriptor{noopMigration1, noopMigration2},
- []migrationDescriptor{noopMigration1, noopMigration2, errorMigration},
- fmt.Sprintf("failed to run migration %q", errorMigration.name),
- },
- }
-
- defer func(prev []migrationDescriptor) { backwardCompatibleMigrations = prev }(backwardCompatibleMigrations)
-
- for _, tc := range testCases {
- t.Run("", func(t *testing.T) {
- db.kvs = make(map[string][]byte)
- for _, name := range tc.preCompleted {
- db.kvs[string(migrationKey(codec, name))] = []byte{}
- }
- backwardCompatibleMigrations = tc.migrations
-
- err := mgr.EnsureMigrations(context.Background(), roachpb.Version{} /* bootstrapVersion */)
- if !testutils.IsError(err, tc.expectedErr) {
- t.Errorf("expected error %q, got error %v", tc.expectedErr, err)
- }
- if err != nil {
- return
- }
-
- for _, migration := range tc.migrations {
- if _, ok := db.kvs[string(migrationKey(codec, migration))]; !ok {
- t.Errorf("expected key %s to be written, but it wasn't", migrationKey(codec, migration))
- }
- }
- if len(db.kvs) != len(tc.migrations) {
- t.Errorf("expected %d key to be written, but %d were",
- len(tc.migrations), len(db.kvs))
- }
- })
- }
- if !fnGotCalled {
- t.Errorf("expected fnGotCalledDescriptor to be run by the migration coordinator, but it wasn't")
- }
-}
-
-func TestSkipMigrationsIncludedInBootstrap(t *testing.T) {
- defer leaktest.AfterTest(t)()
- ctx := context.Background()
- codec := keys.SystemSQLCodec
- db := newFakeDB(codec)
- mgr := Manager{
- stopper: stop.NewStopper(),
- leaseManager: &fakeLeaseManager{},
- db: db,
- codec: codec,
- }
- defer mgr.stopper.Stop(ctx)
- defer func(prev []migrationDescriptor) {
- backwardCompatibleMigrations = prev
- }(backwardCompatibleMigrations)
-
- v := roachpb.MustParseVersion("19.1")
- fnGotCalled := false
- backwardCompatibleMigrations = []migrationDescriptor{{
- name: "got-called-verifier",
- includedInBootstrap: v,
- workFn: func(context.Context, runner) error {
- fnGotCalled = true
- return nil
- },
- }}
- // If the cluster has been bootstrapped at an old version, the migration should run.
- require.NoError(t, mgr.EnsureMigrations(ctx, roachpb.Version{} /* bootstrapVersion */))
- require.True(t, fnGotCalled)
- fnGotCalled = false
- // If the cluster has been bootstrapped at a new version, the migration should
- // not run.
- require.NoError(t, mgr.EnsureMigrations(ctx, v /* bootstrapVersion */))
- require.False(t, fnGotCalled)
-}
-
-func TestClusterWideMigrationOnlyRunBySystemTenant(t *testing.T) {
- defer leaktest.AfterTest(t)()
- testutils.RunTrueAndFalse(t, "system tenant", func(t *testing.T, systemTenant bool) {
- var codec keys.SQLCodec
- if systemTenant {
- codec = keys.SystemSQLCodec
- } else {
- codec = keys.MakeSQLCodec(roachpb.MustMakeTenantID(5))
- }
-
- ctx := context.Background()
- db := newFakeDB(codec)
- mgr := Manager{
- stopper: stop.NewStopper(),
- leaseManager: &fakeLeaseManager{},
- db: db,
- codec: codec,
- }
- defer mgr.stopper.Stop(ctx)
- defer func(prev []migrationDescriptor) {
- backwardCompatibleMigrations = prev
- }(backwardCompatibleMigrations)
-
- fnGotCalled := false
- backwardCompatibleMigrations = []migrationDescriptor{{
- name: "got-called-verifier",
- clusterWide: true,
- workFn: func(context.Context, runner) error {
- fnGotCalled = true
- return nil
- },
- }}
- // The migration should only be run by the system tenant.
- require.NoError(t, mgr.EnsureMigrations(ctx, roachpb.Version{} /* bootstrapVersion */))
- require.Equal(t, systemTenant, fnGotCalled)
- })
-}
-
-func TestDBErrors(t *testing.T) {
- defer leaktest.AfterTest(t)()
- codec := keys.SystemSQLCodec
- db := newFakeDB(codec)
- mgr := Manager{
- stopper: stop.NewStopper(),
- leaseManager: &fakeLeaseManager{},
- db: db,
- codec: codec,
- }
- defer mgr.stopper.Stop(context.Background())
-
- migration := noopMigration1
- defer func(prev []migrationDescriptor) { backwardCompatibleMigrations = prev }(backwardCompatibleMigrations)
- backwardCompatibleMigrations = []migrationDescriptor{migration}
- testCases := []struct {
- scanErr error
- putErr error
- expectedErr string
- }{
- {
- nil,
- nil,
- "",
- },
- {
- fmt.Errorf("context deadline exceeded"),
- nil,
- "failed to get list of completed migrations.*context deadline exceeded",
- },
- {
- nil,
- fmt.Errorf("context deadline exceeded"),
- "failed to persist record of completing migration.*context deadline exceeded",
- },
- }
- for _, tc := range testCases {
- t.Run("", func(t *testing.T) {
- db.scanErr = tc.scanErr
- db.putErr = tc.putErr
- db.kvs = make(map[string][]byte)
- err := mgr.EnsureMigrations(context.Background(), roachpb.Version{} /* bootstrapVersion */)
- if !testutils.IsError(err, tc.expectedErr) {
- t.Errorf("expected error %q, got error %v", tc.expectedErr, err)
- }
- if err != nil {
- return
- }
- if _, ok := db.kvs[string(migrationKey(codec, migration))]; !ok {
- t.Errorf("expected key %s to be written, but it wasn't", migrationKey(codec, migration))
- }
- if len(db.kvs) != len(backwardCompatibleMigrations) {
- t.Errorf("expected %d key to be written, but %d were",
- len(backwardCompatibleMigrations), len(db.kvs))
- }
- })
- }
-}
-
-// ExtendLease and ReleaseLease errors should not, by themselves, cause the
-// migration process to fail. Not being able to acquire the lease should, but
-// we don't test that here due to the added code that would be needed to change
-// its retry settings to allow for testing it in a reasonable amount of time.
-func TestLeaseErrors(t *testing.T) {
- defer leaktest.AfterTest(t)()
- codec := keys.SystemSQLCodec
- db := newFakeDB(codec)
- mgr := Manager{
- stopper: stop.NewStopper(),
- leaseManager: &fakeLeaseManager{
- extendErr: fmt.Errorf("context deadline exceeded"),
- releaseErr: fmt.Errorf("context deadline exceeded"),
- },
- db: db,
- codec: codec,
- }
- defer mgr.stopper.Stop(context.Background())
-
- migration := noopMigration1
- defer func(prev []migrationDescriptor) { backwardCompatibleMigrations = prev }(backwardCompatibleMigrations)
- backwardCompatibleMigrations = []migrationDescriptor{migration}
- if err := mgr.EnsureMigrations(context.Background(), roachpb.Version{} /* bootstrapVersion */); err != nil {
- t.Error(err)
- }
- if _, ok := db.kvs[string(migrationKey(codec, migration))]; !ok {
- t.Errorf("expected key %s to be written, but it wasn't", migrationKey(codec, migration))
- }
- if len(db.kvs) != len(backwardCompatibleMigrations) {
- t.Errorf("expected %d key to be written, but %d were",
- len(backwardCompatibleMigrations), len(db.kvs))
- }
-}
-
-// The lease not having enough time left on it to finish migrations should
-// cause the process to exit via a call to log.Fatal.
-func TestLeaseExpiration(t *testing.T) {
- defer leaktest.AfterTest(t)()
- codec := keys.SystemSQLCodec
- db := newFakeDB(codec)
- mgr := Manager{
- stopper: stop.NewStopper(),
- leaseManager: &fakeLeaseManager{leaseTimeRemaining: time.Nanosecond},
- db: db,
- codec: codec,
- }
- defer mgr.stopper.Stop(context.Background())
-
- oldLeaseRefreshInterval := leaseRefreshInterval
- leaseRefreshInterval = time.Microsecond
- defer func() { leaseRefreshInterval = oldLeaseRefreshInterval }()
-
- exitCalled := make(chan bool)
- log.SetExitFunc(true /* hideStack */, func(exit.Code) { exitCalled <- true })
- defer log.ResetExitFunc()
- // Disable stack traces to make the test output in teamcity less deceiving.
- defer log.DisableTracebacks()()
-
- waitForExitMigration := migrationDescriptor{
- name: "wait for exit to be called",
- workFn: func(context.Context, runner) error {
- select {
- case <-exitCalled:
- return nil
- case <-time.After(10 * time.Second):
- return fmt.Errorf("timed out waiting for exit to be called")
- }
- },
- }
- defer func(prev []migrationDescriptor) { backwardCompatibleMigrations = prev }(backwardCompatibleMigrations)
- backwardCompatibleMigrations = []migrationDescriptor{waitForExitMigration}
- if err := mgr.EnsureMigrations(context.Background(), roachpb.Version{} /* bootstrapVersion */); err != nil {
- t.Error(err)
- }
-}
-
-// migrationTest assists in testing the effects of a migration. It provides
-// methods to edit the list of migrations run at server startup, start a test
-// server, and run an individual migration. The test server's KV and SQL
-// interfaces are intended to be accessed directly to verify the effect of the
-// migration. Any mutations to the list of migrations run at server startup are
-// reverted at the end of the test.
-type migrationTest struct {
- oldMigrations []migrationDescriptor
- server serverutils.TestServerInterface
- sqlDB *sqlutils.SQLRunner
- kvDB *kv.DB
- memMetrics *sql.MemoryMetrics
-}
-
-// makeMigrationTest creates a new migrationTest.
-//
-// The caller is responsible for calling the test's close method at the end of
-// the test.
-func makeMigrationTest(ctx context.Context, t testing.TB) migrationTest {
- t.Helper()
-
- oldMigrations := append([]migrationDescriptor(nil), backwardCompatibleMigrations...)
- memMetrics := sql.MakeMemMetrics("migration-test-internal", time.Minute)
- return migrationTest{
- oldMigrations: oldMigrations,
- memMetrics: &memMetrics,
- }
-}
-
-// pop removes the migration whose name starts with namePrefix from the list of
-// migrations run at server startup. It fails the test if the number of
-// migrations that match namePrefix is not exactly one.
-//
-// You must not call pop after calling start.
-func (mt *migrationTest) pop(t testing.TB, namePrefix string) migrationDescriptor {
- t.Helper()
-
- if mt.server != nil {
- t.Fatal("migrationTest.pop must be called before mt.start")
- }
-
- var migration migrationDescriptor
- var newMigrations []migrationDescriptor
- for _, m := range backwardCompatibleMigrations {
- if strings.HasPrefix(m.name, namePrefix) {
- migration = m
- continue
- }
- newMigrations = append(newMigrations, m)
- }
- if n := len(backwardCompatibleMigrations) - len(newMigrations); n != 1 {
- t.Fatalf("expected prefix %q to match exactly one migration, but matched %d", namePrefix, n)
- }
- backwardCompatibleMigrations = newMigrations
- return migration
-}
-
-// start starts a test server with the given serverArgs.
-func (mt *migrationTest) start(t testing.TB, serverArgs base.TestServerArgs) {
- server, sqlDB, kvDB := serverutils.StartServer(t, serverArgs)
- mt.server = server
- mt.sqlDB = sqlutils.MakeSQLRunner(sqlDB)
- mt.kvDB = kvDB
-}
-
-// runMigration triggers a manual run of migration. It does not mark migration
-// as completed, so subsequent calls will cause migration to be re-executed.
-// This is useful for verifying idempotency.
-//
-// You must call start before calling runMigration.
-func (mt *migrationTest) runMigration(ctx context.Context, m migrationDescriptor) error {
- if m.workFn == nil {
- // Migration has been baked in. Ignore it.
- return nil
- }
- return m.workFn(ctx, runner{
- settings: mt.server.ClusterSettings(),
- codec: keys.SystemSQLCodec,
- db: dbAdapter{DB: mt.kvDB},
- sqlExecutor: mt.server.InternalExecutor().(*sql.InternalExecutor),
- })
-}
-
-// close stops the test server and restores package-global state.
-func (mt *migrationTest) close(ctx context.Context) {
- if mt.server != nil {
- mt.server.Stopper().Stop(ctx)
- }
- backwardCompatibleMigrations = mt.oldMigrations
-}
-
-func TestAdminUserExists(t *testing.T) {
- defer leaktest.AfterTest(t)()
- ctx := context.Background()
-
- mt := makeMigrationTest(ctx, t)
- defer mt.close(ctx)
-
- migration := mt.pop(t, "add system.users isRole column and create admin role")
- mt.start(t, base.TestServerArgs{})
-
- // Create a user named "admin". We have to do a manual insert as "CREATE USER"
- // knows about "isRole", but the migration hasn't run yet.
- mt.sqlDB.Exec(t, `INSERT INTO system.users (username, "hashedPassword", user_id) VALUES ($1, '', $2)`,
- username.AdminRole, username.AdminRoleID)
-
- // The revised migration in v2.1 upserts the admin user, so this should succeed.
- if err := mt.runMigration(ctx, migration); err != nil {
- t.Errorf("expected success, got %q", err)
- }
-}
-
-func TestPublicRoleExists(t *testing.T) {
- defer leaktest.AfterTest(t)()
- ctx := context.Background()
-
- mt := makeMigrationTest(ctx, t)
- defer mt.close(ctx)
-
- migration := mt.pop(t, "disallow public user or role name")
- mt.start(t, base.TestServerArgs{})
-
- // Create a user (we check for user or role) named "public".
- // We have to do a manual insert as "CREATE USER" knows to disallow "public".
- mt.sqlDB.Exec(t, `INSERT INTO system.users (username, "hashedPassword", "isRole", user_id) VALUES ($1, '', false, $2)`,
- username.PublicRole, username.PublicRoleID)
-
- e := `found a user named public which is now a reserved name.`
- // The revised migration in v2.1 upserts the admin user, so this should succeed.
- if err := mt.runMigration(ctx, migration); !testutils.IsError(err, e) {
- t.Errorf("expected error %q got %q", e, err)
- }
-
- // Now try with a role instead of a user.
- mt.sqlDB.Exec(t, `DELETE FROM system.users WHERE username = $1`, username.PublicRole)
- mt.sqlDB.Exec(t, `INSERT INTO system.users (username, "hashedPassword", "isRole", user_id) VALUES ($1, '', true, $2)`,
- username.PublicRole, username.PublicRoleID)
-
- e = `found a role named public which is now a reserved name.`
- // The revised migration in v2.1 upserts the admin user, so this should succeed.
- if err := mt.runMigration(ctx, migration); !testutils.IsError(err, e) {
- t.Errorf("expected error %q got %q", e, err)
- }
-
- // Drop it and try again.
- mt.sqlDB.Exec(t, `DELETE FROM system.users WHERE username = $1`, username.PublicRole)
- if err := mt.runMigration(ctx, migration); err != nil {
- t.Errorf("expected success, got %q", err)
- }
-}
-
-func TestReplayMigrations(t *testing.T) {
- defer leaktest.AfterTest(t)()
- ctx := context.Background()
-
- mt := makeMigrationTest(ctx, t)
- defer mt.close(ctx)
-
- mt.start(t, base.TestServerArgs{})
-
- // Test all migrations again. Starting the server did the first round.
- for _, m := range backwardCompatibleMigrations {
- if err := mt.runMigration(ctx, m); err != nil {
- t.Error(err)
- }
- }
-}
-
-func TestExpectedInitialRangeCount(t *testing.T) {
- defer leaktest.AfterTest(t)()
-
- ctx := context.Background()
- s, sqlDB, kvDB := serverutils.StartServer(t, base.TestServerArgs{})
- defer s.Stopper().Stop(ctx)
-
- testutils.SucceedsSoon(t, func() error {
- lastMigration := backwardCompatibleMigrations[len(backwardCompatibleMigrations)-1]
- if _, err := kvDB.Get(ctx, migrationKey(keys.SystemSQLCodec, lastMigration)); err != nil {
- return errors.New("last migration has not completed")
- }
-
- sysCfg := s.SystemConfigProvider().GetSystemConfig()
- if sysCfg == nil {
- return errors.New("gossipped system config not available")
- }
-
- rows, err := sqlDB.Query(`SELECT range_id, start_key, end_key FROM crdb_internal.ranges`)
- if err != nil {
- return err
- }
- defer rows.Close()
- nranges := 0
- for rows.Next() {
- var rangeID int
- var startKey, endKey []byte
- if err := rows.Scan(&rangeID, &startKey, &endKey); err != nil {
- return err
- }
- if sysCfg.NeedsSplit(ctx, startKey, endKey) {
- return fmt.Errorf("range %d needs split", rangeID)
- }
- nranges++
- }
- if rows.Err() != nil {
- return err
- }
-
- expectedRanges, err := s.ExpectedInitialRangeCount()
- if err != nil {
- return err
- }
- if expectedRanges != nranges {
- return fmt.Errorf("expected %d ranges but got %d", expectedRanges, nranges)
- }
-
- return nil
- })
-}
-
-func TestUpdateSystemLocationData(t *testing.T) {
- defer leaktest.AfterTest(t)()
- ctx := context.Background()
-
- mt := makeMigrationTest(ctx, t)
- defer mt.close(ctx)
-
- migration := mt.pop(t, "update system.locations with default location data")
- mt.start(t, base.TestServerArgs{})
-
- // Check that we don't have any data in the system.locations table without the migration.
- var count int
- mt.sqlDB.QueryRow(t, `SELECT count(*) FROM system.locations`).Scan(&count)
- if count != 0 {
- t.Fatalf("Exected to find 0 rows in system.locations. Found %d instead", count)
- }
-
- // Run the migration to insert locations.
- if err := mt.runMigration(ctx, migration); err != nil {
- t.Errorf("expected success, got %q", err)
- }
-
- // Check that we have all of the expected locations.
- mt.sqlDB.QueryRow(t, `SELECT count(*) FROM system.locations`).Scan(&count)
- if count != len(roachpb.DefaultLocationInformation) {
- t.Fatalf("Exected to find 0 rows in system.locations. Found %d instead", count)
- }
-}
diff --git a/pkg/upgrade/BUILD.bazel b/pkg/upgrade/BUILD.bazel
index 4d77c2971237..b0ad207cbfea 100644
--- a/pkg/upgrade/BUILD.bazel
+++ b/pkg/upgrade/BUILD.bazel
@@ -7,13 +7,11 @@ go_library(
"helpers.go",
"system_upgrade.go",
"tenant_upgrade.go",
- "testing_knobs.go",
"upgrade.go",
],
importpath = "github.com/cockroachdb/cockroach/pkg/upgrade",
visibility = ["//visibility:public"],
deps = [
- "//pkg/base",
"//pkg/clusterversion",
"//pkg/jobs",
"//pkg/keys",
@@ -28,6 +26,7 @@ go_library(
"//pkg/sql/catalog/resolver",
"//pkg/sql/sessiondata",
"//pkg/sql/sqlutil",
+ "//pkg/upgrade/upgradebase",
"//pkg/util/log",
"//pkg/util/stop",
"@com_github_cockroachdb_logtags//:logtags",
diff --git a/pkg/upgrade/migrationstable/BUILD.bazel b/pkg/upgrade/migrationstable/BUILD.bazel
new file mode 100644
index 000000000000..65a6d289b4c4
--- /dev/null
+++ b/pkg/upgrade/migrationstable/BUILD.bazel
@@ -0,0 +1,20 @@
+load("//build/bazelutil/unused_checker:unused.bzl", "get_x_data")
+load("@io_bazel_rules_go//go:def.bzl", "go_library")
+
+go_library(
+ name = "migrationstable",
+ srcs = ["migrations_table.go"],
+ importpath = "github.com/cockroachdb/cockroach/pkg/upgrade/migrationstable",
+ visibility = ["//visibility:public"],
+ deps = [
+ "//pkg/kv",
+ "//pkg/roachpb",
+ "//pkg/sql/sem/tree",
+ "//pkg/sql/sessiondata",
+ "//pkg/sql/sqlutil",
+ "//pkg/util/timeutil",
+ "@com_github_cockroachdb_errors//:errors",
+ ],
+)
+
+get_x_data(name = "get_x_data")
diff --git a/pkg/upgrade/migrationstable/migrations_table.go b/pkg/upgrade/migrationstable/migrations_table.go
new file mode 100644
index 000000000000..2fe2584b33e0
--- /dev/null
+++ b/pkg/upgrade/migrationstable/migrations_table.go
@@ -0,0 +1,112 @@
+// Copyright 2022 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 migrationstable
+
+import (
+ "context"
+ "fmt"
+
+ "github.com/cockroachdb/cockroach/pkg/kv"
+ "github.com/cockroachdb/cockroach/pkg/roachpb"
+ "github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
+ "github.com/cockroachdb/cockroach/pkg/sql/sessiondata"
+ "github.com/cockroachdb/cockroach/pkg/sql/sqlutil"
+ "github.com/cockroachdb/cockroach/pkg/util/timeutil"
+ "github.com/cockroachdb/errors"
+)
+
+// MarkMigrationCompleted inserts a row in system.migrations.
+func MarkMigrationCompleted(
+ ctx context.Context, ie sqlutil.InternalExecutor, v roachpb.Version,
+) error {
+ _, err := ie.ExecEx(
+ ctx,
+ "migration-job-mark-job-succeeded",
+ nil, /* txn */
+ sessiondata.NodeUserSessionDataOverride,
+ `
+INSERT
+ INTO system.migrations
+ (
+ major,
+ minor,
+ patch,
+ internal,
+ completed_at
+ )
+VALUES ($1, $2, $3, $4, $5)`,
+ v.Major,
+ v.Minor,
+ v.Patch,
+ v.Internal,
+ timeutil.Now())
+ return err
+}
+
+type StaleReadOpt bool
+
+const (
+ StaleRead StaleReadOpt = true
+ ConsistentRead = false
+)
+
+// CheckIfMigrationCompleted queries the system.upgrades table to determine
+// if the upgrade associated with this version has already been completed.
+// The txn may be nil, in which case the check will be run in its own
+// transaction.
+//
+// staleOpt dictates whether the check will run a consistent read or a stale,
+// follower read. If txn is not nil, only ConsistentRead can be specified. If
+// enterpriseEnabled is set and the StaleRead option is passed, then AS OF
+// SYSTEM TIME with_max_staleness('1h') is used for the query.
+func CheckIfMigrationCompleted(
+ ctx context.Context,
+ v roachpb.Version,
+ txn *kv.Txn,
+ ie sqlutil.InternalExecutor,
+ enterpriseEnabled bool,
+ staleOpt StaleReadOpt,
+) (alreadyCompleted bool, _ error) {
+ if txn != nil && staleOpt == StaleRead {
+ return false, errors.AssertionFailedf(
+ "CheckIfMigrationCompleted: cannot ask for stale read when running in a txn")
+ }
+ queryFormat := `
+SELECT count(*)
+ FROM system.migrations
+ %s
+ WHERE major = $1
+ AND minor = $2
+ AND patch = $3
+ AND internal = $4
+`
+ var query string
+ if staleOpt == StaleRead && enterpriseEnabled {
+ query = fmt.Sprintf(queryFormat, "AS OF SYSTEM TIME with_max_staleness('1h')")
+ } else {
+ query = fmt.Sprintf(queryFormat, "")
+ }
+
+ row, err := ie.QueryRow(
+ ctx,
+ "migration-job-find-already-completed",
+ txn,
+ query,
+ v.Major,
+ v.Minor,
+ v.Patch,
+ v.Internal)
+ if err != nil {
+ return false, err
+ }
+ count := *row[0].(*tree.DInt)
+ return count != 0, nil
+}
diff --git a/pkg/upgrade/system_upgrade.go b/pkg/upgrade/system_upgrade.go
index 3f41ba2fff49..3c8ba49c4b3f 100644
--- a/pkg/upgrade/system_upgrade.go
+++ b/pkg/upgrade/system_upgrade.go
@@ -19,6 +19,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/kv/kvclient/kvcoord"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/server/serverpb"
+ "github.com/cockroachdb/cockroach/pkg/sql/sqlutil"
"github.com/cockroachdb/cockroach/pkg/util/stop"
"github.com/cockroachdb/logtags"
)
@@ -105,10 +106,11 @@ type Cluster interface {
// SystemDeps are the dependencies of upgrades which perform actions at the
// KV layer on behalf of the system tenant.
type SystemDeps struct {
- Cluster Cluster
- DB *kv.DB
- DistSender *kvcoord.DistSender
- Stopper *stop.Stopper
+ Cluster Cluster
+ DB *kv.DB
+ InternalExecutor sqlutil.InternalExecutor
+ DistSender *kvcoord.DistSender
+ Stopper *stop.Stopper
}
// SystemUpgrade is an implementation of Upgrade for system-level
@@ -134,6 +136,22 @@ func NewSystemUpgrade(description string, v roachpb.Version, fn SystemUpgradeFun
}
}
+// NewPermanentSystemUpgrade constructs a SystemUpgrade that is marked as
+// "permanent": an upgrade that will run regardless of the cluster's bootstrap
+// version.
+func NewPermanentSystemUpgrade(
+ description string, v roachpb.Version, fn SystemUpgradeFunc,
+) *SystemUpgrade {
+ return &SystemUpgrade{
+ upgrade: upgrade{
+ description: description,
+ v: v,
+ permanent: true,
+ },
+ fn: fn,
+ }
+}
+
// Run kickstarts the actual upgrade process for system-level upgrades.
func (m *SystemUpgrade) Run(ctx context.Context, v roachpb.Version, d SystemDeps) error {
ctx = logtags.AddTag(ctx, fmt.Sprintf("upgrade=%s", v), nil)
diff --git a/pkg/upgrade/tenant_upgrade.go b/pkg/upgrade/tenant_upgrade.go
index 72816b5a4c0f..f6293a75415f 100644
--- a/pkg/upgrade/tenant_upgrade.go
+++ b/pkg/upgrade/tenant_upgrade.go
@@ -26,6 +26,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/catalog/resolver"
"github.com/cockroachdb/cockroach/pkg/sql/sessiondata"
"github.com/cockroachdb/cockroach/pkg/sql/sqlutil"
+ "github.com/cockroachdb/cockroach/pkg/upgrade/upgradebase"
"github.com/cockroachdb/logtags"
)
@@ -47,7 +48,7 @@ type TenantDeps struct {
Default roachpb.SpanConfig
}
- TestingKnobs *TestingKnobs
+ TestingKnobs *upgradebase.TestingKnobs
SchemaResolverConstructor func( // A constructor that returns a schema resolver for `descriptors` in `currDb`.
txn *kv.Txn, descriptors *descs.Collection, currDb string,
) (resolver.SchemaResolver, func(), error)
@@ -75,11 +76,17 @@ type PreconditionFunc func(context.Context, clusterversion.ClusterVersion, Tenan
// sql. It includes the system tenant.
type TenantUpgrade struct {
upgrade
- fn TenantUpgradeFunc
+ fn TenantUpgradeFunc
+ // precondition is executed before fn. Note that permanent upgrades (see
+ // upgrade.permanent) cannot have preconditions.
precondition PreconditionFunc
}
-var _ Upgrade = (*TenantUpgrade)(nil)
+var _ upgradebase.Upgrade = (*TenantUpgrade)(nil)
+
+// NoPrecondition can be used with NewTenantUpgrade to signify that the
+// respective upgrade does not need any preconditions checked.
+var NoPrecondition PreconditionFunc = nil
// NewTenantUpgrade constructs a TenantUpgrade.
func NewTenantUpgrade(
@@ -89,6 +96,7 @@ func NewTenantUpgrade(
upgrade: upgrade{
description: description,
v: v,
+ permanent: false,
},
fn: fn,
precondition: precondition,
@@ -96,6 +104,23 @@ func NewTenantUpgrade(
return m
}
+// NewPermanentTenantUpgrade constructs a TenantUpgrade marked as "permanent":
+// an upgrade that will run regardless of the cluster's bootstrap version.
+func NewPermanentTenantUpgrade(
+ description string, v roachpb.Version, fn TenantUpgradeFunc,
+) *TenantUpgrade {
+ m := &TenantUpgrade{
+ upgrade: upgrade{
+ description: description,
+ v: v,
+ permanent: true,
+ },
+ fn: fn,
+ precondition: nil,
+ }
+ return m
+}
+
// Run kick-starts the actual upgrade process for tenant-level upgrades.
func (m *TenantUpgrade) Run(ctx context.Context, v roachpb.Version, d TenantDeps) error {
ctx = logtags.AddTag(ctx, fmt.Sprintf("upgrade=%s", v), nil)
@@ -108,5 +133,8 @@ func (m *TenantUpgrade) Precondition(
ctx context.Context, cv clusterversion.ClusterVersion, d TenantDeps,
) error {
ctx = logtags.AddTag(ctx, fmt.Sprintf("upgrade=%s,precondition", cv), nil)
- return m.precondition(ctx, cv, d)
+ if m.precondition != nil {
+ return m.precondition(ctx, cv, d)
+ }
+ return nil
}
diff --git a/pkg/upgrade/upgrade.go b/pkg/upgrade/upgrade.go
index 4b57286271b7..f6ae01aa6ed0 100644
--- a/pkg/upgrade/upgrade.go
+++ b/pkg/upgrade/upgrade.go
@@ -25,54 +25,16 @@ import (
"fmt"
"github.com/cockroachdb/cockroach/pkg/roachpb"
+ "github.com/cockroachdb/cockroach/pkg/upgrade/upgradebase"
)
-// Upgrade defines a program to be executed once every node in the cluster is
-// (a) running a specific binary version, and (b) has completed all prior
-// upgrades. Note that there are two types of upgrades, a SystemUpgrade
-// and a TenantUpgrade. A SystemUpgrade only runs on the system tenant and
-// is used to migrate state at the KV layer. A TenantUpgrade runs on all
-// tenants (including the system tenant) and should be used whenever state at
-// the SQL layer is being migrated.
-//
-// Each upgrade is associated with a specific internal cluster version and is
-// idempotent in nature. When setting the cluster version (via `SET CLUSTER
-// SETTING version`), the manager process determines the set of upgrades
-// needed to bridge the gap between the current active cluster version, and the
-// target one. See [1] for where that happens.
-//
-// To introduce an upgrade, start by adding version key to pkg/clusterversion
-// and introducing a corresponding internal cluster version for it. See [2] for
-// more details. Following that, define an Upgrade in the upgrades package
-// and add it to the appropriate upgrades slice to the registry. Be sure to
-// key it in with the new cluster version we just added. During cluster
-// upgrades, once the operator is able to set a cluster version setting that's
-// past the version that was introduced (typically the major release version
-// the upgrade was introduced in), the manager will execute the defined
-// upgrade before letting the upgrade finalize.
-//
-// If the upgrade requires below-Raft level changes ([3] is one example),
-// you'll need to add a version switch and the relevant system-level upgrade
-// in [4]. See IterateRangeDescriptors and the Migrate KV request for more
-// details.
-//
-// [1]: `(*Manager).Migrate`
-// [2]: pkg/clusterversion/cockroach_versions.go
-// [3]: truncatedStateMigration
-// [4]: pkg/kv/kvserver/batch_eval/cmd_migrate.go
-type Upgrade interface {
- Version() roachpb.Version
- Name() string
- internal() // restrict implementations to this package
-}
-
// JobDeps are upgrade-specific dependencies used by the upgrade job to run
// upgrades.
type JobDeps interface {
// GetUpgrade returns the upgrade associated with the cluster version
// if one exists.
- GetUpgrade(key roachpb.Version) (Upgrade, bool)
+ GetUpgrade(key roachpb.Version) (upgradebase.Upgrade, bool)
// SystemDeps returns a handle to upgrade dependencies on a system tenant.
SystemDeps() SystemDeps
@@ -80,17 +42,27 @@ type JobDeps interface {
type upgrade struct {
description string
- v roachpb.Version
+ // v is the version that this upgrade is associated with. The upgrade runs
+ // when the cluster's version is incremented to v or, for permanent upgrades
+ // (see below) when the cluster is bootstrapped at v or above.
+ v roachpb.Version
+ // permanent is set for "permanent" upgrades - i.e. upgrades that are not
+ // baked into the bootstrap image and need to be run on new clusters
+ // regardless of the cluster's bootstrap version.
+ permanent bool
}
-// ClusterVersion makes SystemUpgrade an Upgrade.
+// Version is part of the upgradebase.Upgrade interface.
func (m *upgrade) Version() roachpb.Version {
return m.v
}
-// Name returns a human-readable name for this upgrade.
+// Permanent is part of the upgradebase.Upgrade interface.
+func (m *upgrade) Permanent() bool {
+ return m.permanent
+}
+
+// Name is part of the upgradebase.Upgrade interface.
func (m *upgrade) Name() string {
return fmt.Sprintf("Upgrade to %s: %q", m.v.String(), m.description)
}
-
-func (m *upgrade) internal() {}
diff --git a/pkg/upgrade/upgradebase/BUILD.bazel b/pkg/upgrade/upgradebase/BUILD.bazel
new file mode 100644
index 000000000000..447bddfa4c8d
--- /dev/null
+++ b/pkg/upgrade/upgradebase/BUILD.bazel
@@ -0,0 +1,18 @@
+load("//build/bazelutil/unused_checker:unused.bzl", "get_x_data")
+load("@io_bazel_rules_go//go:def.bzl", "go_library")
+
+go_library(
+ name = "upgradebase",
+ srcs = [
+ "testing_knobs.go",
+ "upgrade.go",
+ ],
+ importpath = "github.com/cockroachdb/cockroach/pkg/upgrade/upgradebase",
+ visibility = ["//visibility:public"],
+ deps = [
+ "//pkg/base",
+ "//pkg/roachpb",
+ ],
+)
+
+get_x_data(name = "get_x_data")
diff --git a/pkg/upgrade/testing_knobs.go b/pkg/upgrade/upgradebase/testing_knobs.go
similarity index 74%
rename from pkg/upgrade/testing_knobs.go
rename to pkg/upgrade/upgradebase/testing_knobs.go
index 36c6f178eabf..f266aedfb6c6 100644
--- a/pkg/upgrade/testing_knobs.go
+++ b/pkg/upgrade/upgradebase/testing_knobs.go
@@ -8,7 +8,7 @@
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.
-package upgrade
+package upgradebase
import (
"github.com/cockroachdb/cockroach/pkg/base"
@@ -26,6 +26,16 @@ type TestingKnobs struct {
// RegistryOverride is used to inject upgrades for specific cluster versions.
RegistryOverride func(v roachpb.Version) (Upgrade, bool)
+
+ // DontUseJobs, if set, makes upgrades run without employing jobs. This helps
+ // tests that care about not having random rows in the system.jobs table, and
+ // such. Jobs are not essential for running upgrades, but they help in
+ // production.
+ DontUseJobs bool
+
+ // AfterRunPermanentUpgrades is called after each call to
+ // RunPermanentUpgrades.
+ AfterRunPermanentUpgrades func()
}
// ModuleTestingKnobs makes TestingKnobs a base.ModuleTestingKnobs.
diff --git a/pkg/upgrade/upgradebase/upgrade.go b/pkg/upgrade/upgradebase/upgrade.go
new file mode 100644
index 000000000000..87ecd431f383
--- /dev/null
+++ b/pkg/upgrade/upgradebase/upgrade.go
@@ -0,0 +1,55 @@
+// Copyright 2022 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 upgradebase
+
+import "github.com/cockroachdb/cockroach/pkg/roachpb"
+
+// Upgrade defines a program to be executed once every node in the cluster is
+// (a) running a specific binary version, and (b) has completed all prior
+// upgrades. Note that there are two types of upgrades, a SystemUpgrade
+// and a TenantUpgrade. A SystemUpgrade only runs on the system tenant and
+// is used to migrate state at the KV layer. A TenantUpgrade runs on all
+// tenants (including the system tenant) and should be used whenever state at
+// the SQL layer is being migrated.
+//
+// Each upgrade is associated with a specific internal cluster version and is
+// idempotent in nature. When setting the cluster version (via `SET CLUSTER
+// SETTING version`), the manager process determines the set of upgrades
+// needed to bridge the gap between the current active cluster version, and the
+// target one. See [1] for where that happens.
+//
+// To introduce an upgrade, start by adding version key to pkg/clusterversion
+// and introducing a corresponding internal cluster version for it. See [2] for
+// more details. Following that, define an Upgrade in the upgrades package
+// and add it to the appropriate upgrades slice to the registry. Be sure to
+// key it in with the new cluster version we just added. During cluster
+// upgrades, once the operator is able to set a cluster version setting that's
+// past the version that was introduced (typically the major release version
+// the upgrade was introduced in), the manager will execute the defined
+// upgrade before letting the upgrade finalize.
+//
+// If the upgrade requires below-Raft level changes ([3] is one example),
+// you'll need to add a version switch and the relevant system-level upgrade
+// in [4]. See IterateRangeDescriptors and the Migrate KV request for more
+// details.
+//
+// [1]: `(*Manager).Migrate`
+// [2]: pkg/clusterversion/cockroach_versions.go
+// [3]: truncatedStateMigration
+// [4]: pkg/kv/kvserver/batch_eval/cmd_migrate.go
+type Upgrade interface {
+ Version() roachpb.Version
+ // Name returns a human-readable name for this upgrade.
+ Name() string
+ // Permanent returns true for "permanent" upgrades - i.e. upgrades that are
+ // not baked into the bootstrap image and need to be run on new cluster
+ // regardless of the cluster's bootstrap version.
+ Permanent() bool
+}
diff --git a/pkg/upgrade/upgradejob/BUILD.bazel b/pkg/upgrade/upgradejob/BUILD.bazel
index 489a9062251a..2937d506485f 100644
--- a/pkg/upgrade/upgradejob/BUILD.bazel
+++ b/pkg/upgrade/upgradejob/BUILD.bazel
@@ -7,6 +7,7 @@ go_library(
importpath = "github.com/cockroachdb/cockroach/pkg/upgrade/upgradejob",
visibility = ["//visibility:public"],
deps = [
+ "//pkg/base",
"//pkg/clusterversion",
"//pkg/jobs",
"//pkg/jobs/jobspb",
@@ -17,12 +18,9 @@ go_library(
"//pkg/sql",
"//pkg/sql/catalog/descs",
"//pkg/sql/catalog/resolver",
- "//pkg/sql/sem/tree",
- "//pkg/sql/sessiondata",
"//pkg/sql/sessiondatapb",
- "//pkg/sql/sqlutil",
"//pkg/upgrade",
- "//pkg/util/timeutil",
+ "//pkg/upgrade/migrationstable",
"@com_github_cockroachdb_errors//:errors",
],
)
diff --git a/pkg/upgrade/upgradejob/upgrade_job.go b/pkg/upgrade/upgradejob/upgrade_job.go
index c03b11316aa0..42539569e179 100644
--- a/pkg/upgrade/upgradejob/upgrade_job.go
+++ b/pkg/upgrade/upgradejob/upgrade_job.go
@@ -15,6 +15,7 @@ package upgradejob
import (
"context"
+ "github.com/cockroachdb/cockroach/pkg/base"
"github.com/cockroachdb/cockroach/pkg/clusterversion"
"github.com/cockroachdb/cockroach/pkg/jobs"
"github.com/cockroachdb/cockroach/pkg/jobs/jobspb"
@@ -25,12 +26,9 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descs"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/resolver"
- "github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
- "github.com/cockroachdb/cockroach/pkg/sql/sessiondata"
"github.com/cockroachdb/cockroach/pkg/sql/sessiondatapb"
- "github.com/cockroachdb/cockroach/pkg/sql/sqlutil"
"github.com/cockroachdb/cockroach/pkg/upgrade"
- "github.com/cockroachdb/cockroach/pkg/util/timeutil"
+ "github.com/cockroachdb/cockroach/pkg/upgrade/migrationstable"
"github.com/cockroachdb/errors"
)
@@ -68,7 +66,11 @@ func (r resumer) Resume(ctx context.Context, execCtxI interface{}) error {
v := pl.GetMigration().ClusterVersion.Version
ie := execCtx.ExecCfg().InternalExecutor
- alreadyCompleted, err := CheckIfMigrationCompleted(ctx, nil /* txn */, ie, v)
+ enterpriseEnabled := base.CCLDistributionAndEnterpriseEnabled(
+ execCtx.ExecCfg().Settings, execCtx.ExecCfg().NodeInfo.LogicalClusterID())
+ alreadyCompleted, err := migrationstable.CheckIfMigrationCompleted(
+ ctx, v, nil /* txn */, ie, enterpriseEnabled, migrationstable.ConsistentRead,
+ )
if alreadyCompleted || err != nil {
return errors.Wrapf(err, "checking migration completion for %v", v)
}
@@ -131,70 +133,12 @@ func (r resumer) Resume(ctx context.Context, execCtxI interface{}) error {
// Mark the upgrade as having been completed so that subsequent iterations
// no-op and new jobs are not created.
- if err := markMigrationCompleted(ctx, ie, v); err != nil {
+ if err := migrationstable.MarkMigrationCompleted(ctx, ie, v); err != nil {
return errors.Wrapf(err, "marking migration complete for %v", v)
}
return nil
}
-// CheckIfMigrationCompleted queries the system.migrations table to determine
-// if the upgrade associated with this version has already been completed.
-// The txn may be nil, in which case the check will be run in its own
-// transaction.
-func CheckIfMigrationCompleted(
- ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor, v roachpb.Version,
-) (alreadyCompleted bool, _ error) {
- row, err := ie.QueryRow(
- ctx,
- "migration-job-find-already-completed",
- txn,
- `
-SELECT EXISTS(
- SELECT *
- FROM system.migrations
- WHERE major = $1
- AND minor = $2
- AND patch = $3
- AND internal = $4
- );
-`,
- v.Major,
- v.Minor,
- v.Patch,
- v.Internal)
- if err != nil {
- return false, err
- }
- return bool(*row[0].(*tree.DBool)), nil
-}
-
-func markMigrationCompleted(
- ctx context.Context, ie sqlutil.InternalExecutor, v roachpb.Version,
-) error {
- _, err := ie.ExecEx(
- ctx,
- "migration-job-mark-job-succeeded",
- nil, /* txn */
- sessiondata.NodeUserSessionDataOverride,
- `
-INSERT
- INTO system.migrations
- (
- major,
- minor,
- patch,
- internal,
- completed_at
- )
-VALUES ($1, $2, $3, $4, $5)`,
- v.Major,
- v.Minor,
- v.Patch,
- v.Internal,
- timeutil.Now())
- return err
-}
-
// The long-running upgrade resumer has no reverting logic.
func (r resumer) OnFailOrCancel(ctx context.Context, execCtx interface{}, _ error) error {
return nil
diff --git a/pkg/upgrade/upgrademanager/BUILD.bazel b/pkg/upgrade/upgrademanager/BUILD.bazel
index 543717c21d46..07e199d5ba31 100644
--- a/pkg/upgrade/upgrademanager/BUILD.bazel
+++ b/pkg/upgrade/upgrademanager/BUILD.bazel
@@ -7,6 +7,7 @@ go_library(
importpath = "github.com/cockroachdb/cockroach/pkg/upgrade/upgrademanager",
visibility = ["//visibility:public"],
deps = [
+ "//pkg/base",
"//pkg/clusterversion",
"//pkg/jobs",
"//pkg/jobs/jobspb",
@@ -24,9 +25,12 @@ go_library(
"//pkg/sql/sem/tree",
"//pkg/sql/sqlutil",
"//pkg/upgrade",
+ "//pkg/upgrade/migrationstable",
+ "//pkg/upgrade/upgradebase",
"//pkg/upgrade/upgradejob:upgrade_job",
"//pkg/upgrade/upgrades",
"//pkg/util/log",
+ "//pkg/util/uuid",
"@com_github_cockroachdb_errors//:errors",
"@com_github_cockroachdb_logtags//:logtags",
"@com_github_cockroachdb_redact//:redact",
@@ -60,7 +64,7 @@ go_test(
"//pkg/testutils/sqlutils",
"//pkg/testutils/testcluster",
"//pkg/upgrade",
- "//pkg/upgrade/upgrades",
+ "//pkg/upgrade/upgradebase",
"//pkg/util",
"//pkg/util/leaktest",
"//pkg/util/log",
diff --git a/pkg/upgrade/upgrademanager/manager.go b/pkg/upgrade/upgrademanager/manager.go
index cffe667d41eb..cef75f5041d9 100644
--- a/pkg/upgrade/upgrademanager/manager.go
+++ b/pkg/upgrade/upgrademanager/manager.go
@@ -16,6 +16,7 @@ import (
"context"
"fmt"
+ "github.com/cockroachdb/cockroach/pkg/base"
"github.com/cockroachdb/cockroach/pkg/clusterversion"
"github.com/cockroachdb/cockroach/pkg/jobs"
"github.com/cockroachdb/cockroach/pkg/jobs/jobspb"
@@ -33,9 +34,12 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/sqlutil"
"github.com/cockroachdb/cockroach/pkg/upgrade"
+ "github.com/cockroachdb/cockroach/pkg/upgrade/migrationstable"
+ "github.com/cockroachdb/cockroach/pkg/upgrade/upgradebase"
"github.com/cockroachdb/cockroach/pkg/upgrade/upgradejob"
"github.com/cockroachdb/cockroach/pkg/upgrade/upgrades"
"github.com/cockroachdb/cockroach/pkg/util/log"
+ "github.com/cockroachdb/cockroach/pkg/util/uuid"
"github.com/cockroachdb/errors"
"github.com/cockroachdb/logtags"
"github.com/cockroachdb/redact"
@@ -44,18 +48,19 @@ import (
// Manager is the instance responsible for executing upgrades across the
// cluster.
type Manager struct {
- deps upgrade.SystemDeps
- lm *lease.Manager
- ie sqlutil.InternalExecutor
- ief descs.TxnManager
- jr *jobs.Registry
- codec keys.SQLCodec
- settings *cluster.Settings
- knobs upgrade.TestingKnobs
+ deps upgrade.SystemDeps
+ lm *lease.Manager
+ ie sqlutil.InternalExecutor
+ ief descs.TxnManager
+ jr *jobs.Registry
+ codec keys.SQLCodec
+ settings *cluster.Settings
+ knobs upgradebase.TestingKnobs
+ clusterID uuid.UUID
}
// GetUpgrade returns the upgrade associated with this key.
-func (m *Manager) GetUpgrade(key roachpb.Version) (upgrade.Upgrade, bool) {
+func (m *Manager) GetUpgrade(key roachpb.Version) (upgradebase.Upgrade, bool) {
if m.knobs.RegistryOverride != nil {
if m, ok := m.knobs.RegistryOverride(key); ok {
return m, ok
@@ -80,21 +85,23 @@ func NewManager(
jr *jobs.Registry,
codec keys.SQLCodec,
settings *cluster.Settings,
- testingKnobs *upgrade.TestingKnobs,
+ clusterID uuid.UUID,
+ testingKnobs *upgradebase.TestingKnobs,
) *Manager {
- var knobs upgrade.TestingKnobs
+ var knobs upgradebase.TestingKnobs
if testingKnobs != nil {
knobs = *testingKnobs
}
return &Manager{
- deps: deps,
- lm: lm,
- ie: ie,
- ief: ief,
- jr: jr,
- codec: codec,
- settings: settings,
- knobs: knobs,
+ deps: deps,
+ lm: lm,
+ ie: ie,
+ ief: ief,
+ jr: jr,
+ codec: codec,
+ settings: settings,
+ clusterID: clusterID,
+ knobs: knobs,
}
}
@@ -148,8 +155,103 @@ func safeToUpgradeTenant(
return true, nil
}
-// Migrate runs the set of upgrades required to upgrade the cluster version
-// from the current version to the target one.
+// RunPermanentUpgrades runs all the upgrades associated with cluster versions
+// <= upToVersion that are marked as permanent. Upgrades that have already run
+// to completion, or that are currently running, are not run again, but the call
+// will block for their completion.
+//
+// NOTE: All upgrades, permanent and non-permanent, end up running in order.
+// RunPermanentUpgrades(v1) is called before Migrate(v1,v2). Of course,
+// non-permanent upgrades for versions <= v1 are not run at all; they're assumed
+// to be baked into the bootstrap metadata.
+func (m *Manager) RunPermanentUpgrades(ctx context.Context, upToVersion roachpb.Version) error {
+ log.Infof(ctx, "running permanent upgrades up to version: %v", upToVersion)
+ defer func() {
+ if fn := m.knobs.AfterRunPermanentUpgrades; fn != nil {
+ fn()
+ }
+ }()
+ vers := m.listBetween(roachpb.Version{}, upToVersion)
+ var permanentUpgrades []upgradebase.Upgrade
+ for _, v := range vers {
+ upgrade, exists := m.GetUpgrade(v)
+ if !exists || !upgrade.Permanent() {
+ continue
+ }
+ permanentUpgrades = append(permanentUpgrades, upgrade)
+ }
+
+ user := username.RootUserName()
+
+ if len(permanentUpgrades) == 0 {
+ // If we didn't find any permanent migrations, it must be that a test used
+ // some the testing knobs to inhibit us from finding the migrations.
+ // However, we must run the permanent migrations (at least the one writing
+ // the value of the cluster version to the system.settings table); the test
+ // did not actually mean to inhibit running these. So we'll run them anyway,
+ // without using jobs, so that the side effect of the migrations are
+ // minimized and tests continue to be happy as they were before the
+ // permanent migrations were introduced.
+ return m.runPermanentMigrationsWithoutJobsForTests(ctx, user)
+ }
+
+ // Do a best-effort check to see if all upgrades have already executed and so
+ // there's nothing for us to do. Probably the most common case is that a node
+ // is started in a cluster that has already run all the relevant upgrades, in
+ // which case we'll figure this out cheaply.
+ // We look at whether the last permanent upgrade that we need to run has
+ // already completed successfully. Looking only at the last one is sufficient
+ // because upgrades run in order.
+ latest := permanentUpgrades[len(permanentUpgrades)-1]
+ lastVer := latest.Version()
+ enterpriseEnabled := base.CCLDistributionAndEnterpriseEnabled(m.settings, m.clusterID)
+ lastUpgradeCompleted, err := migrationstable.CheckIfMigrationCompleted(
+ ctx, lastVer, nil /* txn */, m.ie,
+ // We'll do a follower read. This is all best effort anyway, and the
+ // follower read should keep the startup time low in the common case where
+ // all upgrades have run a long time ago before this node start.
+ enterpriseEnabled,
+ migrationstable.StaleRead)
+ if err != nil {
+ return err
+ }
+ if lastUpgradeCompleted {
+ return nil
+ }
+
+ for _, u := range permanentUpgrades {
+ log.Infof(ctx, "running permanent upgrade for version %s", u.Version())
+ if err := m.runMigration(ctx, u, user, u.Version(), !m.knobs.DontUseJobs); err != nil {
+ return err
+ }
+ }
+ return nil
+}
+
+// runPermanentMigrationsWithoutJobsForTests runs all permanent migrations up to
+// VPrimordialMax. They are run without jobs, in order to minimize the side
+// effects left on cluster.
+//
+// NOTE: VPrimordialMax was chosen arbitrarily, since we don't have a great way
+// to tell which migrations are needed and which aren't on the code path leading
+// here.
+func (m *Manager) runPermanentMigrationsWithoutJobsForTests(
+ ctx context.Context, user username.SQLUsername,
+) error {
+ log.Infof(ctx, "found test configuration that eliminated all upgrades; running permanent upgrades anyway")
+ vers := clusterversion.ListBetween(roachpb.Version{}, clusterversion.ByKey(clusterversion.VPrimordialMax))
+ for _, v := range vers {
+ upg, exists := upgrades.GetUpgrade(v)
+ if !exists || !upg.Permanent() {
+ continue
+ }
+ if err := m.runMigration(ctx, upg, user, upg.Version(), false /* useJob */); err != nil {
+ return err
+ }
+ }
+ return nil
+}
+
func (m *Manager) Migrate(
ctx context.Context,
user username.SQLUsername,
@@ -200,8 +302,11 @@ func (m *Manager) Migrate(
log.Infof(ctx, "stepping through %s", clusterVersion)
cv := clusterversion.ClusterVersion{Version: clusterVersion}
// First, run the actual upgrade if any.
- if err := m.runMigration(ctx, user, clusterVersion); err != nil {
- return err
+ mig, exists := m.GetUpgrade(clusterVersion)
+ if exists {
+ if err := m.runMigration(ctx, mig, user, clusterVersion, !m.knobs.DontUseJobs); err != nil {
+ return err
+ }
}
// Next we'll push out the version gate to every node in the cluster.
@@ -353,21 +458,66 @@ func forEveryNodeUntilClusterStable(
}
func (m *Manager) runMigration(
- ctx context.Context, user username.SQLUsername, version roachpb.Version,
+ ctx context.Context,
+ mig upgradebase.Upgrade,
+ user username.SQLUsername,
+ version roachpb.Version,
+ useJob bool,
) error {
- mig, exists := m.GetUpgrade(version)
- if !exists {
- return nil
- }
_, isSystemMigration := mig.(*upgrade.SystemUpgrade)
if isSystemMigration && !m.codec.ForSystemTenant() {
return nil
}
- alreadyCompleted, id, err := m.getOrCreateMigrationJob(ctx, user, version, mig.Name())
- if alreadyCompleted || err != nil {
- return err
+ if !useJob {
+ // Some tests don't like it when jobs are run at server startup, because
+ // they pollute the jobs table. So, we run the upgrade directly.
+
+ alreadyCompleted, err := migrationstable.CheckIfMigrationCompleted(
+ ctx, version, nil /* txn */, m.ie, false /* enterpriseEnabled */, migrationstable.ConsistentRead,
+ )
+ if alreadyCompleted || err != nil {
+ return err
+ }
+
+ switch upg := mig.(type) {
+ case *upgrade.SystemUpgrade:
+ if err := upg.Run(ctx, mig.Version(), m.SystemDeps()); err != nil {
+ return err
+ }
+ case *upgrade.TenantUpgrade:
+ // The TenantDeps used here are incomplete, but enough for the "permanent
+ // upgrades" that run under this testing knob.
+ if err := upg.Run(ctx, mig.Version(), upgrade.TenantDeps{
+ DB: m.deps.DB,
+ Codec: m.codec,
+ Settings: m.settings,
+ LeaseManager: m.lm,
+ InternalExecutor: m.ie,
+ InternalExecutorFactory: m.ief,
+ JobRegistry: m.jr,
+ }); err != nil {
+ return err
+ }
+ }
+
+ if err := migrationstable.MarkMigrationCompleted(ctx, m.ie, mig.Version()); err != nil {
+ return err
+ }
+ } else {
+ // Run a job that, in turn, will run the upgrade. By running upgrades inside
+ // jobs, we get some observability for them and we avoid multiple nodes
+ // attempting to run the same upgrade all at once. Particularly for
+ // long-running upgrades, this is useful.
+ //
+ // If the job already exists, we wait for it to finish.
+ alreadyCompleted, id, err := m.getOrCreateMigrationJob(ctx, user, version, mig.Name())
+ if alreadyCompleted || err != nil {
+ return err
+ }
+ log.Infof(ctx, "running %s", mig.Name())
+ return m.jr.Run(ctx, m.ie, []jobspb.JobID{id})
}
- return m.jr.Run(ctx, m.ie, []jobspb.JobID{id})
+ return nil
}
func (m *Manager) getOrCreateMigrationJob(
@@ -375,7 +525,10 @@ func (m *Manager) getOrCreateMigrationJob(
) (alreadyCompleted bool, jobID jobspb.JobID, _ error) {
newJobID := m.jr.MakeJobID()
if err := m.deps.DB.Txn(ctx, func(ctx context.Context, txn *kv.Txn) (err error) {
- alreadyCompleted, err = upgradejob.CheckIfMigrationCompleted(ctx, txn, m.ie, version)
+ enterpriseEnabled := base.CCLDistributionAndEnterpriseEnabled(m.settings, m.clusterID)
+ alreadyCompleted, err = migrationstable.CheckIfMigrationCompleted(
+ ctx, version, txn, m.ie, enterpriseEnabled, migrationstable.ConsistentRead,
+ )
if err != nil && ctx.Err() == nil {
log.Warningf(ctx, "failed to check if migration already completed: %v", err)
}
diff --git a/pkg/upgrade/upgrademanager/manager_external_test.go b/pkg/upgrade/upgrademanager/manager_external_test.go
index db988bde6693..ca291175cde4 100644
--- a/pkg/upgrade/upgrademanager/manager_external_test.go
+++ b/pkg/upgrade/upgrademanager/manager_external_test.go
@@ -34,7 +34,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/testutils/sqlutils"
"github.com/cockroachdb/cockroach/pkg/testutils/testcluster"
"github.com/cockroachdb/cockroach/pkg/upgrade"
- "github.com/cockroachdb/cockroach/pkg/upgrade/upgrades"
+ "github.com/cockroachdb/cockroach/pkg/upgrade/upgradebase"
"github.com/cockroachdb/cockroach/pkg/util"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
"github.com/cockroachdb/cockroach/pkg/util/log"
@@ -76,15 +76,15 @@ func TestAlreadyRunningJobsAreHandledProperly(t *testing.T) {
// See the TODO below for why we need this.
ProcessorNoTracingSpan: true,
},
- UpgradeManager: &upgrade.TestingKnobs{
+ UpgradeManager: &upgradebase.TestingKnobs{
ListBetweenOverride: func(from, to roachpb.Version) []roachpb.Version {
return []roachpb.Version{to}
},
- RegistryOverride: func(v roachpb.Version) (upgrade.Upgrade, bool) {
+ RegistryOverride: func(v roachpb.Version) (upgradebase.Upgrade, bool) {
if v != endCV {
return nil, false
}
- return upgrade.NewTenantUpgrade("test", v, upgrades.NoPrecondition, func(
+ return upgrade.NewTenantUpgrade("test", v, upgrade.NoPrecondition, func(
ctx context.Context, version clusterversion.ClusterVersion, deps upgrade.TenantDeps,
) error {
canResume := make(chan error)
@@ -223,11 +223,11 @@ func TestMigrateUpdatesReplicaVersion(t *testing.T) {
BinaryVersionOverride: startCV,
DisableAutomaticVersionUpgrade: make(chan struct{}),
},
- UpgradeManager: &upgrade.TestingKnobs{
+ UpgradeManager: &upgradebase.TestingKnobs{
ListBetweenOverride: func(from, to roachpb.Version) []roachpb.Version {
return []roachpb.Version{from, to}
},
- RegistryOverride: func(cv roachpb.Version) (upgrade.Upgrade, bool) {
+ RegistryOverride: func(cv roachpb.Version) (upgradebase.Upgrade, bool) {
if cv != endCV {
return nil, false
}
@@ -337,11 +337,11 @@ func TestConcurrentMigrationAttempts(t *testing.T) {
BinaryVersionOverride: versions[0],
DisableAutomaticVersionUpgrade: make(chan struct{}),
},
- UpgradeManager: &upgrade.TestingKnobs{
+ UpgradeManager: &upgradebase.TestingKnobs{
ListBetweenOverride: func(from, to roachpb.Version) []roachpb.Version {
return versions
},
- RegistryOverride: func(cv roachpb.Version) (upgrade.Upgrade, bool) {
+ RegistryOverride: func(cv roachpb.Version) (upgradebase.Upgrade, bool) {
return upgrade.NewSystemUpgrade("test", cv, func(
ctx context.Context, version clusterversion.ClusterVersion, d upgrade.SystemDeps,
) error {
@@ -419,15 +419,15 @@ func TestPauseMigration(t *testing.T) {
BinaryVersionOverride: startCV,
DisableAutomaticVersionUpgrade: make(chan struct{}),
},
- UpgradeManager: &upgrade.TestingKnobs{
+ UpgradeManager: &upgradebase.TestingKnobs{
ListBetweenOverride: func(from, to roachpb.Version) []roachpb.Version {
return []roachpb.Version{to}
},
- RegistryOverride: func(cv roachpb.Version) (upgrade.Upgrade, bool) {
+ RegistryOverride: func(cv roachpb.Version) (upgradebase.Upgrade, bool) {
if cv != endCV {
return nil, false
}
- return upgrade.NewTenantUpgrade("test", cv, upgrades.NoPrecondition, func(
+ return upgrade.NewTenantUpgrade("test", cv, upgrade.NoPrecondition, func(
ctx context.Context, version clusterversion.ClusterVersion, deps upgrade.TenantDeps,
) error {
canResume := make(chan error)
@@ -542,13 +542,13 @@ func TestPrecondition(t *testing.T) {
},
// Inject an upgrade which would run to upgrade the cluster.
// We'll validate that we never create a job for this upgrade.
- UpgradeManager: &upgrade.TestingKnobs{
+ UpgradeManager: &upgradebase.TestingKnobs{
ListBetweenOverride: func(from, to roachpb.Version) []roachpb.Version {
start := sort.Search(len(versions), func(i int) bool { return from.Less(versions[i]) })
end := sort.Search(len(versions), func(i int) bool { return to.Less(versions[i]) })
return versions[start:end]
},
- RegistryOverride: func(cv roachpb.Version) (upgrade.Upgrade, bool) {
+ RegistryOverride: func(cv roachpb.Version) (upgradebase.Upgrade, bool) {
switch cv {
case v1:
return upgrade.NewTenantUpgrade("v1", cv,
@@ -561,7 +561,7 @@ func TestPrecondition(t *testing.T) {
), true
case v2:
return upgrade.NewTenantUpgrade("v2", cv,
- upgrades.NoPrecondition,
+ upgrade.NoPrecondition,
cf(&migrationRun, &migrationErr),
), true
default:
diff --git a/pkg/upgrade/upgrades/BUILD.bazel b/pkg/upgrade/upgrades/BUILD.bazel
index 48db9cd3a34a..a2c4e33da56e 100644
--- a/pkg/upgrade/upgrades/BUILD.bazel
+++ b/pkg/upgrade/upgrades/BUILD.bazel
@@ -11,6 +11,7 @@ go_library(
"descriptor_utils.go",
"ensure_sql_schema_telemetry_schedule.go",
"fix_userfile_descriptor_corruption.go",
+ "permanent_upgrades.go",
"precondition_before_starting_an_upgrade.go",
"role_id_sequence_migration.go",
"role_options_table_migration.go",
@@ -59,6 +60,7 @@ go_library(
"//pkg/sql/types",
"//pkg/storage",
"//pkg/upgrade",
+ "//pkg/upgrade/upgradebase",
"//pkg/util/hlc",
"//pkg/util/log",
"//pkg/util/protoutil",
@@ -138,6 +140,7 @@ go_test(
"//pkg/testutils/sqlutils",
"//pkg/testutils/testcluster",
"//pkg/upgrade",
+ "//pkg/upgrade/upgradebase",
"//pkg/util",
"//pkg/util/ctxgroup",
"//pkg/util/hlc",
diff --git a/pkg/upgrade/upgrades/permanent_upgrades.go b/pkg/upgrade/upgrades/permanent_upgrades.go
new file mode 100644
index 000000000000..856d2a215bc7
--- /dev/null
+++ b/pkg/upgrade/upgrades/permanent_upgrades.go
@@ -0,0 +1,282 @@
+// Copyright 2022 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 upgrades
+
+import (
+ "context"
+ "fmt"
+
+ "github.com/cockroachdb/cockroach/pkg/clusterversion"
+ "github.com/cockroachdb/cockroach/pkg/keys"
+ "github.com/cockroachdb/cockroach/pkg/kv"
+ "github.com/cockroachdb/cockroach/pkg/roachpb"
+ "github.com/cockroachdb/cockroach/pkg/security/username"
+ "github.com/cockroachdb/cockroach/pkg/settings/cluster"
+ "github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkeys"
+ "github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
+ "github.com/cockroachdb/cockroach/pkg/sql/sessiondata"
+ "github.com/cockroachdb/cockroach/pkg/upgrade"
+ "github.com/cockroachdb/cockroach/pkg/util/log"
+ "github.com/cockroachdb/cockroach/pkg/util/protoutil"
+ "github.com/cockroachdb/cockroach/pkg/util/retry"
+ "github.com/cockroachdb/errors"
+)
+
+func addRootUser(
+ ctx context.Context, _ clusterversion.ClusterVersion, deps upgrade.TenantDeps,
+) error {
+ // Upsert the root user into the table. We intentionally override any existing entry.
+ const upsertRootStmt = `
+ UPSERT INTO system.users (username, "hashedPassword", "isRole", "user_id") VALUES ($1, '', false, 1)
+ `
+ _, err := deps.InternalExecutor.Exec(ctx, "addRootUser", nil /* txn */, upsertRootStmt, username.RootUser)
+ if err != nil {
+ return err
+ }
+
+ // Upsert the admin role into the table. We intentionally override any existing entry.
+ const upsertAdminStmt = `
+ UPSERT INTO system.users (username, "hashedPassword", "isRole", "user_id") VALUES ($1, '', true, 2)
+ `
+ _, err = deps.InternalExecutor.Exec(ctx, "addAdminRole", nil /* txn */, upsertAdminStmt, username.AdminRole)
+ if err != nil {
+ return err
+ }
+
+ // Upsert the role membership into the table. We intentionally override any existing entry.
+ const upsertMembership = `
+ UPSERT INTO system.role_members ("role", "member", "isAdmin") VALUES ($1, $2, true)
+ `
+ _, err = deps.InternalExecutor.Exec(
+ ctx, "addRootToAdminRole", nil /* txn */, upsertMembership, username.AdminRole, username.RootUser)
+ if err != nil {
+ return err
+ }
+
+ // Add the CREATELOGIN option to roles that already have CREATEROLE.
+ const upsertCreateRoleStmt = `
+ UPSERT INTO system.role_options (username, option, value)
+ SELECT username, 'CREATELOGIN', NULL
+ FROM system.role_options
+ WHERE option = 'CREATEROLE'
+ `
+ _, err = deps.InternalExecutor.Exec(
+ ctx, "add CREATELOGIN where a role already has CREATEROLE", nil, /* txn */
+ upsertCreateRoleStmt)
+ return err
+}
+
+func optInToDiagnosticsStatReporting(
+ ctx context.Context, _ clusterversion.ClusterVersion, deps upgrade.TenantDeps,
+) error {
+ // We're opting-out of the automatic opt-in. See discussion in updates.go.
+ if cluster.TelemetryOptOut() {
+ return nil
+ }
+ _, err := deps.InternalExecutor.Exec(
+ ctx, "optInToDiagnosticsStatReporting", nil, /* txn */
+ `SET CLUSTER SETTING diagnostics.reporting.enabled = true`)
+ return err
+}
+
+func populateVersionSetting(
+ ctx context.Context, _ clusterversion.ClusterVersion, deps upgrade.SystemDeps,
+) error {
+ var v roachpb.Version
+ if err := deps.DB.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error {
+ return txn.GetProto(ctx, keys.BootstrapVersionKey, &v)
+ }); err != nil {
+ return err
+ }
+ if v == (roachpb.Version{}) {
+ // The cluster was bootstrapped at v1.0 (or even earlier), so just use
+ // the TestingBinaryMinSupportedVersion of the binary.
+ v = clusterversion.TestingBinaryMinSupportedVersion
+ }
+
+ b, err := protoutil.Marshal(&clusterversion.ClusterVersion{Version: v})
+ if err != nil {
+ return errors.Wrap(err, "while marshaling version")
+ }
+
+ // Add a ON CONFLICT DO NOTHING to avoid changing an existing version.
+ // Again, this can happen if the migration doesn't run to completion
+ // (overwriting also seems reasonable, but what for).
+ // We don't allow users to perform version changes until we have run
+ // the insert below.
+ _, err = deps.InternalExecutor.Exec(
+ ctx, "insert-setting", nil, /* txn */
+ fmt.Sprintf(`INSERT INTO system.settings (name, value, "lastUpdated", "valueType") VALUES ('version', x'%x', now(), 'm') ON CONFLICT(name) DO NOTHING`, b),
+ )
+ if err != nil {
+ return err
+ }
+
+ // Tenant ID 0 indicates that we're overriding the value for all
+ // tenants.
+ tenantID := tree.NewDInt(0)
+ _, err = deps.InternalExecutor.Exec(
+ ctx,
+ "insert-setting", nil, /* txn */
+ fmt.Sprintf(`INSERT INTO system.tenant_settings (tenant_id, name, value, "last_updated", "value_type") VALUES (%d, 'version', x'%x', now(), 'm') ON CONFLICT(tenant_id, name) DO NOTHING`, tenantID, b),
+ )
+ if err != nil {
+ return err
+ }
+
+ return nil
+}
+
+func initializeClusterSecret(
+ ctx context.Context, _ clusterversion.ClusterVersion, deps upgrade.TenantDeps,
+) error {
+ _, err := deps.InternalExecutor.Exec(
+ ctx, "initializeClusterSecret", nil, /* txn */
+ `SET CLUSTER SETTING cluster.secret = gen_random_uuid()::STRING`,
+ )
+ return err
+}
+
+func retireOldTsPurgeIntervalSettings(
+ ctx context.Context, _ clusterversion.ClusterVersion, deps upgrade.SystemDeps,
+) error {
+ // We are going to deprecate `timeseries.storage.10s_resolution_ttl`
+ // into `timeseries.storage.resolution_10s.ttl` if the latter is not
+ // defined.
+ //
+ // Ditto for the `30m` resolution.
+
+ // Copy 'timeseries.storage.10s_resolution_ttl' into
+ // 'timeseries.storage.resolution_10s.ttl' if the former is defined
+ // and the latter is not defined yet.
+ //
+ // We rely on the SELECT returning no row if the original setting
+ // was not defined, and INSERT ON CONFLICT DO NOTHING to ignore the
+ // insert if the new name was already set.
+ _, err := deps.InternalExecutor.Exec(ctx, "copy-setting", nil, /* txn */
+ `
+INSERT INTO system.settings (name, value, "lastUpdated", "valueType")
+ SELECT 'timeseries.storage.resolution_10s.ttl', value, "lastUpdated", "valueType"
+ FROM system.settings WHERE name = 'timeseries.storage.10s_resolution_ttl'
+ON CONFLICT (name) DO NOTHING`,
+ )
+ if err != nil {
+ return err
+ }
+
+ // Ditto 30m.
+ _, err = deps.InternalExecutor.Exec(ctx, "copy-setting", nil, /* txn */
+ `
+INSERT INTO system.settings (name, value, "lastUpdated", "valueType")
+ SELECT 'timeseries.storage.resolution_30m.ttl', value, "lastUpdated", "valueType"
+ FROM system.settings WHERE name = 'timeseries.storage.30m_resolution_ttl'
+ON CONFLICT (name) DO NOTHING`,
+ )
+ if err != nil {
+ return err
+ }
+
+ return nil
+}
+
+func updateSystemLocationData(
+ ctx context.Context, _ clusterversion.ClusterVersion, deps upgrade.TenantDeps,
+) error {
+ // See if the system.locations table already has data in it.
+ // If so, we don't want to do anything.
+ row, err := deps.InternalExecutor.QueryRowEx(ctx, "update-system-locations",
+ nil, /* txn */
+ sessiondata.InternalExecutorOverride{User: username.RootUserName()},
+ `SELECT count(*) FROM system.locations`)
+ if err != nil {
+ return err
+ }
+ if row == nil {
+ return errors.New("failed to update system locations")
+ }
+ count := int(tree.MustBeDInt(row[0]))
+ if count != 0 {
+ return nil
+ }
+
+ for _, loc := range roachpb.DefaultLocationInformation {
+ stmt := `UPSERT INTO system.locations VALUES ($1, $2, $3, $4)`
+ tier := loc.Locality.Tiers[0]
+ _, err := deps.InternalExecutor.Exec(ctx, "update-system-locations", nil, /* txn */
+ stmt, tier.Key, tier.Value, loc.Latitude, loc.Longitude,
+ )
+ if err != nil {
+ return err
+ }
+ }
+ return nil
+}
+
+func disallowPublicUserOrRole(
+ ctx context.Context, _ clusterversion.ClusterVersion, deps upgrade.TenantDeps,
+) error {
+ // Check whether a user or role named "public" exists.
+ const selectPublicStmt = `
+ SELECT username, "isRole" from system.users WHERE username = $1
+ `
+
+ for retry := retry.Start(retry.Options{MaxRetries: 5}); retry.Next(); {
+ row, err := deps.InternalExecutor.QueryRowEx(
+ ctx, "disallowPublicUserOrRole", nil, /* txn */
+ sessiondata.InternalExecutorOverride{
+ User: username.RootUserName(),
+ },
+ selectPublicStmt, username.PublicRole,
+ )
+ if err != nil {
+ continue
+ }
+ if row == nil {
+ // No such user.
+ return nil
+ }
+
+ isRole, ok := tree.AsDBool(row[1])
+ if !ok {
+ log.Fatalf(ctx, "expected 'isRole' column of system.users to be of type bool, got %v", row)
+ }
+
+ if isRole {
+ return fmt.Errorf(`found a role named %s which is now a reserved name. Please drop the role `+
+ `(DROP ROLE %s) using a previous version of CockroachDB and try again`,
+ username.PublicRole, username.PublicRole)
+ }
+ return fmt.Errorf(`found a user named %s which is now a reserved name. Please drop the role `+
+ `(DROP USER %s) using a previous version of CockroachDB and try again`,
+ username.PublicRole, username.PublicRole)
+ }
+ return nil
+}
+
+func createDefaultDbs(
+ ctx context.Context, _ clusterversion.ClusterVersion, deps upgrade.TenantDeps,
+) error {
+ // Create the default databases. These are plain databases with
+ // default permissions. Nothing special happens if they exist
+ // already.
+ const createDbStmt = `CREATE DATABASE IF NOT EXISTS "%s"`
+
+ var err error
+ for _, dbName := range []string{catalogkeys.DefaultDatabaseName, catalogkeys.PgDatabaseName} {
+ stmt := fmt.Sprintf(createDbStmt, dbName)
+ _, err = deps.InternalExecutor.Exec(ctx, "create-default-DB", nil /* txn */, stmt)
+ if err != nil {
+ log.Warningf(ctx, "failed attempt to add database %q: %s", dbName, err)
+ return err
+ }
+ }
+ return nil
+}
diff --git a/pkg/upgrade/upgrades/schema_changes_external_test.go b/pkg/upgrade/upgrades/schema_changes_external_test.go
index fa92f271cc6d..70227e81dfd2 100644
--- a/pkg/upgrade/upgrades/schema_changes_external_test.go
+++ b/pkg/upgrade/upgrades/schema_changes_external_test.go
@@ -36,6 +36,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/testutils/sqlutils"
"github.com/cockroachdb/cockroach/pkg/testutils/testcluster"
"github.com/cockroachdb/cockroach/pkg/upgrade"
+ "github.com/cockroachdb/cockroach/pkg/upgrade/upgradebase"
"github.com/cockroachdb/cockroach/pkg/upgrade/upgrades"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
"github.com/cockroachdb/cockroach/pkg/util/log"
@@ -348,17 +349,17 @@ func testMigrationWithFailures(
}
},
},
- UpgradeManager: &upgrade.TestingKnobs{
+ UpgradeManager: &upgradebase.TestingKnobs{
ListBetweenOverride: func(from, to roachpb.Version) []roachpb.Version {
return []roachpb.Version{
endCV,
}
},
- RegistryOverride: func(cv roachpb.Version) (upgrade.Upgrade, bool) {
+ RegistryOverride: func(cv roachpb.Version) (upgradebase.Upgrade, bool) {
if cv.Equal(endCV) {
return upgrade.NewTenantUpgrade("testing",
endCV,
- upgrades.NoPrecondition,
+ upgrade.NoPrecondition,
migrationFunc,
), true
}
diff --git a/pkg/upgrade/upgrades/upgrades.go b/pkg/upgrade/upgrades/upgrades.go
index 48932eff9041..57dd1fd6efcb 100644
--- a/pkg/upgrade/upgrades/upgrades.go
+++ b/pkg/upgrade/upgrades/upgrades.go
@@ -20,21 +20,28 @@ import (
"github.com/cockroachdb/cockroach/pkg/clusterversion"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/upgrade"
+ "github.com/cockroachdb/cockroach/pkg/upgrade/upgradebase"
"github.com/cockroachdb/errors"
)
+// SettingsDefaultOverrides documents the effect of several migrations that add
+// an explicit value for a setting, effectively changing the "default value"
+// from what was defined in code.
+var SettingsDefaultOverrides = map[string]string{
+ "diagnostics.reporting.enabled": "true",
+ "cluster.secret": "",
+}
+
// GetUpgrade returns the upgrade corresponding to this version if
// one exists.
-func GetUpgrade(key roachpb.Version) (upgrade.Upgrade, bool) {
+func GetUpgrade(key roachpb.Version) (upgradebase.Upgrade, bool) {
m, ok := registry[key]
+ if !ok {
+ return nil, false
+ }
return m, ok
}
-// NoPrecondition is a PreconditionFunc that doesn't check anything.
-func NoPrecondition(context.Context, clusterversion.ClusterVersion, upgrade.TenantDeps) error {
- return nil
-}
-
// NoTenantUpgradeFunc is a TenantUpgradeFunc that doesn't do anything.
func NoTenantUpgradeFunc(context.Context, clusterversion.ClusterVersion, upgrade.TenantDeps) error {
return nil
@@ -43,9 +50,55 @@ func NoTenantUpgradeFunc(context.Context, clusterversion.ClusterVersion, upgrade
// registry defines the global mapping between a cluster version and the
// associated upgrade. The upgrade is only executed after a cluster-wide
// bump of the corresponding version gate.
-var registry = make(map[roachpb.Version]upgrade.Upgrade)
+var registry = make(map[roachpb.Version]upgradebase.Upgrade)
-var upgrades = []upgrade.Upgrade{
+var upgrades = []upgradebase.Upgrade{
+ upgrade.NewPermanentTenantUpgrade(
+ "add users and roles",
+ toCV(clusterversion.VPrimordial1),
+ addRootUser,
+ ),
+ upgrade.NewPermanentTenantUpgrade(
+ "enable diagnostics reporting",
+ toCV(clusterversion.VPrimordial2),
+ optInToDiagnosticsStatReporting,
+ ),
+ upgrade.NewPermanentSystemUpgrade(
+ "populate initial version cluster setting table entry",
+ toCV(clusterversion.VPrimordial3),
+ populateVersionSetting,
+ ),
+ upgrade.NewPermanentTenantUpgrade(
+ "initialize the cluster.secret setting",
+ toCV(clusterversion.VPrimordial4),
+ initializeClusterSecret,
+ ),
+ // Introduced in v19.1.
+ // TODO(knz): bake this migration into v19.2.
+ upgrade.NewPermanentSystemUpgrade(
+ "propagate the ts purge interval to the new setting names",
+ toCV(clusterversion.VPrimordial5),
+ retireOldTsPurgeIntervalSettings,
+ ),
+ upgrade.NewPermanentTenantUpgrade(
+ "update system.locations with default location data",
+ toCV(clusterversion.VPrimordial6),
+ updateSystemLocationData,
+ ),
+ // Introduced in v2.1.
+ // TODO(mberhault): bake into v19.1.
+ upgrade.NewPermanentTenantUpgrade(
+ "disallow public user or role name",
+ toCV(clusterversion.VPrimordial7),
+ disallowPublicUserOrRole,
+ ),
+ // Introduced in v2.1.
+ // TODO(knz): bake this migration into v19.1.
+ upgrade.NewPermanentTenantUpgrade(
+ "create default databases",
+ toCV(clusterversion.VPrimordial8),
+ createDefaultDbs,
+ ),
upgrade.NewTenantUpgrade(
"ensure preconditions are met before starting upgrading to v22.2",
toCV(clusterversion.V22_2Start),
@@ -55,42 +108,42 @@ var upgrades = []upgrade.Upgrade{
upgrade.NewTenantUpgrade(
"upgrade sequences to be referenced by ID",
toCV(clusterversion.V22_2UpgradeSequenceToBeReferencedByID),
- NoPrecondition,
+ upgrade.NoPrecondition,
upgradeSequenceToBeReferencedByID,
),
upgrade.NewTenantUpgrade(
"update system.statement_diagnostics_requests to support sampling probabilities",
toCV(clusterversion.V22_2SampledStmtDiagReqs),
- NoPrecondition,
+ upgrade.NoPrecondition,
sampledStmtDiagReqsMigration,
),
upgrade.NewTenantUpgrade(
"add the system.privileges table",
toCV(clusterversion.V22_2SystemPrivilegesTable),
- NoPrecondition,
+ upgrade.NoPrecondition,
systemPrivilegesTableMigration,
),
upgrade.NewTenantUpgrade(
"add column locality to table system.sql_instances",
toCV(clusterversion.V22_2AlterSystemSQLInstancesAddLocality),
- NoPrecondition,
+ upgrade.NoPrecondition,
alterSystemSQLInstancesAddLocality,
),
upgrade.NewTenantUpgrade(
"add the system.external_connections table",
toCV(clusterversion.V22_2SystemExternalConnectionsTable),
- NoPrecondition,
+ upgrade.NoPrecondition,
systemExternalConnectionsTableMigration,
),
upgrade.NewTenantUpgrade(
"add column index_recommendations to table system.statement_statistics",
toCV(clusterversion.V22_2AlterSystemStatementStatisticsAddIndexRecommendations),
- NoPrecondition,
+ upgrade.NoPrecondition,
alterSystemStatementStatisticsAddIndexRecommendations,
),
upgrade.NewTenantUpgrade("add system.role_id_sequence",
toCV(clusterversion.V22_2RoleIDSequence),
- NoPrecondition,
+ upgrade.NoPrecondition,
roleIDSequenceMigration,
),
// Add user_id column, the column will not be backfilled.
@@ -101,38 +154,38 @@ var upgrades = []upgrade.Upgrade{
// more users can be created without ids.
upgrade.NewTenantUpgrade("alter system.users to include user_id column",
toCV(clusterversion.V22_2AddSystemUserIDColumn),
- NoPrecondition,
+ upgrade.NoPrecondition,
alterSystemUsersAddUserIDColumnWithIndex,
),
upgrade.NewTenantUpgrade("backfill users with ids and add an index on the id column",
toCV(clusterversion.V22_2SystemUsersIDColumnIsBackfilled),
- NoPrecondition,
+ upgrade.NoPrecondition,
backfillSystemUsersIDColumn,
),
upgrade.NewTenantUpgrade("set user_id column to not null",
toCV(clusterversion.V22_2SetSystemUsersUserIDColumnNotNull),
- NoPrecondition,
+ upgrade.NoPrecondition,
setUserIDNotNull,
),
upgrade.NewTenantUpgrade(
"add default SQL schema telemetry schedule",
toCV(clusterversion.V22_2SQLSchemaTelemetryScheduledJobs),
- NoPrecondition,
+ upgrade.NoPrecondition,
ensureSQLSchemaTelemetrySchedule,
),
upgrade.NewTenantUpgrade("alter system.role_options to include user_id column",
toCV(clusterversion.V22_2RoleOptionsTableHasIDColumn),
- NoPrecondition,
+ upgrade.NoPrecondition,
alterSystemRoleOptionsAddUserIDColumnWithIndex,
),
upgrade.NewTenantUpgrade("backfill entries in system.role_options to include IDs",
toCV(clusterversion.V22_2RoleOptionsIDColumnIsBackfilled),
- NoPrecondition,
+ upgrade.NoPrecondition,
backfillSystemRoleOptionsIDColumn,
),
upgrade.NewTenantUpgrade("set system.role_options user_id column to not null",
toCV(clusterversion.V22_2SetRoleOptionsUserIDColumnNotNull),
- NoPrecondition,
+ upgrade.NoPrecondition,
setSystemRoleOptionsUserIDColumnNotNull,
),
upgrade.NewTenantUpgrade("ensure all GC jobs send DeleteRange requests",
@@ -143,32 +196,32 @@ var upgrades = []upgrade.Upgrade{
upgrade.NewTenantUpgrade(
"wait for all in-flight schema changes",
toCV(clusterversion.V22_2NoNonMVCCAddSSTable),
- NoPrecondition,
+ upgrade.NoPrecondition,
waitForAllSchemaChanges,
),
upgrade.NewTenantUpgrade("update invalid column IDs in sequence back references",
toCV(clusterversion.V22_2UpdateInvalidColumnIDsInSequenceBackReferences),
- NoPrecondition,
+ upgrade.NoPrecondition,
updateInvalidColumnIDsInSequenceBackReferences,
),
upgrade.NewTenantUpgrade("fix corrupt user-file related table descriptors",
toCV(clusterversion.V22_2FixUserfileRelatedDescriptorCorruption),
- NoPrecondition,
+ upgrade.NoPrecondition,
fixInvalidObjectsThatLookLikeBadUserfileConstraint,
),
upgrade.NewTenantUpgrade("add a name column to system.tenants and populate a system tenant entry",
toCV(clusterversion.V23_1TenantNames),
- NoPrecondition,
+ upgrade.NoPrecondition,
addTenantNameColumnAndSystemTenantEntry,
),
upgrade.NewTenantUpgrade("set the value or system.descriptor_id_seq for the system tenant",
toCV(clusterversion.V23_1DescIDSequenceForSystemTenant),
- NoPrecondition,
+ upgrade.NoPrecondition,
descIDSequenceForSystemTenant,
),
upgrade.NewTenantUpgrade("add a partial predicate column to system.table_statistics",
toCV(clusterversion.V23_1AddPartialStatisticsPredicateCol),
- NoPrecondition,
+ upgrade.NoPrecondition,
alterSystemTableStatisticsAddPartialPredicate,
),
}
|