Skip to content

Commit

Permalink
kvcoord: Pace rangefeed client goroutine creation
Browse files Browse the repository at this point in the history
Acquire catchup scan quota prior to goroutine creation in order to pace
the goroutine creation rate.

This change results in nice and smooth growth in
goroutine count, thus reducing the pressure on goroutine scheduler,
which in turn reduces the impact on SQL latency during changefeed
startup.

Improve observability over running changefeeds by
adding a column to `crdb_internal.active_rangefeed` virtual table
to indicate if the range is currently in catchup scan mode.

Fixes #98842

Release note (enterprise change): Pace rangefeed goroutine creation
rate to improve scheduler latency.  Improve observability by adding
additional column in the `crdb_internal.active_rangefeed`
table to indicate if the range is currently in catchup scan mode.
  • Loading branch information
Yevgeniy Miretskiy committed Sep 8, 2023
1 parent f316ead commit 8d321d4
Show file tree
Hide file tree
Showing 6 changed files with 128 additions and 87 deletions.
1 change: 1 addition & 0 deletions pkg/cli/zip_table_registry.go
Original file line number Diff line number Diff line change
Expand Up @@ -599,6 +599,7 @@ var zipInternalTablesPerNode = DebugZipTableRegistry{
"range_end",
"resolved",
"last_event_utc",
"catchup",
},
},
"crdb_internal.feature_usage": {
Expand Down
27 changes: 10 additions & 17 deletions pkg/kv/kvclient/kvcoord/dist_sender_mux_rangefeed.go
Original file line number Diff line number Diff line change
Expand Up @@ -95,7 +95,10 @@ func muxRangeFeed(
m.metrics = cfg.knobs.metrics
}

divideAllSpansOnRangeBoundaries(spans, m.startSingleRangeFeed, ds, &m.g)
m.g.GoCtx(func(ctx context.Context) error {
return divideAllSpansOnRangeBoundaries(ctx, spans, m.startSingleRangeFeed, ds, &m.g)
})

return errors.CombineErrors(m.g.Wait(), ctx.Err())
}

Expand Down Expand Up @@ -165,12 +168,6 @@ type activeMuxRangeFeed struct {
roachpb.ReplicaDescriptor
startAfter hlc.Timestamp

// catchupRes is the catchup scan quota acquired upon the
// start of rangefeed.
// It is released when this stream receives first non-empty checkpoint
// (meaning: catchup scan completes).
catchupRes catchupAlloc

// State pertaining to execution of rangefeed call.
token rangecache.EvictionToken
transport Transport
Expand Down Expand Up @@ -218,7 +215,7 @@ func (m *rangefeedMuxer) startSingleRangeFeed(

// Register active mux range feed.
stream := &activeMuxRangeFeed{
activeRangeFeed: newActiveRangeFeed(span, startAfter, m.registry, m.metrics.RangefeedRanges),
activeRangeFeed: newActiveRangeFeed(span, startAfter, m.registry, m.metrics),
rSpan: rs,
startAfter: startAfter,
token: token,
Expand Down Expand Up @@ -409,7 +406,7 @@ func (m *rangefeedMuxer) startNodeMuxRangeFeed(

// make sure that the underlying error is not fatal. If it is, there is no
// reason to restart each rangefeed, so just bail out.
if _, err := handleRangefeedError(ctx, recvErr); err != nil {
if _, err := handleRangefeedError(ctx, m.metrics, recvErr); err != nil {
// Regardless of an error, release any resources (i.e. metrics) still
// being held by active stream.
for _, s := range toRestart {
Expand Down Expand Up @@ -467,9 +464,8 @@ func (m *rangefeedMuxer) receiveEventsFromNode(
case *kvpb.RangeFeedCheckpoint:
if t.Span.Contains(active.Span) {
// If we see the first non-empty checkpoint, we know we're done with the catchup scan.
if !t.ResolvedTS.IsEmpty() && active.catchupRes != nil {
active.catchupRes.Release()
active.catchupRes = nil
if active.catchupRes != nil {
active.releaseCatchupScan()
}
// Note that this timestamp means that all rows in the span with
// writes at or before the timestamp have now been seen. The
Expand Down Expand Up @@ -524,10 +520,7 @@ func (m *rangefeedMuxer) restartActiveRangeFeed(

// Release catchup scan reservation if any -- we will acquire another
// one when we restart.
if active.catchupRes != nil {
active.catchupRes.Release()
active.catchupRes = nil
}
active.releaseCatchupScan()

doRelease := true
defer func() {
Expand All @@ -536,7 +529,7 @@ func (m *rangefeedMuxer) restartActiveRangeFeed(
}
}()

errInfo, err := handleRangefeedError(ctx, reason)
errInfo, err := handleRangefeedError(ctx, m.metrics, reason)
if err != nil {
// If this is an error we cannot recover from, terminate the rangefeed.
return err
Expand Down
Loading

0 comments on commit 8d321d4

Please sign in to comment.