server: support adding stores to existing nodes#55350
server: support adding stores to existing nodes#55350craig[bot] merged 1 commit intocockroachdb:masterfrom
Conversation
adc3d23 to
98bc5bd
Compare
tbg
left a comment
There was a problem hiding this comment.
Looks good! Does it hang as we expect?
Reviewed 1 of 1 files at r1, 2 of 2 files at r2.
Reviewable status:complete! 0 of 0 LGTMs obtained (waiting on @irfansharif and @TheSamHuang)
pkg/server/node_test.go, line 171 at r1 (raw file):
// TestAddNewStoresToExistingNodes starts a cluster with three nodes, // shuts down all nodes and adds a store to each node, and ensures // nodes start back up successfully
nit: .
pkg/server/node_test.go, line 204 at r1 (raw file):
tc.Stopper().Stop(ctx) // Add an additional store to each node
.
pkg/server/node_test.go, line 235 at r1 (raw file):
} // Start all nodes with additional stores
.
pkg/server/node_test.go, line 241 at r1 (raw file):
// Ensure all nodes have 2 stores available testutils.SucceedsSoon(t, func() error { for _, idx := range []int{0, 1, 2} {
Might as well for _, srv := range tc.Servers { ... }
Yes, it does! |
98bc5bd to
225e074
Compare
|
After trying to debug why the new test was failing (in hopes of solving the stores issue), I found out that the testing libraries do not allow for starting up a cluster or even an individual server after they have been stopped. (After calling |
|
I think something got lost in translation when we talked about this test. I did mention that you can't restart an existing server, but that's why we're using the disks - we will start a new server from the existing stores. That should work and if it doesn't, I'm happy to help fix it. I don't think we should throw the towel yet and write a roachtest here. |
|
Honestly looking at your code I'm a bit confused by your comment. You're already doing the right thing in that test, which intentionally doesn't work (since you haven't fixed the store creation issue yet). I was trying out this branch and there are some issues to chew through locally. For one, for some reason once you give disks initial cluster start takes 10+ seconds - I'm sure I looked at this before and couldn't figure it out, so now is the time to double down - and you need the I'll put together a diff for you when I've gotten somewhere. |
We need to bootstrap additional stores asynchronously to avoid a deadlock when restarting the members of the quorum for the range that has the store ID allocator together. Fixes cockroachdb#55350. Release note: None
|
@TheSamHuang I updated your branch with commits that "fix" both the issues with the test and make the change to bootstrap additional stores asynchronously. Interestingly, this is still failing, but with an unrelated storage issue: This occurs in the test when we restart the cluster from the store directories we made. Uh oh. @petermattis I hope this is something silly related to the test setup. Can you have the storage team take a look? There's also a second failure mode:
This is because cockroach/pkg/server/debug/server.go Lines 221 to 230 in 7bed111 |
|
Actually, the pebble issue doesn't show up any more, despite a few minutes of |
|
Pulled the branch down locally, I still see the same pebble failure. |
|
Interesting. Every time? I got it once and then never again. |
111a8d9 to
19d71a8
Compare
We need to bootstrap additional stores asynchronously to avoid a deadlock when restarting the members of the quorum for the range that has the store ID allocator together. Fixes cockroachdb#55350. Release note: None
|
Nope, not every time. I get the other debug server error too, and manually ignoring that error it's hard to repro, but I haven't really tried in earnest. |
be839ed to
3ca7f55
Compare
irfansharif
left a comment
There was a problem hiding this comment.
Thanks for working on this thus far Sam! I think we're super close, I've included in comments below the few changes we'll need to make here before wrapping it all up. I'm excited to have crdb (once again, ha) attain the ability to support nodes with multiple stores.
We'll also want to clean up our commit history here. We'll want to separate out @tbg's changes to the testing infrastructure as the initial commits, and then introduce the test (nicely done by the way) + the async bootstrap process as a single final commit. Lastly, we'll want to change the commit title to be "server: support adding stores to existing nodes". Our convention for the prefix is to have it match the package name in question (for us it's pkg/server).
2fd7fe7 to
b7f4afd
Compare
irfansharif
left a comment
There was a problem hiding this comment.
Nice, almost there. We just have our git commit history to unwrangle and we should be good to go. Happy to help you with it offline if needed!
769c768 to
6e92d8f
Compare
3727f45 to
d88ab46
Compare
4f7a31f to
e6256f4
Compare
|
In addition, moved |
There was a problem hiding this comment.
LGTM! Let's get a final set of eyes from @tbg before moving on from here. Thanks for all the work Sam.
tbg
left a comment
There was a problem hiding this comment.
+1 on Irfan's comments about comments. This bootstrap problem is not well known outside of the reviewers of this PR. It should be clear as day to anyone visiting this area in the future.
Reviewed 4 of 6 files at r4, 2 of 5 files at r11, 3 of 3 files at r16, 3 of 3 files at r17.
Reviewable status:complete! 0 of 0 LGTMs obtained (waiting on @irfansharif, @tbg, and @TheSamHuang)
pkg/server/node.go, line 351 at r14 (raw file):
Previously, irfansharif (irfan sharif) wrote…
Let's use named return values here: https://tour.golang.org/basics/7. Let's also add some commentary above explaining what this new return value is intended for.
By convention, the error always goes last. Also, it doesn't need to be named:
(waitForBootstrapNewStores func(), _ error)
I'm also confused why we stopped returning the channel. A blocking function is IMO a worse choice than a channel. (You've probably discussed this with Irfan before). The upside of a channel is that it provides better cancel-ability for the waiter.
I think there's also a third option that seems appealing, which is that instead of returning something from (*Node).start, we add a new method on *Node that does the waiting:
// bootstrapNewStoresCh returns a channel that is closed once all
// additional empty stores have been assigned a StoreIdent.
func (n *Node) bootstrapNewStoresCh() <-chan struct{} {
return n.bootstrapNewStoresCh
}pkg/server/node.go, line 476 at r17 (raw file):
bootstrapNewStoresDone := make(chan struct{}) waitForBootstrapNewStores = func() { <-bootstrapNewStoresDone } _ = n.stopper.RunAsyncTask(ctx, "bootstrap-stores", func(ctx context.Context) {
if RunAsyncTask returns an error, we need (or should) close the channel as well.
pkg/server/server.go, line 1543 at r11 (raw file):
Previously, irfansharif (irfan sharif) wrote…
Instead of constructing the channel at the caller, we can do it within
node.startitself. What we can do instead is return a function likewaitForBootstrapNewStoresand call that instead, and within that function body (unbeknownst to the caller) it would wait for the right channel.As for why we should structure it this way: it lets us contain the scope of this signal channel to just the bits we care about. This server.PreStart method is already too long (as you're painfully aware), so the fewer things in scope, the clearer things are.
Ah, this is where the suggestion to return a closure comes from. Curious why you prefer this over a <-chan struct{} which imo is the most functional representation of "a thing that signals you".
|
Thanks Toby and Irfan for the comments!
I made one change to your suggestion - instead of defining a function that simply returns the
Instead of closing the channel, I instead opted to |
/shrug no good reason other than generally wanting to avoid channels in APIs.
I'm fine either way, I didn't have strong opinions on what we had before either.
I believe Tobi was talking about the error returned here, which we're currently ignoring: _ = n.stopper.RunAsyncTaskThis isn't from bootstrapping the stores themselves, it's from not being able to start off the async task in the first place. We'll need to handle it appropriately. |
Yes, I agree. I think it will be more clear what I meant when I push the amended commit. I meant that if starting the async task failed, the stores wouldn't get bootstrapped so we should fail. |
fb9234f to
1a4184a
Compare
After some more thought, decided to go with closing the channel and just returning the error to the caller instead! |
irfansharif
left a comment
There was a problem hiding this comment.
LGTM mod comment adjustments below.
We need to bootstrap additional stores asynchronously. Consider the range that houses the store ID allocator. When restarting the set of nodes that holds a quorum of these replicas, when restarting them with additional stores, those additional stores will require store IDs to get fully bootstrapped. But if we're gating node start (specifically opening up the RPC floodgates) on having all stores fully bootstrapped, we'll simply hang when trying to allocate store IDs. See TestAddNewStoresToExistingNodes and cockroachdb#39415 for more details. Instead we opt to bootstrap additional stores asynchronously, and rely on the blocking function to signal to the caller that all stores have been fully bootstrapped. Release note: None Co-authored-by: Sam Huang <samh@cockroachlabs.com> Co-authored-by: Tobias Grieger <tobias.b.grieger@gmail.com>
1a4184a to
991a01d
Compare
|
bors r=irfansharif |
|
Build succeeded: |
55957: builtins: fix incorrect sqrdiff evaluation due to reuse of the results from previous iterations when used as a window function r=yuzefovich a=mneverov builtins: fix incorrect sqrdiff evaluation due to reuse of the results from previous iterations when used as a window function fixes #55944 Release note (bug fix): CockroachDB previously could incorrectly evaluate sqrdiff function when used as a window function in some cases, and now it is fixed. 55975: roachprod: Configure Azure disks. r=miretskiy a=miretskiy Add ability to specify the size of the attached disks for Azure. Use read only caching for Azure premium disks. Release Notes: None 55979: server: remove extra inline comment r=knz a=TheSamHuang Previously, there was an extra comment left over from an earlier iteration of #55350. This comment was no longer accurate and was removed. Release note: None. Co-authored-by: Max Neverov <neverov.max@gmail.com> Co-authored-by: Yevgeniy Miretskiy <yevgeniy@cockroachlabs.com> Co-authored-by: Sam Huang <samh@cockroachlabs.com>
We need to bootstrap additional stores asynchronously. Consider the range that
houses the store ID allocator. When restarting the set of nodes that holds a
quorum of these replicas, when restarting them with additional stores, those
additional stores will require store IDs to get fully bootstrapped. But if
we're gating node start (specifically opening up the RPC floodgates) on having
all stores fully bootstrapped, we'll simply hang when trying to allocate store
IDs. See TestAddNewStoresToExistingNodes and #39415 for more details.
Instead we opt to bootstrap additional stores asynchronously, and rely on the
blocking function to signal to the caller that all stores have been fully
bootstrapped.