Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
114307: sql: remove pre-23.1 version gates r=RaduBerinde a=RaduBerinde

This PR is only for the top commit (it is on top of cockroachdb#114306).

This change removes pre-23.1 version gates in SQL code.
    
Informs: cockroachdb#112501
Release note: None

Co-authored-by: Radu Berinde <[email protected]>
  • Loading branch information
craig[bot] and RaduBerinde committed Nov 15, 2023
2 parents e724061 + 03bcb7a commit 7f96bac
Show file tree
Hide file tree
Showing 51 changed files with 201 additions and 1,228 deletions.
178 changes: 24 additions & 154 deletions pkg/clusterversion/cockroach_versions.go

Large diffs are not rendered by default.

1 change: 1 addition & 0 deletions pkg/jobs/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -145,6 +145,7 @@ go_test(
"//pkg/sql/sem/tree",
"//pkg/sql/sessiondata",
"//pkg/sql/sqlliveness",
"//pkg/sql/sqlliveness/slstorage",
"//pkg/testutils",
"//pkg/testutils/jobutils",
"//pkg/testutils/serverutils",
Expand Down
10 changes: 8 additions & 2 deletions pkg/jobs/registry_external_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -37,6 +37,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/sessiondata"
"github.com/cockroachdb/cockroach/pkg/sql/sqlliveness"
"github.com/cockroachdb/cockroach/pkg/sql/sqlliveness/slstorage"
"github.com/cockroachdb/cockroach/pkg/testutils"
"github.com/cockroachdb/cockroach/pkg/testutils/serverutils"
"github.com/cockroachdb/cockroach/pkg/testutils/skip"
Expand Down Expand Up @@ -112,12 +113,17 @@ RETURNING id;
terminalStatuses := []jobs.Status{jobs.StatusSucceeded, jobs.StatusCanceled, jobs.StatusFailed}
terminalIDs := make([]jobspb.JobID, len(terminalStatuses))
terminalClaims := make([][]byte, len(terminalStatuses))
mkSessionID := func() []byte {
sessionID, err := slstorage.MakeSessionID([]byte("us"), uuid.MakeV4())
require.NoError(t, err)
return []byte(sessionID)
}
for i, s := range terminalStatuses {
terminalClaims[i] = uuid.MakeV4().GetBytes() // bogus claim
terminalClaims[i] = mkSessionID() // bogus claim
tdb.QueryRow(t, insertQuery, s, terminalClaims[i], 42).Scan(&terminalIDs[i])
}
var nonTerminalID jobspb.JobID
tdb.QueryRow(t, insertQuery, jobs.StatusRunning, uuid.MakeV4().GetBytes(), 42).Scan(&nonTerminalID)
tdb.QueryRow(t, insertQuery, jobs.StatusRunning, mkSessionID(), 42).Scan(&nonTerminalID)

checkClaimEqual := func(id jobspb.JobID, exp []byte) error {
const getClaimQuery = `SELECT claim_session_id FROM system.jobs WHERE id = $1`
Expand Down
6 changes: 3 additions & 3 deletions pkg/server/settingswatcher/version_guard_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -109,9 +109,9 @@ func TestVersionGuard(t *testing.T) {
storageVersion: &initialVersion,
settingsVersion: maxVersion,
checkVersions: map[clusterversion.Key]bool{
initialVersion: true,
maxVersion: true,
clusterversion.TODO_Delete_V23_1Start: true,
initialVersion: true,
startVersion: true,
maxVersion: true,
},
},
}
Expand Down
16 changes: 3 additions & 13 deletions pkg/sql/alter_role.go
Original file line number Diff line number Diff line change
Expand Up @@ -15,7 +15,6 @@ import (
"fmt"
"strings"

"github.com/cockroachdb/cockroach/pkg/clusterversion"
"github.com/cockroachdb/cockroach/pkg/security/username"
"github.com/cockroachdb/cockroach/pkg/settings"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
Expand Down Expand Up @@ -428,9 +427,6 @@ func (p *planner) processSetOrResetClause(
}

func (n *alterRoleSetNode) startExec(params runParams) error {
databaseRoleSettingsHasRoleIDCol := params.p.ExecCfg().Settings.Version.IsActive(params.ctx,
clusterversion.TODO_Delete_V23_1DatabaseRoleSettingsHasRoleIDColumn)

var opName string
if n.isRole {
sqltelemetry.IncIAMAlterCounter(sqltelemetry.Role)
Expand All @@ -454,22 +450,16 @@ func (n *alterRoleSetNode) startExec(params runParams) error {
sessioninit.DatabaseRoleSettingsTableName,
)

var upsertQuery = fmt.Sprintf(
`UPSERT INTO %s (database_id, role_name, settings) VALUES ($1, $2, $3)`,
sessioninit.DatabaseRoleSettingsTableName,
)
if databaseRoleSettingsHasRoleIDCol {
upsertQuery = fmt.Sprintf(`
var upsertQuery = fmt.Sprintf(`
UPSERT INTO %s (database_id, role_name, settings, role_id)
VALUES ($1, $2, $3, (
SELECT CASE $2
WHEN '%s' THEN %d
ELSE (SELECT user_id FROM system.users WHERE username = $2)
END
))`,
sessioninit.DatabaseRoleSettingsTableName, username.EmptyRole, username.EmptyRoleID,
)
}
sessioninit.DatabaseRoleSettingsTableName, username.EmptyRole, username.EmptyRoleID,
)

// Instead of inserting an empty settings array, this function will make
// sure the row is deleted instead.
Expand Down
38 changes: 2 additions & 36 deletions pkg/sql/alter_table.go
Original file line number Diff line number Diff line change
Expand Up @@ -1341,49 +1341,15 @@ func insertJSONStatistic(
histogram interface{},
) error {
var (
ctx = params.ctx
txn = params.p.InternalSQLTxn()
settings = params.ExecCfg().Settings
ctx = params.ctx
txn = params.p.InternalSQLTxn()
)

var name interface{}
if s.Name != "" {
name = s.Name
}

if !settings.Version.IsActive(ctx, clusterversion.TODO_Delete_V23_1AddPartialStatisticsColumns) {

if s.PartialPredicate != "" {
return pgerror.Newf(pgcode.ObjectNotInPrerequisiteState, "statistic for columns %v with collection time %s to insert is partial but cluster version is below 23.1", s.Columns, s.CreatedAt)
}

_ /* rows */, err := txn.Exec(
ctx,
"insert-stats",
txn.KV(),
`INSERT INTO system.table_statistics (
"tableID",
"name",
"columnIDs",
"createdAt",
"rowCount",
"distinctCount",
"nullCount",
"avgSize",
histogram
) VALUES ($1, $2, $3, $4, $5, $6, $7, $8, $9)`,
tableID,
name,
columnIDs,
s.CreatedAt,
s.RowCount,
s.DistinctCount,
s.NullCount,
s.AvgSize,
histogram)
return err
}

var predicateValue interface{}
if s.PartialPredicate != "" {
predicateValue = s.PartialPredicate
Expand Down
1 change: 0 additions & 1 deletion pkg/sql/catalog/descidgen/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -6,7 +6,6 @@ go_library(
importpath = "github.com/cockroachdb/cockroach/pkg/sql/catalog/descidgen",
visibility = ["//visibility:public"],
deps = [
"//pkg/clusterversion",
"//pkg/keys",
"//pkg/kv",
"//pkg/roachpb",
Expand Down
20 changes: 0 additions & 20 deletions pkg/sql/catalog/descidgen/generate_id.go
Original file line number Diff line number Diff line change
Expand Up @@ -13,7 +13,6 @@ package descidgen
import (
"context"

"github.com/cockroachdb/cockroach/pkg/clusterversion"
"github.com/cockroachdb/cockroach/pkg/keys"
"github.com/cockroachdb/cockroach/pkg/kv"
"github.com/cockroachdb/cockroach/pkg/roachpb"
Expand Down Expand Up @@ -65,15 +64,6 @@ func (g *generator) run(ctx context.Context, inc int64) (catid.DescID, error) {
return catid.DescID(nextID), err
}

// ErrDescIDSequenceMigrationInProgress is the error returned when the
// descriptor ID generator is unavailable due to migrating the system tenant
// counter from keys.LegacyDescIDGenerator to system.descriptor_id_seq.
//
// TODO(postamar): remove along with clusterversion.TODO_Delete_V23_1DescIDSequenceForSystemTenant
var ErrDescIDSequenceMigrationInProgress = errors.New(
"descriptor ID generator unavailable, migration in progress, retry later",
)

// NewGenerator constructs a non-transactional eval.DescIDGenerator.
//
// In this implementation the value returned by PeekNextUniqueDescID is _not_
Expand Down Expand Up @@ -108,16 +98,6 @@ func key(
ctx context.Context, codec keys.SQLCodec, settings *cluster.Settings,
) (roachpb.Key, error) {
key := codec.SequenceKey(keys.DescIDSequenceID)
if cv := settings.Version; codec.ForSystemTenant() &&
!cv.IsActive(ctx, clusterversion.TODO_Delete_V23_1DescIDSequenceForSystemTenant) {
// At this point, the system tenant may still be using a legacy non-SQL key,
// or may be in the process of undergoing the migration away from it, in
// which case descriptor ID generation is made unavailable.
if cv.IsActive(ctx, clusterversion.TODO_Delete_V23_1DescIDSequenceForSystemTenant-1) {
return nil, ErrDescIDSequenceMigrationInProgress
}
key = keys.LegacyDescIDGenerator
}
return key, nil
}

Expand Down
1 change: 0 additions & 1 deletion pkg/sql/catalog/lease/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,6 @@ go_library(
visibility = ["//visibility:public"],
deps = [
"//pkg/base",
"//pkg/clusterversion",
"//pkg/keys",
"//pkg/kv",
"//pkg/kv/kvclient/rangefeed",
Expand Down
58 changes: 10 additions & 48 deletions pkg/sql/catalog/lease/kv_writer.go
Original file line number Diff line number Diff line change
Expand Up @@ -13,7 +13,6 @@ package lease
import (
"context"

"github.com/cockroachdb/cockroach/pkg/clusterversion"
"github.com/cockroachdb/cockroach/pkg/keys"
"github.com/cockroachdb/cockroach/pkg/kv"
"github.com/cockroachdb/cockroach/pkg/server/settingswatcher"
Expand Down Expand Up @@ -58,63 +57,35 @@ func leaseTableWithID(id descpb.ID) catalog.TableDescriptor {
return mut.ImmutableCopy().(catalog.TableDescriptor)
}

func (w *kvWriter) versionGuard(
ctx context.Context, txn *kv.Txn,
) (settingswatcher.VersionGuard, error) {
return w.settingsWatcher.MakeVersionGuard(ctx, txn, clusterversion.TODO_Delete_V23_1_SystemRbrCleanup)
}

func (w *kvWriter) insertLease(ctx context.Context, txn *kv.Txn, l leaseFields) error {
return w.do(ctx, txn, l, func(guard settingswatcher.VersionGuard, b *kv.Batch) error {
if guard.IsActive(clusterversion.TODO_Delete_V23_1_SystemRbrDualWrite) {
err := w.newWriter.Insert(ctx, b, false /*kvTrace */, leaseAsRbrDatum(l)...)
if err != nil {
return err
}
}
if !guard.IsActive(clusterversion.TODO_Delete_V23_1_SystemRbrSingleWrite) {
err := w.oldWriter.Insert(ctx, b, false /*kvTrace */, leaseAsRbtDatum(l)...)
if err != nil {
return err
}
return w.do(ctx, txn, l, func(b *kv.Batch) error {
err := w.newWriter.Insert(ctx, b, false /*kvTrace */, leaseAsRbrDatum(l)...)
if err != nil {
return err
}
return nil
})
}

func (w *kvWriter) deleteLease(ctx context.Context, txn *kv.Txn, l leaseFields) error {
return w.do(ctx, txn, l, func(guard settingswatcher.VersionGuard, b *kv.Batch) error {
if guard.IsActive(clusterversion.TODO_Delete_V23_1_SystemRbrDualWrite) {
err := w.newWriter.Delete(ctx, b, false /*kvTrace */, leaseAsRbrDatum(l)...)
if err != nil {
return err
}
}
if !guard.IsActive(clusterversion.TODO_Delete_V23_1_SystemRbrSingleWrite) {
err := w.oldWriter.Delete(ctx, b, false /*kvTrace */, leaseAsRbtDatum(l)...)
if err != nil {
return err
}
return w.do(ctx, txn, l, func(b *kv.Batch) error {
err := w.newWriter.Delete(ctx, b, false /*kvTrace */, leaseAsRbrDatum(l)...)
if err != nil {
return err
}
return nil
})
}

type addToBatchFunc = func(settingswatcher.VersionGuard, *kv.Batch) error
type addToBatchFunc = func(*kv.Batch) error

func (w *kvWriter) do(
ctx context.Context, txn *kv.Txn, lease leaseFields, addToBatch addToBatchFunc,
) error {
run := (*kv.Txn).Run
do := func(ctx context.Context, txn *kv.Txn) error {
guard, err := w.versionGuard(ctx, txn)
if err != nil {
return err
}

b := txn.NewBatch()
err = addToBatch(guard, b)
if err != nil {
if err := addToBatch(b); err != nil {
return errors.NewAssertionErrorWithWrappedErrf(err, "failed to encode lease entry")
}
return run(txn, ctx, b)
Expand All @@ -136,12 +107,3 @@ func leaseAsRbrDatum(l leaseFields) []tree.Datum {
}

}

func leaseAsRbtDatum(l leaseFields) []tree.Datum {
return []tree.Datum{
tree.NewDInt(tree.DInt(l.descID)),
tree.NewDInt(tree.DInt(l.version)),
tree.NewDInt(tree.DInt(l.instanceID)),
&l.expiration,
}
}
12 changes: 1 addition & 11 deletions pkg/sql/catalog/lease/lease.go
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,6 @@ import (
"time"

"github.com/cockroachdb/cockroach/pkg/base"
"github.com/cockroachdb/cockroach/pkg/clusterversion"
"github.com/cockroachdb/cockroach/pkg/keys"
"github.com/cockroachdb/cockroach/pkg/kv"
"github.com/cockroachdb/cockroach/pkg/kv/kvclient/rangefeed"
Expand Down Expand Up @@ -1400,18 +1399,9 @@ func (m *Manager) DeleteOrphanedLeases(ctx context.Context, timeThreshold int64)
// doesn't implement AS OF SYSTEM TIME.

// Read orphaned leases.
const (
queryWithRegion = `
query := `
SELECT "descID", version, expiration, crdb_region FROM system.public.lease AS OF SYSTEM TIME %d WHERE "nodeID" = %d
`
queryWithoutRegion = `
SELECT "descID", version, expiration FROM system.public.lease AS OF SYSTEM TIME %d WHERE "nodeID" = %d
`
)
query := queryWithRegion
if !m.settings.Version.IsActive(ctx, clusterversion.TODO_Delete_V23_1_SystemRbrReadNew) {
query = queryWithoutRegion
}
sqlQuery := fmt.Sprintf(query, timeThreshold, instanceID)
var rows []tree.Datums
retryOptions := base.DefaultRetryOptions()
Expand Down
1 change: 0 additions & 1 deletion pkg/sql/colexec/colbuilder/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -9,7 +9,6 @@ go_library(
importpath = "github.com/cockroachdb/cockroach/pkg/sql/colexec/colbuilder",
visibility = ["//visibility:public"],
deps = [
"//pkg/clusterversion",
"//pkg/col/coldata",
"//pkg/col/coldataext",
"//pkg/col/typeconv",
Expand Down
4 changes: 0 additions & 4 deletions pkg/sql/colexec/colbuilder/execplan.go
Original file line number Diff line number Diff line change
Expand Up @@ -15,7 +15,6 @@ import (
"reflect"
"strings"

"github.com/cockroachdb/cockroach/pkg/clusterversion"
"github.com/cockroachdb/cockroach/pkg/col/coldata"
"github.com/cockroachdb/cockroach/pkg/col/coldataext"
"github.com/cockroachdb/cockroach/pkg/col/typeconv"
Expand Down Expand Up @@ -835,9 +834,6 @@ func NewColOperator(
var resultTypes []*types.T
if flowCtx.EvalCtx.SessionData().DirectColumnarScansEnabled {
canUseDirectScan := func() bool {
if !flowCtx.EvalCtx.Settings.Version.IsActive(ctx, clusterversion.TODO_Delete_V23_1_KVDirectColumnarScans) {
return false
}
// We currently don't use the direct scans if TraceKV is
// enabled (due to not being able to tell the KV server
// about it). One idea would be to include this boolean into
Expand Down
18 changes: 0 additions & 18 deletions pkg/sql/conn_executor.go
Original file line number Diff line number Diff line change
Expand Up @@ -3287,10 +3287,6 @@ var retriableMinTimestampBoundUnsatisfiableError = errors.Newf(
func errIsRetriable(err error) bool {
return errors.HasInterface(err, (*pgerror.ClientVisibleRetryError)(nil)) ||
errors.Is(err, retriableMinTimestampBoundUnsatisfiableError) ||
// Note that this error is not handled internally and can make it to the
// client in implicit transactions. This is not great; it should
// be marked as a client visible retry error.
errors.Is(err, descidgen.ErrDescIDSequenceMigrationInProgress) ||
descs.IsTwoVersionInvariantViolationError(err)
}

Expand Down Expand Up @@ -3752,13 +3748,6 @@ func (ex *connExecutor) txnStateTransitionsApplyWrapper(
return advanceInfo{}, err
}
}
// Similarly, if the descriptor ID generator is not available because of
// an ongoing migration, wait for the migration to complete first.
if errors.Is(payloadErr, descidgen.ErrDescIDSequenceMigrationInProgress) {
if err := ex.handleWaitingForDescriptorIDGeneratorMigration(ex.Ctx()); err != nil {
return advanceInfo{}, err
}
}
}

// Handle transaction events which cause updates to txnState.
Expand Down Expand Up @@ -3922,13 +3911,6 @@ func (ex *connExecutor) handleWaitingForConcurrentSchemaChanges(
return ex.resetTransactionOnSchemaChangeRetry(ctx)
}

func (ex *connExecutor) handleWaitingForDescriptorIDGeneratorMigration(ctx context.Context) error {
if err := ex.planner.waitForDescriptorIDGeneratorMigration(ctx); err != nil {
return err
}
return ex.resetTransactionOnSchemaChangeRetry(ctx)
}

// initStatementResult initializes res according to a query.
//
// cols represents the columns of the result rows. Should be nil if
Expand Down
Loading

0 comments on commit 7f96bac

Please sign in to comment.