Skip to content

Commit ea38ebb

Browse files
craig[bot]shubhamdhamaiskettanehmw5hfqazi
committed
141735: sql: enable tenant testing for zone tests r=rafiss a=shubhamdhama `` sql: enable tenant testing for `zone_test.go` `` Updating `TestValidSetShowZones` to use `createTestServerParamsAllowTenants` instead of `createTestServerParams` required some investigation. It turned out simpler than expected. Zone settings for meta ranges aren't supported for secondary tenants, so this change simply exclude them from setup and validating. Informs: #140446 Epic: CRDB-48357 Release note: None `` sql: enable tenant testing for `zone_config_test` `` Since `GetSpanConfigForKey` is only available in system tenant, it is not used when running under secondary tenants. 142186: kvserver: mark replica as unavailable if leaderless for a long time r=iskettaneh a=iskettaneh This PR marks the replica as unavailable in if it has been leaderless for a duration above: `kv.replica_raft.leaderless_unavailable_threshold`. This helps requests to bail early on unavailable ranges without relying on the replica circuit breaker to trip. This has multiple benefits: 1) Faster reaction time than the replica circuit breaker: If two nodes fail (assuming R=3), many ranges will become unavailable. With the replica circuit breaker, a scan query will need to trip the circuit breaker for multiple replicas, causing added delays. However, with this approach, the replica will basically become unavailable on the tick path (rather than on the request path). Meaning that a scan query wouldn't need to trip one circuit breaker after the other. 2) Lighter weight: Instead of relying on the replica circuit breaker to test the replication pipeline before it marks the range as available again, this approach relies on the Raft signal to know when there is a leader again, indicating that the range is available again. 3) With leader leases, a replica won't propose a lease if it's not the leader. This means that with leader leases, the replica circuit breaker might not trip if the range have lost quorum. However, with this commit, the replica will eventually forget who the leader was, and eventually the leaderlessWatcher would mark it as unavailable. Fixes: #139638 Release note: None 142489: row, sql: implement mutations on vector indexes r=mw5h a=mw5h This patch plumbs the output of the vector search operators into rowenc for encoding into vector indexes. The output of these vector search operators is included in the row values for mutation operators after partial index values and are plumbed into a new vector index update helper, which tracks the column values until they're needed by rowenc. While we're here, we homogonize how pkg/sql/{delete,insert,update,upsert}.go consume row values, hopefully improving legibility. Epic: CRDB-42943 Release note: None 142829: sql/catalog: fix object renames for PCR reader catalogs r=fqazi a=fqazi Previously, the PCR reader catalog would only delete namespace entries for a descriptor if it was not modified. This meant the reader catalog logic could leave behind stale entries in the system.namespace table after a object was renamed. To address this, this patch detects if an object is renamed, and allows the old namespace entry to be deleted. Fixes: #142828 Release note (bug fix): PCR reader catalogs could have orphan rows in system.namespace after a object is renamed. 142836: kvserver: deflake TestLeasePreferencesDuringOutage r=kvoli a=arulajmani There was a race here, when heartbeating node liveness epochs, where another node could increment our epoch. This patch retries in such situations. Fixes #142795 Release note: None Co-authored-by: Shubham Dhama <[email protected]> Co-authored-by: Ibrahim Kettaneh <[email protected]> Co-authored-by: Matt White <[email protected]> Co-authored-by: Faizan Qazi <[email protected]> Co-authored-by: Arul Ajmani <[email protected]>
6 parents 42b4638 + 344c262 + a563025 + 5eaf411 + 10cc4df + 4e87632 commit ea38ebb

Some content is hidden

Large Commits have some content hidden by default. Use the searchbox below for content that may be hidden.

46 files changed

+1086
-207
lines changed

docs/generated/settings/settings.html

+1
Original file line numberDiff line numberDiff line change
@@ -116,6 +116,7 @@
116116
<tr><td><div id="setting-kv-rangefeed-closed-timestamp-refresh-interval" class="anchored"><code>kv.rangefeed.closed_timestamp_refresh_interval</code></div></td><td>duration</td><td><code>3s</code></td><td>the interval at which closed-timestamp updatesare delivered to rangefeeds; set to 0 to use kv.closed_timestamp.side_transport_interval</td><td>Dedicated/Self-hosted (read-write); Serverless (read-only)</td></tr>
117117
<tr><td><div id="setting-kv-rangefeed-enabled" class="anchored"><code>kv.rangefeed.enabled</code></div></td><td>boolean</td><td><code>false</code></td><td>if set, rangefeed registration is enabled</td><td>Dedicated/Self-hosted (read-write); Serverless (read-only)</td></tr>
118118
<tr><td><div id="setting-kv-replica-circuit-breaker-slow-replication-threshold" class="anchored"><code>kv.replica_circuit_breaker.slow_replication_threshold</code></div></td><td>duration</td><td><code>1m0s</code></td><td>duration after which slow proposals trip the per-Replica circuit breaker (zero duration disables breakers)</td><td>Dedicated/Self-Hosted</td></tr>
119+
<tr><td><div id="setting-kv-replica-raft-leaderless-unavailable-threshold" class="anchored"><code>kv.replica_raft.leaderless_unavailable_threshold</code></div></td><td>duration</td><td><code>1m0s</code></td><td>duration after which leaderless replicas is considered unavailable. Set to 0 to disable leaderless replica availability checks</td><td>Dedicated/Self-Hosted</td></tr>
119120
<tr><td><div id="setting-kv-replica-stats-addsst-request-size-factor" class="anchored"><code>kv.replica_stats.addsst_request_size_factor</code></div></td><td>integer</td><td><code>50000</code></td><td>the divisor that is applied to addsstable request sizes, then recorded in a leaseholders QPS; 0 means all requests are treated as cost 1</td><td>Dedicated/Self-Hosted</td></tr>
120121
<tr><td><div id="setting-kv-replication-reports-interval" class="anchored"><code>kv.replication_reports.interval</code></div></td><td>duration</td><td><code>1m0s</code></td><td>the frequency for generating the replication_constraint_stats, replication_stats_report and replication_critical_localities reports (set to 0 to disable)</td><td>Dedicated/Self-Hosted</td></tr>
121122
<tr><td><div id="setting-kv-snapshot-rebalance-max-rate" class="anchored"><code>kv.snapshot_rebalance.max_rate</code></div></td><td>byte size</td><td><code>32 MiB</code></td><td>the rate limit (bytes/sec) to use for rebalance and upreplication snapshots</td><td>Dedicated/Self-Hosted</td></tr>

pkg/ccl/partitionccl/zone_test.go

+2-6
Original file line numberDiff line numberDiff line change
@@ -50,9 +50,7 @@ func TestValidIndexPartitionSetShowZones(t *testing.T) {
5050
defer leaktest.AfterTest(t)()
5151
defer log.Scope(t).Close(t)
5252

53-
s, db, _ := serverutils.StartServer(t, base.TestServerArgs{
54-
DefaultTestTenant: base.TODOTestTenantDisabled,
55-
})
53+
s, db, _ := serverutils.StartServer(t, base.TestServerArgs{})
5654
defer s.Stopper().Stop(context.Background())
5755

5856
sqlDB := sqlutils.MakeSQLRunner(db)
@@ -253,9 +251,7 @@ func TestInvalidIndexPartitionSetShowZones(t *testing.T) {
253251
defer leaktest.AfterTest(t)()
254252
defer log.Scope(t).Close(t)
255253

256-
s, db, _ := serverutils.StartServer(t, base.TestServerArgs{
257-
DefaultTestTenant: base.TODOTestTenantDisabled,
258-
})
254+
s, db, _ := serverutils.StartServer(t, base.TestServerArgs{})
259255
defer s.Stopper().Stop(context.Background())
260256

261257
for i, tc := range []struct {

pkg/crosscluster/logical/lww_kv_processor.go

+9-3
Original file line numberDiff line numberDiff line change
@@ -476,13 +476,15 @@ func (p *kvTableWriter) insertRow(ctx context.Context, b *kv.Batch, after cdceve
476476

477477
var ph row.PartialIndexUpdateHelper
478478
// TODO(dt): support partial indexes.
479+
var vh row.VectorIndexUpdateHelper
480+
// TODO(mw5h, drewk): support vector indexes.
479481
oth := &row.OriginTimestampCPutHelper{
480482
OriginTimestamp: after.MvccTimestamp,
481483
// TODO(ssd): We should choose this based by comparing the cluster IDs of the source
482484
// and destination clusters.
483485
// ShouldWinTie: true,
484486
}
485-
return p.ri.InsertRow(ctx, &row.KVBatchAdapter{Batch: b}, p.newVals, ph, oth, row.CPutOp, false /* traceKV */)
487+
return p.ri.InsertRow(ctx, &row.KVBatchAdapter{Batch: b}, p.newVals, ph, vh, oth, row.CPutOp, false /* traceKV */)
486488
}
487489

488490
func (p *kvTableWriter) updateRow(
@@ -497,13 +499,15 @@ func (p *kvTableWriter) updateRow(
497499

498500
var ph row.PartialIndexUpdateHelper
499501
// TODO(dt): support partial indexes.
502+
var vh row.VectorIndexUpdateHelper
503+
// TODO(mw5h, drewk): support vector indexes.
500504
oth := &row.OriginTimestampCPutHelper{
501505
OriginTimestamp: after.MvccTimestamp,
502506
// TODO(ssd): We should choose this based by comparing the cluster IDs of the source
503507
// and destination clusters.
504508
// ShouldWinTie: true,
505509
}
506-
_, err := p.ru.UpdateRow(ctx, b, p.oldVals, p.newVals, ph, oth, false)
510+
_, err := p.ru.UpdateRow(ctx, b, p.oldVals, p.newVals, ph, vh, oth, false)
507511
return err
508512
}
509513

@@ -516,6 +520,8 @@ func (p *kvTableWriter) deleteRow(
516520

517521
var ph row.PartialIndexUpdateHelper
518522
// TODO(dt): support partial indexes.
523+
var vh row.VectorIndexUpdateHelper
524+
// TODO(mw5h, drewk): support vector indexes.
519525
oth := &row.OriginTimestampCPutHelper{
520526
PreviousWasDeleted: before.IsDeleted(),
521527
OriginTimestamp: after.MvccTimestamp,
@@ -524,7 +530,7 @@ func (p *kvTableWriter) deleteRow(
524530
// ShouldWinTie: true,
525531
}
526532

527-
return p.rd.DeleteRow(ctx, b, p.oldVals, ph, oth, false)
533+
return p.rd.DeleteRow(ctx, b, p.oldVals, ph, vh, oth, false)
528534
}
529535

530536
func (p *kvTableWriter) fillOld(vals cdcevent.Row) error {

pkg/kv/kvserver/client_lease_test.go

+11-1
Original file line numberDiff line numberDiff line change
@@ -1178,7 +1178,17 @@ func TestLeasePreferencesDuringOutage(t *testing.T) {
11781178
// allocator on server 0 may see everyone as temporarily dead due to the
11791179
// clock move above.
11801180
for _, i := range []int{0, 3, 4} {
1181-
require.NoError(t, tc.Servers[i].HeartbeatNodeLiveness())
1181+
testutils.SucceedsSoon(t, func() error {
1182+
err := tc.Servers[i].HeartbeatNodeLiveness()
1183+
if err != nil {
1184+
if errors.Is(err, liveness.ErrEpochIncremented) {
1185+
t.Logf("retrying heartbeat after err %s", err)
1186+
return err
1187+
}
1188+
t.Fatalf("unexpected error heartbeating liveness record for server %d: %s", i, err)
1189+
}
1190+
return nil
1191+
})
11821192
require.NoError(t, tc.GetFirstStoreFromServer(t, i).GossipStore(ctx, true))
11831193
}
11841194
}

pkg/kv/kvserver/client_replica_test.go

+137
Original file line numberDiff line numberDiff line change
@@ -39,6 +39,7 @@ import (
3939
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/protectedts/ptutil"
4040
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/raftutil"
4141
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/stateloader"
42+
"github.com/cockroachdb/cockroach/pkg/raft"
4243
"github.com/cockroachdb/cockroach/pkg/raft/raftpb"
4344
"github.com/cockroachdb/cockroach/pkg/roachpb"
4445
"github.com/cockroachdb/cockroach/pkg/server"
@@ -2728,6 +2729,142 @@ func TestChangeReplicasGeneration(t *testing.T) {
27282729
assert.EqualValues(t, repl.Desc().Generation, oldGeneration+3, "\nold: %+v\nnew: %+v", oldDesc, newDesc)
27292730
}
27302731

2732+
// TestLossQuorumCauseLeaderlessWatcherToSignalUnavailable checks that if a range
2733+
// lost its quorum, the remaining replicas in that range will have their
2734+
// leaderlessWatcher indicate that the range is unavailable. Also, it checks
2735+
// that when the range regains quorum, the leaderlessWatcher will indicate that
2736+
// the range is available.
2737+
func TestLossQuorumCauseLeaderWatcherToSignalUnavailable(t *testing.T) {
2738+
defer leaktest.AfterTest(t)()
2739+
defer log.Scope(t).Close(t)
2740+
ctx := context.Background()
2741+
manualClock := hlc.NewHybridManualClock()
2742+
2743+
stickyVFSRegistry := fs.NewStickyRegistry()
2744+
lisReg := listenerutil.NewListenerRegistry()
2745+
defer lisReg.Close()
2746+
2747+
// Perform a basic test setup with two nodes.
2748+
const numServers int = 2
2749+
st := cluster.MakeTestingClusterSettings()
2750+
2751+
// Set `kv.replica_raft.leaderless_unavailable_threshold` to 10 seconds.
2752+
threshold := time.Second * 10
2753+
kvserver.ReplicaLeaderlessUnavailableThreshold.Override(ctx, &st.SV, threshold)
2754+
2755+
stickyServerArgs := make(map[int]base.TestServerArgs)
2756+
for i := 0; i < numServers; i++ {
2757+
stickyServerArgs[i] = base.TestServerArgs{
2758+
Settings: st,
2759+
StoreSpecs: []base.StoreSpec{
2760+
{
2761+
InMemory: true,
2762+
StickyVFSID: strconv.FormatInt(int64(i), 10),
2763+
},
2764+
},
2765+
Knobs: base.TestingKnobs{
2766+
Server: &server.TestingKnobs{
2767+
StickyVFSRegistry: stickyVFSRegistry,
2768+
},
2769+
},
2770+
}
2771+
}
2772+
2773+
tc := testcluster.StartTestCluster(t, numServers,
2774+
base.TestClusterArgs{
2775+
ReplicationMode: base.ReplicationManual,
2776+
ReusableListenerReg: lisReg,
2777+
ServerArgsPerNode: stickyServerArgs,
2778+
})
2779+
2780+
defer tc.Stopper().Stop(ctx)
2781+
2782+
key := tc.ScratchRange(t)
2783+
tc.AddVotersOrFatal(t, key, tc.Targets(1)...)
2784+
desc, err := tc.LookupRange(key)
2785+
require.NoError(t, err)
2786+
2787+
// Randomly stop server index 0 or 1.
2788+
stoppedNodeInx := rand.Intn(2)
2789+
aliveNodeIdx := 1 - stoppedNodeInx
2790+
tc.StopServer(stoppedNodeInx)
2791+
repl := tc.GetFirstStoreFromServer(t, aliveNodeIdx).LookupReplica(roachpb.RKey(key))
2792+
2793+
// The range is available initially.
2794+
require.False(t, repl.LeaderlessWatcher.IsUnavailable())
2795+
2796+
// Wait until the remaining replica becomes leaderless.
2797+
testutils.SucceedsSoon(t, func() error {
2798+
if repl.RaftStatus().Lead != raft.None {
2799+
return errors.New("Leader still exists")
2800+
}
2801+
return nil
2802+
})
2803+
2804+
// Increment the clock by the leaderlessWatcher unavailable threshold.
2805+
manualClock.Increment(threshold.Nanoseconds())
2806+
2807+
// Wait for the leaderlessWatcher to indicate that the range is unavailable.
2808+
testutils.SucceedsSoon(t, func() error {
2809+
tc.GetFirstStoreFromServer(t, aliveNodeIdx).LookupReplica(roachpb.RKey(key))
2810+
if !repl.LeaderlessWatcher.IsUnavailable() {
2811+
return errors.New("range is still available")
2812+
}
2813+
return nil
2814+
})
2815+
2816+
sendPutRequestWithTimeout := func(repl *kvserver.Replica, timeout time.Duration) (*kvpb.Error, error) {
2817+
ctx, cancel := context.WithTimeout(ctx, timeout)
2818+
defer cancel()
2819+
ba := &kvpb.BatchRequest{}
2820+
ba.RangeID = desc.RangeID
2821+
ba.Timestamp = repl.Clock().Now()
2822+
ba.Add(putArgs(key, []byte("foo")))
2823+
_, pErr := repl.Send(ctx, ba)
2824+
return pErr, ctx.Err()
2825+
}
2826+
2827+
// Requests should immediately return an error indicating that the range is
2828+
// unavailable.
2829+
pErr, ctxErr := sendPutRequestWithTimeout(repl, 2*time.Second)
2830+
require.NoError(t, ctxErr)
2831+
require.Regexp(t, "replica has been leaderless for 10s", pErr)
2832+
require.True(t, errors.HasType(pErr.GoError(), (*kvpb.ReplicaUnavailableError)(nil)),
2833+
"expected ReplicaUnavailableError, got %v", err)
2834+
2835+
// At this point we know that the replica is considered unavailable. Regain
2836+
// the quorum and check that the leaderlessWatcher indicates that the range is
2837+
// available.
2838+
require.NoError(t, tc.RestartServer(stoppedNodeInx))
2839+
2840+
testutils.SucceedsSoon(t, func() error {
2841+
repl = tc.GetFirstStoreFromServer(t, aliveNodeIdx).LookupReplica(roachpb.RKey(key))
2842+
tc.GetFirstStoreFromServer(t, aliveNodeIdx).LookupReplica(roachpb.RKey(key))
2843+
if repl.LeaderlessWatcher.IsUnavailable() {
2844+
return errors.New("range is still unavailable")
2845+
}
2846+
return nil
2847+
})
2848+
2849+
// Requests should now succeed. We need to try both replicas to avoid
2850+
// NotLeaseHolderErrors.
2851+
testutils.SucceedsSoon(t, func() error {
2852+
for i := range numServers {
2853+
repl := tc.GetFirstStoreFromServer(t, i).LookupReplica(roachpb.RKey(key))
2854+
pErr, ctxErr = sendPutRequestWithTimeout(repl, 2*time.Second)
2855+
if ctxErr == nil && pErr == nil {
2856+
return nil
2857+
}
2858+
}
2859+
// If we reach this point, we know that the request failed, return the
2860+
// error.
2861+
if ctxErr != nil {
2862+
return ctxErr
2863+
}
2864+
return pErr.GoError()
2865+
})
2866+
}
2867+
27312868
func TestClearRange(t *testing.T) {
27322869
defer leaktest.AfterTest(t)()
27332870
defer log.Scope(t).Close(t)

pkg/kv/kvserver/replica.go

+65
Original file line numberDiff line numberDiff line change
@@ -176,6 +176,67 @@ func (c *atomicConnectionClass) set(cc rpc.ConnectionClass) {
176176
atomic.StoreUint32((*uint32)(c), uint32(cc))
177177
}
178178

179+
// leaderlessWatcher is a lightweight implementation of the signaller interface
180+
// that is used to signal when a replica doesn't know who the leader is for an
181+
// extended period of time. This is used to signal that the range in
182+
// unavailable.
183+
type leaderlessWatcher struct {
184+
mu struct {
185+
syncutil.RWMutex
186+
187+
// leaderlessTimestamp records the timestamp captured when the replica
188+
// didn't know who the leader was. This is reset on every tick if the
189+
// replica knows who the leader is.
190+
leaderlessTimestamp time.Time
191+
192+
// unavailable is set to true if the replica is leaderless for a long time
193+
// (longer than ReplicaLeaderlessUnavailableThreshold).
194+
unavailable bool
195+
}
196+
197+
// err is the error returned when the replica is leaderless for a long time.
198+
err error
199+
200+
// closedChannel is an already closed channel. Requests will use it to know
201+
// that the replica is leaderless, and can be considered unavailable. This
202+
// is primarily due to implementation details of the request path, where
203+
// the request grabs a signaller in signallerForBatch() and then checks if
204+
// the channel is closed to determine if the replica is available.
205+
closedChannel chan struct{}
206+
}
207+
208+
// NewLeaderlessWatcher initializes a new leaderlessWatcher with the default
209+
// values.
210+
func NewLeaderlessWatcher(r *Replica) *leaderlessWatcher {
211+
closedCh := make(chan struct{})
212+
close(closedCh)
213+
return &leaderlessWatcher{
214+
err: r.replicaUnavailableError(
215+
errors.Errorf("replica has been leaderless for %s",
216+
ReplicaLeaderlessUnavailableThreshold.Get(&r.store.cfg.Settings.SV))),
217+
closedChannel: closedCh,
218+
}
219+
}
220+
221+
func (lw *leaderlessWatcher) Err() error {
222+
return lw.err
223+
}
224+
225+
func (lw *leaderlessWatcher) C() <-chan struct{} {
226+
return lw.closedChannel
227+
}
228+
229+
func (lw *leaderlessWatcher) IsUnavailable() bool {
230+
lw.mu.RLock()
231+
defer lw.mu.RUnlock()
232+
return lw.mu.unavailable
233+
}
234+
235+
func (lw *leaderlessWatcher) resetLocked() {
236+
lw.mu.leaderlessTimestamp = time.Time{}
237+
lw.mu.unavailable = false
238+
}
239+
179240
// ReplicaMutex is an RWMutex. It has its own type to make it easier to look for
180241
// usages specific to the replica mutex.
181242
type ReplicaMutex syncutil.RWMutex
@@ -941,6 +1002,10 @@ type Replica struct {
9411002
lastTickTimestamp hlc.ClockTimestamp
9421003
}
9431004

1005+
// LeaderlessWatcher is used to signal when a replica is leaderless for a long
1006+
// time.
1007+
LeaderlessWatcher *leaderlessWatcher
1008+
9441009
// The raft log truncations that are pending. Access is protected by its own
9451010
// mutex. All implementation details should be considered hidden except to
9461011
// the code in raft_log_truncator.go. External code should only use the

pkg/kv/kvserver/replica_backpressure.go

+16-5
Original file line numberDiff line numberDiff line change
@@ -118,18 +118,29 @@ func canBackpressureBatch(ba *kvpb.BatchRequest) bool {
118118
return false
119119
}
120120

121-
// signallerForBatch returns the signaller to use for this batch. This is the
122-
// Replica's breaker's signaller except if any request in the batch uses
123-
// poison.Policy_Wait, in which case it's a neverTripSignaller. In particular,
124-
// `(signaller).C() == nil` signals that the request bypasses the circuit
125-
// breakers.
121+
// signallerForBatch returns the signaller to use for this batch in the
122+
// following priorities:
123+
// 1. If the batch contains a request uses poison.Policy_Wait, we will return
124+
// neverTripSignaller.
125+
// 2. If the replica is leaderless for a time longer than the threshold in
126+
// `kv.replica_raft.leaderless_unavailable_threshold`, use the
127+
// leaderlessWatcher signal.
128+
// 3. Otherwise, use the replica's breaker's signaller
126129
func (r *Replica) signallerForBatch(ba *kvpb.BatchRequest) signaller {
127130
for _, ru := range ba.Requests {
128131
req := ru.GetInner()
129132
if kvpb.BypassesReplicaCircuitBreaker(req) {
130133
return neverTripSignaller{}
131134
}
132135
}
136+
137+
// If the leaderless watcher indicates that this replica is leaderless for a
138+
// long time, use it as the signal.
139+
if r.LeaderlessWatcher.IsUnavailable() {
140+
return r.LeaderlessWatcher
141+
}
142+
143+
// Otherwise, use the replica's breaker.
133144
return r.breaker.Signal()
134145
}
135146

pkg/kv/kvserver/replica_init.go

+1
Original file line numberDiff line numberDiff line change
@@ -245,6 +245,7 @@ func newUninitializedReplicaWithoutRaftGroup(
245245
r.breaker = newReplicaCircuitBreaker(
246246
store.cfg.Settings, store.stopper, r.AmbientContext, r, onTrip, onReset,
247247
)
248+
r.LeaderlessWatcher = NewLeaderlessWatcher(r)
248249
r.mu.currentRACv2Mode = r.replicationAdmissionControlModeToUse(context.TODO())
249250
r.raftMu.flowControlLevel = kvflowcontrol.GetV2EnabledWhenLeaderLevel(
250251
r.raftCtx, store.ClusterSettings(), store.TestingKnobs().FlowControlTestingKnobs)

0 commit comments

Comments
 (0)