-
Notifications
You must be signed in to change notification settings - Fork 3.8k
/
raft.proto
170 lines (142 loc) · 7.16 KB
/
raft.proto
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
// Copyright 2015 The Cockroach Authors.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
// implied. See the License for the specific language governing
// permissions and limitations under the License.
//
// Author: Tamir Duberstein (tamird@gmail.com)
syntax = "proto2";
package cockroach.storage;
option go_package = "storage";
import "cockroach/pkg/roachpb/errors.proto";
import "cockroach/pkg/roachpb/metadata.proto";
import "cockroach/pkg/storage/storagebase/state.proto";
import "etcd/raft/raftpb/raft.proto";
import "gogoproto/gogo.proto";
// RaftHeartbeat is a request that contains the barebones information for a
// raftpb.MsgHeartbeat raftpb.Message. RaftHeartbeats are coalesced and sent
// in a RaftMessageRequest, and reconstructed by the receiver into individual
// raftpb.Message protos.
message RaftHeartbeat {
optional uint64 range_id = 1 [(gogoproto.nullable) = false,
(gogoproto.customname) = "RangeID",
(gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.RangeID"];
optional uint32 from_replica_id = 2 [(gogoproto.nullable) = false,
(gogoproto.customname) = "FromReplicaID",
(gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.ReplicaID"];
optional uint32 to_replica_id = 3 [(gogoproto.nullable) = false,
(gogoproto.customname) = "ToReplicaID",
(gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.ReplicaID"];
optional uint64 term = 4 [(gogoproto.nullable) = false];
optional uint64 commit = 5 [(gogoproto.nullable) = false];
optional bool quiesce = 6 [(gogoproto.nullable) = false];
}
// RaftMessageRequest is the request used to send raft messages using our
// protobuf-based RPC codec. If a RaftMessageRequest has a non-empty number of
// heartbeats or heartbeat_resps, the contents of the message field is treated
// as a dummy message and discarded. A coalesced heartbeat request's replica
// descriptor's range ID must be zero.
message RaftMessageRequest {
optional uint64 range_id = 1 [(gogoproto.nullable) = false,
(gogoproto.customname) = "RangeID",
(gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.RangeID"];
optional roachpb.ReplicaDescriptor from_replica = 2 [(gogoproto.nullable) = false];
optional roachpb.ReplicaDescriptor to_replica = 3 [(gogoproto.nullable) = false];
optional raftpb.Message message = 4 [(gogoproto.nullable) = false];
// Is this a quiesce request? A quiesce request is a MsgHeartbeat
// which is requesting the recipient to stop ticking its local
// replica as long as the current Raft state matches the heartbeat
// Term/Commit. If the Term/Commit match, the recipient is marked as
// quiescent. If they don't match, the message is passed along to
// Raft which will generate a MsgHeartbeatResp that will unquiesce
// the sender.
optional bool quiesce = 5 [(gogoproto.nullable) = false];
// A coalesced heartbeat request is any RaftMessageRequest with a nonzero number of
// heartbeats or heartbeat_resps.
repeated RaftHeartbeat heartbeats = 6 [(gogoproto.nullable) = false];
repeated RaftHeartbeat heartbeat_resps = 7 [(gogoproto.nullable) = false];
}
message RaftMessageRequestBatch {
repeated RaftMessageRequest requests = 1 [(gogoproto.nullable) = false];
}
message RaftMessageResponseUnion {
option (gogoproto.onlyone) = true;
optional roachpb.Error error = 1;
}
// RaftMessageResponse may be sent to the sender of a
// RaftMessageRequest. RaftMessage does not use the usual
// request/response pattern; it is primarily modeled as a one-way
// stream of requests. Normal 'responses' are usually sent as new
// requests on a separate stream in the other direction.
// RaftMessageResponse is not sent for every RaftMessageRequest, but
// may be used for certain error conditions.
message RaftMessageResponse {
optional uint64 range_id = 1 [(gogoproto.nullable) = false,
(gogoproto.customname) = "RangeID",
(gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.RangeID"];
optional roachpb.ReplicaDescriptor from_replica = 2 [(gogoproto.nullable) = false];
optional roachpb.ReplicaDescriptor to_replica = 3 [(gogoproto.nullable) = false];
optional RaftMessageResponseUnion union = 4 [(gogoproto.nullable) = false];
}
// SnapshotRequest is the request used to send streaming snapshot requests.
message SnapshotRequest {
message Header {
optional roachpb.RangeDescriptor range_descriptor = 1 [(gogoproto.nullable) = false];
// The replica state at the time the snapshot was generated. Note
// that ReplicaState.Desc differs from the above range_descriptor
// field which holds the updated descriptor after the new replica
// has been added while ReplicaState.Desc holds the descriptor
// before the new replica has been added.
optional storagebase.ReplicaState state = 5;
// The inner raft message is of type MsgSnap, and its snapshot data contains a UUID.
optional RaftMessageRequest raft_message_request = 2 [(gogoproto.nullable) = false];
// The estimated size of the range, to be used in reservation decisions.
optional int64 range_size = 3 [(gogoproto.nullable) = false];
// can_decline is set on preemptive snapshots, but not those generated
// by raft because at that point it is better to queue up the stream
// than to cancel it.
optional bool can_decline = 4 [(gogoproto.nullable) = false];
}
optional Header header = 1;
// A RocksDB BatchRepr. Multiple kv_batches may be sent across multiple request messages.
optional bytes kv_batch = 2 [(gogoproto.customname) = "KVBatch"];
// These are really raftpb.Entry, but we model them as raw bytes to avoid
// roundtripping through memory. They are separate from the kv_batch to
// allow flexibility in log implementations.
repeated bytes log_entries = 3;
optional bool final = 4 [(gogoproto.nullable) = false];
}
message SnapshotResponse {
enum Status {
UNKNOWN = 0;
ACCEPTED = 1;
APPLIED = 2;
ERROR = 3;
DECLINED = 4;
}
optional Status status = 1 [(gogoproto.nullable) = false];
optional string message = 2 [(gogoproto.nullable) = false];
optional roachpb.StoreCapacity store_capacity = 3;
}
// ConfChangeContext is encoded in the raftpb.ConfChange.Context field.
message ConfChangeContext {
optional string command_id = 1 [(gogoproto.nullable) = false,
(gogoproto.customname) = "CommandID"];
// Payload is the application-level command (i.e. an encoded
// storagebase.RaftCommand).
optional bytes payload = 2;
// Replica contains full details about the replica being added or removed.
optional roachpb.ReplicaDescriptor replica = 3 [(gogoproto.nullable) = false];
}
service MultiRaft {
rpc RaftMessageBatch (stream RaftMessageRequestBatch) returns (stream RaftMessageResponse) {}
rpc RaftSnapshot (stream SnapshotRequest) returns (stream SnapshotResponse) {}
}