From 2cf2dfd0898974179cfeaf0bc925666ed7d28abd Mon Sep 17 00:00:00 2001 From: Connor Date: Sat, 1 Feb 2020 19:51:46 +0800 Subject: [PATCH] add mock pd client (#36) * add mock pd client Signed-off-by: Connor1996 --- .gitignore | 1 + kv/pd/client.go | 49 +-- kv/test_raftstore/pd.go | 544 +++++++++++++++++++++++++ kv/tikv/raftstore/fsm_peer.go | 12 - kv/tikv/raftstore/pd_task_handler.go | 35 +- kv/tikv/raftstore/worker.go | 10 - kv/tikv/worker/worker.go | 8 +- proto/pkg/pdpb/pdpb.pb.go | 582 +++++++++++---------------- proto/proto/pdpb.proto | 12 - 9 files changed, 794 insertions(+), 459 deletions(-) create mode 100644 kv/test_raftstore/pd.go diff --git a/.gitignore b/.gitignore index bdcf34666..12a8fd594 100644 --- a/.gitignore +++ b/.gitignore @@ -1,4 +1,5 @@ .idea* +.vscode* node/node unikv/unikv bin/* diff --git a/kv/pd/client.go b/kv/pd/client.go index c3257e312..e519cb63a 100644 --- a/kv/pd/client.go +++ b/kv/pd/client.go @@ -41,16 +41,12 @@ type Client interface { IsBootstrapped(ctx context.Context) (bool, error) PutStore(ctx context.Context, store *metapb.Store) error GetStore(ctx context.Context, storeID uint64) (*metapb.Store, error) - GetAllStores(ctx context.Context, excludeTombstone bool) ([]*metapb.Store, error) - GetClusterConfig(ctx context.Context) (*metapb.Cluster, error) GetRegion(ctx context.Context, key []byte) (*metapb.Region, *metapb.Peer, error) GetRegionByID(ctx context.Context, regionID uint64) (*metapb.Region, *metapb.Peer, error) - ReportRegion(*pdpb.RegionHeartbeatRequest) AskBatchSplit(ctx context.Context, region *metapb.Region, count int) (*pdpb.AskBatchSplitResponse, error) - ReportBatchSplit(ctx context.Context, regions []*metapb.Region) error - GetGCSafePoint(ctx context.Context) (uint64, error) StoreHeartbeat(ctx context.Context, stats *pdpb.StoreStats) error - SetRegionHeartbeatResponseHandler(h func(*pdpb.RegionHeartbeatResponse)) + RegionHeartbeat(*pdpb.RegionHeartbeatRequest) + SetRegionHeartbeatResponseHandler(storeID uint64, h func(*pdpb.RegionHeartbeatResponse)) Close() } @@ -563,43 +559,6 @@ func (c *client) AskBatchSplit(ctx context.Context, region *metapb.Region, count return resp, nil } -func (c *client) ReportBatchSplit(ctx context.Context, regions []*metapb.Region) error { - var resp *pdpb.ReportBatchSplitResponse - err := c.doRequest(ctx, func(ctx context.Context, client pdpb.PDClient) error { - var err1 error - resp, err1 = client.ReportBatchSplit(ctx, &pdpb.ReportBatchSplitRequest{ - Header: c.requestHeader(), - Regions: regions, - }) - return err1 - }) - if err != nil { - return err - } - if herr := resp.Header.GetError(); herr != nil { - return errors.New(herr.String()) - } - return nil -} - -func (c *client) GetGCSafePoint(ctx context.Context) (uint64, error) { - var resp *pdpb.GetGCSafePointResponse - err := c.doRequest(ctx, func(ctx context.Context, client pdpb.PDClient) error { - var err1 error - resp, err1 = client.GetGCSafePoint(ctx, &pdpb.GetGCSafePointRequest{ - Header: c.requestHeader(), - }) - return err1 - }) - if err != nil { - return 0, err - } - if herr := resp.Header.GetError(); herr != nil { - return 0, errors.New(herr.String()) - } - return resp.SafePoint, nil -} - func (c *client) StoreHeartbeat(ctx context.Context, stats *pdpb.StoreStats) error { var resp *pdpb.StoreHeartbeatResponse err := c.doRequest(ctx, func(ctx context.Context, client pdpb.PDClient) error { @@ -619,11 +578,11 @@ func (c *client) StoreHeartbeat(ctx context.Context, stats *pdpb.StoreStats) err return nil } -func (c *client) ReportRegion(request *pdpb.RegionHeartbeatRequest) { +func (c *client) RegionHeartbeat(request *pdpb.RegionHeartbeatRequest) { c.regionCh <- request } -func (c *client) SetRegionHeartbeatResponseHandler(h func(*pdpb.RegionHeartbeatResponse)) { +func (c *client) SetRegionHeartbeatResponseHandler(_ uint64, h func(*pdpb.RegionHeartbeatResponse)) { if h == nil { h = func(*pdpb.RegionHeartbeatResponse) {} } diff --git a/kv/test_raftstore/pd.go b/kv/test_raftstore/pd.go new file mode 100644 index 000000000..2c655967d --- /dev/null +++ b/kv/test_raftstore/pd.go @@ -0,0 +1,544 @@ +package test_raftstore + +import ( + "bytes" + "context" + "sync" + + "github.com/google/btree" + "github.com/ngaut/log" + "github.com/pingcap-incubator/tinykv/kv/tikv/raftstore" + "github.com/pingcap-incubator/tinykv/proto/pkg/eraftpb" + "github.com/pingcap-incubator/tinykv/proto/pkg/metapb" + "github.com/pingcap-incubator/tinykv/proto/pkg/pdpb" + "github.com/pingcap/errors" +) + +var _ btree.Item = ®ionItem{} + +type regionItem struct { + region metapb.Region +} + +// Less returns true if the region start key is less than the other. +func (r *regionItem) Less(other btree.Item) bool { + left := r.region.GetStartKey() + right := other.(*regionItem).region.GetStartKey() + return bytes.Compare(left, right) < 0 +} + +func (r *regionItem) Contains(key []byte) bool { + start, end := r.region.GetStartKey(), r.region.GetEndKey() + return bytes.Compare(key, start) >= 0 && (len(end) == 0 || bytes.Compare(key, end) < 0) +} + +type OperatorType int64 + +const ( + OperatorTypeAddPeer = 1 + OperatorTypeRemovePeer = 2 + OperatorTypeTransferLeader = 3 +) + +type Operator struct { + Type OperatorType + Data interface{} +} + +type OpAddPeer struct { + peer metapb.Peer + pending bool +} + +type OpRemovePeer struct { + peer metapb.Peer +} + +type OpTransferLeader struct { + peer metapb.Peer +} + +type Store struct { + store metapb.Store + heartbeatResponseHandler func(*pdpb.RegionHeartbeatResponse) +} + +func NewStore(store *metapb.Store) *Store { + return &Store{ + store: *store, + heartbeatResponseHandler: nil, + } +} + +type MockPDClient struct { + sync.RWMutex + + clusterID uint64 + + meta metapb.Cluster + stores map[uint64]*Store + regionsRange btree.BTree // key -> region + regionsKey map[uint64][]byte // regionID -> startKey + + baseID uint64 + + operators map[uint64]*Operator + leaders map[uint64]*metapb.Peer // regionID -> peer + pendingPeers map[uint64]*metapb.Peer // peerID -> peer + + bootstrapped bool +} + +// Implement PDClient interface +func (m *MockPDClient) GetClusterID(ctx context.Context) uint64 { + m.RLock() + defer m.RUnlock() + return m.clusterID +} + +func (m *MockPDClient) AllocID(ctx context.Context) (uint64, error) { + m.Lock() + defer m.Unlock() + ret := m.baseID + m.baseID++ + return ret, nil +} + +func (m *MockPDClient) Bootstrap(ctx context.Context, store *metapb.Store, region *metapb.Region) (*pdpb.BootstrapResponse, error) { + m.Lock() + defer m.Unlock() + + resp := &pdpb.BootstrapResponse{ + Header: &pdpb.ResponseHeader{ClusterId: m.clusterID}, + } + + if m.bootstrapped == true || len(m.regionsKey) != 0 { + m.bootstrapped = true + resp.Header.Error = &pdpb.Error{ + Type: pdpb.ErrorType_NOT_BOOTSTRAPPED, + Message: "cluster is not bootstrapped", + } + return resp, nil + } + + m.stores[store.GetId()] = NewStore(store) + m.addRegionLocked(region) + m.bootstrapped = true + return resp, nil +} + +func (m *MockPDClient) IsBootstrapped(ctx context.Context) (bool, error) { + m.RLock() + defer m.RUnlock() + return m.bootstrapped, nil +} + +func (m *MockPDClient) checkBootstrap() error { + if bootstrapped, _ := m.IsBootstrapped(context.TODO()); !bootstrapped { + return errors.New("not bootstrapped") + } + return nil +} + +func (m *MockPDClient) PutStore(ctx context.Context, store *metapb.Store) error { + if err := m.checkBootstrap(); err != nil { + return err + } + m.Lock() + defer m.Unlock() + + s := NewStore(store) + m.stores[store.GetId()] = s + return nil +} + +func (m *MockPDClient) GetStore(ctx context.Context, storeID uint64) (*metapb.Store, error) { + if err := m.checkBootstrap(); err != nil { + return nil, err + } + m.RLock() + defer m.RUnlock() + + s, ok := m.stores[storeID] + if !ok { + return nil, errors.Errorf("store %d not found", storeID) + } + return &s.store, nil +} + +func (m *MockPDClient) GetRegion(ctx context.Context, key []byte) (*metapb.Region, *metapb.Peer, error) { + if err := m.checkBootstrap(); err != nil { + return nil, nil, err + } + m.RLock() + defer m.RUnlock() + region, leader := m.getRegionLocked(key) + return region, leader, nil +} + +func (m *MockPDClient) getRegionLocked(key []byte) (*metapb.Region, *metapb.Peer) { + result := m.findRegion(key) + if result == nil { + return nil, nil + } + + leader := m.leaders[result.region.GetId()] + return &result.region, leader +} + +func (m *MockPDClient) GetRegionByID(ctx context.Context, regionID uint64) (*metapb.Region, *metapb.Peer, error) { + if err := m.checkBootstrap(); err != nil { + return nil, nil, err + } + m.RLock() + defer m.RUnlock() + return m.getRegionByIDLocked(regionID) +} + +func (m *MockPDClient) getRegionByIDLocked(regionID uint64) (*metapb.Region, *metapb.Peer, error) { + startKey := m.regionsKey[regionID] + region, leader := m.getRegionLocked(startKey) + return region, leader, nil +} + +func (m *MockPDClient) AskBatchSplit(ctx context.Context, region *metapb.Region, count int) (*pdpb.AskBatchSplitResponse, error) { + resp := new(pdpb.AskBatchSplitResponse) + resp.Header = &pdpb.ResponseHeader{ClusterId: m.clusterID} + curRegion, _, err := m.GetRegionByID(ctx, region.GetId()) + if err != nil { + return resp, err + } + if raftstore.IsEpochStale(region.RegionEpoch, curRegion.RegionEpoch) { + return resp, errors.New("epoch is stale") + } + + for i := 0; i < count; i++ { + ids := new(pdpb.SplitID) + id, _ := m.AllocID(ctx) + ids.NewRegionId = id + + for range region.GetPeers() { + id, _ := m.AllocID(ctx) + ids.NewPeerIds = append(ids.NewPeerIds, id) + } + resp.Ids = append(resp.Ids, ids) + } + return resp, nil +} + +func (m *MockPDClient) StoreHeartbeat(ctx context.Context, stats *pdpb.StoreStats) error { + if err := m.checkBootstrap(); err != nil { + return err + } + // nothing need to do + return nil +} + +func (m *MockPDClient) RegionHeartbeat(req *pdpb.RegionHeartbeatRequest) error { + if err := m.checkBootstrap(); err != nil { + return err + } + + m.Lock() + defer m.Unlock() + + regionID := req.Region.GetId() + for _, p := range req.Region.GetPeers() { + delete(m.pendingPeers, p.GetId()) + } + for _, p := range req.GetPendingPeers() { + m.pendingPeers[p.GetId()] = p + } + m.leaders[regionID] = req.Leader + + if err := m.handleHeartbeatVersion(req.Region); err != nil { + return err + } + if err := m.handleHeartbeatConfVersion(req.Region); err != nil { + return err + } + + resp := &pdpb.RegionHeartbeatResponse{ + Header: &pdpb.ResponseHeader{ClusterId: m.clusterID}, + RegionId: regionID, + RegionEpoch: req.Region.GetRegionEpoch(), + TargetPeer: req.Leader, + } + if op := m.operators[regionID]; op != nil { + if m.tryFinished(op, req.Region, req.Leader) { + delete(m.operators, regionID) + } else { + m.makeRegionHeartbeatResponse(op, resp) + } + log.Debugf("[region %d] schedule %v", regionID, op) + } + + store := m.stores[req.Leader.GetStoreId()] + store.heartbeatResponseHandler(resp) + return nil +} + +func (m *MockPDClient) handleHeartbeatVersion(region *metapb.Region) error { + if bytes.Compare(region.GetStartKey(), region.GetEndKey()) > 0 { + panic("start key > end key") + } + + for { + searchRegion, _ := m.getRegionLocked(region.GetStartKey()) + if searchRegion == nil { + m.addRegionLocked(region) + return nil + } else { + if bytes.Equal(searchRegion.GetStartKey(), region.GetStartKey()) && + bytes.Equal(searchRegion.GetEndKey(), region.GetEndKey()) { + // the two regions' range are same, must check epoch + if raftstore.IsEpochStale(region.RegionEpoch, searchRegion.RegionEpoch) { + return errors.New("epoch is stale") + } + if searchRegion.RegionEpoch.Version < region.RegionEpoch.Version { + m.removeRegionLocked(searchRegion) + m.addRegionLocked(region) + } + return nil + } + + if bytes.Compare(searchRegion.GetStartKey(), region.GetEndKey()) > 0 { + // No range covers [start, end) now, insert directly. + m.addRegionLocked(region) + return nil + } else { + // overlap, remove old, insert new. + // E.g, 1 [a, c) -> 1 [a, b) + 2 [b, c), either new 1 or 2 reports, the region + // is overlapped with origin [a, c). + if region.GetRegionEpoch().GetVersion() <= searchRegion.GetRegionEpoch().GetVersion() { + return errors.New("epoch is stale") + } + m.removeRegionLocked(searchRegion) + } + } + } +} + +func (m *MockPDClient) handleHeartbeatConfVersion(region *metapb.Region) error { + searchRegion, _ := m.getRegionLocked(region.GetStartKey()) + if raftstore.IsEpochStale(region.RegionEpoch, searchRegion.RegionEpoch) { + return errors.New("epoch is stale") + } + + regionPeerLen := len(region.GetPeers()) + searchRegionPeerLen := len(searchRegion.GetPeers()) + + if region.RegionEpoch.ConfVer > searchRegion.RegionEpoch.ConfVer { + // If ConfVer changed, TinyKV has added/removed one peer already. + // So scheduler and TinyKV can't have same peer count and can only have + // only one different peer. + if searchRegionPeerLen > regionPeerLen { + if regionPeerLen-searchRegionPeerLen != 1 { + panic("should only one conf change") + } + if len(GetDiffPeers(searchRegion, region)) != 1 { + panic("should only one different peer") + } + if len(GetDiffPeers(region, searchRegion)) != 0 { + panic("should include all peers") + } + } else if searchRegionPeerLen < regionPeerLen { + if regionPeerLen-searchRegionPeerLen != 1 { + panic("should only one conf change") + } + if len(GetDiffPeers(region, searchRegion)) != 1 { + panic("should only one different peer") + } + if len(GetDiffPeers(searchRegion, region)) != 0 { + panic("should include all peers") + } + } else { + MustSamePeers(searchRegion, region) + if searchRegion.RegionEpoch.ConfVer+1 != region.RegionEpoch.ConfVer { + panic("unmatched conf version") + } + if searchRegion.RegionEpoch.Version+1 != region.RegionEpoch.Version { + panic("unmatched version") + } + } + + // update the region. + if m.regionsRange.ReplaceOrInsert(®ionItem{region: *region}) == nil { + panic("update inexistent region ") + } + } else { + MustSamePeers(searchRegion, region) + } + return nil +} + +func (m *MockPDClient) tryFinished(op *Operator, region *metapb.Region, leader *metapb.Peer) bool { + switch op.Type { + case OperatorTypeAddPeer: + add := op.Data.(OpAddPeer) + if !add.pending { + for _, p := range region.GetPeers() { + if add.peer.GetId() == p.GetId() { + add.pending = true + } else { + // TinyKV rejects AddNode. + return false + } + } + } else { + _, found := m.pendingPeers[add.peer.GetId()] + return !found + } + case OperatorTypeRemovePeer: + remove := op.Data.(OpRemovePeer) + for _, p := range region.GetPeers() { + if remove.peer.GetId() == p.GetId() { + return false + } + } + return true + case OperatorTypeTransferLeader: + transfer := op.Data.(OpTransferLeader) + return leader.GetId() == transfer.peer.GetId() + } + panic("unreachable") +} + +func (m *MockPDClient) makeRegionHeartbeatResponse(op *Operator, resp *pdpb.RegionHeartbeatResponse) { + switch op.Type { + case OperatorTypeAddPeer: + add := op.Data.(OpAddPeer) + if !add.pending { + resp.ChangePeer = &pdpb.ChangePeer{ + ChangeType: eraftpb.ConfChangeType_AddNode, + Peer: &add.peer, + } + } + case OperatorTypeRemovePeer: + remove := op.Data.(OpRemovePeer) + resp.ChangePeer = &pdpb.ChangePeer{ + ChangeType: eraftpb.ConfChangeType_RemoveNode, + Peer: &remove.peer, + } + case OperatorTypeTransferLeader: + transfer := op.Data.(OpTransferLeader) + resp.TransferLeader = &pdpb.TransferLeader{ + Peer: &transfer.peer, + } + } +} + +func (m *MockPDClient) SetRegionHeartbeatResponseHandler(storeID uint64, h func(*pdpb.RegionHeartbeatResponse)) { + if h == nil { + h = func(*pdpb.RegionHeartbeatResponse) {} + } + m.Lock() + defer m.Unlock() + store := m.stores[storeID] + store.heartbeatResponseHandler = h +} + +func (m *MockPDClient) Close() { + // do nothing +} + +func (m *MockPDClient) findRegion(key []byte) *regionItem { + item := ®ionItem{region: metapb.Region{StartKey: key}} + + var result *regionItem + m.regionsRange.DescendLessOrEqual(item, func(i btree.Item) bool { + result = i.(*regionItem) + return false + }) + + if result == nil || !result.Contains(key) { + return nil + } + + return result +} + +func (m *MockPDClient) addRegionLocked(region *metapb.Region) { + m.regionsKey[region.GetId()] = region.GetStartKey() + m.regionsRange.ReplaceOrInsert(®ionItem{region: *region}) +} + +func (m *MockPDClient) removeRegionLocked(region *metapb.Region) { + delete(m.regionsKey, region.GetId()) + result := m.findRegion(region.GetStartKey()) + if result == nil || result.region.GetId() != region.GetId() { + return + } + m.regionsRange.Delete(result) +} + +// Extra API for tests +func (m *MockPDClient) AddPeer(regionID uint64, peer metapb.Peer) { + m.scheduleOperator(regionID, &Operator{ + Type: OperatorTypeAddPeer, + Data: &OpAddPeer{ + peer: peer, + pending: false, + }, + }) +} + +func (m *MockPDClient) RemovePeer(regionID uint64, peer metapb.Peer) { + m.scheduleOperator(regionID, &Operator{ + Type: OperatorTypeRemovePeer, + Data: &OpRemovePeer{ + peer: peer, + }, + }) +} + +func (m *MockPDClient) TransferLeader(regionID uint64, peer metapb.Peer) { + m.scheduleOperator(regionID, &Operator{ + Type: OperatorTypeTransferLeader, + Data: &OpTransferLeader{ + peer: peer, + }, + }) +} + +func (m *MockPDClient) scheduleOperator(regionID uint64, op *Operator) { + m.Lock() + defer m.Unlock() + m.operators[regionID] = op +} + +// Utilities +func MustSamePeers(left *metapb.Region, right *metapb.Region) { + if len(left.GetPeers()) != len(right.GetPeers()) { + panic("unmatched peers length") + } + for _, p := range left.GetPeers() { + found := false + for _, p1 := range right.GetPeers() { + if p.GetStoreId() == p1.GetStoreId() { + found = true + break + } + } + if !found { + panic("not found the peer") + } + } +} + +func GetDiffPeers(left *metapb.Region, right *metapb.Region) []*metapb.Peer { + peers := make([]*metapb.Peer, 1) + for _, p := range left.GetPeers() { + found := false + for _, p1 := range right.GetPeers() { + if p.GetStoreId() == p1.GetStoreId() { + found = true + break + } + } + if !found { + peers = append(peers, p) + } + } + return peers +} diff --git a/kv/tikv/raftstore/fsm_peer.go b/kv/tikv/raftstore/fsm_peer.go index 041d621ed..dcdb92ef0 100644 --- a/kv/tikv/raftstore/fsm_peer.go +++ b/kv/tikv/raftstore/fsm_peer.go @@ -589,12 +589,6 @@ func (d *peerMsgHandler) destroyPeer(mergeByTarget bool) { d.ctx.storeMetaLock.Unlock() // send messages out of store meta lock. d.ctx.applyMsgs.appendMsg(regionID, message.NewPeerMsg(message.MsgTypeApplyDestroy, regionID, nil)) - d.ctx.pdTaskSender <- worker.Task{ - Tp: worker.TaskTypePDDestroyPeer, - Data: &pdDestroyPeerTask{ - regionID: regionID, - }, - } }() meta := d.ctx.storeMeta isInitialized := d.peer.isInitialized() @@ -698,12 +692,6 @@ func (d *peerMsgHandler) onReadySplitRegion(derived *metapb.Region, regions []*m d.peer.HeartbeatPd(d.ctx.pdTaskSender) // Notify pd immediately to let it update the region meta. log.Infof("%s notify pd with split count %d", d.tag(), len(regions)) - // Now pd only uses ReportBatchSplit for history operation show, - // so we send it independently here. - d.ctx.pdTaskSender <- worker.Task{ - Tp: worker.TaskTypePDReportBatchSplit, - Data: &pdReportBatchSplitTask{regions: regions}, - } } lastRegion := regions[len(regions)-1] diff --git a/kv/tikv/raftstore/pd_task_handler.go b/kv/tikv/raftstore/pd_task_handler.go index 04b8b3dab..0c93d80bf 100644 --- a/kv/tikv/raftstore/pd_task_handler.go +++ b/kv/tikv/raftstore/pd_task_handler.go @@ -10,7 +10,6 @@ import ( "github.com/pingcap-incubator/tinykv/proto/pkg/metapb" "github.com/pingcap-incubator/tinykv/proto/pkg/pdpb" "github.com/pingcap-incubator/tinykv/proto/pkg/raft_cmdpb" - "github.com/pingcap-incubator/tinykv/proto/pkg/raft_serverpb" "github.com/shirou/gopsutil/disk" ) @@ -36,19 +35,14 @@ func (r *pdTaskHandler) Handle(t worker.Task) { r.onHeartbeat(t.Data.(*pdRegionHeartbeatTask)) case worker.TaskTypePDStoreHeartbeat: r.onStoreHeartbeat(t.Data.(*pdStoreHeartbeatTask)) - case worker.TaskTypePDReportBatchSplit: - r.onReportBatchSplit(t.Data.(*pdReportBatchSplitTask)) - case worker.TaskTypePDDestroyPeer: - r.onDestroyPeer(t.Data.(*pdDestroyPeerTask)) default: log.Error("unsupported worker.Task type:", t.Tp) } } func (r *pdTaskHandler) start() { - r.pdClient.SetRegionHeartbeatResponseHandler(r.onRegionHeartbeatResponse) + r.pdClient.SetRegionHeartbeatResponseHandler(r.storeID, r.onRegionHeartbeatResponse) } - func (r *pdTaskHandler) onRegionHeartbeatResponse(resp *pdpb.RegionHeartbeatResponse) { if changePeer := resp.GetChangePeer(); changePeer != nil { r.sendAdminRequest(resp.RegionId, resp.RegionEpoch, resp.TargetPeer, &raft_cmdpb.AdminRequest{ @@ -92,7 +86,7 @@ func (r *pdTaskHandler) onAskBatchSplit(t *pdAskBatchSplitTask) { } func (r *pdTaskHandler) onHeartbeat(t *pdRegionHeartbeatTask) { - var size, keys int64 + var size int64 if t.approximateSize != nil { size = int64(*t.approximateSize) } @@ -103,9 +97,8 @@ func (r *pdTaskHandler) onHeartbeat(t *pdRegionHeartbeatTask) { DownPeers: t.downPeers, PendingPeers: t.pendingPeers, ApproximateSize: uint64(size), - ApproximateKeys: uint64(keys), } - r.pdClient.ReportRegion(req) + r.pdClient.RegionHeartbeat(req) } func (r *pdTaskHandler) onStoreHeartbeat(t *pdStoreHeartbeatTask) { @@ -133,14 +126,6 @@ func (r *pdTaskHandler) onStoreHeartbeat(t *pdStoreHeartbeatTask) { r.pdClient.StoreHeartbeat(context.TODO(), t.stats) } -func (r *pdTaskHandler) onReportBatchSplit(t *pdReportBatchSplitTask) { - r.pdClient.ReportBatchSplit(context.TODO(), t.regions) -} - -func (r *pdTaskHandler) onDestroyPeer(t *pdDestroyPeerTask) { - // TODO: delete it -} - func (r *pdTaskHandler) sendAdminRequest(regionID uint64, epoch *metapb.RegionEpoch, peer *metapb.Peer, req *raft_cmdpb.AdminRequest, callback *message.Callback) { cmd := &raft_cmdpb.RaftCmdRequest{ Header: &raft_cmdpb.RaftRequestHeader{ @@ -152,17 +137,3 @@ func (r *pdTaskHandler) sendAdminRequest(regionID uint64, epoch *metapb.RegionEp } r.router.SendRaftCommand(cmd, callback) } - -func (r *pdTaskHandler) sendDestroyPeer(local *metapb.Region, peer *metapb.Peer, pdRegion *metapb.Region) { - r.router.Send(local.GetId(), message.Msg{ - Type: message.MsgTypeRaftMessage, - RegionID: local.GetId(), - Data: &raft_serverpb.RaftMessage{ - RegionId: local.GetId(), - FromPeer: peer, - ToPeer: peer, - RegionEpoch: pdRegion.GetRegionEpoch(), - IsTombstone: true, - }, - }) -} diff --git a/kv/tikv/raftstore/worker.go b/kv/tikv/raftstore/worker.go index 360c771d1..922a3a16f 100644 --- a/kv/tikv/raftstore/worker.go +++ b/kv/tikv/raftstore/worker.go @@ -53,8 +53,6 @@ type pdRegionHeartbeatTask struct { peer *metapb.Peer downPeers []*pdpb.PeerStats pendingPeers []*metapb.Peer - writtenBytes uint64 - writtenKeys uint64 approximateSize *uint64 } @@ -65,14 +63,6 @@ type pdStoreHeartbeatTask struct { capacity uint64 } -type pdReportBatchSplitTask struct { - regions []*metapb.Region -} - -type pdDestroyPeerTask struct { - regionID uint64 -} - type splitCheckHandler struct { engine *badger.DB router *router diff --git a/kv/tikv/worker/worker.go b/kv/tikv/worker/worker.go index 07f8c67c5..1beda9184 100644 --- a/kv/tikv/worker/worker.go +++ b/kv/tikv/worker/worker.go @@ -9,11 +9,9 @@ const ( TaskTypeRaftLogGC TaskType = 1 TaskTypeSplitCheck TaskType = 2 - TaskTypePDAskBatchSplit TaskType = 102 - TaskTypePDHeartbeat TaskType = 103 - TaskTypePDStoreHeartbeat TaskType = 104 - TaskTypePDReportBatchSplit TaskType = 105 - TaskTypePDDestroyPeer TaskType = 108 + TaskTypePDAskBatchSplit TaskType = 102 + TaskTypePDHeartbeat TaskType = 103 + TaskTypePDStoreHeartbeat TaskType = 104 TaskTypeRegionGen TaskType = 401 TaskTypeRegionApply TaskType = 402 diff --git a/proto/pkg/pdpb/pdpb.pb.go b/proto/pkg/pdpb/pdpb.pb.go index 6f7c27c6d..7ca8b540f 100644 --- a/proto/pkg/pdpb/pdpb.pb.go +++ b/proto/pkg/pdpb/pdpb.pb.go @@ -67,7 +67,7 @@ func (x ErrorType) String() string { return proto.EnumName(ErrorType_name, int32(x)) } func (ErrorType) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_pdpb_5af6b72fd88b8384, []int{0} + return fileDescriptor_pdpb_4ddb592a4d17b00d, []int{0} } type CheckPolicy int32 @@ -93,7 +93,7 @@ func (x CheckPolicy) String() string { return proto.EnumName(CheckPolicy_name, int32(x)) } func (CheckPolicy) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_pdpb_5af6b72fd88b8384, []int{1} + return fileDescriptor_pdpb_4ddb592a4d17b00d, []int{1} } type OperatorStatus int32 @@ -125,7 +125,7 @@ func (x OperatorStatus) String() string { return proto.EnumName(OperatorStatus_name, int32(x)) } func (OperatorStatus) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_pdpb_5af6b72fd88b8384, []int{2} + return fileDescriptor_pdpb_4ddb592a4d17b00d, []int{2} } type RequestHeader struct { @@ -140,7 +140,7 @@ func (m *RequestHeader) Reset() { *m = RequestHeader{} } func (m *RequestHeader) String() string { return proto.CompactTextString(m) } func (*RequestHeader) ProtoMessage() {} func (*RequestHeader) Descriptor() ([]byte, []int) { - return fileDescriptor_pdpb_5af6b72fd88b8384, []int{0} + return fileDescriptor_pdpb_4ddb592a4d17b00d, []int{0} } func (m *RequestHeader) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -189,7 +189,7 @@ func (m *ResponseHeader) Reset() { *m = ResponseHeader{} } func (m *ResponseHeader) String() string { return proto.CompactTextString(m) } func (*ResponseHeader) ProtoMessage() {} func (*ResponseHeader) Descriptor() ([]byte, []int) { - return fileDescriptor_pdpb_5af6b72fd88b8384, []int{1} + return fileDescriptor_pdpb_4ddb592a4d17b00d, []int{1} } func (m *ResponseHeader) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -244,7 +244,7 @@ func (m *Error) Reset() { *m = Error{} } func (m *Error) String() string { return proto.CompactTextString(m) } func (*Error) ProtoMessage() {} func (*Error) Descriptor() ([]byte, []int) { - return fileDescriptor_pdpb_5af6b72fd88b8384, []int{2} + return fileDescriptor_pdpb_4ddb592a4d17b00d, []int{2} } func (m *Error) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -299,7 +299,7 @@ func (m *TsoRequest) Reset() { *m = TsoRequest{} } func (m *TsoRequest) String() string { return proto.CompactTextString(m) } func (*TsoRequest) ProtoMessage() {} func (*TsoRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_pdpb_5af6b72fd88b8384, []int{3} + return fileDescriptor_pdpb_4ddb592a4d17b00d, []int{3} } func (m *TsoRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -354,7 +354,7 @@ func (m *Timestamp) Reset() { *m = Timestamp{} } func (m *Timestamp) String() string { return proto.CompactTextString(m) } func (*Timestamp) ProtoMessage() {} func (*Timestamp) Descriptor() ([]byte, []int) { - return fileDescriptor_pdpb_5af6b72fd88b8384, []int{4} + return fileDescriptor_pdpb_4ddb592a4d17b00d, []int{4} } func (m *Timestamp) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -410,7 +410,7 @@ func (m *TsoResponse) Reset() { *m = TsoResponse{} } func (m *TsoResponse) String() string { return proto.CompactTextString(m) } func (*TsoResponse) ProtoMessage() {} func (*TsoResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_pdpb_5af6b72fd88b8384, []int{5} + return fileDescriptor_pdpb_4ddb592a4d17b00d, []int{5} } func (m *TsoResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -473,7 +473,7 @@ func (m *BootstrapRequest) Reset() { *m = BootstrapRequest{} } func (m *BootstrapRequest) String() string { return proto.CompactTextString(m) } func (*BootstrapRequest) ProtoMessage() {} func (*BootstrapRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_pdpb_5af6b72fd88b8384, []int{6} + return fileDescriptor_pdpb_4ddb592a4d17b00d, []int{6} } func (m *BootstrapRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -534,7 +534,7 @@ func (m *BootstrapResponse) Reset() { *m = BootstrapResponse{} } func (m *BootstrapResponse) String() string { return proto.CompactTextString(m) } func (*BootstrapResponse) ProtoMessage() {} func (*BootstrapResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_pdpb_5af6b72fd88b8384, []int{7} + return fileDescriptor_pdpb_4ddb592a4d17b00d, []int{7} } func (m *BootstrapResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -581,7 +581,7 @@ func (m *IsBootstrappedRequest) Reset() { *m = IsBootstrappedRequest{} } func (m *IsBootstrappedRequest) String() string { return proto.CompactTextString(m) } func (*IsBootstrappedRequest) ProtoMessage() {} func (*IsBootstrappedRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_pdpb_5af6b72fd88b8384, []int{8} + return fileDescriptor_pdpb_4ddb592a4d17b00d, []int{8} } func (m *IsBootstrappedRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -629,7 +629,7 @@ func (m *IsBootstrappedResponse) Reset() { *m = IsBootstrappedResponse{} func (m *IsBootstrappedResponse) String() string { return proto.CompactTextString(m) } func (*IsBootstrappedResponse) ProtoMessage() {} func (*IsBootstrappedResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_pdpb_5af6b72fd88b8384, []int{9} + return fileDescriptor_pdpb_4ddb592a4d17b00d, []int{9} } func (m *IsBootstrappedResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -683,7 +683,7 @@ func (m *AllocIDRequest) Reset() { *m = AllocIDRequest{} } func (m *AllocIDRequest) String() string { return proto.CompactTextString(m) } func (*AllocIDRequest) ProtoMessage() {} func (*AllocIDRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_pdpb_5af6b72fd88b8384, []int{10} + return fileDescriptor_pdpb_4ddb592a4d17b00d, []int{10} } func (m *AllocIDRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -731,7 +731,7 @@ func (m *AllocIDResponse) Reset() { *m = AllocIDResponse{} } func (m *AllocIDResponse) String() string { return proto.CompactTextString(m) } func (*AllocIDResponse) ProtoMessage() {} func (*AllocIDResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_pdpb_5af6b72fd88b8384, []int{11} + return fileDescriptor_pdpb_4ddb592a4d17b00d, []int{11} } func (m *AllocIDResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -786,7 +786,7 @@ func (m *GetStoreRequest) Reset() { *m = GetStoreRequest{} } func (m *GetStoreRequest) String() string { return proto.CompactTextString(m) } func (*GetStoreRequest) ProtoMessage() {} func (*GetStoreRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_pdpb_5af6b72fd88b8384, []int{12} + return fileDescriptor_pdpb_4ddb592a4d17b00d, []int{12} } func (m *GetStoreRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -842,7 +842,7 @@ func (m *GetStoreResponse) Reset() { *m = GetStoreResponse{} } func (m *GetStoreResponse) String() string { return proto.CompactTextString(m) } func (*GetStoreResponse) ProtoMessage() {} func (*GetStoreResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_pdpb_5af6b72fd88b8384, []int{13} + return fileDescriptor_pdpb_4ddb592a4d17b00d, []int{13} } func (m *GetStoreResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -904,7 +904,7 @@ func (m *PutStoreRequest) Reset() { *m = PutStoreRequest{} } func (m *PutStoreRequest) String() string { return proto.CompactTextString(m) } func (*PutStoreRequest) ProtoMessage() {} func (*PutStoreRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_pdpb_5af6b72fd88b8384, []int{14} + return fileDescriptor_pdpb_4ddb592a4d17b00d, []int{14} } func (m *PutStoreRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -958,7 +958,7 @@ func (m *PutStoreResponse) Reset() { *m = PutStoreResponse{} } func (m *PutStoreResponse) String() string { return proto.CompactTextString(m) } func (*PutStoreResponse) ProtoMessage() {} func (*PutStoreResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_pdpb_5af6b72fd88b8384, []int{15} + return fileDescriptor_pdpb_4ddb592a4d17b00d, []int{15} } func (m *PutStoreResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1007,7 +1007,7 @@ func (m *GetAllStoresRequest) Reset() { *m = GetAllStoresRequest{} } func (m *GetAllStoresRequest) String() string { return proto.CompactTextString(m) } func (*GetAllStoresRequest) ProtoMessage() {} func (*GetAllStoresRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_pdpb_5af6b72fd88b8384, []int{16} + return fileDescriptor_pdpb_4ddb592a4d17b00d, []int{16} } func (m *GetAllStoresRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1062,7 +1062,7 @@ func (m *GetAllStoresResponse) Reset() { *m = GetAllStoresResponse{} } func (m *GetAllStoresResponse) String() string { return proto.CompactTextString(m) } func (*GetAllStoresResponse) ProtoMessage() {} func (*GetAllStoresResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_pdpb_5af6b72fd88b8384, []int{17} + return fileDescriptor_pdpb_4ddb592a4d17b00d, []int{17} } func (m *GetAllStoresResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1117,7 +1117,7 @@ func (m *GetRegionRequest) Reset() { *m = GetRegionRequest{} } func (m *GetRegionRequest) String() string { return proto.CompactTextString(m) } func (*GetRegionRequest) ProtoMessage() {} func (*GetRegionRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_pdpb_5af6b72fd88b8384, []int{18} + return fileDescriptor_pdpb_4ddb592a4d17b00d, []int{18} } func (m *GetRegionRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1174,7 +1174,7 @@ func (m *GetRegionResponse) Reset() { *m = GetRegionResponse{} } func (m *GetRegionResponse) String() string { return proto.CompactTextString(m) } func (*GetRegionResponse) ProtoMessage() {} func (*GetRegionResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_pdpb_5af6b72fd88b8384, []int{19} + return fileDescriptor_pdpb_4ddb592a4d17b00d, []int{19} } func (m *GetRegionResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1243,7 +1243,7 @@ func (m *GetRegionByIDRequest) Reset() { *m = GetRegionByIDRequest{} } func (m *GetRegionByIDRequest) String() string { return proto.CompactTextString(m) } func (*GetRegionByIDRequest) ProtoMessage() {} func (*GetRegionByIDRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_pdpb_5af6b72fd88b8384, []int{20} + return fileDescriptor_pdpb_4ddb592a4d17b00d, []int{20} } func (m *GetRegionByIDRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1300,7 +1300,7 @@ func (m *ScanRegionsRequest) Reset() { *m = ScanRegionsRequest{} } func (m *ScanRegionsRequest) String() string { return proto.CompactTextString(m) } func (*ScanRegionsRequest) ProtoMessage() {} func (*ScanRegionsRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_pdpb_5af6b72fd88b8384, []int{21} + return fileDescriptor_pdpb_4ddb592a4d17b00d, []int{21} } func (m *ScanRegionsRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1370,7 +1370,7 @@ func (m *ScanRegionsResponse) Reset() { *m = ScanRegionsResponse{} } func (m *ScanRegionsResponse) String() string { return proto.CompactTextString(m) } func (*ScanRegionsResponse) ProtoMessage() {} func (*ScanRegionsResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_pdpb_5af6b72fd88b8384, []int{22} + return fileDescriptor_pdpb_4ddb592a4d17b00d, []int{22} } func (m *ScanRegionsResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1431,7 +1431,7 @@ func (m *GetClusterConfigRequest) Reset() { *m = GetClusterConfigRequest func (m *GetClusterConfigRequest) String() string { return proto.CompactTextString(m) } func (*GetClusterConfigRequest) ProtoMessage() {} func (*GetClusterConfigRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_pdpb_5af6b72fd88b8384, []int{23} + return fileDescriptor_pdpb_4ddb592a4d17b00d, []int{23} } func (m *GetClusterConfigRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1479,7 +1479,7 @@ func (m *GetClusterConfigResponse) Reset() { *m = GetClusterConfigRespon func (m *GetClusterConfigResponse) String() string { return proto.CompactTextString(m) } func (*GetClusterConfigResponse) ProtoMessage() {} func (*GetClusterConfigResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_pdpb_5af6b72fd88b8384, []int{24} + return fileDescriptor_pdpb_4ddb592a4d17b00d, []int{24} } func (m *GetClusterConfigResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1534,7 +1534,7 @@ func (m *PutClusterConfigRequest) Reset() { *m = PutClusterConfigRequest func (m *PutClusterConfigRequest) String() string { return proto.CompactTextString(m) } func (*PutClusterConfigRequest) ProtoMessage() {} func (*PutClusterConfigRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_pdpb_5af6b72fd88b8384, []int{25} + return fileDescriptor_pdpb_4ddb592a4d17b00d, []int{25} } func (m *PutClusterConfigRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1588,7 +1588,7 @@ func (m *PutClusterConfigResponse) Reset() { *m = PutClusterConfigRespon func (m *PutClusterConfigResponse) String() string { return proto.CompactTextString(m) } func (*PutClusterConfigResponse) ProtoMessage() {} func (*PutClusterConfigResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_pdpb_5af6b72fd88b8384, []int{26} + return fileDescriptor_pdpb_4ddb592a4d17b00d, []int{26} } func (m *PutClusterConfigResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1641,7 +1641,7 @@ func (m *Member) Reset() { *m = Member{} } func (m *Member) String() string { return proto.CompactTextString(m) } func (*Member) ProtoMessage() {} func (*Member) Descriptor() ([]byte, []int) { - return fileDescriptor_pdpb_5af6b72fd88b8384, []int{27} + return fileDescriptor_pdpb_4ddb592a4d17b00d, []int{27} } func (m *Member) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1716,7 +1716,7 @@ func (m *GetMembersRequest) Reset() { *m = GetMembersRequest{} } func (m *GetMembersRequest) String() string { return proto.CompactTextString(m) } func (*GetMembersRequest) ProtoMessage() {} func (*GetMembersRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_pdpb_5af6b72fd88b8384, []int{28} + return fileDescriptor_pdpb_4ddb592a4d17b00d, []int{28} } func (m *GetMembersRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1766,7 +1766,7 @@ func (m *GetMembersResponse) Reset() { *m = GetMembersResponse{} } func (m *GetMembersResponse) String() string { return proto.CompactTextString(m) } func (*GetMembersResponse) ProtoMessage() {} func (*GetMembersResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_pdpb_5af6b72fd88b8384, []int{29} + return fileDescriptor_pdpb_4ddb592a4d17b00d, []int{29} } func (m *GetMembersResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1835,7 +1835,7 @@ func (m *PeerStats) Reset() { *m = PeerStats{} } func (m *PeerStats) String() string { return proto.CompactTextString(m) } func (*PeerStats) ProtoMessage() {} func (*PeerStats) Descriptor() ([]byte, []int) { - return fileDescriptor_pdpb_5af6b72fd88b8384, []int{30} + return fileDescriptor_pdpb_4ddb592a4d17b00d, []int{30} } func (m *PeerStats) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1911,7 +1911,7 @@ func (m *RegionHeartbeatRequest) Reset() { *m = RegionHeartbeatRequest{} func (m *RegionHeartbeatRequest) String() string { return proto.CompactTextString(m) } func (*RegionHeartbeatRequest) ProtoMessage() {} func (*RegionHeartbeatRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_pdpb_5af6b72fd88b8384, []int{31} + return fileDescriptor_pdpb_4ddb592a4d17b00d, []int{31} } func (m *RegionHeartbeatRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2043,7 +2043,7 @@ func (m *ChangePeer) Reset() { *m = ChangePeer{} } func (m *ChangePeer) String() string { return proto.CompactTextString(m) } func (*ChangePeer) ProtoMessage() {} func (*ChangePeer) Descriptor() ([]byte, []int) { - return fileDescriptor_pdpb_5af6b72fd88b8384, []int{32} + return fileDescriptor_pdpb_4ddb592a4d17b00d, []int{32} } func (m *ChangePeer) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2097,7 +2097,7 @@ func (m *TransferLeader) Reset() { *m = TransferLeader{} } func (m *TransferLeader) String() string { return proto.CompactTextString(m) } func (*TransferLeader) ProtoMessage() {} func (*TransferLeader) Descriptor() ([]byte, []int) { - return fileDescriptor_pdpb_5af6b72fd88b8384, []int{33} + return fileDescriptor_pdpb_4ddb592a4d17b00d, []int{33} } func (m *TransferLeader) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2144,7 +2144,7 @@ func (m *Merge) Reset() { *m = Merge{} } func (m *Merge) String() string { return proto.CompactTextString(m) } func (*Merge) ProtoMessage() {} func (*Merge) Descriptor() ([]byte, []int) { - return fileDescriptor_pdpb_5af6b72fd88b8384, []int{34} + return fileDescriptor_pdpb_4ddb592a4d17b00d, []int{34} } func (m *Merge) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2192,7 +2192,7 @@ func (m *SplitRegion) Reset() { *m = SplitRegion{} } func (m *SplitRegion) String() string { return proto.CompactTextString(m) } func (*SplitRegion) ProtoMessage() {} func (*SplitRegion) Descriptor() ([]byte, []int) { - return fileDescriptor_pdpb_5af6b72fd88b8384, []int{35} + return fileDescriptor_pdpb_4ddb592a4d17b00d, []int{35} } func (m *SplitRegion) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2271,7 +2271,7 @@ func (m *RegionHeartbeatResponse) Reset() { *m = RegionHeartbeatResponse func (m *RegionHeartbeatResponse) String() string { return proto.CompactTextString(m) } func (*RegionHeartbeatResponse) ProtoMessage() {} func (*RegionHeartbeatResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_pdpb_5af6b72fd88b8384, []int{36} + return fileDescriptor_pdpb_4ddb592a4d17b00d, []int{36} } func (m *RegionHeartbeatResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2368,7 +2368,7 @@ func (m *AskSplitRequest) Reset() { *m = AskSplitRequest{} } func (m *AskSplitRequest) String() string { return proto.CompactTextString(m) } func (*AskSplitRequest) ProtoMessage() {} func (*AskSplitRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_pdpb_5af6b72fd88b8384, []int{37} + return fileDescriptor_pdpb_4ddb592a4d17b00d, []int{37} } func (m *AskSplitRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2428,7 +2428,7 @@ func (m *AskSplitResponse) Reset() { *m = AskSplitResponse{} } func (m *AskSplitResponse) String() string { return proto.CompactTextString(m) } func (*AskSplitResponse) ProtoMessage() {} func (*AskSplitResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_pdpb_5af6b72fd88b8384, []int{38} + return fileDescriptor_pdpb_4ddb592a4d17b00d, []int{38} } func (m *AskSplitResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2491,7 +2491,7 @@ func (m *ReportSplitRequest) Reset() { *m = ReportSplitRequest{} } func (m *ReportSplitRequest) String() string { return proto.CompactTextString(m) } func (*ReportSplitRequest) ProtoMessage() {} func (*ReportSplitRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_pdpb_5af6b72fd88b8384, []int{39} + return fileDescriptor_pdpb_4ddb592a4d17b00d, []int{39} } func (m *ReportSplitRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2552,7 +2552,7 @@ func (m *ReportSplitResponse) Reset() { *m = ReportSplitResponse{} } func (m *ReportSplitResponse) String() string { return proto.CompactTextString(m) } func (*ReportSplitResponse) ProtoMessage() {} func (*ReportSplitResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_pdpb_5af6b72fd88b8384, []int{40} + return fileDescriptor_pdpb_4ddb592a4d17b00d, []int{40} } func (m *ReportSplitResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2601,7 +2601,7 @@ func (m *AskBatchSplitRequest) Reset() { *m = AskBatchSplitRequest{} } func (m *AskBatchSplitRequest) String() string { return proto.CompactTextString(m) } func (*AskBatchSplitRequest) ProtoMessage() {} func (*AskBatchSplitRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_pdpb_5af6b72fd88b8384, []int{41} + return fileDescriptor_pdpb_4ddb592a4d17b00d, []int{41} } func (m *AskBatchSplitRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2663,7 +2663,7 @@ func (m *SplitID) Reset() { *m = SplitID{} } func (m *SplitID) String() string { return proto.CompactTextString(m) } func (*SplitID) ProtoMessage() {} func (*SplitID) Descriptor() ([]byte, []int) { - return fileDescriptor_pdpb_5af6b72fd88b8384, []int{42} + return fileDescriptor_pdpb_4ddb592a4d17b00d, []int{42} } func (m *SplitID) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2718,7 +2718,7 @@ func (m *AskBatchSplitResponse) Reset() { *m = AskBatchSplitResponse{} } func (m *AskBatchSplitResponse) String() string { return proto.CompactTextString(m) } func (*AskBatchSplitResponse) ProtoMessage() {} func (*AskBatchSplitResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_pdpb_5af6b72fd88b8384, []int{43} + return fileDescriptor_pdpb_4ddb592a4d17b00d, []int{43} } func (m *AskBatchSplitResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2773,7 +2773,7 @@ func (m *ReportBatchSplitRequest) Reset() { *m = ReportBatchSplitRequest func (m *ReportBatchSplitRequest) String() string { return proto.CompactTextString(m) } func (*ReportBatchSplitRequest) ProtoMessage() {} func (*ReportBatchSplitRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_pdpb_5af6b72fd88b8384, []int{44} + return fileDescriptor_pdpb_4ddb592a4d17b00d, []int{44} } func (m *ReportBatchSplitRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2827,7 +2827,7 @@ func (m *ReportBatchSplitResponse) Reset() { *m = ReportBatchSplitRespon func (m *ReportBatchSplitResponse) String() string { return proto.CompactTextString(m) } func (*ReportBatchSplitResponse) ProtoMessage() {} func (*ReportBatchSplitResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_pdpb_5af6b72fd88b8384, []int{45} + return fileDescriptor_pdpb_4ddb592a4d17b00d, []int{45} } func (m *ReportBatchSplitResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2877,7 +2877,7 @@ func (m *TimeInterval) Reset() { *m = TimeInterval{} } func (m *TimeInterval) String() string { return proto.CompactTextString(m) } func (*TimeInterval) ProtoMessage() {} func (*TimeInterval) Descriptor() ([]byte, []int) { - return fileDescriptor_pdpb_5af6b72fd88b8384, []int{46} + return fileDescriptor_pdpb_4ddb592a4d17b00d, []int{46} } func (m *TimeInterval) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2932,7 +2932,7 @@ func (m *RecordPair) Reset() { *m = RecordPair{} } func (m *RecordPair) String() string { return proto.CompactTextString(m) } func (*RecordPair) ProtoMessage() {} func (*RecordPair) Descriptor() ([]byte, []int) { - return fileDescriptor_pdpb_5af6b72fd88b8384, []int{47} + return fileDescriptor_pdpb_4ddb592a4d17b00d, []int{47} } func (m *RecordPair) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3022,7 +3022,7 @@ func (m *StoreStats) Reset() { *m = StoreStats{} } func (m *StoreStats) String() string { return proto.CompactTextString(m) } func (*StoreStats) ProtoMessage() {} func (*StoreStats) Descriptor() ([]byte, []int) { - return fileDescriptor_pdpb_5af6b72fd88b8384, []int{48} + return fileDescriptor_pdpb_4ddb592a4d17b00d, []int{48} } func (m *StoreStats) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3196,7 +3196,7 @@ func (m *StoreHeartbeatRequest) Reset() { *m = StoreHeartbeatRequest{} } func (m *StoreHeartbeatRequest) String() string { return proto.CompactTextString(m) } func (*StoreHeartbeatRequest) ProtoMessage() {} func (*StoreHeartbeatRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_pdpb_5af6b72fd88b8384, []int{49} + return fileDescriptor_pdpb_4ddb592a4d17b00d, []int{49} } func (m *StoreHeartbeatRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3250,7 +3250,7 @@ func (m *StoreHeartbeatResponse) Reset() { *m = StoreHeartbeatResponse{} func (m *StoreHeartbeatResponse) String() string { return proto.CompactTextString(m) } func (*StoreHeartbeatResponse) ProtoMessage() {} func (*StoreHeartbeatResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_pdpb_5af6b72fd88b8384, []int{50} + return fileDescriptor_pdpb_4ddb592a4d17b00d, []int{50} } func (m *StoreHeartbeatResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3302,7 +3302,7 @@ func (m *ScatterRegionRequest) Reset() { *m = ScatterRegionRequest{} } func (m *ScatterRegionRequest) String() string { return proto.CompactTextString(m) } func (*ScatterRegionRequest) ProtoMessage() {} func (*ScatterRegionRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_pdpb_5af6b72fd88b8384, []int{51} + return fileDescriptor_pdpb_4ddb592a4d17b00d, []int{51} } func (m *ScatterRegionRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3370,7 +3370,7 @@ func (m *ScatterRegionResponse) Reset() { *m = ScatterRegionResponse{} } func (m *ScatterRegionResponse) String() string { return proto.CompactTextString(m) } func (*ScatterRegionResponse) ProtoMessage() {} func (*ScatterRegionResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_pdpb_5af6b72fd88b8384, []int{52} + return fileDescriptor_pdpb_4ddb592a4d17b00d, []int{52} } func (m *ScatterRegionResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3417,7 +3417,7 @@ func (m *GetGCSafePointRequest) Reset() { *m = GetGCSafePointRequest{} } func (m *GetGCSafePointRequest) String() string { return proto.CompactTextString(m) } func (*GetGCSafePointRequest) ProtoMessage() {} func (*GetGCSafePointRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_pdpb_5af6b72fd88b8384, []int{53} + return fileDescriptor_pdpb_4ddb592a4d17b00d, []int{53} } func (m *GetGCSafePointRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3465,7 +3465,7 @@ func (m *GetGCSafePointResponse) Reset() { *m = GetGCSafePointResponse{} func (m *GetGCSafePointResponse) String() string { return proto.CompactTextString(m) } func (*GetGCSafePointResponse) ProtoMessage() {} func (*GetGCSafePointResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_pdpb_5af6b72fd88b8384, []int{54} + return fileDescriptor_pdpb_4ddb592a4d17b00d, []int{54} } func (m *GetGCSafePointResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3520,7 +3520,7 @@ func (m *UpdateGCSafePointRequest) Reset() { *m = UpdateGCSafePointReque func (m *UpdateGCSafePointRequest) String() string { return proto.CompactTextString(m) } func (*UpdateGCSafePointRequest) ProtoMessage() {} func (*UpdateGCSafePointRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_pdpb_5af6b72fd88b8384, []int{55} + return fileDescriptor_pdpb_4ddb592a4d17b00d, []int{55} } func (m *UpdateGCSafePointRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3575,7 +3575,7 @@ func (m *UpdateGCSafePointResponse) Reset() { *m = UpdateGCSafePointResp func (m *UpdateGCSafePointResponse) String() string { return proto.CompactTextString(m) } func (*UpdateGCSafePointResponse) ProtoMessage() {} func (*UpdateGCSafePointResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_pdpb_5af6b72fd88b8384, []int{56} + return fileDescriptor_pdpb_4ddb592a4d17b00d, []int{56} } func (m *UpdateGCSafePointResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3633,7 +3633,7 @@ func (m *SyncRegionRequest) Reset() { *m = SyncRegionRequest{} } func (m *SyncRegionRequest) String() string { return proto.CompactTextString(m) } func (*SyncRegionRequest) ProtoMessage() {} func (*SyncRegionRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_pdpb_5af6b72fd88b8384, []int{57} + return fileDescriptor_pdpb_4ddb592a4d17b00d, []int{57} } func (m *SyncRegionRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3698,7 +3698,7 @@ func (m *SyncRegionResponse) Reset() { *m = SyncRegionResponse{} } func (m *SyncRegionResponse) String() string { return proto.CompactTextString(m) } func (*SyncRegionResponse) ProtoMessage() {} func (*SyncRegionResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_pdpb_5af6b72fd88b8384, []int{58} + return fileDescriptor_pdpb_4ddb592a4d17b00d, []int{58} } func (m *SyncRegionResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3760,7 +3760,7 @@ func (m *GetOperatorRequest) Reset() { *m = GetOperatorRequest{} } func (m *GetOperatorRequest) String() string { return proto.CompactTextString(m) } func (*GetOperatorRequest) ProtoMessage() {} func (*GetOperatorRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_pdpb_5af6b72fd88b8384, []int{59} + return fileDescriptor_pdpb_4ddb592a4d17b00d, []int{59} } func (m *GetOperatorRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3818,7 +3818,7 @@ func (m *GetOperatorResponse) Reset() { *m = GetOperatorResponse{} } func (m *GetOperatorResponse) String() string { return proto.CompactTextString(m) } func (*GetOperatorResponse) ProtoMessage() {} func (*GetOperatorResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_pdpb_5af6b72fd88b8384, []int{60} + return fileDescriptor_pdpb_4ddb592a4d17b00d, []int{60} } func (m *GetOperatorResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3976,10 +3976,7 @@ type PDClient interface { GetPrevRegion(ctx context.Context, in *GetRegionRequest, opts ...grpc.CallOption) (*GetRegionResponse, error) GetRegionByID(ctx context.Context, in *GetRegionByIDRequest, opts ...grpc.CallOption) (*GetRegionResponse, error) ScanRegions(ctx context.Context, in *ScanRegionsRequest, opts ...grpc.CallOption) (*ScanRegionsResponse, error) - AskSplit(ctx context.Context, in *AskSplitRequest, opts ...grpc.CallOption) (*AskSplitResponse, error) - ReportSplit(ctx context.Context, in *ReportSplitRequest, opts ...grpc.CallOption) (*ReportSplitResponse, error) AskBatchSplit(ctx context.Context, in *AskBatchSplitRequest, opts ...grpc.CallOption) (*AskBatchSplitResponse, error) - ReportBatchSplit(ctx context.Context, in *ReportBatchSplitRequest, opts ...grpc.CallOption) (*ReportBatchSplitResponse, error) GetClusterConfig(ctx context.Context, in *GetClusterConfigRequest, opts ...grpc.CallOption) (*GetClusterConfigResponse, error) PutClusterConfig(ctx context.Context, in *PutClusterConfigRequest, opts ...grpc.CallOption) (*PutClusterConfigResponse, error) ScatterRegion(ctx context.Context, in *ScatterRegionRequest, opts ...grpc.CallOption) (*ScatterRegionResponse, error) @@ -4167,26 +4164,6 @@ func (c *pDClient) ScanRegions(ctx context.Context, in *ScanRegionsRequest, opts return out, nil } -// Deprecated: Do not use. -func (c *pDClient) AskSplit(ctx context.Context, in *AskSplitRequest, opts ...grpc.CallOption) (*AskSplitResponse, error) { - out := new(AskSplitResponse) - err := c.cc.Invoke(ctx, "/pdpb.PD/AskSplit", in, out, opts...) - if err != nil { - return nil, err - } - return out, nil -} - -// Deprecated: Do not use. -func (c *pDClient) ReportSplit(ctx context.Context, in *ReportSplitRequest, opts ...grpc.CallOption) (*ReportSplitResponse, error) { - out := new(ReportSplitResponse) - err := c.cc.Invoke(ctx, "/pdpb.PD/ReportSplit", in, out, opts...) - if err != nil { - return nil, err - } - return out, nil -} - func (c *pDClient) AskBatchSplit(ctx context.Context, in *AskBatchSplitRequest, opts ...grpc.CallOption) (*AskBatchSplitResponse, error) { out := new(AskBatchSplitResponse) err := c.cc.Invoke(ctx, "/pdpb.PD/AskBatchSplit", in, out, opts...) @@ -4196,15 +4173,6 @@ func (c *pDClient) AskBatchSplit(ctx context.Context, in *AskBatchSplitRequest, return out, nil } -func (c *pDClient) ReportBatchSplit(ctx context.Context, in *ReportBatchSplitRequest, opts ...grpc.CallOption) (*ReportBatchSplitResponse, error) { - out := new(ReportBatchSplitResponse) - err := c.cc.Invoke(ctx, "/pdpb.PD/ReportBatchSplit", in, out, opts...) - if err != nil { - return nil, err - } - return out, nil -} - func (c *pDClient) GetClusterConfig(ctx context.Context, in *GetClusterConfigRequest, opts ...grpc.CallOption) (*GetClusterConfigResponse, error) { out := new(GetClusterConfigResponse) err := c.cc.Invoke(ctx, "/pdpb.PD/GetClusterConfig", in, out, opts...) @@ -4309,10 +4277,7 @@ type PDServer interface { GetPrevRegion(context.Context, *GetRegionRequest) (*GetRegionResponse, error) GetRegionByID(context.Context, *GetRegionByIDRequest) (*GetRegionResponse, error) ScanRegions(context.Context, *ScanRegionsRequest) (*ScanRegionsResponse, error) - AskSplit(context.Context, *AskSplitRequest) (*AskSplitResponse, error) - ReportSplit(context.Context, *ReportSplitRequest) (*ReportSplitResponse, error) AskBatchSplit(context.Context, *AskBatchSplitRequest) (*AskBatchSplitResponse, error) - ReportBatchSplit(context.Context, *ReportBatchSplitRequest) (*ReportBatchSplitResponse, error) GetClusterConfig(context.Context, *GetClusterConfigRequest) (*GetClusterConfigResponse, error) PutClusterConfig(context.Context, *PutClusterConfigRequest) (*PutClusterConfigResponse, error) ScatterRegion(context.Context, *ScatterRegionRequest) (*ScatterRegionResponse, error) @@ -4594,42 +4559,6 @@ func _PD_ScanRegions_Handler(srv interface{}, ctx context.Context, dec func(inte return interceptor(ctx, in, info, handler) } -func _PD_AskSplit_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { - in := new(AskSplitRequest) - if err := dec(in); err != nil { - return nil, err - } - if interceptor == nil { - return srv.(PDServer).AskSplit(ctx, in) - } - info := &grpc.UnaryServerInfo{ - Server: srv, - FullMethod: "/pdpb.PD/AskSplit", - } - handler := func(ctx context.Context, req interface{}) (interface{}, error) { - return srv.(PDServer).AskSplit(ctx, req.(*AskSplitRequest)) - } - return interceptor(ctx, in, info, handler) -} - -func _PD_ReportSplit_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { - in := new(ReportSplitRequest) - if err := dec(in); err != nil { - return nil, err - } - if interceptor == nil { - return srv.(PDServer).ReportSplit(ctx, in) - } - info := &grpc.UnaryServerInfo{ - Server: srv, - FullMethod: "/pdpb.PD/ReportSplit", - } - handler := func(ctx context.Context, req interface{}) (interface{}, error) { - return srv.(PDServer).ReportSplit(ctx, req.(*ReportSplitRequest)) - } - return interceptor(ctx, in, info, handler) -} - func _PD_AskBatchSplit_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { in := new(AskBatchSplitRequest) if err := dec(in); err != nil { @@ -4648,24 +4577,6 @@ func _PD_AskBatchSplit_Handler(srv interface{}, ctx context.Context, dec func(in return interceptor(ctx, in, info, handler) } -func _PD_ReportBatchSplit_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { - in := new(ReportBatchSplitRequest) - if err := dec(in); err != nil { - return nil, err - } - if interceptor == nil { - return srv.(PDServer).ReportBatchSplit(ctx, in) - } - info := &grpc.UnaryServerInfo{ - Server: srv, - FullMethod: "/pdpb.PD/ReportBatchSplit", - } - handler := func(ctx context.Context, req interface{}) (interface{}, error) { - return srv.(PDServer).ReportBatchSplit(ctx, req.(*ReportBatchSplitRequest)) - } - return interceptor(ctx, in, info, handler) -} - func _PD_GetClusterConfig_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { in := new(GetClusterConfigRequest) if err := dec(in); err != nil { @@ -4852,22 +4763,10 @@ var _PD_serviceDesc = grpc.ServiceDesc{ MethodName: "ScanRegions", Handler: _PD_ScanRegions_Handler, }, - { - MethodName: "AskSplit", - Handler: _PD_AskSplit_Handler, - }, - { - MethodName: "ReportSplit", - Handler: _PD_ReportSplit_Handler, - }, { MethodName: "AskBatchSplit", Handler: _PD_AskBatchSplit_Handler, }, - { - MethodName: "ReportBatchSplit", - Handler: _PD_ReportBatchSplit_Handler, - }, { MethodName: "GetClusterConfig", Handler: _PD_GetClusterConfig_Handler, @@ -16672,184 +16571,181 @@ var ( ErrIntOverflowPdpb = fmt.Errorf("proto: integer overflow") ) -func init() { proto.RegisterFile("pdpb.proto", fileDescriptor_pdpb_5af6b72fd88b8384) } +func init() { proto.RegisterFile("pdpb.proto", fileDescriptor_pdpb_4ddb592a4d17b00d) } -var fileDescriptor_pdpb_5af6b72fd88b8384 = []byte{ - // 2815 bytes of a gzipped FileDescriptorProto +var fileDescriptor_pdpb_4ddb592a4d17b00d = []byte{ + // 2763 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xb4, 0x3a, 0xcd, 0x72, 0x1b, 0xc7, - 0xd1, 0x5c, 0x10, 0x00, 0x81, 0xc6, 0x2f, 0x87, 0x14, 0x09, 0x41, 0x96, 0x2c, 0xaf, 0xf4, 0xf9, - 0x93, 0x1d, 0x9b, 0xb6, 0x65, 0x95, 0xcb, 0x55, 0x29, 0xbb, 0x0c, 0x82, 0x10, 0x0d, 0x8b, 0x04, - 0x50, 0x03, 0xd0, 0x8e, 0xab, 0x52, 0x46, 0x96, 0xbb, 0x43, 0x70, 0x43, 0x70, 0x77, 0xbd, 0xbb, - 0xa0, 0x0c, 0x57, 0x0e, 0x39, 0x25, 0xa9, 0x8a, 0x73, 0xc8, 0xc1, 0xa9, 0x54, 0x4e, 0x79, 0x82, - 0xdc, 0x92, 0x6b, 0xae, 0x39, 0xe6, 0x11, 0x52, 0xce, 0x13, 0xe4, 0x0d, 0x52, 0xf3, 0xb7, 0x7f, - 0x00, 0x24, 0x66, 0x65, 0x9f, 0x88, 0xed, 0xee, 0xe9, 0xe9, 0xbf, 0xe9, 0xe9, 0xe9, 0x26, 0x80, - 0x63, 0x38, 0xa7, 0x7b, 0x8e, 0x6b, 0xfb, 0x36, 0xca, 0xd2, 0xdf, 0xcd, 0xf2, 0x25, 0xf1, 0x35, - 0x09, 0x6b, 0x56, 0x88, 0xab, 0x9d, 0xf9, 0xc1, 0xe7, 0xf6, 0xc4, 0x9e, 0xd8, 0xec, 0xe7, 0x5b, - 0xf4, 0x17, 0x87, 0xaa, 0x7b, 0x50, 0xc1, 0xe4, 0xcb, 0x19, 0xf1, 0xfc, 0x8f, 0x89, 0x66, 0x10, - 0x17, 0xdd, 0x06, 0xd0, 0xa7, 0x33, 0xcf, 0x27, 0xee, 0xd8, 0x34, 0x1a, 0xca, 0x5d, 0xe5, 0x41, - 0x16, 0x17, 0x05, 0xa4, 0x6b, 0xa8, 0x18, 0xaa, 0x98, 0x78, 0x8e, 0x6d, 0x79, 0xe4, 0x5a, 0x0b, - 0xd0, 0x2b, 0x90, 0x23, 0xae, 0x6b, 0xbb, 0x8d, 0xcc, 0x5d, 0xe5, 0x41, 0xe9, 0x61, 0x69, 0x8f, - 0x49, 0xdd, 0xa1, 0x20, 0xcc, 0x31, 0xea, 0x63, 0xc8, 0xb1, 0x6f, 0x74, 0x0f, 0xb2, 0xfe, 0xdc, - 0x21, 0x8c, 0x49, 0xf5, 0x61, 0x2d, 0x42, 0x3a, 0x9a, 0x3b, 0x04, 0x33, 0x24, 0x6a, 0xc0, 0xc6, - 0x25, 0xf1, 0x3c, 0x6d, 0x42, 0x18, 0xcb, 0x22, 0x96, 0x9f, 0x6a, 0x1f, 0x60, 0xe4, 0xd9, 0x42, - 0x1d, 0xf4, 0x23, 0xc8, 0x9f, 0x33, 0x09, 0x19, 0xbb, 0xd2, 0xc3, 0x2d, 0xce, 0x2e, 0xa6, 0x2d, - 0x16, 0x24, 0x68, 0x1b, 0x72, 0xba, 0x3d, 0xb3, 0x7c, 0xc6, 0xb2, 0x82, 0xf9, 0x87, 0xda, 0x82, - 0xe2, 0xc8, 0xbc, 0x24, 0x9e, 0xaf, 0x5d, 0x3a, 0xa8, 0x09, 0x05, 0xe7, 0x7c, 0xee, 0x99, 0xba, - 0x36, 0x65, 0x1c, 0xd7, 0x71, 0xf0, 0x4d, 0x65, 0x9a, 0xda, 0x13, 0x86, 0xca, 0x30, 0x94, 0xfc, - 0x54, 0x7f, 0xa9, 0x40, 0x89, 0x09, 0xc5, 0x6d, 0x86, 0xde, 0x48, 0x48, 0xb5, 0x2d, 0xa5, 0x8a, - 0xda, 0xf4, 0xd9, 0x62, 0xa1, 0x37, 0xa1, 0xe8, 0x4b, 0xb1, 0x1a, 0xeb, 0x8c, 0x8d, 0xb0, 0x55, - 0x20, 0x2d, 0x0e, 0x29, 0xd4, 0x6f, 0x14, 0xa8, 0xef, 0xdb, 0xb6, 0xef, 0xf9, 0xae, 0xe6, 0xa4, - 0xb2, 0xce, 0x3d, 0xc8, 0x79, 0xbe, 0xed, 0x12, 0xe1, 0xc3, 0xca, 0x9e, 0x88, 0xb3, 0x21, 0x05, - 0x62, 0x8e, 0x43, 0xaf, 0x42, 0xde, 0x25, 0x13, 0xd3, 0xb6, 0x84, 0x48, 0x55, 0x49, 0x85, 0x19, - 0x14, 0x0b, 0xac, 0xda, 0x82, 0xcd, 0x88, 0x34, 0x69, 0xcc, 0xa2, 0x1e, 0xc0, 0x8d, 0xae, 0x17, - 0x30, 0x71, 0x88, 0x91, 0x46, 0x2b, 0xf5, 0xe7, 0xb0, 0x93, 0xe4, 0x92, 0xca, 0x49, 0x2a, 0x94, - 0x4f, 0x23, 0x5c, 0x98, 0x91, 0x0a, 0x38, 0x06, 0x53, 0x3f, 0x80, 0x6a, 0x6b, 0x3a, 0xb5, 0xf5, - 0xee, 0x41, 0x2a, 0x51, 0xfb, 0x50, 0x0b, 0x96, 0xa7, 0x92, 0xb1, 0x0a, 0x19, 0x93, 0x4b, 0x96, - 0xc5, 0x19, 0xd3, 0x50, 0x3f, 0x87, 0xda, 0x21, 0xf1, 0xb9, 0xff, 0xd2, 0x44, 0xc4, 0x4d, 0x28, - 0x30, 0xaf, 0x8f, 0x03, 0xae, 0x1b, 0xec, 0xbb, 0x6b, 0xa8, 0xbf, 0x53, 0xa0, 0x1e, 0xf2, 0x4e, - 0x25, 0xed, 0x35, 0xe3, 0x2d, 0xe7, 0xf9, 0x9a, 0xef, 0x89, 0x70, 0xab, 0x73, 0x8e, 0x8c, 0x64, - 0x48, 0xe1, 0x98, 0xa3, 0x55, 0x1d, 0x6a, 0x83, 0xd9, 0x0b, 0xa8, 0x7a, 0x1d, 0x61, 0xd4, 0x8f, - 0xa0, 0x1e, 0x6e, 0x92, 0x2a, 0xa6, 0x7f, 0x01, 0x5b, 0x87, 0xc4, 0x6f, 0x4d, 0xa7, 0x8c, 0x89, - 0x97, 0x4a, 0xd4, 0xf7, 0xa1, 0x41, 0xbe, 0xd2, 0xa7, 0x33, 0x83, 0x8c, 0x7d, 0xfb, 0xf2, 0xd4, - 0xf3, 0x6d, 0x8b, 0x8c, 0x99, 0x80, 0x9e, 0x88, 0xca, 0x1d, 0x81, 0x1f, 0x49, 0x34, 0xdf, 0x4d, - 0xbd, 0x80, 0xed, 0xf8, 0xee, 0xa9, 0xfc, 0xf6, 0x7f, 0x90, 0x0f, 0x76, 0x5b, 0x5f, 0xb4, 0x95, - 0x40, 0xaa, 0x5f, 0xb0, 0x00, 0x11, 0x69, 0x21, 0x8d, 0x9e, 0xb7, 0x01, 0x78, 0x32, 0x19, 0x5f, - 0x90, 0x39, 0xd3, 0xac, 0x8c, 0x8b, 0x1c, 0xf2, 0x84, 0xcc, 0xd5, 0xbf, 0x2a, 0xb0, 0x19, 0xd9, - 0x20, 0x95, 0x2a, 0x61, 0x36, 0xcb, 0x3c, 0x2b, 0x9b, 0xa1, 0xfb, 0x90, 0x9f, 0x72, 0xae, 0x3c, - 0x0c, 0xcb, 0x92, 0x6e, 0x40, 0x28, 0x37, 0x8e, 0xa3, 0x54, 0xde, 0x54, 0xbb, 0x22, 0x5e, 0x23, - 0xcb, 0x0c, 0x93, 0xa0, 0xe2, 0x38, 0xf5, 0x67, 0xcc, 0x09, 0x7c, 0x83, 0xfd, 0x79, 0xba, 0x54, - 0x81, 0x6e, 0x81, 0xb0, 0x44, 0x78, 0x34, 0x0b, 0x1c, 0xc0, 0xcf, 0x26, 0x1a, 0xea, 0x9a, 0xc5, - 0xf7, 0xf0, 0xd2, 0x6e, 0xe0, 0xf9, 0x9a, 0xeb, 0x47, 0x6c, 0x5f, 0x60, 0x80, 0x27, 0x64, 0x4e, - 0x2f, 0xac, 0xa9, 0x79, 0x69, 0xfa, 0xcc, 0x1a, 0x39, 0xcc, 0x3f, 0xd0, 0x2e, 0x6c, 0x10, 0xcb, - 0x60, 0x0b, 0xb2, 0x6c, 0x41, 0x9e, 0x58, 0x06, 0xf5, 0xd4, 0xb7, 0x0a, 0x6c, 0xc5, 0xe4, 0x49, - 0xe5, 0xab, 0x07, 0xb0, 0xc1, 0x35, 0x94, 0x71, 0x97, 0x74, 0x96, 0x44, 0xa3, 0x57, 0x61, 0x83, - 0x7b, 0x84, 0x66, 0x8d, 0x45, 0x47, 0x48, 0xa4, 0xfa, 0x18, 0x76, 0x0f, 0x89, 0xdf, 0xe6, 0x45, - 0x4c, 0xdb, 0xb6, 0xce, 0xcc, 0x49, 0xaa, 0xbc, 0xed, 0x41, 0x63, 0x91, 0x4f, 0x2a, 0x1d, 0x5f, - 0x83, 0x0d, 0x51, 0x53, 0x89, 0x80, 0xac, 0x49, 0xc9, 0x05, 0x77, 0x2c, 0xf1, 0xea, 0x97, 0xb0, - 0x3b, 0x98, 0xbd, 0xb8, 0xf0, 0xff, 0xcb, 0x96, 0x1f, 0x43, 0x63, 0x71, 0xcb, 0x54, 0x69, 0xf0, - 0xcf, 0x0a, 0xe4, 0x8f, 0xc9, 0xe5, 0x29, 0x71, 0x11, 0x82, 0xac, 0xa5, 0x5d, 0xf2, 0x6a, 0xb0, - 0x88, 0xd9, 0x6f, 0x1a, 0x7c, 0x97, 0x0c, 0x1b, 0x89, 0x6e, 0x0e, 0xe8, 0x1a, 0x14, 0xe9, 0x10, - 0xe2, 0x8e, 0x67, 0xee, 0x94, 0xfb, 0xb7, 0x88, 0x0b, 0x14, 0x70, 0xe2, 0x4e, 0x3d, 0xf4, 0x32, - 0x94, 0xf4, 0xa9, 0x49, 0x2c, 0x9f, 0xa3, 0xb3, 0x0c, 0x0d, 0x1c, 0xc4, 0x08, 0xfe, 0x1f, 0x6a, - 0xdc, 0xfd, 0x63, 0xc7, 0x35, 0x6d, 0xd7, 0xf4, 0xe7, 0x8d, 0x1c, 0x0b, 0xe2, 0x2a, 0x07, 0x0f, - 0x04, 0x54, 0xfd, 0x88, 0x65, 0x17, 0x2e, 0x64, 0xaa, 0x23, 0xa4, 0xfe, 0x5d, 0x01, 0x14, 0x65, - 0x91, 0x32, 0x43, 0x6d, 0x70, 0xcd, 0x65, 0xd4, 0x97, 0x39, 0x39, 0xe7, 0x8a, 0x25, 0x72, 0x49, - 0x86, 0x8a, 0x92, 0xc9, 0x0c, 0xf5, 0x26, 0x94, 0x88, 0xaf, 0x1b, 0x63, 0x41, 0x9a, 0x5d, 0x42, - 0x0a, 0x94, 0xe0, 0x88, 0x6b, 0x30, 0x80, 0x22, 0x3d, 0x31, 0xec, 0xa2, 0x45, 0x77, 0x21, 0x4b, - 0xcd, 0x2c, 0xa4, 0x8e, 0x1f, 0x29, 0x86, 0x41, 0xaf, 0x40, 0xd9, 0xb0, 0x9f, 0x5a, 0x63, 0x8f, - 0xe8, 0xb6, 0x65, 0x78, 0xc2, 0x73, 0x25, 0x0a, 0x1b, 0x72, 0x90, 0xfa, 0xa7, 0x2c, 0xec, 0xf0, - 0xe3, 0xfa, 0x31, 0xd1, 0x5c, 0xff, 0x94, 0x68, 0x7e, 0xaa, 0xa8, 0xfd, 0x7e, 0x13, 0xf7, 0x1e, - 0x00, 0x13, 0x9c, 0x6a, 0x21, 0x93, 0xb7, 0xa8, 0xb5, 0x03, 0xfd, 0x71, 0x91, 0x92, 0xd0, 0x4f, - 0x0f, 0xbd, 0x03, 0x15, 0x87, 0x58, 0x86, 0x69, 0x4d, 0xc4, 0x92, 0xdc, 0x92, 0x34, 0x53, 0x16, - 0x24, 0x7c, 0xc9, 0x3d, 0xa8, 0x9c, 0xce, 0x7d, 0xe2, 0x8d, 0x9f, 0xba, 0xa6, 0xef, 0x13, 0xab, - 0x91, 0x67, 0xc6, 0x29, 0x33, 0xe0, 0x67, 0x1c, 0x46, 0x6f, 0x3c, 0x4e, 0xe4, 0x12, 0xcd, 0x68, - 0x6c, 0xf0, 0x47, 0x16, 0x83, 0x60, 0xa2, 0xd1, 0x47, 0x56, 0xf9, 0x82, 0xcc, 0x43, 0x16, 0x05, - 0x6e, 0x5f, 0x0a, 0x93, 0x1c, 0x6e, 0x41, 0x91, 0x91, 0x30, 0x06, 0x45, 0x7e, 0x72, 0x28, 0x80, - 0xad, 0x7f, 0x0d, 0xea, 0x9a, 0xe3, 0xb8, 0xf6, 0x57, 0xe6, 0xa5, 0xe6, 0x93, 0xb1, 0x67, 0x7e, - 0x4d, 0x1a, 0xc0, 0x68, 0x6a, 0x11, 0xf8, 0xd0, 0xfc, 0x9a, 0xa0, 0x3d, 0x28, 0x98, 0x96, 0x4f, - 0xdc, 0x2b, 0x6d, 0xda, 0x28, 0x33, 0xcb, 0xa1, 0xf0, 0xed, 0xd1, 0x15, 0x18, 0x1c, 0xd0, 0x24, - 0x59, 0xd3, 0x2d, 0x1b, 0x95, 0x05, 0xd6, 0x4f, 0xc8, 0xdc, 0xa3, 0x07, 0xde, 0x27, 0xee, 0x65, - 0xa3, 0xca, 0xd0, 0xec, 0xf7, 0x27, 0xd9, 0x42, 0xa9, 0x5e, 0x56, 0xcf, 0x01, 0xda, 0xe7, 0x9a, - 0x35, 0x21, 0xd4, 0x64, 0xd7, 0x88, 0xb7, 0xf7, 0xa1, 0xa4, 0x33, 0xfa, 0x31, 0x7b, 0x4f, 0x66, - 0xd8, 0x7b, 0x72, 0x77, 0x4f, 0x3e, 0x88, 0x69, 0x86, 0xe2, 0xfc, 0xd8, 0xbb, 0x12, 0xf4, 0xe0, - 0xb7, 0xfa, 0x10, 0xaa, 0x23, 0x57, 0xb3, 0xbc, 0x33, 0xe2, 0xf2, 0x50, 0x7f, 0xfe, 0x6e, 0xea, - 0x5b, 0x90, 0x3b, 0x26, 0xee, 0x84, 0x3d, 0x81, 0x7c, 0xcd, 0x9d, 0x10, 0x5f, 0x10, 0x2f, 0xc4, - 0x1e, 0xc7, 0xaa, 0x47, 0x50, 0x1a, 0x3a, 0x53, 0x53, 0x5c, 0xf5, 0xe8, 0x35, 0xc8, 0x3b, 0xf6, - 0xd4, 0xd4, 0xe7, 0xe2, 0xe1, 0xbb, 0xc9, 0x0d, 0xda, 0x3e, 0x27, 0xfa, 0xc5, 0x80, 0x21, 0xb0, - 0x20, 0xa0, 0x26, 0x62, 0x16, 0xa4, 0x27, 0xbe, 0x8c, 0xd9, 0x6f, 0xf5, 0x0f, 0xeb, 0xb0, 0xbb, - 0x70, 0x72, 0x52, 0xa5, 0x94, 0x77, 0x02, 0xb3, 0x31, 0x8d, 0x33, 0xd1, 0xc2, 0x3a, 0xb4, 0xbf, - 0xb4, 0x17, 0xf3, 0xc5, 0x07, 0x50, 0xf3, 0x85, 0xbd, 0xc6, 0xb1, 0xf3, 0x24, 0x76, 0x8a, 0x1b, - 0x13, 0x57, 0xfd, 0xb8, 0x71, 0x63, 0xd5, 0x4a, 0x36, 0x5e, 0xad, 0xa0, 0xf7, 0xa0, 0x2c, 0x90, - 0xc4, 0xb1, 0xf5, 0x73, 0x96, 0x8e, 0xe9, 0xe9, 0x8f, 0x19, 0xb5, 0x43, 0x51, 0xb8, 0xe4, 0x86, - 0x1f, 0x34, 0x97, 0x71, 0x43, 0x73, 0x35, 0xf2, 0x4b, 0x1c, 0x07, 0x9c, 0x60, 0xc0, 0x93, 0x53, - 0xee, 0x92, 0xba, 0x8f, 0x1d, 0xab, 0xa0, 0x43, 0xc1, 0x3c, 0x8a, 0x39, 0x06, 0x3d, 0x82, 0xb2, - 0x47, 0x1d, 0x36, 0x16, 0xa9, 0xa5, 0xc0, 0x28, 0x85, 0x9f, 0x22, 0xae, 0xc4, 0x25, 0x2f, 0xfc, - 0x50, 0xcf, 0xa0, 0xd6, 0xf2, 0x2e, 0x04, 0xfa, 0x87, 0x4b, 0x65, 0xea, 0xaf, 0x14, 0xa8, 0x87, - 0x1b, 0xa5, 0x7c, 0xc3, 0x56, 0x2c, 0xf2, 0x74, 0x9c, 0xac, 0x1c, 0x4b, 0x16, 0x79, 0x8a, 0xa5, - 0x3b, 0xee, 0x42, 0x99, 0xd2, 0xb0, 0x2b, 0xd6, 0x34, 0xf8, 0x0d, 0x9b, 0xc5, 0x60, 0x91, 0xa7, - 0xd4, 0x8c, 0x5d, 0xc3, 0x53, 0x7f, 0xab, 0x00, 0xc2, 0xc4, 0xb1, 0x5d, 0x3f, 0xbd, 0xd2, 0x2a, - 0x64, 0xa7, 0xe4, 0xcc, 0x5f, 0xa1, 0x32, 0xc3, 0xa1, 0xfb, 0x90, 0x73, 0xcd, 0xc9, 0xb9, 0xbf, - 0xa2, 0xd3, 0xc0, 0x91, 0x6a, 0x1b, 0xb6, 0x62, 0xc2, 0xa4, 0xaa, 0x47, 0xbe, 0x51, 0x60, 0xbb, - 0xe5, 0x5d, 0xec, 0x6b, 0xbe, 0x7e, 0xfe, 0x83, 0x7b, 0x92, 0x16, 0x29, 0x3c, 0xce, 0x78, 0xd7, - 0x67, 0x9d, 0x75, 0x7d, 0x80, 0x81, 0xda, 0xac, 0x23, 0xd5, 0x87, 0x0d, 0x26, 0x45, 0xf7, 0x60, - 0xd1, 0x65, 0xca, 0xf3, 0x5d, 0x96, 0x59, 0x70, 0xd9, 0x19, 0xdc, 0x48, 0xa8, 0x97, 0x2a, 0x7e, - 0x5e, 0x86, 0x75, 0xc9, 0x9f, 0x3e, 0xfb, 0xc2, 0x73, 0xd1, 0x3d, 0xc0, 0x14, 0xa3, 0x3a, 0x34, - 0x47, 0x51, 0x67, 0xbc, 0xa0, 0x25, 0xaf, 0x5d, 0xeb, 0xd3, 0x9a, 0x74, 0x71, 0xc7, 0x54, 0x31, - 0xf0, 0x53, 0x28, 0x47, 0x2f, 0x37, 0x5a, 0x29, 0xf2, 0x17, 0x50, 0xd8, 0x85, 0xe3, 0xb6, 0xaf, - 0x32, 0x70, 0xd8, 0x32, 0xbc, 0x07, 0x15, 0xfa, 0xee, 0x09, 0xc9, 0xf8, 0xa9, 0x2a, 0x13, 0xcb, - 0x08, 0x88, 0xd4, 0x47, 0x00, 0x98, 0xe8, 0xb6, 0x6b, 0x0c, 0x34, 0xd3, 0x45, 0x75, 0x58, 0xa7, - 0xcf, 0x24, 0x5e, 0xf3, 0xd2, 0x9f, 0xf4, 0x49, 0x75, 0xa5, 0x4d, 0x67, 0x44, 0x2c, 0xe6, 0x1f, - 0xea, 0x7f, 0x72, 0x00, 0x61, 0xaf, 0x23, 0xd6, 0x8f, 0x51, 0x62, 0xfd, 0x18, 0xd4, 0x84, 0x82, - 0xae, 0x39, 0x9a, 0x4e, 0x0b, 0x5a, 0x51, 0x31, 0xcb, 0x6f, 0xf4, 0x12, 0x14, 0xb5, 0x2b, 0xcd, - 0x9c, 0x6a, 0xa7, 0x53, 0xc2, 0xa2, 0x2d, 0x8b, 0x43, 0x00, 0xad, 0x2a, 0x44, 0x74, 0xf1, 0x70, - 0xcc, 0xb2, 0x70, 0x14, 0xa9, 0x96, 0xc5, 0x23, 0x7a, 0x03, 0x90, 0x27, 0xea, 0x1d, 0xcf, 0xd2, - 0x1c, 0x41, 0x98, 0x63, 0x84, 0x75, 0x81, 0x19, 0x5a, 0x9a, 0xc3, 0xa9, 0xdf, 0x86, 0x6d, 0x97, - 0xe8, 0xc4, 0xbc, 0x4a, 0xd0, 0xe7, 0x19, 0x3d, 0x0a, 0x70, 0xe1, 0x8a, 0xdb, 0x00, 0xa1, 0xa9, - 0x59, 0x82, 0xae, 0xe0, 0x62, 0x60, 0x65, 0xb4, 0x07, 0x5b, 0x9a, 0xe3, 0x4c, 0xe7, 0x09, 0x7e, - 0x05, 0x46, 0xb7, 0x29, 0x51, 0x21, 0xbb, 0x5d, 0xd8, 0x30, 0xbd, 0xf1, 0xe9, 0xcc, 0x9b, 0xb3, - 0x12, 0xa8, 0x80, 0xf3, 0xa6, 0xb7, 0x3f, 0xf3, 0xe6, 0xf4, 0x1e, 0x9a, 0x79, 0xc4, 0x88, 0x56, - 0x3e, 0x05, 0x0a, 0x60, 0x25, 0xcf, 0x42, 0x85, 0x56, 0x5a, 0x52, 0xa1, 0x25, 0x4b, 0xb0, 0xf2, - 0x62, 0x09, 0x16, 0x2f, 0xe2, 0x2a, 0xc9, 0x22, 0x2e, 0x56, 0xa1, 0x55, 0x13, 0x15, 0x5a, 0xb4, - 0xec, 0xaa, 0x5d, 0xa3, 0xec, 0x7a, 0x0b, 0x40, 0x77, 0x66, 0xe3, 0x99, 0xa7, 0x4d, 0x88, 0xd7, - 0xa8, 0xb3, 0xa3, 0x52, 0x97, 0x51, 0x2e, 0xa3, 0x0d, 0x17, 0x75, 0x67, 0x76, 0xc2, 0x48, 0xd0, - 0x23, 0xa8, 0xd0, 0x8d, 0xc7, 0xa6, 0x3d, 0x76, 0x35, 0x9f, 0x78, 0x8d, 0xcd, 0x15, 0x6b, 0x4a, - 0x94, 0xac, 0x6b, 0x63, 0x4a, 0x84, 0xde, 0x83, 0x2a, 0x55, 0x98, 0x84, 0xcb, 0xd0, 0x8a, 0x65, - 0x65, 0x46, 0x27, 0xd7, 0xbd, 0x0b, 0x65, 0xdb, 0x19, 0x4f, 0x35, 0x9f, 0x58, 0xba, 0x49, 0xbc, - 0xc6, 0xd6, 0xaa, 0xcd, 0x6c, 0xe7, 0x48, 0x12, 0xa9, 0x53, 0xb8, 0xc1, 0x42, 0xfe, 0x45, 0x1f, - 0x08, 0xa2, 0x6f, 0x98, 0x79, 0x76, 0xdf, 0xf0, 0x31, 0xec, 0x24, 0x77, 0x4b, 0x95, 0x3d, 0xfe, - 0xa2, 0xc0, 0xf6, 0x50, 0xd7, 0x7c, 0xfa, 0x60, 0x4e, 0xdf, 0xf2, 0x7a, 0x56, 0x5b, 0xe7, 0xba, - 0xad, 0xf7, 0xc8, 0x9b, 0x27, 0xbb, 0xfa, 0xcd, 0xa3, 0x76, 0xe0, 0x46, 0x42, 0xde, 0xb4, 0x4d, - 0xfa, 0x43, 0xe2, 0x1f, 0xb6, 0x87, 0xda, 0x19, 0x19, 0xd8, 0xa6, 0x95, 0xca, 0x5b, 0x2a, 0x81, - 0x9d, 0x24, 0x97, 0x54, 0x17, 0x14, 0x4d, 0x24, 0xda, 0x19, 0x19, 0x3b, 0x94, 0x87, 0x30, 0x60, - 0xd1, 0x93, 0x4c, 0xd5, 0x33, 0x68, 0x9c, 0x38, 0x86, 0xe6, 0x93, 0x17, 0x94, 0xf7, 0x79, 0xfb, - 0xd8, 0x70, 0x73, 0xc9, 0x3e, 0xa9, 0x34, 0xba, 0x0f, 0x55, 0x7a, 0xb7, 0x2f, 0xec, 0x46, 0x6f, - 0xfc, 0x80, 0xb7, 0xfa, 0x6b, 0x05, 0x36, 0x87, 0x73, 0x4b, 0x7f, 0x81, 0xd0, 0xbb, 0x0f, 0x79, - 0xde, 0x4b, 0x10, 0x27, 0x26, 0xd1, 0x40, 0xe0, 0x38, 0x56, 0xba, 0xb0, 0x4c, 0x6d, 0x5a, 0x06, - 0xf9, 0x4a, 0x5c, 0x26, 0x3c, 0x79, 0x77, 0x29, 0x84, 0xf7, 0x1e, 0x23, 0x92, 0xfc, 0xc0, 0xad, - 0xbe, 0xe7, 0xca, 0xf3, 0x05, 0xeb, 0xc1, 0xf4, 0x1d, 0xe2, 0x6a, 0xbe, 0xed, 0x7e, 0xff, 0xbd, - 0xd6, 0xbf, 0x29, 0xac, 0xa3, 0x1f, 0x6e, 0x90, 0x4a, 0xe1, 0x67, 0x9e, 0x7b, 0x04, 0x59, 0x83, - 0x78, 0x3a, 0x53, 0xae, 0x8c, 0xd9, 0x6f, 0xca, 0x9e, 0xe6, 0xaf, 0x99, 0xc7, 0xce, 0x78, 0x55, - 0xb2, 0x97, 0x62, 0x0c, 0x19, 0x0e, 0x0b, 0x1a, 0xf6, 0x9e, 0x34, 0x2d, 0x83, 0xdd, 0xd8, 0xf4, - 0x3d, 0x69, 0x5a, 0xc6, 0xeb, 0xdf, 0x2a, 0x50, 0x0c, 0x86, 0xae, 0x28, 0x0f, 0x99, 0xfe, 0x93, - 0xfa, 0x1a, 0x2a, 0xc1, 0xc6, 0x49, 0xef, 0x49, 0xaf, 0xff, 0x59, 0xaf, 0xae, 0xa0, 0x6d, 0xa8, - 0xf7, 0xfa, 0xa3, 0xf1, 0x7e, 0xbf, 0x3f, 0x1a, 0x8e, 0x70, 0x6b, 0x30, 0xe8, 0x1c, 0xd4, 0x33, - 0x68, 0x0b, 0x6a, 0xc3, 0x51, 0x1f, 0x77, 0xc6, 0xa3, 0xfe, 0xf1, 0xfe, 0x70, 0xd4, 0xef, 0x75, - 0xea, 0xeb, 0xa8, 0x01, 0xdb, 0xad, 0x23, 0xdc, 0x69, 0x1d, 0x7c, 0x1e, 0x27, 0xcf, 0x52, 0x4c, - 0xb7, 0xd7, 0xee, 0x1f, 0x0f, 0x5a, 0xa3, 0xee, 0xfe, 0x51, 0x67, 0xfc, 0x69, 0x07, 0x0f, 0xbb, - 0xfd, 0x5e, 0x3d, 0x47, 0xd9, 0xe3, 0xce, 0x61, 0xb7, 0xdf, 0x1b, 0xd3, 0x5d, 0x1e, 0xf7, 0x4f, - 0x7a, 0x07, 0xf5, 0xfc, 0xeb, 0x8f, 0xa0, 0x14, 0x79, 0x12, 0xa3, 0x02, 0x64, 0x87, 0xed, 0x56, - 0xaf, 0xbe, 0x86, 0x6a, 0x50, 0x6a, 0x0d, 0x06, 0xb8, 0xff, 0x93, 0xee, 0x71, 0x6b, 0xd4, 0xa9, - 0x2b, 0x08, 0x20, 0x7f, 0x32, 0xec, 0x3c, 0xe9, 0x7c, 0x5e, 0xcf, 0xbc, 0x3e, 0x80, 0x6a, 0x5c, - 0x77, 0xaa, 0xc9, 0xf0, 0xa4, 0xdd, 0xee, 0x0c, 0x87, 0x5c, 0xad, 0x51, 0xf7, 0xb8, 0xd3, 0x3f, - 0x19, 0xf1, 0x75, 0xed, 0x56, 0xaf, 0xdd, 0x39, 0xaa, 0x67, 0x28, 0x02, 0x77, 0x06, 0x47, 0xad, - 0x36, 0x55, 0x82, 0x7e, 0x9c, 0xf4, 0x7a, 0xdd, 0xde, 0x61, 0x3d, 0xfb, 0xf0, 0xf7, 0x55, 0xc8, - 0x0c, 0x0e, 0x50, 0x0b, 0x20, 0xec, 0xe1, 0xa1, 0x5d, 0x6e, 0xe6, 0x85, 0xc6, 0x60, 0xb3, 0xb1, - 0x88, 0xe0, 0x8e, 0x56, 0xd7, 0xd0, 0xdb, 0xb0, 0x3e, 0xf2, 0x6c, 0x24, 0xae, 0xa0, 0x70, 0x76, - 0xdd, 0xdc, 0x8c, 0x40, 0x24, 0xf5, 0x03, 0xe5, 0x6d, 0x05, 0x7d, 0x08, 0xc5, 0x60, 0x62, 0x89, - 0x76, 0x38, 0x55, 0x72, 0xb6, 0xdb, 0xdc, 0x5d, 0x80, 0x07, 0x3b, 0x1e, 0x43, 0x35, 0x3e, 0xf3, - 0x44, 0xb7, 0x38, 0xf1, 0xd2, 0x79, 0x6a, 0xf3, 0xa5, 0xe5, 0xc8, 0x80, 0xdd, 0xfb, 0xb0, 0x21, - 0xe6, 0x92, 0x48, 0xc4, 0x59, 0x7c, 0xca, 0xd9, 0xbc, 0x91, 0x80, 0x06, 0x2b, 0x7f, 0x0c, 0x05, - 0x39, 0x24, 0x44, 0x37, 0x02, 0x13, 0x45, 0xa7, 0x74, 0xcd, 0x9d, 0x24, 0x38, 0xba, 0x58, 0x4e, - 0xdb, 0xe4, 0xe2, 0xc4, 0x88, 0x4f, 0x2e, 0x4e, 0x0e, 0xe5, 0xd4, 0x35, 0x74, 0x08, 0xe5, 0xe8, - 0xa8, 0x0b, 0xdd, 0x0c, 0xb6, 0x49, 0x0e, 0xdf, 0x9a, 0xcd, 0x65, 0xa8, 0xa8, 0x2d, 0xe3, 0x05, - 0x82, 0xb4, 0xe5, 0xd2, 0x22, 0x45, 0xda, 0x72, 0x79, 0x4d, 0xa1, 0xae, 0xa1, 0x11, 0xd4, 0x12, - 0x5d, 0x1c, 0xf4, 0x92, 0x4c, 0x0d, 0xcb, 0xda, 0xa2, 0xcd, 0xdb, 0x2b, 0xb0, 0xc9, 0x80, 0x09, - 0x66, 0x4a, 0x28, 0xb4, 0x68, 0xec, 0x3a, 0x68, 0xee, 0x2e, 0xc0, 0x03, 0xa9, 0xf6, 0xa1, 0x72, - 0x48, 0xfc, 0x81, 0x4b, 0xae, 0xd2, 0xf3, 0x78, 0xcc, 0x78, 0x84, 0x73, 0x2d, 0xd4, 0x4c, 0xd0, - 0x46, 0x86, 0x5d, 0xcf, 0xe2, 0x73, 0x00, 0xa5, 0xc8, 0xb0, 0x08, 0x89, 0x93, 0xb5, 0x38, 0xcf, - 0x6a, 0xde, 0x5c, 0x82, 0x09, 0xb8, 0x7c, 0x08, 0x05, 0xd9, 0x2c, 0x91, 0xc1, 0x93, 0xe8, 0xd2, - 0xc8, 0xe0, 0x49, 0xf6, 0x54, 0xd4, 0xf5, 0xdf, 0x64, 0x14, 0x74, 0x08, 0xa5, 0x48, 0x5b, 0x41, - 0x4a, 0xb1, 0xd8, 0xf6, 0x90, 0x52, 0x2c, 0xe9, 0x41, 0x70, 0x46, 0x9f, 0x40, 0x25, 0xf6, 0xf4, - 0x96, 0x66, 0x59, 0xd6, 0x6e, 0x68, 0xde, 0x5a, 0x8a, 0x0b, 0x94, 0x1a, 0x42, 0x3d, 0xf9, 0xd8, - 0x45, 0xb7, 0xa3, 0xfb, 0x2f, 0x72, 0xbc, 0xb3, 0x0a, 0x1d, 0x65, 0x9a, 0x9c, 0x5e, 0x49, 0xa6, - 0x2b, 0xa6, 0x63, 0x92, 0xe9, 0xaa, 0xa1, 0x17, 0x67, 0x9a, 0x1c, 0x15, 0x49, 0xa6, 0x2b, 0xa6, - 0x56, 0x92, 0xe9, 0xaa, 0x09, 0x93, 0xba, 0x46, 0x4d, 0x19, 0x2b, 0x59, 0xa5, 0x29, 0x97, 0xd5, - 0xdd, 0xd2, 0x94, 0x4b, 0x6b, 0x5c, 0x7e, 0xac, 0xe3, 0x15, 0xa7, 0x3c, 0xd6, 0x4b, 0xab, 0x59, - 0x79, 0xac, 0x97, 0x17, 0xa9, 0xea, 0x1a, 0xfa, 0x14, 0x36, 0x17, 0x2a, 0x3e, 0x24, 0x34, 0x5a, - 0x55, 0x72, 0x36, 0x5f, 0x5e, 0x89, 0x8f, 0x1c, 0xaa, 0x52, 0x58, 0x4d, 0x05, 0xf7, 0xcf, 0x42, - 0xa9, 0x27, 0xef, 0x9f, 0xc5, 0xca, 0x4b, 0x24, 0x88, 0x03, 0x28, 0x45, 0xaa, 0x14, 0x14, 0x5e, - 0x57, 0x89, 0xca, 0xa8, 0x79, 0x73, 0x09, 0x46, 0x72, 0xda, 0x57, 0xff, 0xf1, 0xdd, 0x1d, 0xe5, - 0x9f, 0xdf, 0xdd, 0x51, 0xfe, 0xf5, 0xdd, 0x1d, 0xe5, 0x8f, 0xff, 0xbe, 0xb3, 0x06, 0x75, 0xdb, - 0x9d, 0xec, 0xf9, 0xe6, 0xc5, 0xd5, 0xde, 0xc5, 0x15, 0xfb, 0x57, 0xb3, 0xd3, 0x3c, 0xfb, 0xf3, - 0xee, 0x7f, 0x03, 0x00, 0x00, 0xff, 0xff, 0xeb, 0xc1, 0xd2, 0x13, 0xb8, 0x26, 0x00, 0x00, + 0xd1, 0x5c, 0x10, 0x00, 0x81, 0xc6, 0x0f, 0xc1, 0x21, 0x45, 0x42, 0x90, 0x25, 0xcb, 0x2b, 0x7d, + 0xfe, 0x64, 0xc7, 0xa6, 0x6d, 0x59, 0xe5, 0x72, 0x55, 0xca, 0x29, 0x83, 0x20, 0x44, 0xc3, 0x22, + 0x01, 0xd4, 0x00, 0xb4, 0xe3, 0xaa, 0x94, 0x37, 0xcb, 0xdd, 0x21, 0xb8, 0x21, 0xb8, 0xbb, 0xde, + 0x59, 0x50, 0x86, 0x2b, 0x87, 0x9c, 0x92, 0x43, 0x9c, 0xa3, 0x53, 0xa9, 0x9c, 0xf2, 0x04, 0xb9, + 0xa4, 0x92, 0x6b, 0xae, 0x39, 0xe6, 0x11, 0x52, 0xce, 0x13, 0xe4, 0x0d, 0x52, 0xf3, 0xb3, 0xbf, + 0x00, 0x25, 0x66, 0x65, 0x9d, 0x88, 0xed, 0xee, 0xe9, 0xe9, 0xbf, 0xe9, 0xe9, 0xe9, 0x26, 0x80, + 0x6b, 0xba, 0x27, 0xbb, 0xae, 0xe7, 0xf8, 0x0e, 0xca, 0xb3, 0xdf, 0xad, 0xea, 0x05, 0xf1, 0xf5, + 0x00, 0xd6, 0xaa, 0x11, 0x4f, 0x3f, 0xf5, 0xc3, 0xcf, 0xad, 0x89, 0x33, 0x71, 0xf8, 0xcf, 0x77, + 0xd8, 0x2f, 0x01, 0x55, 0x77, 0xa1, 0x86, 0xc9, 0x57, 0x33, 0x42, 0xfd, 0x4f, 0x88, 0x6e, 0x12, + 0x0f, 0xdd, 0x06, 0x30, 0xa6, 0x33, 0xea, 0x13, 0x4f, 0xb3, 0xcc, 0xa6, 0x72, 0x57, 0x79, 0x90, + 0xc7, 0x65, 0x09, 0xe9, 0x99, 0x2a, 0x86, 0x3a, 0x26, 0xd4, 0x75, 0x6c, 0x4a, 0xae, 0xb5, 0x00, + 0xbd, 0x06, 0x05, 0xe2, 0x79, 0x8e, 0xd7, 0xcc, 0xdd, 0x55, 0x1e, 0x54, 0x1e, 0x56, 0x76, 0xb9, + 0xd4, 0x5d, 0x06, 0xc2, 0x02, 0xa3, 0x3e, 0x86, 0x02, 0xff, 0x46, 0xf7, 0x20, 0xef, 0xcf, 0x5d, + 0xc2, 0x99, 0xd4, 0x1f, 0xae, 0xc7, 0x48, 0xc7, 0x73, 0x97, 0x60, 0x8e, 0x44, 0x4d, 0x58, 0xbb, + 0x20, 0x94, 0xea, 0x13, 0xc2, 0x59, 0x96, 0x71, 0xf0, 0xa9, 0x0e, 0x00, 0xc6, 0xd4, 0x91, 0xea, + 0xa0, 0x1f, 0x41, 0xf1, 0x8c, 0x4b, 0xc8, 0xd9, 0x55, 0x1e, 0x6e, 0x0a, 0x76, 0x09, 0x6d, 0xb1, + 0x24, 0x41, 0x5b, 0x50, 0x30, 0x9c, 0x99, 0xed, 0x73, 0x96, 0x35, 0x2c, 0x3e, 0xd4, 0x36, 0x94, + 0xc7, 0xd6, 0x05, 0xa1, 0xbe, 0x7e, 0xe1, 0xa2, 0x16, 0x94, 0xdc, 0xb3, 0x39, 0xb5, 0x0c, 0x7d, + 0xca, 0x39, 0xae, 0xe2, 0xf0, 0x9b, 0xc9, 0x34, 0x75, 0x26, 0x1c, 0x95, 0xe3, 0xa8, 0xe0, 0x53, + 0xfd, 0x95, 0x02, 0x15, 0x2e, 0x94, 0xb0, 0x19, 0x7a, 0x2b, 0x25, 0xd5, 0x56, 0x20, 0x55, 0xdc, + 0xa6, 0xcf, 0x16, 0x0b, 0xbd, 0x0d, 0x65, 0x3f, 0x10, 0xab, 0xb9, 0xca, 0xd9, 0x48, 0x5b, 0x85, + 0xd2, 0xe2, 0x88, 0x42, 0xfd, 0x56, 0x81, 0xc6, 0x9e, 0xe3, 0xf8, 0xd4, 0xf7, 0x74, 0x37, 0x93, + 0x75, 0xee, 0x41, 0x81, 0xfa, 0x8e, 0x47, 0xa4, 0x0f, 0x6b, 0xbb, 0x32, 0xce, 0x46, 0x0c, 0x88, + 0x05, 0x0e, 0xbd, 0x0e, 0x45, 0x8f, 0x4c, 0x2c, 0xc7, 0x96, 0x22, 0xd5, 0x03, 0x2a, 0xcc, 0xa1, + 0x58, 0x62, 0xd5, 0x36, 0x6c, 0xc4, 0xa4, 0xc9, 0x62, 0x16, 0x75, 0x1f, 0x6e, 0xf4, 0x68, 0xc8, + 0xc4, 0x25, 0x66, 0x16, 0xad, 0xd4, 0x5f, 0xc0, 0x76, 0x9a, 0x4b, 0x26, 0x27, 0xa9, 0x50, 0x3d, + 0x89, 0x71, 0xe1, 0x46, 0x2a, 0xe1, 0x04, 0x4c, 0xfd, 0x08, 0xea, 0xed, 0xe9, 0xd4, 0x31, 0x7a, + 0xfb, 0x99, 0x44, 0x1d, 0xc0, 0x7a, 0xb8, 0x3c, 0x93, 0x8c, 0x75, 0xc8, 0x59, 0x42, 0xb2, 0x3c, + 0xce, 0x59, 0xa6, 0xfa, 0x05, 0xac, 0x1f, 0x10, 0x5f, 0xf8, 0x2f, 0x4b, 0x44, 0xdc, 0x84, 0x12, + 0xf7, 0xba, 0x16, 0x72, 0x5d, 0xe3, 0xdf, 0x3d, 0x53, 0xfd, 0x9d, 0x02, 0x8d, 0x88, 0x77, 0x26, + 0x69, 0xaf, 0x19, 0x6f, 0x05, 0xea, 0xeb, 0x3e, 0x95, 0xe1, 0xd6, 0x10, 0x1c, 0x39, 0xc9, 0x88, + 0xc1, 0xb1, 0x40, 0xab, 0x06, 0xac, 0x0f, 0x67, 0x2f, 0xa0, 0xea, 0x75, 0x84, 0x51, 0x3f, 0x86, + 0x46, 0xb4, 0x49, 0xa6, 0x98, 0xfe, 0x25, 0x6c, 0x1e, 0x10, 0xbf, 0x3d, 0x9d, 0x72, 0x26, 0x34, + 0x93, 0xa8, 0x1f, 0x42, 0x93, 0x7c, 0x6d, 0x4c, 0x67, 0x26, 0xd1, 0x7c, 0xe7, 0xe2, 0x84, 0xfa, + 0x8e, 0x4d, 0x34, 0x2e, 0x20, 0x95, 0x51, 0xb9, 0x2d, 0xf1, 0xe3, 0x00, 0x2d, 0x76, 0x53, 0xcf, + 0x61, 0x2b, 0xb9, 0x7b, 0x26, 0xbf, 0xfd, 0x1f, 0x14, 0xc3, 0xdd, 0x56, 0x17, 0x6d, 0x25, 0x91, + 0xea, 0x97, 0x3c, 0x40, 0x64, 0x5a, 0xc8, 0xa2, 0xe7, 0x6d, 0x00, 0x91, 0x4c, 0xb4, 0x73, 0x32, + 0xe7, 0x9a, 0x55, 0x71, 0x59, 0x40, 0x9e, 0x90, 0xb9, 0xfa, 0x57, 0x05, 0x36, 0x62, 0x1b, 0x64, + 0x52, 0x25, 0xca, 0x66, 0xb9, 0x67, 0x65, 0x33, 0x74, 0x1f, 0x8a, 0x53, 0xc1, 0x55, 0x84, 0x61, + 0x35, 0xa0, 0x1b, 0x12, 0xc6, 0x4d, 0xe0, 0x18, 0x15, 0x9d, 0xea, 0x97, 0x84, 0x36, 0xf3, 0xdc, + 0x30, 0x29, 0x2a, 0x81, 0x53, 0x7f, 0xce, 0x9d, 0x20, 0x36, 0xd8, 0x9b, 0x67, 0x4b, 0x15, 0xe8, + 0x16, 0x48, 0x4b, 0x44, 0x47, 0xb3, 0x24, 0x00, 0xe2, 0x6c, 0xa2, 0x91, 0xa1, 0xdb, 0x62, 0x0f, + 0x9a, 0x75, 0x03, 0xea, 0xeb, 0x9e, 0x1f, 0xb3, 0x7d, 0x89, 0x03, 0x9e, 0x90, 0x39, 0xbb, 0xb0, + 0xa6, 0xd6, 0x85, 0xe5, 0x73, 0x6b, 0x14, 0xb0, 0xf8, 0x40, 0x3b, 0xb0, 0x46, 0x6c, 0x93, 0x2f, + 0xc8, 0xf3, 0x05, 0x45, 0x62, 0x9b, 0xcc, 0x53, 0xdf, 0x29, 0xb0, 0x99, 0x90, 0x27, 0x93, 0xaf, + 0x1e, 0xc0, 0x9a, 0xd0, 0x30, 0x88, 0xbb, 0xb4, 0xb3, 0x02, 0x34, 0x7a, 0x1d, 0xd6, 0x84, 0x47, + 0x58, 0xd6, 0x58, 0x74, 0x44, 0x80, 0x54, 0x1f, 0xc3, 0xce, 0x01, 0xf1, 0x3b, 0xa2, 0x88, 0xe9, + 0x38, 0xf6, 0xa9, 0x35, 0xc9, 0x94, 0xb7, 0x29, 0x34, 0x17, 0xf9, 0x64, 0xd2, 0xf1, 0x0d, 0x58, + 0x93, 0x35, 0x95, 0x0c, 0xc8, 0xf5, 0x40, 0x72, 0xc9, 0x1d, 0x07, 0x78, 0xf5, 0x2b, 0xd8, 0x19, + 0xce, 0x5e, 0x5c, 0xf8, 0xff, 0x65, 0xcb, 0x4f, 0xa0, 0xb9, 0xb8, 0x65, 0xa6, 0x34, 0xf8, 0x27, + 0x05, 0x8a, 0x47, 0xe4, 0xe2, 0x84, 0x78, 0x08, 0x41, 0xde, 0xd6, 0x2f, 0x44, 0x35, 0x58, 0xc6, + 0xfc, 0x37, 0x0b, 0xbe, 0x0b, 0x8e, 0x8d, 0x45, 0xb7, 0x00, 0xf4, 0x4c, 0x86, 0x74, 0x09, 0xf1, + 0xb4, 0x99, 0x37, 0x15, 0xfe, 0x2d, 0xe3, 0x12, 0x03, 0x1c, 0x7b, 0x53, 0x8a, 0x5e, 0x85, 0x8a, + 0x31, 0xb5, 0x88, 0xed, 0x0b, 0x74, 0x9e, 0xa3, 0x41, 0x80, 0x38, 0xc1, 0xff, 0xc3, 0xba, 0x70, + 0xbf, 0xe6, 0x7a, 0x96, 0xe3, 0x59, 0xfe, 0xbc, 0x59, 0xe0, 0x41, 0x5c, 0x17, 0xe0, 0xa1, 0x84, + 0xaa, 0x1f, 0xf3, 0xec, 0x22, 0x84, 0xcc, 0x74, 0x84, 0xd4, 0xbf, 0x2b, 0x80, 0xe2, 0x2c, 0x32, + 0x66, 0xa8, 0x35, 0xa1, 0x79, 0x10, 0xf5, 0x55, 0x41, 0x2e, 0xb8, 0xe2, 0x00, 0xb9, 0x24, 0x43, + 0xc5, 0xc9, 0x82, 0x0c, 0xf5, 0x36, 0x54, 0x88, 0x6f, 0x98, 0x9a, 0x24, 0xcd, 0x2f, 0x21, 0x05, + 0x46, 0x70, 0x28, 0x34, 0x18, 0x42, 0x99, 0x9d, 0x18, 0x7e, 0xd1, 0xa2, 0xbb, 0x90, 0x67, 0x66, + 0x96, 0x52, 0x27, 0x8f, 0x14, 0xc7, 0xa0, 0xd7, 0xa0, 0x6a, 0x3a, 0x4f, 0x6d, 0x8d, 0x12, 0xc3, + 0xb1, 0x4d, 0x2a, 0x3d, 0x57, 0x61, 0xb0, 0x91, 0x00, 0xa9, 0x7f, 0xcc, 0xc3, 0xb6, 0x38, 0xae, + 0x9f, 0x10, 0xdd, 0xf3, 0x4f, 0x88, 0xee, 0x67, 0x8a, 0xda, 0x1f, 0x36, 0x71, 0xef, 0x02, 0x70, + 0xc1, 0x99, 0x16, 0x41, 0xf2, 0x96, 0xb5, 0x76, 0xa8, 0x3f, 0x2e, 0x33, 0x12, 0xf6, 0x49, 0xd1, + 0x7b, 0x50, 0x73, 0x89, 0x6d, 0x5a, 0xf6, 0x44, 0x2e, 0x29, 0x2c, 0x49, 0x33, 0x55, 0x49, 0x22, + 0x96, 0xdc, 0x83, 0xda, 0xc9, 0xdc, 0x27, 0x54, 0x7b, 0xea, 0x59, 0xbe, 0x4f, 0xec, 0x66, 0x91, + 0x1b, 0xa7, 0xca, 0x81, 0x9f, 0x0b, 0x18, 0xbb, 0xf1, 0x04, 0x91, 0x47, 0x74, 0xb3, 0xb9, 0x26, + 0x1e, 0x59, 0x1c, 0x82, 0x89, 0xce, 0x1e, 0x59, 0xd5, 0x73, 0x32, 0x8f, 0x58, 0x94, 0x84, 0x7d, + 0x19, 0x2c, 0xe0, 0x70, 0x0b, 0xca, 0x9c, 0x84, 0x33, 0x28, 0x8b, 0x93, 0xc3, 0x00, 0x7c, 0xfd, + 0x1b, 0xd0, 0xd0, 0x5d, 0xd7, 0x73, 0xbe, 0xb6, 0x2e, 0x74, 0x9f, 0x68, 0xd4, 0xfa, 0x86, 0x34, + 0x81, 0xd3, 0xac, 0xc7, 0xe0, 0x23, 0xeb, 0x1b, 0x82, 0x76, 0xa1, 0x64, 0xd9, 0x3e, 0xf1, 0x2e, + 0xf5, 0x69, 0xb3, 0xca, 0x2d, 0x87, 0xa2, 0xb7, 0x47, 0x4f, 0x62, 0x70, 0x48, 0x93, 0x66, 0xcd, + 0xb6, 0x6c, 0xd6, 0x16, 0x58, 0x3f, 0x21, 0x73, 0xca, 0x0e, 0xbc, 0x4f, 0xbc, 0x8b, 0x66, 0x9d, + 0xa3, 0xf9, 0xef, 0x4f, 0xf3, 0xa5, 0x4a, 0xa3, 0xaa, 0x9e, 0x01, 0x74, 0xce, 0x74, 0x7b, 0x42, + 0x98, 0xc9, 0xae, 0x11, 0x6f, 0x1f, 0x42, 0xc5, 0xe0, 0xf4, 0x1a, 0x7f, 0x4f, 0xe6, 0xf8, 0x7b, + 0x72, 0x67, 0x37, 0x78, 0x10, 0xb3, 0x0c, 0x25, 0xf8, 0xf1, 0x77, 0x25, 0x18, 0xe1, 0x6f, 0xf5, + 0x21, 0xd4, 0xc7, 0x9e, 0x6e, 0xd3, 0x53, 0xe2, 0x89, 0x50, 0x7f, 0xfe, 0x6e, 0xea, 0x3b, 0x50, + 0x38, 0x22, 0xde, 0x84, 0x3f, 0x81, 0x7c, 0xdd, 0x9b, 0x10, 0x5f, 0x12, 0x2f, 0xc4, 0x9e, 0xc0, + 0xaa, 0x87, 0x50, 0x19, 0xb9, 0x53, 0x4b, 0x5e, 0xf5, 0xe8, 0x0d, 0x28, 0xba, 0xce, 0xd4, 0x32, + 0xe6, 0xf2, 0xe1, 0xbb, 0x21, 0x0c, 0xda, 0x39, 0x23, 0xc6, 0xf9, 0x90, 0x23, 0xb0, 0x24, 0x60, + 0x26, 0xe2, 0x16, 0x64, 0x27, 0xbe, 0x8a, 0xf9, 0x6f, 0xf5, 0xf7, 0xab, 0xb0, 0xb3, 0x70, 0x72, + 0x32, 0xa5, 0x94, 0xf7, 0x42, 0xb3, 0x71, 0x8d, 0x73, 0xf1, 0xc2, 0x3a, 0xb2, 0x7f, 0x60, 0x2f, + 0xee, 0x8b, 0x8f, 0x60, 0xdd, 0x97, 0xf6, 0xd2, 0x12, 0xe7, 0x49, 0xee, 0x94, 0x34, 0x26, 0xae, + 0xfb, 0x49, 0xe3, 0x26, 0xaa, 0x95, 0x7c, 0xb2, 0x5a, 0x41, 0x1f, 0x40, 0x55, 0x22, 0x89, 0xeb, + 0x18, 0x67, 0x3c, 0x1d, 0xb3, 0xd3, 0x9f, 0x30, 0x6a, 0x97, 0xa1, 0x70, 0xc5, 0x8b, 0x3e, 0x58, + 0x2e, 0x13, 0x86, 0x16, 0x6a, 0x14, 0x97, 0x38, 0x0e, 0x04, 0xc1, 0x50, 0x24, 0xa7, 0xc2, 0x05, + 0x73, 0x1f, 0x3f, 0x56, 0x61, 0x87, 0x82, 0x7b, 0x14, 0x0b, 0x0c, 0x7a, 0x04, 0x55, 0xca, 0x1c, + 0xa6, 0xc9, 0xd4, 0x52, 0xe2, 0x94, 0xd2, 0x4f, 0x31, 0x57, 0xe2, 0x0a, 0x8d, 0x3e, 0xd4, 0x53, + 0x58, 0x6f, 0xd3, 0x73, 0x89, 0x7e, 0x79, 0xa9, 0x4c, 0xfd, 0xb5, 0x02, 0x8d, 0x68, 0xa3, 0x8c, + 0x6f, 0xd8, 0x9a, 0x4d, 0x9e, 0x6a, 0xe9, 0xca, 0xb1, 0x62, 0x93, 0xa7, 0x38, 0x70, 0xc7, 0x5d, + 0xa8, 0x32, 0x1a, 0x7e, 0xc5, 0x5a, 0xa6, 0xb8, 0x61, 0xf3, 0x18, 0x6c, 0xf2, 0x94, 0x99, 0xb1, + 0x67, 0x52, 0xf5, 0xb7, 0x0a, 0x20, 0x4c, 0x5c, 0xc7, 0xf3, 0xb3, 0x2b, 0xad, 0x42, 0x7e, 0x4a, + 0x4e, 0xfd, 0x2b, 0x54, 0xe6, 0x38, 0x74, 0x1f, 0x0a, 0x9e, 0x35, 0x39, 0xf3, 0xaf, 0xe8, 0x34, + 0x08, 0xa4, 0xda, 0x81, 0xcd, 0x84, 0x30, 0x99, 0xea, 0x91, 0x6f, 0x15, 0xd8, 0x6a, 0xd3, 0xf3, + 0x3d, 0xdd, 0x37, 0xce, 0x5e, 0xba, 0x27, 0x59, 0x91, 0x22, 0xe2, 0x4c, 0x74, 0x7d, 0x56, 0x79, + 0xd7, 0x07, 0x38, 0xa8, 0xc3, 0x3b, 0x52, 0x03, 0x58, 0xe3, 0x52, 0xf4, 0xf6, 0x17, 0x5d, 0xa6, + 0x3c, 0xdf, 0x65, 0xb9, 0x05, 0x97, 0x9d, 0xc2, 0x8d, 0x94, 0x7a, 0x99, 0xe2, 0xe7, 0x55, 0x58, + 0x0d, 0xf8, 0xb3, 0x67, 0x5f, 0x74, 0x2e, 0x7a, 0xfb, 0x98, 0x61, 0x54, 0x97, 0xe5, 0x28, 0xe6, + 0x8c, 0x17, 0xb4, 0xe4, 0xb5, 0x6b, 0x7d, 0x56, 0x93, 0x2e, 0xee, 0x98, 0x29, 0x06, 0x7e, 0x06, + 0xd5, 0xf8, 0xe5, 0xc6, 0x2a, 0x45, 0xf1, 0x02, 0x8a, 0xba, 0x70, 0xc2, 0xf6, 0x75, 0x0e, 0x8e, + 0x5a, 0x86, 0xf7, 0xa0, 0xc6, 0xde, 0x3d, 0x11, 0x99, 0x38, 0x55, 0x55, 0x62, 0x9b, 0x21, 0x91, + 0xfa, 0x08, 0x00, 0x13, 0xc3, 0xf1, 0xcc, 0xa1, 0x6e, 0x79, 0xa8, 0x01, 0xab, 0xec, 0x99, 0x24, + 0x6a, 0x5e, 0xf6, 0x93, 0x3d, 0xa9, 0x2e, 0xf5, 0xe9, 0x8c, 0xc8, 0xc5, 0xe2, 0x43, 0xfd, 0x4f, + 0x01, 0x20, 0xea, 0x75, 0x24, 0xfa, 0x31, 0x4a, 0xa2, 0x1f, 0x83, 0x5a, 0x50, 0x32, 0x74, 0x57, + 0x37, 0x58, 0x41, 0x2b, 0x2b, 0xe6, 0xe0, 0x1b, 0xbd, 0x02, 0x65, 0xfd, 0x52, 0xb7, 0xa6, 0xfa, + 0xc9, 0x94, 0xf0, 0x68, 0xcb, 0xe3, 0x08, 0xc0, 0xaa, 0x0a, 0x19, 0x5d, 0x22, 0x1c, 0xf3, 0x3c, + 0x1c, 0x65, 0xaa, 0xe5, 0xf1, 0x88, 0xde, 0x02, 0x44, 0x65, 0xbd, 0x43, 0x6d, 0xdd, 0x95, 0x84, + 0x05, 0x4e, 0xd8, 0x90, 0x98, 0x91, 0xad, 0xbb, 0x82, 0xfa, 0x5d, 0xd8, 0xf2, 0x88, 0x41, 0xac, + 0xcb, 0x14, 0x7d, 0x91, 0xd3, 0xa3, 0x10, 0x17, 0xad, 0xb8, 0x0d, 0x10, 0x99, 0x9a, 0x27, 0xe8, + 0x1a, 0x2e, 0x87, 0x56, 0x46, 0xbb, 0xb0, 0xa9, 0xbb, 0xee, 0x74, 0x9e, 0xe2, 0x57, 0xe2, 0x74, + 0x1b, 0x01, 0x2a, 0x62, 0xb7, 0x03, 0x6b, 0x16, 0xd5, 0x4e, 0x66, 0x74, 0xce, 0x4b, 0xa0, 0x12, + 0x2e, 0x5a, 0x74, 0x6f, 0x46, 0xe7, 0xec, 0x1e, 0x9a, 0x51, 0x62, 0xc6, 0x2b, 0x9f, 0x12, 0x03, + 0xf0, 0x92, 0x67, 0xa1, 0x42, 0xab, 0x2c, 0xa9, 0xd0, 0xd2, 0x25, 0x58, 0x75, 0xb1, 0x04, 0x4b, + 0x16, 0x71, 0xb5, 0x74, 0x11, 0x97, 0xa8, 0xd0, 0xea, 0xa9, 0x0a, 0x2d, 0x5e, 0x76, 0xad, 0x5f, + 0xa3, 0xec, 0x7a, 0x07, 0xc0, 0x70, 0x67, 0xda, 0x8c, 0xea, 0x13, 0x42, 0x9b, 0x0d, 0x7e, 0x54, + 0x1a, 0x41, 0x94, 0x07, 0xd1, 0x86, 0xcb, 0x86, 0x3b, 0x3b, 0xe6, 0x24, 0xe8, 0x11, 0xd4, 0xd8, + 0xc6, 0x9a, 0xe5, 0x68, 0x9e, 0xee, 0x13, 0xda, 0xdc, 0xb8, 0x62, 0x4d, 0x85, 0x91, 0xf5, 0x1c, + 0xcc, 0x88, 0xd0, 0x07, 0x50, 0x67, 0x0a, 0x93, 0x68, 0x19, 0xba, 0x62, 0x59, 0x95, 0xd3, 0x05, + 0xeb, 0xde, 0x87, 0xaa, 0xe3, 0x6a, 0x53, 0xdd, 0x27, 0xb6, 0x61, 0x11, 0xda, 0xdc, 0xbc, 0x6a, + 0x33, 0xc7, 0x3d, 0x0c, 0x88, 0xd4, 0x29, 0xdc, 0xe0, 0x21, 0xff, 0xa2, 0x0f, 0x04, 0xd9, 0x37, + 0xcc, 0x3d, 0xbb, 0x6f, 0xf8, 0x18, 0xb6, 0xd3, 0xbb, 0x65, 0xca, 0x1e, 0x7f, 0x56, 0x60, 0x6b, + 0x64, 0xe8, 0x3e, 0x7b, 0x30, 0x67, 0x6f, 0x79, 0x3d, 0xab, 0xad, 0x73, 0xdd, 0xd6, 0x7b, 0xec, + 0xcd, 0x93, 0xbf, 0xfa, 0xcd, 0xa3, 0x76, 0xe1, 0x46, 0x4a, 0xde, 0xac, 0x4d, 0xfa, 0x03, 0xe2, + 0x1f, 0x74, 0x46, 0xfa, 0x29, 0x19, 0x3a, 0x96, 0x9d, 0xc9, 0x5b, 0x2a, 0x81, 0xed, 0x34, 0x97, + 0x4c, 0x17, 0x14, 0x4b, 0x24, 0xfa, 0x29, 0xd1, 0x5c, 0xc6, 0x43, 0x1a, 0xb0, 0x4c, 0x03, 0xa6, + 0xea, 0x29, 0x34, 0x8f, 0x5d, 0x53, 0xf7, 0xc9, 0x0b, 0xca, 0xfb, 0xbc, 0x7d, 0x1c, 0xb8, 0xb9, + 0x64, 0x9f, 0x4c, 0x1a, 0xdd, 0x87, 0x3a, 0xbb, 0xdb, 0x17, 0x76, 0x63, 0x37, 0x7e, 0xc8, 0x5b, + 0xfd, 0x8d, 0x02, 0x1b, 0xa3, 0xb9, 0x6d, 0xbc, 0x40, 0xe8, 0xdd, 0x87, 0xa2, 0xe8, 0x25, 0xc8, + 0x13, 0x93, 0x6a, 0x20, 0x08, 0x1c, 0x2f, 0x5d, 0x78, 0xa6, 0xb6, 0x6c, 0x93, 0x7c, 0x2d, 0x2f, + 0x13, 0x91, 0xbc, 0x7b, 0x0c, 0x22, 0x7a, 0x8f, 0x31, 0x49, 0x5e, 0x72, 0xab, 0xef, 0xb9, 0xf2, + 0x7c, 0xc9, 0x7b, 0x30, 0x03, 0x97, 0x78, 0xba, 0xef, 0x78, 0x3f, 0x7c, 0xaf, 0xf5, 0x6f, 0x0a, + 0xef, 0xe8, 0x47, 0x1b, 0x64, 0x52, 0xf8, 0x99, 0xe7, 0x1e, 0x41, 0xde, 0x24, 0xd4, 0xe0, 0xca, + 0x55, 0x31, 0xff, 0xcd, 0xd8, 0xb3, 0xfc, 0x35, 0xa3, 0xfc, 0x8c, 0xd7, 0x03, 0xf6, 0x81, 0x18, + 0x23, 0x8e, 0xc3, 0x92, 0x86, 0xbf, 0x27, 0x2d, 0xdb, 0xe4, 0x37, 0x36, 0x7b, 0x4f, 0x5a, 0xb6, + 0xf9, 0xe6, 0x77, 0x0a, 0x94, 0xc3, 0xa1, 0x2b, 0x2a, 0x42, 0x6e, 0xf0, 0xa4, 0xb1, 0x82, 0x2a, + 0xb0, 0x76, 0xdc, 0x7f, 0xd2, 0x1f, 0x7c, 0xde, 0x6f, 0x28, 0x68, 0x0b, 0x1a, 0xfd, 0xc1, 0x58, + 0xdb, 0x1b, 0x0c, 0xc6, 0xa3, 0x31, 0x6e, 0x0f, 0x87, 0xdd, 0xfd, 0x46, 0x0e, 0x6d, 0xc2, 0xfa, + 0x68, 0x3c, 0xc0, 0x5d, 0x6d, 0x3c, 0x38, 0xda, 0x1b, 0x8d, 0x07, 0xfd, 0x6e, 0x63, 0x15, 0x35, + 0x61, 0xab, 0x7d, 0x88, 0xbb, 0xed, 0xfd, 0x2f, 0x92, 0xe4, 0x79, 0x86, 0xe9, 0xf5, 0x3b, 0x83, + 0xa3, 0x61, 0x7b, 0xdc, 0xdb, 0x3b, 0xec, 0x6a, 0x9f, 0x75, 0xf1, 0xa8, 0x37, 0xe8, 0x37, 0x0a, + 0x8c, 0x3d, 0xee, 0x1e, 0xf4, 0x06, 0x7d, 0x8d, 0xed, 0xf2, 0x78, 0x70, 0xdc, 0xdf, 0x6f, 0x14, + 0xdf, 0x7c, 0x04, 0x95, 0xd8, 0x93, 0x18, 0x95, 0x20, 0x3f, 0xea, 0xb4, 0xfb, 0x8d, 0x15, 0xb4, + 0x0e, 0x95, 0xf6, 0x70, 0x88, 0x07, 0x3f, 0xed, 0x1d, 0xb5, 0xc7, 0xdd, 0x86, 0x82, 0x00, 0x8a, + 0xc7, 0xa3, 0xee, 0x93, 0xee, 0x17, 0x8d, 0xdc, 0x9b, 0x43, 0xa8, 0x27, 0x75, 0x67, 0x9a, 0x8c, + 0x8e, 0x3b, 0x9d, 0xee, 0x68, 0x24, 0xd4, 0x1a, 0xf7, 0x8e, 0xba, 0x83, 0xe3, 0xb1, 0x58, 0xd7, + 0x69, 0xf7, 0x3b, 0xdd, 0xc3, 0x46, 0x8e, 0x21, 0x70, 0x77, 0x78, 0xd8, 0xee, 0x30, 0x25, 0xd8, + 0xc7, 0x71, 0xbf, 0xdf, 0xeb, 0x1f, 0x34, 0xf2, 0x0f, 0xff, 0x52, 0x85, 0xdc, 0x70, 0x1f, 0xb5, + 0x01, 0xa2, 0x1e, 0x1e, 0xda, 0x11, 0x66, 0x5e, 0x68, 0x0c, 0xb6, 0x9a, 0x8b, 0x08, 0xe1, 0x68, + 0x75, 0x05, 0xbd, 0x0b, 0xab, 0x63, 0xea, 0x20, 0x79, 0x05, 0x45, 0xb3, 0xeb, 0xd6, 0x46, 0x0c, + 0x12, 0x50, 0x3f, 0x50, 0xde, 0x55, 0xd0, 0x4f, 0xa0, 0x1c, 0x4e, 0x2c, 0xd1, 0xb6, 0xa0, 0x4a, + 0xcf, 0x76, 0x5b, 0x3b, 0x0b, 0xf0, 0x70, 0xc7, 0x23, 0xa8, 0x27, 0x67, 0x9e, 0xe8, 0x96, 0x20, + 0x5e, 0x3a, 0x4f, 0x6d, 0xbd, 0xb2, 0x1c, 0x19, 0xb2, 0xfb, 0x10, 0xd6, 0xe4, 0x5c, 0x12, 0xc9, + 0x38, 0x4b, 0x4e, 0x39, 0x5b, 0x37, 0x52, 0xd0, 0x70, 0xe5, 0x8f, 0xa1, 0x14, 0x0c, 0x09, 0xd1, + 0x8d, 0xd0, 0x44, 0xf1, 0x29, 0x5d, 0x6b, 0x3b, 0x0d, 0x8e, 0x2f, 0x0e, 0xa6, 0x6d, 0xc1, 0xe2, + 0xd4, 0x88, 0x2f, 0x58, 0x9c, 0x1e, 0xca, 0xa9, 0x2b, 0xe8, 0x00, 0xaa, 0xf1, 0x51, 0x17, 0xba, + 0x19, 0x6e, 0x93, 0x1e, 0xbe, 0xb5, 0x5a, 0xcb, 0x50, 0x71, 0x5b, 0x26, 0x0b, 0x84, 0xc0, 0x96, + 0x4b, 0x8b, 0x94, 0xc0, 0x96, 0xcb, 0x6b, 0x0a, 0x75, 0x05, 0x8d, 0x61, 0x3d, 0xd5, 0xc5, 0x41, + 0xaf, 0x04, 0xa9, 0x61, 0x59, 0x5b, 0xb4, 0x75, 0xfb, 0x0a, 0x6c, 0x3a, 0x60, 0xc2, 0x99, 0x12, + 0x8a, 0x2c, 0x9a, 0xb8, 0x0e, 0x5a, 0x3b, 0x0b, 0xf0, 0x50, 0xaa, 0x3d, 0xa8, 0x1d, 0x10, 0x7f, + 0xe8, 0x91, 0xcb, 0xec, 0x3c, 0x1e, 0x73, 0x1e, 0xd1, 0x5c, 0x0b, 0xb5, 0x52, 0xb4, 0xb1, 0x61, + 0xd7, 0xb3, 0xf8, 0xec, 0x43, 0x25, 0x36, 0x2c, 0x42, 0xf2, 0x64, 0x2d, 0xce, 0xb3, 0x5a, 0x37, + 0x97, 0x60, 0x42, 0x2e, 0x9f, 0x42, 0x2d, 0xf1, 0xe2, 0x0d, 0xa4, 0x59, 0xf6, 0xca, 0x6f, 0xdd, + 0x5a, 0x8a, 0x0b, 0x79, 0x8d, 0xf8, 0x24, 0x33, 0x31, 0xf7, 0x40, 0xb7, 0x43, 0x05, 0x96, 0x8d, + 0x60, 0x5a, 0x77, 0xae, 0x42, 0xc7, 0x99, 0xa6, 0x87, 0x29, 0x01, 0xd3, 0x2b, 0xe6, 0x3a, 0x01, + 0xd3, 0xab, 0x66, 0x30, 0x42, 0xeb, 0x44, 0x51, 0x17, 0x68, 0xbd, 0xac, 0x32, 0x0d, 0xb4, 0x5e, + 0x5a, 0x05, 0x8a, 0xc0, 0x4f, 0xd6, 0x64, 0x41, 0xe0, 0x2f, 0xad, 0xf7, 0x82, 0xc0, 0x5f, 0x5e, + 0xc6, 0xa9, 0x2b, 0xe8, 0x33, 0xd8, 0x58, 0xa8, 0x89, 0x90, 0xd4, 0xe8, 0xaa, 0xa2, 0xac, 0xf5, + 0xea, 0x95, 0xf8, 0x58, 0xd8, 0x55, 0xa2, 0x7a, 0x23, 0xcc, 0xd0, 0x0b, 0xc5, 0x50, 0x90, 0xa1, + 0x17, 0x6b, 0x13, 0x79, 0x84, 0xf6, 0xa1, 0x12, 0xbb, 0xc7, 0x51, 0x94, 0xd0, 0x53, 0xb5, 0x43, + 0xeb, 0xe6, 0x12, 0x4c, 0xc0, 0x69, 0x4f, 0xfd, 0xc7, 0xf7, 0x77, 0x94, 0x7f, 0x7e, 0x7f, 0x47, + 0xf9, 0xd7, 0xf7, 0x77, 0x94, 0x3f, 0xfc, 0xfb, 0xce, 0x0a, 0x34, 0x1c, 0x6f, 0xb2, 0xeb, 0x5b, + 0xe7, 0x97, 0xbb, 0xe7, 0x97, 0xfc, 0x9f, 0xb1, 0x4e, 0x8a, 0xfc, 0xcf, 0xfb, 0xff, 0x0d, 0x00, + 0x00, 0xff, 0xff, 0xee, 0x5f, 0xea, 0xb3, 0xda, 0x25, 0x00, 0x00, } diff --git a/proto/proto/pdpb.proto b/proto/proto/pdpb.proto index 6cfdc99c5..ace74d2a3 100644 --- a/proto/proto/pdpb.proto +++ b/proto/proto/pdpb.proto @@ -43,20 +43,8 @@ service PD { rpc ScanRegions(ScanRegionsRequest) returns (ScanRegionsResponse) {} - rpc AskSplit(AskSplitRequest) returns (AskSplitResponse) { - // Use AskBatchSplit instead. - option deprecated = true; - } - - rpc ReportSplit(ReportSplitRequest) returns (ReportSplitResponse) { - // Use ResportBatchSplit instead. - option deprecated = true; - } - rpc AskBatchSplit(AskBatchSplitRequest) returns (AskBatchSplitResponse) {} - rpc ReportBatchSplit(ReportBatchSplitRequest) returns (ReportBatchSplitResponse) {} - rpc GetClusterConfig(GetClusterConfigRequest) returns (GetClusterConfigResponse) {} rpc PutClusterConfig(PutClusterConfigRequest) returns (PutClusterConfigResponse) {}