From 03a8503c0c96ff902b669ebe4e8ba24934d54385 Mon Sep 17 00:00:00 2001 From: Joe Betz Date: Tue, 19 Jun 2018 16:23:25 -0700 Subject: [PATCH] *: Add progress notify request watch request --- Documentation/dev-guide/api_reference_v3.md | 11 +- .../apispec/swagger/rpc.swagger.json | 9 +- .../apispec/swagger/v3election.swagger.json | 2 +- .../apispec/swagger/v3lock.swagger.json | 2 +- clientv3/integration/watch_test.go | 46 + clientv3/watch.go | 102 +- etcdctl/ctlv3/command/watch_command.go | 41 +- etcdserver/api/v3rpc/watch.go | 7 +- etcdserver/etcdserverpb/etcdserver.pb.go | 1 + etcdserver/etcdserverpb/rpc.pb.go | 921 +++++++++++------- etcdserver/etcdserverpb/rpc.proto | 9 + 11 files changed, 737 insertions(+), 414 deletions(-) diff --git a/Documentation/dev-guide/api_reference_v3.md b/Documentation/dev-guide/api_reference_v3.md index b3481fa83cfa..3278a55e7548 100644 --- a/Documentation/dev-guide/api_reference_v3.md +++ b/Documentation/dev-guide/api_reference_v3.md @@ -740,7 +740,7 @@ Empty field. | ----- | ----------- | ---- | | cluster_id | cluster_id is the ID of the cluster which sent the response. | uint64 | | member_id | member_id is the ID of the member which sent the response. | uint64 | -| revision | revision is the key-value store revision when the request was applied. | int64 | +| revision | revision is the key-value store revision when the request was applied. For watch progress responses, the header.revision indicates progress. All future events recieved in this stream are guarenteed to have a higher revision number than the header.revision number. | int64 | | raft_term | raft_term is the raft term when the request was applied. | uint64 | @@ -840,6 +840,14 @@ From google paxosdb paper: Our implementation hinges around a powerful primitive +##### message `WatchProgressRequest` (etcdserver/etcdserverpb/rpc.proto) + +Requests the a watch stream progress status be sent in the watch response stream as soon as possible. + +Empty field. + + + ##### message `WatchRequest` (etcdserver/etcdserverpb/rpc.proto) | Field | Description | Type | @@ -847,6 +855,7 @@ From google paxosdb paper: Our implementation hinges around a powerful primitive | request_union | request_union is a request to either create a new watcher or cancel an existing watcher. | oneof | | create_request | | WatchCreateRequest | | cancel_request | | WatchCancelRequest | +| progress_request | | WatchProgressRequest | diff --git a/Documentation/dev-guide/apispec/swagger/rpc.swagger.json b/Documentation/dev-guide/apispec/swagger/rpc.swagger.json index a0217d29d6bf..f9099675419f 100644 --- a/Documentation/dev-guide/apispec/swagger/rpc.swagger.json +++ b/Documentation/dev-guide/apispec/swagger/rpc.swagger.json @@ -2195,7 +2195,7 @@ "format": "uint64" }, "revision": { - "description": "revision is the key-value store revision when the request was applied.", + "description": "revision is the key-value store revision when the request was applied.\nFor watch progress responses, the header.revision indicates progress. All future events\nrecieved in this stream are guarenteed to have a higher revision number than the\nheader.revision number.", "type": "string", "format": "int64" } @@ -2396,6 +2396,10 @@ } } }, + "etcdserverpbWatchProgressRequest": { + "description": "Requests the a watch stream progress status be sent in the watch response stream as soon as\npossible.", + "type": "object" + }, "etcdserverpbWatchRequest": { "type": "object", "properties": { @@ -2404,6 +2408,9 @@ }, "create_request": { "$ref": "#/definitions/etcdserverpbWatchCreateRequest" + }, + "progress_request": { + "$ref": "#/definitions/etcdserverpbWatchProgressRequest" } } }, diff --git a/Documentation/dev-guide/apispec/swagger/v3election.swagger.json b/Documentation/dev-guide/apispec/swagger/v3election.swagger.json index bdb9003fe492..db46ddef9994 100644 --- a/Documentation/dev-guide/apispec/swagger/v3election.swagger.json +++ b/Documentation/dev-guide/apispec/swagger/v3election.swagger.json @@ -168,7 +168,7 @@ "revision": { "type": "string", "format": "int64", - "description": "revision is the key-value store revision when the request was applied." + "description": "revision is the key-value store revision when the request was applied.\nFor watch progress responses, the header.revision indicates progress. All future events\nrecieved in this stream are guarenteed to have a higher revision number than the\nheader.revision number." }, "raft_term": { "type": "string", diff --git a/Documentation/dev-guide/apispec/swagger/v3lock.swagger.json b/Documentation/dev-guide/apispec/swagger/v3lock.swagger.json index 9666bf587727..aa14511d258a 100644 --- a/Documentation/dev-guide/apispec/swagger/v3lock.swagger.json +++ b/Documentation/dev-guide/apispec/swagger/v3lock.swagger.json @@ -87,7 +87,7 @@ "revision": { "type": "string", "format": "int64", - "description": "revision is the key-value store revision when the request was applied." + "description": "revision is the key-value store revision when the request was applied.\nFor watch progress responses, the header.revision indicates progress. All future events\nrecieved in this stream are guarenteed to have a higher revision number than the\nheader.revision number." }, "raft_term": { "type": "string", diff --git a/clientv3/integration/watch_test.go b/clientv3/integration/watch_test.go index efe4387abdd4..accb1a28f787 100644 --- a/clientv3/integration/watch_test.go +++ b/clientv3/integration/watch_test.go @@ -582,6 +582,52 @@ func testWatchWithProgressNotify(t *testing.T, watchOnPut bool) { } } +func TestWatchRequestProgress(t *testing.T) { + defer testutil.AfterTest(t) + + watchTimeout := 3 * time.Second + + clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3}) + defer clus.Terminate(t) + + wc := clus.RandClient() + + rch := wc.Watch(context.Background(), "/", clientv3.WithPrefix()) + + _, err := wc.Put(context.Background(), "/a", "1") + if err != nil { + t.Fatal(err) + } + + select { + case resp := <-rch: // wait for notification + if len(resp.Events) != 1 { + t.Fatalf("resp.Events expected 1, got %d", len(resp.Events)) + } + case <-time.After(watchTimeout): + t.Fatalf("watch response expected in %v, but timed out", watchTimeout) + } + + for i := 0; i < 2; i++ { // ensure progress does not change if the only requests are for progress + err = wc.RequestProgress(context.Background()) + if err != nil { + t.Fatal(err) + } + + select { + case resp := <-rch: + if !resp.IsProgressNotify() { + t.Fatalf("expected resp.IsProgressNotify() == true") + } + if resp.Header.Revision != 2 { + t.Fatalf("resp.Header.Revision expected 2, got %d", resp.Header.Revision) + } + case <-time.After(watchTimeout): + t.Fatalf("watch response expected in %v, but timed out", watchTimeout) + } + } +} + func TestWatchEventType(t *testing.T) { cluster := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1}) defer cluster.Terminate(t) diff --git a/clientv3/watch.go b/clientv3/watch.go index 39d149c530ed..c9ab60d79219 100644 --- a/clientv3/watch.go +++ b/clientv3/watch.go @@ -66,6 +66,9 @@ type Watcher interface { // Close closes the watcher and cancels all watch requests. Close() error + + // RequestProgress requests a progress notify response be sent in all WatchChans. + RequestProgress(ctx context.Context) error } type WatchResponse struct { @@ -150,7 +153,7 @@ type watchGrpcStream struct { resuming []*watcherStream // reqc sends a watch request from Watch() to the main goroutine - reqc chan *watchRequest + reqc chan watchStreamRequest // respc receives data from the watch client respc chan *pb.WatchResponse // donec closes to broadcast shutdown @@ -168,6 +171,11 @@ type watchGrpcStream struct { closeErr error } +// watchStreamRequest is a union of the supported watch request operation types +type watchStreamRequest interface { + toPB() *pb.WatchRequest +} + // watchRequest is issued by the subscriber to start a new watcher type watchRequest struct { ctx context.Context @@ -192,6 +200,10 @@ type watchRequest struct { retc chan chan WatchResponse } +// progressRequest is issued by the subscriber to request watch progress +type progressRequest struct { +} + // watcherStream represents a registered watcher type watcherStream struct { // initReq is the request that initiated this request @@ -249,7 +261,7 @@ func (w *watcher) newWatcherGrpcStream(inctx context.Context) *watchGrpcStream { cancel: cancel, substreams: make(map[int64]*watcherStream), respc: make(chan *pb.WatchResponse), - reqc: make(chan *watchRequest), + reqc: make(chan watchStreamRequest), donec: make(chan struct{}), errc: make(chan error, 1), closingc: make(chan *watcherStream), @@ -355,6 +367,44 @@ func (w *watcher) Close() (err error) { return err } +// RequestProgress requests a progress notify response be sent in all WatchChans. +func (w *watcher) RequestProgress(ctx context.Context) (err error) { + ctxKey := streamKeyFromCtx(ctx) + + w.mu.Lock() + if w.streams == nil { + return fmt.Errorf("No stream found for context") + } + wgs := w.streams[ctxKey] + if wgs == nil { + wgs = w.newWatcherGrpcStream(ctx) + w.streams[ctxKey] = wgs + } + donec := wgs.donec + reqc := wgs.reqc + w.mu.Unlock() + + pr := &progressRequest{} + + select { + case reqc <- pr: + return nil + case <-ctx.Done(): + if err == nil { + return ctx.Err() + } + return err + case <-donec: + if wgs.closeErr != nil { + return wgs.closeErr + } + // retry; may have dropped stream from no ctxs + return w.RequestProgress(ctx) + } + // TODO: block for progress response? + return fmt.Errorf("Unexpected error submitting progress request") +} + func (w *watchGrpcStream) close() (err error) { w.cancel() <-w.donec @@ -462,26 +512,31 @@ func (w *watchGrpcStream) run() { for { select { // Watch() requested - case wreq := <-w.reqc: - outc := make(chan WatchResponse, 1) - // TODO: pass custom watch ID? - ws := &watcherStream{ - initReq: *wreq, - id: -1, - outc: outc, - // unbuffered so resumes won't cause repeat events - recvc: make(chan *WatchResponse), - } + case req := <-w.reqc: + switch wreq := req.(type) { + case *watchRequest: + outc := make(chan WatchResponse, 1) + // TODO: pass custom watch ID? + ws := &watcherStream{ + initReq: *wreq, + id: -1, + outc: outc, + // unbuffered so resumes won't cause repeat events + recvc: make(chan *WatchResponse), + } - ws.donec = make(chan struct{}) - w.wg.Add(1) - go w.serveSubstream(ws, w.resumec) + ws.donec = make(chan struct{}) + w.wg.Add(1) + go w.serveSubstream(ws, w.resumec) - // queue up for watcher creation/resume - w.resuming = append(w.resuming, ws) - if len(w.resuming) == 1 { - // head of resume queue, can register a new watcher - wc.Send(ws.initReq.toPB()) + // queue up for watcher creation/resume + w.resuming = append(w.resuming, ws) + if len(w.resuming) == 1 { + // head of resume queue, can register a new watcher + wc.Send(ws.initReq.toPB()) + } + case *progressRequest: + wc.Send(wreq.toPB()) } // new events from the watch client @@ -882,6 +937,13 @@ func (wr *watchRequest) toPB() *pb.WatchRequest { return &pb.WatchRequest{RequestUnion: cr} } +// toPB converts an internal progress request structure to its protobuf WatchRequest structure. +func (pr *progressRequest) toPB() *pb.WatchRequest { + req := &pb.WatchProgressRequest{} + cr := &pb.WatchRequest_ProgressRequest{ProgressRequest: req} + return &pb.WatchRequest{RequestUnion: cr} +} + func streamKeyFromCtx(ctx context.Context) string { if md, ok := metadata.FromOutgoingContext(ctx); ok { return fmt.Sprintf("%+v", md) diff --git a/etcdctl/ctlv3/command/watch_command.go b/etcdctl/ctlv3/command/watch_command.go index 15cc835becb4..240f46b0ff30 100644 --- a/etcdctl/ctlv3/command/watch_command.go +++ b/etcdctl/ctlv3/command/watch_command.go @@ -101,27 +101,33 @@ func watchInteractiveFunc(cmd *cobra.Command, osArgs []string, envKey, envRange l = strings.TrimSuffix(l, "\n") args := argify(l) - if len(args) < 2 && envKey == "" { - fmt.Fprintf(os.Stderr, "Invalid command %s (command type or key is not provided)\n", l) - continue - } + switch args[0] { + case "watch": + if len(args) < 2 && envKey == "" { + fmt.Fprintf(os.Stderr, "Invalid command %s (command type or key is not provided)\n", l) + continue + } + watchArgs, execArgs, perr := parseWatchArgs(osArgs, args, envKey, envRange, true) + if perr != nil { + ExitWithError(ExitBadArgs, perr) + } - if args[0] != "watch" { + ch, err := getWatchChan(c, watchArgs) + if err != nil { + fmt.Fprintf(os.Stderr, "Invalid command %s (%v)\n", l, err) + continue + } + go printWatchCh(c, ch, execArgs) + case "progress": + err := c.RequestProgress(clientv3.WithRequireLeader(context.Background())) + if err != nil { + ExitWithError(ExitError, err) + } + default: fmt.Fprintf(os.Stderr, "Invalid command %s (only support watch)\n", l) continue } - watchArgs, execArgs, perr := parseWatchArgs(osArgs, args, envKey, envRange, true) - if perr != nil { - ExitWithError(ExitBadArgs, perr) - } - - ch, err := getWatchChan(c, watchArgs) - if err != nil { - fmt.Fprintf(os.Stderr, "Invalid command %s (%v)\n", l, err) - continue - } - go printWatchCh(c, ch, execArgs) } } @@ -152,6 +158,9 @@ func printWatchCh(c *clientv3.Client, ch clientv3.WatchChan, execArgs []string) if resp.Canceled { fmt.Fprintf(os.Stderr, "watch was canceled (%v)\n", resp.Err()) } + if resp.IsProgressNotify() { + fmt.Fprintf(os.Stdout, "progress notify: %d\n", resp.Header.Revision) + } display.Watch(resp) if len(execArgs) > 0 { diff --git a/etcdserver/api/v3rpc/watch.go b/etcdserver/api/v3rpc/watch.go index db0ad8e648f5..f9fe38df2de8 100644 --- a/etcdserver/api/v3rpc/watch.go +++ b/etcdserver/api/v3rpc/watch.go @@ -317,7 +317,12 @@ func (sws *serverWatchStream) recvLoop() error { sws.mu.Unlock() } } - + case *pb.WatchRequest_ProgressRequest: + if uv.ProgressRequest != nil { + sws.ctrlStream <- &pb.WatchResponse{ + Header: sws.newResponseHeader(sws.watchStream.Rev()), + } + } default: // we probably should not shutdown the entire stream when // receive an valid command. diff --git a/etcdserver/etcdserverpb/etcdserver.pb.go b/etcdserver/etcdserverpb/etcdserver.pb.go index 90045a5c9759..465588f48a5d 100644 --- a/etcdserver/etcdserverpb/etcdserver.pb.go +++ b/etcdserver/etcdserverpb/etcdserver.pb.go @@ -39,6 +39,7 @@ WatchRequest WatchCreateRequest WatchCancelRequest + WatchProgressRequest WatchResponse LeaseGrantRequest LeaseGrantResponse diff --git a/etcdserver/etcdserverpb/rpc.pb.go b/etcdserver/etcdserverpb/rpc.pb.go index 0aed0d43b7cf..38975595c6fd 100644 --- a/etcdserver/etcdserverpb/rpc.pb.go +++ b/etcdserver/etcdserverpb/rpc.pb.go @@ -211,7 +211,7 @@ func (x AlarmRequest_AlarmAction) String() string { return proto.EnumName(AlarmRequest_AlarmAction_name, int32(x)) } func (AlarmRequest_AlarmAction) EnumDescriptor() ([]byte, []int) { - return fileDescriptorRpc, []int{48, 0} + return fileDescriptorRpc, []int{49, 0} } type ResponseHeader struct { @@ -220,6 +220,9 @@ type ResponseHeader struct { // member_id is the ID of the member which sent the response. MemberId uint64 `protobuf:"varint,2,opt,name=member_id,json=memberId,proto3" json:"member_id,omitempty"` // revision is the key-value store revision when the request was applied. + // For watch progress responses, the header.revision indicates progress. All future events + // recieved in this stream are guarenteed to have a higher revision number than the + // header.revision number. Revision int64 `protobuf:"varint,3,opt,name=revision,proto3" json:"revision,omitempty"` // raft_term is the raft term when the request was applied. RaftTerm uint64 `protobuf:"varint,4,opt,name=raft_term,json=raftTerm,proto3" json:"raft_term,omitempty"` @@ -1480,6 +1483,7 @@ type WatchRequest struct { // Types that are valid to be assigned to RequestUnion: // *WatchRequest_CreateRequest // *WatchRequest_CancelRequest + // *WatchRequest_ProgressRequest RequestUnion isWatchRequest_RequestUnion `protobuf_oneof:"request_union"` } @@ -1500,9 +1504,13 @@ type WatchRequest_CreateRequest struct { type WatchRequest_CancelRequest struct { CancelRequest *WatchCancelRequest `protobuf:"bytes,2,opt,name=cancel_request,json=cancelRequest,oneof"` } +type WatchRequest_ProgressRequest struct { + ProgressRequest *WatchProgressRequest `protobuf:"bytes,3,opt,name=progress_request,json=progressRequest,oneof"` +} -func (*WatchRequest_CreateRequest) isWatchRequest_RequestUnion() {} -func (*WatchRequest_CancelRequest) isWatchRequest_RequestUnion() {} +func (*WatchRequest_CreateRequest) isWatchRequest_RequestUnion() {} +func (*WatchRequest_CancelRequest) isWatchRequest_RequestUnion() {} +func (*WatchRequest_ProgressRequest) isWatchRequest_RequestUnion() {} func (m *WatchRequest) GetRequestUnion() isWatchRequest_RequestUnion { if m != nil { @@ -1525,11 +1533,19 @@ func (m *WatchRequest) GetCancelRequest() *WatchCancelRequest { return nil } +func (m *WatchRequest) GetProgressRequest() *WatchProgressRequest { + if x, ok := m.GetRequestUnion().(*WatchRequest_ProgressRequest); ok { + return x.ProgressRequest + } + return nil +} + // XXX_OneofFuncs is for the internal use of the proto package. func (*WatchRequest) XXX_OneofFuncs() (func(msg proto.Message, b *proto.Buffer) error, func(msg proto.Message, tag, wire int, b *proto.Buffer) (bool, error), func(msg proto.Message) (n int), []interface{}) { return _WatchRequest_OneofMarshaler, _WatchRequest_OneofUnmarshaler, _WatchRequest_OneofSizer, []interface{}{ (*WatchRequest_CreateRequest)(nil), (*WatchRequest_CancelRequest)(nil), + (*WatchRequest_ProgressRequest)(nil), } } @@ -1547,6 +1563,11 @@ func _WatchRequest_OneofMarshaler(msg proto.Message, b *proto.Buffer) error { if err := b.EncodeMessage(x.CancelRequest); err != nil { return err } + case *WatchRequest_ProgressRequest: + _ = b.EncodeVarint(3<<3 | proto.WireBytes) + if err := b.EncodeMessage(x.ProgressRequest); err != nil { + return err + } case nil: default: return fmt.Errorf("WatchRequest.RequestUnion has unexpected type %T", x) @@ -1573,6 +1594,14 @@ func _WatchRequest_OneofUnmarshaler(msg proto.Message, tag, wire int, b *proto.B err := b.DecodeMessage(msg) m.RequestUnion = &WatchRequest_CancelRequest{msg} return true, err + case 3: // request_union.progress_request + if wire != proto.WireBytes { + return true, proto.ErrInternalBadWireType + } + msg := new(WatchProgressRequest) + err := b.DecodeMessage(msg) + m.RequestUnion = &WatchRequest_ProgressRequest{msg} + return true, err default: return false, nil } @@ -1592,6 +1621,11 @@ func _WatchRequest_OneofSizer(msg proto.Message) (n int) { n += proto.SizeVarint(2<<3 | proto.WireBytes) n += proto.SizeVarint(uint64(s)) n += s + case *WatchRequest_ProgressRequest: + s := proto.Size(x.ProgressRequest) + n += proto.SizeVarint(3<<3 | proto.WireBytes) + n += proto.SizeVarint(uint64(s)) + n += s case nil: default: panic(fmt.Sprintf("proto: unexpected type %T in oneof", x)) @@ -1708,6 +1742,16 @@ func (m *WatchCancelRequest) GetWatchId() int64 { return 0 } +// Requests the a watch stream progress status be sent in the watch response stream as soon as +// possible. +type WatchProgressRequest struct { +} + +func (m *WatchProgressRequest) Reset() { *m = WatchProgressRequest{} } +func (m *WatchProgressRequest) String() string { return proto.CompactTextString(m) } +func (*WatchProgressRequest) ProtoMessage() {} +func (*WatchProgressRequest) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{23} } + type WatchResponse struct { Header *ResponseHeader `protobuf:"bytes,1,opt,name=header" json:"header,omitempty"` // watch_id is the ID of the watcher that corresponds to the response. @@ -1739,7 +1783,7 @@ type WatchResponse struct { func (m *WatchResponse) Reset() { *m = WatchResponse{} } func (m *WatchResponse) String() string { return proto.CompactTextString(m) } func (*WatchResponse) ProtoMessage() {} -func (*WatchResponse) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{23} } +func (*WatchResponse) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{24} } func (m *WatchResponse) GetHeader() *ResponseHeader { if m != nil { @@ -1807,7 +1851,7 @@ type LeaseGrantRequest struct { func (m *LeaseGrantRequest) Reset() { *m = LeaseGrantRequest{} } func (m *LeaseGrantRequest) String() string { return proto.CompactTextString(m) } func (*LeaseGrantRequest) ProtoMessage() {} -func (*LeaseGrantRequest) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{24} } +func (*LeaseGrantRequest) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{25} } func (m *LeaseGrantRequest) GetTTL() int64 { if m != nil { @@ -1835,7 +1879,7 @@ type LeaseGrantResponse struct { func (m *LeaseGrantResponse) Reset() { *m = LeaseGrantResponse{} } func (m *LeaseGrantResponse) String() string { return proto.CompactTextString(m) } func (*LeaseGrantResponse) ProtoMessage() {} -func (*LeaseGrantResponse) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{25} } +func (*LeaseGrantResponse) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{26} } func (m *LeaseGrantResponse) GetHeader() *ResponseHeader { if m != nil { @@ -1873,7 +1917,7 @@ type LeaseRevokeRequest struct { func (m *LeaseRevokeRequest) Reset() { *m = LeaseRevokeRequest{} } func (m *LeaseRevokeRequest) String() string { return proto.CompactTextString(m) } func (*LeaseRevokeRequest) ProtoMessage() {} -func (*LeaseRevokeRequest) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{26} } +func (*LeaseRevokeRequest) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{27} } func (m *LeaseRevokeRequest) GetID() int64 { if m != nil { @@ -1889,7 +1933,7 @@ type LeaseRevokeResponse struct { func (m *LeaseRevokeResponse) Reset() { *m = LeaseRevokeResponse{} } func (m *LeaseRevokeResponse) String() string { return proto.CompactTextString(m) } func (*LeaseRevokeResponse) ProtoMessage() {} -func (*LeaseRevokeResponse) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{27} } +func (*LeaseRevokeResponse) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{28} } func (m *LeaseRevokeResponse) GetHeader() *ResponseHeader { if m != nil { @@ -1906,7 +1950,7 @@ type LeaseKeepAliveRequest struct { func (m *LeaseKeepAliveRequest) Reset() { *m = LeaseKeepAliveRequest{} } func (m *LeaseKeepAliveRequest) String() string { return proto.CompactTextString(m) } func (*LeaseKeepAliveRequest) ProtoMessage() {} -func (*LeaseKeepAliveRequest) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{28} } +func (*LeaseKeepAliveRequest) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{29} } func (m *LeaseKeepAliveRequest) GetID() int64 { if m != nil { @@ -1926,7 +1970,7 @@ type LeaseKeepAliveResponse struct { func (m *LeaseKeepAliveResponse) Reset() { *m = LeaseKeepAliveResponse{} } func (m *LeaseKeepAliveResponse) String() string { return proto.CompactTextString(m) } func (*LeaseKeepAliveResponse) ProtoMessage() {} -func (*LeaseKeepAliveResponse) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{29} } +func (*LeaseKeepAliveResponse) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{30} } func (m *LeaseKeepAliveResponse) GetHeader() *ResponseHeader { if m != nil { @@ -1959,7 +2003,7 @@ type LeaseTimeToLiveRequest struct { func (m *LeaseTimeToLiveRequest) Reset() { *m = LeaseTimeToLiveRequest{} } func (m *LeaseTimeToLiveRequest) String() string { return proto.CompactTextString(m) } func (*LeaseTimeToLiveRequest) ProtoMessage() {} -func (*LeaseTimeToLiveRequest) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{30} } +func (*LeaseTimeToLiveRequest) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{31} } func (m *LeaseTimeToLiveRequest) GetID() int64 { if m != nil { @@ -1990,7 +2034,7 @@ type LeaseTimeToLiveResponse struct { func (m *LeaseTimeToLiveResponse) Reset() { *m = LeaseTimeToLiveResponse{} } func (m *LeaseTimeToLiveResponse) String() string { return proto.CompactTextString(m) } func (*LeaseTimeToLiveResponse) ProtoMessage() {} -func (*LeaseTimeToLiveResponse) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{31} } +func (*LeaseTimeToLiveResponse) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{32} } func (m *LeaseTimeToLiveResponse) GetHeader() *ResponseHeader { if m != nil { @@ -2033,7 +2077,7 @@ type LeaseLeasesRequest struct { func (m *LeaseLeasesRequest) Reset() { *m = LeaseLeasesRequest{} } func (m *LeaseLeasesRequest) String() string { return proto.CompactTextString(m) } func (*LeaseLeasesRequest) ProtoMessage() {} -func (*LeaseLeasesRequest) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{32} } +func (*LeaseLeasesRequest) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{33} } type LeaseStatus struct { ID int64 `protobuf:"varint,1,opt,name=ID,proto3" json:"ID,omitempty"` @@ -2042,7 +2086,7 @@ type LeaseStatus struct { func (m *LeaseStatus) Reset() { *m = LeaseStatus{} } func (m *LeaseStatus) String() string { return proto.CompactTextString(m) } func (*LeaseStatus) ProtoMessage() {} -func (*LeaseStatus) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{33} } +func (*LeaseStatus) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{34} } func (m *LeaseStatus) GetID() int64 { if m != nil { @@ -2059,7 +2103,7 @@ type LeaseLeasesResponse struct { func (m *LeaseLeasesResponse) Reset() { *m = LeaseLeasesResponse{} } func (m *LeaseLeasesResponse) String() string { return proto.CompactTextString(m) } func (*LeaseLeasesResponse) ProtoMessage() {} -func (*LeaseLeasesResponse) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{34} } +func (*LeaseLeasesResponse) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{35} } func (m *LeaseLeasesResponse) GetHeader() *ResponseHeader { if m != nil { @@ -2089,7 +2133,7 @@ type Member struct { func (m *Member) Reset() { *m = Member{} } func (m *Member) String() string { return proto.CompactTextString(m) } func (*Member) ProtoMessage() {} -func (*Member) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{35} } +func (*Member) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{36} } func (m *Member) GetID() uint64 { if m != nil { @@ -2127,7 +2171,7 @@ type MemberAddRequest struct { func (m *MemberAddRequest) Reset() { *m = MemberAddRequest{} } func (m *MemberAddRequest) String() string { return proto.CompactTextString(m) } func (*MemberAddRequest) ProtoMessage() {} -func (*MemberAddRequest) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{36} } +func (*MemberAddRequest) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{37} } func (m *MemberAddRequest) GetPeerURLs() []string { if m != nil { @@ -2147,7 +2191,7 @@ type MemberAddResponse struct { func (m *MemberAddResponse) Reset() { *m = MemberAddResponse{} } func (m *MemberAddResponse) String() string { return proto.CompactTextString(m) } func (*MemberAddResponse) ProtoMessage() {} -func (*MemberAddResponse) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{37} } +func (*MemberAddResponse) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{38} } func (m *MemberAddResponse) GetHeader() *ResponseHeader { if m != nil { @@ -2178,7 +2222,7 @@ type MemberRemoveRequest struct { func (m *MemberRemoveRequest) Reset() { *m = MemberRemoveRequest{} } func (m *MemberRemoveRequest) String() string { return proto.CompactTextString(m) } func (*MemberRemoveRequest) ProtoMessage() {} -func (*MemberRemoveRequest) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{38} } +func (*MemberRemoveRequest) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{39} } func (m *MemberRemoveRequest) GetID() uint64 { if m != nil { @@ -2196,7 +2240,7 @@ type MemberRemoveResponse struct { func (m *MemberRemoveResponse) Reset() { *m = MemberRemoveResponse{} } func (m *MemberRemoveResponse) String() string { return proto.CompactTextString(m) } func (*MemberRemoveResponse) ProtoMessage() {} -func (*MemberRemoveResponse) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{39} } +func (*MemberRemoveResponse) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{40} } func (m *MemberRemoveResponse) GetHeader() *ResponseHeader { if m != nil { @@ -2222,7 +2266,7 @@ type MemberUpdateRequest struct { func (m *MemberUpdateRequest) Reset() { *m = MemberUpdateRequest{} } func (m *MemberUpdateRequest) String() string { return proto.CompactTextString(m) } func (*MemberUpdateRequest) ProtoMessage() {} -func (*MemberUpdateRequest) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{40} } +func (*MemberUpdateRequest) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{41} } func (m *MemberUpdateRequest) GetID() uint64 { if m != nil { @@ -2247,7 +2291,7 @@ type MemberUpdateResponse struct { func (m *MemberUpdateResponse) Reset() { *m = MemberUpdateResponse{} } func (m *MemberUpdateResponse) String() string { return proto.CompactTextString(m) } func (*MemberUpdateResponse) ProtoMessage() {} -func (*MemberUpdateResponse) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{41} } +func (*MemberUpdateResponse) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{42} } func (m *MemberUpdateResponse) GetHeader() *ResponseHeader { if m != nil { @@ -2269,7 +2313,7 @@ type MemberListRequest struct { func (m *MemberListRequest) Reset() { *m = MemberListRequest{} } func (m *MemberListRequest) String() string { return proto.CompactTextString(m) } func (*MemberListRequest) ProtoMessage() {} -func (*MemberListRequest) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{42} } +func (*MemberListRequest) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{43} } type MemberListResponse struct { Header *ResponseHeader `protobuf:"bytes,1,opt,name=header" json:"header,omitempty"` @@ -2280,7 +2324,7 @@ type MemberListResponse struct { func (m *MemberListResponse) Reset() { *m = MemberListResponse{} } func (m *MemberListResponse) String() string { return proto.CompactTextString(m) } func (*MemberListResponse) ProtoMessage() {} -func (*MemberListResponse) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{43} } +func (*MemberListResponse) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{44} } func (m *MemberListResponse) GetHeader() *ResponseHeader { if m != nil { @@ -2302,7 +2346,7 @@ type DefragmentRequest struct { func (m *DefragmentRequest) Reset() { *m = DefragmentRequest{} } func (m *DefragmentRequest) String() string { return proto.CompactTextString(m) } func (*DefragmentRequest) ProtoMessage() {} -func (*DefragmentRequest) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{44} } +func (*DefragmentRequest) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{45} } type DefragmentResponse struct { Header *ResponseHeader `protobuf:"bytes,1,opt,name=header" json:"header,omitempty"` @@ -2311,7 +2355,7 @@ type DefragmentResponse struct { func (m *DefragmentResponse) Reset() { *m = DefragmentResponse{} } func (m *DefragmentResponse) String() string { return proto.CompactTextString(m) } func (*DefragmentResponse) ProtoMessage() {} -func (*DefragmentResponse) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{45} } +func (*DefragmentResponse) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{46} } func (m *DefragmentResponse) GetHeader() *ResponseHeader { if m != nil { @@ -2328,7 +2372,7 @@ type MoveLeaderRequest struct { func (m *MoveLeaderRequest) Reset() { *m = MoveLeaderRequest{} } func (m *MoveLeaderRequest) String() string { return proto.CompactTextString(m) } func (*MoveLeaderRequest) ProtoMessage() {} -func (*MoveLeaderRequest) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{46} } +func (*MoveLeaderRequest) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{47} } func (m *MoveLeaderRequest) GetTargetID() uint64 { if m != nil { @@ -2344,7 +2388,7 @@ type MoveLeaderResponse struct { func (m *MoveLeaderResponse) Reset() { *m = MoveLeaderResponse{} } func (m *MoveLeaderResponse) String() string { return proto.CompactTextString(m) } func (*MoveLeaderResponse) ProtoMessage() {} -func (*MoveLeaderResponse) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{47} } +func (*MoveLeaderResponse) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{48} } func (m *MoveLeaderResponse) GetHeader() *ResponseHeader { if m != nil { @@ -2368,7 +2412,7 @@ type AlarmRequest struct { func (m *AlarmRequest) Reset() { *m = AlarmRequest{} } func (m *AlarmRequest) String() string { return proto.CompactTextString(m) } func (*AlarmRequest) ProtoMessage() {} -func (*AlarmRequest) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{48} } +func (*AlarmRequest) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{49} } func (m *AlarmRequest) GetAction() AlarmRequest_AlarmAction { if m != nil { @@ -2401,7 +2445,7 @@ type AlarmMember struct { func (m *AlarmMember) Reset() { *m = AlarmMember{} } func (m *AlarmMember) String() string { return proto.CompactTextString(m) } func (*AlarmMember) ProtoMessage() {} -func (*AlarmMember) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{49} } +func (*AlarmMember) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{50} } func (m *AlarmMember) GetMemberID() uint64 { if m != nil { @@ -2426,7 +2470,7 @@ type AlarmResponse struct { func (m *AlarmResponse) Reset() { *m = AlarmResponse{} } func (m *AlarmResponse) String() string { return proto.CompactTextString(m) } func (*AlarmResponse) ProtoMessage() {} -func (*AlarmResponse) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{50} } +func (*AlarmResponse) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{51} } func (m *AlarmResponse) GetHeader() *ResponseHeader { if m != nil { @@ -2448,7 +2492,7 @@ type StatusRequest struct { func (m *StatusRequest) Reset() { *m = StatusRequest{} } func (m *StatusRequest) String() string { return proto.CompactTextString(m) } func (*StatusRequest) ProtoMessage() {} -func (*StatusRequest) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{51} } +func (*StatusRequest) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{52} } type StatusResponse struct { Header *ResponseHeader `protobuf:"bytes,1,opt,name=header" json:"header,omitempty"` @@ -2473,7 +2517,7 @@ type StatusResponse struct { func (m *StatusResponse) Reset() { *m = StatusResponse{} } func (m *StatusResponse) String() string { return proto.CompactTextString(m) } func (*StatusResponse) ProtoMessage() {} -func (*StatusResponse) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{52} } +func (*StatusResponse) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{53} } func (m *StatusResponse) GetHeader() *ResponseHeader { if m != nil { @@ -2544,7 +2588,7 @@ type AuthEnableRequest struct { func (m *AuthEnableRequest) Reset() { *m = AuthEnableRequest{} } func (m *AuthEnableRequest) String() string { return proto.CompactTextString(m) } func (*AuthEnableRequest) ProtoMessage() {} -func (*AuthEnableRequest) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{53} } +func (*AuthEnableRequest) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{54} } type AuthDisableRequest struct { } @@ -2552,7 +2596,7 @@ type AuthDisableRequest struct { func (m *AuthDisableRequest) Reset() { *m = AuthDisableRequest{} } func (m *AuthDisableRequest) String() string { return proto.CompactTextString(m) } func (*AuthDisableRequest) ProtoMessage() {} -func (*AuthDisableRequest) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{54} } +func (*AuthDisableRequest) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{55} } type AuthenticateRequest struct { Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` @@ -2562,7 +2606,7 @@ type AuthenticateRequest struct { func (m *AuthenticateRequest) Reset() { *m = AuthenticateRequest{} } func (m *AuthenticateRequest) String() string { return proto.CompactTextString(m) } func (*AuthenticateRequest) ProtoMessage() {} -func (*AuthenticateRequest) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{55} } +func (*AuthenticateRequest) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{56} } func (m *AuthenticateRequest) GetName() string { if m != nil { @@ -2586,7 +2630,7 @@ type AuthUserAddRequest struct { func (m *AuthUserAddRequest) Reset() { *m = AuthUserAddRequest{} } func (m *AuthUserAddRequest) String() string { return proto.CompactTextString(m) } func (*AuthUserAddRequest) ProtoMessage() {} -func (*AuthUserAddRequest) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{56} } +func (*AuthUserAddRequest) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{57} } func (m *AuthUserAddRequest) GetName() string { if m != nil { @@ -2609,7 +2653,7 @@ type AuthUserGetRequest struct { func (m *AuthUserGetRequest) Reset() { *m = AuthUserGetRequest{} } func (m *AuthUserGetRequest) String() string { return proto.CompactTextString(m) } func (*AuthUserGetRequest) ProtoMessage() {} -func (*AuthUserGetRequest) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{57} } +func (*AuthUserGetRequest) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{58} } func (m *AuthUserGetRequest) GetName() string { if m != nil { @@ -2626,7 +2670,7 @@ type AuthUserDeleteRequest struct { func (m *AuthUserDeleteRequest) Reset() { *m = AuthUserDeleteRequest{} } func (m *AuthUserDeleteRequest) String() string { return proto.CompactTextString(m) } func (*AuthUserDeleteRequest) ProtoMessage() {} -func (*AuthUserDeleteRequest) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{58} } +func (*AuthUserDeleteRequest) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{59} } func (m *AuthUserDeleteRequest) GetName() string { if m != nil { @@ -2646,7 +2690,7 @@ func (m *AuthUserChangePasswordRequest) Reset() { *m = AuthUserChangePas func (m *AuthUserChangePasswordRequest) String() string { return proto.CompactTextString(m) } func (*AuthUserChangePasswordRequest) ProtoMessage() {} func (*AuthUserChangePasswordRequest) Descriptor() ([]byte, []int) { - return fileDescriptorRpc, []int{59} + return fileDescriptorRpc, []int{60} } func (m *AuthUserChangePasswordRequest) GetName() string { @@ -2673,7 +2717,7 @@ type AuthUserGrantRoleRequest struct { func (m *AuthUserGrantRoleRequest) Reset() { *m = AuthUserGrantRoleRequest{} } func (m *AuthUserGrantRoleRequest) String() string { return proto.CompactTextString(m) } func (*AuthUserGrantRoleRequest) ProtoMessage() {} -func (*AuthUserGrantRoleRequest) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{60} } +func (*AuthUserGrantRoleRequest) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{61} } func (m *AuthUserGrantRoleRequest) GetUser() string { if m != nil { @@ -2697,7 +2741,7 @@ type AuthUserRevokeRoleRequest struct { func (m *AuthUserRevokeRoleRequest) Reset() { *m = AuthUserRevokeRoleRequest{} } func (m *AuthUserRevokeRoleRequest) String() string { return proto.CompactTextString(m) } func (*AuthUserRevokeRoleRequest) ProtoMessage() {} -func (*AuthUserRevokeRoleRequest) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{61} } +func (*AuthUserRevokeRoleRequest) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{62} } func (m *AuthUserRevokeRoleRequest) GetName() string { if m != nil { @@ -2721,7 +2765,7 @@ type AuthRoleAddRequest struct { func (m *AuthRoleAddRequest) Reset() { *m = AuthRoleAddRequest{} } func (m *AuthRoleAddRequest) String() string { return proto.CompactTextString(m) } func (*AuthRoleAddRequest) ProtoMessage() {} -func (*AuthRoleAddRequest) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{62} } +func (*AuthRoleAddRequest) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{63} } func (m *AuthRoleAddRequest) GetName() string { if m != nil { @@ -2737,7 +2781,7 @@ type AuthRoleGetRequest struct { func (m *AuthRoleGetRequest) Reset() { *m = AuthRoleGetRequest{} } func (m *AuthRoleGetRequest) String() string { return proto.CompactTextString(m) } func (*AuthRoleGetRequest) ProtoMessage() {} -func (*AuthRoleGetRequest) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{63} } +func (*AuthRoleGetRequest) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{64} } func (m *AuthRoleGetRequest) GetRole() string { if m != nil { @@ -2752,7 +2796,7 @@ type AuthUserListRequest struct { func (m *AuthUserListRequest) Reset() { *m = AuthUserListRequest{} } func (m *AuthUserListRequest) String() string { return proto.CompactTextString(m) } func (*AuthUserListRequest) ProtoMessage() {} -func (*AuthUserListRequest) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{64} } +func (*AuthUserListRequest) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{65} } type AuthRoleListRequest struct { } @@ -2760,7 +2804,7 @@ type AuthRoleListRequest struct { func (m *AuthRoleListRequest) Reset() { *m = AuthRoleListRequest{} } func (m *AuthRoleListRequest) String() string { return proto.CompactTextString(m) } func (*AuthRoleListRequest) ProtoMessage() {} -func (*AuthRoleListRequest) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{65} } +func (*AuthRoleListRequest) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{66} } type AuthRoleDeleteRequest struct { Role string `protobuf:"bytes,1,opt,name=role,proto3" json:"role,omitempty"` @@ -2769,7 +2813,7 @@ type AuthRoleDeleteRequest struct { func (m *AuthRoleDeleteRequest) Reset() { *m = AuthRoleDeleteRequest{} } func (m *AuthRoleDeleteRequest) String() string { return proto.CompactTextString(m) } func (*AuthRoleDeleteRequest) ProtoMessage() {} -func (*AuthRoleDeleteRequest) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{66} } +func (*AuthRoleDeleteRequest) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{67} } func (m *AuthRoleDeleteRequest) GetRole() string { if m != nil { @@ -2789,7 +2833,7 @@ func (m *AuthRoleGrantPermissionRequest) Reset() { *m = AuthRoleGrantPer func (m *AuthRoleGrantPermissionRequest) String() string { return proto.CompactTextString(m) } func (*AuthRoleGrantPermissionRequest) ProtoMessage() {} func (*AuthRoleGrantPermissionRequest) Descriptor() ([]byte, []int) { - return fileDescriptorRpc, []int{67} + return fileDescriptorRpc, []int{68} } func (m *AuthRoleGrantPermissionRequest) GetName() string { @@ -2816,7 +2860,7 @@ func (m *AuthRoleRevokePermissionRequest) Reset() { *m = AuthRoleRevokeP func (m *AuthRoleRevokePermissionRequest) String() string { return proto.CompactTextString(m) } func (*AuthRoleRevokePermissionRequest) ProtoMessage() {} func (*AuthRoleRevokePermissionRequest) Descriptor() ([]byte, []int) { - return fileDescriptorRpc, []int{68} + return fileDescriptorRpc, []int{69} } func (m *AuthRoleRevokePermissionRequest) GetRole() string { @@ -2847,7 +2891,7 @@ type AuthEnableResponse struct { func (m *AuthEnableResponse) Reset() { *m = AuthEnableResponse{} } func (m *AuthEnableResponse) String() string { return proto.CompactTextString(m) } func (*AuthEnableResponse) ProtoMessage() {} -func (*AuthEnableResponse) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{69} } +func (*AuthEnableResponse) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{70} } func (m *AuthEnableResponse) GetHeader() *ResponseHeader { if m != nil { @@ -2863,7 +2907,7 @@ type AuthDisableResponse struct { func (m *AuthDisableResponse) Reset() { *m = AuthDisableResponse{} } func (m *AuthDisableResponse) String() string { return proto.CompactTextString(m) } func (*AuthDisableResponse) ProtoMessage() {} -func (*AuthDisableResponse) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{70} } +func (*AuthDisableResponse) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{71} } func (m *AuthDisableResponse) GetHeader() *ResponseHeader { if m != nil { @@ -2881,7 +2925,7 @@ type AuthenticateResponse struct { func (m *AuthenticateResponse) Reset() { *m = AuthenticateResponse{} } func (m *AuthenticateResponse) String() string { return proto.CompactTextString(m) } func (*AuthenticateResponse) ProtoMessage() {} -func (*AuthenticateResponse) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{71} } +func (*AuthenticateResponse) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{72} } func (m *AuthenticateResponse) GetHeader() *ResponseHeader { if m != nil { @@ -2904,7 +2948,7 @@ type AuthUserAddResponse struct { func (m *AuthUserAddResponse) Reset() { *m = AuthUserAddResponse{} } func (m *AuthUserAddResponse) String() string { return proto.CompactTextString(m) } func (*AuthUserAddResponse) ProtoMessage() {} -func (*AuthUserAddResponse) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{72} } +func (*AuthUserAddResponse) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{73} } func (m *AuthUserAddResponse) GetHeader() *ResponseHeader { if m != nil { @@ -2921,7 +2965,7 @@ type AuthUserGetResponse struct { func (m *AuthUserGetResponse) Reset() { *m = AuthUserGetResponse{} } func (m *AuthUserGetResponse) String() string { return proto.CompactTextString(m) } func (*AuthUserGetResponse) ProtoMessage() {} -func (*AuthUserGetResponse) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{73} } +func (*AuthUserGetResponse) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{74} } func (m *AuthUserGetResponse) GetHeader() *ResponseHeader { if m != nil { @@ -2944,7 +2988,7 @@ type AuthUserDeleteResponse struct { func (m *AuthUserDeleteResponse) Reset() { *m = AuthUserDeleteResponse{} } func (m *AuthUserDeleteResponse) String() string { return proto.CompactTextString(m) } func (*AuthUserDeleteResponse) ProtoMessage() {} -func (*AuthUserDeleteResponse) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{74} } +func (*AuthUserDeleteResponse) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{75} } func (m *AuthUserDeleteResponse) GetHeader() *ResponseHeader { if m != nil { @@ -2961,7 +3005,7 @@ func (m *AuthUserChangePasswordResponse) Reset() { *m = AuthUserChangePa func (m *AuthUserChangePasswordResponse) String() string { return proto.CompactTextString(m) } func (*AuthUserChangePasswordResponse) ProtoMessage() {} func (*AuthUserChangePasswordResponse) Descriptor() ([]byte, []int) { - return fileDescriptorRpc, []int{75} + return fileDescriptorRpc, []int{76} } func (m *AuthUserChangePasswordResponse) GetHeader() *ResponseHeader { @@ -2978,7 +3022,7 @@ type AuthUserGrantRoleResponse struct { func (m *AuthUserGrantRoleResponse) Reset() { *m = AuthUserGrantRoleResponse{} } func (m *AuthUserGrantRoleResponse) String() string { return proto.CompactTextString(m) } func (*AuthUserGrantRoleResponse) ProtoMessage() {} -func (*AuthUserGrantRoleResponse) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{76} } +func (*AuthUserGrantRoleResponse) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{77} } func (m *AuthUserGrantRoleResponse) GetHeader() *ResponseHeader { if m != nil { @@ -2994,7 +3038,7 @@ type AuthUserRevokeRoleResponse struct { func (m *AuthUserRevokeRoleResponse) Reset() { *m = AuthUserRevokeRoleResponse{} } func (m *AuthUserRevokeRoleResponse) String() string { return proto.CompactTextString(m) } func (*AuthUserRevokeRoleResponse) ProtoMessage() {} -func (*AuthUserRevokeRoleResponse) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{77} } +func (*AuthUserRevokeRoleResponse) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{78} } func (m *AuthUserRevokeRoleResponse) GetHeader() *ResponseHeader { if m != nil { @@ -3010,7 +3054,7 @@ type AuthRoleAddResponse struct { func (m *AuthRoleAddResponse) Reset() { *m = AuthRoleAddResponse{} } func (m *AuthRoleAddResponse) String() string { return proto.CompactTextString(m) } func (*AuthRoleAddResponse) ProtoMessage() {} -func (*AuthRoleAddResponse) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{78} } +func (*AuthRoleAddResponse) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{79} } func (m *AuthRoleAddResponse) GetHeader() *ResponseHeader { if m != nil { @@ -3027,7 +3071,7 @@ type AuthRoleGetResponse struct { func (m *AuthRoleGetResponse) Reset() { *m = AuthRoleGetResponse{} } func (m *AuthRoleGetResponse) String() string { return proto.CompactTextString(m) } func (*AuthRoleGetResponse) ProtoMessage() {} -func (*AuthRoleGetResponse) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{79} } +func (*AuthRoleGetResponse) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{80} } func (m *AuthRoleGetResponse) GetHeader() *ResponseHeader { if m != nil { @@ -3051,7 +3095,7 @@ type AuthRoleListResponse struct { func (m *AuthRoleListResponse) Reset() { *m = AuthRoleListResponse{} } func (m *AuthRoleListResponse) String() string { return proto.CompactTextString(m) } func (*AuthRoleListResponse) ProtoMessage() {} -func (*AuthRoleListResponse) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{80} } +func (*AuthRoleListResponse) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{81} } func (m *AuthRoleListResponse) GetHeader() *ResponseHeader { if m != nil { @@ -3075,7 +3119,7 @@ type AuthUserListResponse struct { func (m *AuthUserListResponse) Reset() { *m = AuthUserListResponse{} } func (m *AuthUserListResponse) String() string { return proto.CompactTextString(m) } func (*AuthUserListResponse) ProtoMessage() {} -func (*AuthUserListResponse) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{81} } +func (*AuthUserListResponse) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{82} } func (m *AuthUserListResponse) GetHeader() *ResponseHeader { if m != nil { @@ -3098,7 +3142,7 @@ type AuthRoleDeleteResponse struct { func (m *AuthRoleDeleteResponse) Reset() { *m = AuthRoleDeleteResponse{} } func (m *AuthRoleDeleteResponse) String() string { return proto.CompactTextString(m) } func (*AuthRoleDeleteResponse) ProtoMessage() {} -func (*AuthRoleDeleteResponse) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{82} } +func (*AuthRoleDeleteResponse) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{83} } func (m *AuthRoleDeleteResponse) GetHeader() *ResponseHeader { if m != nil { @@ -3115,7 +3159,7 @@ func (m *AuthRoleGrantPermissionResponse) Reset() { *m = AuthRoleGrantPe func (m *AuthRoleGrantPermissionResponse) String() string { return proto.CompactTextString(m) } func (*AuthRoleGrantPermissionResponse) ProtoMessage() {} func (*AuthRoleGrantPermissionResponse) Descriptor() ([]byte, []int) { - return fileDescriptorRpc, []int{83} + return fileDescriptorRpc, []int{84} } func (m *AuthRoleGrantPermissionResponse) GetHeader() *ResponseHeader { @@ -3133,7 +3177,7 @@ func (m *AuthRoleRevokePermissionResponse) Reset() { *m = AuthRoleRevoke func (m *AuthRoleRevokePermissionResponse) String() string { return proto.CompactTextString(m) } func (*AuthRoleRevokePermissionResponse) ProtoMessage() {} func (*AuthRoleRevokePermissionResponse) Descriptor() ([]byte, []int) { - return fileDescriptorRpc, []int{84} + return fileDescriptorRpc, []int{85} } func (m *AuthRoleRevokePermissionResponse) GetHeader() *ResponseHeader { @@ -3167,6 +3211,7 @@ func init() { proto.RegisterType((*WatchRequest)(nil), "etcdserverpb.WatchRequest") proto.RegisterType((*WatchCreateRequest)(nil), "etcdserverpb.WatchCreateRequest") proto.RegisterType((*WatchCancelRequest)(nil), "etcdserverpb.WatchCancelRequest") + proto.RegisterType((*WatchProgressRequest)(nil), "etcdserverpb.WatchProgressRequest") proto.RegisterType((*WatchResponse)(nil), "etcdserverpb.WatchResponse") proto.RegisterType((*LeaseGrantRequest)(nil), "etcdserverpb.LeaseGrantRequest") proto.RegisterType((*LeaseGrantResponse)(nil), "etcdserverpb.LeaseGrantResponse") @@ -5916,6 +5961,20 @@ func (m *WatchRequest_CancelRequest) MarshalTo(dAtA []byte) (int, error) { } return i, nil } +func (m *WatchRequest_ProgressRequest) MarshalTo(dAtA []byte) (int, error) { + i := 0 + if m.ProgressRequest != nil { + dAtA[i] = 0x1a + i++ + i = encodeVarintRpc(dAtA, i, uint64(m.ProgressRequest.Size())) + n24, err := m.ProgressRequest.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n24 + } + return i, nil +} func (m *WatchCreateRequest) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) @@ -5959,21 +6018,21 @@ func (m *WatchCreateRequest) MarshalTo(dAtA []byte) (int, error) { i++ } if len(m.Filters) > 0 { - dAtA25 := make([]byte, len(m.Filters)*10) - var j24 int + dAtA26 := make([]byte, len(m.Filters)*10) + var j25 int for _, num := range m.Filters { for num >= 1<<7 { - dAtA25[j24] = uint8(uint64(num)&0x7f | 0x80) + dAtA26[j25] = uint8(uint64(num)&0x7f | 0x80) num >>= 7 - j24++ + j25++ } - dAtA25[j24] = uint8(num) - j24++ + dAtA26[j25] = uint8(num) + j25++ } dAtA[i] = 0x2a i++ - i = encodeVarintRpc(dAtA, i, uint64(j24)) - i += copy(dAtA[i:], dAtA25[:j24]) + i = encodeVarintRpc(dAtA, i, uint64(j25)) + i += copy(dAtA[i:], dAtA26[:j25]) } if m.PrevKv { dAtA[i] = 0x30 @@ -6026,6 +6085,24 @@ func (m *WatchCancelRequest) MarshalTo(dAtA []byte) (int, error) { return i, nil } +func (m *WatchProgressRequest) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *WatchProgressRequest) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + return i, nil +} + func (m *WatchResponse) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) @@ -6045,11 +6122,11 @@ func (m *WatchResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintRpc(dAtA, i, uint64(m.Header.Size())) - n26, err := m.Header.MarshalTo(dAtA[i:]) + n27, err := m.Header.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n26 + i += n27 } if m.WatchId != 0 { dAtA[i] = 0x10 @@ -6159,11 +6236,11 @@ func (m *LeaseGrantResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintRpc(dAtA, i, uint64(m.Header.Size())) - n27, err := m.Header.MarshalTo(dAtA[i:]) + n28, err := m.Header.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n27 + i += n28 } if m.ID != 0 { dAtA[i] = 0x10 @@ -6226,11 +6303,11 @@ func (m *LeaseRevokeResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintRpc(dAtA, i, uint64(m.Header.Size())) - n28, err := m.Header.MarshalTo(dAtA[i:]) + n29, err := m.Header.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n28 + i += n29 } return i, nil } @@ -6277,11 +6354,11 @@ func (m *LeaseKeepAliveResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintRpc(dAtA, i, uint64(m.Header.Size())) - n29, err := m.Header.MarshalTo(dAtA[i:]) + n30, err := m.Header.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n29 + i += n30 } if m.ID != 0 { dAtA[i] = 0x10 @@ -6348,11 +6425,11 @@ func (m *LeaseTimeToLiveResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintRpc(dAtA, i, uint64(m.Header.Size())) - n30, err := m.Header.MarshalTo(dAtA[i:]) + n31, err := m.Header.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n30 + i += n31 } if m.ID != 0 { dAtA[i] = 0x10 @@ -6440,11 +6517,11 @@ func (m *LeaseLeasesResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintRpc(dAtA, i, uint64(m.Header.Size())) - n31, err := m.Header.MarshalTo(dAtA[i:]) + n32, err := m.Header.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n31 + i += n32 } if len(m.Leases) > 0 { for _, msg := range m.Leases { @@ -6572,21 +6649,21 @@ func (m *MemberAddResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintRpc(dAtA, i, uint64(m.Header.Size())) - n32, err := m.Header.MarshalTo(dAtA[i:]) + n33, err := m.Header.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n32 + i += n33 } if m.Member != nil { dAtA[i] = 0x12 i++ i = encodeVarintRpc(dAtA, i, uint64(m.Member.Size())) - n33, err := m.Member.MarshalTo(dAtA[i:]) + n34, err := m.Member.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n33 + i += n34 } if len(m.Members) > 0 { for _, msg := range m.Members { @@ -6645,11 +6722,11 @@ func (m *MemberRemoveResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintRpc(dAtA, i, uint64(m.Header.Size())) - n34, err := m.Header.MarshalTo(dAtA[i:]) + n35, err := m.Header.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n34 + i += n35 } if len(m.Members) > 0 { for _, msg := range m.Members { @@ -6723,11 +6800,11 @@ func (m *MemberUpdateResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintRpc(dAtA, i, uint64(m.Header.Size())) - n35, err := m.Header.MarshalTo(dAtA[i:]) + n36, err := m.Header.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n35 + i += n36 } if len(m.Members) > 0 { for _, msg := range m.Members { @@ -6781,11 +6858,11 @@ func (m *MemberListResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintRpc(dAtA, i, uint64(m.Header.Size())) - n36, err := m.Header.MarshalTo(dAtA[i:]) + n37, err := m.Header.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n36 + i += n37 } if len(m.Members) > 0 { for _, msg := range m.Members { @@ -6839,11 +6916,11 @@ func (m *DefragmentResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintRpc(dAtA, i, uint64(m.Header.Size())) - n37, err := m.Header.MarshalTo(dAtA[i:]) + n38, err := m.Header.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n37 + i += n38 } return i, nil } @@ -6890,11 +6967,11 @@ func (m *MoveLeaderResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintRpc(dAtA, i, uint64(m.Header.Size())) - n38, err := m.Header.MarshalTo(dAtA[i:]) + n39, err := m.Header.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n38 + i += n39 } return i, nil } @@ -6979,11 +7056,11 @@ func (m *AlarmResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintRpc(dAtA, i, uint64(m.Header.Size())) - n39, err := m.Header.MarshalTo(dAtA[i:]) + n40, err := m.Header.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n39 + i += n40 } if len(m.Alarms) > 0 { for _, msg := range m.Alarms { @@ -7037,11 +7114,11 @@ func (m *StatusResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintRpc(dAtA, i, uint64(m.Header.Size())) - n40, err := m.Header.MarshalTo(dAtA[i:]) + n41, err := m.Header.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n40 + i += n41 } if len(m.Version) > 0 { dAtA[i] = 0x12 @@ -7464,11 +7541,11 @@ func (m *AuthRoleGrantPermissionRequest) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x12 i++ i = encodeVarintRpc(dAtA, i, uint64(m.Perm.Size())) - n41, err := m.Perm.MarshalTo(dAtA[i:]) + n42, err := m.Perm.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n41 + i += n42 } return i, nil } @@ -7528,11 +7605,11 @@ func (m *AuthEnableResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintRpc(dAtA, i, uint64(m.Header.Size())) - n42, err := m.Header.MarshalTo(dAtA[i:]) + n43, err := m.Header.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n42 + i += n43 } return i, nil } @@ -7556,11 +7633,11 @@ func (m *AuthDisableResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintRpc(dAtA, i, uint64(m.Header.Size())) - n43, err := m.Header.MarshalTo(dAtA[i:]) + n44, err := m.Header.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n43 + i += n44 } return i, nil } @@ -7584,11 +7661,11 @@ func (m *AuthenticateResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintRpc(dAtA, i, uint64(m.Header.Size())) - n44, err := m.Header.MarshalTo(dAtA[i:]) + n45, err := m.Header.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n44 + i += n45 } if len(m.Token) > 0 { dAtA[i] = 0x12 @@ -7618,11 +7695,11 @@ func (m *AuthUserAddResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintRpc(dAtA, i, uint64(m.Header.Size())) - n45, err := m.Header.MarshalTo(dAtA[i:]) + n46, err := m.Header.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n45 + i += n46 } return i, nil } @@ -7646,11 +7723,11 @@ func (m *AuthUserGetResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintRpc(dAtA, i, uint64(m.Header.Size())) - n46, err := m.Header.MarshalTo(dAtA[i:]) + n47, err := m.Header.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n46 + i += n47 } if len(m.Roles) > 0 { for _, s := range m.Roles { @@ -7689,11 +7766,11 @@ func (m *AuthUserDeleteResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintRpc(dAtA, i, uint64(m.Header.Size())) - n47, err := m.Header.MarshalTo(dAtA[i:]) + n48, err := m.Header.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n47 + i += n48 } return i, nil } @@ -7717,11 +7794,11 @@ func (m *AuthUserChangePasswordResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintRpc(dAtA, i, uint64(m.Header.Size())) - n48, err := m.Header.MarshalTo(dAtA[i:]) + n49, err := m.Header.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n48 + i += n49 } return i, nil } @@ -7745,11 +7822,11 @@ func (m *AuthUserGrantRoleResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintRpc(dAtA, i, uint64(m.Header.Size())) - n49, err := m.Header.MarshalTo(dAtA[i:]) + n50, err := m.Header.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n49 + i += n50 } return i, nil } @@ -7773,11 +7850,11 @@ func (m *AuthUserRevokeRoleResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintRpc(dAtA, i, uint64(m.Header.Size())) - n50, err := m.Header.MarshalTo(dAtA[i:]) + n51, err := m.Header.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n50 + i += n51 } return i, nil } @@ -7801,11 +7878,11 @@ func (m *AuthRoleAddResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintRpc(dAtA, i, uint64(m.Header.Size())) - n51, err := m.Header.MarshalTo(dAtA[i:]) + n52, err := m.Header.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n51 + i += n52 } return i, nil } @@ -7829,11 +7906,11 @@ func (m *AuthRoleGetResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintRpc(dAtA, i, uint64(m.Header.Size())) - n52, err := m.Header.MarshalTo(dAtA[i:]) + n53, err := m.Header.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n52 + i += n53 } if len(m.Perm) > 0 { for _, msg := range m.Perm { @@ -7869,11 +7946,11 @@ func (m *AuthRoleListResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintRpc(dAtA, i, uint64(m.Header.Size())) - n53, err := m.Header.MarshalTo(dAtA[i:]) + n54, err := m.Header.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n53 + i += n54 } if len(m.Roles) > 0 { for _, s := range m.Roles { @@ -7912,11 +7989,11 @@ func (m *AuthUserListResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintRpc(dAtA, i, uint64(m.Header.Size())) - n54, err := m.Header.MarshalTo(dAtA[i:]) + n55, err := m.Header.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n54 + i += n55 } if len(m.Users) > 0 { for _, s := range m.Users { @@ -7955,11 +8032,11 @@ func (m *AuthRoleDeleteResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintRpc(dAtA, i, uint64(m.Header.Size())) - n55, err := m.Header.MarshalTo(dAtA[i:]) + n56, err := m.Header.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n55 + i += n56 } return i, nil } @@ -7983,11 +8060,11 @@ func (m *AuthRoleGrantPermissionResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintRpc(dAtA, i, uint64(m.Header.Size())) - n56, err := m.Header.MarshalTo(dAtA[i:]) + n57, err := m.Header.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n56 + i += n57 } return i, nil } @@ -8011,11 +8088,11 @@ func (m *AuthRoleRevokePermissionResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintRpc(dAtA, i, uint64(m.Header.Size())) - n57, err := m.Header.MarshalTo(dAtA[i:]) + n58, err := m.Header.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n57 + i += n58 } return i, nil } @@ -8497,6 +8574,15 @@ func (m *WatchRequest_CancelRequest) Size() (n int) { } return n } +func (m *WatchRequest_ProgressRequest) Size() (n int) { + var l int + _ = l + if m.ProgressRequest != nil { + l = m.ProgressRequest.Size() + n += 1 + l + sovRpc(uint64(l)) + } + return n +} func (m *WatchCreateRequest) Size() (n int) { var l int _ = l @@ -8542,6 +8628,12 @@ func (m *WatchCancelRequest) Size() (n int) { return n } +func (m *WatchProgressRequest) Size() (n int) { + var l int + _ = l + return n +} + func (m *WatchResponse) Size() (n int) { var l int _ = l @@ -12213,6 +12305,38 @@ func (m *WatchRequest) Unmarshal(dAtA []byte) error { } m.RequestUnion = &WatchRequest_CancelRequest{v} iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ProgressRequest", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthRpc + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + v := &WatchProgressRequest{} + if err := v.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.RequestUnion = &WatchRequest_ProgressRequest{v} + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipRpc(dAtA[iNdEx:]) @@ -12575,6 +12699,56 @@ func (m *WatchCancelRequest) Unmarshal(dAtA []byte) error { } return nil } +func (m *WatchProgressRequest) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: WatchProgressRequest: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: WatchProgressRequest: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + default: + iNdEx = preIndex + skippy, err := skipRpc(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthRpc + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} func (m *WatchResponse) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 @@ -18698,240 +18872,241 @@ var ( func init() { proto.RegisterFile("rpc.proto", fileDescriptorRpc) } var fileDescriptorRpc = []byte{ - // 3750 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xbc, 0x5b, 0xcd, 0x6f, 0x1c, 0x47, - 0x76, 0x67, 0xcf, 0x70, 0xbe, 0xde, 0x7c, 0x70, 0x58, 0x24, 0xa5, 0xd1, 0x48, 0xa2, 0xa8, 0x92, - 0x64, 0xd1, 0x92, 0xcd, 0xb1, 0x69, 0x3b, 0x01, 0x94, 0xc4, 0x30, 0x45, 0x8e, 0x45, 0x9a, 0x14, - 0x49, 0x37, 0x87, 0xf2, 0x07, 0x8c, 0x10, 0xcd, 0x99, 0x22, 0xd9, 0xe1, 0x4c, 0xf7, 0xb8, 0xbb, - 0x67, 0x44, 0x3a, 0x41, 0x1c, 0x18, 0xce, 0x21, 0x39, 0xda, 0x40, 0x90, 0x1c, 0x72, 0x0a, 0x82, - 0xc0, 0x87, 0x00, 0x7b, 0x59, 0x2c, 0xb0, 0x7f, 0xc1, 0xde, 0x76, 0x17, 0xfb, 0x0f, 0x2c, 0xbc, - 0xbe, 0xec, 0x7f, 0xb1, 0xa8, 0xaf, 0xee, 0xea, 0x9e, 0x6e, 0x52, 0xf6, 0xd8, 0xbe, 0x50, 0x5d, - 0x55, 0xaf, 0xde, 0xef, 0xd5, 0xab, 0xaa, 0xf7, 0xaa, 0x7e, 0x35, 0x82, 0x82, 0xd3, 0x6f, 0x2f, - 0xf5, 0x1d, 0xdb, 0xb3, 0x51, 0x89, 0x78, 0xed, 0x8e, 0x4b, 0x9c, 0x21, 0x71, 0xfa, 0x87, 0xf5, - 0xd9, 0x63, 0xfb, 0xd8, 0x66, 0x0d, 0x0d, 0xfa, 0xc5, 0x65, 0xea, 0xd7, 0xa8, 0x4c, 0xa3, 0x37, - 0x6c, 0xb7, 0xd9, 0x9f, 0xfe, 0x61, 0xe3, 0x74, 0x28, 0x9a, 0xae, 0xb3, 0x26, 0x63, 0xe0, 0x9d, - 0xb0, 0x3f, 0xfd, 0x43, 0xf6, 0x8f, 0x68, 0xbc, 0x71, 0x6c, 0xdb, 0xc7, 0x5d, 0xd2, 0x30, 0xfa, - 0x66, 0xc3, 0xb0, 0x2c, 0xdb, 0x33, 0x3c, 0xd3, 0xb6, 0x5c, 0xde, 0x8a, 0xff, 0x55, 0x83, 0x8a, - 0x4e, 0xdc, 0xbe, 0x6d, 0xb9, 0x64, 0x9d, 0x18, 0x1d, 0xe2, 0xa0, 0x9b, 0x00, 0xed, 0xee, 0xc0, - 0xf5, 0x88, 0x73, 0x60, 0x76, 0x6a, 0xda, 0x82, 0xb6, 0x38, 0xa9, 0x17, 0x44, 0xcd, 0x46, 0x07, - 0x5d, 0x87, 0x42, 0x8f, 0xf4, 0x0e, 0x79, 0x6b, 0x8a, 0xb5, 0xe6, 0x79, 0xc5, 0x46, 0x07, 0xd5, - 0x21, 0xef, 0x90, 0xa1, 0xe9, 0x9a, 0xb6, 0x55, 0x4b, 0x2f, 0x68, 0x8b, 0x69, 0xdd, 0x2f, 0xd3, - 0x8e, 0x8e, 0x71, 0xe4, 0x1d, 0x78, 0xc4, 0xe9, 0xd5, 0x26, 0x79, 0x47, 0x5a, 0xd1, 0x22, 0x4e, - 0x0f, 0x7f, 0x99, 0x81, 0x92, 0x6e, 0x58, 0xc7, 0x44, 0x27, 0x9f, 0x0e, 0x88, 0xeb, 0xa1, 0x2a, - 0xa4, 0x4f, 0xc9, 0x39, 0x83, 0x2f, 0xe9, 0xf4, 0x93, 0xf7, 0xb7, 0x8e, 0xc9, 0x01, 0xb1, 0x38, - 0x70, 0x89, 0xf6, 0xb7, 0x8e, 0x49, 0xd3, 0xea, 0xa0, 0x59, 0xc8, 0x74, 0xcd, 0x9e, 0xe9, 0x09, - 0x54, 0x5e, 0x08, 0x99, 0x33, 0x19, 0x31, 0x67, 0x15, 0xc0, 0xb5, 0x1d, 0xef, 0xc0, 0x76, 0x3a, - 0xc4, 0xa9, 0x65, 0x16, 0xb4, 0xc5, 0xca, 0xf2, 0xdd, 0x25, 0x75, 0x22, 0x96, 0x54, 0x83, 0x96, - 0xf6, 0x6c, 0xc7, 0xdb, 0xa1, 0xb2, 0x7a, 0xc1, 0x95, 0x9f, 0xe8, 0x5d, 0x28, 0x32, 0x25, 0x9e, - 0xe1, 0x1c, 0x13, 0xaf, 0x96, 0x65, 0x5a, 0xee, 0x5d, 0xa2, 0xa5, 0xc5, 0x84, 0x75, 0x06, 0xcf, - 0xbf, 0x11, 0x86, 0x92, 0x4b, 0x1c, 0xd3, 0xe8, 0x9a, 0x9f, 0x19, 0x87, 0x5d, 0x52, 0xcb, 0x2d, - 0x68, 0x8b, 0x79, 0x3d, 0x54, 0x47, 0xc7, 0x7f, 0x4a, 0xce, 0xdd, 0x03, 0xdb, 0xea, 0x9e, 0xd7, - 0xf2, 0x4c, 0x20, 0x4f, 0x2b, 0x76, 0xac, 0xee, 0x39, 0x9b, 0x34, 0x7b, 0x60, 0x79, 0xbc, 0xb5, - 0xc0, 0x5a, 0x0b, 0xac, 0x86, 0x35, 0x2f, 0x42, 0xb5, 0x67, 0x5a, 0x07, 0x3d, 0xbb, 0x73, 0xe0, - 0x3b, 0x04, 0x98, 0x43, 0x2a, 0x3d, 0xd3, 0x7a, 0x6a, 0x77, 0x74, 0xe9, 0x16, 0x2a, 0x69, 0x9c, - 0x85, 0x25, 0x8b, 0x42, 0xd2, 0x38, 0x53, 0x25, 0x97, 0x60, 0x86, 0xea, 0x6c, 0x3b, 0xc4, 0xf0, - 0x48, 0x20, 0x5c, 0x62, 0xc2, 0xd3, 0x3d, 0xd3, 0x5a, 0x65, 0x2d, 0x21, 0x79, 0xe3, 0x6c, 0x44, - 0xbe, 0x2c, 0xe4, 0x8d, 0xb3, 0xb0, 0x3c, 0x5e, 0x82, 0x82, 0xef, 0x73, 0x94, 0x87, 0xc9, 0xed, - 0x9d, 0xed, 0x66, 0x75, 0x02, 0x01, 0x64, 0x57, 0xf6, 0x56, 0x9b, 0xdb, 0x6b, 0x55, 0x0d, 0x15, - 0x21, 0xb7, 0xd6, 0xe4, 0x85, 0x14, 0x7e, 0x0c, 0x10, 0x78, 0x17, 0xe5, 0x20, 0xbd, 0xd9, 0xfc, - 0xa8, 0x3a, 0x41, 0x65, 0x9e, 0x35, 0xf5, 0xbd, 0x8d, 0x9d, 0xed, 0xaa, 0x46, 0x3b, 0xaf, 0xea, - 0xcd, 0x95, 0x56, 0xb3, 0x9a, 0xa2, 0x12, 0x4f, 0x77, 0xd6, 0xaa, 0x69, 0x54, 0x80, 0xcc, 0xb3, - 0x95, 0xad, 0xfd, 0x66, 0x75, 0x12, 0x7f, 0xad, 0x41, 0x59, 0xcc, 0x17, 0xdf, 0x13, 0xe8, 0x4d, - 0xc8, 0x9e, 0xb0, 0x7d, 0xc1, 0x96, 0x62, 0x71, 0xf9, 0x46, 0x64, 0x72, 0x43, 0x7b, 0x47, 0x17, - 0xb2, 0x08, 0x43, 0xfa, 0x74, 0xe8, 0xd6, 0x52, 0x0b, 0xe9, 0xc5, 0xe2, 0x72, 0x75, 0x89, 0x6f, - 0xd8, 0xa5, 0x4d, 0x72, 0xfe, 0xcc, 0xe8, 0x0e, 0x88, 0x4e, 0x1b, 0x11, 0x82, 0xc9, 0x9e, 0xed, - 0x10, 0xb6, 0x62, 0xf3, 0x3a, 0xfb, 0xa6, 0xcb, 0x98, 0x4d, 0x9a, 0x58, 0xad, 0xbc, 0x80, 0xbf, - 0xd1, 0x00, 0x76, 0x07, 0x5e, 0xf2, 0xd6, 0x98, 0x85, 0xcc, 0x90, 0x2a, 0x16, 0xdb, 0x82, 0x17, - 0xd8, 0x9e, 0x20, 0x86, 0x4b, 0xfc, 0x3d, 0x41, 0x0b, 0xe8, 0x2a, 0xe4, 0xfa, 0x0e, 0x19, 0x1e, - 0x9c, 0x0e, 0x19, 0x48, 0x5e, 0xcf, 0xd2, 0xe2, 0xe6, 0x10, 0xdd, 0x86, 0x92, 0x79, 0x6c, 0xd9, - 0x0e, 0x39, 0xe0, 0xba, 0x32, 0xac, 0xb5, 0xc8, 0xeb, 0x98, 0xdd, 0x8a, 0x08, 0x57, 0x9c, 0x55, - 0x45, 0xb6, 0x68, 0x15, 0xb6, 0xa0, 0xc8, 0x4c, 0x1d, 0xcb, 0x7d, 0x2f, 0x07, 0x36, 0xa6, 0x58, - 0xb7, 0x51, 0x17, 0x0a, 0xab, 0xf1, 0x27, 0x80, 0xd6, 0x48, 0x97, 0x78, 0x64, 0x9c, 0xe8, 0xa1, - 0xf8, 0x24, 0xad, 0xfa, 0x04, 0x7f, 0xa5, 0xc1, 0x4c, 0x48, 0xfd, 0x58, 0xc3, 0xaa, 0x41, 0xae, - 0xc3, 0x94, 0x71, 0x0b, 0xd2, 0xba, 0x2c, 0xa2, 0x87, 0x90, 0x17, 0x06, 0xb8, 0xb5, 0x74, 0xc2, - 0xa2, 0xc9, 0x71, 0x9b, 0x5c, 0xfc, 0x4d, 0x0a, 0x0a, 0x62, 0xa0, 0x3b, 0x7d, 0xb4, 0x02, 0x65, - 0x87, 0x17, 0x0e, 0xd8, 0x78, 0x84, 0x45, 0xf5, 0xe4, 0x20, 0xb4, 0x3e, 0xa1, 0x97, 0x44, 0x17, - 0x56, 0x8d, 0xfe, 0x06, 0x8a, 0x52, 0x45, 0x7f, 0xe0, 0x09, 0x97, 0xd7, 0xc2, 0x0a, 0x82, 0xf5, - 0xb7, 0x3e, 0xa1, 0x83, 0x10, 0xdf, 0x1d, 0x78, 0xa8, 0x05, 0xb3, 0xb2, 0x33, 0x1f, 0x8d, 0x30, - 0x23, 0xcd, 0xb4, 0x2c, 0x84, 0xb5, 0x8c, 0x4e, 0xd5, 0xfa, 0x84, 0x8e, 0x44, 0x7f, 0xa5, 0x51, - 0x35, 0xc9, 0x3b, 0xe3, 0xc1, 0x7b, 0xc4, 0xa4, 0xd6, 0x99, 0x35, 0x6a, 0x52, 0xeb, 0xcc, 0x7a, - 0x5c, 0x80, 0x9c, 0x28, 0xe1, 0x5f, 0xa5, 0x00, 0xe4, 0x6c, 0xec, 0xf4, 0xd1, 0x1a, 0x54, 0x1c, - 0x51, 0x0a, 0x79, 0xeb, 0x7a, 0xac, 0xb7, 0xc4, 0x24, 0x4e, 0xe8, 0x65, 0xd9, 0x89, 0x1b, 0xf7, - 0x36, 0x94, 0x7c, 0x2d, 0x81, 0xc3, 0xae, 0xc5, 0x38, 0xcc, 0xd7, 0x50, 0x94, 0x1d, 0xa8, 0xcb, - 0x3e, 0x80, 0x39, 0xbf, 0x7f, 0x8c, 0xcf, 0x6e, 0x5f, 0xe0, 0x33, 0x5f, 0xe1, 0x8c, 0xd4, 0xa0, - 0x7a, 0x4d, 0x35, 0x2c, 0x70, 0xdb, 0xb5, 0x18, 0xb7, 0x8d, 0x1a, 0x46, 0x1d, 0x07, 0x34, 0x5f, - 0xf2, 0x22, 0xfe, 0x73, 0x1a, 0x72, 0xab, 0x76, 0xaf, 0x6f, 0x38, 0x74, 0x36, 0xb2, 0x0e, 0x71, - 0x07, 0x5d, 0x8f, 0xb9, 0xab, 0xb2, 0x7c, 0x27, 0xac, 0x51, 0x88, 0xc9, 0x7f, 0x75, 0x26, 0xaa, - 0x8b, 0x2e, 0xb4, 0xb3, 0x48, 0x8f, 0xa9, 0x17, 0xe8, 0x2c, 0x92, 0xa3, 0xe8, 0x22, 0x37, 0x72, - 0x3a, 0xd8, 0xc8, 0x75, 0xc8, 0x0d, 0x89, 0x13, 0xa4, 0xf4, 0xf5, 0x09, 0x5d, 0x56, 0xa0, 0x97, - 0x61, 0x2a, 0x9a, 0x5e, 0x32, 0x42, 0xa6, 0xd2, 0x0e, 0x67, 0xa3, 0x3b, 0x50, 0x0a, 0xe5, 0xb8, - 0xac, 0x90, 0x2b, 0xf6, 0x94, 0x14, 0x77, 0x45, 0xc6, 0x55, 0x9a, 0x8f, 0x4b, 0xeb, 0x13, 0x32, - 0xb2, 0x5e, 0x91, 0x91, 0x35, 0x2f, 0x7a, 0x89, 0xd8, 0x1a, 0x0a, 0x32, 0xef, 0x84, 0x83, 0x0c, - 0x7e, 0x07, 0xca, 0x21, 0x07, 0xd1, 0xbc, 0xd3, 0x7c, 0x7f, 0x7f, 0x65, 0x8b, 0x27, 0xa9, 0x27, - 0x2c, 0x2f, 0xe9, 0x55, 0x8d, 0xe6, 0xba, 0xad, 0xe6, 0xde, 0x5e, 0x35, 0x85, 0xca, 0x50, 0xd8, - 0xde, 0x69, 0x1d, 0x70, 0xa9, 0x34, 0x7e, 0xe2, 0x6b, 0x10, 0x49, 0x4e, 0xc9, 0x6d, 0x13, 0x4a, - 0x6e, 0xd3, 0x64, 0x6e, 0x4b, 0x05, 0xb9, 0x8d, 0xa5, 0xb9, 0xad, 0xe6, 0xca, 0x5e, 0xb3, 0x3a, - 0xf9, 0xb8, 0x02, 0x25, 0xee, 0xdf, 0x83, 0x81, 0x45, 0x53, 0xed, 0xff, 0x68, 0x00, 0xc1, 0x6e, - 0x42, 0x0d, 0xc8, 0xb5, 0x39, 0x4e, 0x4d, 0x63, 0xc1, 0x68, 0x2e, 0x76, 0xca, 0x74, 0x29, 0x85, - 0x5e, 0x87, 0x9c, 0x3b, 0x68, 0xb7, 0x89, 0x2b, 0x53, 0xde, 0xd5, 0x68, 0x3c, 0x14, 0xd1, 0x4a, - 0x97, 0x72, 0xb4, 0xcb, 0x91, 0x61, 0x76, 0x07, 0x2c, 0x01, 0x5e, 0xdc, 0x45, 0xc8, 0xe1, 0xff, - 0xd2, 0xa0, 0xa8, 0x2c, 0xde, 0x1f, 0x18, 0x84, 0x6f, 0x40, 0x81, 0xd9, 0x40, 0x3a, 0x22, 0x0c, - 0xe7, 0xf5, 0xa0, 0x02, 0xfd, 0x15, 0x14, 0xe4, 0x0e, 0x90, 0x91, 0xb8, 0x16, 0xaf, 0x76, 0xa7, - 0xaf, 0x07, 0xa2, 0x78, 0x13, 0xa6, 0x99, 0x57, 0xda, 0xf4, 0x70, 0x2d, 0xfd, 0xa8, 0x1e, 0x3f, - 0xb5, 0xc8, 0xf1, 0xb3, 0x0e, 0xf9, 0xfe, 0xc9, 0xb9, 0x6b, 0xb6, 0x8d, 0xae, 0xb0, 0xc2, 0x2f, - 0xe3, 0xf7, 0x00, 0xa9, 0xca, 0xc6, 0x19, 0x2e, 0x2e, 0x43, 0x71, 0xdd, 0x70, 0x4f, 0x84, 0x49, - 0xf8, 0x21, 0x94, 0x69, 0x71, 0xf3, 0xd9, 0x0b, 0xd8, 0xc8, 0x2e, 0x07, 0x52, 0x7a, 0x2c, 0x9f, - 0x23, 0x98, 0x3c, 0x31, 0xdc, 0x13, 0x36, 0xd0, 0xb2, 0xce, 0xbe, 0xd1, 0xcb, 0x50, 0x6d, 0xf3, - 0x41, 0x1e, 0x44, 0xae, 0x0c, 0x53, 0xa2, 0xde, 0x3f, 0x09, 0x7e, 0x08, 0x25, 0x3e, 0x86, 0x1f, - 0xdb, 0x08, 0x3c, 0x0d, 0x53, 0x7b, 0x96, 0xd1, 0x77, 0x4f, 0x6c, 0x99, 0xdd, 0xe8, 0xa0, 0xab, - 0x41, 0xdd, 0x58, 0x88, 0xf7, 0x61, 0xca, 0x21, 0x3d, 0xc3, 0xb4, 0x4c, 0xeb, 0xf8, 0xe0, 0xf0, - 0xdc, 0x23, 0xae, 0xb8, 0x30, 0x55, 0xfc, 0xea, 0xc7, 0xb4, 0x96, 0x9a, 0x76, 0xd8, 0xb5, 0x0f, - 0x45, 0x98, 0x63, 0xdf, 0xf8, 0x97, 0x1a, 0x94, 0x3e, 0x30, 0xbc, 0xb6, 0x9c, 0x3a, 0xb4, 0x01, - 0x15, 0x3f, 0xb8, 0xb1, 0x1a, 0x61, 0x4b, 0x24, 0xc5, 0xb2, 0x3e, 0xf2, 0x28, 0x2d, 0xb3, 0x63, - 0xb9, 0xad, 0x56, 0x30, 0x55, 0x86, 0xd5, 0x26, 0x5d, 0x5f, 0x55, 0x2a, 0x59, 0x15, 0x13, 0x54, - 0x55, 0xa9, 0x15, 0x8f, 0xa7, 0x82, 0xe3, 0x07, 0x8f, 0x25, 0xbf, 0x4f, 0x01, 0x1a, 0xb5, 0xe1, - 0xfb, 0x9e, 0xc8, 0xee, 0x41, 0xc5, 0xf5, 0x0c, 0x67, 0x64, 0x6d, 0x94, 0x59, 0xad, 0x1f, 0xa0, - 0xef, 0xc3, 0x54, 0xdf, 0xb1, 0x8f, 0x1d, 0xe2, 0xba, 0x07, 0x96, 0xed, 0x99, 0x47, 0xe7, 0xe2, - 0x50, 0x5b, 0x91, 0xd5, 0xdb, 0xac, 0x16, 0x35, 0x21, 0x77, 0x64, 0x76, 0x3d, 0xe2, 0xb8, 0xb5, - 0xcc, 0x42, 0x7a, 0xb1, 0xb2, 0xfc, 0xf0, 0x32, 0xaf, 0x2d, 0xbd, 0xcb, 0xe4, 0x5b, 0xe7, 0x7d, - 0xa2, 0xcb, 0xbe, 0xea, 0x41, 0x31, 0x1b, 0x3a, 0x3c, 0x5f, 0x83, 0xfc, 0x73, 0xaa, 0x82, 0x5e, - 0x8a, 0x73, 0xfc, 0x6c, 0xc7, 0xca, 0xfc, 0x4e, 0x7c, 0xe4, 0x18, 0xc7, 0x3d, 0x62, 0x79, 0xf2, - 0xda, 0x26, 0xcb, 0xf8, 0x1e, 0x40, 0x00, 0x43, 0x23, 0xf4, 0xf6, 0xce, 0xee, 0x7e, 0xab, 0x3a, - 0x81, 0x4a, 0x90, 0xdf, 0xde, 0x59, 0x6b, 0x6e, 0x35, 0x69, 0x38, 0xc7, 0x0d, 0xe9, 0x52, 0xd5, - 0xf5, 0x21, 0x4c, 0x2d, 0x84, 0x49, 0x8f, 0x88, 0x65, 0xb1, 0x78, 0xc6, 0x5a, 0xc1, 0x2a, 0x44, - 0x2a, 0x3c, 0xac, 0x1a, 0xe4, 0xf8, 0xa2, 0xea, 0x88, 0x33, 0xb3, 0x2c, 0xd2, 0x01, 0xf3, 0x35, - 0x42, 0x3a, 0x62, 0x36, 0xfc, 0x72, 0xec, 0xae, 0xcf, 0xc4, 0xee, 0x7a, 0x74, 0x07, 0xca, 0xfe, - 0x22, 0x35, 0x5c, 0x91, 0xa2, 0x0b, 0x7a, 0x49, 0xae, 0x3f, 0x5a, 0x17, 0x72, 0x6e, 0x2e, 0xec, - 0x5c, 0x74, 0x0f, 0xb2, 0x64, 0x48, 0x2c, 0xcf, 0xad, 0x15, 0x59, 0x20, 0x2f, 0xcb, 0x23, 0x75, - 0x93, 0xd6, 0xea, 0xa2, 0x11, 0xbf, 0x05, 0xd3, 0xec, 0xea, 0xf2, 0xc4, 0x31, 0x2c, 0xf5, 0x8e, - 0xd5, 0x6a, 0x6d, 0x09, 0xb7, 0xd2, 0x4f, 0x54, 0x81, 0xd4, 0xc6, 0x9a, 0x70, 0x42, 0x6a, 0x63, - 0x0d, 0x7f, 0xa1, 0x01, 0x52, 0xfb, 0x8d, 0xe5, 0xe7, 0x88, 0x72, 0x09, 0x9f, 0x0e, 0xe0, 0x67, - 0x21, 0x43, 0x1c, 0xc7, 0x76, 0x98, 0x47, 0x0b, 0x3a, 0x2f, 0xe0, 0xbb, 0xc2, 0x06, 0x9d, 0x0c, - 0xed, 0x53, 0x7f, 0xaf, 0x71, 0x6d, 0x9a, 0x6f, 0xea, 0x26, 0xcc, 0x84, 0xa4, 0xc6, 0x4a, 0x28, - 0xf7, 0x61, 0x8e, 0x29, 0xdb, 0x24, 0xa4, 0xbf, 0xd2, 0x35, 0x87, 0x89, 0xa8, 0x7d, 0xb8, 0x12, - 0x15, 0xfc, 0x69, 0x7d, 0x84, 0xff, 0x56, 0x20, 0xb6, 0xcc, 0x1e, 0x69, 0xd9, 0x5b, 0xc9, 0xb6, - 0xd1, 0x80, 0x7b, 0x4a, 0xce, 0x5d, 0x91, 0x79, 0xd9, 0x37, 0xfe, 0x5f, 0x0d, 0xae, 0x8e, 0x74, - 0xff, 0x89, 0x67, 0x75, 0x1e, 0xe0, 0x98, 0x2e, 0x1f, 0xd2, 0xa1, 0x0d, 0xfc, 0xd2, 0xaf, 0xd4, - 0xf8, 0x76, 0xd2, 0x98, 0x55, 0x12, 0x76, 0xce, 0x8a, 0x39, 0x67, 0x7f, 0x5c, 0x99, 0xb6, 0x6e, - 0x42, 0x91, 0x55, 0xec, 0x79, 0x86, 0x37, 0x70, 0x47, 0x26, 0xe3, 0x9f, 0xc5, 0x12, 0x90, 0x9d, - 0xc6, 0x1a, 0xd7, 0xeb, 0x90, 0x65, 0xe7, 0x5d, 0x79, 0xda, 0x8b, 0x5c, 0x30, 0x14, 0x3b, 0x74, - 0x21, 0x88, 0x4f, 0x20, 0xfb, 0x94, 0x91, 0x84, 0x8a, 0x65, 0x93, 0x72, 0x2a, 0x2c, 0xa3, 0xc7, - 0xa9, 0x8b, 0x82, 0xce, 0xbe, 0xd9, 0xe1, 0x88, 0x10, 0x67, 0x5f, 0xdf, 0xe2, 0x87, 0xb0, 0x82, - 0xee, 0x97, 0xa9, 0xcb, 0xda, 0x5d, 0x93, 0x58, 0x1e, 0x6b, 0x9d, 0x64, 0xad, 0x4a, 0x0d, 0x5e, - 0x82, 0x2a, 0x47, 0x5a, 0xe9, 0x74, 0x94, 0x43, 0x8e, 0xaf, 0x4f, 0x0b, 0xeb, 0xc3, 0xff, 0xa7, - 0xc1, 0xb4, 0xd2, 0x61, 0x2c, 0xc7, 0xbc, 0x02, 0x59, 0x4e, 0x85, 0x8a, 0x7c, 0x3a, 0x1b, 0xee, - 0xc5, 0x61, 0x74, 0x21, 0x83, 0x96, 0x20, 0xc7, 0xbf, 0xe4, 0x49, 0x33, 0x5e, 0x5c, 0x0a, 0xe1, - 0x7b, 0x30, 0x23, 0xaa, 0x48, 0xcf, 0x8e, 0x5b, 0xdb, 0xcc, 0xa1, 0xf8, 0x9f, 0x60, 0x36, 0x2c, - 0x36, 0xd6, 0x90, 0x14, 0x23, 0x53, 0x2f, 0x62, 0xe4, 0x8a, 0x34, 0x72, 0xbf, 0xdf, 0x51, 0xd2, - 0x7f, 0x74, 0xd6, 0xd5, 0x19, 0x49, 0x45, 0x66, 0xc4, 0x1f, 0x80, 0x54, 0xf1, 0xb3, 0x0e, 0x60, - 0x46, 0x2e, 0x87, 0x2d, 0xd3, 0xf5, 0x0f, 0x85, 0x9f, 0x01, 0x52, 0x2b, 0x7f, 0x6e, 0x83, 0xd6, - 0x88, 0x4c, 0x6a, 0xd2, 0xa0, 0xf7, 0x00, 0xa9, 0x95, 0x63, 0x45, 0xf4, 0x06, 0x4c, 0x3f, 0xb5, - 0x87, 0x34, 0x34, 0xd0, 0xda, 0x60, 0xcb, 0xf0, 0x2b, 0xa2, 0x3f, 0x6d, 0x7e, 0x99, 0x82, 0xab, - 0x1d, 0xc6, 0x02, 0xff, 0xad, 0x06, 0xa5, 0x95, 0xae, 0xe1, 0xf4, 0x24, 0xf0, 0xdb, 0x90, 0xe5, - 0x17, 0x1f, 0xc1, 0x35, 0xbc, 0x14, 0x56, 0xa3, 0xca, 0xf2, 0xc2, 0x0a, 0xbf, 0x26, 0x89, 0x5e, - 0xd4, 0x70, 0xf1, 0x1c, 0xb1, 0x16, 0x79, 0x9e, 0x58, 0x43, 0xaf, 0x42, 0xc6, 0xa0, 0x5d, 0x58, - 0x08, 0xae, 0x44, 0xaf, 0x9c, 0x4c, 0x1b, 0x3b, 0xef, 0x71, 0x29, 0xfc, 0x26, 0x14, 0x15, 0x04, - 0x7a, 0xa9, 0x7e, 0xd2, 0x14, 0x87, 0xb3, 0x95, 0xd5, 0xd6, 0xc6, 0x33, 0x7e, 0xd7, 0xae, 0x00, - 0xac, 0x35, 0xfd, 0x72, 0x0a, 0x7f, 0x28, 0x7a, 0x89, 0x78, 0xa7, 0xda, 0xa3, 0x25, 0xd9, 0x93, - 0x7a, 0x21, 0x7b, 0xce, 0xa0, 0x2c, 0x86, 0x3f, 0x6e, 0xf8, 0x66, 0xfa, 0x12, 0xc2, 0xb7, 0x62, - 0xbc, 0x2e, 0x04, 0xf1, 0x14, 0x94, 0x45, 0x40, 0x17, 0xeb, 0xef, 0x17, 0x29, 0xa8, 0xc8, 0x9a, - 0x71, 0x39, 0x51, 0x49, 0xe7, 0xf0, 0x0c, 0xe0, 0x93, 0x39, 0x57, 0x20, 0xdb, 0x39, 0xdc, 0x33, - 0x3f, 0x93, 0xfc, 0xb5, 0x28, 0xd1, 0xfa, 0x2e, 0xc7, 0xe1, 0x8f, 0x48, 0xa2, 0x44, 0x2f, 0xf6, - 0x8e, 0x71, 0xe4, 0x6d, 0x58, 0x1d, 0x72, 0xc6, 0xce, 0x94, 0x93, 0x7a, 0x50, 0xc1, 0xee, 0xb9, - 0xe2, 0xb1, 0x89, 0x1d, 0x24, 0x95, 0xc7, 0x27, 0xf4, 0x00, 0xaa, 0xf4, 0x7b, 0xa5, 0xdf, 0xef, - 0x9a, 0xa4, 0xc3, 0x15, 0xe4, 0x98, 0xcc, 0x48, 0x3d, 0x45, 0x67, 0x47, 0x2f, 0xb7, 0x96, 0x67, - 0x61, 0x4b, 0x94, 0xd0, 0x02, 0x14, 0xb9, 0x7d, 0x1b, 0xd6, 0xbe, 0x4b, 0xd8, 0x0b, 0x4c, 0x5a, - 0x57, 0xab, 0xe8, 0x3e, 0x5e, 0x19, 0x78, 0x27, 0x4d, 0xcb, 0x38, 0xec, 0xca, 0xb8, 0x48, 0x93, - 0x39, 0xad, 0x5c, 0x33, 0x5d, 0xb5, 0xb6, 0x09, 0x33, 0xb4, 0x96, 0x58, 0x9e, 0xd9, 0x56, 0x82, - 0xa8, 0x4c, 0x95, 0x5a, 0x24, 0x55, 0x1a, 0xae, 0xfb, 0xdc, 0x76, 0x3a, 0xc2, 0x81, 0x7e, 0x19, - 0xaf, 0x71, 0xe5, 0xfb, 0x6e, 0x28, 0x19, 0x7e, 0x5f, 0x2d, 0x8b, 0x81, 0x96, 0x27, 0xc4, 0xbb, - 0x40, 0x0b, 0x7e, 0x08, 0x73, 0x52, 0x52, 0xb0, 0x92, 0x17, 0x08, 0xef, 0xc0, 0x4d, 0x29, 0xbc, - 0x7a, 0x42, 0xaf, 0x7d, 0xbb, 0x02, 0xf0, 0x87, 0xda, 0xf9, 0x18, 0x6a, 0xbe, 0x9d, 0xec, 0x48, - 0x6e, 0x77, 0x55, 0x03, 0x06, 0xae, 0x58, 0x99, 0x05, 0x9d, 0x7d, 0xd3, 0x3a, 0xc7, 0xee, 0xfa, - 0x07, 0x0f, 0xfa, 0x8d, 0x57, 0xe1, 0x9a, 0xd4, 0x21, 0x0e, 0xcb, 0x61, 0x25, 0x23, 0x06, 0xc5, - 0x29, 0x11, 0x0e, 0xa3, 0x5d, 0x2f, 0x76, 0xbb, 0x2a, 0x19, 0x76, 0x2d, 0xd3, 0xa9, 0x29, 0x3a, - 0xe7, 0xf8, 0x8a, 0xa0, 0x86, 0xa9, 0x79, 0x49, 0x54, 0x53, 0x05, 0x6a, 0xb5, 0x98, 0x08, 0x5a, - 0x3d, 0x32, 0x11, 0x23, 0xaa, 0x3f, 0x81, 0x79, 0xdf, 0x08, 0xea, 0xb7, 0x5d, 0xe2, 0xf4, 0x4c, - 0xd7, 0x55, 0x78, 0xac, 0xb8, 0x81, 0xbf, 0x04, 0x93, 0x7d, 0x22, 0x22, 0x57, 0x71, 0x19, 0x2d, - 0xf1, 0x87, 0xe7, 0x25, 0xa5, 0x33, 0x6b, 0xc7, 0x1d, 0xb8, 0x25, 0xb5, 0x73, 0x8f, 0xc6, 0xaa, - 0x8f, 0x1a, 0x25, 0xe9, 0x82, 0x54, 0x02, 0x5d, 0x90, 0x8e, 0x70, 0xab, 0xef, 0x71, 0x47, 0xca, - 0xbd, 0x35, 0x56, 0x46, 0xda, 0xe4, 0x3e, 0xf5, 0xb7, 0xe4, 0x58, 0xca, 0x0e, 0x61, 0x36, 0xbc, - 0x93, 0xc7, 0x0a, 0x96, 0xb3, 0x90, 0xf1, 0xec, 0x53, 0x22, 0x43, 0x25, 0x2f, 0x48, 0x83, 0xfd, - 0x6d, 0x3e, 0x96, 0xc1, 0x46, 0xa0, 0x8c, 0x2d, 0xc9, 0x71, 0xed, 0xa5, 0xb3, 0x29, 0x8f, 0x78, - 0xbc, 0x80, 0xb7, 0xe1, 0x4a, 0x34, 0x4c, 0x8c, 0x65, 0xf2, 0x33, 0xbe, 0x80, 0xe3, 0x22, 0xc9, - 0x58, 0x7a, 0xdf, 0x0f, 0x82, 0x81, 0x12, 0x50, 0xc6, 0x52, 0xa9, 0x43, 0x3d, 0x2e, 0xbe, 0xfc, - 0x18, 0xeb, 0xd5, 0x0f, 0x37, 0x63, 0x29, 0x73, 0x03, 0x65, 0xe3, 0x4f, 0x7f, 0x10, 0x23, 0xd2, - 0x17, 0xc6, 0x08, 0xb1, 0x49, 0x82, 0x28, 0xf6, 0x13, 0x2c, 0x3a, 0x81, 0x11, 0x04, 0xd0, 0x71, - 0x31, 0x68, 0x0e, 0xf1, 0x31, 0x58, 0x41, 0x2e, 0x6c, 0x35, 0xec, 0x8e, 0x35, 0x19, 0x1f, 0x04, - 0xb1, 0x73, 0x24, 0x32, 0x8f, 0xa5, 0xf8, 0x43, 0x58, 0x48, 0x0e, 0xca, 0xe3, 0x68, 0x7e, 0xd0, - 0x80, 0x82, 0x7f, 0x6c, 0x55, 0x7e, 0xb4, 0x51, 0x84, 0xdc, 0xf6, 0xce, 0xde, 0xee, 0xca, 0x6a, - 0x93, 0xff, 0x6a, 0x63, 0x75, 0x47, 0xd7, 0xf7, 0x77, 0x5b, 0xd5, 0xd4, 0xf2, 0x77, 0x69, 0x48, - 0x6d, 0x3e, 0x43, 0x1f, 0x41, 0x86, 0x3f, 0x61, 0x5e, 0xf0, 0x6e, 0x5d, 0xbf, 0xe8, 0x95, 0x16, - 0x5f, 0xfd, 0xe2, 0x0f, 0xdf, 0x7d, 0x9d, 0x9a, 0xc6, 0xa5, 0xc6, 0xf0, 0x8d, 0xc6, 0xe9, 0xb0, - 0xc1, 0x72, 0xc3, 0x23, 0xed, 0x01, 0x7a, 0x1f, 0xd2, 0xbb, 0x03, 0x0f, 0x25, 0xbe, 0x67, 0xd7, - 0x93, 0x1f, 0x6e, 0xf1, 0x1c, 0x53, 0x3a, 0x85, 0x41, 0x28, 0xed, 0x0f, 0x3c, 0xaa, 0xf2, 0x53, - 0x28, 0xaa, 0xcf, 0xae, 0x97, 0x3e, 0x72, 0xd7, 0x2f, 0x7f, 0xd2, 0xc5, 0x37, 0x19, 0xd4, 0x55, - 0x8c, 0x04, 0x14, 0x7f, 0x18, 0x56, 0x47, 0xd1, 0x3a, 0xb3, 0x50, 0xe2, 0x13, 0x78, 0x3d, 0xf9, - 0x95, 0x77, 0x64, 0x14, 0xde, 0x99, 0x45, 0x55, 0xfe, 0x83, 0x78, 0xe0, 0x6d, 0x7b, 0xe8, 0x56, - 0xcc, 0x03, 0x9f, 0xfa, 0x94, 0x55, 0x5f, 0x48, 0x16, 0x10, 0x20, 0x37, 0x18, 0xc8, 0x15, 0x3c, - 0x2d, 0x40, 0xda, 0xbe, 0xc8, 0x23, 0xed, 0xc1, 0x72, 0x1b, 0x32, 0x8c, 0x8f, 0x46, 0x1f, 0xcb, - 0x8f, 0x7a, 0x0c, 0x01, 0x9f, 0x30, 0xd1, 0x21, 0x26, 0x1b, 0xcf, 0x32, 0xa0, 0x0a, 0x2e, 0x50, - 0x20, 0xc6, 0x46, 0x3f, 0xd2, 0x1e, 0x2c, 0x6a, 0xaf, 0x69, 0xcb, 0xff, 0x9f, 0x81, 0x0c, 0x23, - 0x9f, 0xd0, 0x29, 0x40, 0xc0, 0xcd, 0x46, 0x47, 0x37, 0xc2, 0xf6, 0x46, 0x47, 0x37, 0x4a, 0xeb, - 0xe2, 0x3a, 0x03, 0x9d, 0xc5, 0x53, 0x14, 0x94, 0x71, 0x5a, 0x0d, 0x46, 0xd3, 0x51, 0x3f, 0xfe, - 0x9b, 0x26, 0xb8, 0x37, 0xbe, 0x97, 0x50, 0x9c, 0xb6, 0x10, 0x41, 0x1b, 0x5d, 0x0e, 0x31, 0xe4, - 0x2c, 0x7e, 0x8b, 0x01, 0x36, 0x70, 0x35, 0x00, 0x74, 0x98, 0xc4, 0x23, 0xed, 0xc1, 0xc7, 0x35, - 0x3c, 0x23, 0xbc, 0x1c, 0x69, 0x41, 0x9f, 0x43, 0x25, 0x4c, 0xba, 0xa2, 0x3b, 0x31, 0x58, 0x51, - 0xee, 0xb6, 0x7e, 0xf7, 0x62, 0x21, 0x61, 0xd3, 0x3c, 0xb3, 0x49, 0x80, 0x73, 0xe4, 0x53, 0x42, - 0xfa, 0x06, 0x15, 0x12, 0x73, 0x80, 0xfe, 0x5b, 0x83, 0xa9, 0x08, 0x8b, 0x8a, 0xe2, 0xb4, 0x8f, - 0x70, 0xb4, 0xf5, 0x7b, 0x97, 0x48, 0x09, 0x23, 0xfe, 0x8e, 0x19, 0xf1, 0xd7, 0x78, 0x36, 0x30, - 0xc2, 0x33, 0x7b, 0xc4, 0xb3, 0x85, 0x15, 0x1f, 0xdf, 0xc0, 0x57, 0x43, 0xce, 0x09, 0xb5, 0x06, - 0x93, 0xc5, 0x99, 0xd0, 0xd8, 0xc9, 0x0a, 0x31, 0xab, 0xb1, 0x93, 0x15, 0xa6, 0x51, 0xe3, 0x26, - 0x8b, 0xf3, 0x9e, 0x71, 0x93, 0xe5, 0xb7, 0x2c, 0xb3, 0x9f, 0x58, 0xf0, 0x1f, 0x56, 0x22, 0x1b, - 0x0a, 0x3e, 0x0b, 0x89, 0xe6, 0xe3, 0x18, 0xa1, 0xe0, 0x2e, 0x51, 0xbf, 0x95, 0xd8, 0x2e, 0x0c, - 0xba, 0xcd, 0x0c, 0xba, 0x8e, 0xaf, 0x50, 0x64, 0xf1, 0xdb, 0xcd, 0x06, 0xa7, 0x1d, 0x1a, 0x46, - 0xa7, 0x43, 0x1d, 0xf1, 0x8f, 0x50, 0x52, 0x69, 0x42, 0x74, 0x3b, 0x96, 0x85, 0x52, 0x99, 0xc6, - 0x3a, 0xbe, 0x48, 0x44, 0x20, 0xdf, 0x65, 0xc8, 0xf3, 0xf8, 0x5a, 0x0c, 0xb2, 0xc3, 0x44, 0x43, - 0xe0, 0x9c, 0xe2, 0x8b, 0x07, 0x0f, 0x31, 0x88, 0xf1, 0xe0, 0x61, 0x86, 0xf0, 0x42, 0xf0, 0x01, - 0x13, 0xa5, 0xe0, 0x2e, 0x40, 0x40, 0xe6, 0xa1, 0x58, 0x5f, 0x2a, 0x97, 0xa9, 0x68, 0x70, 0x18, - 0xe5, 0x01, 0x31, 0x66, 0xb0, 0x62, 0xdd, 0x45, 0x60, 0xbb, 0xa6, 0x4b, 0x83, 0xc4, 0xf2, 0xbf, - 0x67, 0xa1, 0xf8, 0xd4, 0x30, 0x2d, 0x8f, 0x58, 0x86, 0xd5, 0x26, 0xe8, 0x10, 0x32, 0x2c, 0x51, - 0x46, 0xe3, 0xa0, 0xca, 0x6f, 0x45, 0xe3, 0x60, 0x88, 0xfc, 0xc1, 0x0b, 0x0c, 0xb5, 0x8e, 0xe7, - 0x28, 0x6a, 0x2f, 0x50, 0xdd, 0x60, 0x9c, 0x0d, 0x1d, 0xe8, 0x11, 0x64, 0xc5, 0x73, 0x40, 0x44, - 0x51, 0x88, 0xcb, 0xa9, 0xdf, 0x88, 0x6f, 0x8c, 0x5b, 0x4a, 0x2a, 0x8c, 0xcb, 0xe4, 0x28, 0xce, - 0x10, 0x20, 0x20, 0x23, 0xa3, 0x0e, 0x1d, 0xe1, 0x2e, 0xeb, 0x0b, 0xc9, 0x02, 0x02, 0xf3, 0x1e, - 0xc3, 0xbc, 0x85, 0xeb, 0x51, 0xcc, 0x8e, 0x2f, 0x4b, 0x71, 0xff, 0x1e, 0x26, 0xd7, 0x0d, 0xf7, - 0x04, 0x45, 0x52, 0x9f, 0xf2, 0x7b, 0x87, 0x7a, 0x3d, 0xae, 0x49, 0xa0, 0xdc, 0x62, 0x28, 0xd7, - 0x78, 0x24, 0x51, 0x51, 0x4e, 0x0c, 0x97, 0xe6, 0x14, 0xd4, 0x81, 0x2c, 0xff, 0xf9, 0x43, 0xd4, - 0x7f, 0xa1, 0x9f, 0x50, 0x44, 0xfd, 0x17, 0xfe, 0xc5, 0xc4, 0xe5, 0x28, 0x7d, 0xc8, 0xcb, 0xdf, - 0x1b, 0xa0, 0x9b, 0x91, 0xa9, 0x08, 0xff, 0x36, 0xa1, 0x3e, 0x9f, 0xd4, 0x2c, 0xb0, 0xee, 0x30, - 0xac, 0x9b, 0xb8, 0x36, 0x32, 0x57, 0x42, 0xf2, 0x91, 0xf6, 0xe0, 0x35, 0x0d, 0x7d, 0x0e, 0x10, - 0xf0, 0xb7, 0x23, 0x1b, 0x20, 0x4a, 0x05, 0x8f, 0x6c, 0x80, 0x11, 0xea, 0x17, 0x2f, 0x31, 0xdc, - 0x45, 0x7c, 0x27, 0x8a, 0xeb, 0x39, 0x86, 0xe5, 0x1e, 0x11, 0xe7, 0x55, 0xce, 0xd1, 0xb9, 0x27, - 0x66, 0x9f, 0x6e, 0x86, 0x5f, 0x4f, 0xc1, 0x24, 0x3d, 0x80, 0xd2, 0x3c, 0x1d, 0xdc, 0xdb, 0xa3, - 0x96, 0x8c, 0xb0, 0x65, 0x51, 0x4b, 0x46, 0xaf, 0xfc, 0xe1, 0x3c, 0xcd, 0x7e, 0x11, 0x4f, 0x98, - 0x00, 0x75, 0xb4, 0x0d, 0x45, 0xe5, 0x62, 0x8f, 0x62, 0x94, 0x85, 0x69, 0xb8, 0x68, 0xe4, 0x8f, - 0x61, 0x05, 0xf0, 0x75, 0x86, 0x37, 0xc7, 0x23, 0x3f, 0xc3, 0xeb, 0x70, 0x09, 0x0a, 0xf8, 0x1c, - 0x4a, 0xea, 0xe5, 0x1f, 0xc5, 0xe8, 0x8b, 0x50, 0x7c, 0xd1, 0x28, 0x17, 0xc7, 0x1d, 0x84, 0x37, - 0xbe, 0xff, 0xab, 0x7f, 0x29, 0x46, 0x81, 0xbb, 0x90, 0x13, 0x6c, 0x40, 0xdc, 0x28, 0xc3, 0x7c, - 0x60, 0xdc, 0x28, 0x23, 0x54, 0x42, 0xf8, 0x6c, 0xc7, 0x10, 0xe9, 0x85, 0x47, 0x66, 0x12, 0x81, - 0xf6, 0x84, 0x78, 0x49, 0x68, 0x01, 0xb9, 0x95, 0x84, 0xa6, 0x5c, 0x36, 0x93, 0xd0, 0x8e, 0x89, - 0x27, 0xb6, 0x8b, 0xbc, 0xc4, 0xa1, 0x04, 0x65, 0x6a, 0xf4, 0xc6, 0x17, 0x89, 0xc4, 0x1d, 0xbd, - 0x03, 0x40, 0x11, 0xba, 0xd1, 0x19, 0x40, 0xc0, 0x55, 0x44, 0xcf, 0x53, 0xb1, 0x84, 0x67, 0xf4, - 0x3c, 0x15, 0x4f, 0x77, 0x84, 0x43, 0x43, 0x80, 0xcb, 0x4f, 0xfe, 0x14, 0xf9, 0x2b, 0x0d, 0xd0, - 0x28, 0xad, 0x81, 0x1e, 0xc6, 0x6b, 0x8f, 0xa5, 0x51, 0xeb, 0xaf, 0xbc, 0x98, 0x70, 0x5c, 0xb4, - 0x0f, 0x4c, 0x6a, 0x33, 0xe9, 0xfe, 0x73, 0x6a, 0xd4, 0xbf, 0x68, 0x50, 0x0e, 0x71, 0x22, 0xe8, - 0xa5, 0x84, 0x39, 0x8d, 0xb0, 0xb0, 0xf5, 0xfb, 0x97, 0xca, 0xc5, 0x1d, 0x34, 0x95, 0x15, 0x20, - 0x4f, 0xdc, 0x5f, 0x6a, 0x50, 0x09, 0x73, 0x28, 0x28, 0x41, 0xf7, 0x08, 0x8b, 0x5b, 0x5f, 0xbc, - 0x5c, 0xf0, 0xe2, 0xe9, 0x09, 0x0e, 0xdb, 0x5d, 0xc8, 0x09, 0xd6, 0x25, 0x6e, 0xe1, 0x87, 0xf9, - 0xdf, 0xb8, 0x85, 0x1f, 0xa1, 0x6c, 0x62, 0x16, 0xbe, 0x63, 0x77, 0x89, 0xb2, 0xcd, 0x04, 0x2d, - 0x93, 0x84, 0x76, 0xf1, 0x36, 0x8b, 0x70, 0x3a, 0x49, 0x68, 0xc1, 0x36, 0x93, 0x7c, 0x0c, 0x4a, - 0x50, 0x76, 0xc9, 0x36, 0x8b, 0xd2, 0x39, 0x31, 0xdb, 0x8c, 0x01, 0x2a, 0xdb, 0x2c, 0x60, 0x4e, - 0xe2, 0xb6, 0xd9, 0x08, 0x9d, 0x1d, 0xb7, 0xcd, 0x46, 0xc9, 0x97, 0x98, 0x79, 0x64, 0xb8, 0xa1, - 0x6d, 0x36, 0x13, 0x43, 0xb2, 0xa0, 0x57, 0x12, 0x9c, 0x18, 0xcb, 0x92, 0xd7, 0x5f, 0x7d, 0x41, - 0xe9, 0xc4, 0x35, 0xce, 0xdd, 0x2f, 0xd7, 0xf8, 0x7f, 0x68, 0x30, 0x1b, 0x47, 0xd0, 0xa0, 0x04, - 0x9c, 0x04, 0x76, 0xbd, 0xbe, 0xf4, 0xa2, 0xe2, 0x17, 0x7b, 0xcb, 0x5f, 0xf5, 0x8f, 0xab, 0xbf, - 0xf9, 0x76, 0x5e, 0xfb, 0xdd, 0xb7, 0xf3, 0xda, 0x1f, 0xbf, 0x9d, 0xd7, 0xfe, 0xf3, 0x4f, 0xf3, - 0x13, 0x87, 0x59, 0xf6, 0x7f, 0xc9, 0xde, 0xf8, 0x4b, 0x00, 0x00, 0x00, 0xff, 0xff, 0x91, 0xd4, - 0xc1, 0xf9, 0xd2, 0x36, 0x00, 0x00, + // 3773 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xbc, 0x5b, 0x5b, 0x6f, 0x1c, 0x47, + 0x76, 0x66, 0xcf, 0x70, 0x6e, 0x67, 0x2e, 0x1c, 0x16, 0x2f, 0x1a, 0x8d, 0x24, 0x8a, 0x2a, 0x49, + 0x16, 0x2d, 0xd9, 0x1c, 0x9b, 0xb6, 0x13, 0x40, 0x49, 0x0c, 0x53, 0xe4, 0x58, 0xa4, 0x49, 0x91, + 0x74, 0x73, 0x28, 0x5f, 0x60, 0x84, 0x68, 0xce, 0x94, 0xc8, 0x0e, 0x67, 0xba, 0xc7, 0xdd, 0x3d, + 0x23, 0xd2, 0x09, 0xe2, 0xc0, 0x70, 0x02, 0x24, 0x8f, 0x36, 0x10, 0x24, 0x0f, 0x79, 0x0a, 0x82, + 0xc0, 0x0f, 0x0b, 0xec, 0xdb, 0x02, 0xfb, 0x0b, 0xf6, 0x6d, 0x77, 0xb1, 0x7f, 0x60, 0xe1, 0xf5, + 0xcb, 0xfe, 0x8b, 0x45, 0xdd, 0xba, 0xab, 0x7b, 0xba, 0x49, 0xd9, 0x63, 0xfb, 0x85, 0xea, 0xaa, + 0x3a, 0x75, 0xbe, 0x53, 0xa7, 0xaa, 0xce, 0xa9, 0xfa, 0x6a, 0x04, 0x05, 0xa7, 0xdf, 0x5e, 0xee, + 0x3b, 0xb6, 0x67, 0xa3, 0x12, 0xf1, 0xda, 0x1d, 0x97, 0x38, 0x43, 0xe2, 0xf4, 0x8f, 0xea, 0xb3, + 0xc7, 0xf6, 0xb1, 0xcd, 0x1a, 0x1a, 0xf4, 0x8b, 0xcb, 0xd4, 0xaf, 0x52, 0x99, 0x46, 0x6f, 0xd8, + 0x6e, 0xb3, 0x3f, 0xfd, 0xa3, 0xc6, 0xe9, 0x50, 0x34, 0x5d, 0x63, 0x4d, 0xc6, 0xc0, 0x3b, 0x61, + 0x7f, 0xfa, 0x47, 0xec, 0x1f, 0xd1, 0x78, 0xfd, 0xd8, 0xb6, 0x8f, 0xbb, 0xa4, 0x61, 0xf4, 0xcd, + 0x86, 0x61, 0x59, 0xb6, 0x67, 0x78, 0xa6, 0x6d, 0xb9, 0xbc, 0x15, 0xff, 0xab, 0x06, 0x15, 0x9d, + 0xb8, 0x7d, 0xdb, 0x72, 0xc9, 0x06, 0x31, 0x3a, 0xc4, 0x41, 0x37, 0x00, 0xda, 0xdd, 0x81, 0xeb, + 0x11, 0xe7, 0xd0, 0xec, 0xd4, 0xb4, 0x45, 0x6d, 0x69, 0x52, 0x2f, 0x88, 0x9a, 0xcd, 0x0e, 0xba, + 0x06, 0x85, 0x1e, 0xe9, 0x1d, 0xf1, 0xd6, 0x14, 0x6b, 0xcd, 0xf3, 0x8a, 0xcd, 0x0e, 0xaa, 0x43, + 0xde, 0x21, 0x43, 0xd3, 0x35, 0x6d, 0xab, 0x96, 0x5e, 0xd4, 0x96, 0xd2, 0xba, 0x5f, 0xa6, 0x1d, + 0x1d, 0xe3, 0x99, 0x77, 0xe8, 0x11, 0xa7, 0x57, 0x9b, 0xe4, 0x1d, 0x69, 0x45, 0x8b, 0x38, 0x3d, + 0xfc, 0x65, 0x06, 0x4a, 0xba, 0x61, 0x1d, 0x13, 0x9d, 0x7c, 0x3a, 0x20, 0xae, 0x87, 0xaa, 0x90, + 0x3e, 0x25, 0xe7, 0x0c, 0xbe, 0xa4, 0xd3, 0x4f, 0xde, 0xdf, 0x3a, 0x26, 0x87, 0xc4, 0xe2, 0xc0, + 0x25, 0xda, 0xdf, 0x3a, 0x26, 0x4d, 0xab, 0x83, 0x66, 0x21, 0xd3, 0x35, 0x7b, 0xa6, 0x27, 0x50, + 0x79, 0x21, 0x64, 0xce, 0x64, 0xc4, 0x9c, 0x35, 0x00, 0xd7, 0x76, 0xbc, 0x43, 0xdb, 0xe9, 0x10, + 0xa7, 0x96, 0x59, 0xd4, 0x96, 0x2a, 0x2b, 0x77, 0x96, 0xd5, 0x89, 0x58, 0x56, 0x0d, 0x5a, 0xde, + 0xb7, 0x1d, 0x6f, 0x97, 0xca, 0xea, 0x05, 0x57, 0x7e, 0xa2, 0x77, 0xa1, 0xc8, 0x94, 0x78, 0x86, + 0x73, 0x4c, 0xbc, 0x5a, 0x96, 0x69, 0xb9, 0x7b, 0x89, 0x96, 0x16, 0x13, 0xd6, 0x19, 0x3c, 0xff, + 0x46, 0x18, 0x4a, 0x2e, 0x71, 0x4c, 0xa3, 0x6b, 0x7e, 0x66, 0x1c, 0x75, 0x49, 0x2d, 0xb7, 0xa8, + 0x2d, 0xe5, 0xf5, 0x50, 0x1d, 0x1d, 0xff, 0x29, 0x39, 0x77, 0x0f, 0x6d, 0xab, 0x7b, 0x5e, 0xcb, + 0x33, 0x81, 0x3c, 0xad, 0xd8, 0xb5, 0xba, 0xe7, 0x6c, 0xd2, 0xec, 0x81, 0xe5, 0xf1, 0xd6, 0x02, + 0x6b, 0x2d, 0xb0, 0x1a, 0xd6, 0xbc, 0x04, 0xd5, 0x9e, 0x69, 0x1d, 0xf6, 0xec, 0xce, 0xa1, 0xef, + 0x10, 0x60, 0x0e, 0xa9, 0xf4, 0x4c, 0xeb, 0x89, 0xdd, 0xd1, 0xa5, 0x5b, 0xa8, 0xa4, 0x71, 0x16, + 0x96, 0x2c, 0x0a, 0x49, 0xe3, 0x4c, 0x95, 0x5c, 0x86, 0x19, 0xaa, 0xb3, 0xed, 0x10, 0xc3, 0x23, + 0x81, 0x70, 0x89, 0x09, 0x4f, 0xf7, 0x4c, 0x6b, 0x8d, 0xb5, 0x84, 0xe4, 0x8d, 0xb3, 0x11, 0xf9, + 0xb2, 0x90, 0x37, 0xce, 0xc2, 0xf2, 0x78, 0x19, 0x0a, 0xbe, 0xcf, 0x51, 0x1e, 0x26, 0x77, 0x76, + 0x77, 0x9a, 0xd5, 0x09, 0x04, 0x90, 0x5d, 0xdd, 0x5f, 0x6b, 0xee, 0xac, 0x57, 0x35, 0x54, 0x84, + 0xdc, 0x7a, 0x93, 0x17, 0x52, 0xf8, 0x11, 0x40, 0xe0, 0x5d, 0x94, 0x83, 0xf4, 0x56, 0xf3, 0xa3, + 0xea, 0x04, 0x95, 0x79, 0xda, 0xd4, 0xf7, 0x37, 0x77, 0x77, 0xaa, 0x1a, 0xed, 0xbc, 0xa6, 0x37, + 0x57, 0x5b, 0xcd, 0x6a, 0x8a, 0x4a, 0x3c, 0xd9, 0x5d, 0xaf, 0xa6, 0x51, 0x01, 0x32, 0x4f, 0x57, + 0xb7, 0x0f, 0x9a, 0xd5, 0x49, 0xfc, 0xb5, 0x06, 0x65, 0x31, 0x5f, 0x7c, 0x4f, 0xa0, 0x37, 0x21, + 0x7b, 0xc2, 0xf6, 0x05, 0x5b, 0x8a, 0xc5, 0x95, 0xeb, 0x91, 0xc9, 0x0d, 0xed, 0x1d, 0x5d, 0xc8, + 0x22, 0x0c, 0xe9, 0xd3, 0xa1, 0x5b, 0x4b, 0x2d, 0xa6, 0x97, 0x8a, 0x2b, 0xd5, 0x65, 0xbe, 0x61, + 0x97, 0xb7, 0xc8, 0xf9, 0x53, 0xa3, 0x3b, 0x20, 0x3a, 0x6d, 0x44, 0x08, 0x26, 0x7b, 0xb6, 0x43, + 0xd8, 0x8a, 0xcd, 0xeb, 0xec, 0x9b, 0x2e, 0x63, 0x36, 0x69, 0x62, 0xb5, 0xf2, 0x02, 0xfe, 0x46, + 0x03, 0xd8, 0x1b, 0x78, 0xc9, 0x5b, 0x63, 0x16, 0x32, 0x43, 0xaa, 0x58, 0x6c, 0x0b, 0x5e, 0x60, + 0x7b, 0x82, 0x18, 0x2e, 0xf1, 0xf7, 0x04, 0x2d, 0xa0, 0x2b, 0x90, 0xeb, 0x3b, 0x64, 0x78, 0x78, + 0x3a, 0x64, 0x20, 0x79, 0x3d, 0x4b, 0x8b, 0x5b, 0x43, 0x74, 0x0b, 0x4a, 0xe6, 0xb1, 0x65, 0x3b, + 0xe4, 0x90, 0xeb, 0xca, 0xb0, 0xd6, 0x22, 0xaf, 0x63, 0x76, 0x2b, 0x22, 0x5c, 0x71, 0x56, 0x15, + 0xd9, 0xa6, 0x55, 0xd8, 0x82, 0x22, 0x33, 0x75, 0x2c, 0xf7, 0xbd, 0x1c, 0xd8, 0x98, 0x62, 0xdd, + 0x46, 0x5d, 0x28, 0xac, 0xc6, 0x9f, 0x00, 0x5a, 0x27, 0x5d, 0xe2, 0x91, 0x71, 0xa2, 0x87, 0xe2, + 0x93, 0xb4, 0xea, 0x13, 0xfc, 0x95, 0x06, 0x33, 0x21, 0xf5, 0x63, 0x0d, 0xab, 0x06, 0xb9, 0x0e, + 0x53, 0xc6, 0x2d, 0x48, 0xeb, 0xb2, 0x88, 0x1e, 0x40, 0x5e, 0x18, 0xe0, 0xd6, 0xd2, 0x09, 0x8b, + 0x26, 0xc7, 0x6d, 0x72, 0xf1, 0x37, 0x29, 0x28, 0x88, 0x81, 0xee, 0xf6, 0xd1, 0x2a, 0x94, 0x1d, + 0x5e, 0x38, 0x64, 0xe3, 0x11, 0x16, 0xd5, 0x93, 0x83, 0xd0, 0xc6, 0x84, 0x5e, 0x12, 0x5d, 0x58, + 0x35, 0xfa, 0x1b, 0x28, 0x4a, 0x15, 0xfd, 0x81, 0x27, 0x5c, 0x5e, 0x0b, 0x2b, 0x08, 0xd6, 0xdf, + 0xc6, 0x84, 0x0e, 0x42, 0x7c, 0x6f, 0xe0, 0xa1, 0x16, 0xcc, 0xca, 0xce, 0x7c, 0x34, 0xc2, 0x8c, + 0x34, 0xd3, 0xb2, 0x18, 0xd6, 0x32, 0x3a, 0x55, 0x1b, 0x13, 0x3a, 0x12, 0xfd, 0x95, 0x46, 0xd5, + 0x24, 0xef, 0x8c, 0x07, 0xef, 0x11, 0x93, 0x5a, 0x67, 0xd6, 0xa8, 0x49, 0xad, 0x33, 0xeb, 0x51, + 0x01, 0x72, 0xa2, 0x84, 0x7f, 0x95, 0x02, 0x90, 0xb3, 0xb1, 0xdb, 0x47, 0xeb, 0x50, 0x71, 0x44, + 0x29, 0xe4, 0xad, 0x6b, 0xb1, 0xde, 0x12, 0x93, 0x38, 0xa1, 0x97, 0x65, 0x27, 0x6e, 0xdc, 0xdb, + 0x50, 0xf2, 0xb5, 0x04, 0x0e, 0xbb, 0x1a, 0xe3, 0x30, 0x5f, 0x43, 0x51, 0x76, 0xa0, 0x2e, 0xfb, + 0x00, 0xe6, 0xfc, 0xfe, 0x31, 0x3e, 0xbb, 0x75, 0x81, 0xcf, 0x7c, 0x85, 0x33, 0x52, 0x83, 0xea, + 0x35, 0xd5, 0xb0, 0xc0, 0x6d, 0x57, 0x63, 0xdc, 0x36, 0x6a, 0x18, 0x75, 0x1c, 0xd0, 0x7c, 0xc9, + 0x8b, 0xf8, 0xcf, 0x69, 0xc8, 0xad, 0xd9, 0xbd, 0xbe, 0xe1, 0xd0, 0xd9, 0xc8, 0x3a, 0xc4, 0x1d, + 0x74, 0x3d, 0xe6, 0xae, 0xca, 0xca, 0xed, 0xb0, 0x46, 0x21, 0x26, 0xff, 0xd5, 0x99, 0xa8, 0x2e, + 0xba, 0xd0, 0xce, 0x22, 0x3d, 0xa6, 0x5e, 0xa0, 0xb3, 0x48, 0x8e, 0xa2, 0x8b, 0xdc, 0xc8, 0xe9, + 0x60, 0x23, 0xd7, 0x21, 0x37, 0x24, 0x4e, 0x90, 0xd2, 0x37, 0x26, 0x74, 0x59, 0x81, 0x5e, 0x86, + 0xa9, 0x68, 0x7a, 0xc9, 0x08, 0x99, 0x4a, 0x3b, 0x9c, 0x8d, 0x6e, 0x43, 0x29, 0x94, 0xe3, 0xb2, + 0x42, 0xae, 0xd8, 0x53, 0x52, 0xdc, 0xbc, 0x8c, 0xab, 0x34, 0x1f, 0x97, 0x36, 0x26, 0x64, 0x64, + 0x9d, 0x97, 0x91, 0x35, 0x2f, 0x7a, 0x89, 0xd8, 0x1a, 0x0a, 0x32, 0xef, 0x84, 0x83, 0x0c, 0x7e, + 0x07, 0xca, 0x21, 0x07, 0xd1, 0xbc, 0xd3, 0x7c, 0xff, 0x60, 0x75, 0x9b, 0x27, 0xa9, 0xc7, 0x2c, + 0x2f, 0xe9, 0x55, 0x8d, 0xe6, 0xba, 0xed, 0xe6, 0xfe, 0x7e, 0x35, 0x85, 0xca, 0x50, 0xd8, 0xd9, + 0x6d, 0x1d, 0x72, 0xa9, 0x34, 0x7e, 0xec, 0x6b, 0x10, 0x49, 0x4e, 0xc9, 0x6d, 0x13, 0x4a, 0x6e, + 0xd3, 0x64, 0x6e, 0x4b, 0x05, 0xb9, 0x8d, 0xa5, 0xb9, 0xed, 0xe6, 0xea, 0x7e, 0xb3, 0x3a, 0xf9, + 0xa8, 0x02, 0x25, 0xee, 0xdf, 0xc3, 0x81, 0x45, 0x53, 0xed, 0xff, 0x6a, 0x00, 0xc1, 0x6e, 0x42, + 0x0d, 0xc8, 0xb5, 0x39, 0x4e, 0x4d, 0x63, 0xc1, 0x68, 0x2e, 0x76, 0xca, 0x74, 0x29, 0x85, 0x5e, + 0x87, 0x9c, 0x3b, 0x68, 0xb7, 0x89, 0x2b, 0x53, 0xde, 0x95, 0x68, 0x3c, 0x14, 0xd1, 0x4a, 0x97, + 0x72, 0xb4, 0xcb, 0x33, 0xc3, 0xec, 0x0e, 0x58, 0x02, 0xbc, 0xb8, 0x8b, 0x90, 0xc3, 0xff, 0xad, + 0x41, 0x51, 0x59, 0xbc, 0x3f, 0x30, 0x08, 0x5f, 0x87, 0x02, 0xb3, 0x81, 0x74, 0x44, 0x18, 0xce, + 0xeb, 0x41, 0x05, 0xfa, 0x2b, 0x28, 0xc8, 0x1d, 0x20, 0x23, 0x71, 0x2d, 0x5e, 0xed, 0x6e, 0x5f, + 0x0f, 0x44, 0xf1, 0x16, 0x4c, 0x33, 0xaf, 0xb4, 0xe9, 0xe1, 0x5a, 0xfa, 0x51, 0x3d, 0x7e, 0x6a, + 0x91, 0xe3, 0x67, 0x1d, 0xf2, 0xfd, 0x93, 0x73, 0xd7, 0x6c, 0x1b, 0x5d, 0x61, 0x85, 0x5f, 0xc6, + 0xef, 0x01, 0x52, 0x95, 0x8d, 0x33, 0x5c, 0x5c, 0x86, 0xe2, 0x86, 0xe1, 0x9e, 0x08, 0x93, 0xf0, + 0x03, 0x28, 0xd3, 0xe2, 0xd6, 0xd3, 0x17, 0xb0, 0x91, 0x5d, 0x0e, 0xa4, 0xf4, 0x58, 0x3e, 0x47, + 0x30, 0x79, 0x62, 0xb8, 0x27, 0x6c, 0xa0, 0x65, 0x9d, 0x7d, 0xa3, 0x97, 0xa1, 0xda, 0xe6, 0x83, + 0x3c, 0x8c, 0x5c, 0x19, 0xa6, 0x44, 0xbd, 0x7f, 0x12, 0xfc, 0x10, 0x4a, 0x7c, 0x0c, 0x3f, 0xb6, + 0x11, 0x78, 0x1a, 0xa6, 0xf6, 0x2d, 0xa3, 0xef, 0x9e, 0xd8, 0x32, 0xbb, 0xd1, 0x41, 0x57, 0x83, + 0xba, 0xb1, 0x10, 0xef, 0xc1, 0x94, 0x43, 0x7a, 0x86, 0x69, 0x99, 0xd6, 0xf1, 0xe1, 0xd1, 0xb9, + 0x47, 0x5c, 0x71, 0x61, 0xaa, 0xf8, 0xd5, 0x8f, 0x68, 0x2d, 0x35, 0xed, 0xa8, 0x6b, 0x1f, 0x89, + 0x30, 0xc7, 0xbe, 0xf1, 0xbf, 0xa5, 0xa0, 0xf4, 0x81, 0xe1, 0xb5, 0xe5, 0xd4, 0xa1, 0x4d, 0xa8, + 0xf8, 0xc1, 0x8d, 0xd5, 0x08, 0x5b, 0x22, 0x29, 0x96, 0xf5, 0x91, 0x47, 0x69, 0x99, 0x1d, 0xcb, + 0x6d, 0xb5, 0x82, 0xa9, 0x32, 0xac, 0x36, 0xe9, 0xfa, 0xaa, 0x52, 0xc9, 0xaa, 0x98, 0xa0, 0xaa, + 0x4a, 0xad, 0x40, 0xbb, 0x50, 0xed, 0x3b, 0xf6, 0xb1, 0x43, 0x5c, 0xd7, 0x57, 0xc6, 0xd3, 0x18, + 0x8e, 0x51, 0xb6, 0x27, 0x44, 0x03, 0x75, 0x53, 0xfd, 0x70, 0xd5, 0xa3, 0xa9, 0xe0, 0x3c, 0xc3, + 0x83, 0xd3, 0xef, 0x53, 0x80, 0x46, 0x07, 0xf5, 0x7d, 0x8f, 0x78, 0x77, 0xa1, 0xe2, 0x7a, 0x86, + 0x33, 0xb2, 0xd8, 0xca, 0xac, 0xd6, 0x8f, 0xf8, 0xf7, 0xc0, 0x37, 0xe8, 0xd0, 0xb2, 0x3d, 0xf3, + 0xd9, 0xb9, 0x38, 0x25, 0x57, 0x64, 0xf5, 0x0e, 0xab, 0x45, 0x4d, 0xc8, 0x3d, 0x33, 0xbb, 0x1e, + 0x71, 0xdc, 0x5a, 0x66, 0x31, 0xbd, 0x54, 0x59, 0x79, 0x70, 0xd9, 0x34, 0x2c, 0xbf, 0xcb, 0xe4, + 0x5b, 0xe7, 0x7d, 0xa2, 0xcb, 0xbe, 0xea, 0xc9, 0x33, 0x1b, 0x3a, 0x8d, 0x5f, 0x85, 0xfc, 0x73, + 0xaa, 0x82, 0xde, 0xb2, 0x73, 0xfc, 0xb0, 0xc8, 0xca, 0xfc, 0x92, 0xfd, 0xcc, 0x31, 0x8e, 0x7b, + 0xc4, 0xf2, 0xe4, 0x3d, 0x50, 0x96, 0xf1, 0x5d, 0x80, 0x00, 0x86, 0x86, 0xfc, 0x9d, 0xdd, 0xbd, + 0x83, 0x56, 0x75, 0x02, 0x95, 0x20, 0xbf, 0xb3, 0xbb, 0xde, 0xdc, 0x6e, 0xd2, 0xfc, 0x80, 0x1b, + 0xd2, 0xa5, 0xa1, 0xb9, 0x54, 0x31, 0xb5, 0x10, 0x26, 0x9e, 0x87, 0xd9, 0xb8, 0x09, 0xa4, 0x67, + 0xd1, 0xb2, 0x58, 0xa5, 0x63, 0x6d, 0x15, 0x15, 0x3a, 0x15, 0x1e, 0x6e, 0x0d, 0x72, 0x7c, 0xf5, + 0x76, 0xc4, 0xe1, 0x5c, 0x16, 0xa9, 0x23, 0xf8, 0x62, 0x24, 0x1d, 0x31, 0x4b, 0x7e, 0x39, 0x36, + 0xbc, 0x64, 0x62, 0xc3, 0x0b, 0xba, 0x0d, 0x65, 0x7f, 0x37, 0x18, 0xae, 0x38, 0x0b, 0x14, 0xf4, + 0x92, 0x5c, 0xe8, 0xb4, 0x2e, 0xe4, 0xf4, 0x5c, 0xd8, 0xe9, 0xe8, 0x2e, 0x64, 0xc9, 0x90, 0x58, + 0x9e, 0x5b, 0x2b, 0xb2, 0x8c, 0x51, 0x96, 0x67, 0xf7, 0x26, 0xad, 0xd5, 0x45, 0x23, 0x7e, 0x0b, + 0xa6, 0xd9, 0x1d, 0xe9, 0xb1, 0x63, 0x58, 0xea, 0x65, 0xae, 0xd5, 0xda, 0x16, 0xee, 0xa6, 0x9f, + 0xa8, 0x02, 0xa9, 0xcd, 0x75, 0xe1, 0x84, 0xd4, 0xe6, 0x3a, 0xfe, 0x42, 0x03, 0xa4, 0xf6, 0x1b, + 0xcb, 0xcf, 0x11, 0xe5, 0x12, 0x3e, 0x1d, 0xc0, 0xcf, 0x42, 0x86, 0x38, 0x8e, 0xed, 0x30, 0x8f, + 0x16, 0x74, 0x5e, 0xc0, 0x77, 0x84, 0x0d, 0x3a, 0x19, 0xda, 0xa7, 0xfe, 0x1e, 0xe4, 0xda, 0x34, + 0xdf, 0xd4, 0x2d, 0x98, 0x09, 0x49, 0x8d, 0x95, 0xb9, 0xee, 0xc1, 0x1c, 0x53, 0xb6, 0x45, 0x48, + 0x7f, 0xb5, 0x6b, 0x0e, 0x13, 0x51, 0xfb, 0x30, 0x1f, 0x15, 0xfc, 0x69, 0x7d, 0x84, 0xff, 0x56, + 0x20, 0xb6, 0xcc, 0x1e, 0x69, 0xd9, 0xdb, 0xc9, 0xb6, 0xd1, 0xc8, 0x7e, 0x4a, 0xce, 0x5d, 0x91, + 0xe2, 0xd9, 0x37, 0xfe, 0x3f, 0x0d, 0xae, 0x8c, 0x74, 0xff, 0x89, 0x67, 0x75, 0x01, 0xe0, 0x98, + 0x2e, 0x1f, 0xd2, 0xa1, 0x0d, 0x9c, 0x5d, 0x50, 0x6a, 0x7c, 0x3b, 0x69, 0x2c, 0x2b, 0x09, 0x3b, + 0x67, 0xc5, 0x9c, 0xb3, 0x3f, 0xfe, 0x8e, 0xbf, 0x01, 0x45, 0x56, 0xb1, 0xef, 0x19, 0xde, 0xc0, + 0x1d, 0x99, 0x8c, 0x7f, 0x16, 0x4b, 0x40, 0x76, 0x1a, 0x6b, 0x5c, 0xaf, 0x43, 0x96, 0x1d, 0xac, + 0xe5, 0xb1, 0x32, 0x72, 0x93, 0x51, 0xec, 0xd0, 0x85, 0x20, 0x3e, 0x81, 0xec, 0x13, 0xc6, 0x46, + 0x2a, 0x96, 0x4d, 0xca, 0xa9, 0xb0, 0x8c, 0x1e, 0xe7, 0x48, 0x0a, 0x3a, 0xfb, 0x66, 0xa7, 0x30, + 0x42, 0x9c, 0x03, 0x7d, 0x9b, 0x9f, 0xf6, 0x0a, 0xba, 0x5f, 0xa6, 0x2e, 0x6b, 0x77, 0x4d, 0x62, + 0x79, 0xac, 0x75, 0x92, 0xb5, 0x2a, 0x35, 0x78, 0x19, 0xaa, 0x1c, 0x69, 0xb5, 0xd3, 0x51, 0x4e, + 0x53, 0xbe, 0x3e, 0x2d, 0xac, 0x0f, 0xff, 0xbf, 0x06, 0xd3, 0x4a, 0x87, 0xb1, 0x1c, 0xf3, 0x0a, + 0x64, 0x39, 0xe7, 0x2a, 0x12, 0xf7, 0x6c, 0xb8, 0x17, 0x87, 0xd1, 0x85, 0x0c, 0x5a, 0x86, 0x1c, + 0xff, 0x92, 0x47, 0xda, 0x78, 0x71, 0x29, 0x84, 0xef, 0xc2, 0x8c, 0xa8, 0x22, 0x3d, 0x3b, 0x6e, + 0x6d, 0x33, 0x87, 0xe2, 0x7f, 0x82, 0xd9, 0xb0, 0xd8, 0x58, 0x43, 0x52, 0x8c, 0x4c, 0xbd, 0x88, + 0x91, 0xab, 0xd2, 0xc8, 0x83, 0x7e, 0x47, 0x39, 0x16, 0x44, 0x67, 0x5d, 0x9d, 0x91, 0x54, 0x64, + 0x46, 0xfc, 0x01, 0x48, 0x15, 0x3f, 0xeb, 0x00, 0x66, 0xe4, 0x72, 0xd8, 0x36, 0x5d, 0xff, 0xf4, + 0xf9, 0x19, 0x20, 0xb5, 0xf2, 0xe7, 0x36, 0x68, 0x9d, 0xc8, 0xa4, 0x26, 0x0d, 0x7a, 0x0f, 0x90, + 0x5a, 0x39, 0x56, 0x44, 0x6f, 0xc0, 0xf4, 0x13, 0x7b, 0x48, 0x43, 0x03, 0xad, 0x0d, 0xb6, 0x0c, + 0xbf, 0x8b, 0xfa, 0xd3, 0xe6, 0x97, 0x29, 0xb8, 0xda, 0x61, 0x2c, 0xf0, 0xdf, 0x6a, 0x50, 0x5a, + 0xed, 0x1a, 0x4e, 0x4f, 0x02, 0xbf, 0x0d, 0x59, 0x7e, 0xc3, 0x12, 0xa4, 0xc6, 0x4b, 0x61, 0x35, + 0xaa, 0x2c, 0x2f, 0xac, 0xf2, 0xfb, 0x98, 0xe8, 0x45, 0x0d, 0x17, 0xef, 0x1e, 0xeb, 0x91, 0x77, + 0x90, 0x75, 0xf4, 0x2a, 0x64, 0x0c, 0xda, 0x85, 0x85, 0xe0, 0x4a, 0xf4, 0x6e, 0xcb, 0xb4, 0xb1, + 0x73, 0x20, 0x97, 0xc2, 0x6f, 0x42, 0x51, 0x41, 0xa0, 0xb7, 0xf7, 0xc7, 0x4d, 0x71, 0x68, 0x5b, + 0x5d, 0x6b, 0x6d, 0x3e, 0xe5, 0x97, 0xfa, 0x0a, 0xc0, 0x7a, 0xd3, 0x2f, 0xa7, 0xf0, 0x87, 0xa2, + 0x97, 0x88, 0x77, 0xaa, 0x3d, 0x5a, 0x92, 0x3d, 0xa9, 0x17, 0xb2, 0xe7, 0x0c, 0xca, 0x62, 0xf8, + 0xe3, 0x86, 0x6f, 0xa6, 0x2f, 0x21, 0x7c, 0x2b, 0xc6, 0xeb, 0x42, 0x10, 0x4f, 0x41, 0x59, 0x04, + 0x74, 0xb1, 0xfe, 0x7e, 0x99, 0x82, 0x8a, 0xac, 0x19, 0x97, 0x7c, 0x95, 0xbc, 0x11, 0xcf, 0x00, + 0x3e, 0x6b, 0x34, 0x0f, 0xd9, 0xce, 0xd1, 0xbe, 0xf9, 0x99, 0x24, 0xca, 0x45, 0x89, 0xd6, 0x77, + 0x39, 0x0e, 0x7f, 0xad, 0x12, 0x25, 0x74, 0x9d, 0x3f, 0x64, 0x6d, 0x5a, 0x1d, 0x72, 0xc6, 0xce, + 0x94, 0x93, 0x7a, 0x50, 0xc1, 0x2e, 0xd4, 0xe2, 0x55, 0x8b, 0x1d, 0x24, 0x95, 0x57, 0x2e, 0x74, + 0x1f, 0xaa, 0xf4, 0x7b, 0xb5, 0xdf, 0xef, 0x9a, 0xa4, 0xc3, 0x15, 0xe4, 0x98, 0xcc, 0x48, 0x3d, + 0x45, 0x67, 0x47, 0x2f, 0xb7, 0x96, 0x67, 0x61, 0x4b, 0x94, 0xd0, 0x22, 0x14, 0xb9, 0x7d, 0x9b, + 0xd6, 0x81, 0x4b, 0xd8, 0x53, 0x4f, 0x5a, 0x57, 0xab, 0xe8, 0x3e, 0x5e, 0x1d, 0x78, 0x27, 0x4d, + 0xcb, 0x38, 0xea, 0xca, 0xb8, 0x48, 0x93, 0x39, 0xad, 0x5c, 0x37, 0x5d, 0xb5, 0xb6, 0x09, 0x33, + 0xb4, 0x96, 0x58, 0x9e, 0xd9, 0x56, 0x82, 0xa8, 0x4c, 0x95, 0x5a, 0x24, 0x55, 0x1a, 0xae, 0xfb, + 0xdc, 0x76, 0x3a, 0xc2, 0x81, 0x7e, 0x19, 0xaf, 0x73, 0xe5, 0x07, 0x6e, 0x28, 0x19, 0x7e, 0x5f, + 0x2d, 0x4b, 0x81, 0x96, 0xc7, 0xc4, 0xbb, 0x40, 0x0b, 0x7e, 0x00, 0x73, 0x52, 0x52, 0xd0, 0x9f, + 0x17, 0x08, 0xef, 0xc2, 0x0d, 0x29, 0xbc, 0x76, 0x42, 0xaf, 0x83, 0x7b, 0x02, 0xf0, 0x87, 0xda, + 0xf9, 0x08, 0x6a, 0xbe, 0x9d, 0xec, 0x48, 0x6e, 0x77, 0x55, 0x03, 0x06, 0xae, 0x58, 0x99, 0x05, + 0x9d, 0x7d, 0xd3, 0x3a, 0xc7, 0xee, 0xfa, 0x07, 0x0f, 0xfa, 0x8d, 0xd7, 0xe0, 0xaa, 0xd4, 0x21, + 0x0e, 0xcb, 0x61, 0x25, 0x23, 0x06, 0xc5, 0x29, 0x11, 0x0e, 0xa3, 0x5d, 0x2f, 0x76, 0xbb, 0x2a, + 0x19, 0x76, 0x2d, 0xd3, 0xa9, 0x29, 0x3a, 0xe7, 0xf8, 0x8a, 0xa0, 0x86, 0xa9, 0x79, 0x49, 0x54, + 0x53, 0x05, 0x6a, 0xb5, 0x98, 0x08, 0x5a, 0x3d, 0x32, 0x11, 0x23, 0xaa, 0x3f, 0x81, 0x05, 0xdf, + 0x08, 0xea, 0xb7, 0x3d, 0xe2, 0xf4, 0x4c, 0xd7, 0x55, 0x08, 0xb3, 0xb8, 0x81, 0xbf, 0x04, 0x93, + 0x7d, 0x22, 0x22, 0x57, 0x71, 0x05, 0x2d, 0xf3, 0x17, 0xee, 0x65, 0xa5, 0x33, 0x6b, 0xc7, 0x1d, + 0xb8, 0x29, 0xb5, 0x73, 0x8f, 0xc6, 0xaa, 0x8f, 0x1a, 0x25, 0x69, 0x84, 0x54, 0x02, 0x8d, 0x90, + 0x8e, 0x90, 0xb8, 0xef, 0x71, 0x47, 0xca, 0xbd, 0x35, 0x56, 0x46, 0xda, 0xe2, 0x3e, 0xf5, 0xb7, + 0xe4, 0x58, 0xca, 0x8e, 0x60, 0x36, 0xbc, 0x93, 0xc7, 0x0a, 0x96, 0xb3, 0x90, 0xf1, 0xec, 0x53, + 0x22, 0x43, 0x25, 0x2f, 0x48, 0x83, 0xfd, 0x6d, 0x3e, 0x96, 0xc1, 0x46, 0xa0, 0x8c, 0x2d, 0xc9, + 0x71, 0xed, 0xa5, 0xb3, 0x29, 0x8f, 0x78, 0xbc, 0x80, 0x77, 0x60, 0x3e, 0x1a, 0x26, 0xc6, 0x32, + 0xf9, 0x29, 0x5f, 0xc0, 0x71, 0x91, 0x64, 0x2c, 0xbd, 0xef, 0x07, 0xc1, 0x40, 0x09, 0x28, 0x63, + 0xa9, 0xd4, 0xa1, 0x1e, 0x17, 0x5f, 0x7e, 0x8c, 0xf5, 0xea, 0x87, 0x9b, 0xb1, 0x94, 0xb9, 0x81, + 0xb2, 0xf1, 0xa7, 0x3f, 0x88, 0x11, 0xe9, 0x0b, 0x63, 0x84, 0xd8, 0x24, 0x41, 0x14, 0xfb, 0x09, + 0x16, 0x9d, 0xc0, 0x08, 0x02, 0xe8, 0xb8, 0x18, 0x34, 0x87, 0xf8, 0x18, 0xac, 0x20, 0x17, 0xb6, + 0x1a, 0x76, 0xc7, 0x9a, 0x8c, 0x0f, 0x82, 0xd8, 0x39, 0x12, 0x99, 0xc7, 0x52, 0xfc, 0x21, 0x2c, + 0x26, 0x07, 0xe5, 0x71, 0x34, 0xdf, 0x6f, 0x40, 0xc1, 0x3f, 0xb6, 0x2a, 0xbf, 0x0e, 0x29, 0x42, + 0x6e, 0x67, 0x77, 0x7f, 0x6f, 0x75, 0xad, 0xc9, 0x7f, 0x1e, 0xb2, 0xb6, 0xab, 0xeb, 0x07, 0x7b, + 0xad, 0x6a, 0x6a, 0xe5, 0xbb, 0x34, 0xa4, 0xb6, 0x9e, 0xa2, 0x8f, 0x20, 0xc3, 0xdf, 0x4a, 0x2f, + 0x78, 0x20, 0xaf, 0x5f, 0xf4, 0x1c, 0x8c, 0xaf, 0x7c, 0xf1, 0x87, 0xef, 0xbe, 0x4e, 0x4d, 0xe3, + 0x52, 0x63, 0xf8, 0x46, 0xe3, 0x74, 0xd8, 0x60, 0xb9, 0xe1, 0xa1, 0x76, 0x1f, 0xbd, 0x0f, 0xe9, + 0xbd, 0x81, 0x87, 0x12, 0x1f, 0xce, 0xeb, 0xc9, 0x2f, 0xc4, 0x78, 0x8e, 0x29, 0x9d, 0xc2, 0x20, + 0x94, 0xf6, 0x07, 0x1e, 0x55, 0xf9, 0x29, 0x14, 0xd5, 0xf7, 0xdd, 0x4b, 0x5f, 0xd3, 0xeb, 0x97, + 0xbf, 0x1d, 0xe3, 0x1b, 0x0c, 0xea, 0x0a, 0x46, 0x02, 0x8a, 0xbf, 0x40, 0xab, 0xa3, 0x68, 0x9d, + 0x59, 0x28, 0xf1, 0xad, 0xbd, 0x9e, 0xfc, 0x9c, 0x3c, 0x32, 0x0a, 0xef, 0xcc, 0xa2, 0x2a, 0xff, + 0x41, 0xbc, 0x24, 0xb7, 0x3d, 0x74, 0x33, 0xe6, 0x25, 0x51, 0x7d, 0x33, 0xab, 0x2f, 0x26, 0x0b, + 0x08, 0x90, 0xeb, 0x0c, 0x64, 0x1e, 0x4f, 0x0b, 0x90, 0xb6, 0x2f, 0xf2, 0x50, 0xbb, 0xbf, 0xd2, + 0x86, 0x0c, 0xe3, 0xa3, 0xd1, 0xc7, 0xf2, 0xa3, 0x1e, 0x43, 0xcc, 0x27, 0x4c, 0x74, 0x88, 0xc9, + 0xc6, 0xb3, 0x0c, 0xa8, 0x82, 0x0b, 0x14, 0x88, 0xb1, 0xd1, 0x0f, 0xb5, 0xfb, 0x4b, 0xda, 0x6b, + 0xda, 0xca, 0x2f, 0x32, 0x90, 0x61, 0xe4, 0x13, 0x3a, 0x05, 0x08, 0xb8, 0xd9, 0xe8, 0xe8, 0x46, + 0xd8, 0xde, 0xe8, 0xe8, 0x46, 0x69, 0x5d, 0x5c, 0x67, 0xa0, 0xb3, 0x78, 0x8a, 0x82, 0x32, 0x4e, + 0xab, 0xc1, 0x68, 0x3a, 0xea, 0xc7, 0x7f, 0xd7, 0x04, 0xf7, 0xc6, 0xf7, 0x12, 0x8a, 0xd3, 0x16, + 0x22, 0x68, 0xa3, 0xcb, 0x21, 0x86, 0x9c, 0xc5, 0x6f, 0x31, 0xc0, 0x06, 0xae, 0x06, 0x80, 0x0e, + 0x93, 0x78, 0xa8, 0xdd, 0xff, 0xb8, 0x86, 0x67, 0x84, 0x97, 0x23, 0x2d, 0xe8, 0x73, 0xa8, 0x84, + 0x49, 0x57, 0x74, 0x3b, 0x06, 0x2b, 0xca, 0xdd, 0xd6, 0xef, 0x5c, 0x2c, 0x24, 0x6c, 0x5a, 0x60, + 0x36, 0x09, 0x70, 0x8e, 0x7c, 0x4a, 0x48, 0xdf, 0xa0, 0x42, 0x62, 0x0e, 0xd0, 0xff, 0x68, 0x30, + 0x15, 0x61, 0x51, 0x51, 0x9c, 0xf6, 0x11, 0x8e, 0xb6, 0x7e, 0xf7, 0x12, 0x29, 0x61, 0xc4, 0xdf, + 0x31, 0x23, 0xfe, 0x1a, 0xcf, 0x06, 0x46, 0x78, 0x66, 0x8f, 0x78, 0xb6, 0xb0, 0xe2, 0xe3, 0xeb, + 0xf8, 0x4a, 0xc8, 0x39, 0xa1, 0xd6, 0x60, 0xb2, 0x38, 0x13, 0x1a, 0x3b, 0x59, 0x21, 0x66, 0x35, + 0x76, 0xb2, 0xc2, 0x34, 0x6a, 0xdc, 0x64, 0x71, 0xde, 0x33, 0x6e, 0xb2, 0xfc, 0x96, 0x15, 0xf6, + 0x5b, 0x0e, 0xfe, 0x0b, 0x4e, 0x64, 0x43, 0xc1, 0x67, 0x21, 0xd1, 0x42, 0x1c, 0x23, 0x14, 0xdc, + 0x25, 0xea, 0x37, 0x13, 0xdb, 0x85, 0x41, 0xb7, 0x98, 0x41, 0xd7, 0xf0, 0x3c, 0x45, 0x16, 0x3f, + 0x12, 0x6d, 0x70, 0xda, 0xa1, 0x61, 0x74, 0x3a, 0xd4, 0x11, 0xff, 0x08, 0x25, 0x95, 0x26, 0x44, + 0xb7, 0x62, 0x59, 0x28, 0x95, 0x69, 0xac, 0xe3, 0x8b, 0x44, 0x04, 0xf2, 0x1d, 0x86, 0xbc, 0x80, + 0xaf, 0xc6, 0x20, 0x3b, 0x4c, 0x34, 0x04, 0xce, 0x29, 0xbe, 0x78, 0xf0, 0x10, 0x83, 0x18, 0x0f, + 0x1e, 0x66, 0x08, 0x2f, 0x04, 0x1f, 0x30, 0x51, 0x0a, 0xee, 0x02, 0x04, 0x64, 0x1e, 0x8a, 0xf5, + 0xa5, 0x72, 0x99, 0x8a, 0x06, 0x87, 0x51, 0x1e, 0x10, 0x63, 0x06, 0x2b, 0xd6, 0x5d, 0x04, 0xb6, + 0x6b, 0xba, 0x34, 0x48, 0xac, 0xfc, 0x47, 0x16, 0x8a, 0x4f, 0x0c, 0xd3, 0xf2, 0x88, 0x65, 0x58, + 0x6d, 0x82, 0x8e, 0x20, 0xc3, 0x12, 0x65, 0x34, 0x0e, 0xaa, 0xfc, 0x56, 0x34, 0x0e, 0x86, 0xc8, + 0x1f, 0xbc, 0xc8, 0x50, 0xeb, 0x78, 0x8e, 0xa2, 0xf6, 0x02, 0xd5, 0x0d, 0xc6, 0xd9, 0xd0, 0x81, + 0x3e, 0x83, 0xac, 0x78, 0x0e, 0x88, 0x28, 0x0a, 0x71, 0x39, 0xf5, 0xeb, 0xf1, 0x8d, 0x71, 0x4b, + 0x49, 0x85, 0x71, 0x99, 0x1c, 0xc5, 0x19, 0x02, 0x04, 0x64, 0x64, 0xd4, 0xa1, 0x23, 0xdc, 0x65, + 0x7d, 0x31, 0x59, 0x40, 0x60, 0xde, 0x65, 0x98, 0x37, 0x71, 0x3d, 0x8a, 0xd9, 0xf1, 0x65, 0x29, + 0xee, 0xdf, 0xc3, 0xe4, 0x86, 0xe1, 0x9e, 0xa0, 0x48, 0xea, 0x53, 0x7e, 0x58, 0x51, 0xaf, 0xc7, + 0x35, 0x09, 0x94, 0x9b, 0x0c, 0xe5, 0x2a, 0x8f, 0x24, 0x2a, 0xca, 0x89, 0xe1, 0xd2, 0x9c, 0x82, + 0x3a, 0x90, 0xe5, 0xbf, 0xb3, 0x88, 0xfa, 0x2f, 0xf4, 0x5b, 0x8d, 0xa8, 0xff, 0xc2, 0x3f, 0xcd, + 0xb8, 0x1c, 0xa5, 0x0f, 0x79, 0xf9, 0xc3, 0x06, 0x74, 0x23, 0x32, 0x15, 0xe1, 0x1f, 0x41, 0xd4, + 0x17, 0x92, 0x9a, 0x05, 0xd6, 0x6d, 0x86, 0x75, 0x03, 0xd7, 0x46, 0xe6, 0x4a, 0x48, 0x3e, 0xd4, + 0xee, 0xbf, 0xa6, 0xa1, 0xcf, 0x01, 0x02, 0xfe, 0x76, 0x64, 0x03, 0x44, 0xa9, 0xe0, 0x91, 0x0d, + 0x30, 0x42, 0xfd, 0xe2, 0x65, 0x86, 0xbb, 0x84, 0x6f, 0x47, 0x71, 0x3d, 0xc7, 0xb0, 0xdc, 0x67, + 0xc4, 0x79, 0x95, 0x73, 0x74, 0xee, 0x89, 0xd9, 0xa7, 0x9b, 0xe1, 0xd7, 0x53, 0x30, 0x49, 0x0f, + 0xa0, 0x34, 0x4f, 0x07, 0xf7, 0xf6, 0xa8, 0x25, 0x23, 0x6c, 0x59, 0xd4, 0x92, 0xd1, 0x2b, 0x7f, + 0x38, 0x4f, 0xb3, 0x9f, 0xde, 0x13, 0x26, 0x40, 0x1d, 0x6d, 0x43, 0x51, 0xb9, 0xd8, 0xa3, 0x18, + 0x65, 0x61, 0x1a, 0x2e, 0x1a, 0xf9, 0x63, 0x58, 0x01, 0x7c, 0x8d, 0xe1, 0xcd, 0xf1, 0xc8, 0xcf, + 0xf0, 0x3a, 0x5c, 0x82, 0x02, 0x3e, 0x87, 0x92, 0x7a, 0xf9, 0x47, 0x31, 0xfa, 0x22, 0x14, 0x5f, + 0x34, 0xca, 0xc5, 0x71, 0x07, 0xe1, 0x8d, 0xef, 0xff, 0xf7, 0x02, 0x29, 0x46, 0x81, 0xbb, 0x90, + 0x13, 0x6c, 0x40, 0xdc, 0x28, 0xc3, 0x7c, 0x60, 0xdc, 0x28, 0x23, 0x54, 0x42, 0xf8, 0x6c, 0xc7, + 0x10, 0xe9, 0x85, 0x47, 0x66, 0x12, 0x81, 0xf6, 0x98, 0x78, 0x49, 0x68, 0x01, 0xb9, 0x95, 0x84, + 0xa6, 0x5c, 0x36, 0x93, 0xd0, 0x8e, 0x89, 0x27, 0xb6, 0x8b, 0xbc, 0xc4, 0xa1, 0x04, 0x65, 0x6a, + 0xf4, 0xc6, 0x17, 0x89, 0xc4, 0x1d, 0xbd, 0x03, 0x40, 0x11, 0xba, 0xd1, 0x19, 0x40, 0xc0, 0x55, + 0x44, 0xcf, 0x53, 0xb1, 0x84, 0x67, 0xf4, 0x3c, 0x15, 0x4f, 0x77, 0x84, 0x43, 0x43, 0x80, 0xcb, + 0x4f, 0xfe, 0x14, 0xf9, 0x2b, 0x0d, 0xd0, 0x28, 0xad, 0x81, 0x1e, 0xc4, 0x6b, 0x8f, 0xa5, 0x51, + 0xeb, 0xaf, 0xbc, 0x98, 0x70, 0x5c, 0xb4, 0x0f, 0x4c, 0x6a, 0x33, 0xe9, 0xfe, 0x73, 0x6a, 0xd4, + 0xbf, 0x68, 0x50, 0x0e, 0x71, 0x22, 0xe8, 0xa5, 0x84, 0x39, 0x8d, 0xb0, 0xb0, 0xf5, 0x7b, 0x97, + 0xca, 0xc5, 0x1d, 0x34, 0x95, 0x15, 0x20, 0x4f, 0xdc, 0x5f, 0x6a, 0x50, 0x09, 0x73, 0x28, 0x28, + 0x41, 0xf7, 0x08, 0x8b, 0x5b, 0x5f, 0xba, 0x5c, 0xf0, 0xe2, 0xe9, 0x09, 0x0e, 0xdb, 0x5d, 0xc8, + 0x09, 0xd6, 0x25, 0x6e, 0xe1, 0x87, 0xf9, 0xdf, 0xb8, 0x85, 0x1f, 0xa1, 0x6c, 0x62, 0x16, 0xbe, + 0x63, 0x77, 0x89, 0xb2, 0xcd, 0x04, 0x2d, 0x93, 0x84, 0x76, 0xf1, 0x36, 0x8b, 0x70, 0x3a, 0x49, + 0x68, 0xc1, 0x36, 0x93, 0x7c, 0x0c, 0x4a, 0x50, 0x76, 0xc9, 0x36, 0x8b, 0xd2, 0x39, 0x31, 0xdb, + 0x8c, 0x01, 0x2a, 0xdb, 0x2c, 0x60, 0x4e, 0xe2, 0xb6, 0xd9, 0x08, 0x9d, 0x1d, 0xb7, 0xcd, 0x46, + 0xc9, 0x97, 0x98, 0x79, 0x64, 0xb8, 0xa1, 0x6d, 0x36, 0x13, 0x43, 0xb2, 0xa0, 0x57, 0x12, 0x9c, + 0x18, 0xcb, 0x92, 0xd7, 0x5f, 0x7d, 0x41, 0xe9, 0xc4, 0x35, 0xce, 0xdd, 0x2f, 0xd7, 0xf8, 0x7f, + 0x6a, 0x30, 0x1b, 0x47, 0xd0, 0xa0, 0x04, 0x9c, 0x04, 0x76, 0xbd, 0xbe, 0xfc, 0xa2, 0xe2, 0x17, + 0x7b, 0xcb, 0x5f, 0xf5, 0x8f, 0xaa, 0xbf, 0xf9, 0x76, 0x41, 0xfb, 0xdd, 0xb7, 0x0b, 0xda, 0x1f, + 0xbf, 0x5d, 0xd0, 0xfe, 0xeb, 0x4f, 0x0b, 0x13, 0x47, 0x59, 0xf6, 0x9f, 0xd6, 0xde, 0xf8, 0x4b, + 0x00, 0x00, 0x00, 0xff, 0xff, 0x40, 0x16, 0xfd, 0x84, 0x3b, 0x37, 0x00, 0x00, } diff --git a/etcdserver/etcdserverpb/rpc.proto b/etcdserver/etcdserverpb/rpc.proto index b4ed52056ab9..daa97752e5c8 100644 --- a/etcdserver/etcdserverpb/rpc.proto +++ b/etcdserver/etcdserverpb/rpc.proto @@ -367,6 +367,9 @@ message ResponseHeader { // member_id is the ID of the member which sent the response. uint64 member_id = 2; // revision is the key-value store revision when the request was applied. + // For watch progress responses, the header.revision indicates progress. All future events + // recieved in this stream are guarenteed to have a higher revision number than the + // header.revision number. int64 revision = 3; // raft_term is the raft term when the request was applied. uint64 raft_term = 4; @@ -655,6 +658,7 @@ message WatchRequest { oneof request_union { WatchCreateRequest create_request = 1; WatchCancelRequest cancel_request = 2; + WatchProgressRequest progress_request = 3; } } @@ -708,6 +712,11 @@ message WatchCancelRequest { int64 watch_id = 1; } +// Requests the a watch stream progress status be sent in the watch response stream as soon as +// possible. +message WatchProgressRequest { +} + message WatchResponse { ResponseHeader header = 1; // watch_id is the ID of the watcher that corresponds to the response.