1syntax = "proto3";
2package etcdserverpb;
3
4import "gogoproto/gogo.proto";
5import "etcd/mvcc/mvccpb/kv.proto";
6import "etcd/auth/authpb/auth.proto";
7
8// for grpc-gateway
9import "google/api/annotations.proto";
10
11option (gogoproto.marshaler_all) = true;
12option (gogoproto.unmarshaler_all) = true;
13
14service KV {
15  // Range gets the keys in the range from the key-value store.
16  rpc Range(RangeRequest) returns (RangeResponse) {
17      option (google.api.http) = {
18        post: "/v3/kv/range"
19        body: "*"
20    };
21  }
22
23  // Put puts the given key into the key-value store.
24  // A put request increments the revision of the key-value store
25  // and generates one event in the event history.
26  rpc Put(PutRequest) returns (PutResponse) {
27      option (google.api.http) = {
28        post: "/v3/kv/put"
29        body: "*"
30    };
31  }
32
33  // DeleteRange deletes the given range from the key-value store.
34  // A delete request increments the revision of the key-value store
35  // and generates a delete event in the event history for every deleted key.
36  rpc DeleteRange(DeleteRangeRequest) returns (DeleteRangeResponse) {
37      option (google.api.http) = {
38        post: "/v3/kv/deleterange"
39        body: "*"
40    };
41  }
42
43  // Txn processes multiple requests in a single transaction.
44  // A txn request increments the revision of the key-value store
45  // and generates events with the same revision for every completed request.
46  // It is not allowed to modify the same key several times within one txn.
47  rpc Txn(TxnRequest) returns (TxnResponse) {
48      option (google.api.http) = {
49        post: "/v3/kv/txn"
50        body: "*"
51    };
52  }
53
54  // Compact compacts the event history in the etcd key-value store. The key-value
55  // store should be periodically compacted or the event history will continue to grow
56  // indefinitely.
57  rpc Compact(CompactionRequest) returns (CompactionResponse) {
58      option (google.api.http) = {
59        post: "/v3/kv/compaction"
60        body: "*"
61    };
62  }
63}
64
65service Watch {
66  // Watch watches for events happening or that have happened. Both input and output
67  // are streams; the input stream is for creating and canceling watchers and the output
68  // stream sends events. One watch RPC can watch on multiple key ranges, streaming events
69  // for several watches at once. The entire event history can be watched starting from the
70  // last compaction revision.
71  rpc Watch(stream WatchRequest) returns (stream WatchResponse) {
72      option (google.api.http) = {
73        post: "/v3/watch"
74        body: "*"
75    };
76  }
77}
78
79service Lease {
80  // LeaseGrant creates a lease which expires if the server does not receive a keepAlive
81  // within a given time to live period. All keys attached to the lease will be expired and
82  // deleted if the lease expires. Each expired key generates a delete event in the event history.
83  rpc LeaseGrant(LeaseGrantRequest) returns (LeaseGrantResponse) {
84      option (google.api.http) = {
85        post: "/v3/lease/grant"
86        body: "*"
87    };
88  }
89
90  // LeaseRevoke revokes a lease. All keys attached to the lease will expire and be deleted.
91  rpc LeaseRevoke(LeaseRevokeRequest) returns (LeaseRevokeResponse) {
92      option (google.api.http) = {
93        post: "/v3/lease/revoke"
94        body: "*"
95        additional_bindings {
96            post: "/v3/kv/lease/revoke"
97            body: "*"
98        }
99    };
100  }
101
102  // LeaseKeepAlive keeps the lease alive by streaming keep alive requests from the client
103  // to the server and streaming keep alive responses from the server to the client.
104  rpc LeaseKeepAlive(stream LeaseKeepAliveRequest) returns (stream LeaseKeepAliveResponse) {
105      option (google.api.http) = {
106        post: "/v3/lease/keepalive"
107        body: "*"
108    };
109  }
110
111  // LeaseTimeToLive retrieves lease information.
112  rpc LeaseTimeToLive(LeaseTimeToLiveRequest) returns (LeaseTimeToLiveResponse) {
113      option (google.api.http) = {
114        post: "/v3/lease/timetolive"
115        body: "*"
116        additional_bindings {
117            post: "/v3/kv/lease/timetolive"
118            body: "*"
119        }
120    };
121  }
122
123  // LeaseLeases lists all existing leases.
124  rpc LeaseLeases(LeaseLeasesRequest) returns (LeaseLeasesResponse) {
125      option (google.api.http) = {
126        post: "/v3/lease/leases"
127        body: "*"
128        additional_bindings {
129            post: "/v3/kv/lease/leases"
130            body: "*"
131        }
132    };
133  }
134}
135
136service Cluster {
137  // MemberAdd adds a member into the cluster.
138  rpc MemberAdd(MemberAddRequest) returns (MemberAddResponse) {
139      option (google.api.http) = {
140        post: "/v3/cluster/member/add"
141        body: "*"
142    };
143  }
144
145  // MemberRemove removes an existing member from the cluster.
146  rpc MemberRemove(MemberRemoveRequest) returns (MemberRemoveResponse) {
147      option (google.api.http) = {
148        post: "/v3/cluster/member/remove"
149        body: "*"
150    };
151  }
152
153  // MemberUpdate updates the member configuration.
154  rpc MemberUpdate(MemberUpdateRequest) returns (MemberUpdateResponse) {
155      option (google.api.http) = {
156        post: "/v3/cluster/member/update"
157        body: "*"
158    };
159  }
160
161  // MemberList lists all the members in the cluster.
162  rpc MemberList(MemberListRequest) returns (MemberListResponse) {
163      option (google.api.http) = {
164        post: "/v3/cluster/member/list"
165        body: "*"
166    };
167  }
168
169  // MemberPromote promotes a member from raft learner (non-voting) to raft voting member.
170  rpc MemberPromote(MemberPromoteRequest) returns (MemberPromoteResponse) {
171      option (google.api.http) = {
172        post: "/v3/cluster/member/promote"
173        body: "*"
174    };
175  }
176}
177
178service Maintenance {
179  // Alarm activates, deactivates, and queries alarms regarding cluster health.
180  rpc Alarm(AlarmRequest) returns (AlarmResponse) {
181      option (google.api.http) = {
182        post: "/v3/maintenance/alarm"
183        body: "*"
184    };
185  }
186
187  // Status gets the status of the member.
188  rpc Status(StatusRequest) returns (StatusResponse) {
189      option (google.api.http) = {
190        post: "/v3/maintenance/status"
191        body: "*"
192    };
193  }
194
195  // Defragment defragments a member's backend database to recover storage space.
196  rpc Defragment(DefragmentRequest) returns (DefragmentResponse) {
197      option (google.api.http) = {
198        post: "/v3/maintenance/defragment"
199        body: "*"
200    };
201  }
202
203  // Hash computes the hash of whole backend keyspace,
204  // including key, lease, and other buckets in storage.
205  // This is designed for testing ONLY!
206  // Do not rely on this in production with ongoing transactions,
207  // since Hash operation does not hold MVCC locks.
208  // Use "HashKV" API instead for "key" bucket consistency checks.
209  rpc Hash(HashRequest) returns (HashResponse) {
210      option (google.api.http) = {
211        post: "/v3/maintenance/hash"
212        body: "*"
213    };
214  }
215
216  // HashKV computes the hash of all MVCC keys up to a given revision.
217  // It only iterates "key" bucket in backend storage.
218  rpc HashKV(HashKVRequest) returns (HashKVResponse) {
219      option (google.api.http) = {
220        post: "/v3/maintenance/hash"
221        body: "*"
222    };
223  }
224
225  // Snapshot sends a snapshot of the entire backend from a member over a stream to a client.
226  rpc Snapshot(SnapshotRequest) returns (stream SnapshotResponse) {
227      option (google.api.http) = {
228        post: "/v3/maintenance/snapshot"
229        body: "*"
230    };
231  }
232
233  // MoveLeader requests current leader node to transfer its leadership to transferee.
234  rpc MoveLeader(MoveLeaderRequest) returns (MoveLeaderResponse) {
235      option (google.api.http) = {
236        post: "/v3/maintenance/transfer-leadership"
237        body: "*"
238    };
239  }
240}
241
242service Auth {
243  // AuthEnable enables authentication.
244  rpc AuthEnable(AuthEnableRequest) returns (AuthEnableResponse) {
245      option (google.api.http) = {
246        post: "/v3/auth/enable"
247        body: "*"
248    };
249  }
250
251  // AuthDisable disables authentication.
252  rpc AuthDisable(AuthDisableRequest) returns (AuthDisableResponse) {
253      option (google.api.http) = {
254        post: "/v3/auth/disable"
255        body: "*"
256    };
257  }
258
259  // Authenticate processes an authenticate request.
260  rpc Authenticate(AuthenticateRequest) returns (AuthenticateResponse) {
261      option (google.api.http) = {
262        post: "/v3/auth/authenticate"
263        body: "*"
264    };
265  }
266
267  // UserAdd adds a new user. User name cannot be empty.
268  rpc UserAdd(AuthUserAddRequest) returns (AuthUserAddResponse) {
269      option (google.api.http) = {
270        post: "/v3/auth/user/add"
271        body: "*"
272    };
273  }
274
275  // UserGet gets detailed user information.
276  rpc UserGet(AuthUserGetRequest) returns (AuthUserGetResponse) {
277      option (google.api.http) = {
278        post: "/v3/auth/user/get"
279        body: "*"
280    };
281  }
282
283  // UserList gets a list of all users.
284  rpc UserList(AuthUserListRequest) returns (AuthUserListResponse) {
285      option (google.api.http) = {
286        post: "/v3/auth/user/list"
287        body: "*"
288    };
289  }
290
291  // UserDelete deletes a specified user.
292  rpc UserDelete(AuthUserDeleteRequest) returns (AuthUserDeleteResponse) {
293      option (google.api.http) = {
294        post: "/v3/auth/user/delete"
295        body: "*"
296    };
297  }
298
299  // UserChangePassword changes the password of a specified user.
300  rpc UserChangePassword(AuthUserChangePasswordRequest) returns (AuthUserChangePasswordResponse) {
301      option (google.api.http) = {
302        post: "/v3/auth/user/changepw"
303        body: "*"
304    };
305  }
306
307  // UserGrant grants a role to a specified user.
308  rpc UserGrantRole(AuthUserGrantRoleRequest) returns (AuthUserGrantRoleResponse) {
309      option (google.api.http) = {
310        post: "/v3/auth/user/grant"
311        body: "*"
312    };
313  }
314
315  // UserRevokeRole revokes a role of specified user.
316  rpc UserRevokeRole(AuthUserRevokeRoleRequest) returns (AuthUserRevokeRoleResponse) {
317      option (google.api.http) = {
318        post: "/v3/auth/user/revoke"
319        body: "*"
320    };
321  }
322
323  // RoleAdd adds a new role. Role name cannot be empty.
324  rpc RoleAdd(AuthRoleAddRequest) returns (AuthRoleAddResponse) {
325      option (google.api.http) = {
326        post: "/v3/auth/role/add"
327        body: "*"
328    };
329  }
330
331  // RoleGet gets detailed role information.
332  rpc RoleGet(AuthRoleGetRequest) returns (AuthRoleGetResponse) {
333      option (google.api.http) = {
334        post: "/v3/auth/role/get"
335        body: "*"
336    };
337  }
338
339  // RoleList gets lists of all roles.
340  rpc RoleList(AuthRoleListRequest) returns (AuthRoleListResponse) {
341      option (google.api.http) = {
342        post: "/v3/auth/role/list"
343        body: "*"
344    };
345  }
346
347  // RoleDelete deletes a specified role.
348  rpc RoleDelete(AuthRoleDeleteRequest) returns (AuthRoleDeleteResponse) {
349      option (google.api.http) = {
350        post: "/v3/auth/role/delete"
351        body: "*"
352    };
353  }
354
355  // RoleGrantPermission grants a permission of a specified key or range to a specified role.
356  rpc RoleGrantPermission(AuthRoleGrantPermissionRequest) returns (AuthRoleGrantPermissionResponse) {
357      option (google.api.http) = {
358        post: "/v3/auth/role/grant"
359        body: "*"
360    };
361  }
362
363  // RoleRevokePermission revokes a key or range permission of a specified role.
364  rpc RoleRevokePermission(AuthRoleRevokePermissionRequest) returns (AuthRoleRevokePermissionResponse) {
365      option (google.api.http) = {
366        post: "/v3/auth/role/revoke"
367        body: "*"
368    };
369  }
370}
371
372message ResponseHeader {
373  // cluster_id is the ID of the cluster which sent the response.
374  uint64 cluster_id = 1;
375  // member_id is the ID of the member which sent the response.
376  uint64 member_id = 2;
377  // revision is the key-value store revision when the request was applied.
378  // For watch progress responses, the header.revision indicates progress. All future events
379  // recieved in this stream are guaranteed to have a higher revision number than the
380  // header.revision number.
381  int64 revision = 3;
382  // raft_term is the raft term when the request was applied.
383  uint64 raft_term = 4;
384}
385
386message RangeRequest {
387  enum SortOrder {
388	NONE = 0; // default, no sorting
389	ASCEND = 1; // lowest target value first
390	DESCEND = 2; // highest target value first
391  }
392  enum SortTarget {
393	KEY = 0;
394	VERSION = 1;
395	CREATE = 2;
396	MOD = 3;
397	VALUE = 4;
398  }
399
400  // key is the first key for the range. If range_end is not given, the request only looks up key.
401  bytes key = 1;
402  // range_end is the upper bound on the requested range [key, range_end).
403  // If range_end is '\0', the range is all keys >= key.
404  // If range_end is key plus one (e.g., "aa"+1 == "ab", "a\xff"+1 == "b"),
405  // then the range request gets all keys prefixed with key.
406  // If both key and range_end are '\0', then the range request returns all keys.
407  bytes range_end = 2;
408  // limit is a limit on the number of keys returned for the request. When limit is set to 0,
409  // it is treated as no limit.
410  int64 limit = 3;
411  // revision is the point-in-time of the key-value store to use for the range.
412  // If revision is less or equal to zero, the range is over the newest key-value store.
413  // If the revision has been compacted, ErrCompacted is returned as a response.
414  int64 revision = 4;
415
416  // sort_order is the order for returned sorted results.
417  SortOrder sort_order = 5;
418
419  // sort_target is the key-value field to use for sorting.
420  SortTarget sort_target = 6;
421
422  // serializable sets the range request to use serializable member-local reads.
423  // Range requests are linearizable by default; linearizable requests have higher
424  // latency and lower throughput than serializable requests but reflect the current
425  // consensus of the cluster. For better performance, in exchange for possible stale reads,
426  // a serializable range request is served locally without needing to reach consensus
427  // with other nodes in the cluster.
428  bool serializable = 7;
429
430  // keys_only when set returns only the keys and not the values.
431  bool keys_only = 8;
432
433  // count_only when set returns only the count of the keys in the range.
434  bool count_only = 9;
435
436  // min_mod_revision is the lower bound for returned key mod revisions; all keys with
437  // lesser mod revisions will be filtered away.
438  int64 min_mod_revision = 10;
439
440  // max_mod_revision is the upper bound for returned key mod revisions; all keys with
441  // greater mod revisions will be filtered away.
442  int64 max_mod_revision = 11;
443
444  // min_create_revision is the lower bound for returned key create revisions; all keys with
445  // lesser create revisions will be filtered away.
446  int64 min_create_revision = 12;
447
448  // max_create_revision is the upper bound for returned key create revisions; all keys with
449  // greater create revisions will be filtered away.
450  int64 max_create_revision = 13;
451}
452
453message RangeResponse {
454  ResponseHeader header = 1;
455  // kvs is the list of key-value pairs matched by the range request.
456  // kvs is empty when count is requested.
457  repeated mvccpb.KeyValue kvs = 2;
458  // more indicates if there are more keys to return in the requested range.
459  bool more = 3;
460  // count is set to the number of keys within the range when requested.
461  int64 count = 4;
462}
463
464message PutRequest {
465  // key is the key, in bytes, to put into the key-value store.
466  bytes key = 1;
467  // value is the value, in bytes, to associate with the key in the key-value store.
468  bytes value = 2;
469  // lease is the lease ID to associate with the key in the key-value store. A lease
470  // value of 0 indicates no lease.
471  int64 lease = 3;
472
473  // If prev_kv is set, etcd gets the previous key-value pair before changing it.
474  // The previous key-value pair will be returned in the put response.
475  bool prev_kv = 4;
476
477  // If ignore_value is set, etcd updates the key using its current value.
478  // Returns an error if the key does not exist.
479  bool ignore_value = 5;
480
481  // If ignore_lease is set, etcd updates the key using its current lease.
482  // Returns an error if the key does not exist.
483  bool ignore_lease = 6;
484}
485
486message PutResponse {
487  ResponseHeader header = 1;
488  // if prev_kv is set in the request, the previous key-value pair will be returned.
489  mvccpb.KeyValue prev_kv = 2;
490}
491
492message DeleteRangeRequest {
493  // key is the first key to delete in the range.
494  bytes key = 1;
495  // range_end is the key following the last key to delete for the range [key, range_end).
496  // If range_end is not given, the range is defined to contain only the key argument.
497  // If range_end is one bit larger than the given key, then the range is all the keys
498  // with the prefix (the given key).
499  // If range_end is '\0', the range is all keys greater than or equal to the key argument.
500  bytes range_end = 2;
501
502  // If prev_kv is set, etcd gets the previous key-value pairs before deleting it.
503  // The previous key-value pairs will be returned in the delete response.
504  bool prev_kv = 3;
505}
506
507message DeleteRangeResponse {
508  ResponseHeader header = 1;
509  // deleted is the number of keys deleted by the delete range request.
510  int64 deleted = 2;
511  // if prev_kv is set in the request, the previous key-value pairs will be returned.
512  repeated mvccpb.KeyValue prev_kvs = 3;
513}
514
515message RequestOp {
516  // request is a union of request types accepted by a transaction.
517  oneof request {
518    RangeRequest request_range = 1;
519    PutRequest request_put = 2;
520    DeleteRangeRequest request_delete_range = 3;
521    TxnRequest request_txn = 4;
522  }
523}
524
525message ResponseOp {
526  // response is a union of response types returned by a transaction.
527  oneof response {
528    RangeResponse response_range = 1;
529    PutResponse response_put = 2;
530    DeleteRangeResponse response_delete_range = 3;
531    TxnResponse response_txn = 4;
532  }
533}
534
535message Compare {
536  enum CompareResult {
537    EQUAL = 0;
538    GREATER = 1;
539    LESS = 2;
540    NOT_EQUAL = 3;
541  }
542  enum CompareTarget {
543    VERSION = 0;
544    CREATE = 1;
545    MOD = 2;
546    VALUE = 3;
547    LEASE = 4;
548  }
549  // result is logical comparison operation for this comparison.
550  CompareResult result = 1;
551  // target is the key-value field to inspect for the comparison.
552  CompareTarget target = 2;
553  // key is the subject key for the comparison operation.
554  bytes key = 3;
555  oneof target_union {
556    // version is the version of the given key
557    int64 version = 4;
558    // create_revision is the creation revision of the given key
559    int64 create_revision = 5;
560    // mod_revision is the last modified revision of the given key.
561    int64 mod_revision = 6;
562    // value is the value of the given key, in bytes.
563    bytes value = 7;
564    // lease is the lease id of the given key.
565    int64 lease = 8;
566    // leave room for more target_union field tags, jump to 64
567  }
568
569  // range_end compares the given target to all keys in the range [key, range_end).
570  // See RangeRequest for more details on key ranges.
571  bytes range_end = 64;
572  // TODO: fill out with most of the rest of RangeRequest fields when needed.
573}
574
575// From google paxosdb paper:
576// Our implementation hinges around a powerful primitive which we call MultiOp. All other database
577// operations except for iteration are implemented as a single call to MultiOp. A MultiOp is applied atomically
578// and consists of three components:
579// 1. A list of tests called guard. Each test in guard checks a single entry in the database. It may check
580// for the absence or presence of a value, or compare with a given value. Two different tests in the guard
581// may apply to the same or different entries in the database. All tests in the guard are applied and
582// MultiOp returns the results. If all tests are true, MultiOp executes t op (see item 2 below), otherwise
583// it executes f op (see item 3 below).
584// 2. A list of database operations called t op. Each operation in the list is either an insert, delete, or
585// lookup operation, and applies to a single database entry. Two different operations in the list may apply
586// to the same or different entries in the database. These operations are executed
587// if guard evaluates to
588// true.
589// 3. A list of database operations called f op. Like t op, but executed if guard evaluates to false.
590message TxnRequest {
591  // compare is a list of predicates representing a conjunction of terms.
592  // If the comparisons succeed, then the success requests will be processed in order,
593  // and the response will contain their respective responses in order.
594  // If the comparisons fail, then the failure requests will be processed in order,
595  // and the response will contain their respective responses in order.
596  repeated Compare compare = 1;
597  // success is a list of requests which will be applied when compare evaluates to true.
598  repeated RequestOp success = 2;
599  // failure is a list of requests which will be applied when compare evaluates to false.
600  repeated RequestOp failure = 3;
601}
602
603message TxnResponse {
604  ResponseHeader header = 1;
605  // succeeded is set to true if the compare evaluated to true or false otherwise.
606  bool succeeded = 2;
607  // responses is a list of responses corresponding to the results from applying
608  // success if succeeded is true or failure if succeeded is false.
609  repeated ResponseOp responses = 3;
610}
611
612// CompactionRequest compacts the key-value store up to a given revision. All superseded keys
613// with a revision less than the compaction revision will be removed.
614message CompactionRequest {
615  // revision is the key-value store revision for the compaction operation.
616  int64 revision = 1;
617  // physical is set so the RPC will wait until the compaction is physically
618  // applied to the local database such that compacted entries are totally
619  // removed from the backend database.
620  bool physical = 2;
621}
622
623message CompactionResponse {
624  ResponseHeader header = 1;
625}
626
627message HashRequest {
628}
629
630message HashKVRequest {
631  // revision is the key-value store revision for the hash operation.
632  int64 revision = 1;
633}
634
635message HashKVResponse {
636  ResponseHeader header = 1;
637  // hash is the hash value computed from the responding member's MVCC keys up to a given revision.
638  uint32 hash = 2;
639  // compact_revision is the compacted revision of key-value store when hash begins.
640  int64 compact_revision = 3;
641}
642
643message HashResponse {
644  ResponseHeader header = 1;
645  // hash is the hash value computed from the responding member's KV's backend.
646  uint32 hash = 2;
647}
648
649message SnapshotRequest {
650}
651
652message SnapshotResponse {
653  // header has the current key-value store information. The first header in the snapshot
654  // stream indicates the point in time of the snapshot.
655  ResponseHeader header = 1;
656
657  // remaining_bytes is the number of blob bytes to be sent after this message
658  uint64 remaining_bytes = 2;
659
660  // blob contains the next chunk of the snapshot in the snapshot stream.
661  bytes blob = 3;
662}
663
664message WatchRequest {
665  // request_union is a request to either create a new watcher or cancel an existing watcher.
666  oneof request_union {
667    WatchCreateRequest create_request = 1;
668    WatchCancelRequest cancel_request = 2;
669    WatchProgressRequest progress_request = 3;
670  }
671}
672
673message WatchCreateRequest {
674  // key is the key to register for watching.
675  bytes key = 1;
676
677  // range_end is the end of the range [key, range_end) to watch. If range_end is not given,
678  // only the key argument is watched. If range_end is equal to '\0', all keys greater than
679  // or equal to the key argument are watched.
680  // If the range_end is one bit larger than the given key,
681  // then all keys with the prefix (the given key) will be watched.
682  bytes range_end = 2;
683
684  // start_revision is an optional revision to watch from (inclusive). No start_revision is "now".
685  int64 start_revision = 3;
686
687  // progress_notify is set so that the etcd server will periodically send a WatchResponse with
688  // no events to the new watcher if there are no recent events. It is useful when clients
689  // wish to recover a disconnected watcher starting from a recent known revision.
690  // The etcd server may decide how often it will send notifications based on current load.
691  bool progress_notify = 4;
692
693  enum FilterType {
694    // filter out put event.
695    NOPUT = 0;
696    // filter out delete event.
697    NODELETE = 1;
698  }
699
700  // filters filter the events at server side before it sends back to the watcher.
701  repeated FilterType filters = 5;
702
703  // If prev_kv is set, created watcher gets the previous KV before the event happens.
704  // If the previous KV is already compacted, nothing will be returned.
705  bool prev_kv = 6;
706
707  // If watch_id is provided and non-zero, it will be assigned to this watcher.
708  // Since creating a watcher in etcd is not a synchronous operation,
709  // this can be used ensure that ordering is correct when creating multiple
710  // watchers on the same stream. Creating a watcher with an ID already in
711  // use on the stream will cause an error to be returned.
712  int64 watch_id = 7;
713
714  // fragment enables splitting large revisions into multiple watch responses.
715  bool fragment = 8;
716}
717
718message WatchCancelRequest {
719  // watch_id is the watcher id to cancel so that no more events are transmitted.
720  int64 watch_id = 1;
721}
722
723// Requests the a watch stream progress status be sent in the watch response stream as soon as
724// possible.
725message WatchProgressRequest {
726}
727
728message WatchResponse {
729  ResponseHeader header = 1;
730  // watch_id is the ID of the watcher that corresponds to the response.
731  int64 watch_id = 2;
732
733  // created is set to true if the response is for a create watch request.
734  // The client should record the watch_id and expect to receive events for
735  // the created watcher from the same stream.
736  // All events sent to the created watcher will attach with the same watch_id.
737  bool created = 3;
738
739  // canceled is set to true if the response is for a cancel watch request.
740  // No further events will be sent to the canceled watcher.
741  bool canceled = 4;
742
743  // compact_revision is set to the minimum index if a watcher tries to watch
744  // at a compacted index.
745  //
746  // This happens when creating a watcher at a compacted revision or the watcher cannot
747  // catch up with the progress of the key-value store.
748  //
749  // The client should treat the watcher as canceled and should not try to create any
750  // watcher with the same start_revision again.
751  int64 compact_revision = 5;
752
753  // cancel_reason indicates the reason for canceling the watcher.
754  string cancel_reason = 6;
755
756  // framgment is true if large watch response was split over multiple responses.
757  bool fragment = 7;
758
759  repeated mvccpb.Event events = 11;
760}
761
762message LeaseGrantRequest {
763  // TTL is the advisory time-to-live in seconds. Expired lease will return -1.
764  int64 TTL = 1;
765  // ID is the requested ID for the lease. If ID is set to 0, the lessor chooses an ID.
766  int64 ID = 2;
767}
768
769message LeaseGrantResponse {
770  ResponseHeader header = 1;
771  // ID is the lease ID for the granted lease.
772  int64 ID = 2;
773  // TTL is the server chosen lease time-to-live in seconds.
774  int64 TTL = 3;
775  string error = 4;
776}
777
778message LeaseRevokeRequest {
779  // ID is the lease ID to revoke. When the ID is revoked, all associated keys will be deleted.
780  int64 ID = 1;
781}
782
783message LeaseRevokeResponse {
784  ResponseHeader header = 1;
785}
786
787message LeaseCheckpoint {
788    // ID is the lease ID to checkpoint.
789  int64 ID = 1;
790
791  // Remaining_TTL is the remaining time until expiry of the lease.
792  int64 remaining_TTL = 2;
793}
794
795message LeaseCheckpointRequest {
796  repeated LeaseCheckpoint checkpoints = 1;
797}
798
799message LeaseCheckpointResponse {
800  ResponseHeader header = 1;
801}
802
803message LeaseKeepAliveRequest {
804  // ID is the lease ID for the lease to keep alive.
805  int64 ID = 1;
806}
807
808message LeaseKeepAliveResponse {
809  ResponseHeader header = 1;
810  // ID is the lease ID from the keep alive request.
811  int64 ID = 2;
812  // TTL is the new time-to-live for the lease.
813  int64 TTL = 3;
814}
815
816message LeaseTimeToLiveRequest {
817  // ID is the lease ID for the lease.
818  int64 ID = 1;
819  // keys is true to query all the keys attached to this lease.
820  bool keys = 2;
821}
822
823message LeaseTimeToLiveResponse {
824  ResponseHeader header = 1;
825  // ID is the lease ID from the keep alive request.
826  int64 ID = 2;
827  // TTL is the remaining TTL in seconds for the lease; the lease will expire in under TTL+1 seconds.
828  int64 TTL = 3;
829  // GrantedTTL is the initial granted time in seconds upon lease creation/renewal.
830  int64 grantedTTL = 4;
831  // Keys is the list of keys attached to this lease.
832  repeated bytes keys = 5;
833}
834
835message LeaseLeasesRequest {
836}
837
838message LeaseStatus {
839  int64 ID = 1;
840  // TODO: int64 TTL = 2;
841}
842
843message LeaseLeasesResponse {
844  ResponseHeader header = 1;
845  repeated LeaseStatus leases = 2;
846}
847
848message Member {
849  // ID is the member ID for this member.
850  uint64 ID = 1;
851  // name is the human-readable name of the member. If the member is not started, the name will be an empty string.
852  string name = 2;
853  // peerURLs is the list of URLs the member exposes to the cluster for communication.
854  repeated string peerURLs = 3;
855  // clientURLs is the list of URLs the member exposes to clients for communication. If the member is not started, clientURLs will be empty.
856  repeated string clientURLs = 4;
857  // isLearner indicates if the member is raft learner.
858  bool isLearner = 5;
859}
860
861message MemberAddRequest {
862  // peerURLs is the list of URLs the added member will use to communicate with the cluster.
863  repeated string peerURLs = 1;
864  // isLearner indicates if the added member is raft learner.
865  bool isLearner = 2;
866}
867
868message MemberAddResponse {
869  ResponseHeader header = 1;
870  // member is the member information for the added member.
871  Member member = 2;
872  // members is a list of all members after adding the new member.
873  repeated Member members = 3;
874}
875
876message MemberRemoveRequest {
877  // ID is the member ID of the member to remove.
878  uint64 ID = 1;
879}
880
881message MemberRemoveResponse {
882  ResponseHeader header = 1;
883  // members is a list of all members after removing the member.
884  repeated Member members = 2;
885}
886
887message MemberUpdateRequest {
888  // ID is the member ID of the member to update.
889  uint64 ID = 1;
890  // peerURLs is the new list of URLs the member will use to communicate with the cluster.
891  repeated string peerURLs = 2;
892}
893
894message MemberUpdateResponse{
895  ResponseHeader header = 1;
896  // members is a list of all members after updating the member.
897  repeated Member members = 2;
898}
899
900message MemberListRequest {
901}
902
903message MemberListResponse {
904  ResponseHeader header = 1;
905  // members is a list of all members associated with the cluster.
906  repeated Member members = 2;
907}
908
909message MemberPromoteRequest {
910  // ID is the member ID of the member to promote.
911  uint64 ID = 1;
912}
913
914message MemberPromoteResponse {
915  ResponseHeader header = 1;
916  // members is a list of all members after promoting the member.
917  repeated Member members = 2;
918}
919
920message DefragmentRequest {
921}
922
923message DefragmentResponse {
924  ResponseHeader header = 1;
925}
926
927message MoveLeaderRequest {
928  // targetID is the node ID for the new leader.
929  uint64 targetID = 1;
930}
931
932message MoveLeaderResponse {
933  ResponseHeader header = 1;
934}
935
936enum AlarmType {
937	NONE = 0; // default, used to query if any alarm is active
938	NOSPACE = 1; // space quota is exhausted
939	CORRUPT = 2; // kv store corruption detected
940}
941
942message AlarmRequest {
943  enum AlarmAction {
944	GET = 0;
945	ACTIVATE = 1;
946	DEACTIVATE = 2;
947  }
948  // action is the kind of alarm request to issue. The action
949  // may GET alarm statuses, ACTIVATE an alarm, or DEACTIVATE a
950  // raised alarm.
951  AlarmAction action = 1;
952  // memberID is the ID of the member associated with the alarm. If memberID is 0, the
953  // alarm request covers all members.
954  uint64 memberID = 2;
955  // alarm is the type of alarm to consider for this request.
956  AlarmType alarm = 3;
957}
958
959message AlarmMember {
960  // memberID is the ID of the member associated with the raised alarm.
961  uint64 memberID = 1;
962  // alarm is the type of alarm which has been raised.
963  AlarmType alarm = 2;
964}
965
966message AlarmResponse {
967  ResponseHeader header = 1;
968  // alarms is a list of alarms associated with the alarm request.
969  repeated AlarmMember alarms = 2;
970}
971
972message StatusRequest {
973}
974
975message StatusResponse {
976  ResponseHeader header = 1;
977  // version is the cluster protocol version used by the responding member.
978  string version = 2;
979  // dbSize is the size of the backend database physically allocated, in bytes, of the responding member.
980  int64 dbSize = 3;
981  // leader is the member ID which the responding member believes is the current leader.
982  uint64 leader = 4;
983  // raftIndex is the current raft committed index of the responding member.
984  uint64 raftIndex = 5;
985  // raftTerm is the current raft term of the responding member.
986  uint64 raftTerm = 6;
987  // raftAppliedIndex is the current raft applied index of the responding member.
988  uint64 raftAppliedIndex = 7;
989  // errors contains alarm/health information and status.
990  repeated string errors = 8;
991  // dbSizeInUse is the size of the backend database logically in use, in bytes, of the responding member.
992  int64 dbSizeInUse = 9;
993  // isLearner indicates if the member is raft learner.
994  bool isLearner = 10;
995}
996
997message AuthEnableRequest {
998}
999
1000message AuthDisableRequest {
1001}
1002
1003message AuthenticateRequest {
1004  string name = 1;
1005  string password = 2;
1006}
1007
1008message AuthUserAddRequest {
1009  string name = 1;
1010  string password = 2;
1011  authpb.UserAddOptions options = 3;
1012}
1013
1014message AuthUserGetRequest {
1015  string name = 1;
1016}
1017
1018message AuthUserDeleteRequest {
1019  // name is the name of the user to delete.
1020  string name = 1;
1021}
1022
1023message AuthUserChangePasswordRequest {
1024  // name is the name of the user whose password is being changed.
1025  string name = 1;
1026  // password is the new password for the user.
1027  string password = 2;
1028}
1029
1030message AuthUserGrantRoleRequest {
1031  // user is the name of the user which should be granted a given role.
1032  string user = 1;
1033  // role is the name of the role to grant to the user.
1034  string role = 2;
1035}
1036
1037message AuthUserRevokeRoleRequest {
1038  string name = 1;
1039  string role = 2;
1040}
1041
1042message AuthRoleAddRequest {
1043  // name is the name of the role to add to the authentication system.
1044  string name = 1;
1045}
1046
1047message AuthRoleGetRequest {
1048  string role = 1;
1049}
1050
1051message AuthUserListRequest {
1052}
1053
1054message AuthRoleListRequest {
1055}
1056
1057message AuthRoleDeleteRequest {
1058  string role = 1;
1059}
1060
1061message AuthRoleGrantPermissionRequest {
1062  // name is the name of the role which will be granted the permission.
1063  string name = 1;
1064  // perm is the permission to grant to the role.
1065  authpb.Permission perm = 2;
1066}
1067
1068message AuthRoleRevokePermissionRequest {
1069  string role = 1;
1070  bytes key = 2;
1071  bytes range_end = 3;
1072}
1073
1074message AuthEnableResponse {
1075  ResponseHeader header = 1;
1076}
1077
1078message AuthDisableResponse {
1079  ResponseHeader header = 1;
1080}
1081
1082message AuthenticateResponse {
1083  ResponseHeader header = 1;
1084  // token is an authorized token that can be used in succeeding RPCs
1085  string token = 2;
1086}
1087
1088message AuthUserAddResponse {
1089  ResponseHeader header = 1;
1090}
1091
1092message AuthUserGetResponse {
1093  ResponseHeader header = 1;
1094
1095  repeated string roles = 2;
1096}
1097
1098message AuthUserDeleteResponse {
1099  ResponseHeader header = 1;
1100}
1101
1102message AuthUserChangePasswordResponse {
1103  ResponseHeader header = 1;
1104}
1105
1106message AuthUserGrantRoleResponse {
1107  ResponseHeader header = 1;
1108}
1109
1110message AuthUserRevokeRoleResponse {
1111  ResponseHeader header = 1;
1112}
1113
1114message AuthRoleAddResponse {
1115  ResponseHeader header = 1;
1116}
1117
1118message AuthRoleGetResponse {
1119  ResponseHeader header = 1;
1120
1121  repeated authpb.Permission perm = 2;
1122}
1123
1124message AuthRoleListResponse {
1125  ResponseHeader header = 1;
1126
1127  repeated string roles = 2;
1128}
1129
1130message AuthUserListResponse {
1131  ResponseHeader header = 1;
1132
1133  repeated string users = 2;
1134}
1135
1136message AuthRoleDeleteResponse {
1137  ResponseHeader header = 1;
1138}
1139
1140message AuthRoleGrantPermissionResponse {
1141  ResponseHeader header = 1;
1142}
1143
1144message AuthRoleRevokePermissionResponse {
1145  ResponseHeader header = 1;
1146}
1147