Browse Source

Merge pull request #9291 from gyuho/fragment-watch

*: fragment watch response by server request limit
Gyuho Lee 7 years ago
parent
commit
53373fecdd
77 changed files with 848 additions and 361 deletions
  1. 21 5
      CHANGELOG-3.4.md
  2. 2 0
      Documentation/dev-guide/api_reference_v3.md
  3. 10 0
      Documentation/dev-guide/apispec/swagger/rpc.swagger.json
  4. 123 0
      clientv3/integration/watch_fragment_test.go
  5. 31 5
      clientv3/op.go
  6. 46 3
      clientv3/watch.go
  7. 1 1
      contrib/raftexample/raft.go
  8. 2 2
      etcdctl/ctlv3/command/migrate_command.go
  9. 1 1
      etcdserver/api/etcdhttp/base.go
  10. 1 1
      etcdserver/api/v2auth/auth.go
  11. 1 1
      etcdserver/api/v2auth/auth_requests.go
  12. 2 2
      etcdserver/api/v2auth/auth_test.go
  13. 0 0
      etcdserver/api/v2error/error.go
  14. 0 0
      etcdserver/api/v2error/error_test.go
  15. 4 4
      etcdserver/api/v2http/client.go
  16. 1 1
      etcdserver/api/v2http/client_auth.go
  17. 1 1
      etcdserver/api/v2http/client_auth_test.go
  18. 2 2
      etcdserver/api/v2http/client_test.go
  19. 1 1
      etcdserver/api/v2http/http.go
  20. 1 1
      etcdserver/api/v2http/http_test.go
  21. 1 1
      etcdserver/api/v2http/metrics.go
  22. 1 1
      etcdserver/api/v2stats/leader.go
  23. 1 1
      etcdserver/api/v2stats/queue.go
  24. 1 1
      etcdserver/api/v2stats/server.go
  25. 3 5
      etcdserver/api/v2stats/stats.go
  26. 0 0
      etcdserver/api/v2store/doc.go
  27. 0 0
      etcdserver/api/v2store/event.go
  28. 1 1
      etcdserver/api/v2store/event_history.go
  29. 0 0
      etcdserver/api/v2store/event_queue.go
  30. 1 1
      etcdserver/api/v2store/event_test.go
  31. 0 0
      etcdserver/api/v2store/heap_test.go
  32. 0 0
      etcdserver/api/v2store/metrics.go
  33. 1 1
      etcdserver/api/v2store/node.go
  34. 0 0
      etcdserver/api/v2store/node_extern.go
  35. 0 0
      etcdserver/api/v2store/node_extern_test.go
  36. 0 0
      etcdserver/api/v2store/node_test.go
  37. 0 0
      etcdserver/api/v2store/stats.go
  38. 0 0
      etcdserver/api/v2store/stats_test.go
  39. 1 1
      etcdserver/api/v2store/store.go
  40. 0 0
      etcdserver/api/v2store/store_bench_test.go
  41. 2 2
      etcdserver/api/v2store/store_test.go
  42. 1 1
      etcdserver/api/v2store/store_ttl_test.go
  43. 1 1
      etcdserver/api/v2store/store_v2_test.go
  44. 1 1
      etcdserver/api/v2store/store_v2v3_test.go
  45. 0 0
      etcdserver/api/v2store/ttl_key_heap.go
  46. 0 0
      etcdserver/api/v2store/watcher.go
  47. 1 1
      etcdserver/api/v2store/watcher_hub.go
  48. 0 0
      etcdserver/api/v2store/watcher_hub_test.go
  49. 0 0
      etcdserver/api/v2store/watcher_test.go
  50. 2 2
      etcdserver/api/v2v3/store.go
  51. 2 2
      etcdserver/api/v2v3/watcher.go
  52. 116 41
      etcdserver/api/v3rpc/watch.go
  53. 95 0
      etcdserver/api/v3rpc/watch_test.go
  54. 1 1
      etcdserver/apply_v2.go
  55. 322 237
      etcdserver/etcdserverpb/rpc.pb.go
  56. 17 4
      etcdserver/etcdserverpb/rpc.proto
  57. 1 1
      etcdserver/membership/cluster.go
  58. 1 1
      etcdserver/membership/cluster_test.go
  59. 1 1
      etcdserver/membership/errors.go
  60. 1 1
      etcdserver/membership/store.go
  61. 2 2
      etcdserver/server.go
  62. 1 1
      etcdserver/server_test.go
  63. 1 1
      etcdserver/v2_server.go
  64. 1 1
      pkg/mock/mockstore/store_recorder.go
  65. 1 1
      rafthttp/functional_test.go
  66. 1 1
      rafthttp/msgappv2_codec.go
  67. 1 1
      rafthttp/msgappv2_codec_test.go
  68. 1 1
      rafthttp/peer.go
  69. 1 1
      rafthttp/pipeline.go
  70. 3 3
      rafthttp/pipeline_test.go
  71. 1 1
      rafthttp/stream.go
  72. 1 1
      rafthttp/stream_test.go
  73. 1 1
      rafthttp/transport.go
  74. 1 1
      rafthttp/transport_bench_test.go
  75. 2 1
      rafthttp/transport_test.go
  76. 1 1
      snapshot/v3_snapshot.go
  77. 1 1
      test

+ 21 - 5
CHANGELOG-3.4.md

@@ -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.

+ 2 - 0
Documentation/dev-guide/api_reference_v3.md

@@ -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 |
 
 

+ 10 - 0
Documentation/dev-guide/apispec/swagger/rpc.swagger.json

@@ -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 - 0
clientv3/integration/watch_fragment_test.go

@@ -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")
+	}
+}

+ 31 - 5
clientv3/op.go

@@ -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.

+ 46 - 3
clientv3/watch.go

@@ -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}

+ 1 - 1
contrib/raftexample/raft.go

@@ -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"

+ 2 - 2
etcdctl/ctlv3/command/migrate_command.go

@@ -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"

+ 1 - 1
etcdserver/api/etcdhttp/base.go

@@ -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"
 

+ 1 - 1
etcdserver/v2auth/auth.go → etcdserver/api/v2auth/auth.go

@@ -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"

+ 1 - 1
etcdserver/v2auth/auth_requests.go → etcdserver/api/v2auth/auth_requests.go

@@ -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"
 )

+ 2 - 2
etcdserver/v2auth/auth_test.go → etcdserver/api/v2auth/auth_test.go

@@ -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"
 )

+ 0 - 0
etcdserver/v2error/error.go → etcdserver/api/v2error/error.go


+ 0 - 0
etcdserver/v2error/error_test.go → etcdserver/api/v2error/error_test.go


+ 4 - 4
etcdserver/api/v2http/client.go

@@ -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"

+ 1 - 1
etcdserver/api/v2http/client_auth.go

@@ -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"
 )

+ 1 - 1
etcdserver/api/v2http/client_auth_test.go

@@ -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"
 )

+ 2 - 2
etcdserver/api/v2http/client_test.go

@@ -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"

+ 1 - 1
etcdserver/api/v2http/http.go

@@ -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"

+ 1 - 1
etcdserver/api/v2http/http_test.go

@@ -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"
 

+ 1 - 1
etcdserver/api/v2http/metrics.go

@@ -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"
 )
 

+ 1 - 1
etcdserver/stats/leader.go → etcdserver/api/v2stats/leader.go

@@ -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"

+ 1 - 1
etcdserver/stats/queue.go → etcdserver/api/v2stats/queue.go

@@ -12,7 +12,7 @@
 // See the License for the specific language governing permissions and
 // limitations under the License.
 
-package stats
+package v2stats
 
 import (
 	"sync"

+ 1 - 1
etcdserver/stats/server.go → etcdserver/api/v2stats/server.go

@@ -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"

+ 3 - 5
etcdserver/stats/stats.go → etcdserver/api/v2stats/stats.go

@@ -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

+ 0 - 0
etcdserver/v2store/doc.go → etcdserver/api/v2store/doc.go


+ 0 - 0
etcdserver/v2store/event.go → etcdserver/api/v2store/event.go


+ 1 - 1
etcdserver/v2store/event_history.go → etcdserver/api/v2store/event_history.go

@@ -20,7 +20,7 @@ import (
 	"strings"
 	"sync"
 
-	"github.com/coreos/etcd/etcdserver/v2error"
+	"github.com/coreos/etcd/etcdserver/api/v2error"
 )
 
 type EventHistory struct {

+ 0 - 0
etcdserver/v2store/event_queue.go → etcdserver/api/v2store/event_queue.go


+ 1 - 1
etcdserver/v2store/event_test.go → etcdserver/api/v2store/event_test.go

@@ -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

+ 0 - 0
etcdserver/v2store/heap_test.go → etcdserver/api/v2store/heap_test.go


+ 0 - 0
etcdserver/v2store/metrics.go → etcdserver/api/v2store/metrics.go


+ 1 - 1
etcdserver/v2store/node.go → etcdserver/api/v2store/node.go

@@ -19,7 +19,7 @@ import (
 	"sort"
 	"time"
 
-	"github.com/coreos/etcd/etcdserver/v2error"
+	"github.com/coreos/etcd/etcdserver/api/v2error"
 
 	"github.com/jonboulle/clockwork"
 )

+ 0 - 0
etcdserver/v2store/node_extern.go → etcdserver/api/v2store/node_extern.go


+ 0 - 0
etcdserver/v2store/node_extern_test.go → etcdserver/api/v2store/node_extern_test.go


+ 0 - 0
etcdserver/v2store/node_test.go → etcdserver/api/v2store/node_test.go


+ 0 - 0
etcdserver/v2store/stats.go → etcdserver/api/v2store/stats.go


+ 0 - 0
etcdserver/v2store/stats_test.go → etcdserver/api/v2store/stats_test.go


+ 1 - 1
etcdserver/v2store/store.go → etcdserver/api/v2store/store.go

@@ -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"

+ 0 - 0
etcdserver/v2store/store_bench_test.go → etcdserver/api/v2store/store_bench_test.go


+ 2 - 2
etcdserver/v2store/store_test.go → etcdserver/api/v2store/store_test.go

@@ -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"
 )
 

+ 1 - 1
etcdserver/v2store/store_ttl_test.go → etcdserver/api/v2store/store_ttl_test.go

@@ -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"

+ 1 - 1
etcdserver/v2store/store_v2_test.go → etcdserver/api/v2store/store_v2_test.go

@@ -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"
 )
 

+ 1 - 1
etcdserver/v2store/store_v2v3_test.go → etcdserver/api/v2store/store_v2v3_test.go

@@ -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"

+ 0 - 0
etcdserver/v2store/ttl_key_heap.go → etcdserver/api/v2store/ttl_key_heap.go


+ 0 - 0
etcdserver/v2store/watcher.go → etcdserver/api/v2store/watcher.go


+ 1 - 1
etcdserver/v2store/watcher_hub.go → etcdserver/api/v2store/watcher_hub.go

@@ -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

+ 0 - 0
etcdserver/v2store/watcher_hub_test.go → etcdserver/api/v2store/watcher_hub_test.go


+ 0 - 0
etcdserver/v2store/watcher_test.go → etcdserver/api/v2store/watcher_test.go


+ 2 - 2
etcdserver/api/v2v3/store.go

@@ -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"
 )
 

+ 2 - 2
etcdserver/api/v2v3/watcher.go

@@ -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) {

+ 116 - 41
etcdserver/api/v3rpc/watch.go

@@ -32,24 +32,31 @@ import (
 )
 
 type watchServer struct {
+	lg *zap.Logger
+
 	clusterID int64
 	memberID  int64
-	sg        etcdserver.RaftStatusGetter
-	watchable mvcc.WatchableKV
 
-	ag AuthGetter
+	maxRequestBytes int
 
-	lg *zap.Logger
+	sg        etcdserver.RaftStatusGetter
+	watchable mvcc.WatchableKV
+	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 - 0
etcdserver/api/v3rpc/watch_test.go

@@ -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
+}

+ 1 - 1
etcdserver/apply_v2.go

@@ -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"

+ 322 - 237
etcdserver/etcdserverpb/rpc.pb.go

@@ -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,
 }

+ 17 - 4
etcdserver/etcdserverpb/rpc.proto

@@ -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;
 }
 

+ 1 - 1
etcdserver/membership/cluster.go

@@ -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"

+ 1 - 1
etcdserver/membership/cluster_test.go

@@ -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"

+ 1 - 1
etcdserver/membership/errors.go

@@ -17,7 +17,7 @@ package membership
 import (
 	"errors"
 
-	"github.com/coreos/etcd/etcdserver/v2error"
+	"github.com/coreos/etcd/etcdserver/api/v2error"
 )
 
 var (

+ 1 - 1
etcdserver/membership/store.go

@@ -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"
 

+ 2 - 2
etcdserver/server.go

@@ -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"

+ 1 - 1
etcdserver/server_test.go

@@ -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"

+ 1 - 1
etcdserver/v2_server.go

@@ -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

+ 1 - 1
pkg/mock/mockstore/store_recorder.go

@@ -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"
 )
 

+ 1 - 1
rafthttp/functional_test.go

@@ -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"

+ 1 - 1
rafthttp/msgappv2_codec.go

@@ -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"

+ 1 - 1
rafthttp/msgappv2_codec_test.go

@@ -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"
 )

+ 1 - 1
rafthttp/peer.go

@@ -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"

+ 1 - 1
rafthttp/pipeline.go

@@ -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"

+ 3 - 3
rafthttp/pipeline_test.go

@@ -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

+ 1 - 1
rafthttp/stream.go

@@ -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"

+ 1 - 1
rafthttp/stream_test.go

@@ -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"

+ 1 - 1
rafthttp/transport.go

@@ -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"

+ 1 - 1
rafthttp/transport_bench_test.go

@@ -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"

+ 2 - 1
rafthttp/transport_test.go

@@ -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"
 )
 

+ 1 - 1
snapshot/v3_snapshot.go

@@ -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"

+ 1 - 1
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[@]}"
 }