-
Notifications
You must be signed in to change notification settings - Fork 3.9k
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
cdc: add elastic CPU control to CDC event processing #91554
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 |
---|---|---|
|
@@ -13,15 +13,19 @@ import ( | |
"hash" | ||
"hash/crc32" | ||
"runtime" | ||
"time" | ||
|
||
"github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl/cdceval" | ||
"github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl/cdcevent" | ||
"github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl/changefeedbase" | ||
"github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl/kvevent" | ||
"github.com/cockroachdb/cockroach/pkg/roachpb" | ||
"github.com/cockroachdb/cockroach/pkg/sql/execinfra" | ||
"github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" | ||
"github.com/cockroachdb/cockroach/pkg/sql/sem/eval" | ||
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree" | ||
"github.com/cockroachdb/cockroach/pkg/util/admission" | ||
"github.com/cockroachdb/cockroach/pkg/util/admission/admissionpb" | ||
"github.com/cockroachdb/cockroach/pkg/util/bufalloc" | ||
"github.com/cockroachdb/cockroach/pkg/util/ctxgroup" | ||
"github.com/cockroachdb/cockroach/pkg/util/hlc" | ||
|
@@ -32,6 +36,10 @@ import ( | |
"github.com/cockroachdb/errors" | ||
) | ||
|
||
// pacerLogEvery is used for logging errors instead of returning terminal | ||
// errors when pacer.Pace returns an error. | ||
var pacerLogEvery log.EveryN = log.Every(100 * time.Millisecond) | ||
|
||
// eventContext holds metadata pertaining to event. | ||
type eventContext struct { | ||
updated, mvcc hlc.Timestamp | ||
|
@@ -62,6 +70,17 @@ type kvEventToRowConsumer struct { | |
|
||
topicDescriptorCache map[TopicIdentifier]TopicDescriptor | ||
topicNamer *TopicNamer | ||
|
||
// This pacer is used to incorporate event consumption to elastic CPU | ||
// control. This helps ensure that event encoding/decoding does not throttle | ||
// foreground SQL traffic. | ||
// | ||
// Note that for pacer to function correctly, | ||
// kvEventToRowConsumer.ConsumeEvent must be called by the same goroutine in a | ||
// tight loop. | ||
// | ||
// The pacer is closed by kvEventToRowConsumer.Close. | ||
pacer *admission.Pacer | ||
} | ||
|
||
func newEventConsumer( | ||
|
@@ -85,6 +104,9 @@ func newEventConsumer( | |
return nil, nil, err | ||
} | ||
|
||
pacerRequestUnit := changefeedbase.EventConsumerPacerRequestSize.Get(&cfg.Settings.SV) | ||
enablePacer := changefeedbase.EventConsumerElasticCPUControlEnabled.Get(&cfg.Settings.SV) | ||
|
||
makeConsumer := func(s EventSink, frontier frontier) (eventConsumer, error) { | ||
var err error | ||
encoder, err := getEncoder(encodingOpts, feed.Targets) | ||
|
@@ -100,22 +122,44 @@ func newEventConsumer( | |
} | ||
} | ||
|
||
// Passing a nil Pacer is effectively a noop Pacer if | ||
// CPU control is disabled. | ||
var pacer *admission.Pacer = nil | ||
if enablePacer { | ||
tenantID, ok := roachpb.TenantFromContext(ctx) | ||
if !ok { | ||
tenantID = roachpb.SystemTenantID | ||
} | ||
pacer = cfg.AdmissionPacerFactory.NewPacer( | ||
pacerRequestUnit, | ||
admission.WorkInfo{ | ||
TenantID: tenantID, | ||
Priority: admissionpb.BulkNormalPri, | ||
CreateTime: timeutil.Now().UnixNano(), | ||
BypassAdmission: false, | ||
}, | ||
) | ||
} | ||
|
||
return newKVEventToRowConsumer(ctx, cfg, evalCtx, frontier, cursor, s, | ||
encoder, details, expr, knobs, topicNamer) | ||
encoder, details, expr, knobs, topicNamer, pacer) | ||
} | ||
|
||
// TODO (jayshrivastava) enable parallel consumers for sinkless changefeeds | ||
numWorkers := changefeedbase.EventConsumerWorkers.Get(&cfg.Settings.SV) | ||
if numWorkers == 0 { | ||
// Pick a reasonable default. | ||
numWorkers = defaultNumWorkers() | ||
} | ||
|
||
// The descriptions for event_consumer_worker settings should also be updated | ||
// when these TODOs are completed. | ||
// | ||
// TODO (ganeshb) Add support for parallel encoding when using parquet. | ||
// We cannot have a separate encoder and sink for parquet format (see | ||
// parquet_sink_cloudstorage.go). Because of this the current nprox solution | ||
// does not work for parquet format. | ||
// | ||
//TODO (ganeshb) Add support for parallel encoding | ||
// TODO (jayshrivastava) enable parallel consumers for sinkless changefeeds. | ||
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. Do you remember why this was disabled in the first place? 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 replied in my latest comment 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. Having hard time finding it, I'm afraid. 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.
|
||
if numWorkers <= 1 || isSinkless || encodingOpts.Format == changefeedbase.OptFormatParquet { | ||
c, err := makeConsumer(sink, spanFrontier) | ||
if err != nil { | ||
|
@@ -174,6 +218,7 @@ func newKVEventToRowConsumer( | |
expr execinfrapb.Expression, | ||
knobs TestingKnobs, | ||
topicNamer *TopicNamer, | ||
pacer *admission.Pacer, | ||
) (*kvEventToRowConsumer, error) { | ||
includeVirtual := details.Opts.IncludeVirtual() | ||
keyOnly := details.Opts.KeyOnly() | ||
|
@@ -215,6 +260,7 @@ func newKVEventToRowConsumer( | |
evaluator: evaluator, | ||
safeExpr: safeExpr, | ||
encodingFormat: encodingOpts.Format, | ||
pacer: pacer, | ||
}, nil | ||
} | ||
|
||
|
@@ -242,6 +288,15 @@ func (c *kvEventToRowConsumer) ConsumeEvent(ctx context.Context, ev kvevent.Even | |
return errors.AssertionFailedf("expected kv ev, got %v", ev.Type()) | ||
} | ||
|
||
// Request CPU time to use for event consumption, block if this time is | ||
// unavailable. If there is unused CPU time left from the last call to | ||
// Pace, then use that time instead of blocking. | ||
if err := c.pacer.Pace(ctx); err != nil { | ||
if pacerLogEvery.ShouldLog() { | ||
log.Errorf(ctx, "automatic pacing: %v", err) | ||
} | ||
} | ||
|
||
schemaTimestamp := ev.KV().Value.Timestamp | ||
prevSchemaTimestamp := schemaTimestamp | ||
mvccTimestamp := ev.MVCCTimestamp() | ||
|
@@ -382,6 +437,7 @@ func (c *kvEventToRowConsumer) ConsumeEvent(ctx context.Context, ev kvevent.Even | |
// Close is a noop for the kvEventToRowConsumer because it | ||
// has no goroutines in flight. | ||
func (c *kvEventToRowConsumer) Close() error { | ||
c.pacer.Close() | ||
return nil | ||
} | ||
|
||
|
@@ -527,6 +583,12 @@ func (c *parallelEventConsumer) startWorkers() error { | |
func (c *parallelEventConsumer) workerLoop( | ||
ctx context.Context, consumer eventConsumer, id int64, | ||
) error { | ||
defer func() { | ||
err := consumer.Close() | ||
if err != nil { | ||
log.Errorf(ctx, "closing consumer: %v", err) | ||
} | ||
}() | ||
for { | ||
select { | ||
case <-ctx.Done(): | ||
|
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.
This workInfo struct should probably have some sort of a tag (an op-string or whatnot) identifying the queue. I know that at this point, pacer does not export per queue stats, but it probably should.
A todo would be fine (in the admission code)
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'm worried about what would happen if you tag a
WorkInfo
with one queue's info, but send it to another. Also, maybe queues should collect stats instead of thePacer
type, because they are underlying mechanism of the `Pacer type.It may make more sense to tag
WorkInfo
with the type/origin of the work and have queues collect metrics about the work coming in. We can group queues by their granter. For example, collect stats about cdc work being granted CPU time. @sumeerbhola, would you mind sharing your opinion about this?