Skip to content

Conversation

@andreimatei
Copy link
Contributor

See individual commits. They all improve the stopper.RunAsyncTask() interface. The only behavior change is that the DistSender's partial batches are not included in the parent's trace.

@cockroach-teamcity
Copy link
Member

This change is Reviewable

Copy link
Contributor

@erikgrinaker erikgrinaker left a comment

Choose a reason for hiding this comment

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

This is great, thanks for cleaning this up!

Another pattern I find myself repeating all over the place, and which gets unnecessarily verbose, is to run the task with a timeout (i.e. in combination with contextutil.RunWithTimeout). See e.g.:

cockroach/pkg/kv/txn.go

Lines 790 to 799 in 4e4f31a

if err := stopper.RunAsyncTask(ctx, "async-rollback", func(ctx context.Context) {
defer cancel()
// A batch with only endTxnReq is not subject to admission control, in
// order to reduce contention by releasing locks. In multi-tenant
// settings, it will be subject to admission control, and the zero
// CreateTime will give it preference within the tenant.
var ba roachpb.BatchRequest
ba.Add(endTxnReq(false /* commit */, nil /* deadline */, false /* systemConfigTrigger */))
_ = contextutil.RunWithTimeout(ctx, "async txn rollback", asyncRollbackTimeout,
func(ctx context.Context) error {

It would be great to have a Timeout parameter as well. This might get a bit more involved, since we'd have to add in some sort of error handling as well (although in the typical case we'd just want to log the error). Would be happy to submit a follow-up PR for this if you'd rather not deal with it now.

Reviewed 1 of 1 files at r1, 9 of 9 files at r2, 4 of 4 files at r3, 3 of 3 files at r4, 4 of 4 files at r5.
Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @andreimatei, @irfansharif, @nvanbenschoten, and @tbg)


pkg/kv/txn.go, line 788 at r4 (raw file):

	// actual cleanup will be independent of this context.
	stopper := txn.db.ctx.Stopper
	ctx, cancel := stopper.WithCancelOnQuiesce(contextutil.WithoutCancel(ctx))

The name may be a bit misleading here. It could be easy to read this and think "well, since the stopper sets cancellation, there's no need to remove it first" -- which would be wrong, of course. Perhaps contextutil.Disconnect() or DisconnectCancel() might be clearer.

@tbg tbg requested a review from erikgrinaker June 14, 2021 10:31
Copy link
Member

@tbg tbg left a comment

Choose a reason for hiding this comment

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

Thanks for working on this Andrei! Nothing super substantial in my comments, just a few suggestions.

@erikgrinaker in your snippet above, the error is discarded and so RunWithTimeout won't be doing anything useful (rather than actually setting up ctx cancellation). So adding a Timeout time.Duration parameter should provide parity.

I think what the snippet probably wants to do is to do the error handling on the error returned from RunWithTimeout. And then once it did that, yeah, for parity the error would have to be passed through the stopper somehow (so that it could do the error decoration). But I think this is getting a little awkward, and I don't want to bake logging into the stopper. I'd stay away from this at least for this PR, happy to help out to get something done in a follow-up.

Reviewed 1 of 1 files at r1, 9 of 9 files at r2, 4 of 4 files at r3, 3 of 3 files at r4, 4 of 4 files at r5.
Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @andreimatei, @erikgrinaker, @irfansharif, and @nvanbenschoten)


pkg/kv/txn.go, line 788 at r4 (raw file):

Previously, erikgrinaker (Erik Grinaker) wrote…

The name may be a bit misleading here. It could be easy to read this and think "well, since the stopper sets cancellation, there's no need to remove it first" -- which would be wrong, of course. Perhaps contextutil.Disconnect() or DisconnectCancel() might be clearer.

Hmm, I'm not sure. Cancellation (like all standard ops on Context) is additive. As you know

ctx, cancel := context.WithDeadline(context.Background(), 1*time.Second)
ctx, cancel = context.WithDeadline(ctx, 5*time.Second)

will result in ctx having a 1s deadline. Wouldn't your argument also imply that this is unexpected? "Oh, I'm setting a 5s deadline, which overrides the 1s deadline, so I can just skip that step"?


pkg/kv/txn.go, line 792 at r5 (raw file):

		stop.TaskOpts{
			TaskName:                "aync-rollback",
			DontInheritCancellation: true,

Forgetting about existing code that would change behavior, what would we want the defaults for DontInheritCancellation and CancelOnQuiesce to be? When we originally introduced the Stopper the idea was that it would be the main workhorse in a graceful shutdown. That hasn't hold true - I think when we quiesce it we mostly want "all tasks to go away promptly", which suggests that the default for CancelOnQuiesce should be true. For DontInheritCancellation, it might be the opposite? If you're launching an async task you usually have to expect that it will want to outlive the caller's context cancellation, and in fact we have had bugs in this area (in intent resolution I think).

While we're here, should we make that change? I'm suggesting we make the change and make sure all callers are updated to keep their old behavior.


pkg/util/contextutil/uncancelled_context.go, line 42 at r4 (raw file):

// WithoutCancel returns a context that doesn't inherit the cancellation of its
// parent, and so it can never be canceled.

If you eat a CI cycle anyway, maybe add that the values are inherited which is the whole point of this method.


pkg/util/stop/stopper.go, line 401 at r2 (raw file):

	}

	taskName := asyncTaskNamePrefix + opt.TaskName

Is asyncTaskNamePrefix worth it? It's extra allocations and also means that the final task name will not be that put into the span, which can be mildly confusing. We could take this refactor as the opportunity to phase this out, seeing how you point out that it was previously inconsistently applied.


pkg/util/stop/stopper.go, line 352 at r3 (raw file):
obligatory nit: even with a FollowsFrom relationship, it's the same trace. You could call this ChildSpan bool to avoid this.
In the comment, we should also be more precise (this stuff is intricate enough as is). Whether the relationship is ChildOf or FollowsFrom really doesn't matter. What we're after is to use WithParentAndAutoCollection instead of WithParentAndManualCollection, because the former will reference the span created for the async operation from the parent span and thus makes it retrievable through its recording. So I'd say something like

ChildSpan, if set, creates the tracing span for this task via tracing.ChildSpan instead of tracing.ForkSpan. This makes the task's span available via the parent span's recording (it is created with the WithParentAndAutoCollection option instead of WithParentAndManualCollection). It also leads to a ChildOf relationship instead of a FollowsFrom relationship to be used for the task's span, which typically implies a non-binding expectation that the parent span will outlive the task's span, i.e. that the parent will wait for the task to complete.

Regardless ....

Setting ...


pkg/util/stop/stopper.go, line 360 at r5 (raw file):

	DontInheritCancellation bool
	// CancelOnQuiesce runs the task in a context that gets canceled when the
	// stopper quiesces. If DontInheritCancellation is not set, then the tasks

task's


pkg/util/stop/stopper.go, line 361 at r5 (raw file):

	// CancelOnQuiesce runs the task in a context that gets canceled when the
	// stopper quiesces. If DontInheritCancellation is not set, then the tasks
	// context inherits the parent's cancelation, as well as the stopper's

cancellation

Copy link
Contributor

@erikgrinaker erikgrinaker left a comment

Choose a reason for hiding this comment

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

I think what the snippet probably wants to do is to do the error handling on the error returned from RunWithTimeout. And then once it did that, yeah, for parity the error would have to be passed through the stopper somehow (so that it could do the error decoration). But I think this is getting a little awkward, and I don't want to bake logging into the stopper. I'd stay away from this at least for this PR, happy to help out to get something done in a follow-up.

I think a better approach for RunWithTimeout generally is to add e.g. contextutil.WithTimeout() which returns a context that decorates the error on Err(). Then we wouldn't have to use a closure, and we wouldn't have to do any error handling in RunAsyncTaskEx().

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @andreimatei, @irfansharif, @nvanbenschoten, and @tbg)


pkg/kv/txn.go, line 788 at r4 (raw file):

Previously, tbg (Tobias Grieger) wrote…

Hmm, I'm not sure. Cancellation (like all standard ops on Context) is additive. As you know

ctx, cancel := context.WithDeadline(context.Background(), 1*time.Second)
ctx, cancel = context.WithDeadline(ctx, 5*time.Second)

will result in ctx having a 1s deadline. Wouldn't your argument also imply that this is unexpected? "Oh, I'm setting a 5s deadline, which overrides the 1s deadline, so I can just skip that step"?

I think it's just the "without cancel/with cancel" wording in that example that looks a bit odd. Not a big deal though, just a suggestion.

Copy link
Member

@tbg tbg left a comment

Choose a reason for hiding this comment

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

Yeah, unfortunately due to the history of context.Context it's a bit unclear if something will break. Err() is really only supposed to return context.DeadlineExceeded or context.Canceled; it says so right in the comment. Since the errors will also pop up in our vendored code, this isn't safe without a giant audit (and re-audit whenever deps change...)

What we could do is to use a convention:

func (ctx *crdbCtx) setErr(err error) {
  if errors.Is(err, context.DeadlineExceeded) {
    ctx.err = context.DeadlineExceeded
  } else {
    ctx.err = context.Canceled
  }
  ctx.origErr = err
}

func (ctx *crdbCtx) Value(k interface{}) interface{} {
  if k == ExtErrKey{} && ctx.origErr != nil {
    return ctx.origErr
  }
  return ctx.wrapped.Value(k)
}

and you could do this:

ctx, _ := contextutil.WithHelpfulTimeout(context.Background, 1*time.Second)
time.Sleep(2*time.Second)
require.Equal(t, context.DeadlineExceeded, ctx.Err())
require.True(t, errors.Is(contextutil.ExtendedError(ctx), (*contextutil.TimeoutError)(nil)))

This isn't that closely related to the stopper any more, but it's a real improvement over RunWithTimeout in my opinion; you don't need a closure which is great in the common case. We "just" need to get in the habit of using ExtendedError instead of ctx.Err(), but hey we have linted that sort of thing to death more than once.

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @andreimatei, @irfansharif, @nvanbenschoten, and @tbg)

@tbg tbg self-requested a review June 14, 2021 12:22
Copy link
Contributor Author

@andreimatei andreimatei 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 @erikgrinaker, @irfansharif, @nvanbenschoten, and @tbg)


pkg/kv/txn.go, line 788 at r4 (raw file):

I think it's just the "without cancel/with cancel" wording in that example that looks a bit odd. Not a big deal though, just a suggestion.

The odd alliteration goes away in the next commit.
I've considered various names for WithoutCancel, and it still seems to me like most suggestive option.


pkg/kv/txn.go, line 792 at r5 (raw file):

Previously, tbg (Tobias Grieger) wrote…

Forgetting about existing code that would change behavior, what would we want the defaults for DontInheritCancellation and CancelOnQuiesce to be? When we originally introduced the Stopper the idea was that it would be the main workhorse in a graceful shutdown. That hasn't hold true - I think when we quiesce it we mostly want "all tasks to go away promptly", which suggests that the default for CancelOnQuiesce should be true. For DontInheritCancellation, it might be the opposite? If you're launching an async task you usually have to expect that it will want to outlive the caller's context cancellation, and in fact we have had bugs in this area (in intent resolution I think).

While we're here, should we make that change? I'm suggesting we make the change and make sure all callers are updated to keep their old behavior.

I too think that CancelOnQuiesce should be the default. But I've had performance concerns - particularly because CancelOnQuiesce is currently wasteful when called on a ctx that itself inherits from a CancelOnQuiesce one - which would become very common. I've appended a commit that improves that implementation. But it gets into a more hands-on approach about the implementations of context.Context that we use, given that the stdlib one is not great. I think you'll welcome this, because you've been talking above about why that standard cancellation facilities suck. So please PTAL at that commit, and then I'll invert the defaults.

I'm also thinking about tying the default of DontInheritCancellation to the value of ChildSpan. Do you have thoughts on that? I'm thinking perhaps adding a simple way of creating a TaskOpts that sets both ChildSpan and InheritCancelation - perhaps as a TaskOpts ctor with a suggestive name (SyncChildTask()?).


pkg/util/stop/stopper.go, line 401 at r2 (raw file):

Previously, tbg (Tobias Grieger) wrote…

Is asyncTaskNamePrefix worth it? It's extra allocations and also means that the final task name will not be that put into the span, which can be mildly confusing. We could take this refactor as the opportunity to phase this out, seeing how you point out that it was previously inconsistently applied.

Indeed it allocates. I've removed it.


pkg/util/stop/stopper.go, line 352 at r3 (raw file):

Previously, tbg (Tobias Grieger) wrote…

obligatory nit: even with a FollowsFrom relationship, it's the same trace. You could call this ChildSpan bool to avoid this.
In the comment, we should also be more precise (this stuff is intricate enough as is). Whether the relationship is ChildOf or FollowsFrom really doesn't matter. What we're after is to use WithParentAndAutoCollection instead of WithParentAndManualCollection, because the former will reference the span created for the async operation from the parent span and thus makes it retrievable through its recording. So I'd say something like

ChildSpan, if set, creates the tracing span for this task via tracing.ChildSpan instead of tracing.ForkSpan. This makes the task's span available via the parent span's recording (it is created with the WithParentAndAutoCollection option instead of WithParentAndManualCollection). It also leads to a ChildOf relationship instead of a FollowsFrom relationship to be used for the task's span, which typically implies a non-binding expectation that the parent span will outlive the task's span, i.e. that the parent will wait for the task to complete.

Regardless ....

Setting ...

done


pkg/util/stop/stopper.go, line 360 at r5 (raw file):

Previously, tbg (Tobias Grieger) wrote…

task's

done


pkg/util/stop/stopper.go, line 361 at r5 (raw file):

Previously, tbg (Tobias Grieger) wrote…

cancellation

done


pkg/util/contextutil/uncancelled_context.go, line 42 at r4 (raw file):

Previously, tbg (Tobias Grieger) wrote…

If you eat a CI cycle anyway, maybe add that the values are inherited which is the whole point of this method.

done

Copy link
Contributor Author

@andreimatei andreimatei left a comment

Choose a reason for hiding this comment

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

I haven't done anything about deadlines in this PR, and I might leave the topic to another PR.
But I've appended another PR with some context hacking - PTAL. I sympathize with the Tobi's sentiment that the stdlib's context.Context is not a great library. I'd like to take steps towards being more aggressive in overcoming it's limitations around cancellation.

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @erikgrinaker, @irfansharif, @nvanbenschoten, and @tbg)

@tbg tbg requested review from erikgrinaker and tbg June 17, 2021 16:06
Copy link
Member

@tbg tbg left a comment

Choose a reason for hiding this comment

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

The broad strokes of this look good but unfortunately I ran out of time to give crdbctx.go a close look (which I think it deserves). LGTM mod that file, perhaps @erikgrinaker can review that.

Reviewed 13 of 13 files at r6, 10 of 10 files at r7, 6 of 6 files at r8, 3 of 3 files at r9, 4 of 4 files at r10, 4 of 4 files at r11.
Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @andreimatei, @erikgrinaker, @irfansharif, @nvanbenschoten, and @tbg)


pkg/util/contextutil/crdbctx.go, line 77 at r11 (raw file):

	uncParent := getCancellationBarrierParent(ctx)
	if uncParent == nil {
		return true

Comment here would be good, this is unintuitive at first.


pkg/util/contextutil/crdbctx.go, line 168 at r11 (raw file):

var cancellationBarrierKey = new(int)

// getUncanceledParent returns the innermost uncanceledContext amongst ctx's

there are a few references to uncanceledContext, is this detritus?


pkg/util/stop/stopper.go, line 352 at r8 (raw file):

	// ChildSpan, if set, creates the tracing span for the task via
	// tracing.ChildSpan() instead of tracing.ForkSpan. This makes the task's span
	// be part of the parent span's recording ((it is created with the

((

Copy link
Contributor

@erikgrinaker erikgrinaker left a comment

Choose a reason for hiding this comment

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

First quick pass over crdbctx.go, dumping some preliminary comments.

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @andreimatei, @irfansharif, @nvanbenschoten, and @tbg)


pkg/util/contextutil/crdbctx.go, line 62 at r11 (raw file):

// certain events (e.g. stopper quiescence) and then, in child tasks, checking
// whether we're still running inside such a context.
func Tagged(ctx context.Context, key interface{}, opt CtxWalkOption) bool {

Is opt important enough that it warrants being required for every call, or could we add an additional method for it when needed?

Also, please use tag instead of key, for symmetry with WithTag() and to make the code easier to follow.


pkg/util/contextutil/crdbctx.go, line 63 at r11 (raw file):

// whether we're still running inside such a context.
func Tagged(ctx context.Context, key interface{}, opt CtxWalkOption) bool {
	p, ok := ctx.Value(key).(*crdbCtx)

nit: could call this e.g. tagged or something, for clarity.


pkg/util/contextutil/crdbctx.go, line 71 at r11 (raw file):

	}

	if opt != CancellationParents {

nit: could use a switch here.


pkg/util/contextutil/crdbctx.go, line 126 at r11 (raw file):

// Value is part of the Context interface.
func (c *crdbCtx) Value(key interface{}) interface{} {
	if key == c.tag {

I'm finding it very unintuitive that we're overloading the semantics of Value to involve tag and cancellation barriers, and it makes Tagged() harder to reason about. I suppose it may be necessary in order to integrate with the stdlib context (is it?), but it should come with a big fat comment in that case.

@tbg tbg requested a review from erikgrinaker June 21, 2021 11:49
Copy link
Member

@tbg tbg 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 @andreimatei, @erikgrinaker, @irfansharif, @nvanbenschoten, and @tbg)


pkg/util/contextutil/crdbctx.go, line 20 at r11 (raw file):

// WithTag returns a context tagged with the provided tag. Calling Tagged(tag)
// on the returned context or on a child context will return true.

This needs to explain how

ctx  = WithTag(ctx, something)
require.True(t, Tagged(ctx))

is different from what you can do with vanilla Context:

ctx = context.WithValue(ctx, something, something)
_, ok := context.Value(something).(*somethingType)
require.True(t, ok)

The answer (only one I can come up with) is that you really care about the CancellationParents option. But you can get that with vanilla too:

ctx = context.WithValue(ctx, something, something)

ctx = context.WithoutCancel(context.WithValue(something, nil))
require.Nil(t, context.Value(something))

pkg/util/contextutil/crdbctx.go, line 42 at r11 (raw file):

type CtxWalkOption int

const (

This stuff just feels overly complex. Let's go back to the problem we actually want to solve. I'm sure we can do something simpler.


pkg/util/contextutil/crdbctx.go, line 126 at r11 (raw file):

Previously, erikgrinaker (Erik Grinaker) wrote…

I'm finding it very unintuitive that we're overloading the semantics of Value to involve tag and cancellation barriers, and it makes Tagged() harder to reason about. I suppose it may be necessary in order to integrate with the stdlib context (is it?), but it should come with a big fat comment in that case.

+1

@tbg tbg self-requested a review June 21, 2021 11:49
Implement RunLimitedAsyncTask in terms of RunAsyncTask. This also
unifies the behavior w.r.t the provided taskName - one of the two
methods was prepending an "[async]" and the other wasn't.

Release note: None
Add a new stoppper.RunAsyncTaskEx(), which takes an optional semaphore
argument. The new method uses an extensible set of options, as more
options are sure to come in the future.

Release note: None
Before this patch, since fairly recently, the spans created for
Stopper.RunAsyncTask[Ex]() were not included in the recording of the
caller's span because these spans were created with the ForkSpan(). This
patch adds an option to RunAsyncEx to make the task's span a child of
the caller's, and thus to be included in the caller's recording. This
option is used by the DistSender when sending partial batch requests,
therefore re-including these requests in higher-level traces (as they
used to be).

The reason why async tasks were detached from the caller's span
in cockroachdb#59815 was because of concerns about parent and children spans with
very different lifetimes becoming tied through a trace, delaying the
garbage collection of the whole trace until the last span in it
finishes. This patch gives the caller of RunAsyncTaskEx control over
this behavior; in the particular case of the DistSender, the traces are
not too long lived and the DistSender waits for the async tasks it
spawns to finish. In such circumstances, tying the allocations of the
child spans to the whole trace should be fine.

Release note (general change): A recent release removed parts of some
queries from the debugging traces of those queries. This information
(i.e. the execution of some low-level RPCs) has been re-included in the
traces.
Add contextutil.WithoutCancel(ctx), which returns a ctx identical to the
argument but without inheriting its cancellation signal. The patch uses
this in one caller, which results in simpler code.

Release note: None
Add an option to not inherit the context cancelation from the caller.
Before this option, the callers had to do this themselves. It's a
frequent enough need that it makes sense for the library to offer it, in
particular in conjunction with the 2nd option below.
Also add an option to cancel the task's context on stopper quiescence.
The caller could do this itself, but again it's quite common to want
this. The particular combination of wanting to not inherit the parent's
cancelation but to want cancelation on quiescence requires care when
done by the caller, because the order of the required context
manipulation matters: the stopper.WithCancelOnQuiescence() cannot be
used outside of the task's closure if the DontInheritCancellation option
is used.
One kv caller is switched to the new interface, resulting in simpler
code.

Release note: None
This patch makes WithCancelOnQuiesce be a no-op when called on a ctx
that was already inheriting cancellation on the quiescing of the
respective Stopper, avoiding multiple allocations. This optimization is
done with an eye towards making cancellation on quiescence the default
for stopper tasks.

Release note: None
Copy link
Contributor Author

@andreimatei andreimatei left a comment

Choose a reason for hiding this comment

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

Tobi, I have received an email with your last round of comments but, for whatever reason, they don't show up in Reviewable. In any case, they were all around WithTag/Tagged. I've removed those in favor of a simpler InheritsCancellation function that can be implemented only through the custom uncanceledContext, without the need for another custom "tagged" context. PTAL.

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @erikgrinaker, @irfansharif, @nvanbenschoten, and @tbg)


pkg/util/stop/stopper.go, line 352 at r8 (raw file):

Previously, tbg (Tobias Grieger) wrote…

((

d


pkg/util/contextutil/crdbctx.go, line 62 at r11 (raw file):

Previously, erikgrinaker (Erik Grinaker) wrote…

Is opt important enough that it warrants being required for every call, or could we add an additional method for it when needed?

Also, please use tag instead of key, for symmetry with WithTag() and to make the code easier to follow.

removed


pkg/util/contextutil/crdbctx.go, line 63 at r11 (raw file):

Previously, erikgrinaker (Erik Grinaker) wrote…

nit: could call this e.g. tagged or something, for clarity.

done


pkg/util/contextutil/crdbctx.go, line 71 at r11 (raw file):

Previously, erikgrinaker (Erik Grinaker) wrote…

nit: could use a switch here.

done


pkg/util/contextutil/crdbctx.go, line 77 at r11 (raw file):

Previously, tbg (Tobias Grieger) wrote…

Comment here would be good, this is unintuitive at first.

done


pkg/util/contextutil/crdbctx.go, line 126 at r11 (raw file):

Previously, erikgrinaker (Erik Grinaker) wrote…

I'm finding it very unintuitive that we're overloading the semantics of Value to involve tag and cancellation barriers, and it makes Tagged() harder to reason about. I suppose it may be necessary in order to integrate with the stdlib context (is it?), but it should come with a big fat comment in that case.

Added a comment. FWIW, the stdlib does the same kind of trick for its context implementations.


pkg/util/contextutil/crdbctx.go, line 168 at r11 (raw file):

Previously, tbg (Tobias Grieger) wrote…

there are a few references to uncanceledContext, is this detritus?

n/a

@tbg tbg self-requested a review June 21, 2021 19:32
Copy link
Member

@tbg tbg left a comment

Choose a reason for hiding this comment

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

For posterity, the email was from a Github review, see #66387 (review), somehow they're also in reviewable? Not sure what's going on but hitting Publish now & then reviewing your changes.

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @andreimatei, @erikgrinaker, @irfansharif, @nvanbenschoten, and @tbg)

Copy link
Contributor Author

@andreimatei andreimatei left a comment

Choose a reason for hiding this comment

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

@tbg, we've discussed offline yesterday replacing the uncanceledContext guy with a facility to copy parts of a context into a new one, and thus get rid of the parent's cancellation that way (by essentially creating a new context). Thinking about it again, I'm no longer sure what we'd gain; I wonder if we fooled ourselves.
The problem with the uncanceledContext is that the InheritsCancellation function only works if uncanceledContext is the only context implementation playing tricks with Done()/Error(). But the thing is, the implementation of InheritsCancellation based on the ctx copy scheme would have exactly the same problem. To recap, the point of the last commit is an optimization: making Stopper.WithCancelOnQuiesce a no-op (in particular, no allocations) if the parent already has a WithCancelOnQuiesce on the chain. Assuming that we want this optimization, we need a way to figure out whether a particular cancellation is inherited. How would we do that in the ctx-copy world? Presumably we'd simply look to see if the child ctx is annotated with the Stopper and if it is, assume that there's nobody on the chain that overrode Done()/Error(). So, we've gotten to the same assumption as the one InheritsCancellation() does in the current patch.

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @andreimatei, @erikgrinaker, @irfansharif, @nvanbenschoten, and @tbg)

Copy link
Member

@tbg tbg left a comment

Choose a reason for hiding this comment

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

Certainly possible that we're fooling ourselves. My reasoning is that we've agreed that it's invalid to "uncancel" a context and so rely on this being true (i.e. part of implementing the interface, in some unspoken sense). If someone else does it, we're screwed anyway; the whole point of contexts is that cancellation is inherited. "Uncancelling" really means, start anew from context.Background.

Reviewed 7 of 10 files at r13, 4 of 7 files at r14, 1 of 5 files at r16, 7 of 7 files at r17.
Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @erikgrinaker, @irfansharif, @nvanbenschoten, and @tbg)

@tbg tbg self-requested a review June 30, 2021 19:36
craig bot pushed a commit that referenced this pull request Jul 20, 2021
67713: stop, kvclient: include DistSender partial batches in traces r=andreimatei a=andreimatei

These are the first few patches from #66387 - extracting the non-contentious prefix dealing with DistSender tracing.

----

Before this patch, since fairly recently, the spans created for
Stopper.RunAsyncTask[Ex]() were not included in the recording of the
caller's span because these spans were created with the ForkSpan(). This
patch adds an option to RunAsyncEx to make the task's span a child of
the caller's, and thus to be included in the caller's recording. This
option is used by the DistSender when sending partial batch requests,
therefore re-including these requests in higher-level traces (as they
used to be).

The reason why async tasks were detached from the caller's span
in #59815 was because of concerns about parent and children spans with
very different lifetimes becoming tied through a trace, delaying the
garbage collection of the whole trace until the last span in it
finishes. This patch gives the caller of RunAsyncTaskEx control over
this behavior; in the particular case of the DistSender, the traces are
not too long lived and the DistSender waits for the async tasks it
spawns to finish. In such circumstances, tying the allocations of the
child spans to the whole trace should be fine.

Release note (general change): A recent release removed parts of some
queries from the debugging traces of those queries. This information
(i.e. the execution of some low-level RPCs) has been re-included in the
traces.

Co-authored-by: Andrei Matei <[email protected]>
@tbg tbg removed their request for review September 15, 2021 08:18
craig bot pushed a commit that referenced this pull request Dec 9, 2021
73554: kvserver: use and assert non-cancellable Raft scheduler context r=tbg a=erikgrinaker

`handleRaftReadyRaftMuLocked` is not prepared to handle context
cancellation. It is typically called via the Raft scheduler, which uses
a background context, but can be called via other paths as well (e.g.
snapshot application).

This patch adds an assertion that the given context is not cancellable,
and creates a new background context for the main scheduler code path
instead of using the CLI's cancellable context.

Release note: None

---

Split off from #73484, see previous discussion there.

Turns out that this fails because the Raft scheduler context is in fact cancellable. It's rooted at the CLI context:

https://github.com/cockroachdb/cockroach/blob/c3e8d8568467809c50a8eb8911fd120fe22661bb/pkg/cli/start.go#L407-L408

There's a few different options here, including:

1. Fixing `handleRaftReady` to to handle context cancellation safely.
2. Using a new background context for the Raft scheduler and populating it with necessary data from the passed context.
3. Getting @andreimatei's `contextutil.WithoutCancel()` from #66387 merged, and use it.

Co-authored-by: Erik Grinaker <[email protected]>
@irfansharif irfansharif removed their request for review January 4, 2022 23:26
@nvb nvb removed their request for review September 27, 2022 19:39
@erikgrinaker erikgrinaker removed their request for review November 8, 2022 12:27
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.

4 participants