Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
90709: sql/schemachanger: inject DML statements into end to end tests r=fqazi a=fqazi

Fixes: #83304

Previously, the declarative schema changer tests only ran DDL
statements for the schema change and had no mechanism for
determining correct behaviour if queries were run concurrently
at each phase. To address this, this patch adds a new framework
to these tests which allows us to inject DML (inserts / selects)
at various stages of a declarative schema change. This gives us
a more powerful framework for validating behaviour.

Release note: None

The first commit here can be ignored, and its a separate PR to make sure this runs stable. A PR is already open for it.

91045: kvserver: separate repl q decision from action r=nvanbenschoten a=kvoli

Same as #90529. with a fix to stop logging an error on replicate queue metrics unsupported action.

```
dev test pkg/cli -f TestPartialZip -v  --stress --race
...
1135 runs so far, 0 failures, over 2m55s
1168 runs so far, 0 failures, over 3m0s
```

Previously, the replicate queue would both plan and apply changes for
in-process replicas within the processOneChange function. This was
problematic for simulation as it was not possible to call
processOneChange directly to apply the simulated result, without
blocking the goroutine.

This patch separates processOneChange into planning (PlanOneChange), the
application of the change (applyChange) and post application tracking
(TrackChangeOutcome).

resolves: #90533

Release note: None

91375: kv: recycle SpanSets returned during optimistic eval validation r=nvanbenschoten a=nvanbenschoten

This commit adds `Release` calls for both SpanSets returned by `collectSpansRead` on the optimistic eval validation path. The failure to recycle these SpanSets was causing a leak from the `SpanSet` memory pool of both top level objects and the recycled inner slices.

We have the same issue at the other caller of `collectSpansRead`, which is being tracked more broadly in #91374.

Release note: None
Epic: None

Co-authored-by: Faizan Qazi <[email protected]>
Co-authored-by: Austen McClernon <[email protected]>
Co-authored-by: Nathan VanBenschoten <[email protected]>
  • Loading branch information
4 people committed Nov 10, 2022
4 parents 8e9b548 + 84a15ea + 7340e07 + 3455572 commit cb796b9
Show file tree
Hide file tree
Showing 59 changed files with 2,113 additions and 610 deletions.
1 change: 1 addition & 0 deletions pkg/kv/kvserver/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -8,6 +8,7 @@ go_library(
name = "kvserver",
srcs = [
"addressing.go",
"allocation_op.go",
"consistency_queue.go",
"debug_print.go",
"doc.go",
Expand Down
148 changes: 148 additions & 0 deletions pkg/kv/kvserver/allocation_op.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,148 @@
// 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 kvserver

import (
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/allocator"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/allocator/storepool"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb"
"github.com/cockroachdb/cockroach/pkg/roachpb"
)

// AllocationOp represents an atomic allocation operation to be applied against
// a specific replica.
//
// TODO(kvoli): Add AllocationRelocateRangeOp.
type AllocationOp interface {
// trackPlanningMetrics tracks the metrics that have been generated in
// creating this operation.
trackPlanningMetrics()
// applyImpact updates the given storepool to reflect the result of
// applying this operation.
applyImpact(storepool *storepool.StorePool)
// lhBeingRemoved returns true when the leaseholder is will be removed if
// this operation succeeds, otherwise false.
lhBeingRemoved() bool
}

// AllocationTransferLeaseOp represents an operation to transfer a range lease to another
// store, from the current one.
type AllocationTransferLeaseOp struct {
target, source roachpb.StoreID
usage allocator.RangeUsageInfo
bypassSafetyChecks bool
sideEffects func()
}

// lhBeingRemoved returns true when the leaseholder is will be removed if this
// operation succeeds, otherwise false. This is always true for lease
// transfers.
func (o AllocationTransferLeaseOp) lhBeingRemoved() bool {
return true
}

func (o AllocationTransferLeaseOp) applyImpact(storepool *storepool.StorePool) {
// TODO(kvoli): Currently the local storepool is updated directly in the
// lease transfer call, rather than in this function. Move the storepool
// tracking from rq.TransferLease to this function once #89771 is merged.
}

// trackPlanningMetrics tracks the metrics that have been generated in creating
// this operation.
func (o AllocationTransferLeaseOp) trackPlanningMetrics() {
if o.sideEffects != nil {
o.sideEffects()
}
}

// AllocationChangeReplicasOp represents an operation to execute a change
// replicas txn.
type AllocationChangeReplicasOp struct {
usage allocator.RangeUsageInfo
lhStore roachpb.StoreID
chgs roachpb.ReplicationChanges
priority kvserverpb.SnapshotRequest_Priority
allocatorPriority float64
reason kvserverpb.RangeLogEventReason
details string
sideEffects func()
}

// lhBeingRemoved returns true when the voter removals for this change replicas
// operation includes the leaseholder store.
func (o AllocationChangeReplicasOp) lhBeingRemoved() bool {
for _, chg := range o.chgs.VoterRemovals() {
if chg.StoreID == o.lhStore {
return true
}
}
return false
}

// applyEstimatedImpact updates the given storepool to reflect the result
// of applying this operation.
func (o AllocationChangeReplicasOp) applyImpact(storepool *storepool.StorePool) {
for _, chg := range o.chgs {
storepool.UpdateLocalStoreAfterRebalance(chg.Target.StoreID, o.usage, chg.ChangeType)
}
}

// trackPlanningMetrics tracks the metrics that have been generated in creating
// this operation.
func (o AllocationChangeReplicasOp) trackPlanningMetrics() {
if o.sideEffects != nil {
o.sideEffects()
}
}

// AllocationFinalizeAtomicReplicationOp represents an operation to finalize an
// atomic change replicas operation and remove any remaining learners.
type AllocationFinalizeAtomicReplicationOp struct{}

// TODO(kvoli): This always returns false, however it is possible that the LH
// may have been removed here.
func (o AllocationFinalizeAtomicReplicationOp) lhBeingRemoved() bool { return false }
func (o AllocationFinalizeAtomicReplicationOp) applyImpact(storepool *storepool.StorePool) {}
func (o AllocationFinalizeAtomicReplicationOp) trackPlanningMetrics() {}

// AllocationNoop represents no operation.
type AllocationNoop struct{}

func (o AllocationNoop) lhBeingRemoved() bool { return false }
func (o AllocationNoop) applyImpact(storepool *storepool.StorePool) {}
func (o AllocationNoop) trackPlanningMetrics() {}

// effectBuilder is a utility struct to track a list of effects, which may be
// used to construct a single effect function that in turn calls all tracked
// effects.
type effectBuilder struct {
e []func()
}

// add appends an effect to be rolled into a single effect when calling f().
// The return value of this function must be used.
func (b effectBuilder) add(effect func()) effectBuilder {
b.e = append(b.e, effect)
return b
}

func (b effectBuilder) f() func() {
// NB: Avoid heap allocations when not necessary.
if len(b.e) == 0 {
return func() {}
}

return func() {
for _, effect := range b.e {
effect()
}
}
}
1 change: 0 additions & 1 deletion pkg/kv/kvserver/allocator/allocatorimpl/allocator_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -5423,7 +5423,6 @@ func TestAllocatorTransferLeaseTargetLoadBased(t *testing.T) {
allocator.TransferLeaseOptions{
ExcludeLeaseRepl: c.excludeLeaseRepl,
CheckCandidateFullness: true,
DryRun: false,
},
)
if c.expected != target.StoreID {
Expand Down
5 changes: 0 additions & 5 deletions pkg/kv/kvserver/allocator/base.go
Original file line number Diff line number Diff line change
Expand Up @@ -167,7 +167,6 @@ type TransferLeaseOptions struct {
// CheckCandidateFullness, when false, tells `TransferLeaseTarget`
// to disregard the existing lease counts on candidates.
CheckCandidateFullness bool
DryRun bool
// AllowUninitializedCandidates allows a lease transfer target to include
// replicas which are not in the existing replica set.
AllowUninitializedCandidates bool
Expand All @@ -181,8 +180,6 @@ const (
TransferErr LeaseTransferOutcome = iota
// TransferOK indicates a successful lease transfer attempt.
TransferOK
// NoTransferDryRun indicates a dry-run (i.e. noop) lease transfer attempt.
NoTransferDryRun
// NoSuitableTarget indicates a lease transfer attempt that found no suitable
// targets.
NoSuitableTarget
Expand All @@ -194,8 +191,6 @@ func (o LeaseTransferOutcome) String() string {
return "err"
case TransferOK:
return "ok"
case NoTransferDryRun:
return "no transfer; dry run"
case NoSuitableTarget:
return "no suitable transfer target found"
default:
Expand Down
2 changes: 2 additions & 0 deletions pkg/kv/kvserver/replica_read.go
Original file line number Diff line number Diff line change
Expand Up @@ -139,6 +139,8 @@ func (r *Replica) executeReadOnlyBatch(
if err != nil {
return nil, g, nil, roachpb.NewError(err)
}
defer latchSpansRead.Release()
defer lockSpansRead.Release()
if ok := g.CheckOptimisticNoConflicts(latchSpansRead, lockSpansRead); !ok {
return nil, g, nil, roachpb.NewError(roachpb.NewOptimisticEvalConflictsError())
}
Expand Down
Loading

0 comments on commit cb796b9

Please sign in to comment.