Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

changefeedccl: Expire protected timestamps #97148

Merged
merged 3 commits into from
Feb 24, 2023
Merged

Conversation

miretskiy
Copy link
Contributor

@miretskiy miretskiy commented Feb 14, 2023

Changefeeds utilize protected timestamp system (PTS)
to ensure that the data targeted by changefeed is not
garbage collected prematurely. PTS record is managed
by running changefeed by periodically updating
PTS record timestamp, so that the data older than
the that timestamp may be GCed. However, if the
changefeed stops running when it is paused (either due
to operator action, or due to on_error=pause option,
the PTS record remains so that the changefeed can
be resumed at a later time. However, it is also possible
that operator may not notice that the job is paused for
too long, thus causing buildup of garbage data.

Excessive buildup of GC work is not great since it
impacts overall cluster performance, and, once GC can resume,
its cost is proportional to how much GC work needs to be done.
This PR introduces a new changefeed option
gc_protect_expires_after to automatically expire PTS records that
are too old. This automatic expiration is a safety mechanism
in case changefeed job gets paused by an operator or due to
an error, while holding onto PTS record due to protect_gc_on_pause
option.
The operator is still expected to monitor changefeed jobs,
and to restart paused changefeeds expediently. If the changefeed
job remains paused, and the underlying PTS records expires, then
the changefeed job will be canceled to prevent build up of GC data.

Epic: CRDB-21953
Informs #84598

Release note (enterprise change): Changefeed will automatically
expire PTS records for paused jobs if changefeed is configured
with gc_protect_expires_after option.

@miretskiy miretskiy requested a review from a team as a code owner February 14, 2023 22:07
@miretskiy miretskiy requested a review from a team February 14, 2023 22:07
@miretskiy miretskiy requested review from a team as code owners February 14, 2023 22:07
@cockroach-teamcity
Copy link
Member

This change is Reviewable

Copy link
Contributor

@jayshrivastava jayshrivastava 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 4 files at r1, 1 of 1 files at r2, 15 of 15 files at r3, 12 of 12 files at r4, 3 of 7 files at r5.
Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @adityamaru, @dt, and @miretskiy)


-- commits line 8 at r2:
typo: processor


-- commits line 16 at r2:
typo: arise


pkg/ccl/changefeedccl/changefeed_processors.go line 1371 at r5 (raw file):

	recordID := progress.ProtectedTimestampRecord
	expiration := changefeedbase.PTSExpiresAfter.Get(&cf.flowCtx.Cfg.Settings.SV)

Maybe add an extra disable if zero check here. This will help in case the downstream code in reconciler.go changes.


pkg/jobs/registry.go line 1472 at r2 (raw file):

// WithJobMetrics returns a RegisterOption which will configure the job
// to use specified job metrics.

"Will configure jobs of this type to use specified metrics" is a bit more clear.


pkg/jobs/registry.go line 1900 at r3 (raw file):

func init() {
	//metricspoller.RegisterPeriodicClusterStatsCollector("paused-jobs", pollMetricsTask)

This init function should be deleted.


pkg/ts/catalog/chart_catalog.go line 3885 at r3 (raw file):

}

func jobTypeCharts(title string, varName string) chartDescription {

Nice.


pkg/jobs/jobs_test.go line 3507 at r3 (raw file):

	ctx := context.Background()
	// Make sure we set polling intervale before we start the server.

typo: interval


pkg/jobs/jobs_test.go line 3509 at r3 (raw file):

	// Make sure we set polling intervale before we start the server.
	// Otherwise, we might pick up the default value (30 second), which would make
	// this test slow.

good catch

Copy link
Contributor Author

@miretskiy miretskiy 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, and @jayshrivastava)


pkg/ccl/changefeedccl/changefeed_processors.go line 1371 at r5 (raw file):

Previously, jayshrivastava (Jayant) wrote…

Maybe add an extra disable if zero check here. This will help in case the downstream code in reconciler.go changes.

Not sure it's needed -- default is 0; and reconciler ignores 0 expiration...


pkg/jobs/registry.go line 1472 at r2 (raw file):

Previously, jayshrivastava (Jayant) wrote…

"Will configure jobs of this type to use specified metrics" is a bit more clear.

Done.


pkg/jobs/registry.go line 1900 at r3 (raw file):

Previously, jayshrivastava (Jayant) wrote…

This init function should be deleted.

Ooops; nice catch.


pkg/ts/catalog/chart_catalog.go line 3885 at r3 (raw file):

Previously, jayshrivastava (Jayant) wrote…

Nice.

Done.


pkg/jobs/jobs_test.go line 3507 at r3 (raw file):

Previously, jayshrivastava (Jayant) wrote…

typo: interval

it's Italian. Done.


pkg/jobs/jobs_test.go line 3509 at r3 (raw file):

Previously, jayshrivastava (Jayant) wrote…

good catch

Done.

@miretskiy
Copy link
Contributor Author

@jayshrivastava @adityamaru -- comments addressed, and CI is green. Would love to get another pass.

Copy link
Contributor

@adityamaru adityamaru left a comment

Choose a reason for hiding this comment

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

PTS and jobs changes LGTM! I'll let Jay give the final stamp though.

shouldRemove, err := task(ctx, txn, rec.Meta)
if err != nil {
return err
}
if !shouldRemove && rec.Target != nil &&
rec.Target.Expiration > 0 &&
rec.Timestamp.Add(int64(rec.Target.Expiration), 0).Less(r.clock.Now()) {
Copy link
Contributor

Choose a reason for hiding this comment

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

do you think this is an interesting enough event that we should log something here? Might be useful when we're looking at debug zips in the future

Copy link
Member

Choose a reason for hiding this comment

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

+1

I'd say we should at least warn here, since the expectation is that this shouldn't happen: the expectation with PTS is that the owner chooses, explicitly, when to release it.

I'm still a little unsure if this is even the layer where we should be doing expirations at all, but I'll make a top-level comment on this instead.

continue
}
p := j.Payload()
stats := ptsStats[p.Type()]
Copy link
Member

Choose a reason for hiding this comment

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

Type() will panic if it sees a job type it doesn't know about. We should get rid of it, as it just isn't a pattern that handles mixed version systems at all gracefully. In meantime, use CheckType.

shouldRemove, err := task(ctx, txn, rec.Meta)
if err != nil {
return err
}
if !shouldRemove && rec.Target != nil &&
rec.Target.Expiration > 0 &&
rec.Timestamp.Add(int64(rec.Target.Expiration), 0).Less(r.clock.Now()) {
Copy link
Member

Choose a reason for hiding this comment

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

+1

I'd say we should at least warn here, since the expectation is that this shouldn't happen: the expectation with PTS is that the owner chooses, explicitly, when to release it.

I'm still a little unsure if this is even the layer where we should be doing expirations at all, but I'll make a top-level comment on this instead.

@dt
Copy link
Member

dt commented Feb 19, 2023

If a foo job wants to protect a span, but expire that protection at time X if nothing else happens before X, it could also just... do that? i.e. run a little poller loop or fire off a background timer that just waits around and if the other thing doesn't happen, it just calls Release() to remove the protection when time X rolls around.

Having the job or at least a job-aware-level task drive the expiration would allow us to e.g. update the foo job to say "I'm giving up my protected timestamp" and go ahead and move to failed or something, which could also give a more useful error like "resumption window expired at X", rather than just getting opaque GC errors that also look like bugs / failures to protect things that we thought were protected.

In short, we don't need to bake the expiration into the protection itself, unless we're specifically worried about the failure mode where the task that is charged with releasing it goes away/gets stuck/etc. But we actually hashed a bunch of this out in the original PTS RFC discussion: we originally started with a lease-like, short-ish expiration and expected the owner to continually heartbeat their PTS record to push it forward and keep it alive, to ensure that we handled orphaned records well. But then we convinced ourselves that that wasn't the failure mode we wanted: we wanted to "fail-safe" and instead, so that if some job was slow/stuck and a little too late in updating its expiration, we didn't start GC and potentially cause data loss in the meantime.

Stepping back a little, there is also a UX here: basically, the behavior is going to be "if a changefeed falls too far behind -- either because it was paused or just slow/stuck, it will will fail and stop protecting history from gc". As it stands in this PR, it would do so by stopping protection of the timestamp and then subsequently failing the job when it can't read that timestamp (with a not very informative error about GC). Presenting this control to the user in this way is sorta weird and relies on them having an understanding of the internals of changefeeds be able to predict what will happen / interpret the errors: you need to know about timestamps and that not protecting a timestamp leads to failing. Also, it won't be consistent or predictable! If your backup schedule is still protecting it, then it might not fail, but then if your backup runs it will fail. I think this will end up with confused users. Instead, I wonder if we should be describing / presenting the behavior to users no in terms of protection, but in terms of "maximum lag" or something: i.e. make a changefeed that will fail, and release associated resources, if it falls further than X behind, including if it is paused for too long. The behavior is (mostly) the same, but I think the mental model is a little easier for users.

Additionally, I think we should be consistent and have the changefeed fail (and release its PTS) if it falls more than X behind even if a backup is still protecting that timestamp since otherwise you have a weird situation where most of the time it works, but then some of the time it fails, and that's bound to cause confusion.

Separately: I don't think a cluster setting is the right UX here: this seems like something that should be per-feed, not cluster-wide, since we should assume a single cluster holds different databases, for different apps, each with different tables where some may be critical and others not, so a cluster-wide setting isn't great. Also, cluster settings require admin so they're not great from granular access control / least-priv, since now you cdc-admin needs cluster-setting-admin which is a big step up in privs.

@dt
Copy link
Member

dt commented Feb 19, 2023

As a concrete counter-suggestion, I'd use the first two commits here but change the third/fourth:

  1. Add a new timestamp field on Payload (not job-specific) called MaximumPTSAge or something.
  2. Add a WITH maximum_allowed_lag or WITH maximum_retained_history or similar option on changefeed stmt, and store the duration if provided in that Payload field.
  3. Extend the metrics poller loop you already are using here, that already loads the PTS record and the Payload, to check if the payload has a max age and if so, if the PTS age exceeds it. Accumulate the set of IDs for which it does.
  4. For each job that had a MaximumPTSAge that was too old, call CANCEL JOB x WITH reason = 'lag / preserved history exceeded configured limit'

Cancelling the job will release the PTS, but do so along with a clear error on the job, and a clear chain of events -- job fell behind -> pts was too old -> job cancelled -> pts was released.

@miretskiy
Copy link
Contributor Author

If a foo job wants to protect a span, but expire that protection at time X if nothing else happens before X, it could also just... do that? i.e. run a little poller loop or fire off a background timer that just waits around and if the other thing doesn't happen, it just calls Release() to remove the protection when time X rolls around

That's sort of the problem I'm trying to solve. A job, w pts, gets paused. There is no longer act I poller for that job.

@dt
Copy link
Member

dt commented Feb 19, 2023

Yeah, I get that -- see my follow-up suggestion: we just added a polling looking at exactly what we want anyway, so we can just use it?

@miretskiy
Copy link
Contributor Author

I have a bone to pick on just going after paused jobs since any job that is blocking gc for a week or more, paused or otherwise, is still going to create the same problem, but we can talk about that later -- I don't want to let perfect be the enemy of good, and paused jobs is certainly amount get most common causes of old PTSs so this is certainly a huge win.

I need to think about the problem of catchup scans... Sometimes it's hard to determine what the right
threshold is... is 9 hours too much? is 12? It all depends on the amount of data...
I think I just went with a bit more restrictive, but safer approach. We can revisit if needed.

On the subject of backporting, what's the thinking? just pull the poller loop from a job into a goroutine that every registry runs instead in the back ported version?

I don't think we'll be backporting. Though, as you point out, there might be options.

@dt
Copy link
Member

dt commented Feb 23, 2023

PR/Merge-commit message needs to be updated to reflect the changes but otherwise LGTM

Changefeeds utilize protected timestamp system (PTS)
to ensure that the data targeted by changefeed is not
garbage collected prematurely.  PTS record is managed
by running changefeed by periodically updating
PTS record timestamp, so that the data older than
the that timestamp may be GCed.  However, if the
changefeed stops running when it is paused (either due
to operator action, or due to `on_error=pause` option,
the PTS record remains so that the changefeed can
be resumed at a later time. However, it is also possible
that operator may not notice that the job is paused for
too long, thus causing buildup of garbage data.

Excessive buildup of GC work is not great since it
impacts overall cluster performance, and, once GC can resume,
its cost is proportional to how much GC work needs to be done.
This PR introduces a new changefeed option
`gc_protect_expires_after` to automatically expire PTS records that
are too old.  This automatic expiration is a safety mechanism
in case changefeed job gets paused by an operator or due to
an error, while holding onto PTS record due to `protect_gc_on_pause`
option.
The operator is still expected to monitor changefeed jobs,
and to restart paused changefeeds expediently.  If the changefeed
job remains paused, and the underlying PTS records expires, then
the changefeed job will be canceled to prevent build up of GC data.

Epic: CRDB-21953
Informs cockroachdb#84598

Release note (enterprise change): Changefeed will automatically
expire PTS records for paused jobs if changefeed is configured
with `gc_protect_expires_after` option.
@miretskiy
Copy link
Contributor Author

PR/Merge-commit message needs to be updated to reflect the changes but otherwise LGTM

done; few more cleanups around failing tests (chart catalog, etc) are also fixed

@miretskiy
Copy link
Contributor Author

bors r+

@craig
Copy link
Contributor

craig bot commented Feb 23, 2023

👎 Rejected by code reviews

@miretskiy
Copy link
Contributor Author

@dt @jayshrivastava seems like bors insists on another vote from you....

@miretskiy
Copy link
Contributor Author

Reviewed 3 of 4 files at r1, 1 of 1 files at r2, 15 of 15 files at r3, 12 of 12 files at r4, 3 of 7 files at r5.
Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @adityamaru, @dt, and @miretskiy)

-- commits line 8 at r2: typo: processor

-- commits line 16 at r2: typo: arise

pkg/ccl/changefeedccl/changefeed_processors.go line 1371 at r5 (raw file):

	recordID := progress.ProtectedTimestampRecord
	expiration := changefeedbase.PTSExpiresAfter.Get(&cf.flowCtx.Cfg.Settings.SV)

Maybe add an extra disable if zero check here. This will help in case the downstream code in reconciler.go changes.

pkg/jobs/registry.go line 1472 at r2 (raw file):

// WithJobMetrics returns a RegisterOption which will configure the job
// to use specified job metrics.

"Will configure jobs of this type to use specified metrics" is a bit more clear.

pkg/jobs/registry.go line 1900 at r3 (raw file):

func init() {
	//metricspoller.RegisterPeriodicClusterStatsCollector("paused-jobs", pollMetricsTask)

This init function should be deleted.

pkg/ts/catalog/chart_catalog.go line 3885 at r3 (raw file):

}

func jobTypeCharts(title string, varName string) chartDescription {

Nice.

pkg/jobs/jobs_test.go line 3507 at r3 (raw file):

	ctx := context.Background()
	// Make sure we set polling intervale before we start the server.

typo: interval

pkg/jobs/jobs_test.go line 3509 at r3 (raw file):

	// Make sure we set polling intervale before we start the server.
	// Otherwise, we might pick up the default value (30 second), which would make
	// this test slow.

good catch

addressed

Copy link
Contributor

@jayshrivastava jayshrivastava left a comment

Choose a reason for hiding this comment

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

Unblocking

@miretskiy
Copy link
Contributor Author

bors r+

@craig
Copy link
Contributor

craig bot commented Feb 24, 2023

Build failed:

@miretskiy
Copy link
Contributor Author

bors r+

@craig
Copy link
Contributor

craig bot commented Feb 24, 2023

Build succeeded:

@craig craig bot merged commit 72bb291 into cockroachdb:master Feb 24, 2023
miretskiy pushed a commit to miretskiy/cockroach that referenced this pull request May 17, 2023
Changefeeds utilize protected timestamp (PTS) in order to ensure
that the data is not garbage collected (GC) prematurely.

This subsystem underwent through many rounds of changes, resulting
in an unintuitive, and potentially dangerous behavior.

This PR updates and improves PTS handling as follows.
PR cockroachdb#97148 introduce capability to cancel jobs that hold on to stale
PTS records.  This PR expands this functionality to apply to all
jobs -- not just paused jobs.

This is necessary because due to cockroachdb#90810, changefeeds will retry almost
every error -- and that means that if a running changefeed jobs fails
to make progress for very long time, it is possible that a PTS record
will protect GC collection for many days, weeks, or even months.

To guard against this case, introduce a new cluster setting
`changefeed.protect_timestamp.max_age`, which defaults to generous 4
days, to make sure that even if the explicit changefeed option
`gc_protect_expires_after` was not specified, the changefeed will fail
after `changefeed.protect_timestamp.max_age` if no progress is made.

The fail safe can be disabled by setting
`changefeed.protect_timestamp.max_age` to 0; Note, however, that doing
so could result in stability issues once stale PTS record released.

In addition, this PR deprecates `protect_data_from_gc_on_pause` option.
This option is not needed since we now employ "active protected
timestamp" management (meaning: there is always a PTS record when
running changefeed jobs), and the handling of this record is consistent
for both running and paused jobs.

Fixes cockroachdb#103464

Release note (enterprise change): Introduce a new
`changefeed.protect_timestamp.max_age` setting (default 4 days), which
will cancel running changefeed jobs if they fail to make forward
progress for much time.  This setting is used if the explicit
`gc_protect_expires_after` option was not set.  In addition, deprecate
`protect_data_from_gc_on_pause` option.  This option is no longer needed
since changefeed jobs always protect data.
miretskiy pushed a commit to miretskiy/cockroach that referenced this pull request May 19, 2023
Changefeeds utilize protected timestamp (PTS) in order to ensure
that the data is not garbage collected (GC) prematurely.

This subsystem underwent through many rounds of changes, resulting
in an unintuitive, and potentially dangerous behavior.

This PR updates and improves PTS handling as follows.
PR cockroachdb#97148 introduce capability to cancel jobs that hold on to stale
PTS records.  This PR expands this functionality to apply to all
jobs -- not just paused jobs.

This is necessary because due to cockroachdb#90810, changefeeds will retry almost
every error -- and that means that if a running changefeed jobs fails
to make progress for very long time, it is possible that a PTS record
will protect GC collection for many days, weeks, or even months.

To guard against this case, introduce a new cluster setting
`changefeed.protect_timestamp.max_age`, which defaults to generous 4
days, to make sure that even if the explicit changefeed option
`gc_protect_expires_after` was not specified, the changefeed will fail
after `changefeed.protect_timestamp.max_age` if no progress is made.
This setting only applies to newly created changefeeds.
Use `ALTER CHANGEFEED` statement to set `gc_protect_expires_after` option
for existing changefeeds to enable PTS expiration.

The fail safe can be disabled by setting
`changefeed.protect_timestamp.max_age` to 0; Note, however, that doing
so could result in stability issues once stale PTS record released.

In addition, this PR deprecates `protect_data_from_gc_on_pause` option.
This option is not needed since we now employ "active protected
timestamp" management (meaning: there is always a PTS record when
running changefeed jobs), and the handling of this record is consistent
for both running and paused jobs.

Fixes cockroachdb#103464

Release note (enterprise change): Introduce a new
`changefeed.protect_timestamp.max_age` setting (default 4 days), which
will cancel running changefeed jobs if they fail to make forward
progress for much time.  This setting is used if the explicit
`gc_protect_expires_after` option was not set.  In addition, deprecate
`protect_data_from_gc_on_pause` option.  This option is no longer needed
since changefeed jobs always protect data.
craig bot pushed a commit that referenced this pull request May 19, 2023
103528: server: Revert server.shutdown.jobs_wait to 0 r=miretskiy a=miretskiy

Revert default setting for `server.shutdown.jobs_wait` to 0
to ensure that shutdown dows not wait for active jobs.

Issues: none
Epic: None

Release note: None

103539: changefeedccl: Improve protected timestamp handling r=miretskiy a=miretskiy

Changefeeds utilize protected timestamp (PTS) in order to ensure that the data is not garbage collected (GC) prematurely.

This subsystem underwent through many rounds of changes, resulting in an unintuitive, and potentially dangerous behavior.

This PR updates and improves PTS handling as follows. PR #97148 introduce capability to cancel jobs that hold on to stale PTS records.  This PR expands this functionality to apply to all jobs -- not just paused jobs.

This is necessary because due to #90810, changefeeds will retry almost every error -- and that means that if a running changefeed jobs fails to make progress for very long time, it is possible that a PTS record will protect GC collection for many days, weeks, or even months.

To guard against this case, introduce a new cluster setting `changefeed.protect_timestamp.max_age`, which defaults to generous 4 days, to make sure that even if the explicit changefeed option `gc_protect_expires_after` was not specified, the changefeed will fail after `changefeed.protect_timestamp.max_age` if no progress is made.

The fail safe can be disabled by setting
`changefeed.protect_timestamp.max_age` to 0; Note, however, that doing so could result in stability issues once stale PTS record released.

In addition, this PR deprecates `protect_data_from_gc_on_pause` option. This option is not needed since we now employ "active protected timestamp" management (meaning: there is always a PTS record when running changefeed jobs), and the handling of this record is consistent for both running and paused jobs.

Fixes #103464

Release note (enterprise change): Introduce a new
`changefeed.protect_timestamp.max_age` setting (default 4 days), which will cancel running changefeed jobs if they fail to make forward progress for much time.  This setting is used if the explicit `gc_protect_expires_after` option was not set.  In addition, deprecate `protect_data_from_gc_on_pause` option.  This option is no longer needed since changefeed jobs always protect data.

Co-authored-by: Yevgeniy Miretskiy <[email protected]>
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