Skip to content

changefeedccl: migrate physical kvfeed to rangefeed.Factory client#168723

Open
aerfrei wants to merge 1 commit intocockroachdb:masterfrom
aerfrei:aerin/cdc-rangefeed-client-migration
Open

changefeedccl: migrate physical kvfeed to rangefeed.Factory client#168723
aerfrei wants to merge 1 commit intocockroachdb:masterfrom
aerfrei:aerin/cdc-rangefeed-client-migration

Conversation

@aerfrei
Copy link
Copy Markdown
Contributor

@aerfrei aerfrei commented Apr 20, 2026

Migrate from the low-level kvcoord.DistSender.RangeFeed API to the
high-level rangefeed.Factory client. The Factory uses typed callbacks
instead of a channel-based event loop and handles transient retry
logic internally.

Notable behavioral change: transient rangefeed errors (range splits,
leaseholder moves) are now retried inside the Factory rather than
surfacing to the kvfeed layer and triggering a full physical feed
restart. The Factory only forwards a narrow set of permanent errors to
the caller; we register an OnSSTable handler that errors so unexpected
SST events fail loud rather than falling into the silent-retry bucket.

The dead RangeObserver plumbing on kvFeed and rangeFeedConfig is
removed in passing — PR #163427 already replaced the
lagging_ranges/total_ranges DistSender-introspection observer with a
per-aggregator frontier poller.

Release note (sql change): The changefeed.bulk_delivery.enabled cluster
setting is now retired. The rangefeed client handles bulk delivery
internally and the setting is no longer consulted.
Epic: none

@trunk-io
Copy link
Copy Markdown
Contributor

trunk-io Bot commented Apr 20, 2026

Merging to master in this repository is managed by Trunk.

  • To merge this pull request, check the box to the left or comment /trunk merge below.

After your PR is submitted to the merge queue, this comment will be automatically updated with its status. If the PR fails, failure details will also be posted here

@cockroach-teamcity
Copy link
Copy Markdown
Member

This change is Reviewable

@aerfrei aerfrei force-pushed the aerin/cdc-rangefeed-client-migration branch from b3f5614 to 080e147 Compare April 21, 2026 14:52
@aerfrei aerfrei changed the title Aerin/cdc rangefeed client migration changefeedccl: migrate physical kvfeed to rangefeed.Factory client Apr 21, 2026
@aerfrei aerfrei force-pushed the aerin/cdc-rangefeed-client-migration branch 4 times, most recently from 29e6a78 to d57d956 Compare April 21, 2026 18:02
@aerfrei aerfrei requested a review from DarrylWong April 21, 2026 18:21
@aerfrei aerfrei marked this pull request as ready for review April 21, 2026 18:52
@aerfrei aerfrei requested a review from a team as a code owner April 21, 2026 18:52
cfg.Knobs.OnRangeFeedStart(cfg.Spans)
}

// TODO(aerfrei): rangefeed.Factory.New takes an initialTimestamp, but it
Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

If we like this PR otherwise, I'll file an issue for this change.

Copy link
Copy Markdown
Contributor

@DarrylWong DarrylWong left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

High level question: Before this change, my understanding is that a physical kv feed took raw kvpb.RangeFeedEvent and converted it to a KVEvents that changefeeds cared about. Do we still need a separate physical kvfeed layer now that we can just interface with the new client which does his for us?

// BulkDelivery enables bulk delivery of rangefeed events, which can improve performance during catchup scans.
// BulkDelivery is no longer used. The rangefeed client now handles bulk
// delivery internally.
var BulkDelivery = settings.RegisterBoolSetting(
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can you explain why we remove this outright rather than plumbing this option through the new rangefeed.Factory?

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The new rangefeed forces bulk delivery so it's not as simple as plumbing it through. This was also a temporary non-public setting that I think we planned to deprecate anyway.

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Gotcha, then I think my nit would be if we could put that info in the comment.

cancel(err)
}

onValue := func(ctx context.Context, value *kvpb.RangeFeedValue) {
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: it'd be nice for readability if these were top level functions instead of inlined.

Have you seen RangefeedHandler in pkg/crosscluster/producer/ordered_event_stream.go? I think something like that would be nice to copy (or perhaps outright extract to a common package and reuse).

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Agreed about these being top level. Can you elaborate more on why you think we should use RangefeedHandler here? I think it would be worthwhile to do if we want to have changefeeds use the ordered event stream, but I don't think it's worth doing immediately as part of this PR.

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Right I don't think we need the literal RangefeedHandler interface right now, I was more pointing to it is as an existing example of "how do we neatly describe all the ways we want to consume rangefeed events". I think copying this structure sets us up to go the interface route if we want as well as is visually easier to parse compared to sticking them in helper funcs.

// flush out every change before the schema change timestamp before we start
// emitting any changes from after the schema change. The KVFeed's
// `SchemaFeed` is responsible for detecting and enforcing these , but the
// after-KVFeed buffer doesn't have access to any of this state. A cleanup is
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can we address this TODO now that the new callbacks have access to frontier state? i.e. get rid of the second buffer and teach On* methods to handle schema changes?

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think this is a little more complicated and should be part of a separate PR, but I agree that we should do this. I think this PR sets us up better for that.

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

That seems reasonable but I think:

  1. It would be nice to do a timeboxed investigation into how hard it would be to actually do this. Naively from the TODO it sounds like all we would have to do is move the schemafeed down a layer into the callbacks but i'm sure there's more complexity than that.
  2. We should update and persist this TODO somewhere so we don't forget.

rangefeed.WithOnSSTable(func(
ctx context.Context, sst *kvpb.RangeFeedSSTable, _ roachpb.Span,
) {
setErr(errors.AssertionFailedf("unexpected SST ingestion: %v", sst))
Copy link
Copy Markdown
Contributor

@dt dt Apr 22, 2026

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: I think you can just not register an sst handler for the same effect; or are you saying we're retrying the default AssertionFailedf (which sounds like a bug)

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I added a test that shows that when I do that, if I include an SST event, the rangefeed hangs. I think generally retrying when we see an SST is intended but it being an AssertionFailedf is probably a bug/should be changed. I'll file an issue for that.

@aerfrei
Copy link
Copy Markdown
Contributor Author

aerfrei commented Apr 22, 2026

High level question: Before this change, my understanding is that a physical kv feed took raw kvpb.RangeFeedEvent and converted it to a KVEvents that changefeeds cared about. Do we still need a separate physical kvfeed layer now that we can just interface with the new client which does his for us?

@DarrylWong I think you're right that we don't really need as much of the scaffolding around the "physicalFeed" including "physicalFeedFactory". I don't think it's really a physical feed anymore, but I think the file separation is good organizationally. I renamed the file and simplified a little. Let me know what you think.

@aerfrei aerfrei requested a review from DarrylWong April 23, 2026 15:03
@aerfrei aerfrei force-pushed the aerin/cdc-rangefeed-client-migration branch from d57d956 to 5c8cecb Compare April 24, 2026 16:50
Migrate from the low-level kvcoord.DistSender.RangeFeed API to the
high-level rangefeed.Factory client. The Factory uses typed callbacks
instead of a channel-based event loop and handles transient retry
logic internally.

Notable behavioral change: transient rangefeed errors (range splits,
leaseholder moves) are now retried inside the Factory rather than
surfacing to the kvfeed layer and triggering a full physical feed
restart. The Factory only forwards a narrow set of permanent errors to
the caller; we register an OnSSTable handler that errors so unexpected
SST events fail loud rather than falling into the silent-retry bucket.

The dead RangeObserver plumbing on kvFeed and rangeFeedConfig is
removed in passing — PR cockroachdb#163427 already replaced the
lagging_ranges/total_ranges DistSender-introspection observer with a
per-aggregator frontier poller.

Release note (sql change): The changefeed.bulk_delivery.enabled cluster
setting is now retired. The rangefeed client handles bulk delivery
internally and the setting is no longer consulted.
Epic: none
@aerfrei aerfrei force-pushed the aerin/cdc-rangefeed-client-migration branch from 5c8cecb to 4e16926 Compare April 24, 2026 16:55
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

4 participants