mirror of
https://github.com/etcd-io/etcd.git
synced 2024-09-27 06:25:44 +00:00
Merge pull request #9291 from gyuho/fragment-watch
*: fragment watch response by server request limit
This commit is contained in:
commit
53373fecdd
@ -84,11 +84,12 @@ See [code changes](https://github.com/coreos/etcd/compare/v3.3.0...v3.4.0) and [
|
||||
- Previously, `Repair(dirpath string) bool`, now `Repair(lg *zap.Logger, dirpath string) bool`.
|
||||
- Previously, `Create(dirpath string, metadata []byte) (*WAL, error)`, now `Create(lg *zap.Logger, dirpath string, metadata []byte) (*WAL, error)`.
|
||||
- Remove [`pkg/cors` package](https://github.com/coreos/etcd/pull/9490).
|
||||
- Change [`--experimental-enable-v2v3`](TODO) flag to `--enable-v2v3`; v2 storage emulation is now stable.
|
||||
- Move internal package `"github.com/coreos/etcd/snap"` to [`"github.com/coreos/etcd/raftsnap"`](https://github.com/coreos/etcd/pull/9211).
|
||||
- Move internal package `"github.com/coreos/etcd/etcdserver/auth"` to [`"github.com/coreos/etcd/etcdserver/v2auth"`](https://github.com/coreos/etcd/pull/9275).
|
||||
- Move internal package `"github.com/coreos/etcd/error"` to [`"github.com/coreos/etcd/etcdserver/v2error"`](https://github.com/coreos/etcd/pull/9274).
|
||||
- Move internal package `"github.com/coreos/etcd/store"` to [`"github.com/coreos/etcd/etcdserver/v2store"`](https://github.com/coreos/etcd/pull/9274).
|
||||
- [`--experimental-enable-v2v3`](TODO) has been deprecated, `--enable-v2v3` flag is now stable.
|
||||
- Move internal package `"github.com/coreos/etcd/etcdserver/auth"` to `"github.com/coreos/etcd/etcdserver/api/v2auth"`.
|
||||
- Move internal package `"github.com/coreos/etcd/etcdserver/stats"` to `"github.com/coreos/etcd/etcdserver/api/v2stats"`.
|
||||
- Move internal package `"github.com/coreos/etcd/error"` to `"github.com/coreos/etcd/etcdserver/api/v2error"`.
|
||||
- Move internal package `"github.com/coreos/etcd/store"` to `"github.com/coreos/etcd/etcdserver/api/v2store"`.
|
||||
|
||||
### Dependency
|
||||
|
||||
@ -198,8 +199,14 @@ See [security doc](https://github.com/coreos/etcd/blob/master/Documentation/op-g
|
||||
### API
|
||||
|
||||
- Add [`snapshot`](https://github.com/coreos/etcd/pull/9118) package for snapshot restore/save operations (see [`godoc.org/github.com/etcd/snapshot`](https://godoc.org/github.com/coreos/etcd/snapshot) for more).
|
||||
- Add [`watch_id` field to `etcdserverpb.WatchCreateRequest`](https://github.com/coreos/etcd/pull/9065), allow user-provided watch ID to `mvcc`.
|
||||
- Add [`watch_id` field to `etcdserverpb.WatchCreateRequest`](https://github.com/coreos/etcd/pull/9065) to allow user-provided watch ID to `mvcc`.
|
||||
- Corresponding `watch_id` is returned via `etcdserverpb.WatchResponse`, if any.
|
||||
- Add [`fragment` field to `etcdserverpb.WatchCreateRequest`](https://github.com/coreos/etcd/pull/9291) to request etcd server to [split watch events](https://github.com/coreos/etcd/issues/9294) when the total size of events exceeds `--max-request-bytes` flag value plus gRPC-overhead 512 bytes.
|
||||
- The default server-side request bytes limit is `embed.DefaultMaxRequestBytes` which is 1.5 MiB plus gRPC-overhead 512 bytes.
|
||||
- If watch response events exceed this server-side request limit and watch request is created with `fragment` field `true`, the server will split watch events into a set of chunks, each of which is a subset of watch events below server-side request limit.
|
||||
- For example, watch response contains 10 events, where each event is 1 MiB. And server `--max-request-bytes` flag value is 1 MiB. Then, server will send 10 separate fragmented events to the client.
|
||||
- For example, watch response contains 5 events, where each event is 2 MiB. And server `--max-request-bytes` flag value is 1 MiB and `clientv3.Config.MaxCallRecvMsgSize` is 1 MiB. Then, server will try to send 5 separate fragmented events to the client, and the client will error with `"code = ResourceExhausted desc = grpc: received message larger than max (...)"`.
|
||||
- Client must implement fragmented watch event merge (which `clientv3` does in etcd v3.4).
|
||||
- Add [`raftAppliedIndex` field to `etcdserverpb.StatusResponse`](https://github.com/coreos/etcd/pull/9176) for current Raft applied index.
|
||||
- Add [`errors` field to `etcdserverpb.StatusResponse`](https://github.com/coreos/etcd/pull/9206) for server-side error.
|
||||
- e.g. `"etcdserver: no leader", "NOSPACE", "CORRUPT"`
|
||||
@ -223,6 +230,15 @@ See [security doc](https://github.com/coreos/etcd/blob/master/Documentation/op-g
|
||||
- Add [`CLUSTER_DEBUG` to enable test cluster logging](https://github.com/coreos/etcd/pull/9678).
|
||||
- Deprecated `capnslog` in integration tests.
|
||||
|
||||
### client v3
|
||||
|
||||
- Add [`WithFragment` `OpOption`](https://github.com/coreos/etcd/pull/9291) to support [watch events fragmentation](https://github.com/coreos/etcd/issues/9294) when the total size of events exceeds `--max-request-bytes` flag value plus gRPC-overhead 512 bytes.
|
||||
- Watch fragmentation is disabled by default.
|
||||
- The default server-side request bytes limit is `embed.DefaultMaxRequestBytes` which is 1.5 MiB plus gRPC-overhead 512 bytes.
|
||||
- If watch response events exceed this server-side request limit and watch request is created with `fragment` field `true`, the server will split watch events into a set of chunks, each of which is a subset of watch events below server-side request limit.
|
||||
- For example, watch response contains 10 events, where each event is 1 MiB. And server `--max-request-bytes` flag value is 1 MiB. Then, server will send 10 separate fragmented events to the client.
|
||||
- For example, watch response contains 5 events, where each event is 2 MiB. And server `--max-request-bytes` flag value is 1 MiB and `clientv3.Config.MaxCallRecvMsgSize` is 1 MiB. Then, server will try to send 5 separate fragmented events to the client, and the client will error with `"code = ResourceExhausted desc = grpc: received message larger than max (...)"`.
|
||||
|
||||
### etcdctl v3
|
||||
|
||||
- Add [`check datascale`](https://github.com/coreos/etcd/pull/9185) command.
|
||||
|
@ -836,6 +836,7 @@ From google paxosdb paper: Our implementation hinges around a powerful primitive
|
||||
| filters | filters filter the events at server side before it sends back to the watcher. | (slice of) FilterType |
|
||||
| prev_kv | If prev_kv is set, created watcher gets the previous KV before the event happens. If the previous KV is already compacted, nothing will be returned. | bool |
|
||||
| watch_id | If watch_id is provided and non-zero, it will be assigned to this watcher. Since creating a watcher in etcd is not a synchronous operation, this can be used ensure that ordering is correct when creating multiple watchers on the same stream. Creating a watcher with an ID already in use on the stream will cause an error to be returned. | int64 |
|
||||
| fragment | fragment enables splitting large revisions into multiple watch responses. | bool |
|
||||
|
||||
|
||||
|
||||
@ -859,6 +860,7 @@ From google paxosdb paper: Our implementation hinges around a powerful primitive
|
||||
| canceled | canceled is set to true if the response is for a cancel watch request. No further events will be sent to the canceled watcher. | bool |
|
||||
| compact_revision | compact_revision is set to the minimum index if a watcher tries to watch at a compacted index. This happens when creating a watcher at a compacted revision or the watcher cannot catch up with the progress of the key-value store. The client should treat the watcher as canceled and should not try to create any watcher with the same start_revision again. | int64 |
|
||||
| cancel_reason | cancel_reason indicates the reason for canceling the watcher. | string |
|
||||
| fragment | framgment is true if large watch response was split over multiple responses. | bool |
|
||||
| events | | (slice of) mvccpb.Event |
|
||||
|
||||
|
||||
|
@ -2359,6 +2359,11 @@
|
||||
"$ref": "#/definitions/WatchCreateRequestFilterType"
|
||||
}
|
||||
},
|
||||
"fragment": {
|
||||
"description": "fragment enables splitting large revisions into multiple watch responses.",
|
||||
"type": "boolean",
|
||||
"format": "boolean"
|
||||
},
|
||||
"key": {
|
||||
"description": "key is the key to register for watching.",
|
||||
"type": "string",
|
||||
@ -2430,6 +2435,11 @@
|
||||
"$ref": "#/definitions/mvccpbEvent"
|
||||
}
|
||||
},
|
||||
"fragment": {
|
||||
"description": "framgment is true if large watch response was split over multiple responses.",
|
||||
"type": "boolean",
|
||||
"format": "boolean"
|
||||
},
|
||||
"header": {
|
||||
"$ref": "#/definitions/etcdserverpbResponseHeader"
|
||||
},
|
||||
|
123
clientv3/integration/watch_fragment_test.go
Normal file
123
clientv3/integration/watch_fragment_test.go
Normal file
@ -0,0 +1,123 @@
|
||||
// Copyright 2018 The etcd Authors
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package integration
|
||||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"strings"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/coreos/etcd/clientv3"
|
||||
"github.com/coreos/etcd/integration"
|
||||
"github.com/coreos/etcd/pkg/testutil"
|
||||
)
|
||||
|
||||
// TestWatchFragmentDisable ensures that large watch
|
||||
// response exceeding server-side request limit can
|
||||
// arrive even without watch response fragmentation.
|
||||
func TestWatchFragmentDisable(t *testing.T) {
|
||||
testWatchFragment(t, false, false)
|
||||
}
|
||||
|
||||
// TestWatchFragmentDisableWithGRPCLimit verifies
|
||||
// large watch response exceeding server-side request
|
||||
// limit and client-side gRPC response receive limit
|
||||
// cannot arrive without watch events fragmentation,
|
||||
// because multiple events exceed client-side gRPC
|
||||
// response receive limit.
|
||||
func TestWatchFragmentDisableWithGRPCLimit(t *testing.T) {
|
||||
testWatchFragment(t, false, true)
|
||||
}
|
||||
|
||||
// TestWatchFragmentEnable ensures that large watch
|
||||
// response exceeding server-side request limit arrive
|
||||
// with watch response fragmentation.
|
||||
func TestWatchFragmentEnable(t *testing.T) {
|
||||
testWatchFragment(t, true, false)
|
||||
}
|
||||
|
||||
// TestWatchFragmentEnableWithGRPCLimit verifies
|
||||
// large watch response exceeding server-side request
|
||||
// limit and client-side gRPC response receive limit
|
||||
// can arrive only when watch events are fragmented.
|
||||
func TestWatchFragmentEnableWithGRPCLimit(t *testing.T) {
|
||||
testWatchFragment(t, true, true)
|
||||
}
|
||||
|
||||
// testWatchFragment triggers watch response that spans over multiple
|
||||
// revisions exceeding server request limits when combined.
|
||||
func testWatchFragment(t *testing.T, fragment, exceedRecvLimit bool) {
|
||||
cfg := &integration.ClusterConfig{
|
||||
Size: 1,
|
||||
MaxRequestBytes: 1.5 * 1024 * 1024,
|
||||
}
|
||||
if exceedRecvLimit {
|
||||
cfg.ClientMaxCallRecvMsgSize = 1.5 * 1024 * 1024
|
||||
}
|
||||
clus := integration.NewClusterV3(t, cfg)
|
||||
defer clus.Terminate(t)
|
||||
|
||||
cli := clus.Client(0)
|
||||
errc := make(chan error)
|
||||
for i := 0; i < 10; i++ {
|
||||
go func(i int) {
|
||||
_, err := cli.Put(context.TODO(),
|
||||
fmt.Sprint("foo", i),
|
||||
strings.Repeat("a", 1024*1024),
|
||||
)
|
||||
errc <- err
|
||||
}(i)
|
||||
}
|
||||
for i := 0; i < 10; i++ {
|
||||
if err := <-errc; err != nil {
|
||||
t.Fatalf("failed to put: %v", err)
|
||||
}
|
||||
}
|
||||
|
||||
opts := []clientv3.OpOption{clientv3.WithPrefix(), clientv3.WithRev(1)}
|
||||
if fragment {
|
||||
opts = append(opts, clientv3.WithFragment())
|
||||
}
|
||||
wch := cli.Watch(context.TODO(), "foo", opts...)
|
||||
|
||||
// expect 10 MiB watch response
|
||||
select {
|
||||
case ws := <-wch:
|
||||
// without fragment, should exceed gRPC client receive limit
|
||||
if !fragment && exceedRecvLimit {
|
||||
if len(ws.Events) != 0 {
|
||||
t.Fatalf("expected 0 events with watch fragmentation, got %d", len(ws.Events))
|
||||
}
|
||||
exp := "code = ResourceExhausted desc = grpc: received message larger than max ("
|
||||
if !strings.Contains(ws.Err().Error(), exp) {
|
||||
t.Fatalf("expected 'ResourceExhausted' error, got %v", ws.Err())
|
||||
}
|
||||
return
|
||||
}
|
||||
|
||||
// still expect merged watch events
|
||||
if len(ws.Events) != 10 {
|
||||
t.Fatalf("expected 10 events with watch fragmentation, got %d", len(ws.Events))
|
||||
}
|
||||
if ws.Err() != nil {
|
||||
t.Fatalf("unexpected error %v", ws.Err())
|
||||
}
|
||||
|
||||
case <-time.After(testutil.RequestTimeout):
|
||||
t.Fatalf("took too long to receive events")
|
||||
}
|
||||
}
|
@ -26,9 +26,7 @@ const (
|
||||
tTxn
|
||||
)
|
||||
|
||||
var (
|
||||
noPrefixEnd = []byte{0}
|
||||
)
|
||||
var noPrefixEnd = []byte{0}
|
||||
|
||||
// Op represents an Operation that kv can execute.
|
||||
type Op struct {
|
||||
@ -53,6 +51,12 @@ type Op struct {
|
||||
// for watch, put, delete
|
||||
prevKV bool
|
||||
|
||||
// for watch
|
||||
// fragmentation should be disabled by default
|
||||
// if true, split watch events when total exceeds
|
||||
// "--max-request-bytes" flag value + 512-byte
|
||||
fragment bool
|
||||
|
||||
// for put
|
||||
ignoreValue bool
|
||||
ignoreLease bool
|
||||
@ -77,8 +81,15 @@ type Op struct {
|
||||
|
||||
// accessors / mutators
|
||||
|
||||
func (op Op) IsTxn() bool { return op.t == tTxn }
|
||||
func (op Op) Txn() ([]Cmp, []Op, []Op) { return op.cmps, op.thenOps, op.elseOps }
|
||||
// IsTxn returns true if the "Op" type is transaction.
|
||||
func (op Op) IsTxn() bool {
|
||||
return op.t == tTxn
|
||||
}
|
||||
|
||||
// Txn returns the comparison(if) operations, "then" operations, and "else" operations.
|
||||
func (op Op) Txn() ([]Cmp, []Op, []Op) {
|
||||
return op.cmps, op.thenOps, op.elseOps
|
||||
}
|
||||
|
||||
// KeyBytes returns the byte slice holding the Op's key.
|
||||
func (op Op) KeyBytes() []byte { return op.key }
|
||||
@ -205,12 +216,14 @@ func (op Op) isWrite() bool {
|
||||
return op.t != tRange
|
||||
}
|
||||
|
||||
// OpGet returns "get" operation based on given key and operation options.
|
||||
func OpGet(key string, opts ...OpOption) Op {
|
||||
ret := Op{t: tRange, key: []byte(key)}
|
||||
ret.applyOpts(opts)
|
||||
return ret
|
||||
}
|
||||
|
||||
// OpDelete returns "delete" operation based on given key and operation options.
|
||||
func OpDelete(key string, opts ...OpOption) Op {
|
||||
ret := Op{t: tDeleteRange, key: []byte(key)}
|
||||
ret.applyOpts(opts)
|
||||
@ -239,6 +252,7 @@ func OpDelete(key string, opts ...OpOption) Op {
|
||||
return ret
|
||||
}
|
||||
|
||||
// OpPut returns "put" operation based on given key-value and operation options.
|
||||
func OpPut(key, val string, opts ...OpOption) Op {
|
||||
ret := Op{t: tPut, key: []byte(key), val: []byte(val)}
|
||||
ret.applyOpts(opts)
|
||||
@ -267,6 +281,7 @@ func OpPut(key, val string, opts ...OpOption) Op {
|
||||
return ret
|
||||
}
|
||||
|
||||
// OpTxn returns "txn" operation based on given transaction conditions.
|
||||
func OpTxn(cmps []Cmp, thenOps []Op, elseOps []Op) Op {
|
||||
return Op{t: tTxn, cmps: cmps, thenOps: thenOps, elseOps: elseOps}
|
||||
}
|
||||
@ -466,6 +481,17 @@ func WithPrevKV() OpOption {
|
||||
}
|
||||
}
|
||||
|
||||
// WithFragment to receive raw watch response with fragmentation.
|
||||
// Fragmentation is disabled by default. If fragmentation is enabled,
|
||||
// etcd watch server will split watch response before sending to clients
|
||||
// when the total size of watch events exceed server-side request limit.
|
||||
// The default server-side request limit is 1.5 MiB, which can be configured
|
||||
// as "--max-request-bytes" flag value + gRPC-overhead 512 bytes.
|
||||
// See "etcdserver/api/v3rpc/watch.go" for more details.
|
||||
func WithFragment() OpOption {
|
||||
return func(op *Op) { op.fragment = true }
|
||||
}
|
||||
|
||||
// WithIgnoreValue updates the key using its current value.
|
||||
// This option can not be combined with non-empty values.
|
||||
// Returns an error if the key does not exist.
|
||||
|
@ -174,10 +174,16 @@ type watchRequest struct {
|
||||
key string
|
||||
end string
|
||||
rev int64
|
||||
|
||||
// send created notification event if this field is true
|
||||
createdNotify bool
|
||||
// progressNotify is for progress updates
|
||||
progressNotify bool
|
||||
// fragmentation should be disabled by default
|
||||
// if true, split watch events when total exceeds
|
||||
// "--max-request-bytes" flag value + 512-byte
|
||||
fragment bool
|
||||
|
||||
// filters is the list of events to filter out
|
||||
filters []pb.WatchCreateRequest_FilterType
|
||||
// get the previous key-value pair before the event happens
|
||||
@ -272,6 +278,7 @@ func (w *watcher) Watch(ctx context.Context, key string, opts ...OpOption) Watch
|
||||
end: string(ow.end),
|
||||
rev: ow.rev,
|
||||
progressNotify: ow.progressNotify,
|
||||
fragment: ow.fragment,
|
||||
filters: filters,
|
||||
prevKV: ow.prevKV,
|
||||
retc: make(chan chan WatchResponse, 1),
|
||||
@ -451,6 +458,7 @@ func (w *watchGrpcStream) run() {
|
||||
|
||||
cancelSet := make(map[int64]struct{})
|
||||
|
||||
var cur *pb.WatchResponse
|
||||
for {
|
||||
select {
|
||||
// Watch() requested
|
||||
@ -475,8 +483,18 @@ func (w *watchGrpcStream) run() {
|
||||
// head of resume queue, can register a new watcher
|
||||
wc.Send(ws.initReq.toPB())
|
||||
}
|
||||
// New events from the watch client
|
||||
|
||||
// new events from the watch client
|
||||
case pbresp := <-w.respc:
|
||||
if cur == nil || pbresp.Created || pbresp.Canceled {
|
||||
cur = pbresp
|
||||
} else if cur != nil && cur.WatchId == pbresp.WatchId {
|
||||
// merge new events
|
||||
cur.Events = append(cur.Events, pbresp.Events...)
|
||||
// update "Fragment" field; last response with "Fragment" == false
|
||||
cur.Fragment = pbresp.Fragment
|
||||
}
|
||||
|
||||
switch {
|
||||
case pbresp.Created:
|
||||
// response to head of queue creation
|
||||
@ -485,9 +503,14 @@ func (w *watchGrpcStream) run() {
|
||||
w.dispatchEvent(pbresp)
|
||||
w.resuming[0] = nil
|
||||
}
|
||||
|
||||
if ws := w.nextResume(); ws != nil {
|
||||
wc.Send(ws.initReq.toPB())
|
||||
}
|
||||
|
||||
// reset for next iteration
|
||||
cur = nil
|
||||
|
||||
case pbresp.Canceled && pbresp.CompactRevision == 0:
|
||||
delete(cancelSet, pbresp.WatchId)
|
||||
if ws, ok := w.substreams[pbresp.WatchId]; ok {
|
||||
@ -495,15 +518,31 @@ func (w *watchGrpcStream) run() {
|
||||
close(ws.recvc)
|
||||
closing[ws] = struct{}{}
|
||||
}
|
||||
|
||||
// reset for next iteration
|
||||
cur = nil
|
||||
|
||||
case cur.Fragment:
|
||||
// watch response events are still fragmented
|
||||
// continue to fetch next fragmented event arrival
|
||||
continue
|
||||
|
||||
default:
|
||||
// dispatch to appropriate watch stream
|
||||
if ok := w.dispatchEvent(pbresp); ok {
|
||||
ok := w.dispatchEvent(cur)
|
||||
|
||||
// reset for next iteration
|
||||
cur = nil
|
||||
|
||||
if ok {
|
||||
break
|
||||
}
|
||||
|
||||
// watch response on unexpected watch id; cancel id
|
||||
if _, ok := cancelSet[pbresp.WatchId]; ok {
|
||||
break
|
||||
}
|
||||
|
||||
cancelSet[pbresp.WatchId] = struct{}{}
|
||||
cr := &pb.WatchRequest_CancelRequest{
|
||||
CancelRequest: &pb.WatchCancelRequest{
|
||||
@ -513,6 +552,7 @@ func (w *watchGrpcStream) run() {
|
||||
req := &pb.WatchRequest{RequestUnion: cr}
|
||||
wc.Send(req)
|
||||
}
|
||||
|
||||
// watch client failed on Recv; spawn another if possible
|
||||
case err := <-w.errc:
|
||||
if isHaltErr(w.ctx, err) || toErr(w.ctx, err) == v3rpc.ErrNoLeader {
|
||||
@ -526,13 +566,15 @@ func (w *watchGrpcStream) run() {
|
||||
wc.Send(ws.initReq.toPB())
|
||||
}
|
||||
cancelSet = make(map[int64]struct{})
|
||||
|
||||
case <-w.ctx.Done():
|
||||
return
|
||||
|
||||
case ws := <-w.closingc:
|
||||
w.closeSubstream(ws)
|
||||
delete(closing, ws)
|
||||
// no more watchers on this stream, shutdown
|
||||
if len(w.substreams)+len(w.resuming) == 0 {
|
||||
// no more watchers on this stream, shutdown
|
||||
return
|
||||
}
|
||||
}
|
||||
@ -820,6 +862,7 @@ func (wr *watchRequest) toPB() *pb.WatchRequest {
|
||||
ProgressNotify: wr.progressNotify,
|
||||
Filters: wr.filters,
|
||||
PrevKv: wr.prevKV,
|
||||
Fragment: wr.fragment,
|
||||
}
|
||||
cr := &pb.WatchRequest_CreateRequest{CreateRequest: req}
|
||||
return &pb.WatchRequest{RequestUnion: cr}
|
||||
|
@ -24,7 +24,7 @@ import (
|
||||
"strconv"
|
||||
"time"
|
||||
|
||||
"github.com/coreos/etcd/etcdserver/stats"
|
||||
stats "github.com/coreos/etcd/etcdserver/api/v2stats"
|
||||
"github.com/coreos/etcd/pkg/fileutil"
|
||||
"github.com/coreos/etcd/pkg/types"
|
||||
"github.com/coreos/etcd/raft"
|
||||
|
@ -27,10 +27,10 @@ import (
|
||||
"github.com/coreos/etcd/client"
|
||||
"github.com/coreos/etcd/etcdserver"
|
||||
"github.com/coreos/etcd/etcdserver/api"
|
||||
"github.com/coreos/etcd/etcdserver/api/v2error"
|
||||
"github.com/coreos/etcd/etcdserver/api/v2store"
|
||||
pb "github.com/coreos/etcd/etcdserver/etcdserverpb"
|
||||
"github.com/coreos/etcd/etcdserver/membership"
|
||||
"github.com/coreos/etcd/etcdserver/v2error"
|
||||
"github.com/coreos/etcd/etcdserver/v2store"
|
||||
"github.com/coreos/etcd/mvcc"
|
||||
"github.com/coreos/etcd/mvcc/backend"
|
||||
"github.com/coreos/etcd/mvcc/mvccpb"
|
||||
|
@ -23,8 +23,8 @@ import (
|
||||
|
||||
"github.com/coreos/etcd/etcdserver"
|
||||
"github.com/coreos/etcd/etcdserver/api"
|
||||
"github.com/coreos/etcd/etcdserver/api/v2error"
|
||||
"github.com/coreos/etcd/etcdserver/api/v2http/httptypes"
|
||||
"github.com/coreos/etcd/etcdserver/v2error"
|
||||
"github.com/coreos/etcd/pkg/logutil"
|
||||
"github.com/coreos/etcd/version"
|
||||
|
||||
|
@ -27,8 +27,8 @@ import (
|
||||
"time"
|
||||
|
||||
"github.com/coreos/etcd/etcdserver"
|
||||
"github.com/coreos/etcd/etcdserver/api/v2error"
|
||||
"github.com/coreos/etcd/etcdserver/etcdserverpb"
|
||||
"github.com/coreos/etcd/etcdserver/v2error"
|
||||
"github.com/coreos/etcd/pkg/types"
|
||||
|
||||
"github.com/coreos/pkg/capnslog"
|
@ -20,8 +20,8 @@ import (
|
||||
"path"
|
||||
|
||||
"github.com/coreos/etcd/etcdserver"
|
||||
"github.com/coreos/etcd/etcdserver/api/v2error"
|
||||
"github.com/coreos/etcd/etcdserver/etcdserverpb"
|
||||
"github.com/coreos/etcd/etcdserver/v2error"
|
||||
|
||||
"go.uber.org/zap"
|
||||
)
|
@ -21,9 +21,9 @@ import (
|
||||
"time"
|
||||
|
||||
"github.com/coreos/etcd/etcdserver"
|
||||
"github.com/coreos/etcd/etcdserver/api/v2error"
|
||||
"github.com/coreos/etcd/etcdserver/api/v2store"
|
||||
"github.com/coreos/etcd/etcdserver/etcdserverpb"
|
||||
"github.com/coreos/etcd/etcdserver/v2error"
|
||||
"github.com/coreos/etcd/etcdserver/v2store"
|
||||
|
||||
"go.uber.org/zap"
|
||||
)
|
@ -30,13 +30,13 @@ import (
|
||||
"github.com/coreos/etcd/etcdserver"
|
||||
"github.com/coreos/etcd/etcdserver/api"
|
||||
"github.com/coreos/etcd/etcdserver/api/etcdhttp"
|
||||
"github.com/coreos/etcd/etcdserver/api/v2auth"
|
||||
"github.com/coreos/etcd/etcdserver/api/v2error"
|
||||
"github.com/coreos/etcd/etcdserver/api/v2http/httptypes"
|
||||
stats "github.com/coreos/etcd/etcdserver/api/v2stats"
|
||||
"github.com/coreos/etcd/etcdserver/api/v2store"
|
||||
"github.com/coreos/etcd/etcdserver/etcdserverpb"
|
||||
"github.com/coreos/etcd/etcdserver/membership"
|
||||
"github.com/coreos/etcd/etcdserver/stats"
|
||||
"github.com/coreos/etcd/etcdserver/v2auth"
|
||||
"github.com/coreos/etcd/etcdserver/v2error"
|
||||
"github.com/coreos/etcd/etcdserver/v2store"
|
||||
"github.com/coreos/etcd/pkg/types"
|
||||
|
||||
"github.com/jonboulle/clockwork"
|
||||
|
@ -21,8 +21,8 @@ import (
|
||||
"strings"
|
||||
|
||||
"github.com/coreos/etcd/etcdserver/api"
|
||||
"github.com/coreos/etcd/etcdserver/api/v2auth"
|
||||
"github.com/coreos/etcd/etcdserver/api/v2http/httptypes"
|
||||
"github.com/coreos/etcd/etcdserver/v2auth"
|
||||
|
||||
"go.uber.org/zap"
|
||||
)
|
||||
|
@ -31,7 +31,7 @@ import (
|
||||
"testing"
|
||||
|
||||
"github.com/coreos/etcd/etcdserver/api"
|
||||
"github.com/coreos/etcd/etcdserver/v2auth"
|
||||
"github.com/coreos/etcd/etcdserver/api/v2auth"
|
||||
|
||||
"go.uber.org/zap"
|
||||
)
|
||||
|
@ -31,11 +31,11 @@ import (
|
||||
|
||||
"github.com/coreos/etcd/etcdserver"
|
||||
"github.com/coreos/etcd/etcdserver/api"
|
||||
"github.com/coreos/etcd/etcdserver/api/v2error"
|
||||
"github.com/coreos/etcd/etcdserver/api/v2http/httptypes"
|
||||
"github.com/coreos/etcd/etcdserver/api/v2store"
|
||||
"github.com/coreos/etcd/etcdserver/etcdserverpb"
|
||||
"github.com/coreos/etcd/etcdserver/membership"
|
||||
"github.com/coreos/etcd/etcdserver/v2error"
|
||||
"github.com/coreos/etcd/etcdserver/v2store"
|
||||
"github.com/coreos/etcd/pkg/testutil"
|
||||
"github.com/coreos/etcd/pkg/types"
|
||||
"github.com/coreos/etcd/raft/raftpb"
|
||||
|
@ -21,8 +21,8 @@ import (
|
||||
"time"
|
||||
|
||||
"github.com/coreos/etcd/etcdserver/api/etcdhttp"
|
||||
"github.com/coreos/etcd/etcdserver/api/v2auth"
|
||||
"github.com/coreos/etcd/etcdserver/api/v2http/httptypes"
|
||||
"github.com/coreos/etcd/etcdserver/v2auth"
|
||||
"github.com/coreos/etcd/pkg/logutil"
|
||||
|
||||
"github.com/coreos/pkg/capnslog"
|
||||
|
@ -23,9 +23,9 @@ import (
|
||||
"testing"
|
||||
|
||||
"github.com/coreos/etcd/etcdserver"
|
||||
"github.com/coreos/etcd/etcdserver/api/v2error"
|
||||
"github.com/coreos/etcd/etcdserver/etcdserverpb"
|
||||
"github.com/coreos/etcd/etcdserver/membership"
|
||||
"github.com/coreos/etcd/etcdserver/v2error"
|
||||
"github.com/coreos/etcd/pkg/types"
|
||||
"github.com/coreos/etcd/raft/raftpb"
|
||||
|
||||
|
@ -20,9 +20,9 @@ import (
|
||||
|
||||
"net/http"
|
||||
|
||||
"github.com/coreos/etcd/etcdserver/api/v2error"
|
||||
"github.com/coreos/etcd/etcdserver/api/v2http/httptypes"
|
||||
"github.com/coreos/etcd/etcdserver/etcdserverpb"
|
||||
"github.com/coreos/etcd/etcdserver/v2error"
|
||||
"github.com/prometheus/client_golang/prometheus"
|
||||
)
|
||||
|
||||
|
@ -12,7 +12,7 @@
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package stats
|
||||
package v2stats
|
||||
|
||||
import (
|
||||
"encoding/json"
|
@ -12,7 +12,7 @@
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package stats
|
||||
package v2stats
|
||||
|
||||
import (
|
||||
"sync"
|
@ -12,7 +12,7 @@
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package stats
|
||||
package v2stats
|
||||
|
||||
import (
|
||||
"encoding/json"
|
@ -12,14 +12,12 @@
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
// Package stats defines a standard interface for etcd cluster statistics.
|
||||
package stats
|
||||
// Package v2stats defines a standard interface for etcd cluster statistics.
|
||||
package v2stats
|
||||
|
||||
import "github.com/coreos/pkg/capnslog"
|
||||
|
||||
var (
|
||||
plog = capnslog.NewPackageLogger("github.com/coreos/etcd", "etcdserver/stats")
|
||||
)
|
||||
var plog = capnslog.NewPackageLogger("github.com/coreos/etcd", "etcdserver/stats")
|
||||
|
||||
type Stats interface {
|
||||
// SelfStats returns the struct representing statistics of this server
|
@ -20,7 +20,7 @@ import (
|
||||
"strings"
|
||||
"sync"
|
||||
|
||||
"github.com/coreos/etcd/etcdserver/v2error"
|
||||
"github.com/coreos/etcd/etcdserver/api/v2error"
|
||||
)
|
||||
|
||||
type EventHistory struct {
|
@ -17,7 +17,7 @@ package v2store
|
||||
import (
|
||||
"testing"
|
||||
|
||||
"github.com/coreos/etcd/etcdserver/v2error"
|
||||
"github.com/coreos/etcd/etcdserver/api/v2error"
|
||||
)
|
||||
|
||||
// TestEventQueue tests a queue with capacity = 100
|
@ -19,7 +19,7 @@ import (
|
||||
"sort"
|
||||
"time"
|
||||
|
||||
"github.com/coreos/etcd/etcdserver/v2error"
|
||||
"github.com/coreos/etcd/etcdserver/api/v2error"
|
||||
|
||||
"github.com/jonboulle/clockwork"
|
||||
)
|
@ -23,7 +23,7 @@ import (
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
"github.com/coreos/etcd/etcdserver/v2error"
|
||||
"github.com/coreos/etcd/etcdserver/api/v2error"
|
||||
"github.com/coreos/etcd/pkg/types"
|
||||
|
||||
"github.com/jonboulle/clockwork"
|
@ -18,8 +18,8 @@ import (
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/coreos/etcd/etcdserver/v2error"
|
||||
"github.com/coreos/etcd/etcdserver/v2store"
|
||||
"github.com/coreos/etcd/etcdserver/api/v2error"
|
||||
"github.com/coreos/etcd/etcdserver/api/v2store"
|
||||
"github.com/coreos/etcd/pkg/testutil"
|
||||
)
|
||||
|
@ -18,7 +18,7 @@ import (
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/coreos/etcd/etcdserver/v2error"
|
||||
"github.com/coreos/etcd/etcdserver/api/v2error"
|
||||
"github.com/coreos/etcd/pkg/testutil"
|
||||
|
||||
"github.com/jonboulle/clockwork"
|
@ -19,7 +19,7 @@ package v2store_test
|
||||
import (
|
||||
"testing"
|
||||
|
||||
"github.com/coreos/etcd/etcdserver/v2store"
|
||||
"github.com/coreos/etcd/etcdserver/api/v2store"
|
||||
"github.com/coreos/etcd/pkg/testutil"
|
||||
)
|
||||
|
@ -21,8 +21,8 @@ import (
|
||||
"testing"
|
||||
|
||||
"github.com/coreos/etcd/clientv3"
|
||||
"github.com/coreos/etcd/etcdserver/api/v2store"
|
||||
"github.com/coreos/etcd/etcdserver/api/v2v3"
|
||||
"github.com/coreos/etcd/etcdserver/v2store"
|
||||
"github.com/coreos/etcd/integration"
|
||||
|
||||
"google.golang.org/grpc/grpclog"
|
@ -21,7 +21,7 @@ import (
|
||||
"sync"
|
||||
"sync/atomic"
|
||||
|
||||
"github.com/coreos/etcd/etcdserver/v2error"
|
||||
"github.com/coreos/etcd/etcdserver/api/v2error"
|
||||
)
|
||||
|
||||
// A watcherHub contains all subscribed watchers
|
@ -24,8 +24,8 @@ import (
|
||||
|
||||
"github.com/coreos/etcd/clientv3"
|
||||
"github.com/coreos/etcd/clientv3/concurrency"
|
||||
"github.com/coreos/etcd/etcdserver/v2error"
|
||||
"github.com/coreos/etcd/etcdserver/v2store"
|
||||
"github.com/coreos/etcd/etcdserver/api/v2error"
|
||||
"github.com/coreos/etcd/etcdserver/api/v2store"
|
||||
"github.com/coreos/etcd/mvcc/mvccpb"
|
||||
)
|
||||
|
||||
|
@ -19,8 +19,8 @@ import (
|
||||
"strings"
|
||||
|
||||
"github.com/coreos/etcd/clientv3"
|
||||
"github.com/coreos/etcd/etcdserver/v2error"
|
||||
"github.com/coreos/etcd/etcdserver/v2store"
|
||||
"github.com/coreos/etcd/etcdserver/api/v2error"
|
||||
"github.com/coreos/etcd/etcdserver/api/v2store"
|
||||
)
|
||||
|
||||
func (s *v2v3Store) Watch(prefix string, recursive, stream bool, sinceIndex uint64) (v2store.Watcher, error) {
|
||||
|
@ -32,24 +32,31 @@ import (
|
||||
)
|
||||
|
||||
type watchServer struct {
|
||||
lg *zap.Logger
|
||||
|
||||
clusterID int64
|
||||
memberID int64
|
||||
|
||||
maxRequestBytes int
|
||||
|
||||
sg etcdserver.RaftStatusGetter
|
||||
watchable mvcc.WatchableKV
|
||||
|
||||
ag AuthGetter
|
||||
|
||||
lg *zap.Logger
|
||||
ag AuthGetter
|
||||
}
|
||||
|
||||
// NewWatchServer returns a new watch server.
|
||||
func NewWatchServer(s *etcdserver.EtcdServer) pb.WatchServer {
|
||||
return &watchServer{
|
||||
lg: s.Cfg.Logger,
|
||||
|
||||
clusterID: int64(s.Cluster().ID()),
|
||||
memberID: int64(s.ID()),
|
||||
|
||||
maxRequestBytes: int(s.Cfg.MaxRequestBytes + grpcOverheadBytes),
|
||||
|
||||
sg: s,
|
||||
watchable: s.Watchable(),
|
||||
ag: s,
|
||||
lg: s.Cfg.Logger,
|
||||
}
|
||||
}
|
||||
|
||||
@ -61,6 +68,7 @@ var (
|
||||
progressReportIntervalMu sync.RWMutex
|
||||
)
|
||||
|
||||
// GetProgressReportInterval returns the current progress report interval (for testing).
|
||||
func GetProgressReportInterval() time.Duration {
|
||||
progressReportIntervalMu.RLock()
|
||||
interval := progressReportInterval
|
||||
@ -74,74 +82,80 @@ func GetProgressReportInterval() time.Duration {
|
||||
return interval + jitter
|
||||
}
|
||||
|
||||
// SetProgressReportInterval updates the current progress report interval (for testing).
|
||||
func SetProgressReportInterval(newTimeout time.Duration) {
|
||||
progressReportIntervalMu.Lock()
|
||||
defer progressReportIntervalMu.Unlock()
|
||||
progressReportInterval = newTimeout
|
||||
progressReportIntervalMu.Unlock()
|
||||
}
|
||||
|
||||
const (
|
||||
// We send ctrl response inside the read loop. We do not want
|
||||
// send to block read, but we still want ctrl response we sent to
|
||||
// be serialized. Thus we use a buffered chan to solve the problem.
|
||||
// A small buffer should be OK for most cases, since we expect the
|
||||
// ctrl requests are infrequent.
|
||||
ctrlStreamBufLen = 16
|
||||
)
|
||||
// We send ctrl response inside the read loop. We do not want
|
||||
// send to block read, but we still want ctrl response we sent to
|
||||
// be serialized. Thus we use a buffered chan to solve the problem.
|
||||
// A small buffer should be OK for most cases, since we expect the
|
||||
// ctrl requests are infrequent.
|
||||
const ctrlStreamBufLen = 16
|
||||
|
||||
// serverWatchStream is an etcd server side stream. It receives requests
|
||||
// from client side gRPC stream. It receives watch events from mvcc.WatchStream,
|
||||
// and creates responses that forwarded to gRPC stream.
|
||||
// It also forwards control message like watch created and canceled.
|
||||
type serverWatchStream struct {
|
||||
lg *zap.Logger
|
||||
|
||||
clusterID int64
|
||||
memberID int64
|
||||
sg etcdserver.RaftStatusGetter
|
||||
|
||||
maxRequestBytes int
|
||||
|
||||
sg etcdserver.RaftStatusGetter
|
||||
watchable mvcc.WatchableKV
|
||||
ag AuthGetter
|
||||
|
||||
gRPCStream pb.Watch_WatchServer
|
||||
watchStream mvcc.WatchStream
|
||||
ctrlStream chan *pb.WatchResponse
|
||||
|
||||
// mu protects progress, prevKV
|
||||
mu sync.Mutex
|
||||
// progress tracks the watchID that stream might need to send
|
||||
// progress to.
|
||||
// mu protects progress, prevKV, fragment
|
||||
mu sync.RWMutex
|
||||
// tracks the watchID that stream might need to send progress to
|
||||
// TODO: combine progress and prevKV into a single struct?
|
||||
progress map[mvcc.WatchID]bool
|
||||
prevKV map[mvcc.WatchID]bool
|
||||
// record watch IDs that need return previous key-value pair
|
||||
prevKV map[mvcc.WatchID]bool
|
||||
// records fragmented watch IDs
|
||||
fragment map[mvcc.WatchID]bool
|
||||
|
||||
// closec indicates the stream is closed.
|
||||
closec chan struct{}
|
||||
|
||||
// wg waits for the send loop to complete
|
||||
wg sync.WaitGroup
|
||||
|
||||
ag AuthGetter
|
||||
|
||||
lg *zap.Logger
|
||||
}
|
||||
|
||||
func (ws *watchServer) Watch(stream pb.Watch_WatchServer) (err error) {
|
||||
sws := serverWatchStream{
|
||||
lg: ws.lg,
|
||||
|
||||
clusterID: ws.clusterID,
|
||||
memberID: ws.memberID,
|
||||
sg: ws.sg,
|
||||
|
||||
maxRequestBytes: ws.maxRequestBytes,
|
||||
|
||||
sg: ws.sg,
|
||||
watchable: ws.watchable,
|
||||
ag: ws.ag,
|
||||
|
||||
gRPCStream: stream,
|
||||
watchStream: ws.watchable.NewWatchStream(),
|
||||
// chan for sending control response like watcher created and canceled.
|
||||
ctrlStream: make(chan *pb.WatchResponse, ctrlStreamBufLen),
|
||||
progress: make(map[mvcc.WatchID]bool),
|
||||
prevKV: make(map[mvcc.WatchID]bool),
|
||||
closec: make(chan struct{}),
|
||||
|
||||
ag: ws.ag,
|
||||
progress: make(map[mvcc.WatchID]bool),
|
||||
prevKV: make(map[mvcc.WatchID]bool),
|
||||
fragment: make(map[mvcc.WatchID]bool),
|
||||
|
||||
lg: ws.lg,
|
||||
closec: make(chan struct{}),
|
||||
}
|
||||
|
||||
sws.wg.Add(1)
|
||||
@ -173,9 +187,11 @@ func (ws *watchServer) Watch(stream pb.Watch_WatchServer) (err error) {
|
||||
errc <- rerr
|
||||
}
|
||||
}()
|
||||
|
||||
select {
|
||||
case err = <-errc:
|
||||
close(sws.ctrlStream)
|
||||
|
||||
case <-stream.Context().Done():
|
||||
err = stream.Context().Err()
|
||||
// the only server-side cancellation is noleader for now.
|
||||
@ -183,6 +199,7 @@ func (ws *watchServer) Watch(stream pb.Watch_WatchServer) (err error) {
|
||||
err = rpctypes.ErrGRPCNoLeader
|
||||
}
|
||||
}
|
||||
|
||||
sws.close()
|
||||
return err
|
||||
}
|
||||
@ -196,7 +213,6 @@ func (sws *serverWatchStream) isWatchPermitted(wcr *pb.WatchCreateRequest) bool
|
||||
// if auth is enabled, IsRangePermitted() can cause an error
|
||||
authInfo = &auth.AuthInfo{}
|
||||
}
|
||||
|
||||
return sws.ag.AuthStore().IsRangePermitted(authInfo, wcr.Key, wcr.RangeEnd) == nil
|
||||
}
|
||||
|
||||
@ -263,6 +279,9 @@ func (sws *serverWatchStream) recvLoop() error {
|
||||
if creq.PrevKv {
|
||||
sws.prevKV[id] = true
|
||||
}
|
||||
if creq.Fragment {
|
||||
sws.fragment[id] = true
|
||||
}
|
||||
sws.mu.Unlock()
|
||||
}
|
||||
wr := &pb.WatchResponse{
|
||||
@ -279,6 +298,7 @@ func (sws *serverWatchStream) recvLoop() error {
|
||||
case <-sws.closec:
|
||||
return nil
|
||||
}
|
||||
|
||||
case *pb.WatchRequest_CancelRequest:
|
||||
if uv.CancelRequest != nil {
|
||||
id := uv.CancelRequest.WatchId
|
||||
@ -292,9 +312,11 @@ func (sws *serverWatchStream) recvLoop() error {
|
||||
sws.mu.Lock()
|
||||
delete(sws.progress, mvcc.WatchID(id))
|
||||
delete(sws.prevKV, mvcc.WatchID(id))
|
||||
delete(sws.fragment, mvcc.WatchID(id))
|
||||
sws.mu.Unlock()
|
||||
}
|
||||
}
|
||||
|
||||
default:
|
||||
// we probably should not shutdown the entire stream when
|
||||
// receive an valid command.
|
||||
@ -338,12 +360,11 @@ func (sws *serverWatchStream) sendLoop() {
|
||||
// or define protocol buffer with []mvccpb.Event.
|
||||
evs := wresp.Events
|
||||
events := make([]*mvccpb.Event, len(evs))
|
||||
sws.mu.Lock()
|
||||
sws.mu.RLock()
|
||||
needPrevKV := sws.prevKV[wresp.WatchID]
|
||||
sws.mu.Unlock()
|
||||
sws.mu.RUnlock()
|
||||
for i := range evs {
|
||||
events[i] = &evs[i]
|
||||
|
||||
if needPrevKV {
|
||||
opt := mvcc.RangeOptions{Rev: evs[i].Kv.ModRevision - 1}
|
||||
r, err := sws.watchable.Range(evs[i].Kv.Key, nil, opt)
|
||||
@ -362,7 +383,7 @@ func (sws *serverWatchStream) sendLoop() {
|
||||
Canceled: canceled,
|
||||
}
|
||||
|
||||
if _, hasId := ids[wresp.WatchID]; !hasId {
|
||||
if _, okID := ids[wresp.WatchID]; !okID {
|
||||
// buffer if id not yet announced
|
||||
wrs := append(pending[wresp.WatchID], wr)
|
||||
pending[wresp.WatchID] = wrs
|
||||
@ -370,18 +391,30 @@ func (sws *serverWatchStream) sendLoop() {
|
||||
}
|
||||
|
||||
mvcc.ReportEventReceived(len(evs))
|
||||
if err := sws.gRPCStream.Send(wr); err != nil {
|
||||
if isClientCtxErr(sws.gRPCStream.Context().Err(), err) {
|
||||
|
||||
sws.mu.RLock()
|
||||
fragmented, ok := sws.fragment[wresp.WatchID]
|
||||
sws.mu.RUnlock()
|
||||
|
||||
var serr error
|
||||
if !fragmented && !ok {
|
||||
serr = sws.gRPCStream.Send(wr)
|
||||
} else {
|
||||
serr = sendFragments(wr, sws.maxRequestBytes, sws.gRPCStream.Send)
|
||||
}
|
||||
|
||||
if serr != nil {
|
||||
if isClientCtxErr(sws.gRPCStream.Context().Err(), serr) {
|
||||
if sws.lg != nil {
|
||||
sws.lg.Debug("failed to send watch response to gRPC stream", zap.Error(err))
|
||||
sws.lg.Debug("failed to send watch response to gRPC stream", zap.Error(serr))
|
||||
} else {
|
||||
plog.Debugf("failed to send watch response to gRPC stream (%q)", err.Error())
|
||||
plog.Debugf("failed to send watch response to gRPC stream (%q)", serr.Error())
|
||||
}
|
||||
} else {
|
||||
if sws.lg != nil {
|
||||
sws.lg.Warn("failed to send watch response to gRPC stream", zap.Error(err))
|
||||
sws.lg.Warn("failed to send watch response to gRPC stream", zap.Error(serr))
|
||||
} else {
|
||||
plog.Warningf("failed to send watch response to gRPC stream (%q)", err.Error())
|
||||
plog.Warningf("failed to send watch response to gRPC stream (%q)", serr.Error())
|
||||
}
|
||||
}
|
||||
return
|
||||
@ -446,6 +479,7 @@ func (sws *serverWatchStream) sendLoop() {
|
||||
}
|
||||
delete(pending, wid)
|
||||
}
|
||||
|
||||
case <-progressTicker.C:
|
||||
sws.mu.Lock()
|
||||
for id, ok := range sws.progress {
|
||||
@ -455,12 +489,52 @@ func (sws *serverWatchStream) sendLoop() {
|
||||
sws.progress[id] = true
|
||||
}
|
||||
sws.mu.Unlock()
|
||||
|
||||
case <-sws.closec:
|
||||
return
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func sendFragments(
|
||||
wr *pb.WatchResponse,
|
||||
maxRequestBytes int,
|
||||
sendFunc func(*pb.WatchResponse) error) error {
|
||||
// no need to fragment if total request size is smaller
|
||||
// than max request limit or response contains only one event
|
||||
if wr.Size() < maxRequestBytes || len(wr.Events) < 2 {
|
||||
return sendFunc(wr)
|
||||
}
|
||||
|
||||
ow := *wr
|
||||
ow.Events = make([]*mvccpb.Event, 0)
|
||||
ow.Fragment = true
|
||||
|
||||
var idx int
|
||||
for {
|
||||
cur := ow
|
||||
for _, ev := range wr.Events[idx:] {
|
||||
cur.Events = append(cur.Events, ev)
|
||||
if len(cur.Events) > 1 && cur.Size() >= maxRequestBytes {
|
||||
cur.Events = cur.Events[:len(cur.Events)-1]
|
||||
break
|
||||
}
|
||||
idx++
|
||||
}
|
||||
if idx == len(wr.Events) {
|
||||
// last response has no more fragment
|
||||
cur.Fragment = false
|
||||
}
|
||||
if err := sendFunc(&cur); err != nil {
|
||||
return err
|
||||
}
|
||||
if !cur.Fragment {
|
||||
break
|
||||
}
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (sws *serverWatchStream) close() {
|
||||
sws.watchStream.Close()
|
||||
close(sws.closec)
|
||||
@ -484,6 +558,7 @@ func filterNoPut(e mvccpb.Event) bool {
|
||||
return e.Type == mvccpb.PUT
|
||||
}
|
||||
|
||||
// FiltersFromRequest returns "mvcc.FilterFunc" from a given watch create request.
|
||||
func FiltersFromRequest(creq *pb.WatchCreateRequest) []mvcc.FilterFunc {
|
||||
filters := make([]mvcc.FilterFunc, 0, len(creq.Filters))
|
||||
for _, ft := range creq.Filters {
|
||||
|
95
etcdserver/api/v3rpc/watch_test.go
Normal file
95
etcdserver/api/v3rpc/watch_test.go
Normal file
@ -0,0 +1,95 @@
|
||||
// Copyright 2018 The etcd Authors
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package v3rpc
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"math"
|
||||
"testing"
|
||||
|
||||
pb "github.com/coreos/etcd/etcdserver/etcdserverpb"
|
||||
"github.com/coreos/etcd/mvcc/mvccpb"
|
||||
)
|
||||
|
||||
func TestSendFragment(t *testing.T) {
|
||||
tt := []struct {
|
||||
wr *pb.WatchResponse
|
||||
maxRequestBytes int
|
||||
fragments int
|
||||
werr error
|
||||
}{
|
||||
{ // large limit should not fragment
|
||||
wr: createResponse(100, 1),
|
||||
maxRequestBytes: math.MaxInt32,
|
||||
fragments: 1,
|
||||
},
|
||||
{ // large limit for two messages, expect no fragment
|
||||
wr: createResponse(10, 2),
|
||||
maxRequestBytes: 50,
|
||||
fragments: 1,
|
||||
},
|
||||
{ // limit is small but only one message, expect no fragment
|
||||
wr: createResponse(1024, 1),
|
||||
maxRequestBytes: 1,
|
||||
fragments: 1,
|
||||
},
|
||||
{ // exceed limit only when combined, expect fragments
|
||||
wr: createResponse(11, 5),
|
||||
maxRequestBytes: 20,
|
||||
fragments: 5,
|
||||
},
|
||||
{ // 5 events with each event exceeding limits, expect fragments
|
||||
wr: createResponse(15, 5),
|
||||
maxRequestBytes: 10,
|
||||
fragments: 5,
|
||||
},
|
||||
{ // 4 events with some combined events exceeding limits
|
||||
wr: createResponse(10, 4),
|
||||
maxRequestBytes: 35,
|
||||
fragments: 2,
|
||||
},
|
||||
}
|
||||
|
||||
for i := range tt {
|
||||
fragmentedResp := make([]*pb.WatchResponse, 0)
|
||||
testSend := func(wr *pb.WatchResponse) error {
|
||||
fragmentedResp = append(fragmentedResp, wr)
|
||||
return nil
|
||||
}
|
||||
err := sendFragments(tt[i].wr, tt[i].maxRequestBytes, testSend)
|
||||
if err != tt[i].werr {
|
||||
t.Errorf("#%d: expected error %v, got %v", i, tt[i].werr, err)
|
||||
}
|
||||
got := len(fragmentedResp)
|
||||
if got != tt[i].fragments {
|
||||
t.Errorf("#%d: expected response number %d, got %d", i, tt[i].fragments, got)
|
||||
}
|
||||
if got > 0 && fragmentedResp[got-1].Fragment {
|
||||
t.Errorf("#%d: expected fragment=false in last response, got %+v", i, fragmentedResp[got-1])
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func createResponse(dataSize, events int) (resp *pb.WatchResponse) {
|
||||
resp = &pb.WatchResponse{Events: make([]*mvccpb.Event, events)}
|
||||
for i := range resp.Events {
|
||||
resp.Events[i] = &mvccpb.Event{
|
||||
Kv: &mvccpb.KeyValue{
|
||||
Key: bytes.Repeat([]byte("a"), dataSize),
|
||||
},
|
||||
}
|
||||
}
|
||||
return resp
|
||||
}
|
@ -20,8 +20,8 @@ import (
|
||||
"time"
|
||||
|
||||
"github.com/coreos/etcd/etcdserver/api"
|
||||
"github.com/coreos/etcd/etcdserver/api/v2store"
|
||||
"github.com/coreos/etcd/etcdserver/membership"
|
||||
"github.com/coreos/etcd/etcdserver/v2store"
|
||||
"github.com/coreos/etcd/pkg/pbutil"
|
||||
|
||||
"github.com/coreos/go-semver/semver"
|
||||
|
@ -1626,6 +1626,8 @@ type WatchCreateRequest struct {
|
||||
// watchers on the same stream. Creating a watcher with an ID already in
|
||||
// use on the stream will cause an error to be returned.
|
||||
WatchId int64 `protobuf:"varint,7,opt,name=watch_id,json=watchId,proto3" json:"watch_id,omitempty"`
|
||||
// fragment enables splitting large revisions into multiple watch responses.
|
||||
Fragment bool `protobuf:"varint,8,opt,name=fragment,proto3" json:"fragment,omitempty"`
|
||||
}
|
||||
|
||||
func (m *WatchCreateRequest) Reset() { *m = WatchCreateRequest{} }
|
||||
@ -1682,6 +1684,13 @@ func (m *WatchCreateRequest) GetWatchId() int64 {
|
||||
return 0
|
||||
}
|
||||
|
||||
func (m *WatchCreateRequest) GetFragment() bool {
|
||||
if m != nil {
|
||||
return m.Fragment
|
||||
}
|
||||
return false
|
||||
}
|
||||
|
||||
type WatchCancelRequest struct {
|
||||
// watch_id is the watcher id to cancel so that no more events are transmitted.
|
||||
WatchId int64 `protobuf:"varint,1,opt,name=watch_id,json=watchId,proto3" json:"watch_id,omitempty"`
|
||||
@ -1721,8 +1730,10 @@ type WatchResponse struct {
|
||||
// watcher with the same start_revision again.
|
||||
CompactRevision int64 `protobuf:"varint,5,opt,name=compact_revision,json=compactRevision,proto3" json:"compact_revision,omitempty"`
|
||||
// cancel_reason indicates the reason for canceling the watcher.
|
||||
CancelReason string `protobuf:"bytes,6,opt,name=cancel_reason,json=cancelReason,proto3" json:"cancel_reason,omitempty"`
|
||||
Events []*mvccpb.Event `protobuf:"bytes,11,rep,name=events" json:"events,omitempty"`
|
||||
CancelReason string `protobuf:"bytes,6,opt,name=cancel_reason,json=cancelReason,proto3" json:"cancel_reason,omitempty"`
|
||||
// framgment is true if large watch response was split over multiple responses.
|
||||
Fragment bool `protobuf:"varint,7,opt,name=fragment,proto3" json:"fragment,omitempty"`
|
||||
Events []*mvccpb.Event `protobuf:"bytes,11,rep,name=events" json:"events,omitempty"`
|
||||
}
|
||||
|
||||
func (m *WatchResponse) Reset() { *m = WatchResponse{} }
|
||||
@ -1772,6 +1783,13 @@ func (m *WatchResponse) GetCancelReason() string {
|
||||
return ""
|
||||
}
|
||||
|
||||
func (m *WatchResponse) GetFragment() bool {
|
||||
if m != nil {
|
||||
return m.Fragment
|
||||
}
|
||||
return false
|
||||
}
|
||||
|
||||
func (m *WatchResponse) GetEvents() []*mvccpb.Event {
|
||||
if m != nil {
|
||||
return m.Events
|
||||
@ -5972,6 +5990,16 @@ func (m *WatchCreateRequest) MarshalTo(dAtA []byte) (int, error) {
|
||||
i++
|
||||
i = encodeVarintRpc(dAtA, i, uint64(m.WatchId))
|
||||
}
|
||||
if m.Fragment {
|
||||
dAtA[i] = 0x40
|
||||
i++
|
||||
if m.Fragment {
|
||||
dAtA[i] = 1
|
||||
} else {
|
||||
dAtA[i] = 0
|
||||
}
|
||||
i++
|
||||
}
|
||||
return i, nil
|
||||
}
|
||||
|
||||
@ -6059,6 +6087,16 @@ func (m *WatchResponse) MarshalTo(dAtA []byte) (int, error) {
|
||||
i = encodeVarintRpc(dAtA, i, uint64(len(m.CancelReason)))
|
||||
i += copy(dAtA[i:], m.CancelReason)
|
||||
}
|
||||
if m.Fragment {
|
||||
dAtA[i] = 0x38
|
||||
i++
|
||||
if m.Fragment {
|
||||
dAtA[i] = 1
|
||||
} else {
|
||||
dAtA[i] = 0
|
||||
}
|
||||
i++
|
||||
}
|
||||
if len(m.Events) > 0 {
|
||||
for _, msg := range m.Events {
|
||||
dAtA[i] = 0x5a
|
||||
@ -8489,6 +8527,9 @@ func (m *WatchCreateRequest) Size() (n int) {
|
||||
if m.WatchId != 0 {
|
||||
n += 1 + sovRpc(uint64(m.WatchId))
|
||||
}
|
||||
if m.Fragment {
|
||||
n += 2
|
||||
}
|
||||
return n
|
||||
}
|
||||
|
||||
@ -8524,6 +8565,9 @@ func (m *WatchResponse) Size() (n int) {
|
||||
if l > 0 {
|
||||
n += 1 + l + sovRpc(uint64(l))
|
||||
}
|
||||
if m.Fragment {
|
||||
n += 2
|
||||
}
|
||||
if len(m.Events) > 0 {
|
||||
for _, e := range m.Events {
|
||||
l = e.Size()
|
||||
@ -12421,6 +12465,26 @@ func (m *WatchCreateRequest) Unmarshal(dAtA []byte) error {
|
||||
break
|
||||
}
|
||||
}
|
||||
case 8:
|
||||
if wireType != 0 {
|
||||
return fmt.Errorf("proto: wrong wireType = %d for field Fragment", wireType)
|
||||
}
|
||||
var v int
|
||||
for shift := uint(0); ; shift += 7 {
|
||||
if shift >= 64 {
|
||||
return ErrIntOverflowRpc
|
||||
}
|
||||
if iNdEx >= l {
|
||||
return io.ErrUnexpectedEOF
|
||||
}
|
||||
b := dAtA[iNdEx]
|
||||
iNdEx++
|
||||
v |= (int(b) & 0x7F) << shift
|
||||
if b < 0x80 {
|
||||
break
|
||||
}
|
||||
}
|
||||
m.Fragment = bool(v != 0)
|
||||
default:
|
||||
iNdEx = preIndex
|
||||
skippy, err := skipRpc(dAtA[iNdEx:])
|
||||
@ -12680,6 +12744,26 @@ func (m *WatchResponse) Unmarshal(dAtA []byte) error {
|
||||
}
|
||||
m.CancelReason = string(dAtA[iNdEx:postIndex])
|
||||
iNdEx = postIndex
|
||||
case 7:
|
||||
if wireType != 0 {
|
||||
return fmt.Errorf("proto: wrong wireType = %d for field Fragment", wireType)
|
||||
}
|
||||
var v int
|
||||
for shift := uint(0); ; shift += 7 {
|
||||
if shift >= 64 {
|
||||
return ErrIntOverflowRpc
|
||||
}
|
||||
if iNdEx >= l {
|
||||
return io.ErrUnexpectedEOF
|
||||
}
|
||||
b := dAtA[iNdEx]
|
||||
iNdEx++
|
||||
v |= (int(b) & 0x7F) << shift
|
||||
if b < 0x80 {
|
||||
break
|
||||
}
|
||||
}
|
||||
m.Fragment = bool(v != 0)
|
||||
case 11:
|
||||
if wireType != 2 {
|
||||
return fmt.Errorf("proto: wrong wireType = %d for field Events", wireType)
|
||||
@ -18614,239 +18698,240 @@ var (
|
||||
func init() { proto.RegisterFile("rpc.proto", fileDescriptorRpc) }
|
||||
|
||||
var fileDescriptorRpc = []byte{
|
||||
// 3738 bytes of a gzipped FileDescriptorProto
|
||||
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xbc, 0x5b, 0xdd, 0x6f, 0x1b, 0xc7,
|
||||
0x76, 0xd7, 0x92, 0xe2, 0xd7, 0xe1, 0x87, 0xa8, 0x91, 0x64, 0xd3, 0xb4, 0x2d, 0xcb, 0x63, 0x3b,
|
||||
0x56, 0xec, 0x44, 0x4c, 0x94, 0xa4, 0x05, 0xdc, 0x36, 0x88, 0x2c, 0x31, 0x96, 0x22, 0x59, 0x52,
|
||||
0x56, 0x94, 0xf3, 0x81, 0xa0, 0xc2, 0x8a, 0x1c, 0x49, 0x5b, 0x91, 0xbb, 0xcc, 0xee, 0x92, 0x96,
|
||||
0xd2, 0xa2, 0x29, 0x82, 0xf4, 0xa1, 0x05, 0xfa, 0x92, 0x00, 0x45, 0xfb, 0xd0, 0xa7, 0xa2, 0x28,
|
||||
0xf2, 0x50, 0xa0, 0x2f, 0xc1, 0x05, 0xee, 0x5f, 0x70, 0xdf, 0xee, 0x05, 0xee, 0x3f, 0x70, 0x91,
|
||||
0x9b, 0x97, 0xfb, 0x5f, 0x5c, 0xcc, 0xd7, 0xee, 0xec, 0x72, 0x57, 0x72, 0xc2, 0x24, 0x2f, 0xf2,
|
||||
0xce, 0xcc, 0x99, 0xf3, 0x3b, 0x73, 0x66, 0xe6, 0x9c, 0x99, 0xdf, 0xd0, 0x50, 0x70, 0xfa, 0xed,
|
||||
0xa5, 0xbe, 0x63, 0x7b, 0x36, 0x2a, 0x11, 0xaf, 0xdd, 0x71, 0x89, 0x33, 0x24, 0x4e, 0xff, 0xb0,
|
||||
0x3e, 0x7b, 0x6c, 0x1f, 0xdb, 0xac, 0xa1, 0x41, 0xbf, 0xb8, 0x4c, 0xfd, 0x1a, 0x95, 0x69, 0xf4,
|
||||
0x86, 0xed, 0x36, 0xfb, 0xd3, 0x3f, 0x6c, 0x9c, 0x0e, 0x45, 0xd3, 0x75, 0xd6, 0x64, 0x0c, 0xbc,
|
||||
0x13, 0xf6, 0xa7, 0x7f, 0xc8, 0xfe, 0x11, 0x8d, 0x37, 0x8e, 0x6d, 0xfb, 0xb8, 0x4b, 0x1a, 0x46,
|
||||
0xdf, 0x6c, 0x18, 0x96, 0x65, 0x7b, 0x86, 0x67, 0xda, 0x96, 0xcb, 0x5b, 0xf1, 0x3f, 0x6b, 0x50,
|
||||
0xd1, 0x89, 0xdb, 0xb7, 0x2d, 0x97, 0xac, 0x13, 0xa3, 0x43, 0x1c, 0x74, 0x13, 0xa0, 0xdd, 0x1d,
|
||||
0xb8, 0x1e, 0x71, 0x0e, 0xcc, 0x4e, 0x4d, 0x5b, 0xd0, 0x16, 0x27, 0xf5, 0x82, 0xa8, 0xd9, 0xe8,
|
||||
0xa0, 0xeb, 0x50, 0xe8, 0x91, 0xde, 0x21, 0x6f, 0x4d, 0xb1, 0xd6, 0x3c, 0xaf, 0xd8, 0xe8, 0xa0,
|
||||
0x3a, 0xe4, 0x1d, 0x32, 0x34, 0x5d, 0xd3, 0xb6, 0x6a, 0xe9, 0x05, 0x6d, 0x31, 0xad, 0xfb, 0x65,
|
||||
0xda, 0xd1, 0x31, 0x8e, 0xbc, 0x03, 0x8f, 0x38, 0xbd, 0xda, 0x24, 0xef, 0x48, 0x2b, 0x5a, 0xc4,
|
||||
0xe9, 0xe1, 0x2f, 0x33, 0x50, 0xd2, 0x0d, 0xeb, 0x98, 0xe8, 0xe4, 0xd3, 0x01, 0x71, 0x3d, 0x54,
|
||||
0x85, 0xf4, 0x29, 0x39, 0x67, 0xf0, 0x25, 0x9d, 0x7e, 0xf2, 0xfe, 0xd6, 0x31, 0x39, 0x20, 0x16,
|
||||
0x07, 0x2e, 0xd1, 0xfe, 0xd6, 0x31, 0x69, 0x5a, 0x1d, 0x34, 0x0b, 0x99, 0xae, 0xd9, 0x33, 0x3d,
|
||||
0x81, 0xca, 0x0b, 0x21, 0x73, 0x26, 0x23, 0xe6, 0xac, 0x02, 0xb8, 0xb6, 0xe3, 0x1d, 0xd8, 0x4e,
|
||||
0x87, 0x38, 0xb5, 0xcc, 0x82, 0xb6, 0x58, 0x59, 0xbe, 0xbb, 0xa4, 0x4e, 0xc4, 0x92, 0x6a, 0xd0,
|
||||
0xd2, 0x9e, 0xed, 0x78, 0x3b, 0x54, 0x56, 0x2f, 0xb8, 0xf2, 0x13, 0xbd, 0x0b, 0x45, 0xa6, 0xc4,
|
||||
0x33, 0x9c, 0x63, 0xe2, 0xd5, 0xb2, 0x4c, 0xcb, 0xbd, 0x4b, 0xb4, 0xb4, 0x98, 0xb0, 0xce, 0xe0,
|
||||
0xf9, 0x37, 0xc2, 0x50, 0x72, 0x89, 0x63, 0x1a, 0x5d, 0xf3, 0x33, 0xe3, 0xb0, 0x4b, 0x6a, 0xb9,
|
||||
0x05, 0x6d, 0x31, 0xaf, 0x87, 0xea, 0xe8, 0xf8, 0x4f, 0xc9, 0xb9, 0x7b, 0x60, 0x5b, 0xdd, 0xf3,
|
||||
0x5a, 0x9e, 0x09, 0xe4, 0x69, 0xc5, 0x8e, 0xd5, 0x3d, 0x67, 0x93, 0x66, 0x0f, 0x2c, 0x8f, 0xb7,
|
||||
0x16, 0x58, 0x6b, 0x81, 0xd5, 0xb0, 0xe6, 0x45, 0xa8, 0xf6, 0x4c, 0xeb, 0xa0, 0x67, 0x77, 0x0e,
|
||||
0x7c, 0x87, 0x00, 0x73, 0x48, 0xa5, 0x67, 0x5a, 0x4f, 0xed, 0x8e, 0x2e, 0xdd, 0x42, 0x25, 0x8d,
|
||||
0xb3, 0xb0, 0x64, 0x51, 0x48, 0x1a, 0x67, 0xaa, 0xe4, 0x12, 0xcc, 0x50, 0x9d, 0x6d, 0x87, 0x18,
|
||||
0x1e, 0x09, 0x84, 0x4b, 0x4c, 0x78, 0xba, 0x67, 0x5a, 0xab, 0xac, 0x25, 0x24, 0x6f, 0x9c, 0x8d,
|
||||
0xc8, 0x97, 0x85, 0xbc, 0x71, 0x16, 0x96, 0xc7, 0x4b, 0x50, 0xf0, 0x7d, 0x8e, 0xf2, 0x30, 0xb9,
|
||||
0xbd, 0xb3, 0xdd, 0xac, 0x4e, 0x20, 0x80, 0xec, 0xca, 0xde, 0x6a, 0x73, 0x7b, 0xad, 0xaa, 0xa1,
|
||||
0x22, 0xe4, 0xd6, 0x9a, 0xbc, 0x90, 0xc2, 0x8f, 0x01, 0x02, 0xef, 0xa2, 0x1c, 0xa4, 0x37, 0x9b,
|
||||
0x1f, 0x55, 0x27, 0xa8, 0xcc, 0xb3, 0xa6, 0xbe, 0xb7, 0xb1, 0xb3, 0x5d, 0xd5, 0x68, 0xe7, 0x55,
|
||||
0xbd, 0xb9, 0xd2, 0x6a, 0x56, 0x53, 0x54, 0xe2, 0xe9, 0xce, 0x5a, 0x35, 0x8d, 0x0a, 0x90, 0x79,
|
||||
0xb6, 0xb2, 0xb5, 0xdf, 0xac, 0x4e, 0xe2, 0xaf, 0x35, 0x28, 0x8b, 0xf9, 0xe2, 0x7b, 0x02, 0xbd,
|
||||
0x09, 0xd9, 0x13, 0xb6, 0x2f, 0xd8, 0x52, 0x2c, 0x2e, 0xdf, 0x88, 0x4c, 0x6e, 0x68, 0xef, 0xe8,
|
||||
0x42, 0x16, 0x61, 0x48, 0x9f, 0x0e, 0xdd, 0x5a, 0x6a, 0x21, 0xbd, 0x58, 0x5c, 0xae, 0x2e, 0xf1,
|
||||
0x0d, 0xbb, 0xb4, 0x49, 0xce, 0x9f, 0x19, 0xdd, 0x01, 0xd1, 0x69, 0x23, 0x42, 0x30, 0xd9, 0xb3,
|
||||
0x1d, 0xc2, 0x56, 0x6c, 0x5e, 0x67, 0xdf, 0x74, 0x19, 0xb3, 0x49, 0x13, 0xab, 0x95, 0x17, 0xf0,
|
||||
0x37, 0x1a, 0xc0, 0xee, 0xc0, 0x4b, 0xde, 0x1a, 0xb3, 0x90, 0x19, 0x52, 0xc5, 0x62, 0x5b, 0xf0,
|
||||
0x02, 0xdb, 0x13, 0xc4, 0x70, 0x89, 0xbf, 0x27, 0x68, 0x01, 0x5d, 0x85, 0x5c, 0xdf, 0x21, 0xc3,
|
||||
0x83, 0xd3, 0x21, 0x03, 0xc9, 0xeb, 0x59, 0x5a, 0xdc, 0x1c, 0xa2, 0xdb, 0x50, 0x32, 0x8f, 0x2d,
|
||||
0xdb, 0x21, 0x07, 0x5c, 0x57, 0x86, 0xb5, 0x16, 0x79, 0x1d, 0xb3, 0x5b, 0x11, 0xe1, 0x8a, 0xb3,
|
||||
0xaa, 0xc8, 0x16, 0xad, 0xc2, 0x16, 0x14, 0x99, 0xa9, 0x63, 0xb9, 0xef, 0xe5, 0xc0, 0xc6, 0x14,
|
||||
0xeb, 0x36, 0xea, 0x42, 0x61, 0x35, 0xfe, 0x04, 0xd0, 0x1a, 0xe9, 0x12, 0x8f, 0x8c, 0x13, 0x3d,
|
||||
0x14, 0x9f, 0xa4, 0x55, 0x9f, 0xe0, 0xaf, 0x34, 0x98, 0x09, 0xa9, 0x1f, 0x6b, 0x58, 0x35, 0xc8,
|
||||
0x75, 0x98, 0x32, 0x6e, 0x41, 0x5a, 0x97, 0x45, 0xf4, 0x10, 0xf2, 0xc2, 0x00, 0xb7, 0x96, 0x4e,
|
||||
0x58, 0x34, 0x39, 0x6e, 0x93, 0x8b, 0xbf, 0x49, 0x41, 0x41, 0x0c, 0x74, 0xa7, 0x8f, 0x56, 0xa0,
|
||||
0xec, 0xf0, 0xc2, 0x01, 0x1b, 0x8f, 0xb0, 0xa8, 0x9e, 0x1c, 0x84, 0xd6, 0x27, 0xf4, 0x92, 0xe8,
|
||||
0xc2, 0xaa, 0xd1, 0x5f, 0x41, 0x51, 0xaa, 0xe8, 0x0f, 0x3c, 0xe1, 0xf2, 0x5a, 0x58, 0x41, 0xb0,
|
||||
0xfe, 0xd6, 0x27, 0x74, 0x10, 0xe2, 0xbb, 0x03, 0x0f, 0xb5, 0x60, 0x56, 0x76, 0xe6, 0xa3, 0x11,
|
||||
0x66, 0xa4, 0x99, 0x96, 0x85, 0xb0, 0x96, 0xd1, 0xa9, 0x5a, 0x9f, 0xd0, 0x91, 0xe8, 0xaf, 0x34,
|
||||
0xaa, 0x26, 0x79, 0x67, 0x3c, 0x78, 0x8f, 0x98, 0xd4, 0x3a, 0xb3, 0x46, 0x4d, 0x6a, 0x9d, 0x59,
|
||||
0x8f, 0x0b, 0x90, 0x13, 0x25, 0xfc, 0xab, 0x14, 0x80, 0x9c, 0x8d, 0x9d, 0x3e, 0x5a, 0x83, 0x8a,
|
||||
0x23, 0x4a, 0x21, 0x6f, 0x5d, 0x8f, 0xf5, 0x96, 0x98, 0xc4, 0x09, 0xbd, 0x2c, 0x3b, 0x71, 0xe3,
|
||||
0xde, 0x86, 0x92, 0xaf, 0x25, 0x70, 0xd8, 0xb5, 0x18, 0x87, 0xf9, 0x1a, 0x8a, 0xb2, 0x03, 0x75,
|
||||
0xd9, 0x07, 0x30, 0xe7, 0xf7, 0x8f, 0xf1, 0xd9, 0xed, 0x0b, 0x7c, 0xe6, 0x2b, 0x9c, 0x91, 0x1a,
|
||||
0x54, 0xaf, 0xa9, 0x86, 0x05, 0x6e, 0xbb, 0x16, 0xe3, 0xb6, 0x51, 0xc3, 0xa8, 0xe3, 0x80, 0xe6,
|
||||
0x4b, 0x5e, 0xc4, 0x7f, 0x4a, 0x43, 0x6e, 0xd5, 0xee, 0xf5, 0x0d, 0x87, 0xce, 0x46, 0xd6, 0x21,
|
||||
0xee, 0xa0, 0xeb, 0x31, 0x77, 0x55, 0x96, 0xef, 0x84, 0x35, 0x0a, 0x31, 0xf9, 0xaf, 0xce, 0x44,
|
||||
0x75, 0xd1, 0x85, 0x76, 0x16, 0xe9, 0x31, 0xf5, 0x02, 0x9d, 0x45, 0x72, 0x14, 0x5d, 0xe4, 0x46,
|
||||
0x4e, 0x07, 0x1b, 0xb9, 0x0e, 0xb9, 0x21, 0x71, 0x82, 0x94, 0xbe, 0x3e, 0xa1, 0xcb, 0x0a, 0xf4,
|
||||
0x32, 0x4c, 0x45, 0xd3, 0x4b, 0x46, 0xc8, 0x54, 0xda, 0xe1, 0x6c, 0x74, 0x07, 0x4a, 0xa1, 0x1c,
|
||||
0x97, 0x15, 0x72, 0xc5, 0x9e, 0x92, 0xe2, 0xae, 0xc8, 0xb8, 0x4a, 0xf3, 0x71, 0x69, 0x7d, 0x42,
|
||||
0x46, 0xd6, 0x2b, 0x32, 0xb2, 0xe6, 0x45, 0x2f, 0x11, 0x5b, 0x43, 0x41, 0xe6, 0x9d, 0x70, 0x90,
|
||||
0xc1, 0xef, 0x40, 0x39, 0xe4, 0x20, 0x9a, 0x77, 0x9a, 0xef, 0xef, 0xaf, 0x6c, 0xf1, 0x24, 0xf5,
|
||||
0x84, 0xe5, 0x25, 0xbd, 0xaa, 0xd1, 0x5c, 0xb7, 0xd5, 0xdc, 0xdb, 0xab, 0xa6, 0x50, 0x19, 0x0a,
|
||||
0xdb, 0x3b, 0xad, 0x03, 0x2e, 0x95, 0xc6, 0x4f, 0x7c, 0x0d, 0x22, 0xc9, 0x29, 0xb9, 0x6d, 0x42,
|
||||
0xc9, 0x6d, 0x9a, 0xcc, 0x6d, 0xa9, 0x20, 0xb7, 0xb1, 0x34, 0xb7, 0xd5, 0x5c, 0xd9, 0x6b, 0x56,
|
||||
0x27, 0x1f, 0x57, 0xa0, 0xc4, 0xfd, 0x7b, 0x30, 0xb0, 0x68, 0xaa, 0xfd, 0x6f, 0x0d, 0x20, 0xd8,
|
||||
0x4d, 0xa8, 0x01, 0xb9, 0x36, 0xc7, 0xa9, 0x69, 0x2c, 0x18, 0xcd, 0xc5, 0x4e, 0x99, 0x2e, 0xa5,
|
||||
0xd0, 0xeb, 0x90, 0x73, 0x07, 0xed, 0x36, 0x71, 0x65, 0xca, 0xbb, 0x1a, 0x8d, 0x87, 0x22, 0x5a,
|
||||
0xe9, 0x52, 0x8e, 0x76, 0x39, 0x32, 0xcc, 0xee, 0x80, 0x25, 0xc0, 0x8b, 0xbb, 0x08, 0x39, 0xfc,
|
||||
0x9f, 0x1a, 0x14, 0x95, 0xc5, 0xfb, 0x23, 0x83, 0xf0, 0x0d, 0x28, 0x30, 0x1b, 0x48, 0x47, 0x84,
|
||||
0xe1, 0xbc, 0x1e, 0x54, 0xa0, 0xbf, 0x80, 0x82, 0xdc, 0x01, 0x32, 0x12, 0xd7, 0xe2, 0xd5, 0xee,
|
||||
0xf4, 0xf5, 0x40, 0x14, 0x6f, 0xc2, 0x34, 0xf3, 0x4a, 0x9b, 0x1e, 0xae, 0xa5, 0x1f, 0xd5, 0xe3,
|
||||
0xa7, 0x16, 0x39, 0x7e, 0xd6, 0x21, 0xdf, 0x3f, 0x39, 0x77, 0xcd, 0xb6, 0xd1, 0x15, 0x56, 0xf8,
|
||||
0x65, 0xfc, 0x1e, 0x20, 0x55, 0xd9, 0x38, 0xc3, 0xc5, 0x65, 0x28, 0xae, 0x1b, 0xee, 0x89, 0x30,
|
||||
0x09, 0x3f, 0x84, 0x32, 0x2d, 0x6e, 0x3e, 0x7b, 0x01, 0x1b, 0xd9, 0xe5, 0x40, 0x4a, 0x8f, 0xe5,
|
||||
0x73, 0x04, 0x93, 0x27, 0x86, 0x7b, 0xc2, 0x06, 0x5a, 0xd6, 0xd9, 0x37, 0x7a, 0x19, 0xaa, 0x6d,
|
||||
0x3e, 0xc8, 0x83, 0xc8, 0x95, 0x61, 0x4a, 0xd4, 0xfb, 0x27, 0xc1, 0x0f, 0xa1, 0xc4, 0xc7, 0xf0,
|
||||
0x53, 0x1b, 0x81, 0xa7, 0x61, 0x6a, 0xcf, 0x32, 0xfa, 0xee, 0x89, 0x2d, 0xb3, 0x1b, 0x1d, 0x74,
|
||||
0x35, 0xa8, 0x1b, 0x0b, 0xf1, 0x3e, 0x4c, 0x39, 0xa4, 0x67, 0x98, 0x96, 0x69, 0x1d, 0x1f, 0x1c,
|
||||
0x9e, 0x7b, 0xc4, 0x15, 0x17, 0xa6, 0x8a, 0x5f, 0xfd, 0x98, 0xd6, 0x52, 0xd3, 0x0e, 0xbb, 0xf6,
|
||||
0xa1, 0x08, 0x73, 0xec, 0x1b, 0x7f, 0xab, 0x41, 0xe9, 0x03, 0xc3, 0x6b, 0xcb, 0xa9, 0x43, 0x1b,
|
||||
0x50, 0xf1, 0x83, 0x1b, 0xab, 0x11, 0xb6, 0x44, 0x52, 0x2c, 0xeb, 0x23, 0x8f, 0xd2, 0x32, 0x3b,
|
||||
0x96, 0xdb, 0x6a, 0x05, 0x53, 0x65, 0x58, 0x6d, 0xd2, 0xf5, 0x55, 0xa5, 0x92, 0x55, 0x31, 0x41,
|
||||
0x55, 0x95, 0x5a, 0xf1, 0x78, 0x2a, 0x38, 0x7e, 0xf0, 0x58, 0xf2, 0x6d, 0x0a, 0xd0, 0xa8, 0x0d,
|
||||
0x3f, 0xf4, 0x44, 0x76, 0x0f, 0x2a, 0xae, 0x67, 0x38, 0x23, 0x6b, 0xa3, 0xcc, 0x6a, 0xfd, 0x00,
|
||||
0x7d, 0x1f, 0xa6, 0xfa, 0x8e, 0x7d, 0xec, 0x10, 0xd7, 0x3d, 0xb0, 0x6c, 0xcf, 0x3c, 0x3a, 0x17,
|
||||
0x87, 0xda, 0x8a, 0xac, 0xde, 0x66, 0xb5, 0xa8, 0x09, 0xb9, 0x23, 0xb3, 0xeb, 0x11, 0xc7, 0xad,
|
||||
0x65, 0x16, 0xd2, 0x8b, 0x95, 0xe5, 0x87, 0x97, 0x79, 0x6d, 0xe9, 0x5d, 0x26, 0xdf, 0x3a, 0xef,
|
||||
0x13, 0x5d, 0xf6, 0x55, 0x0f, 0x8a, 0xd9, 0xd0, 0xe1, 0xf9, 0x1a, 0xe4, 0x9f, 0x53, 0x15, 0xf4,
|
||||
0x52, 0x9c, 0xe3, 0x67, 0x3b, 0x56, 0xde, 0xe8, 0xe0, 0x7b, 0x00, 0x81, 0x2a, 0x1a, 0x85, 0xb7,
|
||||
0x77, 0x76, 0xf7, 0x5b, 0xd5, 0x09, 0x54, 0x82, 0xfc, 0xf6, 0xce, 0x5a, 0x73, 0xab, 0x49, 0x43,
|
||||
0x36, 0x6e, 0x48, 0xb7, 0xa9, 0xee, 0x0d, 0xe9, 0xd5, 0xc2, 0x7a, 0xff, 0x2d, 0x05, 0x65, 0xb1,
|
||||
0x40, 0xc6, 0x5a, 0xa5, 0x2a, 0x44, 0x2a, 0x04, 0x41, 0x0f, 0xac, 0x7c, 0xe1, 0x74, 0xc4, 0xb9,
|
||||
0x58, 0x16, 0x69, 0xd8, 0xe0, 0xeb, 0x80, 0x74, 0x84, 0xc7, 0xfd, 0x72, 0xec, 0xce, 0xce, 0xc4,
|
||||
0xee, 0x6c, 0x74, 0x07, 0xca, 0xfe, 0x42, 0x34, 0x5c, 0x91, 0x86, 0x0b, 0x7a, 0x49, 0xae, 0x31,
|
||||
0x5a, 0x87, 0xee, 0x41, 0x96, 0x0c, 0x89, 0xe5, 0xb9, 0xb5, 0x22, 0x0b, 0xc8, 0x65, 0x79, 0x34,
|
||||
0x6e, 0xd2, 0x5a, 0x5d, 0x34, 0xe2, 0xb7, 0x60, 0x9a, 0x5d, 0x41, 0x9e, 0x38, 0x86, 0xa5, 0xde,
|
||||
0x95, 0x5a, 0xad, 0x2d, 0xe1, 0x3a, 0xfa, 0x89, 0x2a, 0x90, 0xda, 0x58, 0x13, 0x03, 0x4d, 0x6d,
|
||||
0xac, 0xe1, 0x2f, 0x34, 0x40, 0x6a, 0xbf, 0xb1, 0x7c, 0x19, 0x51, 0x2e, 0xe1, 0xd3, 0x01, 0xfc,
|
||||
0x2c, 0x64, 0x88, 0xe3, 0xd8, 0x0e, 0xf3, 0x5a, 0x41, 0xe7, 0x05, 0x7c, 0x57, 0xd8, 0xa0, 0x93,
|
||||
0xa1, 0x7d, 0xea, 0xef, 0x19, 0xae, 0x4d, 0xf3, 0x4d, 0xdd, 0x84, 0x99, 0x90, 0xd4, 0x58, 0x89,
|
||||
0xe1, 0x3e, 0xcc, 0x31, 0x65, 0x9b, 0x84, 0xf4, 0x57, 0xba, 0xe6, 0x30, 0x11, 0xb5, 0x0f, 0x57,
|
||||
0xa2, 0x82, 0x3f, 0xaf, 0x8f, 0xf0, 0x5f, 0x0b, 0xc4, 0x96, 0xd9, 0x23, 0x2d, 0x7b, 0x2b, 0xd9,
|
||||
0x36, 0x1a, 0x38, 0x4f, 0xc9, 0xb9, 0x2b, 0x32, 0x28, 0xfb, 0xc6, 0xff, 0xa3, 0xc1, 0xd5, 0x91,
|
||||
0xee, 0x3f, 0xf3, 0xac, 0xce, 0x03, 0x1c, 0xd3, 0xe5, 0x43, 0x3a, 0xb4, 0x81, 0x5f, 0xde, 0x95,
|
||||
0x1a, 0xdf, 0x4e, 0x1a, 0x7b, 0x4a, 0xc2, 0xce, 0x59, 0x31, 0xe7, 0xec, 0x8f, 0x2b, 0xd3, 0xcf,
|
||||
0x4d, 0x28, 0xb2, 0x8a, 0x3d, 0xcf, 0xf0, 0x06, 0xee, 0xc8, 0x64, 0xfc, 0xa3, 0x58, 0x02, 0xb2,
|
||||
0xd3, 0x58, 0xe3, 0x7a, 0x1d, 0xb2, 0xec, 0xdc, 0x2a, 0x4f, 0x6d, 0x91, 0x8b, 0x82, 0x62, 0x87,
|
||||
0x2e, 0x04, 0xf1, 0x09, 0x64, 0x9f, 0x32, 0xb2, 0x4f, 0xb1, 0x6c, 0x52, 0x4e, 0x85, 0x65, 0xf4,
|
||||
0x38, 0x05, 0x51, 0xd0, 0xd9, 0x37, 0x3b, 0xe4, 0x10, 0xe2, 0xec, 0xeb, 0x5b, 0xfc, 0x30, 0x55,
|
||||
0xd0, 0xfd, 0x32, 0x75, 0x59, 0xbb, 0x6b, 0x12, 0xcb, 0x63, 0xad, 0x93, 0xac, 0x55, 0xa9, 0xc1,
|
||||
0x4b, 0x50, 0xe5, 0x48, 0x2b, 0x9d, 0x8e, 0x72, 0x58, 0xf1, 0xf5, 0x69, 0x61, 0x7d, 0xf8, 0x7f,
|
||||
0x35, 0x98, 0x56, 0x3a, 0x8c, 0xe5, 0x98, 0x57, 0x20, 0xcb, 0x29, 0x4d, 0x91, 0x17, 0x67, 0xc3,
|
||||
0xbd, 0x38, 0x8c, 0x2e, 0x64, 0xd0, 0x12, 0xe4, 0xf8, 0x97, 0x3c, 0x31, 0xc6, 0x8b, 0x4b, 0x21,
|
||||
0x7c, 0x0f, 0x66, 0x44, 0x15, 0xe9, 0xd9, 0x71, 0x6b, 0x9b, 0x39, 0x14, 0xff, 0x03, 0xcc, 0x86,
|
||||
0xc5, 0xc6, 0x1a, 0x92, 0x62, 0x64, 0xea, 0x45, 0x8c, 0x5c, 0x91, 0x46, 0xee, 0xf7, 0x3b, 0x4a,
|
||||
0x1a, 0x8f, 0xce, 0xba, 0x3a, 0x23, 0xa9, 0xc8, 0x8c, 0xf8, 0x03, 0x90, 0x2a, 0x7e, 0xd1, 0x01,
|
||||
0xcc, 0xc8, 0xe5, 0xb0, 0x65, 0xba, 0xfe, 0xe1, 0xee, 0x33, 0x40, 0x6a, 0xe5, 0x2f, 0x6d, 0xd0,
|
||||
0x1a, 0x39, 0x72, 0x8c, 0xe3, 0x1e, 0xf1, 0xf3, 0x13, 0x3d, 0xea, 0xab, 0x95, 0x63, 0x45, 0xf4,
|
||||
0x06, 0x4c, 0x3f, 0xb5, 0x87, 0x34, 0x34, 0xd0, 0xda, 0x60, 0xcb, 0xf0, 0xab, 0x9e, 0x3f, 0x6d,
|
||||
0x7e, 0x99, 0x82, 0xab, 0x1d, 0xc6, 0x02, 0xff, 0xad, 0x06, 0xa5, 0x95, 0xae, 0xe1, 0xf4, 0x24,
|
||||
0xf0, 0xdb, 0x90, 0xe5, 0x17, 0x18, 0xc1, 0x19, 0xbc, 0x14, 0x56, 0xa3, 0xca, 0xf2, 0xc2, 0x0a,
|
||||
0xbf, 0xee, 0x88, 0x5e, 0xd4, 0x70, 0xf1, 0xac, 0xb0, 0x16, 0x79, 0x66, 0x58, 0x43, 0xaf, 0x42,
|
||||
0xc6, 0xa0, 0x5d, 0x58, 0x08, 0xae, 0x44, 0xaf, 0x8e, 0x4c, 0x1b, 0x3b, 0xb7, 0x71, 0x29, 0xfc,
|
||||
0x26, 0x14, 0x15, 0x04, 0x7a, 0x39, 0x7e, 0xd2, 0x14, 0x07, 0xb0, 0x95, 0xd5, 0xd6, 0xc6, 0x33,
|
||||
0x7e, 0x67, 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, 0xff,
|
||||
0x53, 0x50, 0x91, 0x35, 0xe3, 0x72, 0x9b, 0x92, 0x96, 0xe1, 0x19, 0xc0, 0x27, 0x65, 0xae, 0x40,
|
||||
0xb6, 0x73, 0xb8, 0x67, 0x7e, 0x26, 0x79, 0x68, 0x51, 0xa2, 0xf5, 0x5d, 0x8e, 0xc3, 0x1f, 0x83,
|
||||
0x44, 0x89, 0x5e, 0xd0, 0x1d, 0xe3, 0xc8, 0xdb, 0xb0, 0x3a, 0xe4, 0x8c, 0x9d, 0x1b, 0x27, 0xf5,
|
||||
0xa0, 0x82, 0xdd, 0x57, 0xc5, 0xa3, 0x11, 0x3b, 0x2c, 0x2a, 0x8f, 0x48, 0xe8, 0x01, 0x54, 0xe9,
|
||||
0xf7, 0x4a, 0xbf, 0xdf, 0x35, 0x49, 0x87, 0x2b, 0xc8, 0x31, 0x99, 0x91, 0x7a, 0x8a, 0xce, 0x8e,
|
||||
0x5e, 0x6e, 0x2d, 0xcf, 0xc2, 0x96, 0x28, 0xa1, 0x05, 0x28, 0x72, 0xfb, 0x36, 0xac, 0x7d, 0x97,
|
||||
0xb0, 0x97, 0x94, 0xb4, 0xae, 0x56, 0xd1, 0x7d, 0xbc, 0x32, 0xf0, 0x4e, 0x9a, 0x96, 0x71, 0xd8,
|
||||
0x95, 0x71, 0x91, 0x26, 0x73, 0x5a, 0xb9, 0x66, 0xba, 0x6a, 0x6d, 0x13, 0x66, 0x68, 0x2d, 0xb1,
|
||||
0x3c, 0xb3, 0xad, 0x04, 0x51, 0x99, 0x2a, 0xb5, 0x48, 0xaa, 0x34, 0x5c, 0xf7, 0xb9, 0xed, 0x74,
|
||||
0x84, 0x03, 0xfd, 0x32, 0x5e, 0xe3, 0xca, 0xf7, 0xdd, 0x50, 0x32, 0xfc, 0xa1, 0x5a, 0x16, 0x03,
|
||||
0x2d, 0x4f, 0x88, 0x77, 0x81, 0x16, 0xfc, 0x10, 0xe6, 0xa4, 0xa4, 0x60, 0x17, 0x2f, 0x10, 0xde,
|
||||
0x81, 0x9b, 0x52, 0x78, 0xf5, 0x84, 0x5e, 0xdf, 0x76, 0x05, 0xe0, 0x8f, 0xb5, 0xf3, 0x31, 0xd4,
|
||||
0x7c, 0x3b, 0xd9, 0x91, 0xdc, 0xee, 0xaa, 0x06, 0x0c, 0x5c, 0xb1, 0x32, 0x0b, 0x3a, 0xfb, 0xa6,
|
||||
0x75, 0x8e, 0xdd, 0xf5, 0x0f, 0x1e, 0xf4, 0x1b, 0xaf, 0xc2, 0x35, 0xa9, 0x43, 0x1c, 0x96, 0xc3,
|
||||
0x4a, 0x46, 0x0c, 0x8a, 0x53, 0x22, 0x1c, 0x46, 0xbb, 0x5e, 0xec, 0x76, 0x55, 0x32, 0xec, 0x5a,
|
||||
0xa6, 0x53, 0x53, 0x74, 0xce, 0xf1, 0x15, 0x41, 0x0d, 0x53, 0xf3, 0x92, 0xa8, 0xa6, 0x0a, 0xd4,
|
||||
0x6a, 0x31, 0x11, 0xb4, 0x7a, 0x64, 0x22, 0x46, 0x54, 0x7f, 0x02, 0xf3, 0xbe, 0x11, 0xd4, 0x6f,
|
||||
0xbb, 0xc4, 0xe9, 0x99, 0xae, 0xab, 0xf0, 0x51, 0x71, 0x03, 0x7f, 0x09, 0x26, 0xfb, 0x44, 0x44,
|
||||
0xae, 0xe2, 0x32, 0x5a, 0xe2, 0x0f, 0xc8, 0x4b, 0x4a, 0x67, 0xd6, 0x8e, 0x3b, 0x70, 0x4b, 0x6a,
|
||||
0xe7, 0x1e, 0x8d, 0x55, 0x1f, 0x35, 0x4a, 0x5e, 0xfb, 0x53, 0x09, 0xd7, 0xfe, 0x74, 0x84, 0x23,
|
||||
0x7d, 0x8f, 0x3b, 0x52, 0xee, 0xad, 0xb1, 0x32, 0xd2, 0x26, 0xf7, 0xa9, 0xbf, 0x25, 0xc7, 0x52,
|
||||
0x76, 0x08, 0xb3, 0xe1, 0x9d, 0x3c, 0x56, 0xb0, 0x9c, 0x85, 0x8c, 0x67, 0x9f, 0x12, 0x19, 0x2a,
|
||||
0x79, 0x41, 0x1a, 0xec, 0x6f, 0xf3, 0xb1, 0x0c, 0x36, 0x02, 0x65, 0x6c, 0x49, 0x8e, 0x6b, 0x2f,
|
||||
0x9d, 0x4d, 0x79, 0xc4, 0xe3, 0x05, 0xbc, 0x0d, 0x57, 0xa2, 0x61, 0x62, 0x2c, 0x93, 0x9f, 0xf1,
|
||||
0x05, 0x1c, 0x17, 0x49, 0xc6, 0xd2, 0xfb, 0x7e, 0x10, 0x0c, 0x94, 0x80, 0x32, 0x96, 0x4a, 0x1d,
|
||||
0xea, 0x71, 0xf1, 0xe5, 0xa7, 0x58, 0xaf, 0x7e, 0xb8, 0x19, 0x4b, 0x99, 0x1b, 0x28, 0x1b, 0x7f,
|
||||
0xfa, 0x83, 0x18, 0x91, 0xbe, 0x30, 0x46, 0x88, 0x4d, 0x12, 0x44, 0xb1, 0x9f, 0x61, 0xd1, 0x09,
|
||||
0x8c, 0x20, 0x80, 0x8e, 0x8b, 0x41, 0x73, 0x88, 0x8f, 0xc1, 0x0a, 0x72, 0x61, 0xab, 0x61, 0x77,
|
||||
0xac, 0xc9, 0xf8, 0x20, 0x88, 0x9d, 0x23, 0x91, 0x79, 0x2c, 0xc5, 0x1f, 0xc2, 0x42, 0x72, 0x50,
|
||||
0x1e, 0x47, 0xf3, 0x83, 0x06, 0x14, 0xfc, 0x63, 0xab, 0xf2, 0xe3, 0x8b, 0x22, 0xe4, 0xb6, 0x77,
|
||||
0xf6, 0x76, 0x57, 0x56, 0x9b, 0xfc, 0xd7, 0x17, 0xab, 0x3b, 0xba, 0xbe, 0xbf, 0xdb, 0xaa, 0xa6,
|
||||
0x96, 0xbf, 0x4f, 0x43, 0x6a, 0xf3, 0x19, 0xfa, 0x08, 0x32, 0xfc, 0x29, 0xf2, 0x82, 0xf7, 0xe7,
|
||||
0xfa, 0x45, 0xaf, 0xad, 0xf8, 0xea, 0x17, 0xbf, 0xff, 0xfe, 0xeb, 0xd4, 0x34, 0x2e, 0x35, 0x86,
|
||||
0x6f, 0x34, 0x4e, 0x87, 0x0d, 0x96, 0x1b, 0x1e, 0x69, 0x0f, 0xd0, 0xfb, 0x90, 0xde, 0x1d, 0x78,
|
||||
0x28, 0xf1, 0x5d, 0xba, 0x9e, 0xfc, 0x00, 0x8b, 0xe7, 0x98, 0xd2, 0x29, 0x0c, 0x42, 0x69, 0x7f,
|
||||
0xe0, 0x51, 0x95, 0x9f, 0x42, 0x51, 0x7d, 0x3e, 0xbd, 0xf4, 0xb1, 0xba, 0x7e, 0xf9, 0xd3, 0x2c,
|
||||
0xbe, 0xc9, 0xa0, 0xae, 0x62, 0x24, 0xa0, 0xf8, 0x03, 0xaf, 0x3a, 0x8a, 0xd6, 0x99, 0x85, 0x12,
|
||||
0x9f, 0xb2, 0xeb, 0xc9, 0xaf, 0xb5, 0x23, 0xa3, 0xf0, 0xce, 0x2c, 0xaa, 0xf2, 0xef, 0xc4, 0x43,
|
||||
0x6d, 0xdb, 0x43, 0xb7, 0x62, 0x1e, 0xea, 0xd4, 0x27, 0xa9, 0xfa, 0x42, 0xb2, 0x80, 0x00, 0xb9,
|
||||
0xc1, 0x40, 0xae, 0xe0, 0x69, 0x01, 0xd2, 0xf6, 0x45, 0x1e, 0x69, 0x0f, 0x96, 0xdb, 0x90, 0x61,
|
||||
0x9c, 0x33, 0xfa, 0x58, 0x7e, 0xd4, 0x63, 0x88, 0xf4, 0x84, 0x89, 0x0e, 0xb1, 0xd5, 0x78, 0x96,
|
||||
0x01, 0x55, 0x70, 0x81, 0x02, 0x31, 0xc6, 0xf9, 0x91, 0xf6, 0x60, 0x51, 0x7b, 0x4d, 0x5b, 0xfe,
|
||||
0xbf, 0x0c, 0x64, 0x18, 0xf9, 0x84, 0x4e, 0x01, 0x02, 0x6e, 0x36, 0x3a, 0xba, 0x11, 0xb6, 0x37,
|
||||
0x3a, 0xba, 0x51, 0x5a, 0x17, 0xd7, 0x19, 0xe8, 0x2c, 0x9e, 0xa2, 0xa0, 0x8c, 0xd3, 0x6a, 0x30,
|
||||
0x9a, 0x8e, 0xfa, 0xf1, 0x5f, 0x34, 0xc1, 0xbd, 0xf1, 0xbd, 0x84, 0xe2, 0xb4, 0x85, 0x08, 0xda,
|
||||
0xe8, 0x72, 0x88, 0x21, 0x67, 0xf1, 0x5b, 0x0c, 0xb0, 0x81, 0xab, 0x01, 0xa0, 0xc3, 0x24, 0x1e,
|
||||
0x69, 0x0f, 0x3e, 0xae, 0xe1, 0x19, 0xe1, 0xe5, 0x48, 0x0b, 0xfa, 0x1c, 0x2a, 0x61, 0xd2, 0x15,
|
||||
0xdd, 0x89, 0xc1, 0x8a, 0x72, 0xb7, 0xf5, 0xbb, 0x17, 0x0b, 0x09, 0x9b, 0xe6, 0x99, 0x4d, 0x02,
|
||||
0x9c, 0x23, 0x9f, 0x12, 0xd2, 0x37, 0xa8, 0x90, 0x98, 0x03, 0xf4, 0x5f, 0x1a, 0x4c, 0x45, 0x58,
|
||||
0x54, 0x14, 0xa7, 0x7d, 0x84, 0xa3, 0xad, 0xdf, 0xbb, 0x44, 0x4a, 0x18, 0xf1, 0x37, 0xcc, 0x88,
|
||||
0xbf, 0xc4, 0xb3, 0x81, 0x11, 0x9e, 0xd9, 0x23, 0x9e, 0x2d, 0xac, 0xf8, 0xf8, 0x06, 0xbe, 0x1a,
|
||||
0x72, 0x4e, 0xa8, 0x35, 0x98, 0x2c, 0xce, 0x84, 0xc6, 0x4e, 0x56, 0x88, 0x59, 0x8d, 0x9d, 0xac,
|
||||
0x30, 0x8d, 0x1a, 0x37, 0x59, 0x9c, 0xf7, 0x8c, 0x9b, 0x2c, 0xbf, 0x65, 0x99, 0xfd, 0x54, 0x82,
|
||||
0xff, 0x40, 0x12, 0xd9, 0x50, 0xf0, 0x59, 0x48, 0x34, 0x1f, 0xc7, 0x08, 0x05, 0x77, 0x89, 0xfa,
|
||||
0xad, 0xc4, 0x76, 0x61, 0xd0, 0x6d, 0x66, 0xd0, 0x75, 0x7c, 0x85, 0x22, 0x8b, 0xdf, 0x60, 0x36,
|
||||
0x38, 0xed, 0xd0, 0x30, 0x3a, 0x1d, 0xea, 0x88, 0xbf, 0x87, 0x92, 0x4a, 0x13, 0xa2, 0xdb, 0xb1,
|
||||
0x2c, 0x94, 0xca, 0x34, 0xd6, 0xf1, 0x45, 0x22, 0x02, 0xf9, 0x2e, 0x43, 0x9e, 0xc7, 0xd7, 0x62,
|
||||
0x90, 0x1d, 0x26, 0x1a, 0x02, 0xe7, 0x14, 0x5f, 0x3c, 0x78, 0x88, 0x41, 0x8c, 0x07, 0x0f, 0x33,
|
||||
0x84, 0x17, 0x82, 0x0f, 0x98, 0x28, 0x05, 0x77, 0x01, 0x02, 0x32, 0x0f, 0xc5, 0xfa, 0x52, 0xb9,
|
||||
0x4c, 0x45, 0x83, 0xc3, 0x28, 0x0f, 0x88, 0x31, 0x83, 0x15, 0xeb, 0x2e, 0x02, 0xdb, 0x35, 0x5d,
|
||||
0x1a, 0x24, 0x96, 0xff, 0x35, 0x0b, 0xc5, 0xa7, 0x86, 0x69, 0x79, 0xc4, 0x32, 0xac, 0x36, 0x41,
|
||||
0x87, 0x90, 0x61, 0x89, 0x32, 0x1a, 0x07, 0x55, 0x7e, 0x2b, 0x1a, 0x07, 0x43, 0xe4, 0x0f, 0x5e,
|
||||
0x60, 0xa8, 0x75, 0x3c, 0x47, 0x51, 0x7b, 0x81, 0xea, 0x06, 0xe3, 0x6c, 0xe8, 0x40, 0x8f, 0x20,
|
||||
0x2b, 0x9e, 0x03, 0x22, 0x8a, 0x42, 0x5c, 0x4e, 0xfd, 0x46, 0x7c, 0x63, 0xdc, 0x52, 0x52, 0x61,
|
||||
0x5c, 0x26, 0x47, 0x71, 0x86, 0x00, 0x01, 0x19, 0x19, 0x75, 0xe8, 0x08, 0x77, 0x59, 0x5f, 0x48,
|
||||
0x16, 0x10, 0x98, 0xf7, 0x18, 0xe6, 0x2d, 0x5c, 0x8f, 0x62, 0x76, 0x7c, 0x59, 0x8a, 0xfb, 0xb7,
|
||||
0x30, 0xb9, 0x6e, 0xb8, 0x27, 0x28, 0x92, 0xfa, 0x94, 0xdf, 0x2d, 0xd4, 0xeb, 0x71, 0x4d, 0x02,
|
||||
0xe5, 0x16, 0x43, 0xb9, 0xc6, 0x23, 0x89, 0x8a, 0x72, 0x62, 0xb8, 0x34, 0xa7, 0xa0, 0x0e, 0x64,
|
||||
0xf9, 0xcf, 0x18, 0xa2, 0xfe, 0x0b, 0xfd, 0x14, 0x22, 0xea, 0xbf, 0xf0, 0x2f, 0x1f, 0x2e, 0x47,
|
||||
0xe9, 0x43, 0x5e, 0xfe, 0x6e, 0x00, 0xdd, 0x8c, 0x4c, 0x45, 0xf8, 0x37, 0x06, 0xf5, 0xf9, 0xa4,
|
||||
0x66, 0x81, 0x75, 0x87, 0x61, 0xdd, 0xc4, 0xb5, 0x91, 0xb9, 0x12, 0x92, 0x8f, 0xb4, 0x07, 0xaf,
|
||||
0x69, 0xe8, 0x73, 0x80, 0x80, 0xbf, 0x1d, 0xd9, 0x00, 0x51, 0x2a, 0x78, 0x64, 0x03, 0x8c, 0x50,
|
||||
0xbf, 0x78, 0x89, 0xe1, 0x2e, 0xe2, 0x3b, 0x51, 0x5c, 0xcf, 0x31, 0x2c, 0xf7, 0x88, 0x38, 0xaf,
|
||||
0x72, 0x8e, 0xce, 0x3d, 0x31, 0xfb, 0x74, 0x33, 0xfc, 0x7a, 0x0a, 0x26, 0xe9, 0x01, 0x94, 0xe6,
|
||||
0xe9, 0xe0, 0xde, 0x1e, 0xb5, 0x64, 0x84, 0x2d, 0x8b, 0x5a, 0x32, 0x7a, 0xe5, 0x0f, 0xe7, 0x69,
|
||||
0xf6, 0xcb, 0x76, 0xc2, 0x04, 0xa8, 0xa3, 0x6d, 0x28, 0x2a, 0x17, 0x7b, 0x14, 0xa3, 0x2c, 0x4c,
|
||||
0xc3, 0x45, 0x23, 0x7f, 0x0c, 0x2b, 0x80, 0xaf, 0x33, 0xbc, 0x39, 0x1e, 0xf9, 0x19, 0x5e, 0x87,
|
||||
0x4b, 0x50, 0xc0, 0xe7, 0x50, 0x52, 0x2f, 0xff, 0x28, 0x46, 0x5f, 0x84, 0xe2, 0x8b, 0x46, 0xb9,
|
||||
0x38, 0xee, 0x20, 0xbc, 0xf1, 0xfd, 0x5f, 0xef, 0x4b, 0x31, 0x0a, 0xdc, 0x85, 0x9c, 0x60, 0x03,
|
||||
0xe2, 0x46, 0x19, 0xe6, 0x03, 0xe3, 0x46, 0x19, 0xa1, 0x12, 0xc2, 0x67, 0x3b, 0x86, 0x48, 0x2f,
|
||||
0x3c, 0x32, 0x93, 0x08, 0xb4, 0x27, 0xc4, 0x4b, 0x42, 0x0b, 0xc8, 0xad, 0x24, 0x34, 0xe5, 0xb2,
|
||||
0x99, 0x84, 0x76, 0x4c, 0x3c, 0xb1, 0x5d, 0xe4, 0x25, 0x0e, 0x25, 0x28, 0x53, 0xa3, 0x37, 0xbe,
|
||||
0x48, 0x24, 0xee, 0xe8, 0x1d, 0x00, 0x8a, 0xd0, 0x8d, 0xce, 0x00, 0x02, 0xae, 0x22, 0x7a, 0x9e,
|
||||
0x8a, 0x25, 0x3c, 0xa3, 0xe7, 0xa9, 0x78, 0xba, 0x23, 0x1c, 0x1a, 0x02, 0x5c, 0x7e, 0xf2, 0xa7,
|
||||
0xc8, 0x5f, 0x69, 0x80, 0x46, 0x69, 0x0d, 0xf4, 0x30, 0x5e, 0x7b, 0x2c, 0x8d, 0x5a, 0x7f, 0xe5,
|
||||
0xc5, 0x84, 0xe3, 0xa2, 0x7d, 0x60, 0x52, 0x9b, 0x49, 0xf7, 0x9f, 0x53, 0xa3, 0xfe, 0x49, 0x83,
|
||||
0x72, 0x88, 0x13, 0x41, 0x2f, 0x25, 0xcc, 0x69, 0x84, 0x85, 0xad, 0xdf, 0xbf, 0x54, 0x2e, 0xee,
|
||||
0xa0, 0xa9, 0xac, 0x00, 0x79, 0xe2, 0xfe, 0x52, 0x83, 0x4a, 0x98, 0x43, 0x41, 0x09, 0xba, 0x47,
|
||||
0x58, 0xdc, 0xfa, 0xe2, 0xe5, 0x82, 0x17, 0x4f, 0x4f, 0x70, 0xd8, 0xee, 0x42, 0x4e, 0xb0, 0x2e,
|
||||
0x71, 0x0b, 0x3f, 0xcc, 0xff, 0xc6, 0x2d, 0xfc, 0x08, 0x65, 0x13, 0xb3, 0xf0, 0x1d, 0xbb, 0x4b,
|
||||
0x94, 0x6d, 0x26, 0x68, 0x99, 0x24, 0xb4, 0x8b, 0xb7, 0x59, 0x84, 0xd3, 0x49, 0x42, 0x0b, 0xb6,
|
||||
0x99, 0xe4, 0x63, 0x50, 0x82, 0xb2, 0x4b, 0xb6, 0x59, 0x94, 0xce, 0x89, 0xd9, 0x66, 0x0c, 0x50,
|
||||
0xd9, 0x66, 0x01, 0x73, 0x12, 0xb7, 0xcd, 0x46, 0xe8, 0xec, 0xb8, 0x6d, 0x36, 0x4a, 0xbe, 0xc4,
|
||||
0xcc, 0x23, 0xc3, 0x0d, 0x6d, 0xb3, 0x99, 0x18, 0x92, 0x05, 0xbd, 0x92, 0xe0, 0xc4, 0x58, 0x96,
|
||||
0xbc, 0xfe, 0xea, 0x0b, 0x4a, 0x27, 0xae, 0x71, 0xee, 0x7e, 0xb9, 0xc6, 0xff, 0x5d, 0x83, 0xd9,
|
||||
0x38, 0x82, 0x06, 0x25, 0xe0, 0x24, 0xb0, 0xeb, 0xf5, 0xa5, 0x17, 0x15, 0xbf, 0xd8, 0x5b, 0xfe,
|
||||
0xaa, 0x7f, 0x5c, 0xfd, 0xcd, 0x77, 0xf3, 0xda, 0xef, 0xbe, 0x9b, 0xd7, 0xfe, 0xf0, 0xdd, 0xbc,
|
||||
0xf6, 0x1f, 0x7f, 0x9c, 0x9f, 0x38, 0xcc, 0xb2, 0xff, 0x13, 0xf6, 0xc6, 0x9f, 0x03, 0x00, 0x00,
|
||||
0xff, 0xff, 0x3f, 0x89, 0x92, 0xdc, 0x9a, 0x36, 0x00, 0x00,
|
||||
// 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,
|
||||
}
|
||||
|
@ -661,14 +661,17 @@ message WatchRequest {
|
||||
message WatchCreateRequest {
|
||||
// key is the key to register for watching.
|
||||
bytes key = 1;
|
||||
|
||||
// range_end is the end of the range [key, range_end) to watch. If range_end is not given,
|
||||
// only the key argument is watched. If range_end is equal to '\0', all keys greater than
|
||||
// or equal to the key argument are watched.
|
||||
// If the range_end is one bit larger than the given key,
|
||||
// then all keys with the prefix (the given key) will be watched.
|
||||
bytes range_end = 2;
|
||||
|
||||
// start_revision is an optional revision to watch from (inclusive). No start_revision is "now".
|
||||
int64 start_revision = 3;
|
||||
|
||||
// progress_notify is set so that the etcd server will periodically send a WatchResponse with
|
||||
// no events to the new watcher if there are no recent events. It is useful when clients
|
||||
// wish to recover a disconnected watcher starting from a recent known revision.
|
||||
@ -676,11 +679,12 @@ message WatchCreateRequest {
|
||||
bool progress_notify = 4;
|
||||
|
||||
enum FilterType {
|
||||
// filter out put event.
|
||||
NOPUT = 0;
|
||||
// filter out delete event.
|
||||
NODELETE = 1;
|
||||
// filter out put event.
|
||||
NOPUT = 0;
|
||||
// filter out delete event.
|
||||
NODELETE = 1;
|
||||
}
|
||||
|
||||
// filters filter the events at server side before it sends back to the watcher.
|
||||
repeated FilterType filters = 5;
|
||||
|
||||
@ -694,6 +698,9 @@ message WatchCreateRequest {
|
||||
// watchers on the same stream. Creating a watcher with an ID already in
|
||||
// use on the stream will cause an error to be returned.
|
||||
int64 watch_id = 7;
|
||||
|
||||
// fragment enables splitting large revisions into multiple watch responses.
|
||||
bool fragment = 8;
|
||||
}
|
||||
|
||||
message WatchCancelRequest {
|
||||
@ -705,14 +712,17 @@ message WatchResponse {
|
||||
ResponseHeader header = 1;
|
||||
// watch_id is the ID of the watcher that corresponds to the response.
|
||||
int64 watch_id = 2;
|
||||
|
||||
// created is set to true if the response is for a create watch request.
|
||||
// The client should record the watch_id and expect to receive events for
|
||||
// the created watcher from the same stream.
|
||||
// All events sent to the created watcher will attach with the same watch_id.
|
||||
bool created = 3;
|
||||
|
||||
// canceled is set to true if the response is for a cancel watch request.
|
||||
// No further events will be sent to the canceled watcher.
|
||||
bool canceled = 4;
|
||||
|
||||
// compact_revision is set to the minimum index if a watcher tries to watch
|
||||
// at a compacted index.
|
||||
//
|
||||
@ -726,6 +736,9 @@ message WatchResponse {
|
||||
// cancel_reason indicates the reason for canceling the watcher.
|
||||
string cancel_reason = 6;
|
||||
|
||||
// framgment is true if large watch response was split over multiple responses.
|
||||
bool fragment = 7;
|
||||
|
||||
repeated mvccpb.Event events = 11;
|
||||
}
|
||||
|
||||
|
@ -27,7 +27,7 @@ import (
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
"github.com/coreos/etcd/etcdserver/v2store"
|
||||
"github.com/coreos/etcd/etcdserver/api/v2store"
|
||||
"github.com/coreos/etcd/mvcc/backend"
|
||||
"github.com/coreos/etcd/pkg/netutil"
|
||||
"github.com/coreos/etcd/pkg/types"
|
||||
|
@ -21,7 +21,7 @@ import (
|
||||
"reflect"
|
||||
"testing"
|
||||
|
||||
"github.com/coreos/etcd/etcdserver/v2store"
|
||||
"github.com/coreos/etcd/etcdserver/api/v2store"
|
||||
"github.com/coreos/etcd/pkg/mock/mockstore"
|
||||
"github.com/coreos/etcd/pkg/testutil"
|
||||
"github.com/coreos/etcd/pkg/types"
|
||||
|
@ -17,7 +17,7 @@ package membership
|
||||
import (
|
||||
"errors"
|
||||
|
||||
"github.com/coreos/etcd/etcdserver/v2error"
|
||||
"github.com/coreos/etcd/etcdserver/api/v2error"
|
||||
)
|
||||
|
||||
var (
|
||||
|
@ -19,7 +19,7 @@ import (
|
||||
"fmt"
|
||||
"path"
|
||||
|
||||
"github.com/coreos/etcd/etcdserver/v2store"
|
||||
"github.com/coreos/etcd/etcdserver/api/v2store"
|
||||
"github.com/coreos/etcd/mvcc/backend"
|
||||
"github.com/coreos/etcd/pkg/types"
|
||||
|
||||
|
@ -35,10 +35,10 @@ import (
|
||||
"github.com/coreos/etcd/discovery"
|
||||
"github.com/coreos/etcd/etcdserver/api"
|
||||
"github.com/coreos/etcd/etcdserver/api/v2http/httptypes"
|
||||
stats "github.com/coreos/etcd/etcdserver/api/v2stats"
|
||||
"github.com/coreos/etcd/etcdserver/api/v2store"
|
||||
pb "github.com/coreos/etcd/etcdserver/etcdserverpb"
|
||||
"github.com/coreos/etcd/etcdserver/membership"
|
||||
"github.com/coreos/etcd/etcdserver/stats"
|
||||
"github.com/coreos/etcd/etcdserver/v2store"
|
||||
"github.com/coreos/etcd/lease"
|
||||
"github.com/coreos/etcd/lease/leasehttp"
|
||||
"github.com/coreos/etcd/mvcc"
|
||||
|
@ -29,9 +29,9 @@ import (
|
||||
|
||||
"go.uber.org/zap"
|
||||
|
||||
"github.com/coreos/etcd/etcdserver/api/v2store"
|
||||
pb "github.com/coreos/etcd/etcdserver/etcdserverpb"
|
||||
"github.com/coreos/etcd/etcdserver/membership"
|
||||
"github.com/coreos/etcd/etcdserver/v2store"
|
||||
"github.com/coreos/etcd/lease"
|
||||
"github.com/coreos/etcd/mvcc"
|
||||
"github.com/coreos/etcd/mvcc/backend"
|
||||
|
@ -18,8 +18,8 @@ import (
|
||||
"context"
|
||||
"time"
|
||||
|
||||
"github.com/coreos/etcd/etcdserver/api/v2store"
|
||||
pb "github.com/coreos/etcd/etcdserver/etcdserverpb"
|
||||
"github.com/coreos/etcd/etcdserver/v2store"
|
||||
)
|
||||
|
||||
type RequestV2 pb.Request
|
||||
|
@ -17,7 +17,7 @@ package mockstore
|
||||
import (
|
||||
"time"
|
||||
|
||||
"github.com/coreos/etcd/etcdserver/v2store"
|
||||
"github.com/coreos/etcd/etcdserver/api/v2store"
|
||||
"github.com/coreos/etcd/pkg/testutil"
|
||||
)
|
||||
|
||||
|
@ -21,7 +21,7 @@ import (
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/coreos/etcd/etcdserver/stats"
|
||||
stats "github.com/coreos/etcd/etcdserver/api/v2stats"
|
||||
"github.com/coreos/etcd/pkg/types"
|
||||
"github.com/coreos/etcd/raft"
|
||||
"github.com/coreos/etcd/raft/raftpb"
|
||||
|
@ -20,7 +20,7 @@ import (
|
||||
"io"
|
||||
"time"
|
||||
|
||||
"github.com/coreos/etcd/etcdserver/stats"
|
||||
stats "github.com/coreos/etcd/etcdserver/api/v2stats"
|
||||
"github.com/coreos/etcd/pkg/pbutil"
|
||||
"github.com/coreos/etcd/pkg/types"
|
||||
"github.com/coreos/etcd/raft/raftpb"
|
||||
|
@ -19,7 +19,7 @@ import (
|
||||
"reflect"
|
||||
"testing"
|
||||
|
||||
"github.com/coreos/etcd/etcdserver/stats"
|
||||
stats "github.com/coreos/etcd/etcdserver/api/v2stats"
|
||||
"github.com/coreos/etcd/pkg/types"
|
||||
"github.com/coreos/etcd/raft/raftpb"
|
||||
)
|
||||
|
@ -19,7 +19,7 @@ import (
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
"github.com/coreos/etcd/etcdserver/stats"
|
||||
stats "github.com/coreos/etcd/etcdserver/api/v2stats"
|
||||
"github.com/coreos/etcd/pkg/types"
|
||||
"github.com/coreos/etcd/raft"
|
||||
"github.com/coreos/etcd/raft/raftpb"
|
||||
|
@ -22,7 +22,7 @@ import (
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
"github.com/coreos/etcd/etcdserver/stats"
|
||||
stats "github.com/coreos/etcd/etcdserver/api/v2stats"
|
||||
"github.com/coreos/etcd/pkg/pbutil"
|
||||
"github.com/coreos/etcd/pkg/types"
|
||||
"github.com/coreos/etcd/raft"
|
||||
|
@ -24,13 +24,13 @@ import (
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"go.uber.org/zap"
|
||||
|
||||
"github.com/coreos/etcd/etcdserver/stats"
|
||||
stats "github.com/coreos/etcd/etcdserver/api/v2stats"
|
||||
"github.com/coreos/etcd/pkg/testutil"
|
||||
"github.com/coreos/etcd/pkg/types"
|
||||
"github.com/coreos/etcd/raft/raftpb"
|
||||
"github.com/coreos/etcd/version"
|
||||
|
||||
"go.uber.org/zap"
|
||||
)
|
||||
|
||||
// TestPipelineSend tests that pipeline could send data using roundtripper
|
||||
|
@ -25,7 +25,7 @@ import (
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
"github.com/coreos/etcd/etcdserver/stats"
|
||||
stats "github.com/coreos/etcd/etcdserver/api/v2stats"
|
||||
"github.com/coreos/etcd/pkg/httputil"
|
||||
"github.com/coreos/etcd/pkg/transport"
|
||||
"github.com/coreos/etcd/pkg/types"
|
||||
|
@ -26,7 +26,7 @@ import (
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/coreos/etcd/etcdserver/stats"
|
||||
stats "github.com/coreos/etcd/etcdserver/api/v2stats"
|
||||
"github.com/coreos/etcd/pkg/testutil"
|
||||
"github.com/coreos/etcd/pkg/types"
|
||||
"github.com/coreos/etcd/raft/raftpb"
|
||||
|
@ -20,7 +20,7 @@ import (
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
"github.com/coreos/etcd/etcdserver/stats"
|
||||
stats "github.com/coreos/etcd/etcdserver/api/v2stats"
|
||||
"github.com/coreos/etcd/pkg/logutil"
|
||||
"github.com/coreos/etcd/pkg/transport"
|
||||
"github.com/coreos/etcd/pkg/types"
|
||||
|
@ -21,7 +21,7 @@ import (
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/coreos/etcd/etcdserver/stats"
|
||||
stats "github.com/coreos/etcd/etcdserver/api/v2stats"
|
||||
"github.com/coreos/etcd/pkg/types"
|
||||
"github.com/coreos/etcd/raft"
|
||||
"github.com/coreos/etcd/raft/raftpb"
|
||||
|
@ -20,10 +20,11 @@ import (
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/coreos/etcd/etcdserver/stats"
|
||||
stats "github.com/coreos/etcd/etcdserver/api/v2stats"
|
||||
"github.com/coreos/etcd/pkg/testutil"
|
||||
"github.com/coreos/etcd/pkg/types"
|
||||
"github.com/coreos/etcd/raft/raftpb"
|
||||
|
||||
"github.com/xiang90/probing"
|
||||
)
|
||||
|
||||
|
@ -29,9 +29,9 @@ import (
|
||||
|
||||
"github.com/coreos/etcd/clientv3"
|
||||
"github.com/coreos/etcd/etcdserver"
|
||||
"github.com/coreos/etcd/etcdserver/api/v2store"
|
||||
"github.com/coreos/etcd/etcdserver/etcdserverpb"
|
||||
"github.com/coreos/etcd/etcdserver/membership"
|
||||
"github.com/coreos/etcd/etcdserver/v2store"
|
||||
"github.com/coreos/etcd/lease"
|
||||
"github.com/coreos/etcd/mvcc"
|
||||
"github.com/coreos/etcd/mvcc/backend"
|
||||
|
2
test
2
test
@ -193,7 +193,7 @@ function integration_extra {
|
||||
go test -timeout 1m -v ${RACE} -cpu "${TEST_CPUS}" "$@" "${REPO_PATH}/client/integration"
|
||||
go test -timeout 25m -v ${RACE} -cpu "${TEST_CPUS}" "$@" "${REPO_PATH}/clientv3/integration"
|
||||
go test -timeout 1m -v -cpu "${TEST_CPUS}" "$@" "${REPO_PATH}/contrib/raftexample"
|
||||
go test -timeout 5m -v ${RACE} -tags v2v3 "$@" "${REPO_PATH}/etcdserver/v2store"
|
||||
go test -timeout 5m -v ${RACE} -tags v2v3 "$@" "${REPO_PATH}/etcdserver/api/v2store"
|
||||
go test -timeout 1m -v ${RACE} -cpu "${TEST_CPUS}" -run=Example "$@" "${TEST[@]}"
|
||||
}
|
||||
|
||||
|
Loading…
x
Reference in New Issue
Block a user