Skip to content

changefeedccl: fix timestamp for desc fetches during planning#155872

Merged
craig[bot] merged 1 commit intocockroachdb:masterfrom
log-head:log-head/db-level-changefeed-fixed-timestamps-3
Nov 12, 2025
Merged

changefeedccl: fix timestamp for desc fetches during planning#155872
craig[bot] merged 1 commit intocockroachdb:masterfrom
log-head:log-head/db-level-changefeed-fixed-timestamps-3

Conversation

@log-head
Copy link
Copy Markdown
Contributor

@log-head log-head commented Oct 22, 2025

When a changefeed is planned and executed, there are several places where target table descriptors are fetched. With a db-level changefeed, the set of tables can change during changefeed startup. Now, the timestamp is set to the schema timestamp for retrieving table descriptors.

Adding a schema_ts to the protobuf for ChangeAggregatorSpec and ChangeFrontierSpec.

Fixes: #154549
Epic: CRDB-1421

Release note: None

@blathers-crl
Copy link
Copy Markdown

blathers-crl bot commented Oct 22, 2025

It looks like your PR touches production code but doesn't add or edit any test code. Did you consider adding tests to your PR?

🦉 Hoot! I am a Blathers, a bot for CockroachDB. My owner is dev-inf.

@cockroach-teamcity
Copy link
Copy Markdown
Member

This change is Reviewable

@log-head log-head force-pushed the log-head/db-level-changefeed-fixed-timestamps-3 branch 5 times, most recently from 0b92c08 to 6d98f5b Compare October 24, 2025 18:57
@log-head log-head marked this pull request as ready for review October 24, 2025 20:02
@log-head log-head requested a review from a team as a code owner October 24, 2025 20:02
@log-head log-head requested review from KeithCh and aerfrei and removed request for a team and KeithCh October 24, 2025 20:02
@github-actions
Copy link
Copy Markdown
Contributor

Potential Bug(s) Detected

The three-stage Claude Code analysis has identified potential bug(s) in this PR that may warrant investigation.

Next Steps:
Please review the detailed findings in the workflow run.

After you review the findings, please tag the issue as follows:

  • If the detected issue is real or was helpful in any way, please tag the issue with O-AI-Review-Real-Issue-Found
  • If the detected issue was not helpful in any way, please tag the issue with O-AI-Review-Not-Helpful

@github-actions github-actions bot added the o-AI-Review-Potential-Issue-Detected AI reviewer found potential issue. Never assign manually—auto-applied by GH action only. label Oct 24, 2025
g := ctxgroup.WithContext(ctx)
targets, err := AllTargets(ctx, details, execCfg)
var changefeedStartTS hlc.Timestamp
if h := localState.progress.GetHighWater(); h != nil && !h.IsEmpty() {
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.

Is the localstate highwater the right thing to have here so long as it's set? When do we expect to end up in each branch of this if?

Nit: initializing changefeedStartTS with an immediately invoked function might allow you to do early returns and might be a little nicer here.

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.

This logic was taken from distChangefeedFlow, which computed the highWater and schemaTS before starting the changefeed. I've broken out computing those two timestamps into computeDistChangefeedTimestamps.

To answer your question, this part was just to use the highwater if it already exists (changefeed resuming after pause or restart); otherwise, since there's no highwater, start from the statement time.

}

prevTargets, err := AllTargets(ctx, prevDetails, p.ExecCfg())
prevTargets, err := AllTargets(ctx, prevDetails, p.ExecCfg(), statementTime)
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.

In this case we're altering a changefeed and getting the previous targets (i.e. before the alter). Is the right timestamp for this the statement time or something like prevProgress.GetHighWater()? My sense is it shouldn't make much of a difference, both should be at the same schema version etc and neither should have been GC'd.

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.

Good callout, fixed this.

execCfg = ca.knobs.OverrideExecCfg(execCfg)
}
ca.targets, err = AllTargets(ctx, ca.spec.Feed, execCfg)
ca.targets, err = AllTargets(ctx, ca.spec.Feed, execCfg, *ca.spec.SchemaTs)
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.

Since the schemaTs is marked as optional, do we need to check here in case it's null? Perhaps it shouldn't be optional.

Also confirming that since we expect to restart changefeeds on schemaChanges, this schemaTs should never need to be updated and it's never inaccurate.

nit: I think our preferred capitalization is SchemaTS

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.

Yes, need to check if it's null. Added that.

That's right, it's just for when the changefeed is first being started.

The capitalization for this is auto-generated from the protobuf, where it's schema_ts.

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.

You can override this with gogoproto.customname

// when the changefeed resumes.
repeated cockroach.sql.jobs.jobspb.ResolvedSpan resolved_spans = 11 [(gogoproto.nullable) = false];

optional util.hlc.Timestamp schema_ts = 12;
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.

I think you should call out that we're adding a timestamp into the spec in the commit body/PR desc but that change seems appropriate.

var schemaDescriptor catalog.SchemaDescriptor
var err error
f := func(ctx context.Context, txn descs.Txn) error {
if err := txn.KV().SetFixedTimestamp(ctx, schemaTS); err != nil {
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.

Is there a scenario where this fails in a similar way to what we're seeing elsewhere? If so, a test for that case would be cool.

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.

Made a unit test for this, similar to the others, but not able to get it to fail on master.

@aerfrei
Copy link
Copy Markdown
Contributor

aerfrei commented Oct 28, 2025

Looking good, I like the approach. Left some comments.

@log-head log-head force-pushed the log-head/db-level-changefeed-fixed-timestamps-3 branch from 6d98f5b to 964bff9 Compare October 29, 2025 19:15
@github-actions
Copy link
Copy Markdown
Contributor

Potential Bug(s) Detected

The three-stage Claude Code analysis has identified potential bug(s) in this PR that may warrant investigation.

Next Steps:
Please review the detailed findings in the workflow run.

Note: When viewing the workflow output, scroll to the bottom to find the Final Analysis Summary.

After you review the findings, please tag the issue as follows:

  • If the detected issue is real or was helpful in any way, please tag the issue with O-AI-Review-Real-Issue-Found
  • If the detected issue was not helpful in any way, please tag the issue with O-AI-Review-Not-Helpful

@log-head log-head force-pushed the log-head/db-level-changefeed-fixed-timestamps-3 branch from 964bff9 to 7260c71 Compare October 29, 2025 19:32
@github-actions
Copy link
Copy Markdown
Contributor

Potential Bug(s) Detected

The three-stage Claude Code analysis has identified potential bug(s) in this PR that may warrant investigation.

Next Steps:
Please review the detailed findings in the workflow run.

Note: When viewing the workflow output, scroll to the bottom to find the Final Analysis Summary.

After you review the findings, please tag the issue as follows:

  • If the detected issue is real or was helpful in any way, please tag the issue with O-AI-Review-Real-Issue-Found
  • If the detected issue was not helpful in any way, please tag the issue with O-AI-Review-Not-Helpful

@log-head log-head force-pushed the log-head/db-level-changefeed-fixed-timestamps-3 branch 3 times, most recently from 428e1d2 to 3388e6d Compare October 30, 2025 18:06
@log-head log-head added the O-AI-Review-Real-Issue-Found AI reviewer found real issue label Oct 30, 2025
@log-head log-head requested a review from aerfrei October 30, 2025 19:05
@log-head log-head force-pushed the log-head/db-level-changefeed-fixed-timestamps-3 branch from 968f023 to 7d4d3c9 Compare November 5, 2025 19:51
@log-head log-head requested a review from andyyang890 November 5, 2025 21:26
@log-head log-head force-pushed the log-head/db-level-changefeed-fixed-timestamps-3 branch from 7d4d3c9 to 792e1fb Compare November 6, 2025 18:24
@log-head log-head requested a review from andyyang890 November 6, 2025 18:25
Copy link
Copy Markdown
Collaborator

@andyyang890 andyyang890 left a comment

Choose a reason for hiding this comment

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

Looking good! Just have a few more minor comments

@log-head log-head force-pushed the log-head/db-level-changefeed-fixed-timestamps-3 branch from 792e1fb to 267c416 Compare November 7, 2025 17:00
@log-head log-head requested a review from andyyang890 November 7, 2025 17:16
@andyyang890 andyyang890 changed the title changefeedccl: Fix timestamp for desc fetches during planning changefeedccl: fix timestamp for desc fetches during planning Nov 7, 2025
Copy link
Copy Markdown
Collaborator

@andyyang890 andyyang890 left a comment

Choose a reason for hiding this comment

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

LGTM!

@log-head log-head force-pushed the log-head/db-level-changefeed-fixed-timestamps-3 branch 2 times, most recently from 056efe4 to 127e6c6 Compare November 7, 2025 21:36
case <-time.After(5 * time.Second):
t.Error("callback timed out waiting for table creation")
case <-ctx.Done():
t.Error("context canceled")
Copy link
Copy Markdown
Collaborator

@andyyang890 andyyang890 Nov 10, 2025

Choose a reason for hiding this comment

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

Do we need to error when the context gets canceled? Could we just have an empty case/return here?

@log-head log-head force-pushed the log-head/db-level-changefeed-fixed-timestamps-3 branch 3 times, most recently from 174403d to cfcf402 Compare November 10, 2025 21:43
@github-actions
Copy link
Copy Markdown
Contributor

Potential Bug(s) Detected

The three-stage Claude Code analysis has identified potential bug(s) in this PR that may warrant investigation.

Next Steps:
Please review the detailed findings in the workflow run.

Note: When viewing the workflow output, scroll to the bottom to find the Final Analysis Summary.

After you review the findings, please tag the issue as follows:

  • If the detected issue is real or was helpful in any way, please tag the issue with O-AI-Review-Real-Issue-Found
  • If the detected issue was not helpful in any way, please tag the issue with O-AI-Review-Not-Helpful

When a changefeed is planned and executed, there are several places
where target table descriptors are fetched. With a db-level changefeed,
the set of tables can change during changefeed startup. Now, the
timestamp is set to the schema timestamp for retrieving table
descriptors.

Adding a schema_ts to the protobuf for ChangeAggregatorSpec and
ChangeFrontierSpec.

Fixes: cockroachdb#154549
Epic: CRDB-1421

Release note: None
@log-head log-head force-pushed the log-head/db-level-changefeed-fixed-timestamps-3 branch from cfcf402 to 1d556c8 Compare November 12, 2025 16:20
@log-head
Copy link
Copy Markdown
Contributor Author

bors r=andyyang890,aerfrei

craig bot pushed a commit that referenced this pull request Nov 12, 2025
155872: changefeedccl: fix timestamp for desc fetches during planning r=andyyang890,aerfrei a=log-head

When a changefeed is planned and executed, there are several places where target table descriptors are fetched. With a db-level changefeed, the set of tables can change during changefeed startup. Now, the timestamp is set to the schema timestamp for retrieving table descriptors.

Adding a schema_ts to the protobuf for ChangeAggregatorSpec and ChangeFrontierSpec.

Fixes: #154549
Epic: CRDB-1421

Release note: None

157144: sql: add `ALTER TABLE ... SET/ADD IDENTITY` to the declarative schema… r=shghasemi a=shghasemi

… changer

This change adds support for `ALTER TABLE ... SET/ADD IDENTITY` to the declarative schema changer. This schema change operation runs using the legacy schema changer in versions older than 26.1.

Epic: CRDB-31283

Fixes #142918

Release note (sql change): `ALTER TABLE ... SET/ADD GENERATED AS IDENTITY` is supported by the declerative schema changer in 26.1 or later.

157158: roachtest: do not build arm64 and fips if probabilitiy is 0 r=DarrylWong a=rail

Previously, even when the probabilities for arm64 and fips builds were set to 0, the nightly roachtest script would still build those variants, which is unnecessary. This change modifies the script to skip building arm64 and fips variants when their respective probabilities are set to 0.

Epic: none
Release note: none

Co-authored-by: Matthew Lougheed <matthew.lougheed@cockroachlabs.com>
Co-authored-by: Shadi Ghasemitaheri <shadi.ghasemitaheri@cockroachlabs.com>
Co-authored-by: Rail Aliiev <rail@iqchoice.com>
@craig
Copy link
Copy Markdown
Contributor

craig bot commented Nov 12, 2025

Build failed (retrying...):

craig bot pushed a commit that referenced this pull request Nov 12, 2025
155872: changefeedccl: fix timestamp for desc fetches during planning r=andyyang890,aerfrei a=log-head

When a changefeed is planned and executed, there are several places where target table descriptors are fetched. With a db-level changefeed, the set of tables can change during changefeed startup. Now, the timestamp is set to the schema timestamp for retrieving table descriptors.

Adding a schema_ts to the protobuf for ChangeAggregatorSpec and ChangeFrontierSpec.

Fixes: #154549
Epic: CRDB-1421

Release note: None

157144: sql: add `ALTER TABLE ... SET/ADD IDENTITY` to the declarative schema… r=shghasemi a=shghasemi

… changer

This change adds support for `ALTER TABLE ... SET/ADD IDENTITY` to the declarative schema changer. This schema change operation runs using the legacy schema changer in versions older than 26.1.

Epic: CRDB-31283

Fixes #142918

Release note (sql change): `ALTER TABLE ... SET/ADD GENERATED AS IDENTITY` is supported by the declerative schema changer in 26.1 or later.

Co-authored-by: Matthew Lougheed <matthew.lougheed@cockroachlabs.com>
Co-authored-by: Shadi Ghasemitaheri <shadi.ghasemitaheri@cockroachlabs.com>
@craig
Copy link
Copy Markdown
Contributor

craig bot commented Nov 12, 2025

This PR was included in a batch that successfully built, but then failed to merge into master (it was a non-fast-forward update). It will be automatically retried.

@craig
Copy link
Copy Markdown
Contributor

craig bot commented Nov 12, 2025

@craig craig bot merged commit b4ddc73 into cockroachdb:master Nov 12, 2025
24 checks passed
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

o-AI-Review-Potential-Issue-Detected AI reviewer found potential issue. Never assign manually—auto-applied by GH action only. O-AI-Review-Real-Issue-Found AI reviewer found real issue v26.1.0-prerelease

Projects

None yet

Development

Successfully merging this pull request may close these issues.

changefeedccl: db-level feeds: use fixed timestamp for descriptor fetches

4 participants