Skip to content

Commit 5fc80c3

Browse files
craig[bot]kvoliThomas Hardymsbutlerhealthy-pod
committed
108175: kvserver: unskip lease preferences during outage r=andrewbaptist a=kvoli Previously, `TestLeasePreferenceDuringOutage` would force replication queue processing of the test range, then assert that the range up-replicated and lease transferred to a preferred locality. This test was skipped, and two of the assumptions it relied on to pass were no longer true. After #85219, the replicate queue no longer re-processes replicas. Instead, the queue requeues replicas after processing, at the appropriate priority. This broke the test due to the replicate queue being disabled, making the re-queue a no-op. After #94023, the replicate queue no longer looked for lease transfers, after processing a replication action. Combined with #85219, the queue would now be guaranteed to not process both up-replication and lease transfers from a single enqueue. Update the test to not require a manual process, instead using a queue range filter, which allows tests which disable automatic replication, to still process filtered ranges via the various replica queues. Also, ensure that the non-stopped stores are considered live targets, after simulating an outage (bumping manual clocks, stopping servers) -- so that the expected up-replication, then lease transfer can proceed. Fixes: #88769 Release note: None 109432: cluster-ui: handle partial response errors on the database details page r=THardy98 a=THardy98 Part of: #102386 **Demos** (Note: these demos show this same logic applied to both the databases and database table pages as well): DB-Console - https://www.loom.com/share/5108dd655ad342f28323e72eaf68219c - https://www.loom.com/share/1973383dacd7494a84e10bf39e5b85a3 This change applies the same error handling ideas from #109245 to the database details page, enabling non-admin users to use the database details page and providing better transparency to data fetching issues. Errors encountered while fetching table details can be viewed via the tooltip provided by the `Caution` icon at the table's name. `unavailable` cells also provide a tooltip that displays the error impacting that exact cell. Release note (ui change): Non-admin users are able to use the database details page. 110292: c2c: use seperate spanConfigEventStreamSpec in the span config event stream r=stevendanna a=msbutler Previously, the spanConfigEventStream used a streamPartitionSpec, which contained a bunch of fields unecessary for span config streaming. This patch creates a new spanConfigEventStreamSpec which contains the fields only necessary for span config event streaming. Informs #109059 Release note: None 110309: teamcity-trigger: ensure that `race` tag is only passed once r=healthy-pod a=healthy-pod By running under `-race`, the go command defines the `race` build tag for us [1]. Previously, we defined it under `TAGS` to let the issue poster know that this is a failure under `race` and indicate that in the issue. At the time, defining the tag twice didn't cause issues but after #109773, it led to build failures [2]. To reproduce locally: ``` bazel test -s --config=race pkg/util/ctxgroup:all --test_env=GOTRACEBACK=all --define gotags=bazel,gss,race ``` As a follow-up, we should find another way to let the issue poster know that a failure was running under `race`. [1] https://go.dev/doc/articles/race_detector#Excluding_Tests [2] #109994 (comment) Epic: none Release note: None Co-authored-by: Austen McClernon <[email protected]> Co-authored-by: Thomas Hardy <[email protected]> Co-authored-by: Michael Butler <[email protected]> Co-authored-by: healthy-pod <[email protected]>
5 parents 2810022 + fe8b67b + 001ca87 + fcf2650 + d99f8d5 commit 5fc80c3

File tree

17 files changed

+499
-395
lines changed

17 files changed

+499
-395
lines changed

pkg/ccl/streamingccl/streamproducer/event_stream.go

Lines changed: 7 additions & 14 deletions
Original file line numberDiff line numberDiff line change
@@ -484,10 +484,11 @@ func (s *eventStream) streamLoop(ctx context.Context, frontier *span.Frontier) e
484484
}
485485
}
486486

487+
const defaultBatchSize = 1 << 20
488+
487489
func setConfigDefaults(cfg *streampb.StreamPartitionSpec_ExecutionConfig) {
488490
const defaultInitialScanParallelism = 16
489491
const defaultMinCheckpointFrequency = 10 * time.Second
490-
const defaultBatchSize = 1 << 20
491492

492493
if cfg.InitialScanParallelism <= 0 {
493494
cfg.InitialScanParallelism = defaultInitialScanParallelism
@@ -502,27 +503,19 @@ func setConfigDefaults(cfg *streampb.StreamPartitionSpec_ExecutionConfig) {
502503
}
503504
}
504505

505-
func validateSpecs(evalCtx *eval.Context, spec streampb.StreamPartitionSpec) error {
506-
if !evalCtx.SessionData().AvoidBuffering {
507-
return errors.New("partition streaming requires 'SET avoid_buffering = true' option")
508-
}
509-
if len(spec.Spans) == 0 {
510-
return errors.AssertionFailedf("expected at least one span, got none")
511-
}
512-
return nil
513-
}
514-
515506
func streamPartition(
516507
evalCtx *eval.Context, streamID streampb.StreamID, opaqueSpec []byte,
517508
) (eval.ValueGenerator, error) {
518509
var spec streampb.StreamPartitionSpec
519510
if err := protoutil.Unmarshal(opaqueSpec, &spec); err != nil {
520511
return nil, errors.Wrapf(err, "invalid partition spec for stream %d", streamID)
521512
}
522-
if err := validateSpecs(evalCtx, spec); err != nil {
523-
return nil, err
513+
if !evalCtx.SessionData().AvoidBuffering {
514+
return nil, errors.New("partition streaming requires 'SET avoid_buffering = true' option")
515+
}
516+
if len(spec.Spans) == 0 {
517+
return nil, errors.AssertionFailedf("expected at least one span, got none")
524518
}
525-
526519
setConfigDefaults(&spec.Config)
527520

528521
execCfg := evalCtx.Planner.ExecutorConfig().(*sql.ExecutorConfig)

pkg/ccl/streamingccl/streamproducer/span_config_event_stream.go

Lines changed: 17 additions & 22 deletions
Original file line numberDiff line numberDiff line change
@@ -33,7 +33,7 @@ import (
3333

3434
type spanConfigEventStream struct {
3535
execCfg *sql.ExecutorConfig
36-
spec streampb.StreamPartitionSpec
36+
spec streampb.SpanConfigEventStreamSpec
3737
mon *mon.BytesMonitor
3838
acc mon.BoundAccount
3939

@@ -82,15 +82,15 @@ func (s *spanConfigEventStream) Start(ctx context.Context, txn *kv.Txn) error {
8282
s.doneChan = make(chan struct{})
8383

8484
// Reserve batch kvsSize bytes from monitor.
85-
if err := s.acc.Grow(ctx, s.spec.Config.BatchByteSize); err != nil {
86-
return errors.Wrapf(err, "failed to allocated %d bytes from monitor", s.spec.Config.BatchByteSize)
85+
if err := s.acc.Grow(ctx, defaultBatchSize); err != nil {
86+
return errors.Wrapf(err, "failed to allocated %d bytes from monitor", defaultBatchSize)
8787
}
8888

8989
s.rfc = rangefeedcache.NewWatcher(
9090
"spanconfig-subscriber",
9191
s.execCfg.Clock, s.execCfg.RangeFeedFactory,
92-
int(s.spec.Config.BatchByteSize),
93-
s.spec.Spans,
92+
defaultBatchSize,
93+
roachpb.Spans{s.spec.Span},
9494
true, // withPrevValue
9595
spanconfigkvsubscriber.NewSpanConfigDecoder().TranslateEvent,
9696
s.handleUpdate,
@@ -212,13 +212,10 @@ func (s *spanConfigEventStream) streamLoop(ctx context.Context) error {
212212
// TODO(msbutler): We may not need a pacer, given how little traffic will come from this
213213
// stream. That being said, we'd still want to buffer updates to ensure we
214214
// don't clog up the rangefeed. Consider using async flushing.
215-
pacer := makeCheckpointPacer(s.spec.Config.MinCheckpointFrequency)
215+
pacer := makeCheckpointPacer(s.spec.MinCheckpointFrequency)
216216
bufferedEvents := make([]streampb.StreamedSpanConfigEntry, 0)
217217
batcher := makeStreamEventBatcher()
218-
frontier, err := makeSpanConfigFrontier(s.spec.Spans)
219-
if err != nil {
220-
return err
221-
}
218+
frontier := makeSpanConfigFrontier(s.spec.Span)
222219

223220
for {
224221
select {
@@ -247,7 +244,7 @@ func (s *spanConfigEventStream) streamLoop(ctx context.Context) error {
247244
if err != nil {
248245
return err
249246
}
250-
if !tenantID.Equal(s.spec.Config.SpanConfigForTenant) {
247+
if !tenantID.Equal(s.spec.TenantID) {
251248
continue
252249
}
253250

@@ -278,19 +275,17 @@ func (s *spanConfigEventStream) streamLoop(ctx context.Context) error {
278275
}
279276
}
280277

281-
func makeSpanConfigFrontier(spans roachpb.Spans) (*spanConfigFrontier, error) {
282-
if len(spans) != 1 {
283-
return nil, errors.AssertionFailedf("unexpected input span length %d", len(spans))
284-
}
278+
func makeSpanConfigFrontier(span roachpb.Span) *spanConfigFrontier {
279+
285280
checkpoint := streampb.StreamEvent_StreamCheckpoint{
286281
ResolvedSpans: []jobspb.ResolvedSpan{{
287-
Span: spans[0],
282+
Span: span,
288283
},
289284
},
290285
}
291286
return &spanConfigFrontier{
292287
checkpoint: checkpoint,
293-
}, nil
288+
}
294289
}
295290

296291
type spanConfigFrontier struct {
@@ -301,13 +296,13 @@ func (spf *spanConfigFrontier) update(frontier hlc.Timestamp) {
301296
spf.checkpoint.ResolvedSpans[0].Timestamp = frontier
302297
}
303298

304-
func streamSpanConfigPartition(
305-
evalCtx *eval.Context, spec streampb.StreamPartitionSpec,
299+
func streamSpanConfigs(
300+
evalCtx *eval.Context, spec streampb.SpanConfigEventStreamSpec,
306301
) (eval.ValueGenerator, error) {
307-
if err := validateSpecs(evalCtx, spec); err != nil {
308-
return nil, err
302+
303+
if !evalCtx.SessionData().AvoidBuffering {
304+
return nil, errors.New("partition streaming requires 'SET avoid_buffering = true' option")
309305
}
310-
setConfigDefaults(&spec.Config)
311306

312307
return &spanConfigEventStream{
313308
spec: spec,

pkg/ccl/streamingccl/streamproducer/stream_lifetime.go

Lines changed: 6 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -354,19 +354,17 @@ func setupSpanConfigsStream(
354354
}); err != nil {
355355
return nil, err
356356
}
357-
358357
spanConfigKey := evalCtx.Codec.TablePrefix(uint32(spanConfigID))
359358

360359
// TODO(msbutler): crop this span to the keyspan within the span config
361360
// table relevant to this specific tenant once I teach the client.Subscribe()
362361
// to stream span configs, which will make testing easier.
363362
span := roachpb.Span{Key: spanConfigKey, EndKey: spanConfigKey.PrefixEnd()}
364363

365-
spec := streampb.StreamPartitionSpec{
366-
Spans: roachpb.Spans{span},
367-
Config: streampb.StreamPartitionSpec_ExecutionConfig{
368-
MinCheckpointFrequency: streamingccl.StreamReplicationMinCheckpointFrequency.Get(&evalCtx.Settings.SV),
369-
SpanConfigForTenant: tenantID,
370-
}}
371-
return streamSpanConfigPartition(evalCtx, spec)
364+
spec := streampb.SpanConfigEventStreamSpec{
365+
Span: span,
366+
TenantID: tenantID,
367+
MinCheckpointFrequency: streamingccl.StreamReplicationMinCheckpointFrequency.Get(&evalCtx.Settings.SV),
368+
}
369+
return streamSpanConfigs(evalCtx, spec)
372370
}

pkg/cmd/teamcity-trigger/main.go

Lines changed: 0 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -194,7 +194,6 @@ func runTC(queueBuild func(string, map[string]string)) {
194194
opts["env.EXTRA_BAZEL_FLAGS"] = fmt.Sprintf("%s --@io_bazel_rules_go//go/config:race --test_env=GORACE=halt_on_error=1", extraBazelFlags)
195195
opts["env.STRESSFLAGS"] = fmt.Sprintf("-maxruns %d -maxtime %s -maxfails %d -p %d",
196196
maxRuns, maxTime, maxFails, noParallelism)
197-
opts["env.TAGS"] = "race"
198197
queueBuildThenWait(buildID, opts)
199198
delete(opts, "env.TAGS")
200199
}

0 commit comments

Comments
 (0)