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

server: support adding stores to existing nodes #55350

Merged

Conversation

TheSamHuang
Copy link
Contributor

@TheSamHuang TheSamHuang commented Oct 8, 2020

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.

@cockroach-teamcity
Copy link
Member

This change is Reviewable

@TheSamHuang TheSamHuang requested a review from tbg October 8, 2020 20:17
@TheSamHuang TheSamHuang force-pushed the support-adding-stores-to-existing-nodes branch from adc3d23 to 98bc5bd Compare October 8, 2020 20: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.

Looks good! Does it hang as we expect?

Reviewed 1 of 1 files at r1, 2 of 2 files at r2.
Reviewable status: :shipit: 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 { ... }

@TheSamHuang
Copy link
Contributor Author

Looks good! Does it hang as we expect?

Yes, it does!

@TheSamHuang TheSamHuang linked an issue Oct 12, 2020 that may be closed by this pull request
@TheSamHuang TheSamHuang force-pushed the support-adding-stores-to-existing-nodes branch from 98bc5bd to 225e074 Compare October 13, 2020 15:35
@TheSamHuang
Copy link
Contributor Author

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 testCluster.Stopper().Stop() or server.Stopper().Stop(), testCluster.StartTestCluster() and server.Start() will always fail). Since this unit test will always fail, I will opt for writing a roachtest instead.

@tbg
Copy link
Member

tbg commented Oct 14, 2020

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.

@tbg
Copy link
Member

tbg commented Oct 14, 2020

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 ParallelStart: true setting when restarting the cluster.

I'll put together a diff for you when I've gotten somewhere.

tbg added a commit to TheSamHuang/cockroach that referenced this pull request Oct 14, 2020
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
@tbg
Copy link
Member

tbg commented Oct 14, 2020

@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:

anic: open /tmp/TestAddNewStoresToExistingNodes119630251/000304.log: no such file or directory [recovered]
	panic: open /tmp/TestAddNewStoresToExistingNodes119630251/000304.log: no such file or directory

goroutine 5848 [running]:
github.com/cockroachdb/cockroach/pkg/util/stop.(*Stopper).Recover(0xc00316a000, 0x5933c40, 0xc0031d15f0)
	/home/tobias/go/src/github.com/cockroachdb/cockroach/pkg/util/stop/stopper.go:207 +0x11f
panic(0x4442840, 0xc001cfcff0)
	/usr/local/go/src/runtime/panic.go:679 +0x1b2
github.com/cockroachdb/pebble.(*DB).makeRoomForWrite(0xc000a7d800, 0xc001252140, 0x14426, 0x16000)
	/home/tobias/go/src/github.com/cockroachdb/cockroach/vendor/github.com/cockroachdb/pebble/db.go:1393 +0x119d
github.com/cockroachdb/pebble.(*DB).commitWrite(0xc000a7d800, 0xc001252140, 0x0, 0x0, 0xc00384e3a8, 0xc001252140, 0xc00384e3a8)
	/home/tobias/go/src/github.com/cockroachdb/cockroach/vendor/github.com/cockroachdb/pebble/db.go:628 +0xbb
github.com/cockroachdb/pebble.(*commitPipeline).prepare(0xc00384e000, 0xc001252140, 0x0, 0x5933c40, 0xc003f9bc50, 0x5a8e640)
	/home/tobias/go/src/github.com/cockroachdb/cockroach/vendor/github.com/cockroachdb/pebble/commit.go:377 +0x162
github.com/cockroachdb/pebble.(*commitPipeline).Commit(0xc00384e000, 0xc001252140, 0x0, 0x0, 0x20)
	/home/tobias/go/src/github.com/cockroachdb/cockroach/vendor/github.com/cockroachdb/pebble/commit.go:253 +0x73
github.com/cockroachdb/pebble.(*DB).Apply(0xc000a7d800, 0xc001252140, 0x80adb00, 0xc002d42a00, 0xc00026b800)
	/home/tobias/go/src/github.com/cockroachdb/cockroach/vendor/github.com/cockroachdb/pebble/db.go:556 +0x10a
github.com/cockroachdb/pebble.(*Batch).Commit(...)
	/home/tobias/go/src/github.com/cockroachdb/cockroach/vendor/github.com/cockroachdb/pebble/batch.go:727
github.com/cockroachdb/cockroach/pkg/storage.(*pebbleBatch).Commit(0xc0018bec00, 0x5933c00, 0xc003f9bc50, 0x0)
	/home/tobias/go/src/github.com/cockroachdb/cockroach/pkg/storage/pebble_batch.go:341 +0x5e
github.com/cockroachdb/cockroach/pkg/kv/kvserver.(*replicaAppBatch).ApplyToStateMachine(0xc00026b8c0, 0x5933c40, 0xc003f9bc50, 0x7f4d574be180, 0xc00026bad0)
	/home/tobias/go/src/github.com/cockroachdb/cockroach/pkg/kv/kvserver/replica_application_state_machine.go:868 +0xce
github.com/cockroachdb/cockroach/pkg/kv/kvserver/apply.(*Task).applyOneBatch(0xc00205d798, 0x5933c40, 0xc003f9bc50, 0x5976080, 0xc00026ba70, 0x0, 0x0)
	/home/tobias/go/src/github.com/cockroachdb/cockroach/pkg/kv/kvserver/apply/task.go:286 +0x1d7
github.com/cockroachdb/cockroach/pkg/kv/kvserver/apply.(*Task).ApplyCommittedEntries(0xc00205d798, 0x5933c40, 0xc003f9bc50, 0x2, 0x49a7643)
	/home/tobias/go/src/github.com/cockroachdb/cockroach/pkg/kv/kvserver/apply/task.go:247 +0xb9
github.com/cockroachdb/cockroach/pkg/kv/kvserver.(*Replica).handleRaftReadyRaftMuLocked(0xc00026b800, 0x5933c40, 0xc003f9bc50, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, ...)
	/home/tobias/go/src/github.com/cockroachdb/cockroach/pkg/kv/kvserver/replica_raft.go:765 +0xff9
github.com/cockroachdb/cockroach/pkg/kv/kvserver.(*Replica).handleRaftReady(0xc00026b800, 0x5933c40, 0xc003f9bc50, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, ...)

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:

testcluster.go:260: failed to register engines with debug server: store has not been bootstrapped

This is because

// RegisterEngines setups up debug engine endpoints for the known storage engines.
func (ds *Server) RegisterEngines(specs []base.StoreSpec, engines []storage.Engine) error {
if len(specs) != len(engines) {
// TODO(yevgeniy): Consider adding accessors to storage.Engine to get their path.
return errors.New("number of store specs must match number of engines")
}
for i := 0; i < len(specs); i++ {
if specs[i].InMemory {
// TODO(yevgeniy): Add plumbing to support LSM visualization for in memory engines.
continue
expects all stores bootstrapped by the time we register them, but this is not true any more. I'm adding a hacky commit to avoid this failure mode for now.

@tbg
Copy link
Member

tbg commented Oct 14, 2020

Actually, the pebble issue doesn't show up any more, despite a few minutes of make stress PKG=./pkg/server/ TESTS=TestAddNewStoresToExisting STRESSFLAGS='-p 4'. Unclear what's going on here. Maybe build jank? I was running the test through Goland which isn't hooked up to the makefile.

@irfansharif
Copy link
Contributor

Pulled the branch down locally, I still see the same pebble failure.

$ make test PKG=./pkg/server/ TESTS=TestAddNewStoresToExisting
Running make with -j16
GOPATH set to /Users/irfansharif/Software
mkdir -p lib
make[1]: Nothing to be done for `build_lib_static'.
/Applications/Xcode.app/Contents/Developer/usr/bin/make  all-am
make[2]: Nothing to be done for `all-am'.
rm -f lib/lib{geos,geos_c}.dylib
cp -L /Users/irfansharif/Software/native/x86_64-apple-darwin19.6.0/geos/lib/lib{geos,geos_c}.dylib lib
GOFLAGS= go test   -mod=vendor -tags ' make x86_64_apple_darwin19.6.0' -ldflags '-X github.com/cockroachdb/cockroach/pkg/build.typ=development -extldflags "" -X "github.com/cockroachdb/cockroach/pkg/build.tag=v20.2.0-alpha.3-1742-gca9439be3a-dirty" -X "github.com/cockroachdb/cockroach/pkg/build.rev=ca9439be3ab21d33d508e052e88c85461dac0cb1" -X "github.com/cockroachdb/cockroach/pkg/build.cgoTargetTriple=x86_64-apple-darwin19.6.0"  ' -run "TestAddNewStoresToExisting"  -timeout 30m ./pkg/server/
panic: open /var/folders/3j/6fxg9ckn21g5xm0z_06r7gzr0000gn/T/TestAddNewStoresToExistingNodes027872051/000296.log: no such file or directory [recovered]
        panic: open /var/folders/3j/6fxg9ckn21g5xm0z_06r7gzr0000gn/T/TestAddNewStoresToExistingNodes027872051/000296.log: no such file or directory

goroutine 7322 [running]:
github.com/cockroachdb/cockroach/pkg/util/stop.(*Stopper).Recover(0xc001f02e10, 0x9275840, 0xc00308c810)
        /Users/irfansharif/Software/src/github.com/cockroachdb/cockroach/pkg/util/stop/stopper.go:207 +0x11f
panic(0x7d86c80, 0xc004d776e0)
        /usr/local/Cellar/[email protected]/1.13.15/libexec/src/runtime/panic.go:679 +0x1b2
github.com/cockroachdb/pebble.(*DB).makeRoomForWrite(0xc000f20000, 0xc004ce6c80, 0x75d4, 0x8000)
        /Users/irfansharif/Software/src/github.com/cockroachdb/cockroach/vendor/github.com/cockroachdb/pebble/db.go:1393 +0x119d
github.com/cockroachdb/pebble.(*DB).commitWrite(0xc000f20000, 0xc004ce6c80, 0x0, 0x0, 0xc003128de0, 0xc004ce6c80, 0xc003128de0)
        /Users/irfansharif/Software/src/github.com/cockroachdb/cockroach/vendor/github.com/cockroachdb/pebble/db.go:628 +0xbb
github.com/cockroachdb/pebble.(*commitPipeline).prepare(0xc003128600, 0xc004ce6c80, 0x0, 0x9275840, 0xc0026c2420, 0x93d00a0)
        /Users/irfansharif/Software/src/github.com/cockroachdb/cockroach/vendor/github.com/cockroachdb/pebble/commit.go:377 +0x162
github.com/cockroachdb/pebble.(*commitPipeline).Commit(0xc003128600, 0xc004ce6c80, 0x0, 0x0, 0x20)
        /Users/irfansharif/Software/src/github.com/cockroachdb/cockroach/vendor/github.com/cockroachdb/pebble/commit.go:253 +0x73
github.com/cockroachdb/pebble.(*DB).Apply(0xc000f20000, 0xc004ce6c80, 0xb915f21, 0xc004bec700, 0xc000ce5800)
        /Users/irfansharif/Software/src/github.com/cockroachdb/cockroach/vendor/github.com/cockroachdb/pebble/db.go:556 +0x10a
github.com/cockroachdb/pebble.(*Batch).Commit(...)
        /Users/irfansharif/Software/src/github.com/cockroachdb/cockroach/vendor/github.com/cockroachdb/pebble/batch.go:727
github.com/cockroachdb/cockroach/pkg/storage.(*pebbleBatch).Commit(0xc002416200, 0x9275800, 0xc0026c2420, 0x0)
        /Users/irfansharif/Software/src/github.com/cockroachdb/cockroach/pkg/storage/pebble_batch.go:341 +0x5e

@tbg
Copy link
Member

tbg commented Oct 14, 2020

Interesting. Every time? I got it once and then never again.

@tbg tbg force-pushed the support-adding-stores-to-existing-nodes branch from 111a8d9 to 19d71a8 Compare October 14, 2020 15:23
tbg added a commit to TheSamHuang/cockroach that referenced this pull request Oct 14, 2020
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
@irfansharif
Copy link
Contributor

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.

@TheSamHuang TheSamHuang force-pushed the support-adding-stores-to-existing-nodes branch 3 times, most recently from be839ed to 3ca7f55 Compare October 14, 2020 20:33
Copy link
Contributor

@irfansharif irfansharif 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 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).

pkg/server/server.go Outdated Show resolved Hide resolved
pkg/server/server.go Outdated Show resolved Hide resolved
pkg/server/server.go Outdated Show resolved Hide resolved
pkg/server/node.go Outdated Show resolved Hide resolved
pkg/server/node.go Show resolved Hide resolved
pkg/server/connectivity_test.go Outdated Show resolved Hide resolved
pkg/server/node.go Outdated Show resolved Hide resolved
@TheSamHuang TheSamHuang force-pushed the support-adding-stores-to-existing-nodes branch 2 times, most recently from 2fd7fe7 to b7f4afd Compare October 19, 2020 17:09
Copy link
Contributor

@irfansharif irfansharif left a comment

Choose a reason for hiding this comment

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

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!

pkg/server/connectivity_test.go Outdated Show resolved Hide resolved
pkg/server/node.go Outdated Show resolved Hide resolved
pkg/server/server.go Outdated Show resolved Hide resolved
pkg/server/node.go Outdated Show resolved Hide resolved
pkg/server/node.go Outdated Show resolved Hide resolved
pkg/server/node.go Outdated Show resolved Hide resolved
pkg/server/server.go Outdated Show resolved Hide resolved
@TheSamHuang TheSamHuang force-pushed the support-adding-stores-to-existing-nodes branch 2 times, most recently from 769c768 to 6e92d8f Compare October 19, 2020 17:57
pkg/server/node.go Outdated Show resolved Hide resolved
pkg/server/node.go Outdated Show resolved Hide resolved
pkg/server/node.go Outdated Show resolved Hide resolved
pkg/server/node.go Outdated Show resolved Hide resolved
@irfansharif irfansharif force-pushed the support-adding-stores-to-existing-nodes branch from 3727f45 to d88ab46 Compare October 20, 2020 15:03
@irfansharif irfansharif changed the title Support adding stores to existing nodes server: support adding stores to existing nodes Oct 20, 2020
@TheSamHuang TheSamHuang force-pushed the support-adding-stores-to-existing-nodes branch 2 times, most recently from 4f7a31f to e6256f4 Compare October 20, 2020 19:21
@TheSamHuang
Copy link
Contributor Author

TestAddNewStoresToExistingNodes should not be run under stress so I added skip.UnderStress(t, "too many new stores and nodes for stress") (running it under stress leads to a disk stall after a large number of runs).

In addition, moved TestAddNewStoresToExistingNodes to its own file as none of the existing test files seemed appropriate for it.

@TheSamHuang TheSamHuang marked this pull request as ready for review October 20, 2020 19:26
@TheSamHuang TheSamHuang requested a review from a team as a code owner October 20, 2020 19:26
Copy link
Contributor

@irfansharif irfansharif left a comment

Choose a reason for hiding this comment

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

LGTM! Let's get a final set of eyes from @tbg before moving on from here. Thanks for all the work Sam.

pkg/server/node.go Outdated Show resolved Hide resolved
pkg/server/server.go Outdated Show resolved Hide resolved
@tbg tbg requested a review from irfansharif October 21, 2020 08:18
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.

+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: :shipit: 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.start itself. What we can do instead is return a function like waitForBootstrapNewStores and 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".

@tbg tbg self-requested a review October 21, 2020 08:18
@TheSamHuang
Copy link
Contributor Author

Thanks Toby and Irfan for the comments!

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 }

I made one change to your suggestion - instead of defining a function that simply returns the bootstrapNewStoresCh, I defined a function on node that blocks until the stores are bootstrapped. This way, it's a bit more clear to the caller what is happening: they can simply call s.node.waitForBootstrapNewStores(). What do you think?

if RunAsyncTask returns an error, we need (or should) close the channel as well.

Instead of closing the channel, I instead opted to log.Fatalf() instead because if the async task to bootstrap the new stores failed, we should fail here rather than fail transparently to the caller. If the async task failed, I think the node would fail later on by getting stuck in a loop like we were seeing. It would be harder to debug/troubleshoot in that case, than just logging and crashing when the async task failed.

@irfansharif
Copy link
Contributor

Curious why you prefer this over a <-chan struct{} which imo is the most functional representation of "a thing that signals you".

/shrug no good reason other than generally wanting to avoid channels in APIs.

s.node.waitForBootstrapNewStores()

I'm fine either way, I didn't have strong opinions on what we had before either.

Instead of closing the channel, I instead opted to log.Fatalf() instead because if the async task to bootstrap the new stores failed, we should fail here rather than fail transparently to the caller.

I believe Tobi was talking about the error returned here, which we're currently ignoring:

_ = n.stopper.RunAsyncTask

This 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.

@TheSamHuang
Copy link
Contributor Author

This 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.

@TheSamHuang TheSamHuang force-pushed the support-adding-stores-to-existing-nodes branch 4 times, most recently from fb9234f to 1a4184a Compare October 22, 2020 17:06
@TheSamHuang
Copy link
Contributor Author

This 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.

After some more thought, decided to go with closing the channel and just returning the error to the caller instead!

Copy link
Contributor

@irfansharif irfansharif left a comment

Choose a reason for hiding this comment

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

LGTM mod comment adjustments below.

pkg/server/node.go Outdated Show resolved Hide resolved
pkg/server/node.go Outdated Show resolved Hide resolved
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 <[email protected]>
Co-authored-by: Tobias Grieger <[email protected]>
@TheSamHuang TheSamHuang force-pushed the support-adding-stores-to-existing-nodes branch from 1a4184a to 991a01d Compare October 22, 2020 18:11
@TheSamHuang
Copy link
Contributor Author

bors r=irfansharif

@craig
Copy link
Contributor

craig bot commented Oct 22, 2020

Build succeeded:

@craig craig bot merged commit 4e88183 into cockroachdb:master Oct 22, 2020
craig bot pushed a commit that referenced this pull request Oct 26, 2020
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 <[email protected]>
Co-authored-by: Yevgeniy Miretskiy <[email protected]>
Co-authored-by: Sam Huang <[email protected]>
@TheSamHuang TheSamHuang self-assigned this Nov 25, 2020
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.

storage: support adding Stores to existing Nodes
4 participants