Skip to content

sql: backfill indexes at fixed TS#63945

Merged
craig[bot] merged 1 commit intocockroachdb:masterfrom
dt:idx-write-fixed-ts
Apr 22, 2021
Merged

sql: backfill indexes at fixed TS#63945
craig[bot] merged 1 commit intocockroachdb:masterfrom
dt:idx-write-fixed-ts

Conversation

@dt
Copy link
Copy Markdown
Contributor

@dt dt commented Apr 20, 2021

Previously the index backfiller would pick a timestamp on starting up or
resuming, and then use that timestamp for reading the source table and
for writing the index entries to the new index.

This however meant that if it restarted, it would need to re-scan the
partially built index in its entirety at that new time before it could
start writing at that new timestamp. For large indexes, this scan could
be very expensive.

This changes the backfiller to, once it has done a scan (when first
starting, while that index is still empty and can be scanned quickly and
cheaply) to pick a safe write time, then persist that time in the job.
If it resumes, it can then continue to write at that picked time without
re-scanning to pick one again, potentially then on a big, slow-to-scan
index.

Release note: none.

@dt dt requested review from a team, adityamaru and ajwerner April 20, 2021 19:09
@cockroach-teamcity
Copy link
Copy Markdown
Member

This change is Reviewable

@dt dt force-pushed the idx-write-fixed-ts branch 3 times, most recently from b9c4f2f to 84a5f5d Compare April 20, 2021 22:38
Copy link
Copy Markdown
Collaborator

@stevendanna stevendanna left a comment

Choose a reason for hiding this comment

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

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @adityamaru, @ajwerner, and @dt)

a discussion (no related file):
This seems reasonable to me, but I'm imaging you want some review by others more familiar with the code.

For my own edification:

I assume that writing at this older timestamp on resume is safe for the same reason that it was safe to write at that timestamp to begin with: any newer writes to that index (which should be DELETE_AND_WRITE_ONLY while this is happening) will contain the columns we are backfilling and will be at newer timestamps so our older writes here don't matter. Is that about right?


Copy link
Copy Markdown
Contributor Author

@dt dt left a comment

Choose a reason for hiding this comment

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

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @adityamaru, @ajwerner, and @stevendanna)

a discussion (no related file):

Previously, stevendanna (Steven Danna) wrote…

This seems reasonable to me, but I'm imaging you want some review by others more familiar with the code.

For my own edification:

I assume that writing at this older timestamp on resume is safe for the same reason that it was safe to write at that timestamp to begin with: any newer writes to that index (which should be DELETE_AND_WRITE_ONLY while this is happening) will contain the columns we are backfilling and will be at newer timestamps so our older writes here don't matter. Is that about right?

That's correct.

More precisely, that is also what was happening before -- we were already reading and writing at a fixed timestamp on each attempt to backfill the index. The change here is that that timestamp is now fixed across attempts for writes.

As you say, prior to backfill, the schema change advances the index to DELETE_AND_WRITE_ONLY so all new SQL writes during the backfill will create their own entries, so the backfill is only responsible for rows pre-dating that. We could correctly just read and write as of when we moved to WRITE_ONLY and that is indeed what we do for backfills that run to completion on the first attempt, but that time can fall out of GC window for long-running or paused and resumed backfills. So we consult the wall clock when we start or restart backfilling, so on restarts (including automatic ones if we hit TTL errors), we read and (prior to this change) write at a new fixed time for that attempt.

This is all well and good, except for one tiny detail with those writes: we can't blindly write at a time where SQL may have written too, or risk splitting an intent. So we protect against that by, prior to starting the backfill and its blind writes, scanning the target index at the intended write (and read) time to ensure existing intents are resolved and the timestamp cache is populated to prevent any new ones. This scan is trivial when the target index is empty prior to the first attempt, but if we've spent a day backfilling already, this scan could be very non-trivial.

However, we already scanned, on that first attempt, and identified a safe-to-write-at timestamp. Can we just keep writing at that time? We know we can't keep reading at that time as it could fall out of GC, so this question reduces to: Can we write at a time earlier than we read?

I believe the answer is yes. The difference between this write time and the later read time is entirely within the span where SQL writes, due to WTITE_ONLY state, are also updating the index so for anything where we would be reading a more recent change than our index write time, SQL will have a more recent index write anyway that will shadow whatever we write, so it doesn't matter what we do. Only where the backfill's write will be the last write by the end of the backfill does it really matter what the backfill writes, and that will only be true for rows where they were last touched prior to the backfill, so the later-read-than-write time is moot.

The only observable difference I can think of would be an as-of time scan of the index at a mid-backfill timestamp will now observe index entries at that time for rows added after that timestamp (which will, again, then have that same entry at the actual time they were added). That is, entries will appear before the row that added them, for example in an AOST query. However, I do not believe this is observable by an end user, via SQL, since the index is not public during this timespan, so you can't plan a query against it. You could only query that index with AOST times after it went public, which will be well after the SQL row is also visible, long after the window in which the back-dated index entry pre-dates its row.


Copy link
Copy Markdown

@postamar postamar left a comment

Choose a reason for hiding this comment

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

Please excuse my drive-by reviewing, I'm not able to give more than a superficial review to this PR, however I'm curious about a few things if you don't mind educating me.

if err := sc.job.RunningStatus(ctx, nil /* txn */, func(_ context.Context, _ jobspb.Details) (jobs.RunningStatus, error) {
return jobs.RunningStatus("scanning target index for in-progress transactions"), nil
}); err != nil {
return errors.Wrapf(err, "failed to update running status of job %d", errors.Safe(sc.job.ID()))
Copy link
Copy Markdown

Choose a reason for hiding this comment

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

My understanding is wrapping format args in errors.Safe is no longer recommended, as errors.Wrapf and the like will elide potential PII based on argument type alone.

// anything else from sneaking under us. Since these are new indexes, these
// spans should be essentially empty, so this should be a pretty quick and
// cheap scan.
const pageSize = 10000
Copy link
Copy Markdown

Choose a reason for hiding this comment

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

For my own education: how was this batch size chosen? 10k seems to be the norm. I'm guessing it doesn't matter too much as long as it's large enough to amortize the overhead but not too large?
I realize this code was already there and only shows up in the diff because it was wrapped in an if.

if err := sc.job.RunningStatus(ctx, nil /* txn */, func(_ context.Context, _ jobspb.Details) (jobs.RunningStatus, error) {
return RunningStatusBackfill, nil
}); err != nil {
return errors.Wrapf(err, "failed to update running status of job %d", errors.Safe(sc.job.ID()))
Copy link
Copy Markdown

Choose a reason for hiding this comment

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

ditto re. errors.Safe.

optional util.hlc.Timestamp readAsOf = 7 [(gogoproto.nullable) = false];
optional util.hlc.Timestamp writeAsOf = 7 [(gogoproto.nullable) = false];

optional util.hlc.Timestamp readAsOf = 9 [(gogoproto.nullable) = false];
Copy link
Copy Markdown

Choose a reason for hiding this comment

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

For my own education: why is it safe here to reassign tag 7 to writeAsOf and give readAsOf a new tag? I'm guessing backward and forward compatibility isn't a concern here, but I can't help to naively wonder what happens when we're running a mixed-version cluster? wouldn't you have old protos and new protos both being moved around and interpreted differently on old vs new version nodes? I can't stress the "naive" part of my understanding enough.

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.

Backwards compatibility / mixed-version behavior is absolutely a concern, which is precisely why I used the old tag (7) for the "new" write-as-of time field and used a new tag (9) for the "old" read-as-time field.

A time we scanned at is the only safe-to-write-at time -- we must not tell processors to write at a time unless we also scanned at it.

With this patch, if a new gateway node decides it does not want to re-scan at a new wall-time on a resume, but rather wants the whole flow to write at the old time, it needs to be sure the remote processors -- even if they haven't upgraded yet -- will write at that timestamp. What time do remote processors (pre-patch) write at? the one in field 7 (which they call "readAsOf"). By putting the write time in field 7, an old processor will write at that time. It'll also read at that time which means it might, if that time is far in the past, be under the TTL and the proc might get an error and fail. But that's fine -- you can update node and try again.

Copy link
Copy Markdown

Choose a reason for hiding this comment

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

Thank you for the explanation, this makes sense to me now.

Copy link
Copy Markdown
Contributor Author

@dt dt left a comment

Choose a reason for hiding this comment

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

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @adityamaru, @ajwerner, @dt, @postamar, and @stevendanna)


pkg/sql/backfill.go, line 1007 at r1 (raw file):

Previously, postamar (Marius Posta) wrote…

My understanding is wrapping format args in errors.Safe is no longer recommended, as errors.Wrapf and the like will elide potential PII based on argument type alone.

I was copying several other call-sites that set RunningStatus. I'm also hoping to backport this so if the redaction isn't the same on 20.2/21.1, I'd just as soon keep it like the other call-sites, then change them all at once on master?


pkg/sql/backfill.go, line 1018 at r1 (raw file):

Previously, postamar (Marius Posta) wrote…

For my own education: how was this batch size chosen? 10k seems to be the norm. I'm guessing it doesn't matter too much as long as it's large enough to amortize the overhead but not too large?
I realize this code was already there and only shows up in the diff because it was wrapped in an if.

pulled from thin air I think.

Hopefully, after this change, we'd never more than a handful of rows, let alone 10k, since we'd run this once, immediately after entering DELETE_AND_WRITE_ONLY, and then never again on the non-empty index.

Copy link
Copy Markdown
Contributor

@ajwerner ajwerner left a comment

Choose a reason for hiding this comment

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

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @adityamaru, @dt, @postamar, and @stevendanna)

a discussion (no related file):

Previously, dt (David Taylor) wrote…

That's correct.

More precisely, that is also what was happening before -- we were already reading and writing at a fixed timestamp on each attempt to backfill the index. The change here is that that timestamp is now fixed across attempts for writes.

As you say, prior to backfill, the schema change advances the index to DELETE_AND_WRITE_ONLY so all new SQL writes during the backfill will create their own entries, so the backfill is only responsible for rows pre-dating that. We could correctly just read and write as of when we moved to WRITE_ONLY and that is indeed what we do for backfills that run to completion on the first attempt, but that time can fall out of GC window for long-running or paused and resumed backfills. So we consult the wall clock when we start or restart backfilling, so on restarts (including automatic ones if we hit TTL errors), we read and (prior to this change) write at a new fixed time for that attempt.

This is all well and good, except for one tiny detail with those writes: we can't blindly write at a time where SQL may have written too, or risk splitting an intent. So we protect against that by, prior to starting the backfill and its blind writes, scanning the target index at the intended write (and read) time to ensure existing intents are resolved and the timestamp cache is populated to prevent any new ones. This scan is trivial when the target index is empty prior to the first attempt, but if we've spent a day backfilling already, this scan could be very non-trivial.

However, we already scanned, on that first attempt, and identified a safe-to-write-at timestamp. Can we just keep writing at that time? We know we can't keep reading at that time as it could fall out of GC, so this question reduces to: Can we write at a time earlier than we read?

I believe the answer is yes. The difference between this write time and the later read time is entirely within the span where SQL writes, due to WTITE_ONLY state, are also updating the index so for anything where we would be reading a more recent change than our index write time, SQL will have a more recent index write anyway that will shadow whatever we write, so it doesn't matter what we do. Only where the backfill's write will be the last write by the end of the backfill does it really matter what the backfill writes, and that will only be true for rows where they were last touched prior to the backfill, so the later-read-than-write time is moot.

The only observable difference I can think of would be an as-of time scan of the index at a mid-backfill timestamp will now observe index entries at that time for rows added after that timestamp (which will, again, then have that same entry at the actual time they were added). That is, entries will appear before the row that added them, for example in an AOST query. However, I do not believe this is observable by an end user, via SQL, since the index is not public during this timespan, so you can't plan a query against it. You could only query that index with AOST times after it went public, which will be well after the SQL row is also visible, long after the window in which the back-dated index entry pre-dates its row.

I'd love to see a test.



pkg/jobs/jobspb/jobs.proto, line 472 at r2 (raw file):

  uint32 format_version = 7 [(gogoproto.casttype) = "SchemaChangeDetailsFormatVersion"];

  util.hlc.Timestamp write_timestamp = 10 [(gogoproto.nullable) = false];

This deserves commentary


pkg/sql/index_backfiller.go, line 68 at r2 (raw file):

	}

	// TODO(dt): persist a write ts, don't rescan above.

Will you be addressing this in this PR?


pkg/sql/execinfrapb/processors_bulk_io.proto, line 64 at r2 (raw file):

  optional util.hlc.Timestamp writeAsOf = 7 [(gogoproto.nullable) = false];

  optional util.hlc.Timestamp readAsOf = 9 [(gogoproto.nullable) = false];

Commentary please

@dt dt force-pushed the idx-write-fixed-ts branch from fdebd26 to ce5cd28 Compare April 22, 2021 04:58
Copy link
Copy Markdown
Contributor Author

@dt dt left a comment

Choose a reason for hiding this comment

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

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @adityamaru, @ajwerner, @dt, @postamar, and @stevendanna)

a discussion (no related file):

Previously, ajwerner wrote…

I'd love to see a test.

Hm. What kind of test are you thinking? I ran the existing index backfill tests before/after and they don't change, but I don't think that is surprising since this shouldn't have any observable behavior change, since it is just about performance, right?



pkg/jobs/jobspb/jobs.proto, line 472 at r2 (raw file):

Previously, ajwerner wrote…

This deserves commentary

Done.


pkg/sql/index_backfiller.go, line 68 at r2 (raw file):

Previously, ajwerner wrote…

Will you be addressing this in this PR?

Wasn't planning on it -- this is dead code right now, right?


pkg/sql/execinfrapb/processors_bulk_io.proto, line 64 at r2 (raw file):

Previously, ajwerner wrote…

Commentary please

Done.

Copy link
Copy Markdown
Contributor

@ajwerner ajwerner left a comment

Choose a reason for hiding this comment

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

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @adityamaru, @dt, @postamar, and @stevendanna)

a discussion (no related file):

Previously, dt (David Taylor) wrote…

Hm. What kind of test are you thinking? I ran the existing index backfill tests before/after and they don't change, but I don't think that is surprising since this shouldn't have any observable behavior change, since it is just about performance, right?

I'm thinking, in particular, that you inject a failure during a backfill and ensure that the read moves forward but the writes remain the same.



pkg/sql/index_backfiller.go, line 68 at r2 (raw file):

Previously, dt (David Taylor) wrote…

Wasn't planning on it -- this is dead code right now, right?

oof right forgot what this was. carry on.

Previously the index backfiller would pick a timestamp on starting up or
resuming, and then use that timestamp for reading the source table and
for writing the index entries to the new index.

This however meant that if it restarted, it would need to re-scan the
partially built index in its entirety at that new time before it could
start writing at that new timestamp. For large indexes, this scan could
be very expensive.

This changes the backfiller to, once it has done a scan (when first
starting, while that index is still empty and can be scanned quickly and
cheaply) to pick a safe write time, then persist that time in the job.
If it resumes, it can then continue to write at that picked time without
re-scanning to pick one again, potentially then on a big, slow-to-scan
index.

Release note: none.
@dt dt force-pushed the idx-write-fixed-ts branch from ce5cd28 to 2f4545c Compare April 22, 2021 13:12
Copy link
Copy Markdown
Contributor Author

@dt dt left a comment

Choose a reason for hiding this comment

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

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @adityamaru, @dt, @postamar, and @stevendanna)

a discussion (no related file):

Previously, ajwerner wrote…

I'm thinking, in particular, that you inject a failure during a backfill and ensure that the read moves forward but the writes remain the same.

Added to the existing test that injects a GC error to cause a restart a check afterwards that the job's persisted write-at time is before that GC error.


Copy link
Copy Markdown
Contributor

@ajwerner ajwerner left a comment

Choose a reason for hiding this comment

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

Reviewed 3 of 8 files at r1, 1 of 1 files at r2, 5 of 5 files at r3.
Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @adityamaru, @dt, @postamar, and @stevendanna)

a discussion (no related file):

Previously, dt (David Taylor) wrote…

Added to the existing test that injects a GC error to cause a restart a check afterwards that the job's persisted write-at time is before that GC error.

LGTM


@dt
Copy link
Copy Markdown
Contributor Author

dt commented Apr 22, 2021

TFTR!

bors r+

@craig
Copy link
Copy Markdown
Contributor

craig bot commented Apr 22, 2021

Build succeeded:

@craig craig bot merged commit ee8b38e into cockroachdb:master Apr 22, 2021
@dt dt deleted the idx-write-fixed-ts branch April 27, 2021 01:29
rafiss added a commit to rafiss/cockroach that referenced this pull request Mar 26, 2025
PR cockroachdb#64023 made it so we use the backfill's read TS as the batch TS for
the AddSSTable operation. Simultaneously with that change, cockroachdb#63945 was
being worked on, which made it so that we use a fixed _write_ timestamp
for the keys that are backfilled. This write timestamp is the actual
minimum lower bound timestamp for the backfill operation, and the read
timstamp is allowed to change, so it makes more sense to use the write
timestamp for the AddSSTable operation. Looking at the diffs in the PRs
makes it seem pretty likely that this was just missed as part of a
trivial branch skew.

This commit does not cause any behavior change, since at the time of
writing, the read timestamp and write timestamp are always identical for
the backfill.

Release note: None
rafiss added a commit to rafiss/cockroach that referenced this pull request Mar 26, 2025
When cockroachdb#73861 got merged, we seem to have accidentally made it so the index
backfiller will never select a new read timestamp. That is contrary to
the goals of cockroachdb#63945, where we initially added the ability to advance the
read timestamp forward.

This mistake is why we were seeing behavior where a GC threshold error
would cause the backfill to retry infinitely (which in turn led us to treat
as a permanent error in cockroachdb#139203, which was something we never should have
done). We never noticed the bug because when CREATE INDEX was added to the
declarative schema changer in cockroachdb#92128, we turned off the declarative schema
changer in the test that would have caught this (i.e. TestIndexBackfillAfterGC).

This commit makes it so we choose a current timestamp as the read
timestamp when planning the backfill, and fixes up that test to show
that this works for index backfills in the declarative schema changer.

Release note (bug fix): Fixed a bug where a GC threshold error (which
appears as "batch timestamp must be after replica GC threshold ...")
could cause a schema change that backfills data to fail. Now, the error
will cause the backfill to be retried at a higher timestamp to avoid the
error.
craig bot pushed a commit that referenced this pull request Mar 26, 2025
143270: kvserver: better obs in TestTxnReadWithinUncertaintyIntervalAfterRangeMerge r=tbg a=tbg

Closes #143260.

This is a complex test with a rare failure mode. Trace all of the relevant operations so that we can meaningfully engage with it.

Epic: none
Release note: none

143451: sql,backfill: avoid holding a protected timestamp during the backfill r=rafiss a=rafiss

### sql,rowexec: use WriteTS as timestamp for AddSSTable in backfill

PR #64023 made it so we use the backfill's read TS as the batch TS for
the AddSSTable operation. Simultaneously with that change, #63945 was
being worked on, which made it so that we use a fixed _write_ timestamp
for the keys that are backfilled. This write timestamp is the actual
minimum lower bound timestamp for the backfill operation, and the read
timstamp is allowed to change, so it makes more sense to use the write
timestamp for the AddSSTable operation. Looking at the diffs in the PRs
makes it seem pretty likely that this was just missed as part of a
trivial branch skew.

This commit does not cause any behavior change, since at the time of
writing, the read timestamp and write timestamp are always identical for
the backfill.

Release note: None

### sql,backfill: choose new read timestamp when backfill job is resumed

When #73861 got merged, we seem to have accidentally made it so the index
backfiller will never select a new read timestamp. That is contrary to
the goals of #63945, where we initially added the ability to advance the
read timestamp forward.

This mistake is why we were seeing behavior where a GC threshold error
would cause the backfill to retry infinitely (which in turn led us to treat
as a permanent error in #139203, which was something we never should have
done). We never noticed the bug because when CREATE INDEX was added to the
declarative schema changer in #92128, we turned off the declarative schema
changer in the test that would have caught this (i.e. TestIndexBackfillAfterGC).

This commit makes it so we choose a current timestamp as the read
timestamp when planning the backfill, and fixes up that test to show
that this works for index backfills in the declarative schema changer.

Release note (bug fix): Fixed a bug where a GC threshold error (which
appears as "batch timestamp must be after replica GC threshold ...")
could cause a schema change that backfills data to fail. Now, the error
will cause the backfill to be retried at a higher timestamp to avoid the
error.

### sql, backfill: use a current timestamp to scan each chunk of the source index

There's no need for the backfill to use a fixed read timestamp for the
entire job. Instead, each chunk of the original index that needs to be
scanned can use a current timestamp. This allows us to remove all the
logic for adding protected timestamps for the backfill.

Release note (performance improvement): Schema changes that require
data to be backfilled no longer hold a protected timestamp for the
entire duration of the backfill, which means there is less overhead
caused by MVCC garbage collection after the backfill completes.

### sql: stop setting readAsOf in index backfiller

As of the parent commit, this is unused by the index backfiller.

Release note: None

---

fixes #140629
informs #142339
informs #142117
informs #141773

Co-authored-by: Tobias Grieger <tobias.b.grieger@gmail.com>
Co-authored-by: Rafi Shamim <rafi@cockroachlabs.com>
rafiss added a commit to rafiss/cockroach that referenced this pull request Mar 26, 2025
PR cockroachdb#64023 made it so we use the backfill's read TS as the batch TS for
the AddSSTable operation. Simultaneously with that change, cockroachdb#63945 was
being worked on, which made it so that we use a fixed _write_ timestamp
for the keys that are backfilled. This write timestamp is the actual
minimum lower bound timestamp for the backfill operation, and the read
timstamp is allowed to change, so it makes more sense to use the write
timestamp for the AddSSTable operation. Looking at the diffs in the PRs
makes it seem pretty likely that this was just missed as part of a
trivial branch skew.

This commit does not cause any behavior change, since at the time of
writing, the read timestamp and write timestamp are always identical for
the backfill.

Release note: None
rafiss added a commit to rafiss/cockroach that referenced this pull request Mar 26, 2025
When cockroachdb#73861 got merged, we seem to have accidentally made it so the index
backfiller will never select a new read timestamp. That is contrary to
the goals of cockroachdb#63945, where we initially added the ability to advance the
read timestamp forward.

This mistake is why we were seeing behavior where a GC threshold error
would cause the backfill to retry infinitely (which in turn led us to treat
as a permanent error in cockroachdb#139203, which was something we never should have
done). We never noticed the bug because when CREATE INDEX was added to the
declarative schema changer in cockroachdb#92128, we turned off the declarative schema
changer in the test that would have caught this (i.e. TestIndexBackfillAfterGC).

This commit makes it so we choose a current timestamp as the read
timestamp when planning the backfill, and fixes up that test to show
that this works for index backfills in the declarative schema changer.

Release note (bug fix): Fixed a bug where a GC threshold error (which
appears as "batch timestamp must be after replica GC threshold ...")
could cause a schema change that backfills data to fail. Now, the error
will cause the backfill to be retried at a higher timestamp to avoid the
error.
rafiss added a commit to rafiss/cockroach that referenced this pull request Mar 27, 2025
PR cockroachdb#64023 made it so we use the backfill's read TS as the batch TS for
the AddSSTable operation. Simultaneously with that change, cockroachdb#63945 was
being worked on, which made it so that we use a fixed _write_ timestamp
for the keys that are backfilled. This write timestamp is the actual
minimum lower bound timestamp for the backfill operation, and the read
timstamp is allowed to change, so it makes more sense to use the write
timestamp for the AddSSTable operation. Looking at the diffs in the PRs
makes it seem pretty likely that this was just missed as part of a
trivial branch skew.

This commit does not cause any behavior change, since at the time of
writing, the read timestamp and write timestamp are always identical for
the backfill.

Release note: None
rafiss added a commit to rafiss/cockroach that referenced this pull request Mar 27, 2025
When cockroachdb#73861 got merged, we seem to have accidentally made it so the index
backfiller will never select a new read timestamp. That is contrary to
the goals of cockroachdb#63945, where we initially added the ability to advance the
read timestamp forward.

This mistake is why we were seeing behavior where a GC threshold error
would cause the backfill to retry infinitely (which in turn led us to treat
as a permanent error in cockroachdb#139203, which was something we never should have
done). We never noticed the bug because when CREATE INDEX was added to the
declarative schema changer in cockroachdb#92128, we turned off the declarative schema
changer in the test that would have caught this (i.e. TestIndexBackfillAfterGC).

This commit makes it so we choose a current timestamp as the read
timestamp when planning the backfill, and fixes up that test to show
that this works for index backfills in the declarative schema changer.

Release note (bug fix): Fixed a bug where a GC threshold error (which
appears as "batch timestamp must be after replica GC threshold ...")
could cause a schema change that backfills data to fail. Now, the error
will cause the backfill to be retried at a higher timestamp to avoid the
error.
rafiss added a commit to rafiss/cockroach that referenced this pull request Mar 27, 2025
PR cockroachdb#64023 made it so we use the backfill's read TS as the batch TS for
the AddSSTable operation. Simultaneously with that change, cockroachdb#63945 was
being worked on, which made it so that we use a fixed _write_ timestamp
for the keys that are backfilled. This write timestamp is the actual
minimum lower bound timestamp for the backfill operation, and the read
timstamp is allowed to change, so it makes more sense to use the write
timestamp for the AddSSTable operation. Looking at the diffs in the PRs
makes it seem pretty likely that this was just missed as part of a
trivial branch skew.

This commit does not cause any behavior change, since at the time of
writing, the read timestamp and write timestamp are always identical for
the backfill.

Release note: None
rafiss added a commit to rafiss/cockroach that referenced this pull request Mar 27, 2025
When cockroachdb#73861 got merged, we seem to have accidentally made it so the index
backfiller will never select a new read timestamp. That is contrary to
the goals of cockroachdb#63945, where we initially added the ability to advance the
read timestamp forward.

This mistake is why we were seeing behavior where a GC threshold error
would cause the backfill to retry infinitely (which in turn led us to treat
as a permanent error in cockroachdb#139203, which was something we never should have
done). We never noticed the bug because when CREATE INDEX was added to the
declarative schema changer in cockroachdb#92128, we turned off the declarative schema
changer in the test that would have caught this (i.e. TestIndexBackfillAfterGC).

This commit makes it so we choose a current timestamp as the read
timestamp when planning the backfill, and fixes up that test to show
that this works for index backfills in the declarative schema changer.

Release note (bug fix): Fixed a bug where a GC threshold error (which
appears as "batch timestamp must be after replica GC threshold ...")
could cause a schema change that backfills data to fail. Now, the error
will cause the backfill to be retried at a higher timestamp to avoid the
error.
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.

5 participants