Skip to content

Commit

Permalink
kvserver: prevent etcd-io/raft from dropping conf changes
Browse files Browse the repository at this point in the history
Relies on etcd-io/raft#81.

We don't want it to, since that causes issues due to false positives. We are
taking responsibility for carrying out only valid conf changes, as we always
have.

See also etcd-io/raft#80.

Fixes #105797.
Epic: CRDB-25287
Release note (bug fix): under rare circumstances, a replication change could get
stuck when proposed near lease/leadership changes (and likely under overload),
and the replica circuit breakers could trip. This problem has been addressed.
Note to editors: this time it's really addressed (fingers crossed); a previous
attempt with an identical release note had to be reverted.
  • Loading branch information
tbg committed Jul 17, 2023
1 parent 404188d commit 4585dde
Show file tree
Hide file tree
Showing 2 changed files with 76 additions and 12 deletions.
63 changes: 63 additions & 0 deletions pkg/kv/kvserver/client_raft_test.go
Expand Up @@ -32,6 +32,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/kv/kvpb"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/allocator/storepool"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency/isolation"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvflowcontrol/kvflowdispatch"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb"
Expand Down Expand Up @@ -7030,3 +7031,65 @@ func TestStoreMetricsOnIncomingOutgoingMsg(t *testing.T) {
require.Equal(t, expected, actual)
})
}

// TestInvalidConfChangeRejection is a regression test for [1]. It proposes
// an (intentionally) invalid configuration change and makes sure that raft
// does not drop it.
//
// [1]: https://github.com/cockroachdb/cockroach/issues/105797
func TestInvalidConfChangeRejection(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)

// This is a regression test against a stuck command, so set a timeout to get
// a shot at a graceful failure on regression.
ctx, cancel := context.WithTimeout(context.Background(), testutils.DefaultSucceedsSoonDuration)
defer cancel()

// When our configuration change shows up below raft, we need to apply it as a
// no-op, since the config change is intentionally invalid and assertions
// would fail if we were to try to actually apply it.
injErr := errors.New("injected error")
tc := testcluster.StartTestCluster(t, 1, base.TestClusterArgs{
ReplicationMode: base.ReplicationManual,
ServerArgs: base.TestServerArgs{Knobs: base.TestingKnobs{Store: &kvserver.StoreTestingKnobs{
TestingApplyCalledTwiceFilter: func(args kvserverbase.ApplyFilterArgs) (int, *kvpb.Error) {
if args.Req != nil && args.Req.Txn != nil && args.Req.Txn.Name == "fake" {
return 0, kvpb.NewError(injErr)
}
return 0, nil
}}}},
})
defer tc.Stopper().Stop(ctx)

k := tc.ScratchRange(t)

repl := tc.GetFirstStoreFromServer(t, 0).LookupReplica(keys.MustAddr(k))

// Try to leave a joint config even though we're not in one. This is something
// that will lead raft to propose an empty entry instead of our conf change.
//
// See: https://github.com/cockroachdb/cockroach/issues/105797
var ba kvpb.BatchRequest
now := tc.Server(0).Clock().Now()
txn := roachpb.MakeTransaction("fake", k, isolation.Serializable, roachpb.NormalUserPriority, now, 500*time.Millisecond.Nanoseconds(), 1)
ba.Txn = &txn
ba.Timestamp = now
ba.Add(&kvpb.EndTxnRequest{
RequestHeader: kvpb.RequestHeader{
Key: k,
},
Commit: true,
InternalCommitTrigger: &roachpb.InternalCommitTrigger{
ChangeReplicasTrigger: &roachpb.ChangeReplicasTrigger{
Desc: repl.Desc(),
},
},
})

_, pErr := repl.Send(ctx, &ba)
// Verify that we see the configuration change below raft, where we rejected it
// (since it would've otherwise blow up the Replica: after all, we intentionally
// proposed an invalid configuration change.
require.True(t, errors.Is(pErr.GoError(), injErr), "%+v", pErr.GoError())
}
25 changes: 13 additions & 12 deletions pkg/kv/kvserver/store.go
Expand Up @@ -305,18 +305,19 @@ func newRaftConfig(
logger raft.Logger,
) *raft.Config {
return &raft.Config{
ID: id,
Applied: uint64(appliedIndex),
AsyncStorageWrites: true,
ElectionTick: storeCfg.RaftElectionTimeoutTicks,
HeartbeatTick: storeCfg.RaftHeartbeatIntervalTicks,
MaxUncommittedEntriesSize: storeCfg.RaftMaxUncommittedEntriesSize,
MaxCommittedSizePerReady: storeCfg.RaftMaxCommittedSizePerReady,
MaxSizePerMsg: storeCfg.RaftMaxSizePerMsg,
MaxInflightMsgs: storeCfg.RaftMaxInflightMsgs,
MaxInflightBytes: storeCfg.RaftMaxInflightBytes,
Storage: strg,
Logger: logger,
ID: id,
Applied: uint64(appliedIndex),
AsyncStorageWrites: true,
ElectionTick: storeCfg.RaftElectionTimeoutTicks,
HeartbeatTick: storeCfg.RaftHeartbeatIntervalTicks,
MaxUncommittedEntriesSize: storeCfg.RaftMaxUncommittedEntriesSize,
MaxCommittedSizePerReady: storeCfg.RaftMaxCommittedSizePerReady,
DisableConfChangeValidation: true, // see https://github.com/cockroachdb/cockroach/issues/105797
MaxSizePerMsg: storeCfg.RaftMaxSizePerMsg,
MaxInflightMsgs: storeCfg.RaftMaxInflightMsgs,
MaxInflightBytes: storeCfg.RaftMaxInflightBytes,
Storage: strg,
Logger: logger,

PreVote: true,
CheckQuorum: storeCfg.RaftEnableCheckQuorum,
Expand Down

0 comments on commit 4585dde

Please sign in to comment.