Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
105052: kvserver: don't campaign on Raft message without leader r=erikgrinaker a=erikgrinaker

In cf56549, we began waking up the leader if we received a message from a non-leader replica. This implicitly also caused us to campaign if we didn't have a leader.

However, we did this regardless of whether we actually have a leader to wake up. This is likely to lead to election ties during cold starts, where all replicas will campaign simultaneously.

This patch no longer campaigns if we receive a message without a leader. The election tie risk still remains when replicas unquiesce and discover a dead leader -- this will be addressed separately.

Epic: none
Release note: None

105054: publish-artifacts: add a suffix for ARM `workload` binary r=rail a=rickystewart

Epic: none
Release note: None

Co-authored-by: Erik Grinaker <[email protected]>
Co-authored-by: Ricky Stewart <[email protected]>
  • Loading branch information
3 people committed Jun 16, 2023
3 parents b6d0385 + f26fadb + f190457 commit 6d845fc
Show file tree
Hide file tree
Showing 4 changed files with 15 additions and 7 deletions.
3 changes: 3 additions & 0 deletions pkg/cmd/publish-artifacts/main.go
Original file line number Diff line number Diff line change
Expand Up @@ -146,6 +146,9 @@ func buildAndPublishWorkload(providers []release.ObjectPutGetter, o opts, execFn
log.Fatal(err)
}
o.AbsolutePath = filepath.Join(o.PkgDir, "bin", "workload")
if o.Platform == release.PlatformLinuxArm {
o.AbsolutePath += release.SuffixFromPlatform(o.Platform)
}
for _, provider := range providers {
release.PutNonRelease(
provider,
Expand Down
4 changes: 2 additions & 2 deletions pkg/cmd/publish-artifacts/main_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -253,8 +253,8 @@ func TestPublish(t *testing.T) {
"'--workspace_status_command=./build/bazelutil/stamp.sh aarch64-unknown-linux-gnu official-binary' " +
"-c opt --config=ci --config=force_build_cdeps --config=crosslinuxarmbase",
"gs://edge-binaries-bucket/cockroach/lib/libgeos_c.linux-gnu-arm64.so.LATEST/no-cache REDIRECT /cockroach/lib/libgeos_c.linux-gnu-arm64.1234567890abcdef.so",
"gs://edge-binaries-bucket/cockroach/workload.1234567890abcdef CONTENTS env=[] args=bazel build //pkg/cmd/workload -c opt --config=crosslinuxarmbase --config=ci",
"gs://edge-binaries-bucket/cockroach/workload.LATEST/no-cache REDIRECT /cockroach/workload.1234567890abcdef",
"gs://edge-binaries-bucket/cockroach/workload.linux-gnu-arm64.1234567890abcdef CONTENTS env=[] args=bazel build //pkg/cmd/workload -c opt --config=crosslinuxarmbase --config=ci",
"gs://edge-binaries-bucket/cockroach/workload.linux-gnu-arm64.LATEST/no-cache REDIRECT /cockroach/workload.linux-gnu-arm64.1234567890abcdef",
"gs://edge-binaries-bucket/cockroach/cockroach.darwin-amd64.1234567890abcdef CONTENTS env=[] args=bazel build " +
"//pkg/cmd/cockroach //pkg/cmd/cockroach-sql //c-deps:libgeos " +
"'--workspace_status_command=./build/bazelutil/stamp.sh x86_64-apple-darwin19 official-binary' " +
Expand Down
11 changes: 8 additions & 3 deletions pkg/kv/kvserver/replica_raft.go
Original file line number Diff line number Diff line change
Expand Up @@ -647,8 +647,8 @@ func (r *Replica) stepRaftGroup(req *kvserverpb.RaftMessageRequest) error {
// include MsgVotes), so don't campaign if we wake up our raft
// group.
return r.withRaftGroup(false, func(raftGroup *raft.RawNode) (bool, error) {
// If we're not the leader, and we receive a message from a non-leader
// replica while quiesced, we wake up the leader too. This prevents spurious
// If we're a follower, and we receive a message from a non-leader replica
// while quiesced, we wake up the leader too. This prevents spurious
// elections.
//
// This typically happens in the case of a partial network partition where
Expand All @@ -665,7 +665,12 @@ func (r *Replica) stepRaftGroup(req *kvserverpb.RaftMessageRequest) error {
// Note that such partial partitions will typically result in persistent
// mass unquiescence due to the continuous prevotes.
if r.mu.quiescent {
if !r.isRaftLeaderRLocked() && req.FromReplica.ReplicaID != r.mu.leaderID {
st := r.raftBasicStatusRLocked()
hasLeader := st.RaftState == raft.StateFollower && st.Lead != 0
fromLeader := uint64(req.FromReplica.ReplicaID) == st.Lead
if hasLeader && !fromLeader {
// TODO(erikgrinaker): This is likely to result in election ties, find
// some way to avoid that.
r.maybeUnquiesceAndWakeLeaderLocked()
} else {
r.maybeUnquiesceWithOptionsLocked(false /* campaignOnWake */)
Expand Down
4 changes: 2 additions & 2 deletions pkg/release/build.go
Original file line number Diff line number Diff line change
Expand Up @@ -131,7 +131,7 @@ func SharedLibraryExtensionFromPlatform(platform Platform) string {
}

// MakeWorkload makes the bin/workload binary. It is only ever built in the
// crosslinux configuration.
// crosslinux and crosslinuxarm configurations.
func MakeWorkload(platform Platform, opts BuildOptions, pkgDir string) error {
if opts.Release {
return errors.Newf("cannot build workload in Release mode")
Expand All @@ -151,7 +151,7 @@ func MakeWorkload(platform Platform, opts BuildOptions, pkgDir string) error {
if err != nil {
return err
}
return stageBinary("//pkg/cmd/workload", platform, bazelBin, filepath.Join(pkgDir, "bin"), false)
return stageBinary("//pkg/cmd/workload", platform, bazelBin, filepath.Join(pkgDir, "bin"), platform == PlatformLinuxArm)
}

// MakeRelease makes the release binary and associated files.
Expand Down

0 comments on commit 6d845fc

Please sign in to comment.