-
Notifications
You must be signed in to change notification settings - Fork 4k
changefeedccl: block db-level changefeed creation for non-empty tableset #156771
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
base: master
Are you sure you want to change the base?
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -18,6 +18,7 @@ import ( | |
| "github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl/changefeedbase" | ||
| "github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl/changefeedvalidators" | ||
| "github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl/checkpoint" | ||
| "github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl/tableset" | ||
| "github.com/cockroachdb/cockroach/pkg/ccl/utilccl" | ||
| "github.com/cockroachdb/cockroach/pkg/cloud" | ||
| "github.com/cockroachdb/cockroach/pkg/cloud/externalconn" | ||
|
|
@@ -40,9 +41,11 @@ import ( | |
| "github.com/cockroachdb/cockroach/pkg/sql/catalog" | ||
| "github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo" | ||
| "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" | ||
| "github.com/cockroachdb/cockroach/pkg/sql/execinfra" | ||
| "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" | ||
| "github.com/cockroachdb/cockroach/pkg/sql/exprutil" | ||
| "github.com/cockroachdb/cockroach/pkg/sql/isql" | ||
| "github.com/cockroachdb/cockroach/pkg/sql/parser" | ||
| "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" | ||
| "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" | ||
| "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgnotice" | ||
|
|
@@ -59,6 +62,7 @@ import ( | |
| "github.com/cockroachdb/cockroach/pkg/util/log/channel" | ||
| "github.com/cockroachdb/cockroach/pkg/util/log/eventpb" | ||
| "github.com/cockroachdb/cockroach/pkg/util/log/severity" | ||
| "github.com/cockroachdb/cockroach/pkg/util/mon" | ||
| "github.com/cockroachdb/cockroach/pkg/util/randutil" | ||
| "github.com/cockroachdb/cockroach/pkg/util/span" | ||
| "github.com/cockroachdb/cockroach/pkg/util/syncutil" | ||
|
|
@@ -1413,6 +1417,24 @@ func requiresTopicInValue(s Sink) bool { | |
| return s.getConcreteType() == sinkTypeWebhook | ||
| } | ||
|
|
||
| // shouldStartDatabaseWatcher determines whether we should initialize a database-level | ||
| // tableset watcher. This is only applicable when: | ||
| // - there is exactly one target specification, and | ||
| // - that specification is a DATABASE target, and | ||
| // - there are currently no resolved table targets (i.e., zero unique tables). | ||
| func shouldStartDatabaseWatcher( | ||
| details jobspb.ChangefeedDetails, targets changefeedbase.Targets, | ||
| ) bool { | ||
| if len(details.TargetSpecifications) != 1 { | ||
| return false | ||
| } | ||
| ts := details.TargetSpecifications[0] | ||
| if ts.Type != jobspb.ChangefeedTargetSpecification_DATABASE { | ||
| return false | ||
| } | ||
| return targets.NumUniqueTables() == 0 | ||
| } | ||
|
|
||
| func makeChangefeedDescription( | ||
| ctx context.Context, | ||
| changefeed *tree.CreateChangefeed, | ||
|
|
@@ -1546,6 +1568,7 @@ type changefeedResumer struct { | |
|
|
||
| mu struct { | ||
| syncutil.Mutex | ||
| watcherDiffSent bool | ||
| // perNodeAggregatorStats is a per component running aggregate of trace | ||
| // driven AggregatorStats emitted by the processors. | ||
| perNodeAggregatorStats bulk.ComponentAggregatorStats | ||
|
|
@@ -1769,6 +1792,12 @@ func (b *changefeedResumer) resumeWithRetries( | |
|
|
||
| maxBackoff := changefeedbase.MaxRetryBackoff.Get(&execCfg.Settings.SV) | ||
| backoffReset := changefeedbase.RetryBackoffReset.Get(&execCfg.Settings.SV) | ||
|
|
||
| // Create memory monitor for tableset watcher. Similar to how processors create | ||
| // their monitors at function start, we create this unconditionally. | ||
| watcherMemMonitor := execinfra.NewMonitor(ctx, execCfg.DistSQLSrv.ChangefeedMonitor, mon.MakeName("changefeed-tableset-watcher-mem")) | ||
| defer watcherMemMonitor.Stop(ctx) | ||
|
|
||
| for r := getRetry(ctx, maxBackoff, backoffReset); r.Next(); { | ||
| flowErr := maybeUpgradePreProductionReadyExpression(ctx, jobID, details, jobExec) | ||
|
|
||
|
|
@@ -1782,7 +1811,7 @@ func (b *changefeedResumer) resumeWithRetries( | |
| knobs.BeforeDistChangefeed() | ||
| } | ||
|
|
||
| confPoller := make(chan struct{}) | ||
| changefeedDoneCh := make(chan struct{}) | ||
| g := ctxgroup.WithContext(ctx) | ||
| initialHighWater, schemaTS, err := computeDistChangefeedTimestamps(ctx, jobExec, details, localState) | ||
| if err != nil { | ||
|
|
@@ -1796,28 +1825,125 @@ func (b *changefeedResumer) resumeWithRetries( | |
| if err != nil { | ||
| return err | ||
| } | ||
|
|
||
| // This watcher is only used for db-level changefeeds with no watched tables. | ||
| var watcher *tableset.Watcher | ||
| watcherChan := make(chan []tableset.TableDiff, 1) | ||
| var cancelWatcher context.CancelFunc | ||
| var watcherCtx context.Context | ||
| waitForTables := shouldStartDatabaseWatcher(details, targets) | ||
| if waitForTables { | ||
| // Create a watcher for the database. | ||
| dbID := details.TargetSpecifications[0].DescID | ||
| filter := tableset.Filter{ | ||
| DatabaseID: dbID, | ||
| } | ||
| if details.TargetSpecifications[0].FilterList != nil && len(details.TargetSpecifications[0].FilterList.Tables) > 0 { | ||
| if details.TargetSpecifications[0].FilterList.FilterType == tree.IncludeFilter { | ||
| filter.IncludeTables = make(map[string]struct{}) | ||
| for fqTableName := range details.TargetSpecifications[0].FilterList.Tables { | ||
| // TODO: use fully qualified names once watcher supports it. | ||
| // Extract just the table name from the fully qualified name (e.g., "db.public.table" -> "table") | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. We definitely want to keep at least the schema name. Will need to wait for @asg0451 's change to the tableset watcher to know what to pass in.
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I'll update this once that inflight change is ready |
||
| tn, err := parser.ParseQualifiedTableName(fqTableName) | ||
| if err != nil { | ||
| return errors.Wrapf(err, "failed to parse name in filter list: %s", fqTableName) | ||
| } | ||
| filter.IncludeTables[tn.Object()] = struct{}{} | ||
| } | ||
| } else if details.TargetSpecifications[0].FilterList.FilterType == tree.ExcludeFilter { | ||
| filter.ExcludeTables = make(map[string]struct{}) | ||
| for fqTableName := range details.TargetSpecifications[0].FilterList.Tables { | ||
| // TODO: use fully qualified names once watcher supports it. | ||
| // Extract just the table name from the fully qualified name (e.g., "db.public.table" -> "table") | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Flagging this for removal before merge. |
||
| tn, err := parser.ParseQualifiedTableName(fqTableName) | ||
| if err != nil { | ||
| return errors.Wrapf(err, "failed to parse name in filter list: %s", fqTableName) | ||
| } | ||
| filter.ExcludeTables[tn.Object()] = struct{}{} | ||
| } | ||
| } | ||
| } | ||
|
|
||
| b.mu.Lock() | ||
| b.mu.watcherDiffSent = false | ||
| b.mu.Unlock() | ||
|
|
||
| watcher = tableset.NewWatcher(filter, execCfg, watcherMemMonitor, int64(jobID)) | ||
| g.GoCtx(func(ctx context.Context) error { | ||
| // This method runs the watcher until its context is canceled. | ||
| // The watcher context is canceled when diffs are sent to the | ||
| // watcherChan or when the the parent group context is canceled. | ||
| watcherCtx, cancelWatcher = context.WithCancel(ctx) | ||
| defer cancelWatcher() | ||
| err := watcher.Start(watcherCtx, schemaTS) | ||
| b.mu.Lock() | ||
| defer b.mu.Unlock() | ||
| if err != nil && !b.mu.watcherDiffSent { | ||
| return err | ||
| } | ||
| return nil | ||
| }) | ||
| } | ||
|
|
||
| // Run the changefeed until it completes or is shut down. | ||
| // If the changefeed's target tableset is empty, it will need to block | ||
| // until a diff in the tableset is found from the watcher. | ||
| g.GoCtx(func(ctx context.Context) error { | ||
| defer close(confPoller) | ||
| return startDistChangefeed(ctx, jobExec, jobID, schemaTS, details, description, | ||
| initialHighWater, localState, startedCh, onTracingEvent, targets) | ||
| defer close(changefeedDoneCh) | ||
| if waitForTables { | ||
| select { | ||
| case <-ctx.Done(): | ||
| return ctx.Err() | ||
| case diffs := <-watcherChan: | ||
| schemaTS = diffs[0].AsOf | ||
| initialHighWater = diffs[0].AsOf | ||
| targets, err = AllTargets(ctx, details, execCfg, schemaTS) | ||
| if err != nil { | ||
| return err | ||
|
Comment on lines
+1894
to
+1902
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. so we wait for the first event from the watcher before starting? i dont think that's right... what if the tables are already there and none change?
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I changed the conditional check for this if statement so it's more clear. But it will only enter into this select statement when there was an empty tableset and the watcher was created. If there's already tables, it will just go straight to running |
||
| } | ||
| } | ||
| } | ||
| return startDistChangefeed(ctx, jobExec, jobID, schemaTS, details, description, initialHighWater, localState, startedCh, onTracingEvent, targets) | ||
| }) | ||
|
|
||
| // Poll for updated configuration or new database tables if hibernating. | ||
| g.GoCtx(func(ctx context.Context) error { | ||
| t := time.NewTicker(15 * time.Second) | ||
| defer t.Stop() | ||
| for { | ||
| select { | ||
| case <-ctx.Done(): | ||
| return ctx.Err() | ||
| case <-confPoller: | ||
| case <-changefeedDoneCh: | ||
| return nil | ||
| case <-t.C: | ||
| case tick := <-t.C: | ||
| newDest, err := reloadDest(ctx, jobID, execCfg) | ||
| if err != nil { | ||
| log.Changefeed.Warningf(ctx, "failed to check for updated configuration: %v", err) | ||
| } else if newDest != resolvedDest { | ||
| resolvedDest = newDest | ||
| return replanErr | ||
| } | ||
| b.mu.Lock() | ||
| diffSent := b.mu.watcherDiffSent | ||
| b.mu.Unlock() | ||
| if waitForTables && !diffSent { | ||
| unchanged, diffs, err := watcher.PopUnchangedUpTo(ctx, hlc.Timestamp{WallTime: tick.UnixNano()}) | ||
| if err != nil { | ||
| return err | ||
| } | ||
| if !unchanged && len(diffs) > 0 { | ||
| select { | ||
| case watcherChan <- diffs: | ||
| b.mu.Lock() | ||
| b.mu.watcherDiffSent = true | ||
| b.mu.Unlock() | ||
| cancelWatcher() | ||
| case <-ctx.Done(): | ||
| return ctx.Err() | ||
| } | ||
| } | ||
| } | ||
| } | ||
| } | ||
| }) | ||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
why buffer?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
So the poller goroutine doesn't block on sending the diff and it can keep polling. Though the goroutine waiting for the diff, before starting the changefeed, should already be listening on this channel.