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