-
Notifications
You must be signed in to change notification settings - Fork 3.8k
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
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
Showing
59 changed files
with
2,113 additions
and
610 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
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() | ||
} | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.