diff --git a/pkg/storage/client_raft_test.go b/pkg/storage/client_raft_test.go index 110f45ac2bd7..3b6559532782 100644 --- a/pkg/storage/client_raft_test.go +++ b/pkg/storage/client_raft_test.go @@ -1042,13 +1042,22 @@ func TestFailedSnapshotFillsReservation(t *testing.T) { mtc.Start(t, 3) rep, err := mtc.stores[0].GetReplica(1) - if err != nil { - t.Fatal(err) - } + require.NoError(t, err) + repDesc, err := rep.GetReplicaDescriptor() + require.NoError(t, err) + desc := protoutil.Clone(rep.Desc()).(*roachpb.RangeDescriptor) + desc.AddReplica(2, 2, roachpb.LEARNER) + rep2Desc, found := desc.GetReplicaDescriptor(2) + require.True(t, found) header := storage.SnapshotRequest_Header{ CanDecline: true, RangeSize: 100, - State: storagepb.ReplicaState{Desc: rep.Desc()}, + State: storagepb.ReplicaState{Desc: desc}, + RaftMessageRequest: storage.RaftMessageRequest{ + RangeID: rep.RangeID, + FromReplica: repDesc, + ToReplica: rep2Desc, + }, } header.RaftMessageRequest.Message.Snapshot.Data = uuid.UUID{}.GetBytes() // Cause this stream to return an error as soon as we ask it for something. @@ -3643,90 +3652,6 @@ func TestRemovedReplicaError(t *testing.T) { }) } -// TestRemoveRangeWithoutGC ensures that we do not panic when a -// replica has been removed but not yet GC'd (and therefore -// does not have an active raft group). -func TestRemoveRangeWithoutGC(t *testing.T) { - defer leaktest.AfterTest(t)() - - sc := storage.TestStoreConfig(nil) - sc.TestingKnobs.DisableReplicaGCQueue = true - sc.TestingKnobs.DisableEagerReplicaRemoval = true - mtc := &multiTestContext{storeConfig: &sc} - defer mtc.Stop() - mtc.Start(t, 2) - const rangeID roachpb.RangeID = 1 - mtc.replicateRange(rangeID, 1) - mtc.transferLease(context.TODO(), rangeID, 0, 1) - mtc.unreplicateRange(rangeID, 0) - - // Wait for store 0 to process the removal. The in-memory replica - // object still exists but store 0 is no longer present in the - // configuration. - testutils.SucceedsSoon(t, func() error { - rep, err := mtc.stores[0].GetReplica(rangeID) - if err != nil { - return err - } - desc := rep.Desc() - if len(desc.InternalReplicas) != 1 { - return errors.Errorf("range has %d replicas", len(desc.InternalReplicas)) - } - return nil - }) - - // The replica's data is still on disk. - var desc roachpb.RangeDescriptor - descKey := keys.RangeDescriptorKey(roachpb.RKeyMin) - if ok, err := engine.MVCCGetProto(context.Background(), mtc.stores[0].Engine(), descKey, - mtc.stores[0].Clock().Now(), &desc, engine.MVCCGetOptions{}); err != nil { - t.Fatal(err) - } else if !ok { - t.Fatal("expected range descriptor to be present") - } - - // Stop and restart the store. The primary motivation for this test - // is to ensure that the store does not panic on restart (as was - // previously the case). - mtc.stopStore(0) - mtc.restartStore(0) - - // Initially, the in-memory Replica object is recreated from the - // on-disk state. - if _, err := mtc.stores[0].GetReplica(rangeID); err != nil { - t.Fatal(err) - } - - // Re-enable the GC queue to allow the replica to be destroyed - // (after the simulated passage of time). - mtc.advanceClock(context.TODO()) - mtc.manualClock.Increment(int64(storage.ReplicaGCQueueInactivityThreshold + 1)) - mtc.stores[0].SetReplicaGCQueueActive(true) - // There's a fun flake where between when the queue detects that this replica - // needs to be removed and when it actually gets processed whereby an older - // replica will send this replica a raft message which will give it an ID. - // When our replica ID changes the queue will ignore the previous addition and - // we won't be removed. - testutils.SucceedsSoon(t, func() error { - mtc.stores[0].MustForceReplicaGCScanAndProcess() - - // The Replica object should be removed. - const msg = "r[0-9]+ was not found" - if _, err := mtc.stores[0].GetReplica(rangeID); !testutils.IsError(err, msg) { - return errors.Errorf("expected %s, got %v", msg, err) - } - return nil - }) - - // And the data should no longer be on disk. - if ok, err := engine.MVCCGetProto(context.Background(), mtc.stores[0].Engine(), descKey, - mtc.stores[0].Clock().Now(), &desc, engine.MVCCGetOptions{}); err != nil { - t.Fatal(err) - } else if ok { - t.Fatalf("expected range descriptor to be absent") - } -} - func TestTransferRaftLeadership(t *testing.T) { defer leaktest.AfterTest(t)() diff --git a/pkg/storage/metrics.go b/pkg/storage/metrics.go index 9659b6510273..da3409de5ec8 100644 --- a/pkg/storage/metrics.go +++ b/pkg/storage/metrics.go @@ -412,12 +412,6 @@ var ( Measurement: "Snapshots", Unit: metric.Unit_COUNT, } - metaRangeSnapshotsPreemptiveApplied = metric.Metadata{ - Name: "range.snapshots.preemptive-applied", - Help: "Number of applied pre-emptive snapshots", - Measurement: "Snapshots", - Unit: metric.Unit_COUNT, - } metaRangeRaftLeaderTransfers = metric.Metadata{ Name: "range.raftleadertransfers", Help: "Number of raft leader transfers", @@ -1060,15 +1054,14 @@ type StoreMetrics struct { // accordingly. // Range event metrics. - RangeSplits *metric.Counter - RangeMerges *metric.Counter - RangeAdds *metric.Counter - RangeRemoves *metric.Counter - RangeSnapshotsGenerated *metric.Counter - RangeSnapshotsNormalApplied *metric.Counter - RangeSnapshotsLearnerApplied *metric.Counter - RangeSnapshotsPreemptiveApplied *metric.Counter - RangeRaftLeaderTransfers *metric.Counter + RangeSplits *metric.Counter + RangeMerges *metric.Counter + RangeAdds *metric.Counter + RangeRemoves *metric.Counter + RangeSnapshotsGenerated *metric.Counter + RangeSnapshotsNormalApplied *metric.Counter + RangeSnapshotsLearnerApplied *metric.Counter + RangeRaftLeaderTransfers *metric.Counter // Raft processing metrics. RaftTicks *metric.Counter @@ -1267,15 +1260,14 @@ func newStoreMetrics(histogramWindow time.Duration) *StoreMetrics { RdbPendingCompaction: metric.NewGauge(metaRdbPendingCompaction), // Range event metrics. - RangeSplits: metric.NewCounter(metaRangeSplits), - RangeMerges: metric.NewCounter(metaRangeMerges), - RangeAdds: metric.NewCounter(metaRangeAdds), - RangeRemoves: metric.NewCounter(metaRangeRemoves), - RangeSnapshotsGenerated: metric.NewCounter(metaRangeSnapshotsGenerated), - RangeSnapshotsNormalApplied: metric.NewCounter(metaRangeSnapshotsNormalApplied), - RangeSnapshotsLearnerApplied: metric.NewCounter(metaRangeSnapshotsLearnerApplied), - RangeSnapshotsPreemptiveApplied: metric.NewCounter(metaRangeSnapshotsPreemptiveApplied), - RangeRaftLeaderTransfers: metric.NewCounter(metaRangeRaftLeaderTransfers), + RangeSplits: metric.NewCounter(metaRangeSplits), + RangeMerges: metric.NewCounter(metaRangeMerges), + RangeAdds: metric.NewCounter(metaRangeAdds), + RangeRemoves: metric.NewCounter(metaRangeRemoves), + RangeSnapshotsGenerated: metric.NewCounter(metaRangeSnapshotsGenerated), + RangeSnapshotsNormalApplied: metric.NewCounter(metaRangeSnapshotsNormalApplied), + RangeSnapshotsLearnerApplied: metric.NewCounter(metaRangeSnapshotsLearnerApplied), + RangeRaftLeaderTransfers: metric.NewCounter(metaRangeRaftLeaderTransfers), // Raft processing metrics. RaftTicks: metric.NewCounter(metaRaftTicks), diff --git a/pkg/storage/raft.pb.go b/pkg/storage/raft.pb.go index 05c6e6ce9c58..3596aac08569 100644 --- a/pkg/storage/raft.pb.go +++ b/pkg/storage/raft.pb.go @@ -70,7 +70,7 @@ func (x *SnapshotRequest_Priority) UnmarshalJSON(data []byte) error { return nil } func (SnapshotRequest_Priority) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_raft_c419318fe988e310, []int{5, 0} + return fileDescriptor_raft_4f9bfa2e6827576b, []int{5, 0} } type SnapshotRequest_Strategy int32 @@ -107,26 +107,23 @@ func (x *SnapshotRequest_Strategy) UnmarshalJSON(data []byte) error { return nil } func (SnapshotRequest_Strategy) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_raft_c419318fe988e310, []int{5, 1} + return fileDescriptor_raft_4f9bfa2e6827576b, []int{5, 1} } type SnapshotRequest_Type int32 const ( - SnapshotRequest_RAFT SnapshotRequest_Type = 0 - SnapshotRequest_LEARNER SnapshotRequest_Type = 1 - SnapshotRequest_PREEMPTIVE SnapshotRequest_Type = 2 + SnapshotRequest_RAFT SnapshotRequest_Type = 0 + SnapshotRequest_LEARNER SnapshotRequest_Type = 1 ) var SnapshotRequest_Type_name = map[int32]string{ 0: "RAFT", 1: "LEARNER", - 2: "PREEMPTIVE", } var SnapshotRequest_Type_value = map[string]int32{ - "RAFT": 0, - "LEARNER": 1, - "PREEMPTIVE": 2, + "RAFT": 0, + "LEARNER": 1, } func (x SnapshotRequest_Type) Enum() *SnapshotRequest_Type { @@ -146,7 +143,7 @@ func (x *SnapshotRequest_Type) UnmarshalJSON(data []byte) error { return nil } func (SnapshotRequest_Type) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_raft_c419318fe988e310, []int{5, 2} + return fileDescriptor_raft_4f9bfa2e6827576b, []int{5, 2} } type SnapshotResponse_Status int32 @@ -191,7 +188,7 @@ func (x *SnapshotResponse_Status) UnmarshalJSON(data []byte) error { return nil } func (SnapshotResponse_Status) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_raft_c419318fe988e310, []int{6, 0} + return fileDescriptor_raft_4f9bfa2e6827576b, []int{6, 0} } // RaftHeartbeat is a request that contains the barebones information for a @@ -220,7 +217,7 @@ func (m *RaftHeartbeat) Reset() { *m = RaftHeartbeat{} } func (m *RaftHeartbeat) String() string { return proto.CompactTextString(m) } func (*RaftHeartbeat) ProtoMessage() {} func (*RaftHeartbeat) Descriptor() ([]byte, []int) { - return fileDescriptor_raft_c419318fe988e310, []int{0} + return fileDescriptor_raft_4f9bfa2e6827576b, []int{0} } func (m *RaftHeartbeat) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -276,7 +273,7 @@ func (m *RaftMessageRequest) Reset() { *m = RaftMessageRequest{} } func (m *RaftMessageRequest) String() string { return proto.CompactTextString(m) } func (*RaftMessageRequest) ProtoMessage() {} func (*RaftMessageRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_raft_c419318fe988e310, []int{1} + return fileDescriptor_raft_4f9bfa2e6827576b, []int{1} } func (m *RaftMessageRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -309,7 +306,7 @@ func (m *RaftMessageRequestBatch) Reset() { *m = RaftMessageRequestBatch func (m *RaftMessageRequestBatch) String() string { return proto.CompactTextString(m) } func (*RaftMessageRequestBatch) ProtoMessage() {} func (*RaftMessageRequestBatch) Descriptor() ([]byte, []int) { - return fileDescriptor_raft_c419318fe988e310, []int{2} + return fileDescriptor_raft_4f9bfa2e6827576b, []int{2} } func (m *RaftMessageRequestBatch) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -342,7 +339,7 @@ func (m *RaftMessageResponseUnion) Reset() { *m = RaftMessageResponseUni func (m *RaftMessageResponseUnion) String() string { return proto.CompactTextString(m) } func (*RaftMessageResponseUnion) ProtoMessage() {} func (*RaftMessageResponseUnion) Descriptor() ([]byte, []int) { - return fileDescriptor_raft_c419318fe988e310, []int{3} + return fileDescriptor_raft_4f9bfa2e6827576b, []int{3} } func (m *RaftMessageResponseUnion) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -385,7 +382,7 @@ func (m *RaftMessageResponse) Reset() { *m = RaftMessageResponse{} } func (m *RaftMessageResponse) String() string { return proto.CompactTextString(m) } func (*RaftMessageResponse) ProtoMessage() {} func (*RaftMessageResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_raft_c419318fe988e310, []int{4} + return fileDescriptor_raft_4f9bfa2e6827576b, []int{4} } func (m *RaftMessageResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -426,7 +423,7 @@ func (m *SnapshotRequest) Reset() { *m = SnapshotRequest{} } func (m *SnapshotRequest) String() string { return proto.CompactTextString(m) } func (*SnapshotRequest) ProtoMessage() {} func (*SnapshotRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_raft_c419318fe988e310, []int{5} + return fileDescriptor_raft_4f9bfa2e6827576b, []int{5} } func (m *SnapshotRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -488,7 +485,7 @@ func (m *SnapshotRequest_Header) Reset() { *m = SnapshotRequest_Header{} func (m *SnapshotRequest_Header) String() string { return proto.CompactTextString(m) } func (*SnapshotRequest_Header) ProtoMessage() {} func (*SnapshotRequest_Header) Descriptor() ([]byte, []int) { - return fileDescriptor_raft_c419318fe988e310, []int{5, 0} + return fileDescriptor_raft_4f9bfa2e6827576b, []int{5, 0} } func (m *SnapshotRequest_Header) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -522,7 +519,7 @@ func (m *SnapshotResponse) Reset() { *m = SnapshotResponse{} } func (m *SnapshotResponse) String() string { return proto.CompactTextString(m) } func (*SnapshotResponse) ProtoMessage() {} func (*SnapshotResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_raft_c419318fe988e310, []int{6} + return fileDescriptor_raft_4f9bfa2e6827576b, []int{6} } func (m *SnapshotResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -559,7 +556,7 @@ func (m *ConfChangeContext) Reset() { *m = ConfChangeContext{} } func (m *ConfChangeContext) String() string { return proto.CompactTextString(m) } func (*ConfChangeContext) ProtoMessage() {} func (*ConfChangeContext) Descriptor() ([]byte, []int) { - return fileDescriptor_raft_c419318fe988e310, []int{7} + return fileDescriptor_raft_4f9bfa2e6827576b, []int{7} } func (m *ConfChangeContext) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2852,86 +2849,86 @@ var ( ErrIntOverflowRaft = fmt.Errorf("proto: integer overflow") ) -func init() { proto.RegisterFile("storage/raft.proto", fileDescriptor_raft_c419318fe988e310) } - -var fileDescriptor_raft_c419318fe988e310 = []byte{ - // 1243 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe4, 0x56, 0x4b, 0x6f, 0xdb, 0x46, - 0x10, 0x16, 0x2d, 0x5a, 0x8f, 0x91, 0x64, 0x33, 0xdb, 0xa0, 0x25, 0xd4, 0x54, 0x16, 0x94, 0x26, - 0x55, 0x52, 0x54, 0x4a, 0x8d, 0xb4, 0x87, 0xde, 0xf4, 0xa0, 0x63, 0xc5, 0x2f, 0x81, 0x56, 0x5c, - 0xb4, 0x40, 0x20, 0x50, 0xd4, 0x5a, 0x22, 0x2c, 0x71, 0x99, 0xe5, 0x2a, 0xad, 0xf2, 0x2b, 0xfa, - 0x13, 0x7a, 0xed, 0x3f, 0xf1, 0xa5, 0x40, 0x8e, 0x01, 0x5a, 0x18, 0x8d, 0xd3, 0x5f, 0x91, 0x53, - 0xb1, 0x0f, 0xca, 0x8c, 0xed, 0x36, 0x36, 0x50, 0xf4, 0xd2, 0x8b, 0x44, 0xce, 0xe3, 0x1b, 0xce, - 0x7c, 0x33, 0xb3, 0x0b, 0x28, 0x64, 0x84, 0x3a, 0x23, 0x5c, 0xa7, 0xce, 0x21, 0xab, 0x05, 0x94, - 0x30, 0x82, 0x6e, 0xb8, 0xc4, 0x3d, 0xa2, 0xc4, 0x71, 0xc7, 0x35, 0xa5, 0x2d, 0xde, 0x14, 0xaf, - 0xc1, 0xa0, 0x8e, 0x29, 0x25, 0x34, 0x94, 0x86, 0xc5, 0x0f, 0x23, 0xe9, 0x14, 0x33, 0x67, 0xe8, - 0x30, 0x47, 0xc9, 0x3f, 0x89, 0x40, 0xd5, 0x7f, 0x30, 0xa8, 0x87, 0xcc, 0x61, 0x58, 0xa9, 0x3f, - 0xc6, 0xcc, 0x1d, 0x8a, 0x80, 0xe2, 0x27, 0x18, 0xc4, 0x82, 0x17, 0x6f, 0x8e, 0xc8, 0x88, 0x88, - 0xc7, 0x3a, 0x7f, 0x92, 0xd2, 0xca, 0xeb, 0x24, 0x14, 0x6c, 0xe7, 0x90, 0x6d, 0x62, 0x87, 0xb2, - 0x01, 0x76, 0x18, 0x1a, 0x40, 0x86, 0x3a, 0xfe, 0x08, 0xf7, 0xbd, 0xa1, 0xa9, 0x95, 0xb5, 0xaa, - 0xde, 0x7c, 0x74, 0x7c, 0xb2, 0x96, 0x38, 0x3d, 0x59, 0x4b, 0xdb, 0x5c, 0xde, 0x69, 0xbf, 0x3d, - 0x59, 0x7b, 0x38, 0xf2, 0xd8, 0x78, 0x36, 0xa8, 0xb9, 0x64, 0x5a, 0x5f, 0x24, 0x35, 0x1c, 0x9c, - 0x3d, 0xd7, 0x83, 0xa3, 0x51, 0x5d, 0x65, 0x51, 0x53, 0x7e, 0x76, 0x5a, 0x00, 0x77, 0x86, 0xe8, - 0x07, 0x58, 0x3d, 0xa4, 0x64, 0xda, 0xa7, 0x38, 0x98, 0x78, 0xae, 0xc3, 0x43, 0x2d, 0x95, 0xb5, - 0x6a, 0xa1, 0xb9, 0xa7, 0x42, 0x15, 0x36, 0x28, 0x99, 0xda, 0x52, 0x2b, 0x02, 0x7e, 0x7d, 0xbd, - 0x80, 0x91, 0xa7, 0x5d, 0x38, 0x8c, 0x01, 0x0d, 0xd1, 0x33, 0x28, 0x30, 0x12, 0x0f, 0x9b, 0x14, - 0x61, 0x77, 0x54, 0xd8, 0x5c, 0x8f, 0xfc, 0x1b, 0x41, 0x73, 0x8c, 0x9c, 0x85, 0x34, 0x41, 0x67, - 0x98, 0x4e, 0x4d, 0x5d, 0xd4, 0x52, 0xe7, 0x91, 0x6c, 0x21, 0x41, 0xb7, 0x20, 0xe5, 0x92, 0xe9, - 0xd4, 0x63, 0xe6, 0x72, 0x4c, 0xa7, 0x64, 0xa8, 0x04, 0xe9, 0x67, 0x33, 0x0f, 0x87, 0x2e, 0x36, - 0x53, 0x65, 0xad, 0x9a, 0x51, 0xea, 0x48, 0x88, 0xaa, 0x22, 0x15, 0x2f, 0xec, 0x4f, 0xb0, 0x43, - 0x7d, 0x4c, 0xcd, 0x74, 0xcc, 0x2a, 0xc7, 0x48, 0x27, 0xdc, 0x96, 0x8a, 0xca, 0x6f, 0x3a, 0x20, - 0xce, 0xf1, 0x0e, 0x0e, 0x43, 0x67, 0x84, 0x6d, 0xfc, 0x6c, 0x86, 0xc3, 0xff, 0x86, 0xe8, 0x1d, - 0xc8, 0xc7, 0x89, 0x16, 0x2c, 0xe7, 0xd6, 0x3f, 0xad, 0x9d, 0x0d, 0xc2, 0xb9, 0xea, 0xb5, 0x71, - 0xe8, 0x52, 0x2f, 0x60, 0x84, 0x46, 0x99, 0xc4, 0x08, 0x44, 0x1d, 0x80, 0x33, 0xfa, 0x04, 0x77, - 0xd7, 0x03, 0xcb, 0x2e, 0x88, 0x41, 0x75, 0x48, 0x4f, 0x65, 0x3d, 0x04, 0x33, 0xb9, 0xf5, 0xd5, - 0x9a, 0x9c, 0x99, 0x9a, 0x2a, 0x53, 0x54, 0x6f, 0x65, 0x15, 0xe7, 0x63, 0xf9, 0x32, 0x3e, 0x36, - 0x00, 0xc6, 0xd1, 0x10, 0x85, 0x66, 0xaa, 0x9c, 0xac, 0xe6, 0xd6, 0xcb, 0xb5, 0x0b, 0x13, 0x5f, - 0x7b, 0x67, 0xda, 0x14, 0x48, 0xcc, 0x13, 0xed, 0xc1, 0xea, 0xe2, 0xad, 0x4f, 0x71, 0x18, 0x84, - 0x66, 0xfa, 0x5a, 0x60, 0x2b, 0x0b, 0x77, 0x9b, 0x7b, 0xa3, 0xa7, 0xb0, 0x2a, 0x79, 0x0e, 0x99, - 0x43, 0x59, 0xff, 0x08, 0xcf, 0xcd, 0x4c, 0x59, 0xab, 0xe6, 0x9b, 0x5f, 0xbd, 0x3d, 0x59, 0xfb, - 0xf2, 0x7a, 0xfc, 0x6e, 0xe1, 0xb9, 0x5d, 0x10, 0x68, 0xfb, 0x1c, 0x6c, 0x0b, 0xcf, 0x2b, 0x03, - 0xf8, 0xe8, 0x62, 0x73, 0x35, 0x1d, 0xe6, 0x8e, 0xd1, 0x23, 0xc8, 0x50, 0xf9, 0x1e, 0x9a, 0x9a, - 0xc8, 0xe1, 0xce, 0xdf, 0xe4, 0x70, 0xce, 0x5b, 0x26, 0xb2, 0x70, 0xae, 0x74, 0xc1, 0x7c, 0xc7, - 0x2a, 0x0c, 0x88, 0x1f, 0xe2, 0x27, 0xbe, 0x47, 0x7c, 0x54, 0x83, 0x65, 0xb1, 0x3b, 0x45, 0x0f, - 0xe7, 0xd6, 0xcd, 0x4b, 0xda, 0xc1, 0xe2, 0x7a, 0x5b, 0x9a, 0x7d, 0xa3, 0x1f, 0xff, 0xbc, 0xa6, - 0x55, 0x7e, 0x5f, 0x82, 0x0f, 0x2e, 0x81, 0xfc, 0x9f, 0x0f, 0xc5, 0x23, 0x58, 0x9e, 0xf1, 0xa2, - 0xaa, 0x91, 0xf8, 0xfc, 0x7d, 0x6c, 0xc5, 0x78, 0x50, 0x60, 0xd2, 0xbf, 0xf2, 0x67, 0x0a, 0x56, - 0xf7, 0x7d, 0x27, 0x08, 0xc7, 0x84, 0x45, 0xfb, 0xa6, 0x01, 0xa9, 0x31, 0x76, 0x86, 0x38, 0x62, - 0xea, 0xde, 0x25, 0xe8, 0xe7, 0x7c, 0x6a, 0x9b, 0xc2, 0xc1, 0x56, 0x8e, 0xe8, 0x2e, 0x64, 0x8e, - 0x9e, 0xf7, 0x07, 0xbc, 0xb9, 0x44, 0xd5, 0xf2, 0xcd, 0x1c, 0x67, 0x66, 0xeb, 0x40, 0xf4, 0x9b, - 0x9d, 0x3e, 0x7a, 0x2e, 0x1b, 0x6f, 0x0d, 0x72, 0x13, 0x32, 0xea, 0x63, 0x9f, 0x51, 0x0f, 0x87, - 0x66, 0xb2, 0x9c, 0xac, 0xe6, 0x6d, 0x98, 0x90, 0x91, 0x25, 0x25, 0xa8, 0x08, 0xcb, 0x87, 0x9e, - 0xef, 0x4c, 0x44, 0xa2, 0xd1, 0x28, 0x4b, 0x51, 0xf1, 0x17, 0x1d, 0x52, 0x32, 0x2e, 0x7a, 0x0a, - 0x37, 0xf9, 0x52, 0xe8, 0xab, 0x1d, 0xd0, 0x57, 0x0d, 0xa9, 0x18, 0xbb, 0x56, 0x33, 0x23, 0x7a, - 0x71, 0x03, 0xdf, 0x06, 0x50, 0x93, 0xe9, 0xbd, 0xc0, 0x82, 0xb9, 0x64, 0xc4, 0x89, 0x9c, 0x31, - 0xef, 0x05, 0x46, 0x77, 0x20, 0xe7, 0x3a, 0x7e, 0x7f, 0x88, 0xdd, 0x89, 0xe7, 0xe3, 0x77, 0x3e, - 0x18, 0x5c, 0xc7, 0x6f, 0x4b, 0x39, 0xb2, 0x60, 0x59, 0x5c, 0x05, 0xc4, 0x72, 0xba, 0xbc, 0xb8, - 0x8b, 0x4b, 0x43, 0xd4, 0x0a, 0xfb, 0xdc, 0x21, 0x4a, 0x5e, 0x78, 0xa3, 0x1d, 0xc8, 0x04, 0xd4, - 0x23, 0xd4, 0x63, 0x73, 0x71, 0xec, 0xac, 0x5c, 0xda, 0x04, 0xe7, 0x69, 0xea, 0x2a, 0x97, 0x68, - 0x70, 0x23, 0x08, 0x0e, 0x17, 0x32, 0xea, 0x30, 0x3c, 0x9a, 0x8b, 0xf3, 0xe9, 0x6a, 0x70, 0xfb, - 0xca, 0x25, 0x82, 0x8b, 0x20, 0xd0, 0x06, 0xdc, 0x9a, 0xf9, 0xaa, 0xd3, 0x19, 0x1e, 0xf6, 0x19, - 0x9d, 0xf9, 0xf2, 0x49, 0xe6, 0x9e, 0x89, 0x15, 0xa7, 0x18, 0xb7, 0xec, 0x45, 0x86, 0x22, 0x65, - 0xd4, 0x00, 0x9d, 0xcd, 0x03, 0x6c, 0x66, 0xc5, 0x27, 0x7d, 0x76, 0x85, 0x4f, 0xea, 0xcd, 0x03, - 0xbc, 0x38, 0xbc, 0xe7, 0x01, 0x7e, 0xac, 0x67, 0x34, 0x63, 0xa9, 0xf2, 0x10, 0x32, 0x51, 0xee, - 0x28, 0x07, 0xe9, 0x27, 0xbb, 0x5b, 0xbb, 0x7b, 0xdf, 0xee, 0x1a, 0x09, 0x94, 0x87, 0x8c, 0x6d, - 0xb5, 0xf6, 0x0e, 0x2c, 0xfb, 0x3b, 0x43, 0x43, 0x05, 0xc8, 0xda, 0x56, 0xb3, 0xb1, 0xdd, 0xd8, - 0x6d, 0x59, 0xc6, 0x52, 0xc5, 0x84, 0x4c, 0x94, 0x22, 0x37, 0xdc, 0x3a, 0xe8, 0x37, 0x1b, 0xbd, - 0xd6, 0xa6, 0x91, 0xa8, 0x7c, 0x01, 0x3a, 0x8f, 0x84, 0x32, 0xa0, 0xdb, 0x8d, 0x8d, 0x9e, 0x91, - 0xe0, 0xa8, 0xdb, 0x56, 0xc3, 0xde, 0xb5, 0x6c, 0x43, 0x43, 0x2b, 0x00, 0x5d, 0xdb, 0xb2, 0x76, - 0xba, 0xbd, 0xce, 0x01, 0x07, 0xfa, 0x55, 0x03, 0xe3, 0xec, 0x4b, 0xd5, 0x0a, 0xdb, 0x84, 0x14, - 0xaf, 0xc6, 0x2c, 0x14, 0x73, 0xb6, 0xb2, 0x7e, 0xff, 0x1f, 0xd3, 0x93, 0x4e, 0xb5, 0x7d, 0xe1, - 0x11, 0x5d, 0x41, 0xa4, 0x3f, 0x3f, 0xf2, 0xa2, 0x33, 0x92, 0x77, 0x7c, 0xf6, 0xdc, 0x91, 0x58, - 0xe9, 0x40, 0x4a, 0xfa, 0x5d, 0xc8, 0xbd, 0xd1, 0x6a, 0x59, 0xdd, 0x9e, 0xd5, 0x36, 0x34, 0xae, - 0x6a, 0x74, 0xbb, 0xdb, 0x1d, 0xab, 0x6d, 0x2c, 0xa1, 0x2c, 0x2c, 0x5b, 0xb6, 0xbd, 0x67, 0x1b, - 0x49, 0x6e, 0xd5, 0xb6, 0x5a, 0xdb, 0x9d, 0x5d, 0xab, 0x6d, 0xe8, 0x8f, 0xf5, 0x4c, 0xd2, 0xd0, - 0x2b, 0x7d, 0xb8, 0xd1, 0x22, 0xfe, 0x61, 0x6b, 0xcc, 0xbb, 0xbf, 0x45, 0x7c, 0x86, 0x7f, 0x64, - 0xe8, 0x01, 0x00, 0xbf, 0x12, 0x39, 0xfe, 0x30, 0x5a, 0xca, 0xd9, 0xe6, 0x0d, 0xb5, 0x94, 0xb3, - 0x2d, 0xa9, 0xe9, 0xb4, 0xed, 0xac, 0x32, 0x12, 0x57, 0xae, 0x74, 0xe0, 0xcc, 0x27, 0xc4, 0x91, - 0xd7, 0xca, 0xbc, 0x1d, 0xbd, 0xae, 0xbf, 0xd2, 0x20, 0xbb, 0x33, 0x9b, 0x30, 0x8f, 0xcf, 0x29, - 0x9a, 0x80, 0x11, 0x9b, 0x57, 0xb9, 0x3a, 0xee, 0x5f, 0x6d, 0xa8, 0xb9, 0x6d, 0xf1, 0xee, 0xd5, - 0xf6, 0x63, 0x25, 0x51, 0xd5, 0x1e, 0x68, 0xe8, 0x29, 0xe4, 0xb9, 0x32, 0x2a, 0x3d, 0xaa, 0xbc, - 0xbf, 0xed, 0x8a, 0xb7, 0xaf, 0xc0, 0x9d, 0x84, 0x6f, 0xde, 0x3b, 0x7e, 0x5d, 0x4a, 0x1c, 0x9f, - 0x96, 0xb4, 0x97, 0xa7, 0x25, 0xed, 0xd5, 0x69, 0x49, 0xfb, 0xe3, 0xb4, 0xa4, 0xfd, 0xf4, 0xa6, - 0x94, 0x78, 0xf9, 0xa6, 0x94, 0x78, 0xf5, 0xa6, 0x94, 0xf8, 0x3e, 0xad, 0x10, 0xfe, 0x0a, 0x00, - 0x00, 0xff, 0xff, 0xd1, 0xe2, 0x72, 0x64, 0x9c, 0x0c, 0x00, 0x00, +func init() { proto.RegisterFile("storage/raft.proto", fileDescriptor_raft_4f9bfa2e6827576b) } + +var fileDescriptor_raft_4f9bfa2e6827576b = []byte{ + // 1238 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe4, 0x56, 0xcd, 0x6e, 0xdb, 0xc6, + 0x16, 0x16, 0x25, 0x4a, 0xa2, 0x8e, 0xa4, 0x98, 0x99, 0x1b, 0xdc, 0x4b, 0xe8, 0xe6, 0xca, 0x02, + 0x73, 0x93, 0x2a, 0x29, 0x20, 0xa5, 0x46, 0xda, 0x45, 0x77, 0xfa, 0xa1, 0x63, 0xc5, 0xbf, 0xa0, + 0x9d, 0x14, 0x2d, 0x10, 0x08, 0x94, 0x34, 0x96, 0x08, 0x4b, 0x1c, 0x66, 0x38, 0x4a, 0xab, 0x3c, + 0x45, 0x1f, 0xa1, 0xdb, 0xbe, 0x89, 0x37, 0x05, 0xb2, 0x0c, 0xda, 0xc2, 0x68, 0x9c, 0xb7, 0xc8, + 0xaa, 0x98, 0x1f, 0xca, 0x8c, 0xed, 0x36, 0x36, 0x50, 0x74, 0xd3, 0x8d, 0x44, 0x9e, 0x9f, 0xef, + 0xf0, 0x9c, 0xef, 0x9c, 0x33, 0x03, 0x28, 0x62, 0x84, 0x7a, 0x63, 0xdc, 0xa4, 0xde, 0x21, 0x6b, + 0x84, 0x94, 0x30, 0x82, 0x6e, 0x0e, 0xc9, 0xf0, 0x88, 0x12, 0x6f, 0x38, 0x69, 0x28, 0x6d, 0xe5, + 0x96, 0x78, 0x0d, 0x07, 0x4d, 0x4c, 0x29, 0xa1, 0x91, 0x34, 0xac, 0xfc, 0x3b, 0x96, 0xce, 0x30, + 0xf3, 0x46, 0x1e, 0xf3, 0x94, 0xfc, 0x7f, 0x31, 0xa8, 0xfa, 0x0f, 0x07, 0xcd, 0x88, 0x79, 0x0c, + 0x2b, 0xf5, 0x7f, 0x31, 0x1b, 0x8e, 0x44, 0x40, 0xf1, 0x13, 0x0e, 0x12, 0xc1, 0x2b, 0xb7, 0xc6, + 0x64, 0x4c, 0xc4, 0x63, 0x93, 0x3f, 0x49, 0xa9, 0xfd, 0x36, 0x03, 0x65, 0xd7, 0x3b, 0x64, 0x1b, + 0xd8, 0xa3, 0x6c, 0x80, 0x3d, 0x86, 0x06, 0x60, 0x50, 0x2f, 0x18, 0xe3, 0xbe, 0x3f, 0xb2, 0xb4, + 0x9a, 0x56, 0xd7, 0xdb, 0x8f, 0x8f, 0x4f, 0x56, 0x53, 0xa7, 0x27, 0xab, 0x79, 0x97, 0xcb, 0x7b, + 0xdd, 0xf7, 0x27, 0xab, 0x8f, 0xc6, 0x3e, 0x9b, 0xcc, 0x07, 0x8d, 0x21, 0x99, 0x35, 0x97, 0x49, + 0x8d, 0x06, 0x67, 0xcf, 0xcd, 0xf0, 0x68, 0xdc, 0x54, 0x59, 0x34, 0x94, 0x9f, 0x9b, 0x17, 0xc0, + 0xbd, 0x11, 0xfa, 0x16, 0x56, 0x0e, 0x29, 0x99, 0xf5, 0x29, 0x0e, 0xa7, 0xfe, 0xd0, 0xe3, 0xa1, + 0xd2, 0x35, 0xad, 0x5e, 0x6e, 0xef, 0xaa, 0x50, 0xe5, 0x75, 0x4a, 0x66, 0xae, 0xd4, 0x8a, 0x80, + 0x5f, 0x5c, 0x2f, 0x60, 0xec, 0xe9, 0x96, 0x0f, 0x13, 0x40, 0x23, 0xf4, 0x02, 0xca, 0x8c, 0x24, + 0xc3, 0x66, 0x44, 0xd8, 0x6d, 0x15, 0xb6, 0x78, 0x40, 0xfe, 0x8a, 0xa0, 0x45, 0x46, 0xce, 0x42, + 0x5a, 0xa0, 0x33, 0x4c, 0x67, 0x96, 0x2e, 0x6a, 0xa9, 0xf3, 0x48, 0xae, 0x90, 0xa0, 0xdb, 0x90, + 0x1b, 0x92, 0xd9, 0xcc, 0x67, 0x56, 0x36, 0xa1, 0x53, 0x32, 0x54, 0x85, 0xfc, 0x8b, 0xb9, 0x8f, + 0xa3, 0x21, 0xb6, 0x72, 0x35, 0xad, 0x6e, 0x28, 0x75, 0x2c, 0x44, 0x75, 0x91, 0x8a, 0x1f, 0xf5, + 0xa7, 0xd8, 0xa3, 0x01, 0xa6, 0x56, 0x3e, 0x61, 0x55, 0x64, 0xa4, 0x17, 0x6d, 0x49, 0x85, 0xfd, + 0x8b, 0x0e, 0x88, 0x73, 0xbc, 0x8d, 0xa3, 0xc8, 0x1b, 0x63, 0x17, 0xbf, 0x98, 0xe3, 0xe8, 0xef, + 0x21, 0x7a, 0x1b, 0x4a, 0x49, 0xa2, 0x05, 0xcb, 0xc5, 0xb5, 0xff, 0x37, 0xce, 0x06, 0xe1, 0x5c, + 0xf5, 0xba, 0x38, 0x1a, 0x52, 0x3f, 0x64, 0x84, 0xc6, 0x99, 0x24, 0x08, 0x44, 0x3d, 0x80, 0x33, + 0xfa, 0x04, 0x77, 0xd7, 0x03, 0x2b, 0x2c, 0x89, 0x41, 0x4d, 0xc8, 0xcf, 0x64, 0x3d, 0x04, 0x33, + 0xc5, 0xb5, 0x95, 0x86, 0x9c, 0x99, 0x86, 0x2a, 0x53, 0x5c, 0x6f, 0x65, 0x95, 0xe4, 0x23, 0x7b, + 0x19, 0x1f, 0xeb, 0x00, 0x93, 0x78, 0x88, 0x22, 0x2b, 0x57, 0xcb, 0xd4, 0x8b, 0x6b, 0xb5, 0xc6, + 0x85, 0x89, 0x6f, 0x7c, 0x30, 0x6d, 0x0a, 0x24, 0xe1, 0x89, 0x76, 0x61, 0x65, 0xf9, 0xd6, 0xa7, + 0x38, 0x0a, 0x23, 0x2b, 0x7f, 0x2d, 0xb0, 0x1b, 0x4b, 0x77, 0x97, 0x7b, 0xa3, 0xe7, 0xb0, 0x22, + 0x79, 0x8e, 0x98, 0x47, 0x59, 0xff, 0x08, 0x2f, 0x2c, 0xa3, 0xa6, 0xd5, 0x4b, 0xed, 0xcf, 0xdf, + 0x9f, 0xac, 0x7e, 0x76, 0x3d, 0x7e, 0x37, 0xf1, 0xc2, 0x2d, 0x0b, 0xb4, 0x7d, 0x0e, 0xb6, 0x89, + 0x17, 0xf6, 0x00, 0xfe, 0x73, 0xb1, 0xb9, 0xda, 0x1e, 0x1b, 0x4e, 0xd0, 0x63, 0x30, 0xa8, 0x7c, + 0x8f, 0x2c, 0x4d, 0xe4, 0x70, 0xf7, 0x0f, 0x72, 0x38, 0xe7, 0x2d, 0x13, 0x59, 0x3a, 0xdb, 0x7b, + 0x60, 0x7d, 0x60, 0x15, 0x85, 0x24, 0x88, 0xf0, 0xd3, 0xc0, 0x27, 0x01, 0x6a, 0x40, 0x56, 0xec, + 0x4e, 0xd1, 0xc3, 0xc5, 0x35, 0xeb, 0x92, 0x76, 0x70, 0xb8, 0xde, 0x95, 0x66, 0x5f, 0xea, 0xc7, + 0x3f, 0xac, 0x6a, 0xf6, 0xaf, 0x69, 0xf8, 0xd7, 0x25, 0x90, 0xff, 0xf0, 0xa1, 0x78, 0x0c, 0xd9, + 0x39, 0x2f, 0xaa, 0x1a, 0x89, 0x4f, 0x3f, 0xc6, 0x56, 0x82, 0x07, 0x05, 0x26, 0xfd, 0xed, 0x9f, + 0x73, 0xb0, 0xb2, 0x1f, 0x78, 0x61, 0x34, 0x21, 0x2c, 0xde, 0x37, 0x2d, 0xc8, 0x4d, 0xb0, 0x37, + 0xc2, 0x31, 0x53, 0xf7, 0x2f, 0x41, 0x3f, 0xe7, 0xd3, 0xd8, 0x10, 0x0e, 0xae, 0x72, 0x44, 0xf7, + 0xc0, 0x38, 0x7a, 0xd9, 0x1f, 0xf0, 0xe6, 0x12, 0x55, 0x2b, 0xb5, 0x8b, 0x9c, 0x99, 0xcd, 0x67, + 0xa2, 0xdf, 0xdc, 0xfc, 0xd1, 0x4b, 0xd9, 0x78, 0xab, 0x50, 0x9c, 0x92, 0x71, 0x1f, 0x07, 0x8c, + 0xfa, 0x38, 0xb2, 0x32, 0xb5, 0x4c, 0xbd, 0xe4, 0xc2, 0x94, 0x8c, 0x1d, 0x29, 0x41, 0x15, 0xc8, + 0x1e, 0xfa, 0x81, 0x37, 0x15, 0x89, 0xc6, 0xa3, 0x2c, 0x45, 0x95, 0x1f, 0x75, 0xc8, 0xc9, 0xb8, + 0xe8, 0x39, 0xdc, 0xe2, 0x4b, 0xa1, 0xaf, 0x76, 0x40, 0x5f, 0x35, 0xa4, 0x62, 0xec, 0x5a, 0xcd, + 0x8c, 0xe8, 0xc5, 0x0d, 0x7c, 0x07, 0x40, 0x4d, 0xa6, 0xff, 0x0a, 0x0b, 0xe6, 0x32, 0x31, 0x27, + 0x72, 0xc6, 0xfc, 0x57, 0x18, 0xdd, 0x85, 0xe2, 0xd0, 0x0b, 0xfa, 0x23, 0x3c, 0x9c, 0xfa, 0x01, + 0xfe, 0xe0, 0x83, 0x61, 0xe8, 0x05, 0x5d, 0x29, 0x47, 0x0e, 0x64, 0xc5, 0x55, 0x40, 0x2c, 0xa7, + 0xcb, 0x8b, 0xbb, 0xbc, 0x34, 0xc4, 0xad, 0xb0, 0xcf, 0x1d, 0xe2, 0xe4, 0x85, 0x37, 0xda, 0x06, + 0x23, 0xa4, 0x3e, 0xa1, 0x3e, 0x5b, 0x88, 0x63, 0xe7, 0xc6, 0xa5, 0x4d, 0x70, 0x9e, 0xa6, 0x3d, + 0xe5, 0x12, 0x0f, 0x6e, 0x0c, 0xc1, 0xe1, 0x22, 0x46, 0x3d, 0x86, 0xc7, 0x0b, 0x71, 0x3e, 0x5d, + 0x0d, 0x6e, 0x5f, 0xb9, 0xc4, 0x70, 0x31, 0x04, 0x5a, 0x87, 0xdb, 0xf3, 0x40, 0x75, 0x3a, 0xc3, + 0xa3, 0x3e, 0xa3, 0xf3, 0x40, 0x3e, 0xc9, 0xdc, 0x8d, 0x44, 0x71, 0x2a, 0x49, 0xcb, 0x83, 0xd8, + 0x50, 0xa4, 0x8c, 0x5a, 0xa0, 0xb3, 0x45, 0x88, 0xad, 0x82, 0xf8, 0xa4, 0x4f, 0xae, 0xf0, 0x49, + 0x07, 0x8b, 0x10, 0x2f, 0x0f, 0xef, 0x45, 0x88, 0x9f, 0xe8, 0x86, 0x66, 0xa6, 0xed, 0x47, 0x60, + 0xc4, 0xb9, 0xa3, 0x22, 0xe4, 0x9f, 0xee, 0x6c, 0xee, 0xec, 0x7e, 0xb5, 0x63, 0xa6, 0x50, 0x09, + 0x0c, 0xd7, 0xe9, 0xec, 0x3e, 0x73, 0xdc, 0xaf, 0x4d, 0x0d, 0x95, 0xa1, 0xe0, 0x3a, 0xed, 0xd6, + 0x56, 0x6b, 0xa7, 0xe3, 0x98, 0x69, 0xdb, 0x02, 0x23, 0x4e, 0x91, 0x1b, 0x6e, 0x3e, 0xeb, 0xb7, + 0x5b, 0x07, 0x9d, 0x0d, 0x33, 0x65, 0xdf, 0x01, 0x9d, 0x47, 0x42, 0x06, 0xe8, 0x6e, 0x6b, 0xfd, + 0xc0, 0x4c, 0x71, 0xd4, 0x2d, 0xa7, 0xe5, 0xee, 0x38, 0xae, 0xa9, 0xd9, 0xba, 0x91, 0x36, 0xd3, + 0xf6, 0x4f, 0x1a, 0x98, 0x67, 0xdf, 0xa7, 0x16, 0xd7, 0x06, 0xe4, 0x78, 0x0d, 0xe6, 0x91, 0x98, + 0xae, 0x1b, 0x6b, 0x0f, 0xfe, 0x34, 0x29, 0xe9, 0xd4, 0xd8, 0x17, 0x1e, 0xf1, 0xc5, 0x43, 0xfa, + 0xf3, 0x83, 0x2e, 0x3e, 0x19, 0x79, 0x9f, 0x17, 0xce, 0x1d, 0x84, 0x76, 0x0f, 0x72, 0xd2, 0xef, + 0x42, 0xc6, 0xad, 0x4e, 0xc7, 0xd9, 0x3b, 0x70, 0xba, 0xa6, 0xc6, 0x55, 0xad, 0xbd, 0xbd, 0xad, + 0x9e, 0xd3, 0x35, 0xd3, 0xa8, 0x00, 0x59, 0xc7, 0x75, 0x77, 0x5d, 0x33, 0xc3, 0xad, 0xba, 0x4e, + 0x67, 0xab, 0xb7, 0xe3, 0x74, 0x4d, 0xfd, 0x89, 0x6e, 0x64, 0x4c, 0xdd, 0xee, 0xc3, 0xcd, 0x0e, + 0x09, 0x0e, 0x3b, 0x13, 0xde, 0xf3, 0x1d, 0x12, 0x30, 0xfc, 0x1d, 0x43, 0x0f, 0x01, 0xf8, 0x45, + 0xc8, 0x0b, 0x46, 0xf1, 0x2a, 0x2e, 0xb4, 0x6f, 0xaa, 0x55, 0x5c, 0xe8, 0x48, 0x4d, 0xaf, 0xeb, + 0x16, 0x94, 0x91, 0xb8, 0x68, 0xe5, 0x43, 0x6f, 0x31, 0x25, 0x9e, 0xbc, 0x4c, 0x96, 0xdc, 0xf8, + 0x75, 0xed, 0x8d, 0x06, 0x85, 0xed, 0xf9, 0x94, 0xf9, 0x7c, 0x3a, 0xd1, 0x14, 0xcc, 0xc4, 0x94, + 0xca, 0x85, 0xf1, 0xe0, 0x6a, 0xa3, 0xcc, 0x6d, 0x2b, 0xf7, 0xae, 0xb6, 0x15, 0xed, 0x54, 0x5d, + 0x7b, 0xa8, 0xa1, 0xe7, 0x50, 0xe2, 0xca, 0xb8, 0xf4, 0xc8, 0xfe, 0x78, 0xb3, 0x55, 0xee, 0x5c, + 0x81, 0x3b, 0x09, 0xdf, 0xbe, 0x7f, 0xfc, 0xb6, 0x9a, 0x3a, 0x3e, 0xad, 0x6a, 0xaf, 0x4f, 0xab, + 0xda, 0x9b, 0xd3, 0xaa, 0xf6, 0xdb, 0x69, 0x55, 0xfb, 0xfe, 0x5d, 0x35, 0xf5, 0xfa, 0x5d, 0x35, + 0xf5, 0xe6, 0x5d, 0x35, 0xf5, 0x4d, 0x5e, 0x21, 0xfc, 0x1e, 0x00, 0x00, 0xff, 0xff, 0x19, 0x0e, + 0xbc, 0x77, 0x92, 0x0c, 0x00, 0x00, } diff --git a/pkg/storage/raft.proto b/pkg/storage/raft.proto index 3cad2bc76021..b22c0169645a 100644 --- a/pkg/storage/raft.proto +++ b/pkg/storage/raft.proto @@ -131,7 +131,7 @@ message SnapshotRequest { enum Type { RAFT = 0; LEARNER = 1; - PREEMPTIVE = 2; + reserved 2; } message Header { diff --git a/pkg/storage/replica.go b/pkg/storage/replica.go index 1023d5b3bba8..dff7799f106b 100644 --- a/pkg/storage/replica.go +++ b/pkg/storage/replica.go @@ -332,10 +332,7 @@ type Replica struct { // the raftMu. proposals map[storagebase.CmdIDKey]*ProposalData internalRaftGroup *raft.RawNode - // The ID of the replica within the Raft group. May be 0 if the replica has - // been created from a preemptive snapshot (i.e. before being added to the - // Raft group). The replica ID will be non-zero whenever the replica is - // part of a Raft group. + // The ID of the replica within the Raft group. This value may never be 0. replicaID roachpb.ReplicaID // The minimum allowed ID for this replica. Initialized from // RaftTombstone.NextReplicaID. @@ -528,17 +525,6 @@ var _ KeyRange = &Replica{} var _ client.Sender = &Replica{} -// NewReplica initializes the replica using the given metadata. If the -// replica is initialized (i.e. desc contains more than a RangeID), -// replicaID should be 0 and the replicaID will be discovered from the -// descriptor. -func NewReplica( - desc *roachpb.RangeDescriptor, store *Store, replicaID roachpb.ReplicaID, -) (*Replica, error) { - r := newReplica(desc.RangeID, store) - return r, r.init(desc, replicaID) -} - // String returns the string representation of the replica using an // inconsistent copy of the range descriptor. Therefore, String does not // require a lock and its output may not be atomic with other ongoing work in @@ -555,18 +541,6 @@ func (r *Replica) ReplicaID() roachpb.ReplicaID { return r.mu.replicaID } -// minReplicaID returns the minimum replica ID this replica could ever possibly -// have. If this replica currently knows its replica ID (i.e. ReplicaID() is -// non-zero) then it returns it. Otherwise it returns r.mu.tombstoneMinReplicaID. -func (r *Replica) minReplicaID() roachpb.ReplicaID { - r.mu.RLock() - defer r.mu.RUnlock() - if r.mu.replicaID != 0 { - return r.mu.replicaID - } - return r.mu.tombstoneMinReplicaID -} - // cleanupFailedProposal cleans up after a proposal that has failed. It // clears any references to the proposal and releases associated quota. // It requires that both Replica.mu and Replica.raftMu are exclusively held. @@ -1132,7 +1106,7 @@ func (r *Replica) isNewerThanSplit(split *roachpb.SplitTrigger) bool { } func (r *Replica) isNewerThanSplitRLocked(split *roachpb.SplitTrigger) bool { - rightDesc, hasRightDesc := split.RightDesc.GetReplicaDescriptor(r.StoreID()) + rightDesc, _ := split.RightDesc.GetReplicaDescriptor(r.StoreID()) // If we have written a tombstone for this range then we know that the RHS // must have already been removed at the split replica ID. return r.mu.tombstoneMinReplicaID != 0 || @@ -1140,11 +1114,7 @@ func (r *Replica) isNewerThanSplitRLocked(split *roachpb.SplitTrigger) bool { // ID which is above the replica ID of the split then we would not have // written a tombstone but we will have a replica ID that will exceed the // split replica ID. - (r.mu.replicaID > rightDesc.ReplicaID && - // If we're catching up from a preemptive snapshot we won't be in the split. - // and we won't know whether our current replica ID indicates we've been - // removed. - hasRightDesc) + r.mu.replicaID > rightDesc.ReplicaID } // endCmds holds necessary information to end a batch after Raft diff --git a/pkg/storage/replica_application_result.go b/pkg/storage/replica_application_result.go index 4f947be17772..ba6a1aae58dd 100644 --- a/pkg/storage/replica_application_result.go +++ b/pkg/storage/replica_application_result.go @@ -253,7 +253,7 @@ func (r *Replica) handleMergeResult(ctx context.Context, merge *storagepb.Merge) } func (r *Replica) handleDescResult(ctx context.Context, desc *roachpb.RangeDescriptor) { - r.setDesc(ctx, desc) + r.setDescRaftMuLocked(ctx, desc) } func (r *Replica) handleLeaseResult(ctx context.Context, lease *roachpb.Lease) { diff --git a/pkg/storage/replica_command.go b/pkg/storage/replica_command.go index b4200098ac4a..14cc322e9de5 100644 --- a/pkg/storage/replica_command.go +++ b/pkg/storage/replica_command.go @@ -847,8 +847,7 @@ func (s *snapshotError) Error() string { return fmt.Sprintf("snapshot failed: %s", s.cause.Error()) } -// IsSnapshotError returns true iff the error indicates a preemptive -// snapshot failed. +// IsSnapshotError returns true iff the error indicates a snapshot failed. func IsSnapshotError(err error) bool { return causer.Visit(err, func(err error) bool { _, ok := errors.Cause(err).(*snapshotError) @@ -994,26 +993,6 @@ func (r *Replica) changeReplicasImpl( return nil, err } - settings := r.ClusterSettings() - if useLearners := cluster.Version.IsActive( - ctx, settings, cluster.VersionLearnerReplicas, - ); !useLearners { - // NB: we will never use atomic replication changes while learners are not - // also active. - if len(chgs) != 1 { - return nil, errors.Errorf("need exactly one change, got %+v", chgs) - } - switch chgs[0].ChangeType { - case roachpb.ADD_REPLICA: - return r.addReplicaLegacyPreemptiveSnapshot(ctx, chgs[0].Target, desc, priority, reason, details) - case roachpb.REMOVE_REPLICA: - // We're removing a single voter. - return r.atomicReplicationChange(ctx, desc, priority, reason, details, chgs) - default: - return nil, errors.Errorf("unknown change type %d", chgs[0].ChangeType) - } - } - if adds := chgs.Additions(); len(adds) > 0 { // Lock learner snapshots even before we run the ConfChange txn to add them // to prevent a race with the raft snapshot queue trying to send it first. @@ -1387,75 +1366,10 @@ func (r *Replica) tryRollBackLearnerReplica( } } -func (r *Replica) addReplicaLegacyPreemptiveSnapshot( - ctx context.Context, - target roachpb.ReplicationTarget, - desc *roachpb.RangeDescriptor, - priority SnapshotRequest_Priority, - reason storagepb.RangeLogEventReason, - details string, -) (*roachpb.RangeDescriptor, error) { - if desc == nil { - return nil, errors.Errorf("%s: the current RangeDescriptor must not be nil", r) - } - - nodeUsed := false // tracks NodeID only - for _, existingRep := range desc.Replicas().All() { - nodeUsedByExistingRep := existingRep.NodeID == target.NodeID - nodeUsed = nodeUsed || nodeUsedByExistingRep - } - - // If the replica exists on the remote node, no matter in which store, - // abort the replica add. - if nodeUsed { - return nil, errors.Errorf("%s: unable to add replica %v; node already has a replica", r, target) - } - - // Send a pre-emptive snapshot. Note that the replica to which this - // snapshot is addressed has not yet had its replica ID initialized; this - // is intentional, and serves to avoid the following race with the replica - // GC queue: - // - // - snapshot received, a replica is lazily created with the "real" replica ID - // - the replica is eligible for GC because it is not yet a member of the range - // - GC queue runs, creating a raft tombstone with the replica's ID - // - the replica is added to the range - // - lazy creation of the replica fails due to the raft tombstone - // - // Instead, the replica GC queue will create a tombstone with replica ID - // zero, which is never legitimately used, and thus never interferes with - // raft operations. Racing with the replica GC queue can still partially - // negate the benefits of pre-emptive snapshots, but that is a recoverable - // degradation, not a catastrophic failure. - // - // NB: A closure is used here so that we can release the snapshot as soon - // as it has been applied on the remote and before the ChangeReplica - // operation is processed. This is important to allow other ranges to make - // progress which might be required for this ChangeReplicas operation to - // complete. See #10409. - { - preemptiveRepDesc := roachpb.ReplicaDescriptor{ - NodeID: target.NodeID, - StoreID: target.StoreID, - // NB: if we're still sending preemptive snapshot, the recipient is - // very likely a 19.1 node and does not understand this field. It - // won't matter to set it here, but don't anyway. - Type: nil, - ReplicaID: 0, // intentional - } - if err := r.sendSnapshot(ctx, preemptiveRepDesc, SnapshotRequest_PREEMPTIVE, priority); err != nil { - return nil, err - } - } - - iChgs := []internalReplicationChange{{target: target, typ: internalChangeTypeAddVoterViaPreemptiveSnap}} - return execChangeReplicasTxn(ctx, r.store, desc, reason, details, iChgs) -} - type internalChangeType byte const ( - internalChangeTypeAddVoterViaPreemptiveSnap internalChangeType = iota + 1 + _ internalChangeType = iota + 1 internalChangeTypeAddLearner internalChangeTypePromoteLearner // internalChangeTypeDemote changes a voter to a learner. This will @@ -1517,10 +1431,6 @@ func prepareChangeReplicasTrigger( } for _, chg := range chgs { switch chg.typ { - case internalChangeTypeAddVoterViaPreemptiveSnap: - // Legacy code. - added = append(added, - updatedDesc.AddReplica(chg.target.NodeID, chg.target.StoreID, roachpb.VOTER_FULL)) case internalChangeTypeAddLearner: added = append(added, updatedDesc.AddReplica(chg.target.NodeID, chg.target.StoreID, roachpb.LEARNER)) @@ -1782,7 +1692,7 @@ func execChangeReplicasTxn( // ConfChange. It then uses the normal mechanisms to catch up with whatever got // committed to the Raft log during the snapshot transfer. In contrast to adding // the voting replica directly, this avoids a period of fragility when the -// replica would be a full member, but very far behind. [1] +// replica would be a full member, but very far behind. // // Snapshots are expensive and mostly unexpected (except learner snapshots // during rebalancing). The quota pool is responsible for keeping a leader from @@ -1816,7 +1726,7 @@ func execChangeReplicasTxn( // into `receiveSnapshot`, which does the bulk of the work. `receiveSnapshot` // starts by waiting for a reservation in the snapshot rate limiter. It then // reads the header message and hands it to `shouldAcceptSnapshotData` to -// determine if it can use the snapshot [2]. `shouldAcceptSnapshotData` is +// determine if it can use the snapshot [1]. `shouldAcceptSnapshotData` is // advisory and can return false positives. If `shouldAcceptSnapshotData` // returns true, this is communicated back to the sender, which then proceeds to // call `kvBatchSnapshotStrategy.Send`. This uses the iterator captured earlier @@ -1837,7 +1747,7 @@ func execChangeReplicasTxn( // `(Store).processRaftSnapshotRequest` to be applied. First, this re-checks // the same things as `shouldAcceptSnapshotData` to make sure nothing has // changed while the snapshot was being transferred. It then guarantees that -// there is either an initialized[3] replica or a `ReplicaPlaceholder`[4] to +// there is either an initialized[2] replica or a `ReplicaPlaceholder`[3] to // accept the snapshot by creating a placeholder if necessary. Finally, a *Raft // snapshot* message is manually handed to the replica's Raft node (by calling // `stepRaftGroup` + `handleRaftReadyRaftMuLocked`). During the application @@ -1850,27 +1760,19 @@ func execChangeReplicasTxn( // number of subsumed replicas. In the case where there are no subsumed // replicas, 4 SSTs will be created. // -// [1]: There is a third kind of snapshot, called "preemptive", which is how we -// avoided the above fragility before learner replicas were introduced in the -// 19.2 cycle. It's essentially a snapshot that we made very fast by staging it -// on a remote node right before we added a replica on that node. However, -// preemptive snapshots came with all sorts of complexity that we're delighted -// to be rid of. They have to stay around for clusters with mixed 19.1 and 19.2 -// nodes, but after 19.2, we can remove them entirely. -// -// [2]: The largest class of rejections here is if the store contains a replica +// [1]: The largest class of rejections here is if the store contains a replica // that overlaps the snapshot but has a different id (we maintain an invariant // that replicas on a store never overlap). This usually happens when the // recipient has an old copy of a replica that is no longer part of a range and // the `replicaGCQueue` hasn't gotten around to collecting it yet. So if this // happens, `shouldAcceptSnapshotData` will queue it up for consideration. // -// [3]: A uninitialized replica is created when a replica that's being added +// [2]: A uninitialized replica is created when a replica that's being added // gets traffic from its new peers before it gets a snapshot. It may be possible // to get rid of uninitialized replicas (by dropping all Raft traffic except // votes on the floor), but this is a cleanup that hasn't happened yet. // -// [4]: The placeholder is essentially a snapshot lock, making any future +// [3]: The placeholder is essentially a snapshot lock, making any future // callers of `shouldAcceptSnapshotData` return an error so that we no longer // have to worry about racing with a second snapshot. See the comment on // ReplicaPlaceholder for details. @@ -1881,11 +1783,9 @@ func (r *Replica) sendSnapshot( priority SnapshotRequest_Priority, ) (retErr error) { defer func() { - if snapType != SnapshotRequest_PREEMPTIVE { - // Report the snapshot status to Raft, which expects us to do this once we - // finish sending the snapshot. - r.reportSnapshotStatus(ctx, recipient.ReplicaID, retErr) - } + // Report the snapshot status to Raft, which expects us to do this once we + // finish sending the snapshot. + r.reportSnapshotStatus(ctx, recipient.ReplicaID, retErr) }() snap, err := r.GetSnapshot(ctx, snapType, recipient.StoreID) @@ -1956,8 +1856,11 @@ func (r *Replica) sendSnapshot( }, }, RangeSize: r.GetMVCCStats().Total(), - // Recipients can choose to decline preemptive snapshots. - CanDecline: snapType == SnapshotRequest_PREEMPTIVE, + // Recipients currently cannot choose to decline any snapshots. + // In 19.2 and earlier versions pre-emptive snapshots could be declined. + // + // TODO(ajwerner): Consider removing the CanDecline flag. + CanDecline: false, Priority: priority, Strategy: SnapshotRequest_KV_BATCH, Type: snapType, diff --git a/pkg/storage/replica_destroy.go b/pkg/storage/replica_destroy.go index 3f9d8309ac09..821c73148c71 100644 --- a/pkg/storage/replica_destroy.go +++ b/pkg/storage/replica_destroy.go @@ -23,6 +23,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/timeutil" + "github.com/pkg/errors" ) // DestroyReason indicates if a replica is alive, destroyed, corrupted or pending destruction. @@ -63,6 +64,58 @@ func (s destroyStatus) Removed() bool { return s.reason == destroyReasonRemoved } +// removePreemptiveSnapshot is a migration put in place during the 20.1 release +// to remove any on-disk preemptive snapshots which may still be resident on a +// 19.2 node's store due to a rapid upgrade. +// +// As of 19.2, a range can never process commands while it is not part of a +// range. It is still possible that on-disk replica state exists which does not +// contain this Store exist from when this store was running 19.1 and was not +// removed during 19.2. For the sake of this method and elsewhere we'll define +// a preemptive snapshot as on-disk data corresponding to an initialized range +// which has a range descriptor which does not contain this store. +// +// In 19.1 when a Replica processes a command which removes it from the range +// it does not immediately destroy its data. In fact it just assumed that it +// would not be getting more commands appended to its log. In some cases a +// Replica would continue to apply commands even while it was not a part of the +// range. This was unfortunate as it is not possible to uphold certain invariants +// for stores which are not a part of a range when it processes a command. Not +// only did the above behavior of processing commands regardless of whether the +// current store was in the range wasn't just happenstance, it was a fundamental +// property that was relied upon for the change replicas protocol. In 19.2 we +// adopt learner Replicas which are added to the raft group as a non-voting +// member before receiving a snapshot of the state. In all previous versions +// we did not have voters. The legacy protocol instead relied on sending a +// snapshot of raft state and then the Replica had to apply log entries up to +// the point where it was added to the range. +// +// TODO(ajwerner): Remove during 20.2. +func removePreemptiveSnapshot( + ctx context.Context, s *Store, desc *roachpb.RangeDescriptor, +) (err error) { + batch := s.Engine().NewWriteOnlyBatch() + defer batch.Close() + const rangeIDLocalOnly = false + const mustClearRange = false + defer func() { + if err != nil { + err = errors.Wrapf(err, "failed to remove preemptive snapshot for range %v", desc) + } + }() + if err := clearRangeData(desc, s.Engine(), batch, rangeIDLocalOnly, mustClearRange); err != nil { + return err + } + if err := writeTombstoneKey(ctx, batch, desc.RangeID, desc.NextReplicaID); err != nil { + return err + } + if err := batch.Commit(true); err != nil { + return err + } + log.Infof(ctx, "removed preemptive snapshot for %v", desc) + return nil +} + // mergedTombstoneReplicaID is the replica ID written into the tombstone // for replicas which are part of a range which is known to have been merged. // This value should prevent any messages from stale replicas of that range from @@ -222,8 +275,16 @@ func (r *Replica) setTombstoneKey( r.mu.tombstoneMinReplicaID = nextReplicaID } r.mu.Unlock() + return writeTombstoneKey(ctx, writer, r.RangeID, nextReplicaID) +} - tombstoneKey := keys.RaftTombstoneKey(r.RangeID) +func writeTombstoneKey( + ctx context.Context, + writer engine.Writer, + rangeID roachpb.RangeID, + nextReplicaID roachpb.ReplicaID, +) error { + tombstoneKey := keys.RaftTombstoneKey(rangeID) tombstone := &roachpb.RaftTombstone{ NextReplicaID: nextReplicaID, } diff --git a/pkg/storage/replica_gc_queue.go b/pkg/storage/replica_gc_queue.go index 229838c9affb..5bf53884d2cb 100644 --- a/pkg/storage/replica_gc_queue.go +++ b/pkg/storage/replica_gc_queue.go @@ -235,11 +235,6 @@ func (rgcq *replicaGCQueue) process( } replyDesc := rs[0] - repl.mu.RLock() - replicaID := repl.mu.replicaID - ticks := repl.mu.ticks - repl.mu.RUnlock() - // Now check whether the replica is meant to still exist. // Maybe it was deleted "under us" by being moved. currentDesc, currentMember := replyDesc.GetReplicaDescriptor(repl.store.StoreID()) @@ -268,23 +263,7 @@ func (rgcq *replicaGCQueue) process( // We are no longer a member of this range, but the range still exists. // Clean up our local data. - if replicaID == 0 { - // This is a preemptive replica. GC'ing a preemptive replica is a - // good idea if and only if the up-replication that it was a part of - // did *NOT* commit. If it *did* commit and we're removing the - // preemptive snapshot, the newly added follower will first need a - // Raft snapshot to catch up, and that snapshot will be delayed by - // #31875. - // Log if the replica hasn't been around for very long. - // - // TODO(tschottdorf): avoid these, ideally without a time-based mechanism. - // The replica carrying out the replication change could keep the - // snapshot alive until it has either committed or aborted the txn. - // Or we try to use Raft learners for this purpose. - if ticks < 10 { - log.Infof(ctx, "removing young preemptive snapshot (%d ticks)", ticks) - } - } else if replyDesc.EndKey.Less(desc.EndKey) { + if replyDesc.EndKey.Less(desc.EndKey) { // The meta records indicate that the range has split but that this // replica hasn't processed the split trigger yet. By removing this // replica, we're also wiping out the data of what would become the diff --git a/pkg/storage/replica_init.go b/pkg/storage/replica_init.go index de5f069422da..d393dd1630d6 100644 --- a/pkg/storage/replica_init.go +++ b/pkg/storage/replica_init.go @@ -36,21 +36,54 @@ const ( mergeQueueThrottleDuration = 5 * time.Second ) -func newReplica(rangeID roachpb.RangeID, store *Store) *Replica { +// newReplica constructs a new Replica. If the desc is initialized, the store +// must be present in it and the corresponding replica descriptor must have +// replicaID as its ReplicaID. +func newReplica( + ctx context.Context, desc *roachpb.RangeDescriptor, store *Store, replicaID roachpb.ReplicaID, +) (*Replica, error) { + repl := newUnloadedReplica(ctx, desc, store, replicaID) + repl.raftMu.Lock() + defer repl.raftMu.Unlock() + repl.mu.Lock() + defer repl.mu.Unlock() + if err := repl.loadRaftMuLockedReplicaMuLocked(desc); err != nil { + return nil, err + } + return repl, nil +} + +// newUnloadedReplica partially constructs a replica. The primary reason this +// function exists separately from Replica.loadRaftMuLockedReplicaMuLocked() is +// to avoid attempting to fully constructing a Replica prior to proving that it +// can exist during the delicate synchronization dance that occurs in +// Store.tryGetOrCreateReplica(). A Replica returned from this function must not +// be used in any way until it's load() method has been called. +func newUnloadedReplica( + ctx context.Context, desc *roachpb.RangeDescriptor, store *Store, replicaID roachpb.ReplicaID, +) *Replica { + if replicaID == 0 { + log.Fatalf(context.TODO(), "cannot construct a replica for range %d with a 0 replica ID", desc.RangeID) + } r := &Replica{ AmbientContext: store.cfg.AmbientCtx, - RangeID: rangeID, + RangeID: desc.RangeID, store: store, - abortSpan: abortspan.New(rangeID), + abortSpan: abortspan.New(desc.RangeID), txnWaitQueue: txnwait.NewQueue(store), } r.mu.pendingLeaseRequest = makePendingLeaseRequest(r) - r.mu.stateLoader = stateloader.Make(rangeID) + r.mu.stateLoader = stateloader.Make(desc.RangeID) r.mu.quiescent = true r.mu.zone = store.cfg.DefaultZoneConfig + r.mu.replicaID = replicaID split.Init(&r.loadBasedSplitter, rand.Intn, func() float64 { return float64(SplitByLoadQPSThreshold.Get(&store.cfg.Settings.SV)) }) + r.latchMgr = spanlatch.Make(r.store.stopper, r.store.metrics.SlowLatchRequests) + r.mu.proposals = map[storagebase.CmdIDKey]*ProposalData{} + r.mu.checksums = map[uuid.UUID]ReplicaChecksum{} + r.mu.proposalBuf.Init((*replicaProposer)(r)) if leaseHistoryMaxEntries > 0 { r.leaseHistory = newLeaseHistory() @@ -63,51 +96,69 @@ func newReplica(rangeID roachpb.RangeID, store *Store) *Replica { r.writeStats = newReplicaStats(store.Clock(), nil) // Init rangeStr with the range ID. - r.rangeStr.store(0, &roachpb.RangeDescriptor{RangeID: rangeID}) + r.rangeStr.store(replicaID, &roachpb.RangeDescriptor{RangeID: desc.RangeID}) // Add replica log tag - the value is rangeStr.String(). r.AmbientContext.AddLogTag("r", &r.rangeStr) // Add replica pointer value. NB: this was historically useful for debugging // replica GC issues, but is a distraction at the moment. // r.AmbientContext.AddLogTag("@", fmt.Sprintf("%x", unsafe.Pointer(r))) - r.raftMu.stateLoader = stateloader.Make(rangeID) + r.raftMu.stateLoader = stateloader.Make(desc.RangeID) r.splitQueueThrottle = util.Every(splitQueueThrottleDuration) r.mergeQueueThrottle = util.Every(mergeQueueThrottleDuration) return r } -func (r *Replica) init(desc *roachpb.RangeDescriptor, replicaID roachpb.ReplicaID) error { - r.raftMu.Lock() - defer r.raftMu.Unlock() - r.mu.Lock() - defer r.mu.Unlock() - return r.initRaftMuLockedReplicaMuLocked(desc, replicaID) -} - -func (r *Replica) initRaftMuLockedReplicaMuLocked( - desc *roachpb.RangeDescriptor, replicaID roachpb.ReplicaID, -) error { +// loadRaftMuLockedReplicaMuLocked will load the state of the replica from disk. +// If desc is initialized, the Replica will be initialized when this method +// returns. An initialized Replica may not be reloaded. If this method is called +// with an uninitialized desc it may be called again later with an initialized +// desc. +// +// This method is called in three places: +// +// 1) newReplica - used when the store is initializing and during testing +// 2) tryGetOrCreateReplica - see newUnloadedReplica +// 3) splitPostApply - this call initializes a previously uninitialized Replica. +// +func (r *Replica) loadRaftMuLockedReplicaMuLocked(desc *roachpb.RangeDescriptor) error { ctx := r.AnnotateCtx(context.TODO()) if r.mu.state.Desc != nil && r.isInitializedRLocked() { log.Fatalf(ctx, "r%d: cannot reinitialize an initialized replica", desc.RangeID) - } - if desc.IsInitialized() && replicaID != 0 { - return errors.Errorf("replicaID must be 0 when creating an initialized replica") + } else if r.mu.replicaID == 0 { + // NB: This is just a defensive check as r.mu.replicaID should never be 0. + log.Fatalf(ctx, "r%d: cannot initialize replica without a replicaID", desc.RangeID) } - r.latchMgr = spanlatch.Make(r.store.stopper, r.store.metrics.SlowLatchRequests) - r.mu.proposals = map[storagebase.CmdIDKey]*ProposalData{} - r.mu.checksums = map[uuid.UUID]ReplicaChecksum{} // Clear the internal raft group in case we're being reset. Since we're // reloading the raft state below, it isn't safe to use the existing raft // group. r.mu.internalRaftGroup = nil - r.mu.proposalBuf.Init((*replicaProposer)(r)) var err error if r.mu.state, err = r.mu.stateLoader.Load(ctx, r.Engine(), desc); err != nil { return err } + r.mu.lastIndex, err = r.mu.stateLoader.LoadLastIndex(ctx, r.Engine()) + if err != nil { + return err + } + r.mu.lastTerm = invalidLastTerm + + // Ensure that we're not trying to load a replica with a different ID than + // was used to construct this Replica. + replicaID := r.mu.replicaID + if replicaDesc, found := r.mu.state.Desc.GetReplicaDescriptor(r.StoreID()); found { + replicaID = replicaDesc.ReplicaID + } else if desc.IsInitialized() { + log.Fatalf(ctx, "r%d: cannot initialize replica which is not in descriptor %v", desc.RangeID, desc) + } + if r.mu.replicaID != replicaID { + log.Fatalf(ctx, "attempting to initialize a replica which has ID %d with ID %d", + r.mu.replicaID, replicaID) + } + + r.setDescLockedRaftMuLocked(ctx, desc) // Init the minLeaseProposedTS such that we won't use an existing lease (if // any). This is so that, after a restart, we don't propose under old leases. @@ -127,72 +178,10 @@ func (r *Replica) initRaftMuLockedReplicaMuLocked( } } - r.rangeStr.store(0, r.mu.state.Desc) - - r.mu.lastIndex, err = r.mu.stateLoader.LoadLastIndex(ctx, r.Engine()) - if err != nil { - return err - } - r.mu.lastTerm = invalidLastTerm - - if replicaID == 0 { - repDesc, ok := desc.GetReplicaDescriptor(r.store.StoreID()) - if !ok { - // This is intentionally not an error and is the code path exercised - // during preemptive snapshots. The replica ID will be sent when the - // actual raft replica change occurs. - return nil - } - replicaID = repDesc.ReplicaID - } - r.rangeStr.store(replicaID, r.mu.state.Desc) - r.connectionClass.set(rpc.ConnectionClassForKey(desc.StartKey)) - if r.mu.replicaID == 0 { - if err := r.setReplicaIDRaftMuLockedMuLocked(ctx, replicaID); err != nil { - return err - } - } else if r.mu.replicaID != replicaID { - log.Fatalf(ctx, "attempting to initialize a replica which has ID %d with ID %d", - r.mu.replicaID, replicaID) - } - r.assertStateLocked(ctx, r.store.Engine()) - return nil -} - -func (r *Replica) setReplicaIDRaftMuLockedMuLocked( - ctx context.Context, replicaID roachpb.ReplicaID, -) error { - if r.mu.replicaID != 0 { - log.Fatalf(ctx, "cannot set replica ID from anything other than 0, currently %d", - r.mu.replicaID) - } else if replicaID == 0 { - log.Fatalf(ctx, "cannot set replica ID from anything to 0: %v", r) - } - if replicaID < r.mu.tombstoneMinReplicaID { - return &roachpb.RaftGroupDeletedError{} - } - if r.mu.replicaID > replicaID { - return errors.Errorf("replicaID cannot move backwards from %d to %d", r.mu.replicaID, replicaID) - } - if r.mu.destroyStatus.Removed() { - // This replica has been marked for removal and we're trying to resurrect it. - log.Fatalf(ctx, "cannot resurect replica %d", r.mu.replicaID) - } - - // Initialize or update the sideloaded storage. If the sideloaded storage - // already exists (which is iff the previous replicaID was non-zero), then - // we have to move the contained files over (this corresponds to the case in - // which our replica is removed and re-added to the range, without having - // the replica GC'ed in the meantime). - // - // Note that we can't race with a concurrent replicaGC here because both that - // and this is under raftMu. ssBase := r.Engine().GetAuxiliaryDir() - rangeID := r.mu.state.Desc.RangeID - var err error if r.raftMu.sideloaded, err = newDiskSideloadStorage( r.store.cfg.Settings, - rangeID, + desc.RangeID, replicaID, ssBase, r.store.limiters.BulkIOWriteRate, @@ -200,15 +189,7 @@ func (r *Replica) setReplicaIDRaftMuLockedMuLocked( ); err != nil { return errors.Wrap(err, "while initializing sideloaded storage") } - - r.mu.replicaID = replicaID - - // Sanity check that we do not already have a raft group as we did not - // know our replica ID before this call. - if r.mu.internalRaftGroup != nil { - log.Fatalf(ctx, "somehow had an initialized raft group on a zero valued replica") - } - + r.assertStateLocked(ctx, r.store.Engine()) return nil } @@ -263,27 +244,47 @@ func (r *Replica) maybeInitializeRaftGroup(ctx context.Context) { } } -// setDesc atomically sets the replica's descriptor. It requires raftMu to be +// setDescRaftMuLocked atomically sets the replica's descriptor. It requires raftMu to be // locked. -func (r *Replica) setDesc(ctx context.Context, desc *roachpb.RangeDescriptor) { +func (r *Replica) setDescRaftMuLocked(ctx context.Context, desc *roachpb.RangeDescriptor) { r.mu.Lock() defer r.mu.Unlock() + r.setDescLockedRaftMuLocked(ctx, desc) +} +func (r *Replica) setDescLockedRaftMuLocked(ctx context.Context, desc *roachpb.RangeDescriptor) { if desc.RangeID != r.RangeID { log.Fatalf(ctx, "range descriptor ID (%d) does not match replica's range ID (%d)", desc.RangeID, r.RangeID) } - if r.mu.state.Desc != nil && r.mu.state.Desc.IsInitialized() && + if r.mu.state.Desc.IsInitialized() && (desc == nil || !desc.IsInitialized()) { log.Fatalf(ctx, "cannot replace initialized descriptor with uninitialized one: %+v -> %+v", r.mu.state.Desc, desc) } - if r.mu.state.Desc != nil && r.mu.state.Desc.IsInitialized() && + if r.mu.state.Desc.IsInitialized() && !r.mu.state.Desc.StartKey.Equal(desc.StartKey) { log.Fatalf(ctx, "attempted to change replica's start key from %s to %s", r.mu.state.Desc.StartKey, desc.StartKey) } + // NB: It might be nice to assert that the current replica exists in desc + // however we allow it to not be present for three reasons: + // + // 1) When removing the current replica we update the descriptor to the point + // of removal even though we will delete the Replica's data in the same + // batch. We could avoid setting the local descriptor in this case. + // 2) When the DisableEagerReplicaRemoval testing knob is enabled. We + // could remove all tests which utilize this behavior now that there's + // no other mechanism for range state which does not contain the current + // store to exist on disk. + // 3) Various unit tests do not provide a valid descriptor. + replDesc, found := desc.GetReplicaDescriptor(r.StoreID()) + if found && replDesc.ReplicaID != r.mu.replicaID { + log.Fatalf(ctx, "attempted to change replica's ID from %d to %d", + r.mu.replicaID, replDesc.ReplicaID) + } + // Determine if a new replica was added. This is true if the new max replica // ID is greater than the old max replica ID. oldMaxID := maxReplicaIDOfAny(r.mu.state.Desc) diff --git a/pkg/storage/replica_init_test.go b/pkg/storage/replica_init_test.go index 65191d3243af..35db94eaadda 100644 --- a/pkg/storage/replica_init_test.go +++ b/pkg/storage/replica_init_test.go @@ -16,6 +16,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/util/leaktest" + "github.com/cockroachdb/cockroach/pkg/util/stop" ) func TestReplicaUpdateLastReplicaAdded(t *testing.T) { @@ -50,12 +51,19 @@ func TestReplicaUpdateLastReplicaAdded(t *testing.T) { {desc(1, 2, 3), desc(1, 3), 3, 3}, {desc(1, 2, 3), desc(1, 2), 3, 0}, } + + tc := testContext{} + stopper := stop.NewStopper() + ctx := context.Background() + defer stopper.Stop(ctx) + tc.Start(t, stopper) for _, c := range testCases { t.Run("", func(t *testing.T) { var r Replica r.mu.state.Desc = &c.oldDesc r.mu.lastReplicaAdded = c.lastReplicaAdded - r.setDesc(context.Background(), &c.newDesc) + r.store = tc.store + r.setDescRaftMuLocked(context.Background(), &c.newDesc) if c.expectedLastReplicaAdded != r.mu.lastReplicaAdded { t.Fatalf("expected %d, but found %d", c.expectedLastReplicaAdded, r.mu.lastReplicaAdded) diff --git a/pkg/storage/replica_proposal_quota.go b/pkg/storage/replica_proposal_quota.go index 4a6e8ff9c975..f50887bb7b6f 100644 --- a/pkg/storage/replica_proposal_quota.go +++ b/pkg/storage/replica_proposal_quota.go @@ -84,12 +84,6 @@ func (r *Replica) updateProposalQuotaRaftMuLocked( r.mu.Lock() defer r.mu.Unlock() - if r.mu.replicaID == 0 { - // The replica was created from preemptive snapshot and has not been - // added to the Raft group. - return - } - status := r.mu.internalRaftGroup.BasicStatus() if r.mu.leaderID != lastLeaderID { if r.mu.replicaID == r.mu.leaderID { diff --git a/pkg/storage/replica_raft.go b/pkg/storage/replica_raft.go index 0260737f493a..1e7bc7701d2d 100644 --- a/pkg/storage/replica_raft.go +++ b/pkg/storage/replica_raft.go @@ -1086,11 +1086,7 @@ func (r *Replica) sendRaftMessage(ctx context.Context, msg raftpb.Message) { fromReplica, fromErr := r.getReplicaDescriptorByIDRLocked(roachpb.ReplicaID(msg.From), r.mu.lastToReplica) toReplica, toErr := r.getReplicaDescriptorByIDRLocked(roachpb.ReplicaID(msg.To), r.mu.lastFromReplica) var startKey roachpb.RKey - if msg.Type == raftpb.MsgHeartbeat { - if r.mu.replicaID == 0 { - log.Fatalf(ctx, "preemptive snapshot attempted to send a heartbeat: %+v", msg) - } - } else if msg.Type == raftpb.MsgApp && r.mu.internalRaftGroup != nil { + if msg.Type == raftpb.MsgApp && r.mu.internalRaftGroup != nil { // When the follower is potentially an uninitialized replica waiting for // a split trigger, send the replica's StartKey along. See the method // below for more context: @@ -1320,12 +1316,6 @@ func (r *Replica) withRaftGroupLocked( return errRemoved } - if r.mu.replicaID == 0 { - // The replica's raft group has not yet been configured (i.e. the replica - // was created from a preemptive snapshot). - return nil - } - if r.mu.internalRaftGroup == nil { ctx := r.AnnotateCtx(context.TODO()) raftGroup, err := raft.NewRawNode(newRaftConfig( @@ -1545,16 +1535,15 @@ func (r *Replica) maybeAcquireSplitMergeLock( func (r *Replica) acquireSplitLock( ctx context.Context, split *roachpb.SplitTrigger, ) (func(), error) { - // We pass a 0 replicaID because we want to lock the RHS even if we know - // it to be newer than the split so that we can properly clean up its - // state. We could imagine alternatively handling the RaftGroupDeleted - // error here and then not being guaranteed a Replica in the pre and post - // split apply hooks but that doesn't necessarily seem worth it. - const replicaID = 0 rightReplDesc, _ := split.RightDesc.GetReplicaDescriptor(r.StoreID()) rightRng, _, err := r.store.getOrCreateReplica(ctx, split.RightDesc.RangeID, - replicaID, nil, /* creatingReplica */ + rightReplDesc.ReplicaID, nil, /* creatingReplica */ rightReplDesc.GetType() == roachpb.LEARNER) + // If getOrCreateReplica returns RaftGroupDeletedError we know that the RHS + // has already been removed. This case is handled properly in splitPostApply. + if _, isRaftGroupDeletedError := err.(*roachpb.RaftGroupDeletedError); isRaftGroupDeletedError { + return func() {}, nil + } if err != nil { return nil, err } diff --git a/pkg/storage/replica_raftstorage.go b/pkg/storage/replica_raftstorage.go index 62d4512d4329..4dc71208b274 100644 --- a/pkg/storage/replica_raftstorage.go +++ b/pkg/storage/replica_raftstorage.go @@ -738,10 +738,6 @@ func (r *Replica) applySnapshot( log.Fatalf(ctx, "unexpected range ID %d", s.Desc.RangeID) } - r.mu.RLock() - replicaID := r.mu.replicaID - r.mu.RUnlock() - snapType := inSnap.snapType defer func() { if err == nil { @@ -750,8 +746,6 @@ func (r *Replica) applySnapshot( r.store.metrics.RangeSnapshotsNormalApplied.Inc(1) case SnapshotRequest_LEARNER: r.store.metrics.RangeSnapshotsLearnerApplied.Inc(1) - case SnapshotRequest_PREEMPTIVE: - r.store.metrics.RangeSnapshotsPreemptiveApplied.Inc(1) } } }() @@ -836,19 +830,13 @@ func (r *Replica) applySnapshot( return err } } - // If this replica doesn't know its ReplicaID yet, we're applying a - // preemptive snapshot. In this case, we're going to have to write the - // sideloaded proposals into the Raft log. Otherwise, sideload. - if replicaID != 0 { - var err error - var sideloadedEntriesSize int64 - logEntries, sideloadedEntriesSize, err = r.maybeSideloadEntriesRaftMuLocked(ctx, logEntries) - if err != nil { - return err - } - raftLogSize += sideloadedEntriesSize - } + var sideloadedEntriesSize int64 var err error + logEntries, sideloadedEntriesSize, err = r.maybeSideloadEntriesRaftMuLocked(ctx, logEntries) + if err != nil { + return err + } + raftLogSize += sideloadedEntriesSize _, lastTerm, raftLogSize, err = r.append(ctx, &unreplicatedSST, 0, invalidLastTerm, raftLogSize, logEntries) if err != nil { return err @@ -940,7 +928,7 @@ func (r *Replica) applySnapshot( if r.store.removePlaceholderLocked(ctx, r.RangeID) { atomic.AddInt32(&r.store.counts.filledPlaceholders, 1) } - r.setDesc(ctx, s.Desc) + r.setDescRaftMuLocked(ctx, s.Desc) if err := r.store.maybeMarkReplicaInitializedLocked(ctx, r); err != nil { log.Fatalf(ctx, "unable to mark replica initialized while applying snapshot: %+v", err) } @@ -967,7 +955,7 @@ func (r *Replica) applySnapshot( r.store.metrics.subtractMVCCStats(*r.mu.state.Stats) r.store.metrics.addMVCCStats(*s.Stats) // Update the rest of the Raft state. Changes to r.mu.state.Desc must be - // managed by r.setDesc and changes to r.mu.state.Lease must be handled + // managed by r.setDescRaftMuLocked and changes to r.mu.state.Lease must be handled // by r.leasePostApply, but we called those above, so now it's safe to // wholesale replace r.mu.state. r.mu.state = s diff --git a/pkg/storage/replica_test.go b/pkg/storage/replica_test.go index be97aa09d57f..2e74e93c6fde 100644 --- a/pkg/storage/replica_test.go +++ b/pkg/storage/replica_test.go @@ -277,7 +277,7 @@ func (tc *testContext) StartWithStoreConfigAndVersion( ); err != nil { t.Fatal(err) } - repl, err := NewReplica(testDesc, tc.store, 0) + repl, err := newReplica(ctx, testDesc, tc.store, 1) if err != nil { t.Fatal(err) } @@ -389,7 +389,7 @@ func (tc *testContext) addBogusReplicaToRangeDesc( return roachpb.ReplicaDescriptor{}, err } - tc.repl.setDesc(ctx, &newDesc) + tc.repl.setDescRaftMuLocked(ctx, &newDesc) tc.repl.raftMu.Lock() tc.repl.mu.Lock() tc.repl.assertStateLocked(ctx, tc.engine) @@ -6869,30 +6869,6 @@ func TestReplicaDestroy(t *testing.T) { t.Fatal(err) } - // First try and fail with a stale descriptor. - origDesc := repl.Desc() - newDesc := protoutil.Clone(origDesc).(*roachpb.RangeDescriptor) - for i := range newDesc.InternalReplicas { - if newDesc.InternalReplicas[i].StoreID == tc.store.StoreID() { - newDesc.InternalReplicas[i].ReplicaID++ - newDesc.NextReplicaID++ - break - } - } - - repl.setDesc(ctx, newDesc) - expectedErr := "replica descriptor's ID has changed" - func() { - tc.repl.raftMu.Lock() - defer tc.repl.raftMu.Unlock() - if err := tc.store.removeInitializedReplicaRaftMuLocked(ctx, tc.repl, origDesc.NextReplicaID, RemoveOptions{ - DestroyData: true, - }); !testutils.IsError(err, expectedErr) { - t.Fatalf("expected error %q but got %v", expectedErr, err) - } - }() - - // Now try a fresh descriptor and succeed. func() { tc.repl.raftMu.Lock() defer tc.repl.raftMu.Unlock() diff --git a/pkg/storage/split_queue_test.go b/pkg/storage/split_queue_test.go index 5df34a64e6a5..10bd29543868 100644 --- a/pkg/storage/split_queue_test.go +++ b/pkg/storage/split_queue_test.go @@ -69,15 +69,15 @@ func TestSplitQueueShouldQueue(t *testing.T) { if cfg == nil { t.Fatal("config not set") } - + ctx := context.Background() for i, test := range testCases { // Create a replica for testing that is not hooked up to the store. This // ensures that the store won't be mucking with our replica concurrently // during testing (e.g. via the system config gossip update). - copy := *tc.repl.Desc() - copy.StartKey = test.start - copy.EndKey = test.end - repl, err := NewReplica(©, tc.store, 0) + cpy := *tc.repl.Desc() + cpy.StartKey = test.start + cpy.EndKey = test.end + repl, err := newReplica(ctx, &cpy, tc.store, cpy.Replicas().Voters()[0].ReplicaID) if err != nil { t.Fatal(err) } diff --git a/pkg/storage/split_trigger_helper_test.go b/pkg/storage/split_trigger_helper_test.go index 698c7d46b4bf..c7b847b60f1d 100644 --- a/pkg/storage/split_trigger_helper_test.go +++ b/pkg/storage/split_trigger_helper_test.go @@ -47,7 +47,7 @@ func TestMaybeDropMsgApp(t *testing.T) { defer leaktest.AfterTest(t)() testCases := map[testMsgAppDropper]bool{ - // Already init'ed. + // Already load'ed. {initialized: true}: false, // Left hand side not found. {initialized: false}: false, diff --git a/pkg/storage/store.go b/pkg/storage/store.go index 121754d97d55..56beee5de077 100644 --- a/pkg/storage/store.go +++ b/pkg/storage/store.go @@ -1200,7 +1200,9 @@ func IterateIDPrefixKeys( // from the provided Engine. The return values of this method and fn have // semantics similar to engine.MVCCIterate. func IterateRangeDescriptors( - ctx context.Context, reader engine.Reader, fn func(desc roachpb.RangeDescriptor) (bool, error), + ctx context.Context, + reader engine.Reader, + fn func(desc roachpb.RangeDescriptor) (done bool, err error), ) error { log.Event(ctx, "beginning range descriptor iteration") // Iterator over all range-local key-based data. @@ -1332,8 +1334,21 @@ func (s *Store) Start(ctx context.Context, stopper *stop.Stopper) error { if !desc.IsInitialized() { return false, errors.Errorf("found uninitialized RangeDescriptor: %+v", desc) } + replicaDesc, found := desc.GetReplicaDescriptor(s.StoreID()) + if !found { + // This is a pre-emptive snapshot. It's also possible that this is a + // range which has processed a raft command to remove itself (which is + // possible prior to 19.2 or if the DisableEagerReplicaRemoval is + // enabled) and has not yet been removed by the replica gc queue. + // We treat both cases the same way. + // + // TODO(ajwerner): Remove this migration in 20.2. It exists in 20.1 to + // find and remove any pre-emptive snapshots which may have been sent by + // a 19.1 or older node to this node while it was running 19.2. + return false /* done */, removePreemptiveSnapshot(ctx, s, &desc) + } - rep, err := NewReplica(&desc, s, 0) + rep, err := newReplica(ctx, &desc, s, replicaDesc.ReplicaID) if err != nil { return false, err } diff --git a/pkg/storage/store_create_replica.go b/pkg/storage/store_create_replica.go index 99928da434cd..024aaff76dc5 100644 --- a/pkg/storage/store_create_replica.go +++ b/pkg/storage/store_create_replica.go @@ -37,6 +37,9 @@ func (s *Store) getOrCreateReplica( creatingReplica *roachpb.ReplicaDescriptor, isLearner bool, ) (_ *Replica, created bool, _ error) { + if replicaID == 0 { + log.Fatalf(ctx, "cannot construct a Replica for range %d with 0 id", rangeID) + } // We need a retry loop as the replica we find in the map may be in the // process of being removed or may need to be removed. Retries in the loop // imply that a removal is actually being carried out, not that we're waiting @@ -99,23 +102,11 @@ func (s *Store) tryGetOrCreateReplica( return nil, false, roachpb.NewReplicaTooOldError(creatingReplica.ReplicaID) } - toTooOld := toReplicaIsTooOld(repl, replicaID) - isPreemptiveSnapshot := repl.mu.replicaID == 0 && repl.isInitializedRLocked() - // We need to remove preemptive snapshots when we determine that we're now a - // learner. Otherwise we risk appending and applying log entries while we're - // not a member of the range and potentially applying a merge which would be - // unsafe. See the comment in Replica.acquireMergeLock(). - removePreemptiveSnapshot := isPreemptiveSnapshot && isLearner // The current replica needs to be removed, remove it and go back around. - if toTooOld || removePreemptiveSnapshot { - - if shouldLog := log.V(1); shouldLog && toTooOld { + if toTooOld := repl.mu.replicaID < replicaID; toTooOld { + if shouldLog := log.V(1); shouldLog { log.Infof(ctx, "found message for replica ID %d which is newer than %v", replicaID, repl) - } else if shouldLog && removePreemptiveSnapshot { - log.Infof(ctx, "found message for replica ID %v as non-voter but "+ - "currently not part of the range, destroying preemptive snapshot", - replicaID) } repl.mu.Unlock() @@ -129,30 +120,18 @@ func (s *Store) tryGetOrCreateReplica( } defer repl.mu.Unlock() - // If this is intended for replicaID 0 then it's either a preemptive - // snapshot or a split/merge lock in which case we'll let it go through. - if replicaID == 0 { - return repl, false, nil - } - var err error - if repl.mu.replicaID == 0 { - // This message is telling us about our replica ID. - // This is a common case when dealing with preemptive snapshots. - err = repl.setReplicaIDRaftMuLockedMuLocked(repl.AnnotateCtx(ctx), replicaID) - } else if repl.mu.replicaID > replicaID { + if repl.mu.replicaID > replicaID { // The sender is behind and is sending to an old replica. // We could silently drop this message but this way we'll inform the // sender that they may no longer exist. - err = roachpb.NewRangeNotFoundError(rangeID, s.StoreID()) - } else if repl.mu.replicaID != replicaID { + repl.raftMu.Unlock() + return nil, false, &roachpb.RaftGroupDeletedError{} + } + if repl.mu.replicaID != replicaID { // This case should have been caught by handleToReplicaTooOld. log.Fatalf(ctx, "intended replica id %d unexpectedly does not match the current replica %v", replicaID, repl) } - if err != nil { - repl.raftMu.Unlock() - return nil, false, err - } return repl, false, nil } @@ -174,7 +153,12 @@ func (s *Store) tryGetOrCreateReplica( } // Create a new replica and lock it for raft processing. - repl := newReplica(rangeID, s) + uninitializedDesc := &roachpb.RangeDescriptor{ + RangeID: rangeID, + // NB: other fields are unknown; need to populate them from + // snapshot. + } + repl := newUnloadedReplica(ctx, uninitializedDesc, s, replicaID) repl.creatingReplica = creatingReplica repl.raftMu.Lock() // not unlocked @@ -187,11 +171,7 @@ func (s *Store) tryGetOrCreateReplica( // Store.mu to maintain lock ordering invariant. repl.mu.Lock() repl.mu.tombstoneMinReplicaID = tombstone.NextReplicaID - uninitializedDesc := &roachpb.RangeDescriptor{ - RangeID: rangeID, - // NB: other fields are unknown; need to populate them from - // snapshot. - } + // NB: A Replica should never be in the store's replicas map with a nil // descriptor. Assign it directly here. In the case that the Replica should // exist (which we confirm with another check of the Tombstone below), we'll @@ -228,7 +208,7 @@ func (s *Store) tryGetOrCreateReplica( ctx, s.Engine(), tombstoneKey, hlc.Timestamp{}, &tombstone, engine.MVCCGetOptions{}, ); err != nil { return err - } else if ok && replicaID != 0 && replicaID < tombstone.NextReplicaID { + } else if ok && replicaID < tombstone.NextReplicaID { return &roachpb.RaftGroupDeletedError{} } @@ -240,7 +220,7 @@ func (s *Store) tryGetOrCreateReplica( } else if hs.Commit != 0 { log.Fatalf(ctx, "found non-zero HardState.Commit on uninitialized replica %s. HS=%+v", repl, hs) } - return repl.initRaftMuLockedReplicaMuLocked(uninitializedDesc, replicaID) + return repl.loadRaftMuLockedReplicaMuLocked(uninitializedDesc) }(); err != nil { // Mark the replica as destroyed and remove it from the replicas maps to // ensure nobody tries to use it. @@ -269,15 +249,6 @@ func fromReplicaIsTooOld(toReplica *Replica, fromReplica *roachpb.ReplicaDescrip return !found && fromReplica.ReplicaID < desc.NextReplicaID } -// toReplicaIsTooOld returns true if replicaID is newer than toReplica -// indicating that the Replica needs to be removed. -// Assumes toReplica.mu is held. -func toReplicaIsTooOld(toReplica *Replica, replicaID roachpb.ReplicaID) bool { - toReplica.mu.AssertHeld() - return replicaID != 0 && toReplica.mu.replicaID != 0 && - toReplica.mu.replicaID < replicaID -} - // addReplicaInternalLocked adds the replica to the replicas map and the // replicasByKey btree. Returns an error if a replica with // the same Range ID or a KeyRange that overlaps has already been added to diff --git a/pkg/storage/store_merge.go b/pkg/storage/store_merge.go index 481feae8f789..5a75dc86dc16 100644 --- a/pkg/storage/store_merge.go +++ b/pkg/storage/store_merge.go @@ -108,6 +108,6 @@ func (s *Store) MergeRange( } // Update the subsuming range's descriptor. - leftRepl.setDesc(ctx, &newLeftDesc) + leftRepl.setDescRaftMuLocked(ctx, &newLeftDesc) return nil } diff --git a/pkg/storage/store_pool_test.go b/pkg/storage/store_pool_test.go index 65eb35de7f31..70af03364dc8 100644 --- a/pkg/storage/store_pool_test.go +++ b/pkg/storage/store_pool_test.go @@ -592,11 +592,13 @@ func TestStorePoolUpdateLocalStoreBeforeGossip(t *testing.T) { // Create replica. rg := roachpb.RangeDescriptor{ - RangeID: 1, - StartKey: roachpb.RKey([]byte("a")), - EndKey: roachpb.RKey([]byte("b")), + RangeID: 1, + StartKey: roachpb.RKey([]byte("a")), + EndKey: roachpb.RKey([]byte("b")), + NextReplicaID: 1, } - replica, err := NewReplica(&rg, store, roachpb.ReplicaID(0)) + rg.AddReplica(1, 1, roachpb.VOTER_FULL) + replica, err := newReplica(ctx, &rg, store, 1) if err != nil { t.Fatalf("make replica error : %+v", err) } diff --git a/pkg/storage/store_remove_replica.go b/pkg/storage/store_remove_replica.go index 219cce8673aa..db67f3ab001b 100644 --- a/pkg/storage/store_remove_replica.go +++ b/pkg/storage/store_remove_replica.go @@ -83,19 +83,12 @@ func (s *Store) removeInitializedReplicaRaftMuLocked( return nil // already removed, noop } - // We check both rep.mu.ReplicaID and rep.mu.state.Desc's replica ID because - // they can differ in cases when a replica's ID is increased due to an - // incoming raft message (see #14231 for background). - if rep.mu.replicaID >= nextReplicaID { - rep.mu.Unlock() - return errors.Errorf("cannot remove replica %s; replica ID has changed (%s >= %s)", - rep, rep.mu.replicaID, nextReplicaID) - } desc = rep.mu.state.Desc if repDesc, ok := desc.GetReplicaDescriptor(s.StoreID()); ok && repDesc.ReplicaID >= nextReplicaID { rep.mu.Unlock() - return errors.Errorf("cannot remove replica %s; replica descriptor's ID has changed (%s >= %s)", - rep, repDesc.ReplicaID, nextReplicaID) + // NB: This should not in any way be possible starting in 20.1. + log.Fatalf(ctx, "replica descriptor's ID has changed (%s >= %s)", + repDesc.ReplicaID, nextReplicaID) } // This is a fatal error as an initialized replica can never become diff --git a/pkg/storage/store_snapshot.go b/pkg/storage/store_snapshot.go index 99b4e7b84ffe..fb44795321af 100644 --- a/pkg/storage/store_snapshot.go +++ b/pkg/storage/store_snapshot.go @@ -35,7 +35,7 @@ import ( ) const ( - // Messages that provide detail about why a preemptive snapshot was rejected. + // Messages that provide detail about why a snapshot was rejected. snapshotStoreTooFullMsg = "store almost out of disk space" snapshotApplySemBusyMsg = "store busy applying snapshots" storeDrainingMsg = "store is draining" @@ -295,19 +295,6 @@ func (kvSS *kvBatchSnapshotStrategy) Receive( inSnap.String(), len(logEntries), expLen) } - // 19.1 nodes don't set the Type field on the SnapshotRequest_Header proto - // when sending this RPC, so in a mixed cluster setting we may have gotten - // the zero value of RAFT. Since the RPC didn't have type information - // previously, a 19.1 node receiving a snapshot distinguished between RAFT - // and PREEMPTIVE (19.1 nodes never sent LEARNER snapshots) by checking - // whether the replica was a placeholder (ReplicaID == 0). - // - // This adjustment can be removed after 19.2. - if inSnap.snapType == SnapshotRequest_RAFT && - header.RaftMessageRequest.ToReplica.ReplicaID == 0 { - inSnap.snapType = SnapshotRequest_PREEMPTIVE - } - kvSS.status = fmt.Sprintf("log entries: %d, ssts: %d", len(logEntries), len(kvSS.scratch.SSTs())) return inSnap, nil } @@ -387,31 +374,6 @@ func (kvSS *kvBatchSnapshotStrategy) Send( if err == nil { logEntries = append(logEntries, bytes) raftLogBytes += int64(len(bytes)) - if snap.snapType == SnapshotRequest_PREEMPTIVE && - raftLogBytes > 4*kvSS.raftCfg.RaftLogTruncationThreshold { - // If the raft log is too large, abort the snapshot instead of - // potentially running out of memory. However, if this is a - // raft-initiated snapshot (instead of a preemptive one), we - // have a dilemma. It may be impossible to truncate the raft - // log until we have caught up a peer with a snapshot. Since - // we don't know the exact size at which we will run out of - // memory, we err on the size of allowing the snapshot if it - // is raft-initiated, while aborting preemptive snapshots at a - // reasonable threshold. (Empirically, this is good enough: - // the situations that result in large raft logs have not been - // observed to result in raft-initiated snapshots). - // - // By aborting preemptive snapshots here, we disallow replica - // changes until the current replicas have caught up and - // truncated the log (either the range is available, in which - // case this will eventually happen, or it's not,in which case - // the preemptive snapshot would be wasted anyway because the - // change replicas transaction would be unable to commit). - return false, errors.Errorf( - "aborting snapshot because raft log is too large "+ - "(%d bytes after processing %d of %d entries)", - raftLogBytes, len(logEntries), endIndex-firstIndex) - } } return false, err } @@ -763,15 +725,16 @@ func (s *Store) receiveSnapshot( } } - // Defensive check that any non-preemptive snapshot contains this store in the - // descriptor. - if !header.IsPreemptive() { - storeID := s.StoreID() - if _, ok := header.State.Desc.GetReplicaDescriptor(storeID); !ok { - return crdberrors.AssertionFailedf( - `snapshot of type %s was sent to s%d which did not contain it as a replica: %s`, - header.Type, storeID, header.State.Desc.Replicas()) - } + if header.IsPreemptive() { + return crdberrors.AssertionFailedf(`expected a raft or learner snapshot`) + } + + // Defensive check that any snapshot contains this store in the descriptor. + storeID := s.StoreID() + if _, ok := header.State.Desc.GetReplicaDescriptor(storeID); !ok { + return crdberrors.AssertionFailedf( + `snapshot of type %s was sent to s%d which did not contain it as a replica: %s`, + header.Type, storeID, header.State.Desc.Replicas()) } cleanup, rejectionMsg, err := s.reserveSnapshot(ctx, header) @@ -791,18 +754,10 @@ func (s *Store) receiveSnapshot( // We'll perform this check again later after receiving the rest of the // snapshot data - this is purely an optimization to prevent downloading // a snapshot that we know we won't be able to apply. - if header.IsPreemptive() { - if _, err := s.canApplyPreemptiveSnapshot(ctx, header, false /* authoritative */); err != nil { - return sendSnapshotError(stream, - errors.Wrapf(err, "%s,r%d: cannot apply snapshot", s, header.State.Desc.RangeID), - ) - } - } else { - if err := s.shouldAcceptSnapshotData(ctx, header); err != nil { - return sendSnapshotError(stream, - errors.Wrapf(err, "%s,r%d: cannot apply snapshot", s, header.State.Desc.RangeID), - ) - } + if err := s.shouldAcceptSnapshotData(ctx, header); err != nil { + return sendSnapshotError(stream, + errors.Wrapf(err, "%s,r%d: cannot apply snapshot", s, header.State.Desc.RangeID), + ) } // Determine which snapshot strategy the sender is using to send this @@ -841,14 +796,8 @@ func (s *Store) receiveSnapshot( if err != nil { return err } - if header.IsPreemptive() { - if err := s.processPreemptiveSnapshotRequest(ctx, header, inSnap); err != nil { - return sendSnapshotError(stream, errors.Wrap(err.GoError(), "failed to apply snapshot")) - } - } else { - if err := s.processRaftSnapshotRequest(ctx, header, inSnap); err != nil { - return sendSnapshotError(stream, errors.Wrap(err.GoError(), "failed to apply snapshot")) - } + if err := s.processRaftSnapshotRequest(ctx, header, inSnap); err != nil { + return sendSnapshotError(stream, errors.Wrap(err.GoError(), "failed to apply snapshot")) } return stream.Send(&SnapshotResponse{Status: SnapshotResponse_APPLIED}) diff --git a/pkg/storage/store_snapshot_preemptive.go b/pkg/storage/store_snapshot_preemptive.go deleted file mode 100644 index e43b09593cae..000000000000 --- a/pkg/storage/store_snapshot_preemptive.go +++ /dev/null @@ -1,357 +0,0 @@ -// Copyright 2019 The Cockroach Authors. -// -// Use of this software is governed by the Business Source License -// included in the file licenses/BSL.txt. -// -// As of the Change Date specified in that file, in accordance with -// the Business Source License, use of this software will be governed -// by the Apache License, Version 2.0, included in the file -// licenses/APL.txt. - -package storage - -import ( - "context" - "sync/atomic" - - "github.com/cockroachdb/cockroach/pkg/roachpb" - "github.com/cockroachdb/cockroach/pkg/util/log" - crdberrors "github.com/cockroachdb/errors" - "github.com/pkg/errors" - "go.etcd.io/etcd/raft" - "go.etcd.io/etcd/raft/raftpb" -) - -// canApplyPreemptiveSnapshot returns (_, nil) if the preemptive snapshot can be -// applied to this store's replica (i.e. the snapshot is not from an older -// incarnation of the replica) and a placeholder can be added to the -// replicasByKey map (if necessary). If a placeholder is required, it is -// returned as the first value. The authoritative bool determines whether the -// check is carried out with the intention of actually applying the snapshot (in -// which case an existing replica must exist and have its raftMu locked) or as a -// preliminary check. -func (s *Store) canApplyPreemptiveSnapshot( - ctx context.Context, snapHeader *SnapshotRequest_Header, authoritative bool, -) (*ReplicaPlaceholder, error) { - s.mu.Lock() - defer s.mu.Unlock() - return s.canApplyPreemptiveSnapshotLocked(ctx, snapHeader, authoritative) -} - -func (s *Store) canApplyPreemptiveSnapshotLocked( - ctx context.Context, snapHeader *SnapshotRequest_Header, authoritative bool, -) (*ReplicaPlaceholder, error) { - if !snapHeader.IsPreemptive() { - return nil, crdberrors.AssertionFailedf(`expected a preemptive snapshot`) - } - - // TODO(tbg): see the comment on desc.Generation for what seems to be a much - // saner way to handle overlap via generational semantics. - desc := *snapHeader.State.Desc - - // First, check for an existing Replica. - // - // We call canApplyPreemptiveSnapshotLocked twice for each snapshot - // application. In the first case, it's an optimization early before having - // received any data (and we don't use the placeholder if one is returned), - // and the replica may or may not be present. - // - // The second call happens right before we actually plan to apply the snapshot - // (and a Replica is always in place at that point). This means that without a - // Replica, we can have false positives, but if we have a replica it needs to - // take everything into account. - if v, ok := s.mu.replicas.Load( - int64(desc.RangeID), - ); !ok { - if authoritative { - return nil, errors.Errorf("authoritative call requires a replica present") - } - } else { - existingRepl := (*Replica)(v) - // The raftMu is held which allows us to use the existing replica as a - // placeholder when we decide that the snapshot can be applied. As long - // as the caller releases the raftMu only after feeding the snapshot - // into the replica, this is safe. - if authoritative { - existingRepl.raftMu.AssertHeld() - } - - existingRepl.mu.RLock() - existingDesc := existingRepl.descRLocked() - existingIsInitialized := existingRepl.isInitializedRLocked() - existingIsPreemptive := existingRepl.mu.replicaID == 0 - existingRepl.mu.RUnlock() - - if existingIsInitialized { - if existingIsPreemptive { - // Allow applying a preemptive snapshot on top of another - // preemptive snapshot. We only need to acquire a placeholder - // for the part (if any) of the new snapshot that extends past - // the old one. If there's no such overlap, return early; if - // there is, "forward" the descriptor's StartKey so that the - // later code will only check the overlap. - // - // NB: morally it would be cleaner to ask for the existing - // replica to be GC'ed first, but consider that the preemptive - // snapshot was likely left behind by a failed attempt to - // up-replicate. This is a relatively common scenario and not - // worth discarding and resending another snapshot for. Let the - // snapshot through, which means "pretending that it doesn't - // intersect the existing replica". - if !existingDesc.EndKey.Less(desc.EndKey) { - return nil, nil - } - desc.StartKey = existingDesc.EndKey - } - // NB: If the existing snapshot is *not* preemptive (i.e. the above - // branch wasn't taken), the overlap check below will hit an error. - // This path is hit after a rapid up-down-upreplication to the same - // store and will resolve as the replicaGCQueue removes the existing - // replica. We are pretty sure that the existing replica is gc'able, - // because a preemptive snapshot implies that someone is trying to - // add this replica to the group at the moment. (We are not however, - // sure enough that this couldn't happen by accident to GC the - // replica ourselves - the replica GC queue will perform the proper - // check). - } else { - // Morally, the existing replica now has a nonzero replica ID - // because we already know that it is not initialized (i.e. has no - // data). Interestingly, the case in which it has a zero replica ID - // is also possible and should see the snapshot accepted as it - // occurs when a preemptive snapshot is handled: we first create a - // Replica in this state, run this check, and then apply the - // preemptive snapshot. - if !existingIsPreemptive { - // This is similar to the case of a preemptive snapshot hitting - // a fully initialized replica (i.e. not a preemptive snapshot) - // at the end of the last branch (which we don't allow), so we - // want to reject the snapshot. There is a tricky problem to - // to solve here, though: existingRepl doesn't know anything - // about its key bounds, and so to check whether it is actually - // gc'able would require a full scan of the meta2 entries (and - // we would also need to teach the queues how to deal with un- - // initialized replicas). - // - // So we let the snapshot through (by falling through to the - // overlap check, where it either picks up placeholders or - // fails). This is safe (or at least we assume so) because we - // carry out all snapshot decisions through Raft (though it - // still is an odd path that we would be wise to avoid if it - // weren't so difficult). - // - // A consequence of letting this snapshot through is opening this - // replica up to the possibility of erroneous replicaGC. This is - // because it will retain the replicaID of the current replica, - // which is going to be initialized after the snapshot (and thus - // gc'able). - _ = 0 // avoid staticcheck failure - } - } - } - - // We have a key range [desc.StartKey,desc.EndKey) which we want to apply a - // snapshot for. Is there a conflicting existing placeholder or an - // overlapping range? - if err := s.checkSnapshotOverlapLocked(ctx, snapHeader); err != nil { - return nil, err - } - - placeholder := &ReplicaPlaceholder{ - rangeDesc: desc, - } - return placeholder, nil -} - -// processPreemptiveSnapshotRequest processes the incoming preemptive snapshot -// request on the request's specified replica. -func (s *Store) processPreemptiveSnapshotRequest( - ctx context.Context, snapHeader *SnapshotRequest_Header, inSnap IncomingSnapshot, -) *roachpb.Error { - if !snapHeader.IsPreemptive() { - return roachpb.NewError(crdberrors.AssertionFailedf(`expected a preemptive snapshot`)) - } - - return s.withReplicaForRequest(ctx, &snapHeader.RaftMessageRequest, func( - ctx context.Context, r *Replica, - ) (pErr *roachpb.Error) { - if snapHeader.RaftMessageRequest.Message.Type != raftpb.MsgSnap { - log.Fatalf(ctx, "expected snapshot: %+v", snapHeader.RaftMessageRequest) - } - - // Check to see if a snapshot can be applied. Snapshots can always be applied - // to initialized replicas. Note that if we add a placeholder we need to - // already be holding Replica.raftMu in order to prevent concurrent - // raft-ready processing of uninitialized replicas. - var addedPlaceholder bool - var removePlaceholder bool - if err := func() error { - s.mu.Lock() - defer s.mu.Unlock() - placeholder, err := s.canApplyPreemptiveSnapshotLocked(ctx, snapHeader, true /* authoritative */) - if err != nil { - // If the storage cannot accept the snapshot, return an - // error before passing it to RawNode.Step, since our - // error handling options past that point are limited. - log.Infof(ctx, "cannot apply snapshot: %s", err) - return err - } - - if placeholder != nil { - // NB: The placeholder added here is either removed below after a - // preemptive snapshot is applied or after the next call to - // Replica.handleRaftReady. Note that we can only get here if the - // replica doesn't exist or is uninitialized. - if err := s.addPlaceholderLocked(placeholder); err != nil { - log.Fatalf(ctx, "could not add vetted placeholder %s: %+v", placeholder, err) - } - addedPlaceholder = true - } - return nil - }(); err != nil { - return roachpb.NewError(err) - } - - if addedPlaceholder { - // If we added a placeholder remove it before we return unless some other - // part of the code takes ownership of the removal (indicated by setting - // removePlaceholder to false). - removePlaceholder = true - defer func() { - if removePlaceholder { - if s.removePlaceholder(ctx, snapHeader.RaftMessageRequest.RangeID) { - atomic.AddInt32(&s.counts.removedPlaceholders, 1) - } - } - }() - } - - // Requiring that the Term is set in a message makes sure that we - // get all of Raft's internal safety checks (it confuses messages - // at term zero for internal messages). The sending side uses the - // term from the snapshot itself, but we'll just check nonzero. - if snapHeader.RaftMessageRequest.Message.Term == 0 { - return roachpb.NewErrorf( - "preemptive snapshot from term %d received with zero term", - snapHeader.RaftMessageRequest.Message.Snapshot.Metadata.Term, - ) - } - // TODO(tschottdorf): A lot of locking of the individual Replica - // going on below as well. I think that's more easily refactored - // away; what really matters is that the Store doesn't do anything - // else with that same Replica (or one that might conflict with us - // while we still run). In effect, we'd want something like: - // - // 1. look up the snapshot's key range - // 2. get an exclusive lock for operations on that key range from - // the store (or discard the snapshot) - // (at the time of writing, we have checked the key range in - // canApplyPreemptiveSnapshotLocked above, but there are concerns - // about two conflicting operations passing that check simultaneously, - // see #7830) - // 3. do everything below (apply the snapshot through temp Raft group) - // 4. release the exclusive lock on the snapshot's key range - // - // There are two future outcomes: Either we begin receiving - // legitimate Raft traffic for this Range (hence learning the - // ReplicaID and becoming a real Replica), or the Replica GC queue - // decides that the ChangeReplicas as a part of which the - // preemptive snapshot was sent has likely failed and removes both - // in-memory and on-disk state. - r.mu.Lock() - // We are paranoid about applying preemptive snapshots (which - // were constructed via our code rather than raft) to the "real" - // raft group. Instead, we destroy the "real" raft group if one - // exists (this is rare in production, although it occurs in - // tests), apply the preemptive snapshot to a temporary raft - // group, then discard that one as well to be replaced by a real - // raft group when we get a new replica ID. - // - // It might be OK instead to apply preemptive snapshots just - // like normal ones (essentially switching between regular and - // preemptive mode based on whether or not we have a raft group, - // instead of based on the replica ID of the snapshot message). - // However, this is a risk that we're not yet willing to take. - // Additionally, without some additional plumbing work, doing so - // would limit the effectiveness of RaftTransport.SendSync for - // preemptive snapshots. - r.mu.internalRaftGroup = nil - needTombstone := r.mu.state.Desc.NextReplicaID != 0 - r.mu.Unlock() - - appliedIndex, _, err := r.raftMu.stateLoader.LoadAppliedIndex(ctx, r.store.Engine()) - if err != nil { - return roachpb.NewError(err) - } - // We need to create a temporary RawNode to process the snapshot. Raft - // internally runs safety checks on the snapshot, among them one that - // verifies that the peer is actually part of the configuration encoded - // in the snapshot. Awkwardly, it isn't actually a peer (preemptive - // snapshot...). To get around this, pretend the RawNode has the ID of a - // peer we know exists, namely the one that sent us the snap. This won't - // be persisted anywhere, and since we're only using the RawNode for - // this one snapshot, everything is ok. However, we'll make sure that - // no messages are sent in the resulting Ready. - preemptiveSnapshotRaftGroupID := uint64(snapHeader.RaftMessageRequest.FromReplica.ReplicaID) - raftGroup, err := raft.NewRawNode( - newRaftConfig( - raft.Storage((*replicaRaftStorage)(r)), - preemptiveSnapshotRaftGroupID, - // We pass the "real" applied index here due to subtleties - // arising in the case in which Raft discards the snapshot: - // It would instruct us to apply entries, which would have - // crashing potential for any choice of dummy value below. - appliedIndex, - r.store.cfg, - &raftLogger{ctx: ctx}, - )) - if err != nil { - return roachpb.NewError(err) - } - // We have a Raft group; feed it the message. - if err := raftGroup.Step(snapHeader.RaftMessageRequest.Message); err != nil { - return roachpb.NewError(errors.Wrap(err, "unable to process preemptive snapshot")) - } - // In the normal case, the group should ask us to apply a snapshot. - // If it doesn't, our snapshot was probably stale. In that case we - // still go ahead and apply a noop because we want that case to be - // counted by stats as a successful application. - var ready raft.Ready - if raftGroup.HasReady() { - ready = raftGroup.Ready() - // See the comment above - we don't want this temporary Raft group - // to contact the outside world. Apply the snapshot and that's it. - ready.Messages = nil - } - - if needTombstone { - // Bump the min replica ID, but don't write the tombstone key. The - // tombstone key is not expected to be present when normal replica data - // is present and applySnapshot would delete the key in most cases. If - // Raft has decided the snapshot shouldn't be applied we would be - // writing the tombstone key incorrectly. - r.mu.Lock() - if r.mu.state.Desc.NextReplicaID > r.mu.tombstoneMinReplicaID { - r.mu.tombstoneMinReplicaID = r.mu.state.Desc.NextReplicaID - } - r.mu.Unlock() - } - - // Apply the snapshot, as Raft told us to. Preemptive snapshots never - // subsume replicas (this is guaranteed by - // Store.canApplyPreemptiveSnapshot), so we can simply pass nil for the - // subsumedRepls parameter. - if err := r.applySnapshot( - ctx, inSnap, ready.Snapshot, ready.HardState, nil, /* subsumedRepls */ - ); err != nil { - return roachpb.NewError(err) - } - // applySnapshot has already removed the placeholder. - removePlaceholder = false - - // At this point, the Replica has data but no ReplicaID. We hope - // that it turns into a "real" Replica by means of receiving Raft - // messages addressed to it with a ReplicaID, but if that doesn't - // happen, at some point the Replica GC queue will have to grab it. - return nil - }) -} diff --git a/pkg/storage/store_snapshot_test.go b/pkg/storage/store_snapshot_test.go deleted file mode 100644 index 6c866db8ecc7..000000000000 --- a/pkg/storage/store_snapshot_test.go +++ /dev/null @@ -1,144 +0,0 @@ -// Copyright 2018 The Cockroach Authors. -// -// Use of this software is governed by the Business Source License -// included in the file licenses/BSL.txt. -// -// As of the Change Date specified in that file, in accordance with -// the Business Source License, use of this software will be governed -// by the Apache License, Version 2.0, included in the file -// licenses/APL.txt. - -package storage - -import ( - "context" - "strings" - "testing" - - "github.com/cockroachdb/cockroach/pkg/internal/client" - "github.com/cockroachdb/cockroach/pkg/roachpb" - "github.com/cockroachdb/cockroach/pkg/storage/rditer" - "github.com/cockroachdb/cockroach/pkg/testutils" - "github.com/cockroachdb/cockroach/pkg/util/leaktest" - "github.com/cockroachdb/cockroach/pkg/util/stop" - "go.etcd.io/etcd/raft/raftpb" - "golang.org/x/time/rate" -) - -func TestSnapshotRaftLogLimit(t *testing.T) { - defer leaktest.AfterTest(t)() - ctx := context.Background() - stopper := stop.NewStopper() - defer stopper.Stop(ctx) - - store, _ := createTestStore(t, - testStoreOpts{ - // This test was written before test stores could start with more than one - // range and was not adapted. - createSystemRanges: false, - }, - stopper) - store.SetRaftLogQueueActive(false) - repl, err := store.GetReplica(1) - if err != nil { - t.Fatal(err) - } - - var bytesWritten int64 - blob := []byte(strings.Repeat("a", 1024*1024)) - for i := 0; bytesWritten < 5*store.cfg.RaftLogTruncationThreshold; i++ { - pArgs := putArgs(roachpb.Key("a"), blob) - _, pErr := client.SendWrappedWith(ctx, store, roachpb.Header{RangeID: 1}, &pArgs) - if pErr != nil { - t.Fatal(pErr) - } - bytesWritten += int64(len(blob)) - } - - for _, snapType := range []SnapshotRequest_Type{SnapshotRequest_PREEMPTIVE, SnapshotRequest_RAFT} { - t.Run(snapType.String(), func(t *testing.T) { - lastIndex, err := repl.GetLastIndex() - if err != nil { - t.Fatal(err) - } - eng := store.Engine() - snap := eng.NewSnapshot() - defer snap.Close() - - ss := kvBatchSnapshotStrategy{ - raftCfg: &store.cfg.RaftConfig, - limiter: rate.NewLimiter(1<<10, 1), - newBatch: eng.NewBatch, - } - iter := rditer.NewReplicaDataIterator(repl.Desc(), snap, - true /* replicatedOnly */, false /* seekEnd */) - defer iter.Close() - outSnap := &OutgoingSnapshot{ - Iter: iter, - EngineSnap: snap, - snapType: snapType, - RaftSnap: raftpb.Snapshot{ - Metadata: raftpb.SnapshotMetadata{ - Index: lastIndex, - }, - }, - } - - var stream fakeSnapshotStream - header := SnapshotRequest_Header{ - State: repl.State().ReplicaState, - } - - err = ss.Send(ctx, stream, header, outSnap) - if snapType == SnapshotRequest_PREEMPTIVE { - if !testutils.IsError(err, "aborting snapshot because raft log is too large") { - t.Fatalf("unexpected error: %+v", err) - } - } else { - if err != nil { - t.Fatal(err) - } - } - }) - } -} - -// TestSnapshotPreemptiveOnUninitializedReplica is a targeted regression test -// against a bug that once accepted these snapshots without forcing them to -// check for overlapping ranges. -func TestSnapshotPreemptiveOnUninitializedReplica(t *testing.T) { - defer leaktest.AfterTest(t)() - ctx := context.Background() - stopper := stop.NewStopper() - defer stopper.Stop(ctx) - - store, _ := createTestStore(t, testStoreOpts{}, stopper) - - // Create an uninitialized replica. - repl, created, err := store.getOrCreateReplica(ctx, 77, 1, nil, true) - if err != nil { - t.Fatal(err) - } - if !created { - t.Fatal("no replica created") - } - - // Make a descriptor that overlaps r1 (any descriptor does because r1 covers - // all of the keyspace). - desc := *repl.Desc() - desc.StartKey = roachpb.RKey("a") - desc.EndKey = roachpb.RKey("b") - - header := &SnapshotRequest_Header{} - header.State.Desc = &desc - - if !header.IsPreemptive() { - t.Fatal("mock snapshot isn't preemptive") - } - - if _, err := store.canApplyPreemptiveSnapshot( - ctx, header, true, /* authoritative */ - ); !testutils.IsError(err, "intersects existing range") { - t.Fatal(err) - } -} diff --git a/pkg/storage/store_split.go b/pkg/storage/store_split.go index d6f6197e367a..d57a03fcc310 100644 --- a/pkg/storage/store_split.go +++ b/pkg/storage/store_split.go @@ -21,6 +21,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/pkg/errors" "go.etcd.io/etcd/raft" + "go.etcd.io/etcd/raft/raftpb" ) // splitPreApply is called when the raft command is applied. Any @@ -29,68 +30,80 @@ import ( func splitPreApply( ctx context.Context, readWriter engine.ReadWriter, split roachpb.SplitTrigger, r *Replica, ) { - // Check on the RHS, we need to ensure that it exists and has a minReplicaID - // less than or equal to the replica we're about to initialize. + // Sanity check that the store is in the split. // - // The right hand side of the split was already created (and its raftMu - // acquired) in Replica.acquireSplitLock. It must be present here. - rightRepl, err := r.store.GetReplica(split.RightDesc.RangeID) - if err != nil { - log.Fatalf(ctx, "unable to find RHS replica: %+v", err) - } - - // If the RHS is not in the split, sanity check that the LHS is currently - // catching up from a preemptive snapshot. A preemptive snapshot is - // the only reason we should here; replicas delete themselves when they - // apply a command which removes them from the range. The exception to that - // is if the DisableEagerReplicaRemoval testing flag is enabled. + // The exception to that is if the DisableEagerReplicaRemoval testing flag is + // enabled. // // TODO(ajwerner): rethink DisableEagerReplicaRemoval and remove this in // 20.1 after there are no more preemptive snapshots. _, hasRightDesc := split.RightDesc.GetReplicaDescriptor(r.StoreID()) - if !hasRightDesc { - _, lhsExists := r.Desc().GetReplicaDescriptor(r.StoreID()) - if lhsExists { - log.Fatalf(ctx, "cannot process split on s%s which exists in LHS and not in RHS: %+v", - r.StoreID(), split) - } + _, hasLeftDesc := split.LeftDesc.GetReplicaDescriptor(r.StoreID()) + if !hasRightDesc || !hasLeftDesc { + log.Fatalf(ctx, "cannot process split on s%s which does not exist in the split: %+v", + r.StoreID(), split) } + // Check on the RHS, we need to ensure that it exists and has a minReplicaID + // less than or equal to the replica we're about to initialize. + // + // The right hand side of the split was already created (and its raftMu + // acquired) in Replica.acquireSplitLock. It must be present here if it hasn't + // been removed in the meantime (handled below). + rightRepl, err := r.store.GetReplica(split.RightDesc.RangeID) + if roachpb.IsRangeNotFoundError(err) { + // The right hand side we were planning to populate has already been removed. + // We handle this below. + rightRepl = nil + } else if err != nil { + log.Fatal(ctx, errors.Wrap(err, "failed to get RHS replica")) + } // Check to see if we know that the RHS has already been removed from this // store at the replica ID implied by the split. - if rightRepl.isNewerThanSplit(&split) { + if rightRepl == nil || rightRepl.isNewerThanSplit(&split) { // We're in the rare case where we know that the RHS has been removed - // and re-added with a higher replica ID. We know we've never processed a - // snapshot for the right range because up to this point it would overlap - // with the left and ranges cannot move rightwards. + // and re-added with a higher replica ID (and then maybe removed again). // + // To apply the split, we need to "throw away" the data that would belong to + // the RHS, i.e. we clear the user data the RHS would have inherited from the + // LHS due to the split and additionally clear all of the range ID local state + // that the split trigger writes into the RHS. + // + // We know we've never processed a snapshot for the right range because the + // LHS prevents any incoming snapshots until the split has executed (i.e. now). // It is important to preserve the HardState because we might however have // already voted at a higher term. In general this shouldn't happen because // we add learners and then promote them only after we snapshot but we're // going to be extra careful in case future versions of cockroach somehow // promote replicas without ensuring that a snapshot has been received. // - // Clear the user data the RHS would have inherited from the LHS due to the - // split and additionally clear all of the range ID local state that the - // split trigger writes into the RHS. - // // Rather than specifically deleting around the data we want to preserve // we read the HardState to preserve it, clear everything and write back - // the HardState and tombstone. - hs, err := rightRepl.raftMu.stateLoader.LoadHardState(ctx, readWriter) - if err != nil { - log.Fatalf(ctx, "failed to load hard state for removed rhs: %v", err) + // the HardState and tombstone. Note that we only do this if rightRepl + // exists; if it doesn't, there's no Raft state to massage (when rightRepl + // was removed, a tombstone was written instead). + var hs raftpb.HardState + if rightRepl != nil { + // Assert that the rightRepl is not initialized. We're about to clear out + // the data of the RHS of the split; we cannot have already accepted a + // snapshot to initialize this newer RHS. + if rightRepl.IsInitialized() { + log.Fatalf(ctx, "unexpectedly found initialized newer RHS of split: %v", rightRepl.Desc()) + } + hs, err = rightRepl.raftMu.stateLoader.LoadHardState(ctx, readWriter) + if err != nil { + log.Fatalf(ctx, "failed to load hard state for removed rhs: %v", err) + } } const rangeIDLocalOnly = false const mustUseClearRange = false if err := clearRangeData(&split.RightDesc, readWriter, readWriter, rangeIDLocalOnly, mustUseClearRange); err != nil { log.Fatalf(ctx, "failed to clear range data for removed rhs: %v", err) } - if err := rightRepl.raftMu.stateLoader.SetHardState(ctx, readWriter, hs); err != nil { - log.Fatalf(ctx, "failed to set hard state with 0 commit index for removed rhs: %v", err) - } - if err := r.setTombstoneKey(ctx, readWriter, r.minReplicaID()); err != nil { - log.Fatalf(ctx, "failed to set tombstone for removed rhs: %v", err) + if rightRepl != nil { + if err := rightRepl.raftMu.stateLoader.SetHardState(ctx, readWriter, hs); err != nil { + log.Fatalf(ctx, "failed to set hard state with 0 commit index for removed rhs: %v", err) + } } return } @@ -106,78 +119,13 @@ func splitPreApply( // splitPostApply is the part of the split trigger which coordinates the actual // split with the Store. Requires that Replica.raftMu is held. -// -// TODO(tschottdorf): Want to merge this with SplitRange, but some legacy -// testing code calls SplitRange directly. func splitPostApply( ctx context.Context, deltaMS enginepb.MVCCStats, split *roachpb.SplitTrigger, r *Replica, ) { - // The right hand side of the split was already created (and its raftMu - // acquired) in Replica.acquireSplitLock. It must be present here. - rightRngOrNil, err := r.store.GetReplica(split.RightDesc.RangeID) - if err != nil { - log.Fatalf(ctx, "unable to find RHS replica: %+v", err) - } - // Already holding raftMu, see above. - rightRngOrNil.mu.Lock() - - // If we know that the RHS has already been removed at this replica ID - // then we also know that its data has already been removed by the preApply - // so we skip initializing it as the RHS of the split. - if rightRngOrNil.isNewerThanSplitRLocked(split) { - rightRngOrNil.mu.Unlock() - rightRngOrNil = nil - } else { - rightRng := rightRngOrNil - // Finish initialization of the RHS. - err := rightRng.initRaftMuLockedReplicaMuLocked(&split.RightDesc, 0) - rightRng.mu.Unlock() - if err != nil { - log.Fatal(ctx, err) - } - - // This initialMaxClosedValue is created here to ensure that follower reads - // do not regress following the split. After the split occurs there will be no - // information in the closedts subsystem about the newly minted RHS range from - // its leaseholder's store. Furthermore, the RHS will have a lease start time - // equal to that of the LHS which might be quite old. This means that - // timestamps which follow the least StartTime for the LHS part are below the - // current closed timestamp for the LHS would no longer be readable on the RHS - // after the split. It is critical that this call to maxClosed happen during - // the splitPostApply so that it refers to a LAI that is equal to the index at - // which this lease was applied. If it were to refer to a LAI after the split - // then the value of initialMaxClosed might be unsafe. - initialMaxClosed := r.maxClosed(ctx) - r.mu.Lock() - rightRng.mu.Lock() - // Copy the minLeaseProposedTS from the LHS. - rightRng.mu.minLeaseProposedTS = r.mu.minLeaseProposedTS - rightRng.mu.initialMaxClosed = initialMaxClosed - rightLease := *rightRng.mu.state.Lease - rightRng.mu.Unlock() - r.mu.Unlock() - - // We need to explicitly wake up the Raft group on the right-hand range or - // else the range could be underreplicated for an indefinite period of time. - // - // Specifically, suppose one of the replicas of the left-hand range never - // applies this split trigger, e.g., because it catches up via a snapshot that - // advances it past this split. That store won't create the right-hand replica - // until it receives a Raft message addressed to the right-hand range. But - // since new replicas start out quiesced, unless we explicitly awaken the - // Raft group, there might not be any Raft traffic for quite a while. - err = rightRng.withRaftGroup(true, func(r *raft.RawNode) (unquiesceAndWakeLeader bool, _ error) { - return true, nil - }) - if err != nil { - log.Fatalf(ctx, "unable to create raft group for right-hand range in split: %+v", err) - } - - // Invoke the leasePostApply method to ensure we properly initialize - // the replica according to whether it holds the lease. This enables - // the txnWaitQueue. - rightRng.leasePostApply(ctx, rightLease, false /* permitJump */) - } + // rightRngOrNil will be nil if the RHS replica at the ID of the split is + // already known to be removed, generally because we know that this store has + // been re-added at a higher replica ID. + rightRngOrNil := prepareRightReplicaForSplit(ctx, split, r) // Add the RHS replica to the store. This step atomically updates // the EndKey of the LHS replica and also adds the RHS replica // to the store's replica map. @@ -212,7 +160,86 @@ func splitPostApply( r.store.enqueueRaftUpdateCheck(rightRngOrNil.RangeID) } } +} + +// prepareRightReplicaForSplit a helper for splitPostApply. +// Requires that r.raftMu is held. +func prepareRightReplicaForSplit( + ctx context.Context, split *roachpb.SplitTrigger, r *Replica, +) (rightReplicaOrNil *Replica) { + // The right hand side of the split was already created (and its raftMu + // acquired) in Replica.acquireSplitLock. It must be present here. + rightRng, err := r.store.GetReplica(split.RightDesc.RangeID) + // If the RHS replica at the point of the split was known to be removed when + // during the application of the split then we may not find it here. That's + // fine, carry on. See also: + _, _ = r.acquireSplitLock, splitPostApply + if roachpb.IsRangeNotFoundError(err) { + return nil + } + if err != nil { + log.Fatalf(ctx, "unable to find RHS replica: %+v", err) + } + // Already holding raftMu, see above. + rightRng.mu.Lock() + + // If we know that the RHS has already been removed at this replica ID + // then we also know that its data has already been removed by the preApply + // so we skip initializing it as the RHS of the split. + if rightRng.isNewerThanSplitRLocked(split) { + rightRng.mu.Unlock() + return nil + } + + // Finish initialization of the RHS. + err = rightRng.loadRaftMuLockedReplicaMuLocked(&split.RightDesc) + rightRng.mu.Unlock() + if err != nil { + log.Fatal(ctx, err) + } + + // This initialMaxClosedValue is created here to ensure that follower reads + // do not regress following the split. After the split occurs there will be no + // information in the closedts subsystem about the newly minted RHS range from + // its leaseholder's store. Furthermore, the RHS will have a lease start time + // equal to that of the LHS which might be quite old. This means that + // timestamps which follow the least StartTime for the LHS part are below the + // current closed timestamp for the LHS would no longer be readable on the RHS + // after the split. It is critical that this call to maxClosed happen during + // the splitPostApply so that it refers to a LAI that is equal to the index at + // which this lease was applied. If it were to refer to a LAI after the split + // then the value of initialMaxClosed might be unsafe. + initialMaxClosed := r.maxClosed(ctx) + r.mu.Lock() + rightRng.mu.Lock() + // Copy the minLeaseProposedTS from the LHS. + rightRng.mu.minLeaseProposedTS = r.mu.minLeaseProposedTS + rightRng.mu.initialMaxClosed = initialMaxClosed + rightLease := *rightRng.mu.state.Lease + rightRng.mu.Unlock() + r.mu.Unlock() + + // We need to explicitly wake up the Raft group on the right-hand range or + // else the range could be underreplicated for an indefinite period of time. + // + // Specifically, suppose one of the replicas of the left-hand range never + // applies this split trigger, e.g., because it catches up via a snapshot that + // advances it past this split. That store won't create the right-hand replica + // until it receives a Raft message addressed to the right-hand range. But + // since new replicas start out quiesced, unless we explicitly awaken the + // Raft group, there might not be any Raft traffic for quite a while. + err = rightRng.withRaftGroup(true, func(r *raft.RawNode) (unquiesceAndWakeLeader bool, _ error) { + return true, nil + }) + if err != nil { + log.Fatalf(ctx, "unable to create raft group for right-hand range in split: %+v", err) + } + // Invoke the leasePostApply method to ensure we properly initialize + // the replica according to whether it holds the lease. This enables + // the txnWaitQueue. + rightRng.leasePostApply(ctx, rightLease, false /* permitJump */) + return rightRng } // SplitRange shortens the original range to accommodate the new range. The new @@ -248,7 +275,7 @@ func (s *Store) SplitRange( s.replicaQueues.Delete(int64(rightDesc.RangeID)) } - leftRepl.setDesc(ctx, newLeftDesc) + leftRepl.setDescRaftMuLocked(ctx, newLeftDesc) // Clear the LHS txn wait queue, to redirect to the RHS if // appropriate. We do this after setDescWithoutProcessUpdate diff --git a/pkg/storage/store_test.go b/pkg/storage/store_test.go index d914e088c98f..7c319cfac436 100644 --- a/pkg/storage/store_test.go +++ b/pkg/storage/store_test.go @@ -33,7 +33,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/rpc" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" - "github.com/cockroachdb/cockroach/pkg/storage/abortspan" "github.com/cockroachdb/cockroach/pkg/storage/batcheval" "github.com/cockroachdb/cockroach/pkg/storage/engine" "github.com/cockroachdb/cockroach/pkg/storage/engine/enginepb" @@ -43,6 +42,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/storage/storagepb" "github.com/cockroachdb/cockroach/pkg/storage/txnwait" "github.com/cockroachdb/cockroach/pkg/testutils" + "github.com/cockroachdb/cockroach/pkg/util/encoding" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" @@ -548,7 +548,7 @@ func createReplica(s *Store, rangeID roachpb.RangeID, start, end roachpb.RKey) * }}, NextReplicaID: 2, } - r, err := NewReplica(desc, s, 0) + r, err := newReplica(context.Background(), desc, s, 1) if err != nil { log.Fatal(context.Background(), err) } @@ -679,45 +679,6 @@ func TestReplicasByKey(t *testing.T) { } } -func TestStoreRemoveReplicaOldDescriptor(t *testing.T) { - defer leaktest.AfterTest(t)() - ctx := context.Background() - stopper := stop.NewStopper() - defer stopper.Stop(ctx) - store, _ := createTestStore(t, testStoreOpts{createSystemRanges: true}, stopper) - - rep, err := store.GetReplica(1) - if err != nil { - t.Fatal(err) - } - - // First try and fail with a stale descriptor. - origDesc := rep.Desc() - newDesc := protoutil.Clone(origDesc).(*roachpb.RangeDescriptor) - for i := range newDesc.InternalReplicas { - if newDesc.InternalReplicas[i].StoreID == store.StoreID() { - newDesc.InternalReplicas[i].ReplicaID++ - newDesc.NextReplicaID++ - break - } - } - - rep.setDesc(ctx, newDesc) - expectedErr := "replica descriptor's ID has changed" - if err := store.RemoveReplica(ctx, rep, origDesc.NextReplicaID, RemoveOptions{ - DestroyData: true, - }); !testutils.IsError(err, expectedErr) { - t.Fatalf("expected error %q but got %v", expectedErr, err) - } - - // Now try a fresh descriptor and succeed. - if err := store.RemoveReplica(ctx, rep, rep.Desc().NextReplicaID, RemoveOptions{ - DestroyData: true, - }); err != nil { - t.Fatal(err) - } -} - func TestStoreRemoveReplicaDestroy(t *testing.T) { defer leaktest.AfterTest(t)() stopper := stop.NewStopper() @@ -1003,20 +964,10 @@ func TestMaybeMarkReplicaInitialized(t *testing.T) { newRangeID := roachpb.RangeID(3) desc := &roachpb.RangeDescriptor{ RangeID: newRangeID, - InternalReplicas: []roachpb.ReplicaDescriptor{{ - NodeID: 1, - StoreID: 1, - ReplicaID: 1, - }}, - NextReplicaID: 2, } - r := &Replica{ - RangeID: desc.RangeID, - store: store, - abortSpan: abortspan.New(desc.RangeID), - } - if err := r.init(desc, 0); err != nil { + r, err := newReplica(context.Background(), desc, store, 1) + if err != nil { t.Fatal(err) } @@ -1030,11 +981,16 @@ func TestMaybeMarkReplicaInitialized(t *testing.T) { } // Initialize the range with start and end keys. - r.mu.Lock() - r.mu.state.Desc.StartKey = roachpb.RKey("b") - r.mu.state.Desc.EndKey = roachpb.RKey("d") - r.mu.Unlock() - + desc = protoutil.Clone(desc).(*roachpb.RangeDescriptor) + desc.StartKey = roachpb.RKey("b") + desc.EndKey = roachpb.RKey("d") + desc.InternalReplicas = []roachpb.ReplicaDescriptor{{ + NodeID: 1, + StoreID: 1, + ReplicaID: 1, + }} + desc.NextReplicaID = 2 + r.setDescRaftMuLocked(ctx, desc) if err := store.maybeMarkReplicaInitializedLocked(ctx, r); err != nil { t.Errorf("expected maybeMarkReplicaInitializedLocked on a replica that's not in the uninit map to silently succeed, got %v", err) } @@ -1371,7 +1327,7 @@ func splitTestRange(store *Store, key, splitKey roachpb.RKey, t *testing.T) *Rep hlc.Timestamp{}, stateloader.TruncatedStateUnreplicated, ) require.NoError(t, err) - newRng, err := NewReplica(rhsDesc, store, 0) + newRng, err := newReplica(ctx, rhsDesc, store, repl.ReplicaID()) require.NoError(t, err) newLeftDesc := *repl.Desc() newLeftDesc.EndKey = splitKey @@ -2823,80 +2779,6 @@ func TestStoreRangePlaceholders(t *testing.T) { } } -// Test that we remove snapshot placeholders on error conditions. -func TestStoreRemovePlaceholderOnError(t *testing.T) { - defer leaktest.AfterTest(t)() - tc := testContext{} - stopper := stop.NewStopper() - defer stopper.Stop(context.TODO()) - tc.Start(t, stopper) - s := tc.store - ctx := context.Background() - - // Clobber the existing range so we can test nonoverlapping placeholders. - repl1, err := s.GetReplica(1) - if err != nil { - t.Fatal(err) - } - if err := s.RemoveReplica(context.Background(), repl1, repl1.Desc().NextReplicaID, RemoveOptions{ - DestroyData: true, - }); err != nil { - t.Fatal(err) - } - - // Generate a minimal fake snapshot. - snapData := &roachpb.RaftSnapshotData{} - data, err := protoutil.Marshal(snapData) - if err != nil { - t.Fatal(err) - } - - // Wrap the snapshot in a minimal header. The request will error because the - // replica tombstone for the range requires that a new replica have an ID - // greater than 1. - snapHeader := &SnapshotRequest_Header{ - State: storagepb.ReplicaState{Desc: repl1.Desc()}, - RaftMessageRequest: RaftMessageRequest{ - RangeID: 1, - ToReplica: roachpb.ReplicaDescriptor{ - NodeID: 1, - StoreID: 1, - ReplicaID: 0, - }, - FromReplica: roachpb.ReplicaDescriptor{ - NodeID: 2, - StoreID: 2, - ReplicaID: 2, - }, - Message: raftpb.Message{ - Type: raftpb.MsgSnap, - Snapshot: raftpb.Snapshot{ - Data: data, - }, - }, - }, - } - const expected = "preemptive snapshot from term 0 received" - if err := s.processPreemptiveSnapshotRequest(ctx, snapHeader, - IncomingSnapshot{ - SnapUUID: uuid.MakeV4(), - State: &storagepb.ReplicaState{Desc: repl1.Desc()}, - }); !testutils.IsPError(err, expected) { - t.Fatalf("expected %s, but found %v", expected, err) - } - - s.mu.Lock() - numPlaceholders := len(s.mu.replicaPlaceholders) - s.mu.Unlock() - - if numPlaceholders != 0 { - t.Fatalf("expected 0 placeholders, but found %d", numPlaceholders) - } - if n := atomic.LoadInt32(&s.counts.removedPlaceholders); n != 1 { - t.Fatalf("expected 1 removed placeholder, but found %d", n) - } -} - // Test that we remove snapshot placeholders when raft ignores the // snapshot. This is testing the removal of placeholder after handleRaftReady // processing for an uninitialized Replica. @@ -2928,7 +2810,7 @@ func TestStoreRemovePlaceholderOnRaftIgnored(t *testing.T) { ); err != nil { t.Fatal(err) } - uninitRepl1, err := NewReplica(&uninitDesc, s, 0) + uninitRepl1, err := newReplica(ctx, &uninitDesc, s, 2) if err != nil { t.Fatal(err) } @@ -2958,7 +2840,7 @@ func TestStoreRemovePlaceholderOnRaftIgnored(t *testing.T) { FromReplica: roachpb.ReplicaDescriptor{ NodeID: 2, StoreID: 2, - ReplicaID: 2, + ReplicaID: 3, }, Message: raftpb.Message{ Type: raftpb.MsgSnap, @@ -3306,6 +3188,105 @@ func TestSnapshotRateLimit(t *testing.T) { } } +// TestPreemptiveSnapshotsAreRemoved exercises a migration in 20.1 to remove any +// data on disk for a Replica which holds a descriptor in its state that does +// not contain the store. Prior to 20.1 such Replica state could exist on disk +// in two scenario both generally due to a rapid upgrade from 19.1 to 19.2. +// +// See the comment on removePreemptiveSnapshot(). +// +// TODO(ajwerner): remove this in 20.2 with removePreemptiveSnapshot(). +func TestPreemptiveSnapshotsAreRemoved(t *testing.T) { + defer leaktest.AfterTest(t)() + // Create a store with a preemptive snapshot sitting in its engine before it's + // started. Ensure that the preemptive snapshots are removed and the replicas + // for those ranges are not created. + // + ctx := context.Background() + stopper := stop.NewStopper() + defer stopper.Stop(ctx) + config := TestStoreConfig(hlc.NewClock(hlc.UnixNano, base.DefaultMaxClockOffset)) + s := createTestStoreWithoutStart(t, stopper, testStoreOpts{}, &config) + tablePrefix := roachpb.Key(keys.MakeTablePrefix(42)) + tablePrefixEnd := tablePrefix.PrefixEnd() + const rangeID = 42 + + desc := roachpb.NewRangeDescriptor( + rangeID, + roachpb.RKey(tablePrefix), + roachpb.RKey(tablePrefixEnd), + roachpb.MakeReplicaDescriptors([]roachpb.ReplicaDescriptor{ + { + NodeID: 2, + StoreID: 2, + ReplicaID: 2, + }, + }), + ) + const replicatedOnly, seekEnd = false, false + it := rditer.NewReplicaDataIterator(desc, s.Engine(), replicatedOnly, seekEnd) + defer it.Close() + lease := roachpb.Lease{ + Start: s.Clock().Now(), + } + state := storagepb.ReplicaState{ + Desc: desc, + Lease: &lease, + GCThreshold: &hlc.Timestamp{}, + TruncatedState: &roachpb.RaftTruncatedState{ + Term: 1, + Index: 11, + }, + Stats: &enginepb.MVCCStats{ContainsEstimates: 1}, + } + // Write some data into the range and then write the range descriptor and + // state. + func() { + b := s.engine.NewBatch() + defer b.Close() + stl := stateloader.Make(rangeID) + const N = 100 + for i := 0; i < N; i++ { + const colID = 1 + prefix := tablePrefix[0:len(tablePrefix):len(tablePrefix)] + k := roachpb.Key(encoding.EncodeIntValue(prefix, colID, int64(i))) + require.NoError(t, engine.MVCCBlindPutProto(ctx, b, state.Stats, + k, s.Clock().Now(), desc, nil)) + } + require.NoError(t, engine.MVCCBlindPutProto(ctx, b, state.Stats, + keys.RangeDescriptorKey(desc.StartKey), hlc.Timestamp{}, desc, nil)) + _, err := stl.Save(ctx, b, state, stateloader.TruncatedStateUnreplicated) + require.NoError(t, err) + require.NoError(t, b.Commit(true /* sync */)) + }() + + // Start the store. + require.NoError(t, s.Start(ctx, stopper)) + + // Ensure that the data for the preemptive snapshot has been removed. + snap := s.engine.NewSnapshot() + defer snap.Close() + it = rditer.NewReplicaDataIterator(desc, snap, replicatedOnly, seekEnd) + defer it.Close() + // The only key for the range we should find is the tombstone. + tombstoneKey := keys.RaftTombstoneKey(rangeID) + var foundTombstoneKey bool + for ; ; it.Next() { + ok, err := it.Valid() + require.NoError(t, err) + if !ok { + break + } + // There should not be any data other than the tombstone key. + if k := it.UnsafeKey(); k.Equal(engine.MVCCKey{Key: tombstoneKey}) { + foundTombstoneKey = true + } else { + t.Fatalf("found data in the range which should have been removed: %v", k) + } + } + require.True(t, foundTombstoneKey) +} + func BenchmarkStoreGetReplica(b *testing.B) { stopper := stop.NewStopper() defer stopper.Stop(context.TODO()) diff --git a/pkg/storage/stores_test.go b/pkg/storage/stores_test.go index f57fc8821ad0..f864fba244cf 100644 --- a/pkg/storage/stores_test.go +++ b/pkg/storage/stores_test.go @@ -148,7 +148,7 @@ func TestStoresGetReplicaForRangeID(t *testing.T) { }, } - replica, err := NewReplica(desc, store, 0) + replica, err := newReplica(ctx, desc, store, replicaID) if err != nil { t.Fatalf("unexpected error when creating replica: %+v", err) }