diff --git a/pkg/clusterversion/BUILD.bazel b/pkg/clusterversion/BUILD.bazel
index 15663edff181..c3b11920778c 100644
--- a/pkg/clusterversion/BUILD.bazel
+++ b/pkg/clusterversion/BUILD.bazel
@@ -21,6 +21,7 @@ go_library(
     deps = [
         "//pkg/roachpb",
         "//pkg/settings",
+        "//pkg/util/envutil",
         "//pkg/util/log",
         "//pkg/util/metric",
         "//pkg/util/protoutil",
diff --git a/pkg/clusterversion/cockroach_versions.go b/pkg/clusterversion/cockroach_versions.go
index 0482d97b874a..229068bf9faf 100644
--- a/pkg/clusterversion/cockroach_versions.go
+++ b/pkg/clusterversion/cockroach_versions.go
@@ -10,7 +10,10 @@
 
 package clusterversion
 
-import "github.com/cockroachdb/cockroach/pkg/roachpb"
+import (
+	"github.com/cockroachdb/cockroach/pkg/roachpb"
+	"github.com/cockroachdb/cockroach/pkg/util/envutil"
+)
 
 // Key is a unique identifier for a version of CockroachDB.
 type Key int
@@ -472,13 +475,10 @@ var rawVersionsSingleton = keyedVersions{
 }
 
 const (
-	// unstableVersionsAbove is a cluster version Key above which any upgrades in
-	// this version are considered unstable development-only versions if it is not
-	// negative, and upgrading to them should permanently move a cluster to
-	// development versions. On master it should be the minted version of the last
-	// release, while on release branches it can be set to invalidVersionKey to
-	// disable marking any versions as development versions.
-	unstableVersionsAbove = V22_1
+	// developmentBranch should be toggled to false on a release branch once the
+	// set of versions becomes append-only and associated upgrade implementations
+	// are frozen. It is always true on the main development branch.
+	developmentBranch = true
 
 	// finalVersion should be set on a release branch to the minted final cluster
 	// version key, e.g. to V22_2 on the release-22.2 branch once it is minted.
@@ -486,14 +486,27 @@ const (
 	finalVersion = invalidVersionKey
 )
 
+// devVersionsAbove is the version key above which all versions are offset to be
+// development version when developmentBranch is true. By default this is all
+// versions, by setting this to -1, but an env var can override this, to leave
+// the first version un-offset. Doing so means that that version, which is
+// generally minBinaryVersion as well, is unchanged, and thus allows upgrading a
+// stable release data-dir to a dev version if desired.
+var devVersionsAbove Key = func() Key {
+	if envutil.EnvOrDefaultBool("COCKROACH_UPGRADE_TO_DEV_VERSION", false) {
+		return invalidVersionKey + 1
+	}
+	return invalidVersionKey
+}()
+
 var versionsSingleton = func() keyedVersions {
-	if unstableVersionsAbove > invalidVersionKey {
+	if developmentBranch {
 		const devOffset = 1000000
 		// Throw every version above the last release (which will be none on a release
 		// branch) 1 million major versions into the future, so any "upgrade" to a
 		// release branch build will be a downgrade and thus blocked.
 		for i := range rawVersionsSingleton {
-			if rawVersionsSingleton[i].Key > unstableVersionsAbove {
+			if rawVersionsSingleton[i].Key > devVersionsAbove {
 				rawVersionsSingleton[i].Major += devOffset
 			}
 		}
diff --git a/pkg/cmd/roachtest/tests/versionupgrade.go b/pkg/cmd/roachtest/tests/versionupgrade.go
index e59562a984ff..25549ad6cccf 100644
--- a/pkg/cmd/roachtest/tests/versionupgrade.go
+++ b/pkg/cmd/roachtest/tests/versionupgrade.go
@@ -431,6 +431,7 @@ func upgradeNodes(
 
 		binary := uploadVersion(ctx, t, c, c.Node(node), newVersion)
 		settings := install.MakeClusterSettings(install.BinaryOption(binary))
+		settings.Env = append(settings.Env, "COCKROACH_UPGRADE_TO_DEV_VERSION=true")
 		c.Start(ctx, t.L(), startOpts, settings, c.Node(node))
 	}
 }
diff --git a/pkg/kv/kvserver/client_migration_test.go b/pkg/kv/kvserver/client_migration_test.go
index 6efacf3226ad..4e41acb8ede4 100644
--- a/pkg/kv/kvserver/client_migration_test.go
+++ b/pkg/kv/kvserver/client_migration_test.go
@@ -240,8 +240,8 @@ func TestMigrateWaitsForApplication(t *testing.T) {
 	blockApplicationCh := make(chan struct{})
 
 	// We're going to be migrating from startV to endV.
-	startV := roachpb.Version{Major: 41}
-	endV := roachpb.Version{Major: 42}
+	startV := roachpb.Version{Major: 1000041}
+	endV := roachpb.Version{Major: 1000042}
 
 	ctx := context.Background()
 	tc := testcluster.StartTestCluster(t, 3, base.TestClusterArgs{
diff --git a/pkg/sql/logictest/logic.go b/pkg/sql/logictest/logic.go
index 41a81b511011..47e8c9c32252 100644
--- a/pkg/sql/logictest/logic.go
+++ b/pkg/sql/logictest/logic.go
@@ -1207,9 +1207,9 @@ func (t *logicTest) newCluster(
 	tempStorageDiskLimit := int64(512 << 20) /* 512 MiB */
 	// MVCC range tombstones are only available in 22.2 or newer.
 	supportsMVCCRangeTombstones := (t.cfg.BootstrapVersion.Equal(roachpb.Version{}) ||
-		!t.cfg.BootstrapVersion.Less(roachpb.Version{Major: 22, Minor: 2})) &&
+		!t.cfg.BootstrapVersion.Less(clusterversion.ByKey(clusterversion.SetSystemUsersUserIDColumnNotNull))) &&
 		(t.cfg.BinaryVersion.Equal(roachpb.Version{}) ||
-			!t.cfg.BinaryVersion.Less(roachpb.Version{Major: 22, Minor: 2}))
+			!t.cfg.BinaryVersion.Less(clusterversion.ByKey(clusterversion.SetSystemUsersUserIDColumnNotNull)))
 	ignoreMVCCRangeTombstoneErrors := supportsMVCCRangeTombstones &&
 		(globalMVCCRangeTombstone || useMVCCRangeTombstonesForPointDeletes)
 
@@ -1708,7 +1708,7 @@ CREATE DATABASE test; USE test;
 		t.Fatal(err)
 	}
 
-	if !t.cfg.BootstrapVersion.Equal(roachpb.Version{}) && t.cfg.BootstrapVersion.Less(roachpb.Version{Major: 22, Minor: 2}) {
+	if !t.cfg.BootstrapVersion.Equal(roachpb.Version{}) && t.cfg.BootstrapVersion.Less(clusterversion.ByKey(clusterversion.SetSystemUsersUserIDColumnNotNull)) {
 		// Hacky way to create user with an ID if we're on a
 		// bootstrapped binary less than 22.2. The version gate
 		// causes the regular CREATE USER to fail since it will not
diff --git a/pkg/sql/logictest/logictestbase/BUILD.bazel b/pkg/sql/logictest/logictestbase/BUILD.bazel
index 8125bcc75765..4552df7c46c9 100644
--- a/pkg/sql/logictest/logictestbase/BUILD.bazel
+++ b/pkg/sql/logictest/logictestbase/BUILD.bazel
@@ -9,6 +9,7 @@ go_library(
     visibility = ["//visibility:public"],
     deps = [
         "//pkg/build",
+        "//pkg/clusterversion",
         "//pkg/roachpb",
         "//pkg/util",
     ],
diff --git a/pkg/sql/logictest/logictestbase/logictestbase.go b/pkg/sql/logictest/logictestbase/logictestbase.go
index a4fb3743cf28..8d30585deff5 100644
--- a/pkg/sql/logictest/logictestbase/logictestbase.go
+++ b/pkg/sql/logictest/logictestbase/logictestbase.go
@@ -22,6 +22,7 @@ import (
 	"strings"
 
 	"github.com/cockroachdb/cockroach/pkg/build"
+	"github.com/cockroachdb/cockroach/pkg/clusterversion"
 	"github.com/cockroachdb/cockroach/pkg/roachpb"
 	"github.com/cockroachdb/cockroach/pkg/util"
 )
@@ -462,8 +463,8 @@ var LogicTestConfigs = []TestClusterConfig{
 		Name:                        "local-mixed-22.1-22.2",
 		NumNodes:                    1,
 		OverrideDistSQLMode:         "off",
-		BootstrapVersion:            roachpb.Version{Major: 22, Minor: 1},
-		BinaryVersion:               roachpb.Version{Major: 22, Minor: 2},
+		BootstrapVersion:            clusterversion.ByKey(clusterversion.V22_1),
+		BinaryVersion:               clusterversion.ByKey(clusterversion.PrioritizeSnapshots), //TODO: switch to 22.2.
 		DisableUpgrade:              true,
 		DeclarativeCorpusCollection: true,
 	},
diff --git a/pkg/upgrade/upgrades/builtins_test.go b/pkg/upgrade/upgrades/builtins_test.go
index a3bfa3cff216..405749fb2243 100644
--- a/pkg/upgrade/upgrades/builtins_test.go
+++ b/pkg/upgrade/upgrades/builtins_test.go
@@ -46,13 +46,14 @@ func TestIsAtLeastVersionBuiltin(t *testing.T) {
 	)
 	defer tc.Stopper().Stop(ctx)
 
+	v := clusterversion.ByKey(clusterversion.Start22_2).String()
 	// Check that the builtin returns false when comparing against 22.1-2
 	// version because we are still on 22.1-0.
-	sqlDB.CheckQueryResults(t, "SELECT crdb_internal.is_at_least_version('22.1-2')", [][]string{{"false"}})
+	sqlDB.CheckQueryResults(t, "SELECT crdb_internal.is_at_least_version('"+v+"')", [][]string{{"false"}})
 
 	// Run the upgrade.
-	sqlDB.Exec(t, "SET CLUSTER SETTING version = $1", clusterversion.ByKey(clusterversion.Start22_2).String())
+	sqlDB.Exec(t, "SET CLUSTER SETTING version = $1", v)
 
 	// It should now return true.
-	sqlDB.CheckQueryResultsRetry(t, "SELECT crdb_internal.is_at_least_version('22.1-2')", [][]string{{"true"}})
+	sqlDB.CheckQueryResultsRetry(t, "SELECT crdb_internal.is_at_least_version('"+v+"')", [][]string{{"true"}})
 }