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

kvserver: avoid races where replication changes can get interrupted #79379

Conversation

aayushshah15
Copy link
Contributor

@aayushshah15 aayushshah15 commented Apr 4, 2022

This commit adds a safeguard inside
Replica.maybeLeaveAtomicChangeReplicasAndRemoveLearners() to avoid removing
learner replicas when we know that that learner replica is in the process of
receiving its initial snapshot (as indicated by an in-memory lock on log
truncations that we place while the snapshot is in-flight).

This change should considerably reduce the instances where AdminRelocateRange
calls are interrupted by the mergeQueue or the replicateQueue (and vice versa).

Fixes #57129
Relates to #79118

Release note: none

Jira issue: CRDB-14769

@aayushshah15 aayushshah15 requested a review from a team as a code owner April 4, 2022 20:11
@cockroach-teamcity
Copy link
Member

This change is Reviewable

@aayushshah15 aayushshah15 force-pushed the 20220404_avoidLearnerRemovalRacesUsingLogTruncationConstraints branch 4 times, most recently from de97ccf to ca87e67 Compare April 4, 2022 21:34
// initial upreplication snapshot, we don't want to remove them and we bail
// out.
if r.hasOutstandingLearnerSnapshotInFlight() {
return desc, errors.New("cannot remove learner while a snapshot is in flight")
Copy link
Member

Choose a reason for hiding this comment

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

Nothing wrong with this code, but it'll give us new test flakes.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

I've made it such that we don't wait out the grace period for snapshot application when this method checks for the presence of an in-flight snapshot. All the flakes I was seeing (thus far) were because of that 3-second grace period.

@@ -1584,6 +1614,9 @@ func (r *Replica) getAndGCSnapshotLogTruncationConstraintsLocked(
if recipientStore != 0 && item.recipientStore != recipientStore {
continue
}
if item.deadline != (time.Time{}) && ignoreDeadline {
Copy link
Member

Choose a reason for hiding this comment

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

This seems named the wrong way around. If we ignore the deadline, shouldn't we look at all entries? We're ignoring entries that have a deadline, so consider renaming this param to ignoreEntsWithDeadline.
Also add a comment. It's not clear when the deadline is or isn't set. I think a deadline is set only before a transfer is inflight, but I am just guessing (and I wrote this code originally!)

Copy link
Member

Choose a reason for hiding this comment

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

maybe onlyInflight is a better name.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Thanks for taking a look! I've improved an existing comment and added another one explaining when the deadline is assigned. Let me know if you think it still needs to be improved.

@aayushshah15 aayushshah15 force-pushed the 20220404_avoidLearnerRemovalRacesUsingLogTruncationConstraints branch from ca87e67 to cf92d52 Compare April 4, 2022 21:53
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 code looks good, but I'm still surprised this doesn't cause issues in tests. When before a split would nuke the learner and do its thing, it will now fail with an opaque error that isn't going to be handled anywhere, so it will bubble up to the caller. For example, in workload kv init --splits 1000, if the cluster is still up-replicating, the error has to pop up at some point, no? And we certainly would have tests that do this. We should definitely make the error here structured so that we can backfill code to handle it where we find it sorely needed. And we may want to retry the error, at least for a little while. In some sense, what we really want is to block, not to fail fast, and I suspect that if we land the change as is, we will find our hand forced in that direction in practice.

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @aayushshah15 and @nvanbenschoten)


pkg/kv/kvserver/replica_command.go, line 1296 at r2 (raw file):

	// out.
	if r.hasOutstandingLearnerSnapshotInFlight() {
		return desc, errors.New("cannot remove learner while a snapshot is in flight")

make this a sentinel error (i.e. var errCannotRemoveLearnerWithInflightSnapshot = errors.New("...") and then the tests can also reference the error instead of hard-coding the string?


pkg/kv/kvserver/replica_raft.go, line 1539 at r2 (raw file):

		index:          index,
		recipientStore: recipientStore,
		// NB: We explicitly do not want to set a deadline here, since the presence

Consider a rename of the field

https://github.com/cockroachdb/cockroach/blob/0cc83717e5a32c69cd4917d82a7481fc73e1c402/pkg/kv/kvserver/replica_raft.go#L1564-L1565

to something more evocative of its semantics, like completedAt, and move the now.Add(gracePeriod) check into the the method that actually removes the entries. This will work better with the new use case introduced in this PR.


pkg/kv/kvserver/replica_learner_test.go, line 1205 at r2 (raw file):

			_ = tc.AddVotersOrFatal(t, scratchStartKey, tc.Target(1))
		})

spurious


pkg/kv/kvserver/replica_learner_test.go, line 1211 at r2 (raw file):

	atomic.StoreInt64(&activateSnapshotTestingKnob, 1)
	replicationChange := make(chan error)
	go func() {

Please use the stopper here.


pkg/kv/kvserver/replica_learner_test.go, line 1232 at r2 (raw file):

	db.Exec(t, `SET CLUSTER SETTING kv.range_merge.queue_enabled = true`)

	testutils.SucceedsWithin(

Why not just SucceedsSoon?


pkg/kv/kvserver/replica_learner_test.go, line 1255 at r2 (raw file):

	defer leaktest.AfterTest(t)()
	defer log.Scope(t).Close(t)

spurious

@aayushshah15 aayushshah15 force-pushed the 20220404_avoidLearnerRemovalRacesUsingLogTruncationConstraints branch from cf92d52 to 7858c29 Compare April 5, 2022 23:46
Copy link
Contributor Author

@aayushshah15 aayushshah15 left a comment

Choose a reason for hiding this comment

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

When before a split would nuke the learner and do its thing, it will now fail with an opaque error that isn't going to be handled anywhere, so it will bubble up to the caller

It looks like splits only ever cared about exiting joint config states, never about removing learner replicas: https://github.com/cockroachdb/cockroach/blob/7858c2908323c25dcea8083c47c1e459a336f5d5/pkg/kv/kvserver/replica_command.go#L316-L322

And we may want to retry the error, at least for a little while. In some sense, what we really want is to block, not to fail fast, and I suspect that if we land the change as is, we will find our hand forced in that direction in practice.

At the moment, the only entities that will fail-fast when they encounter an in-flight learner snapshot are the mergeQueue, the StoreRebalancer, or the replicateQueue (if it encounters the range while either the mergeQueue or the StoreRebalancer are rebalancing it). That seems good, doesn't it? As long as we're not blocking splits, we should fail-fast and let these queues move on to other ranges, right?

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


pkg/kv/kvserver/replica_command.go, line 1296 at r2 (raw file):

Previously, tbg (Tobias Grieger) wrote…

make this a sentinel error (i.e. var errCannotRemoveLearnerWithInflightSnapshot = errors.New("...") and then the tests can also reference the error instead of hard-coding the string?

Done.


pkg/kv/kvserver/replica_raft.go, line 1539 at r2 (raw file):

Previously, tbg (Tobias Grieger) wrote…

Consider a rename of the field

https://github.com/cockroachdb/cockroach/blob/0cc83717e5a32c69cd4917d82a7481fc73e1c402/pkg/kv/kvserver/replica_raft.go#L1564-L1565

to something more evocative of its semantics, like completedAt, and move the now.Add(gracePeriod) check into the the method that actually removes the entries. This will work better with the new use case introduced in this PR.

Done, much cleaner.


pkg/kv/kvserver/replica_learner_test.go, line 1205 at r2 (raw file):

Previously, tbg (Tobias Grieger) wrote…

spurious

Fixed.


pkg/kv/kvserver/replica_learner_test.go, line 1211 at r2 (raw file):

Previously, tbg (Tobias Grieger) wrote…

Please use the stopper here.

Done.


pkg/kv/kvserver/replica_learner_test.go, line 1232 at r2 (raw file):

Previously, tbg (Tobias Grieger) wrote…

Why not just SucceedsSoon?

No particular reason, changed.


pkg/kv/kvserver/replica_learner_test.go, line 1255 at r2 (raw file):

Previously, tbg (Tobias Grieger) wrote…

spurious

Fixed.

@aayushshah15 aayushshah15 force-pushed the 20220404_avoidLearnerRemovalRacesUsingLogTruncationConstraints branch from 7858c29 to fddc1cd Compare April 6, 2022 00:50
@aayushshah15
Copy link
Contributor Author

It looks like splits only ever cared about exiting joint config states, never about removing learner replicas

If we feel that this is undesirable and we do want splits to be able to remove learners before they begin, we could override that hasOutstandingLearnerSnapshotInFlight() check just for splits.

@aayushshah15 aayushshah15 force-pushed the 20220404_avoidLearnerRemovalRacesUsingLogTruncationConstraints branch from fddc1cd to 1f75586 Compare April 6, 2022 17:20
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.

It looks like splits only ever cared about exiting joint config states, never about removing learner replicas

Ah, thanks for reminding me of that. I think that is fine.

What about SCATTER? Hard to avoid the error there, lest we are willing to retry "for a long time", or return with empty hands.

Reviewed 3 of 5 files at r2, 3 of 3 files at r3, 1 of 1 files at r4, all commit messages.
Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @aayushshah15 and @nvanbenschoten)


pkg/kv/kvserver/replicate_queue.go, line 1255 at r4 (raw file):

}

func (rq *replicateQueue) removeLearner(

🎀

@aayushshah15 aayushshah15 force-pushed the 20220404_avoidLearnerRemovalRacesUsingLogTruncationConstraints branch from 1f75586 to 4f7dfd4 Compare April 7, 2022 01:24
Copy link
Contributor Author

@aayushshah15 aayushshah15 left a comment

Choose a reason for hiding this comment

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

What about SCATTER?

It seems like scatter hitting those errors should be okay. Scatter doesn't return any rebalancing related errors up to its client, so that's not a concern:

requeue, err = rq.processOneChange(
ctx, r, canTransferLease, true /* scatter */, false, /* dryRun */
)
if err != nil {
// TODO(tbg): can this use IsRetriableReplicationError?
if isSnapshotError(err) {
continue
}
break
}

Moreover, if scatter does encounter a range that the replicateQueue is already in the process of moving, it seems behaviorally fine for scatter to not also move it.

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

@aayushshah15 aayushshah15 force-pushed the 20220404_avoidLearnerRemovalRacesUsingLogTruncationConstraints branch 2 times, most recently from 9f2dab1 to ca1d072 Compare April 7, 2022 17:12
Copy link
Contributor Author

@aayushshah15 aayushshah15 left a comment

Choose a reason for hiding this comment

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

@nvanbenschoten I've swapped these commits around like you mentioned.

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

Copy link
Member

@nvanbenschoten nvanbenschoten left a comment

Choose a reason for hiding this comment

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

Reviewed 1 of 5 files at r6, 7 of 7 files at r7, 7 of 7 files at r8, all commit messages.
Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @aayushshah15)


pkg/kv/kvserver/replica_command.go, line 1282 at r8 (raw file):

}

// ErrCannotRemoveLearnerWhileSnapshotInFlight is returned when we cannot remove

Does this need to be marked as a form of IsExpectedRelocateError and of IsRetriableReplicationChangeError?


pkg/kv/kvserver/replica_command.go, line 1295 at r8 (raw file):

	desc, err = r.maybeLeaveAtomicChangeReplicas(ctx, desc)
	if err != nil {
		return nil, learnersRemoved, err

If we know learnersRemoved is 0 here, it's clearer to return 0, not the named return arg. For example, this would be even harder to read as return rangeDesc, learnersRemoved, err. From that line, it's not possible to determine what is being returned. A reader now needs to understand the entire control flow of this function above this point to determine which cases learnersRemoved is non-zero.

I think we should make this change wherever we can in this function.


pkg/kv/kvserver/replica_command.go, line 1302 at r8 (raw file):

	// out.
	if r.hasOutstandingLearnerSnapshotInFlight() {
		return desc, learnersRemoved, ErrCannotRemoveLearnerWhileSnapshotInFlight

How do we expect each of the possible callers into this function to react to this error? Will all of them react gracefully? It would be worthwhile to enumerate them all and ensure that their retry behavior (or lack thereof) in the presence of this error is desirable.


pkg/kv/kvserver/replica_raft.go, line 1557 at r7 (raw file):

	}

	item.completed = true

Do we need this status anymore? Can we delete from r.mu.snapshotLogTruncationConstraints immediately?


pkg/kv/kvserver/replica_raft.go, line 1602 at r8 (raw file):

	learners := r.Desc().Replicas().LearnerDescriptors()
	for _, repl := range learners {
		if yes := r.hasOutstandingSnapshotInFlightToStore(repl.StoreID); yes {

nit: along the lines of my comment above, the following is slightly easier to read because it preserves less state in local variables and allows readers to read backward.

if r.hasOutstandingSnapshotInFlightToStore(repl.StoreID) {
    return true
}

pkg/kv/kvserver/replicate_queue.go, line 670 at r7 (raw file):

		return rq.removeDead(ctx, repl, deadNonVoterReplicas, nonVoterTarget, dryRun)

	case AllocatorRemoveLearner:

Was this just dead code before?


pkg/kv/kvserver/replicate_queue.go, line 663 at r8 (raw file):

		return rq.removeDead(ctx, repl, deadNonVoterReplicas, nonVoterTarget, dryRun)

	case AllocatorRemoveLearner:

Should we combine this with AllocatorFinalizeAtomicReplicationChange to avoid redundancy?

case AllocatorRemoveLearner, AllocatorFinalizeAtomicReplicationChange:

pkg/kv/kvserver/replicate_queue.go, line 690 at r8 (raw file):

		}
		rq.metrics.RemoveLearnerReplicaCount.Inc(learnersRemoved)
		return true, err

return true, nil

@aayushshah15 aayushshah15 force-pushed the 20220404_avoidLearnerRemovalRacesUsingLogTruncationConstraints branch 4 times, most recently from 0e4883f to 15ce64b Compare April 9, 2022 18:57
Copy link
Contributor Author

@aayushshah15 aayushshah15 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 @aayushshah15, @nvanbenschoten, and @tbg)


pkg/kv/kvserver/replica_command.go, line 1282 at r8 (raw file):

Previously, nvanbenschoten (Nathan VanBenschoten) wrote…

Does this need to be marked as a form of IsExpectedRelocateError and of IsRetriableReplicationChangeError?

There don't seem to be any tests that expect to run into a learner replica in the process of receiving its first snapshot, so I don't think it needs to be marked IsExpectedRelocateError or marked retriable.


pkg/kv/kvserver/replica_command.go, line 1295 at r8 (raw file):

Previously, nvanbenschoten (Nathan VanBenschoten) wrote…

If we know learnersRemoved is 0 here, it's clearer to return 0, not the named return arg. For example, this would be even harder to read as return rangeDesc, learnersRemoved, err. From that line, it's not possible to determine what is being returned. A reader now needs to understand the entire control flow of this function above this point to determine which cases learnersRemoved is non-zero.

I think we should make this change wherever we can in this function.

Done, much cleaner.


pkg/kv/kvserver/replica_command.go, line 1302 at r8 (raw file):

Previously, nvanbenschoten (Nathan VanBenschoten) wrote…

How do we expect each of the possible callers into this function to react to this error? Will all of them react gracefully? It would be worthwhile to enumerate them all and ensure that their retry behavior (or lack thereof) in the presence of this error is desirable.

The callers that could potentially run into this error are:

  1. mergeQueue
  2. StoreRebalancer
  3. replicateQueue

In all of these cases, they will fail-fast and retry after their respective intervals (which is 10 mins for the queues, and ~1 min for the StoreRebalancer). It does seem desirable to let these queues move on to other ranges when they encounter this error, as opposed to waiting for a while for the learner to finish receiving its snapshot. Retrying or waiting on this error would make issues like #79249 more likely.


pkg/kv/kvserver/replica_raft.go, line 1557 at r7 (raw file):

Previously, nvanbenschoten (Nathan VanBenschoten) wrote…

Do we need this status anymore? Can we delete from r.mu.snapshotLogTruncationConstraints immediately?

I've cleaned it up now.


pkg/kv/kvserver/replica_raft.go, line 1602 at r8 (raw file):

Previously, nvanbenschoten (Nathan VanBenschoten) wrote…

nit: along the lines of my comment above, the following is slightly easier to read because it preserves less state in local variables and allows readers to read backward.

if r.hasOutstandingSnapshotInFlightToStore(repl.StoreID) {
    return true
}

You're right, done.


pkg/kv/kvserver/replicate_queue.go, line 670 at r7 (raw file):

Previously, nvanbenschoten (Nathan VanBenschoten) wrote…

Was this just dead code before?

Yep.


pkg/kv/kvserver/replicate_queue.go, line 663 at r8 (raw file):

Previously, nvanbenschoten (Nathan VanBenschoten) wrote…

Should we combine this with AllocatorFinalizeAtomicReplicationChange to avoid redundancy?

case AllocatorRemoveLearner, AllocatorFinalizeAtomicReplicationChange:

Done.


pkg/kv/kvserver/replicate_queue.go, line 690 at r8 (raw file):

Previously, nvanbenschoten (Nathan VanBenschoten) wrote…

return true, nil

Done.

Copy link
Member

@nvanbenschoten nvanbenschoten left a comment

Choose a reason for hiding this comment

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

:lgtm:

Reviewed 9 of 9 files at r9, 7 of 7 files at r10, all commit messages.
Reviewable status: :shipit: complete! 1 of 0 LGTMs obtained (waiting on @aayushshah15)


pkg/kv/kvserver/replica_command.go, line 1282 at r8 (raw file):

Previously, aayushshah15 (Aayush Shah) wrote…

There don't seem to be any tests that expect to run into a learner replica in the process of receiving its first snapshot, so I don't think it needs to be marked IsExpectedRelocateError or marked retriable.

I'm surprised that TestKVNemesisMultiNode doesn't run into this. Do you mind stressing that test for a bit?


pkg/kv/kvserver/replica_command.go, line 1302 at r8 (raw file):

The callers that could potentially run into this error are:

And some of the SQL-level admin operations that result in AdminChangeReplicas and AdminRelocateRange requests, right?

In all of these cases ...

Thanks for explaining. This all checks out. Do you mind adding a comment along these lines here?

@aayushshah15 aayushshah15 force-pushed the 20220404_avoidLearnerRemovalRacesUsingLogTruncationConstraints branch 2 times, most recently from e4d222e to 1c56654 Compare April 21, 2022 18:20
Copy link
Contributor Author

@aayushshah15 aayushshah15 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 (and 1 stale) (waiting on @aayushshah15, @nvanbenschoten, and @tbg)


pkg/kv/kvserver/replica_command.go, line 1313 at r12 (raw file):

Previously, nvanbenschoten (Nathan VanBenschoten) wrote…

Can we mark this statically? (i.e. lift the errors.Mark to the global)

Done.


pkg/sql/relocate.go, line 149 at r12 (raw file):

Previously, aayushshah15 (Aayush Shah) wrote…

Yeah, I'm not sure what the right thing to do here is. Backoff seems undesirable compared to just forcing a removal of the learner replica in progress, since we'll likely be backing off to then undo this work anyway. I thought about perhaps adding a parameter to "force" AdminRelocateRange and AdminChangeReplicas to continue with removing learners if they're issued via SQL, but that has mixed version considerations. Do you think that's worth the complexity (as part of this patch specifically)?

Given all that, I do find myself leaning towards the current approach that ignores these ranges and moves on to other ones, maybe with an eye to fix it in a future patch that force-removes these learners. What do you think?

OK, I've actually changed this back to returning an error / failing the SQL statement if we run into this error. That seems better, at least in the short term, than silently ignoring ranges because of the concern you outlined.


pkg/sql/relocate.go, line 149 at r12 (raw file):

Previously, nvanbenschoten (Nathan VanBenschoten) wrote…

nit: missing space before call

Fixed.

@aayushshah15 aayushshah15 force-pushed the 20220404_avoidLearnerRemovalRacesUsingLogTruncationConstraints branch from 1c56654 to 62d73a9 Compare April 21, 2022 19:59
Copy link
Member

@nvanbenschoten nvanbenschoten left a comment

Choose a reason for hiding this comment

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

:lgtm:

Reviewed 1 of 12 files at r13, 11 of 12 files at r14, 1 of 1 files at r15, all commit messages.
Reviewable status: :shipit: complete! 1 of 0 LGTMs obtained (waiting on @aayushshah15)


pkg/sql/relocate.go line 149 at r12 (raw file):

Previously, aayushshah15 (Aayush Shah) wrote…

OK, I've actually changed this back to returning an error / failing the SQL statement if we run into this error. That seems better, at least in the short term, than silently ignoring ranges because of the concern you outlined.

I agree. That seems like a better short-term solution.

This commit removes the grace period (and associated logic) that the sender of
a snapshot used to adhere to before GC-ing that snapshot's log truncation
constraint. This code was only meant to guard against races where a pre-emptive
snapshot was received by a recipient but not applied (which would happen when
the associated replication change was completed). This race is no longer
possible today.

Release note: None
This commit adds a safeguard inside
`Replica.maybeLeaveAtomicChangeReplicasAndRemoveLearners()` to avoid removing
learner replicas _when we know_ that that learner replica is in the process of
receiving its initial snapshot (as indicated by an in-memory lock on log
truncations that we place while the snapshot is in-flight).

This change should considerably reduce the instances where `AdminRelocateRange`
calls are interrupted by the mergeQueue or the replicateQueue (and vice versa).

Fixes cockroachdb#57129
Relates to cockroachdb#79118

Release note: none
@aayushshah15 aayushshah15 force-pushed the 20220404_avoidLearnerRemovalRacesUsingLogTruncationConstraints branch from 62d73a9 to 466f42c Compare April 26, 2022 19:55
@aayushshah15 aayushshah15 added branch-release-22.1 Used to mark GA and release blockers, technical advisories, and bugs for 22.1 and removed GA-blocker branch-release-22.1 Used to mark GA and release blockers, technical advisories, and bugs for 22.1 labels Apr 26, 2022
@aayushshah15
Copy link
Contributor Author

TFTR!

bors r+

@aayushshah15
Copy link
Contributor Author

blathers backport 22.1

@blathers-crl
Copy link

blathers-crl bot commented Apr 26, 2022

Encountered an error creating backports. Some common things that can go wrong:

  1. The backport branch might have already existed.
  2. There was a merge conflict.
  3. The backport branch contained merge commits.

You might need to create your backport manually using the backport tool.


error creating merge commit from 466f42c to blathers/backport-release-22.1-79379: POST https://api.github.com/repos/cockroachdb/cockroach/merges: 409 Merge conflict []

you may need to manually resolve merge conflicts with the backport tool.

Backport to branch 22.1 failed. See errors above.


🦉 Hoot! I am a Blathers, a bot for CockroachDB. My owner is otan.

@craig
Copy link
Contributor

craig bot commented Apr 26, 2022

Build failed (retrying...):

@craig
Copy link
Contributor

craig bot commented Apr 26, 2022

Build failed (retrying...):

@craig
Copy link
Contributor

craig bot commented Apr 27, 2022

Build succeeded:

@craig craig bot merged commit d6240ac into cockroachdb:master Apr 27, 2022
aayushshah15 added a commit to aayushshah15/cockroach that referenced this pull request May 3, 2022
This commit fixes a set of failures noticed in cockroachdb#79887 where a bunch of
`EXPERIMENTAL_RELOCATE` calls failed because they tried removing learner
replicas that are in the process of receiving their initial snapshot.

Removing learners in this state is disallowed as of cockroachdb#79379. This commit fixes a
subset of those failures by making the `kv50/rangelookups` roachtest ignore
these errors.

Relates to cockroachdb#79887

Release note: None
craig bot pushed a commit that referenced this pull request May 4, 2022
80918: kvserver: ignore errors encountered while removing live learners r=aayushshah15 a=aayushshah15

This commit fixes a set of failures noticed in #79887 where a bunch of
`EXPERIMENTAL_RELOCATE` calls failed because they tried removing learner
replicas that are in the process of receiving their initial snapshot. The test simply
randomly issues these `RELOCATE` calls so this behavior is intentional.

Removing learners in this state is disallowed as of #79379. This commit fixes a
subset of those failures by making the `kv50/rangelookups` roachtest ignore
these errors.

Relates to #79887

Release note: None

Co-authored-by: Aayush Shah <[email protected]>
blathers-crl bot pushed a commit that referenced this pull request May 4, 2022
This commit fixes a set of failures noticed in #79887 where a bunch of
`EXPERIMENTAL_RELOCATE` calls failed because they tried removing learner
replicas that are in the process of receiving their initial snapshot.

Removing learners in this state is disallowed as of #79379. This commit fixes a
subset of those failures by making the `kv50/rangelookups` roachtest ignore
these errors.

Relates to #79887

Release note: None
aayushshah15 added a commit to aayushshah15/cockroach that referenced this pull request Jun 8, 2022
This commit fixes a set of failures noticed in cockroachdb#79887 where a bunch of
`EXPERIMENTAL_RELOCATE` calls failed because they tried removing learner
replicas that are in the process of receiving their initial snapshot.

Removing learners in this state is disallowed as of cockroachdb#79379. This commit fixes a
subset of those failures by making the `kv50/rangelookups` roachtest ignore
these errors.

Relates to cockroachdb#79887

Release note: None
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.

kvserver: merge queue racing against replicate queue
4 participants