Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
85864: clusterversion,kvserver: remove EnableLeaseHolderRemoval r=shralex a=celiala

This commit removes the 22.1 `EnableLeaseHolderRemoval` version gates.

Cleanup was done following guidance from [21.2 cleanup](#74270 (comment)):

> For the most part, if the gates were just simple if !version.IsActive { return x } or something, I just removed the block, and even if it was a little more complicated, like args = [x]; if version { args = append(args, y) }; foo(args) I still tried to mostly inline it such that it looked natural (i.e. remove that append and make it args = [x, y]).

> However for just a couple more complicated cases that were referring to <21.2 versions that needed to be replaced when those were deleted, I added a placeholder clusterversion.TODOPre21_2 alias for 21.2. Replacing those calls with this alias shouldn't change their behavior -- it was already always true, since the code today should never run in a <21.2 cluster -- but means we can delete those older versions in the meantime and then the owners of these bits can decide how to update them.

Partially addresses #80663

Release justification: remove version gate

Release note: none

86153: sql: step the transaction before auto-commit r=ajwerner a=ajwerner

We want the pre-commit validation to see the side-effects of all previous
statements. If we don't step the transaction, we won't see writes which
occurred in the last statement. The shocking thing is that this all worked
before this change. The reason it worked was that the internal executor usage
that happened as a part of logging every schema change resulted in the txn
being stepped. That is a separate, and more complex bug to fix. It will be
described and addressed separately.

Fixes #86132

Release justification: Important bug fix 

Release note: None

86419: ci: disable `verify-archive` CI job r=rail,jlinder a=rickystewart

Release justification: Non-production code changes
Release note: None

86430: logictest: add sort order to logic test r=ajwerner a=adityamaru

Fixes a flakey test that lacked a sort order.

Release note: None

Release justification: test only change to fix a flake

Co-authored-by: Celia La <[email protected]>
Co-authored-by: Andrew Werner <[email protected]>
Co-authored-by: Ricky Stewart <[email protected]>
Co-authored-by: Aditya Maru <[email protected]>
  • Loading branch information
5 people committed Aug 19, 2022
5 parents a0d8839 + 9f5f60b + f28caf3 + 8954370 + 2f9d8be commit 8826516
Show file tree
Hide file tree
Showing 14 changed files with 72 additions and 134 deletions.
41 changes: 0 additions & 41 deletions build/teamcity-verify-archive.sh

This file was deleted.

7 changes: 0 additions & 7 deletions pkg/clusterversion/cockroach_versions.go
Original file line number Diff line number Diff line change
Expand Up @@ -190,9 +190,6 @@ const (
// engine running at the required format major version, as do all other nodes
// in the cluster.
EnablePebbleFormatVersionBlockProperties
// EnableLeaseHolderRemoval enables removing a leaseholder and transferring the lease
// during joint configuration, including to VOTER_INCOMING replicas.
EnableLeaseHolderRemoval
// ChangefeedIdleness is the version where changefeed aggregators forward
// idleness-related information alnog with resolved spans to the frontier
ChangefeedIdleness
Expand Down Expand Up @@ -386,10 +383,6 @@ var versionsSingleton = keyedVersions{
Key: EnablePebbleFormatVersionBlockProperties,
Version: roachpb.Version{Major: 21, Minor: 2, Internal: 64},
},
{
Key: EnableLeaseHolderRemoval,
Version: roachpb.Version{Major: 21, Minor: 2, Internal: 70},
},
{
Key: ChangefeedIdleness,
Version: roachpb.Version{Major: 21, Minor: 2, Internal: 82},
Expand Down
77 changes: 38 additions & 39 deletions pkg/clusterversion/key_string.go

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

3 changes: 1 addition & 2 deletions pkg/kv/kvserver/allocator/allocatorimpl/allocator.go
Original file line number Diff line number Diff line change
Expand Up @@ -1478,9 +1478,8 @@ func (a *Allocator) ValidLeaseTargets(
) []roachpb.ReplicaDescriptor {
candidates := make([]roachpb.ReplicaDescriptor, 0, len(existing))
replDescs := roachpb.MakeReplicaSet(existing)
lhRemovalAllowed := a.StorePool.St.Version.IsActive(ctx, clusterversion.EnableLeaseHolderRemoval)
for i := range existing {
if err := roachpb.CheckCanReceiveLease(existing[i], replDescs, lhRemovalAllowed); err != nil {
if err := roachpb.CheckCanReceiveLease(existing[i], replDescs, true /* lhRemovalAllowed */); err != nil {
continue
}
// If we're not allowed to include the current replica, remove it from
Expand Down
1 change: 0 additions & 1 deletion pkg/kv/kvserver/batcheval/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -59,7 +59,6 @@ go_library(
deps = [
"//pkg/cloud",
"//pkg/cloud/cloudpb",
"//pkg/clusterversion",
"//pkg/keys",
"//pkg/kv/kvserver/abortspan",
"//pkg/kv/kvserver/batcheval/result",
Expand Down
5 changes: 1 addition & 4 deletions pkg/kv/kvserver/batcheval/cmd_lease_request.go
Original file line number Diff line number Diff line change
Expand Up @@ -14,7 +14,6 @@ import (
"context"
"time"

"github.com/cockroachdb/cockroach/pkg/clusterversion"
"github.com/cockroachdb/cockroach/pkg/keys"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval/result"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/readsummary/rspb"
Expand Down Expand Up @@ -68,12 +67,10 @@ func RequestLease(
Requested: args.Lease,
}

lhRemovalAllowed :=
cArgs.EvalCtx.ClusterSettings().Version.IsActive(ctx, clusterversion.EnableLeaseHolderRemoval)
// If this check is removed at some point, the filtering of learners on the
// sending side would have to be removed as well.
if err := roachpb.CheckCanReceiveLease(args.Lease.Replica, cArgs.EvalCtx.Desc().Replicas(),
lhRemovalAllowed,
true, /* lhRemovalAllowed */
); err != nil {
rErr.Message = err.Error()
return newFailedLeaseTrigger(false /* isTransfer */), rErr
Expand Down
5 changes: 1 addition & 4 deletions pkg/kv/kvserver/batcheval/cmd_lease_transfer.go
Original file line number Diff line number Diff line change
Expand Up @@ -14,7 +14,6 @@ import (
"context"
"time"

"github.com/cockroachdb/cockroach/pkg/clusterversion"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval/result"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/readsummary/rspb"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/spanset"
Expand Down Expand Up @@ -77,12 +76,10 @@ func TransferLease(
newLease := args.Lease
args.Lease = roachpb.Lease{} // prevent accidental use below

lhRemovalAllowed := cArgs.EvalCtx.ClusterSettings().Version.IsActive(ctx,
clusterversion.EnableLeaseHolderRemoval)
// If this check is removed at some point, the filtering of learners on the
// sending side would have to be removed as well.
if err := roachpb.CheckCanReceiveLease(
newLease.Replica, cArgs.EvalCtx.Desc().Replicas(), lhRemovalAllowed,
newLease.Replica, cArgs.EvalCtx.Desc().Replicas(), true, /* lhRemovalAllowed */
); err != nil {
return newFailedLeaseTrigger(true /* isTransfer */), err
}
Expand Down
4 changes: 1 addition & 3 deletions pkg/kv/kvserver/replica_command.go
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,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/kvserver/allocator"
Expand Down Expand Up @@ -3322,8 +3321,7 @@ func (r *Replica) relocateOne(
}
// Determines whether we can remove the leaseholder without first
// transferring the lease away.
lhRemovalAllowed = len(args.votersToAdd) > 0 &&
r.store.cfg.Settings.Version.IsActive(ctx, clusterversion.EnableLeaseHolderRemoval)
lhRemovalAllowed = len(args.votersToAdd) > 0
curLeaseholder := b.RawResponse().Responses[0].GetLeaseInfo().Lease.Replica
shouldRemove = (curLeaseholder.StoreID != removalTarget.StoreID) || lhRemovalAllowed
if args.targetType == allocatorimpl.VoterTarget {
Expand Down
5 changes: 1 addition & 4 deletions pkg/kv/kvserver/replica_proposal_buf.go
Original file line number Diff line number Diff line change
Expand Up @@ -15,7 +15,6 @@ import (
"sync"
"sync/atomic"

"github.com/cockroachdb/cockroach/pkg/clusterversion"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts/tracker"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/raftutil"
Expand Down Expand Up @@ -1208,9 +1207,7 @@ func (rp *replicaProposer) leaderStatusRLocked(
// lease again, and by then hopefully we will have caught up.
leaderEligibleForLease = true
} else {
lhRemovalAllowed := r.store.cfg.Settings.Version.IsActive(
ctx, clusterversion.EnableLeaseHolderRemoval)
err := roachpb.CheckCanReceiveLease(leaderRep, rangeDesc.Replicas(), lhRemovalAllowed)
err := roachpb.CheckCanReceiveLease(leaderRep, rangeDesc.Replicas(), true /* lhRemovalAllowed */)
leaderEligibleForLease = err == nil
}
}
Expand Down
7 changes: 2 additions & 5 deletions pkg/kv/kvserver/replica_raft.go
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,6 @@ import (
"strings"
"time"

"github.com/cockroachdb/cockroach/pkg/clusterversion"
"github.com/cockroachdb/cockroach/pkg/keys"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/apply"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency"
Expand Down Expand Up @@ -385,8 +384,6 @@ func (r *Replica) propose(
// will be used as a target for the lease transfer). Otherwise, the caller
// is expected to shed the lease before entering a joint configuration.
// See also https://github.com/cockroachdb/cockroach/issues/67740.
lhRemovalAllowed := r.store.cfg.Settings.Version.IsActive(
ctx, clusterversion.EnableLeaseHolderRemoval)
lhDesc, err := r.GetReplicaDescriptor()
if err != nil {
return roachpb.NewError(err)
Expand All @@ -400,11 +397,11 @@ func (r *Replica) propose(
// transferred away. The previous leaseholder is a LEARNER in the target config,
// and therefore shouldn't continue holding the lease.
if err := roachpb.CheckCanReceiveLease(
lhDesc, proposedDesc.Replicas(), lhRemovalAllowed,
lhDesc, proposedDesc.Replicas(), true, /* lhRemovalAllowed */
); err != nil {
e := errors.Mark(errors.Wrapf(err, "%v received invalid ChangeReplicasTrigger %s to "+
"remove self (leaseholder); lhRemovalAllowed: %v; current desc: %v; proposed desc: %v",
lhDesc, crt, lhRemovalAllowed, r.Desc(), proposedDesc), errMarkInvalidReplicationChange)
lhDesc, crt, true /* lhRemovalAllowed */, r.Desc(), proposedDesc), errMarkInvalidReplicationChange)
log.Errorf(p.ctx, "%v", e)
return roachpb.NewError(e)
}
Expand Down
23 changes: 1 addition & 22 deletions pkg/kv/kvserver/replicate_queue.go
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,6 @@ import (
"time"

"github.com/cockroachdb/cockroach/pkg/base"
"github.com/cockroachdb/cockroach/pkg/clusterversion"
"github.com/cockroachdb/cockroach/pkg/gossip"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/allocator"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/allocator/allocatorimpl"
Expand Down Expand Up @@ -985,25 +984,6 @@ func (rq *replicateQueue) addOrReplaceVoters(
break
}
}
// Determines whether we can remove the leaseholder without first
// transferring the lease away. The call to allocator below makes sure
// that we proceed with the change only if we replace the removed replica
// with a VOTER_INCOMING.
lhRemovalAllowed := repl.store.cfg.Settings.Version.IsActive(ctx,
clusterversion.EnableLeaseHolderRemoval)
if !lhRemovalAllowed {
// See about transferring the lease away if we're about to remove the
// leaseholder.
done, err := rq.maybeTransferLeaseAway(
ctx, repl, existingVoters[removeIdx].StoreID, dryRun, nil /* canTransferLeaseFrom */)
if err != nil {
return false, err
}
if done {
// Lease was transferred away. Next leaseholder is going to take over.
return false, nil
}
}
}

lhBeingRemoved := removeIdx >= 0 && existingVoters[removeIdx].StoreID == repl.store.StoreID()
Expand Down Expand Up @@ -1549,8 +1529,7 @@ func (rq *replicateQueue) considerRebalance(

// Determines whether we can remove the leaseholder without first
// transferring the lease away
lhRemovalAllowed := addTarget != (roachpb.ReplicationTarget{}) &&
repl.store.cfg.Settings.Version.IsActive(ctx, clusterversion.EnableLeaseHolderRemoval)
lhRemovalAllowed := addTarget != (roachpb.ReplicationTarget{})
lhBeingRemoved := removeTarget.StoreID == repl.store.StoreID()

if !ok {
Expand Down
4 changes: 4 additions & 0 deletions pkg/sql/conn_executor_exec.go
Original file line number Diff line number Diff line change
Expand Up @@ -2095,6 +2095,10 @@ func (ex *connExecutor) handleAutoCommit(
if err != nil {
return ex.makeErrEvent(err, stmt)
}
if err = ex.state.mu.txn.Step(ctx); err != nil {
log.VEventf(ctx, 2, "AutoCommit. err: %v", err)
return ex.makeErrEvent(err, stmt)
}
ev, payload := ex.commitSQLTransaction(ctx, stmt, ex.commitSQLTransactionInternal)
if perr, ok := payload.(payloadWithError); ok {
err = perr.errorCause()
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,7 @@ statement ok
GRANT USAGE,DROP ON EXTERNAL CONNECTION foo TO testuser

query TTTT
SELECT * FROM system.privileges
SELECT * FROM system.privileges ORDER by username
----
root /externalconn/foo {ALL} {}
testuser /externalconn/foo {DROP,USAGE} {}
Expand All @@ -31,7 +31,7 @@ statement ok
REVOKE USAGE,DROP ON EXTERNAL CONNECTION foo FROM testuser

query TTTT
SELECT * FROM system.privileges
SELECT * FROM system.privileges ORDER by username
----
root /externalconn/foo {ALL} {}

Expand Down
20 changes: 20 additions & 0 deletions pkg/sql/logictest/testdata/logic_test/schema
Original file line number Diff line number Diff line change
Expand Up @@ -858,3 +858,23 @@ SELECT comment FROM system.comments LIMIT 1

statement ok
DROP DATABASE comment_db

statement ok
USE test

# Test that a schema can be created when writing to the eventlog table is
# disabled. This is a regression test for #86132.
subtest create_schema_no_eventlog

statement ok
SET CLUSTER SETTING server.eventlog.enabled = false

statement ok
CREATE SCHEMA sc

statement ok
DROP SCHEMA sc

statement ok
SET CLUSTER SETTING server.eventlog.enabled = false

0 comments on commit 8826516

Please sign in to comment.