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.

This change also improves observability in rangefeed
client by introducing new counters:
 * `distsender.rangefeed.retry.<reason>`: counter keeping track
    of the number of ranges that ecountered a retryable error of
    particular type (e.g. slow counsumer, range split, etc).

Observability also enhanced by adding a column to
 `crdb_internal.active_rangefeed` virtual table augment to indicate
if the range is currently in catchup scan mode.

Fixes cockroachdb#98842

Release note (enterprise change): Pace rangefeed goroutine creation
rate to improve scheduler latency.  Improve observability by adding
additional metrics indicating the reason for rangefeed restart
as well as 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 Aug 28, 2023
1 parent 7437f5e commit ef0e5ca
Show file tree
Hide file tree
Showing 11 changed files with 225 additions and 120 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
91 changes: 78 additions & 13 deletions pkg/kv/kvclient/kvcoord/dist_sender.go
Original file line number Diff line number Diff line change
Expand Up @@ -222,13 +222,6 @@ This counts the number of ranges with an active rangefeed that are performing ca
Measurement: "Ranges",
Unit: metric.Unit_COUNT,
}
metaDistSenderRangefeedRestartStuck = metric.Metadata{
Name: "distsender.rangefeed.restart_stuck",
Help: `Number of times a rangefeed was restarted due to not receiving ` +
`timely updates (kv.rangefeed.range_stuck_threshold cluster setting)`,
Measurement: "Count",
Unit: metric.Unit_COUNT,
}
)

// CanSendToFollower is used by the DistSender to determine if it needs to look
Expand Down Expand Up @@ -310,9 +303,7 @@ type DistSenderMetrics struct {
type DistSenderRangeFeedMetrics struct {
RangefeedRanges *metric.Gauge
RangefeedCatchupRanges *metric.Gauge
RangefeedErrorCatchup *metric.Counter
RangefeedRestartRanges *metric.Counter
RangefeedRestartStuck *metric.Counter
Errors rangeFeedErrorCounters
}

func makeDistSenderMetrics() DistSenderMetrics {
Expand Down Expand Up @@ -353,13 +344,87 @@ func makeDistSenderMetrics() DistSenderMetrics {
return m
}

// rangeFeedErrorCounters are various error related counters for rangefeed.
type rangeFeedErrorCounters struct {
RangefeedRestartRanges *metric.Counter
RangefeedErrorCatchup *metric.Counter
RetryErrors []*metric.Counter
Stuck *metric.Counter
SendErrors *metric.Counter
StoreNotFound *metric.Counter
NodeNotFound *metric.Counter
RangeNotFound *metric.Counter
RangeKeyMismatch *metric.Counter
}

func makeRangeFeedErrorCounters() rangeFeedErrorCounters {
var retryCounters []*metric.Counter
for name := range kvpb.RangeFeedRetryError_Reason_value {
name = strings.TrimPrefix(name, "REASON_")
retryCounters = append(retryCounters, metric.NewCounter(metric.Metadata{
Name: fmt.Sprintf("distsender.rangefeed.retry.%s", strings.ToLower(name)),
Help: fmt.Sprintf(`Number of ranges that encountered retryable %s error`, name),
Measurement: "Ranges",
Unit: metric.Unit_COUNT,
}))
}

retryMeta := func(name string) metric.Metadata {
return metric.Metadata{
Name: fmt.Sprintf("distsender.rangefeed.retry.%s", strings.ReplaceAll(name, " ", "_")),
Help: fmt.Sprintf("Number of ranges that encountered retryable %s error", name),
Measurement: "Ranges",
Unit: metric.Unit_COUNT,
}
}

return rangeFeedErrorCounters{
RangefeedRestartRanges: metric.NewCounter(metaDistSenderRangefeedRestartRanges),
RangefeedErrorCatchup: metric.NewCounter(metaDistSenderRangefeedErrorCatchupRanges),
RetryErrors: retryCounters,
Stuck: metric.NewCounter(retryMeta("stuck")),
SendErrors: metric.NewCounter(retryMeta("send error")),
StoreNotFound: metric.NewCounter(retryMeta("store not found")),
NodeNotFound: metric.NewCounter(retryMeta("node not found")),
RangeNotFound: metric.NewCounter(retryMeta("range not found")),
RangeKeyMismatch: metric.NewCounter(retryMeta("range key mismatch")),
}
}

// GetRangeFeedRetryCounter returns retry reason counter for the specified reason.
// Use this method instead of direct counter access (since this method handles
// potential gaps in retry reason values).
func (c rangeFeedErrorCounters) GetRangeFeedRetryCounter(
reason kvpb.RangeFeedRetryError_Reason,
) *metric.Counter {
// Normally, retry reason values are contiguous. One way gaps could be
// introduced, is if some retry reasons are retired (deletions are
// accomplished by reserving enum value to prevent its re-use), and then more
// reason added after. Then, we can't use reason value as an index into
// retryCounters. Because this scenario is believed to be very unlikely, we
// forego any fancy re-mapping schemes, and instead opt for explicit handling.
switch reason {
case kvpb.RangeFeedRetryError_REASON_REPLICA_REMOVED,
kvpb.RangeFeedRetryError_REASON_RANGE_SPLIT,
kvpb.RangeFeedRetryError_REASON_RANGE_MERGED,
kvpb.RangeFeedRetryError_REASON_RAFT_SNAPSHOT,
kvpb.RangeFeedRetryError_REASON_LOGICAL_OPS_MISSING,
kvpb.RangeFeedRetryError_REASON_SLOW_CONSUMER,
kvpb.RangeFeedRetryError_REASON_NO_LEASEHOLDER,
kvpb.RangeFeedRetryError_REASON_RANGEFEED_CLOSED:
return c.RetryErrors[reason]
default:
panic(errors.AssertionFailedf("unknown retry reason %d", reason))
}
}

func (rangeFeedErrorCounters) MetricStruct() {}

func makeDistSenderRangeFeedMetrics() DistSenderRangeFeedMetrics {
return DistSenderRangeFeedMetrics{
RangefeedRanges: metric.NewGauge(metaDistSenderRangefeedTotalRanges),
RangefeedCatchupRanges: metric.NewGauge(metaDistSenderRangefeedCatchupRanges),
RangefeedErrorCatchup: metric.NewCounter(metaDistSenderRangefeedErrorCatchupRanges),
RangefeedRestartRanges: metric.NewCounter(metaDistSenderRangefeedRestartRanges),
RangefeedRestartStuck: metric.NewCounter(metaDistSenderRangefeedRestartStuck),
Errors: makeRangeFeedErrorCounters(),
}
}

Expand Down
29 changes: 11 additions & 18 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 @@ -468,8 +465,7 @@ func (m *rangefeedMuxer) receiveEventsFromNode(
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
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 All @@ -481,7 +477,7 @@ func (m *rangefeedMuxer) receiveEventsFromNode(
case *kvpb.RangeFeedError:
log.VErrEventf(ctx, 2, "RangeFeedError: %s", t.Error.GoError())
if active.catchupRes != nil {
m.metrics.RangefeedErrorCatchup.Inc(1)
m.metrics.Errors.RangefeedErrorCatchup.Inc(1)
}
ms.deleteStream(event.StreamID)
// Restart rangefeed on another goroutine. Restart might be a bit
Expand Down Expand Up @@ -519,15 +515,12 @@ func (m *rangefeedMuxer) restartActiveRangeFeeds(
func (m *rangefeedMuxer) restartActiveRangeFeed(
ctx context.Context, active *activeMuxRangeFeed, reason error,
) error {
m.metrics.RangefeedRestartRanges.Inc(1)
m.metrics.Errors.RangefeedRestartRanges.Inc(1)
active.setLastError(reason)

// 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 ef0e5ca

Please sign in to comment.