-
Notifications
You must be signed in to change notification settings - Fork 3.8k
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
spanconfig: introduce the spanconfig.SQLWatcher #71968
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 |
---|---|---|
|
@@ -15,6 +15,7 @@ import ( | |
|
||
"github.com/cockroachdb/cockroach/pkg/keys" | ||
"github.com/cockroachdb/cockroach/pkg/roachpb" | ||
"github.com/cockroachdb/cockroach/pkg/sql/catalog" | ||
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" | ||
"github.com/cockroachdb/cockroach/pkg/util/hlc" | ||
) | ||
|
@@ -117,6 +118,59 @@ func FullTranslate( | |
return s.Translate(ctx, descpb.IDs{keys.RootNamespaceID}) | ||
} | ||
|
||
// SQLWatcher watches for events on system.zones and system.descriptors. | ||
type SQLWatcher interface { | ||
// WatchForSQLUpdates watches for updates to zones and descriptors starting at | ||
// the given timestamp (exclusive), informing callers using the handler | ||
// callback. | ||
// | ||
// The handler callback[1] is invoked from time to time with a list of updates | ||
// and a checkpointTS. Invocations of the handler callback provide the | ||
// following semantics: | ||
// 1. Calls to the handler are serial. | ||
// 2. The timestamp supplied to the handler is monotonically increasing. | ||
// 3. The list of DescriptorUpdates supplied to handler includes all events | ||
// in the window (prevInvocationCheckpointTS, checkpointTS]. | ||
// 4. No further calls to the handler are made if a call to the handler | ||
// returns an error. | ||
// | ||
// These guarantees mean that users of this interface are free to persist the | ||
// checkpointTS and later use it to re-establish a SQLWatcher without missing | ||
// any updates. | ||
// | ||
// WatchForSQLUpdates can only ever be called once, effectively making the | ||
// SQLWatcher a single use interface. | ||
// | ||
// WatchForSQLUpdates may run out of memory and return an error if it is | ||
// tracking too many events between two checkpoints. | ||
// | ||
// [1] Users of this interface should not intend to do expensive work in the | ||
// handler callback. | ||
// TODO(arul): Possibly get rid of this limitation. | ||
WatchForSQLUpdates( | ||
ctx context.Context, | ||
startTS hlc.Timestamp, | ||
handler func(ctx context.Context, updates []DescriptorUpdate, checkpointTS hlc.Timestamp) error, | ||
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. nit: give this a type |
||
) error | ||
} | ||
|
||
// DescriptorUpdate captures the ID and type of a descriptor or zone that the | ||
// SQLWatcher has observed updated. | ||
type DescriptorUpdate struct { | ||
// ID of the descriptor/zone that has been updated. | ||
ID descpb.ID | ||
|
||
// DescriptorType of the descriptor/zone that has been updated. Could be either | ||
// the specific type or catalog.Any if no information is available. | ||
DescriptorType catalog.DescriptorType | ||
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. should this be a bitmask so that the events corresponding to a single ID can be combined? |
||
} | ||
|
||
// SQLWatcherFactory is used to construct new SQLWatchers. | ||
type SQLWatcherFactory interface { | ||
// New returns a new SQLWatcher. | ||
New() SQLWatcher | ||
} | ||
|
||
// ReconciliationDependencies captures what's needed by the span config | ||
// reconciliation job to perform its task. The job is responsible for | ||
// reconciling a tenant's zone configurations with the clusters span | ||
|
@@ -126,11 +180,7 @@ type ReconciliationDependencies interface { | |
|
||
SQLTranslator | ||
|
||
// TODO(arul): We'll also want access to a "SQLWatcher", something that | ||
// watches for changes to system.{descriptors, zones} to feed IDs to the | ||
// SQLTranslator. These interfaces will be used by the "Reconciler to perform | ||
// full/partial reconciliation, checkpoint the span config job, and update KV | ||
// with the tenants span config state. | ||
SQLWatcherFactory | ||
} | ||
|
||
// Store is a data structure used to store spans and their corresponding | ||
|
@@ -235,8 +285,7 @@ func (u Update) Deletion() bool { | |
return u.Config.IsEmpty() | ||
} | ||
|
||
// Addition returns true if the update corresponds to a span config being | ||
// added. | ||
// Addition returns true if the update corresponds to a span config being added. | ||
func (u Update) Addition() bool { | ||
return !u.Deletion() | ||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -60,6 +60,7 @@ type Manager struct { | |
knobs *spanconfig.TestingKnobs | ||
|
||
spanconfig.KVAccessor | ||
spanconfig.SQLWatcherFactory | ||
spanconfig.SQLTranslator | ||
} | ||
|
||
|
@@ -73,21 +74,23 @@ func New( | |
stopper *stop.Stopper, | ||
settings *cluster.Settings, | ||
kvAccessor spanconfig.KVAccessor, | ||
sqlWatcherFactory spanconfig.SQLWatcherFactory, | ||
sqlTranslator spanconfig.SQLTranslator, | ||
knobs *spanconfig.TestingKnobs, | ||
) *Manager { | ||
if knobs == nil { | ||
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. This was intentional; I find it better to fold in the knobs nil check into the constructor for cases where the caller doesn't care about plumbing in a testing knob. Can we revert? |
||
knobs = &spanconfig.TestingKnobs{} | ||
} | ||
return &Manager{ | ||
db: db, | ||
jr: jr, | ||
ie: ie, | ||
stopper: stopper, | ||
settings: settings, | ||
KVAccessor: kvAccessor, | ||
SQLTranslator: sqlTranslator, | ||
knobs: knobs, | ||
db: db, | ||
jr: jr, | ||
ie: ie, | ||
stopper: stopper, | ||
settings: settings, | ||
KVAccessor: kvAccessor, | ||
SQLWatcherFactory: sqlWatcherFactory, | ||
SQLTranslator: sqlTranslator, | ||
knobs: knobs, | ||
} | ||
} | ||
|
||
|
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.
I think you should do more to explain the semantics of the memory errors. The caller is going to need to deal with them and restart. That's now part of the contract.