Add persistency of datanode cluster (#5387)

We save channels registered in datanode in etcd and restore the cluster info after
restarting.

Signed-off-by: sunby <bingyi.sun@zilliz.com>
pull/5779/head
sunby 2021-05-25 16:22:02 +08:00 committed by zhenshan.cao
parent 3662b3f0e5
commit 81ac20143b
10 changed files with 1020 additions and 130 deletions

View File

@ -0,0 +1,158 @@
// Copyright (C) 2019-2020 Zilliz. All rights reserved.
//
// Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software distributed under the License
// is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
// or implied. See the License for the specific language governing permissions and limitations under the License.
package dataservice
import (
"github.com/golang/protobuf/proto"
"github.com/milvus-io/milvus/internal/kv"
"github.com/milvus-io/milvus/internal/proto/datapb"
)
const clusterPrefix = "cluster-prefix/"
type dataNodeStatus int8
const (
online dataNodeStatus = iota
offline
)
type dataNodeInfo struct {
info *datapb.DataNodeInfo
status dataNodeStatus
}
type clusterNodeManager struct {
kv kv.TxnKV
dataNodes map[string]*dataNodeInfo
}
func newClusterNodeManager(kv kv.TxnKV) (*clusterNodeManager, error) {
c := &clusterNodeManager{
kv: kv,
dataNodes: make(map[string]*dataNodeInfo),
}
return c, c.loadFromKv()
}
func (c *clusterNodeManager) loadFromKv() error {
_, values, err := c.kv.LoadWithPrefix(clusterPrefix)
if err != nil {
return err
}
for _, v := range values {
info := &datapb.DataNodeInfo{}
if err := proto.UnmarshalText(v, info); err != nil {
return err
}
node := &dataNodeInfo{
info: info,
status: offline,
}
c.dataNodes[info.Address] = node
}
return nil
}
func (c *clusterNodeManager) updateCluster(dataNodes []*datapb.DataNodeInfo) *clusterDeltaChange {
newNodes := make([]string, 0)
offlines := make([]string, 0)
restarts := make([]string, 0)
for _, n := range dataNodes {
node, ok := c.dataNodes[n.Address]
if ok {
node.status = online
if node.info.Version != n.Version {
restarts = append(restarts, n.Address)
}
continue
}
newNode := &dataNodeInfo{
info: &datapb.DataNodeInfo{
Address: n.Address,
Version: n.Version,
Channels: []*datapb.ChannelStatus{},
},
status: online,
}
c.dataNodes[n.Address] = newNode
newNodes = append(newNodes, n.Address)
}
for nAddr, node := range c.dataNodes {
if node.status == offline {
offlines = append(offlines, nAddr)
}
}
return &clusterDeltaChange{
newNodes: newNodes,
offlines: offlines,
restarts: restarts,
}
}
func (c *clusterNodeManager) updateDataNodes(dataNodes []*datapb.DataNodeInfo) error {
for _, node := range dataNodes {
c.dataNodes[node.Address].info = node
}
return c.txnSaveNodes(dataNodes)
}
func (c *clusterNodeManager) getDataNodes(onlyOnline bool) map[string]*datapb.DataNodeInfo {
ret := make(map[string]*datapb.DataNodeInfo)
for k, v := range c.dataNodes {
if !onlyOnline || v.status == online {
ret[k] = proto.Clone(v.info).(*datapb.DataNodeInfo)
}
}
return ret
}
func (c *clusterNodeManager) register(n *datapb.DataNodeInfo) {
node, ok := c.dataNodes[n.Address]
if ok {
node.status = online
node.info.Version = n.Version
} else {
c.dataNodes[n.Address] = &dataNodeInfo{
info: n,
status: online,
}
}
}
func (c *clusterNodeManager) unregister(n *datapb.DataNodeInfo) {
node, ok := c.dataNodes[n.Address]
if !ok {
return
}
node.status = offline
}
func (c *clusterNodeManager) txnSaveNodes(nodes []*datapb.DataNodeInfo) error {
if len(nodes) == 0 {
return nil
}
data := make(map[string]string)
for _, n := range nodes {
c.dataNodes[n.Address].info = n
key := clusterPrefix + n.Address
value := proto.MarshalTextString(n)
data[key] = value
}
return c.kv.MultiSave(data)
}

View File

@ -0,0 +1,71 @@
// Copyright (C) 2019-2020 Zilliz. All rights reserved.
//
// Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software distributed under the License
// is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
// or implied. See the License for the specific language governing permissions and limitations under the License.
package dataservice
import (
"sync"
grpcdatanodeclient "github.com/milvus-io/milvus/internal/distributed/datanode/client"
"github.com/milvus-io/milvus/internal/types"
)
const retryTimes = 2
type sessionManager interface {
sendRequest(addr string, executor func(node types.DataNode) error) error
}
type clusterSessionManager struct {
mu sync.RWMutex
sessions map[string]types.DataNode
}
func newClusterSessionManager() *clusterSessionManager {
return &clusterSessionManager{sessions: make(map[string]types.DataNode)}
}
func (m *clusterSessionManager) createSession(addr string) error {
cli := grpcdatanodeclient.NewClient(addr)
if err := cli.Init(); err != nil {
return err
}
if err := cli.Start(); err != nil {
return err
}
m.sessions[addr] = cli
return nil
}
func (m *clusterSessionManager) getSession(addr string) types.DataNode {
return m.sessions[addr]
}
func (m *clusterSessionManager) hasSession(addr string) bool {
_, ok := m.sessions[addr]
return ok
}
func (m *clusterSessionManager) sendRequest(addr string, executor func(node types.DataNode) error) error {
m.mu.Lock()
defer m.mu.Unlock()
success := false
var err error
for i := 0; !success && i < retryTimes; i++ {
if i != 0 || !m.hasSession(addr) {
m.createSession(addr)
}
err = executor(m.getSession(addr))
if err == nil {
return nil
}
}
return err
}

View File

@ -0,0 +1,229 @@
// Copyright (C) 2019-2020 Zilliz. All rights reserved.
//
// Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software distributed under the License
// is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
// or implied. See the License for the specific language governing permissions and limitations under the License.
package dataservice
import (
"sync"
"github.com/milvus-io/milvus/internal/log"
"github.com/milvus-io/milvus/internal/proto/commonpb"
"github.com/milvus-io/milvus/internal/proto/datapb"
"github.com/milvus-io/milvus/internal/types"
"go.uber.org/zap"
"golang.org/x/net/context"
)
type cluster struct {
mu sync.RWMutex
dataManager *clusterNodeManager
sessionManager sessionManager
startupPolicy clusterStartupPolicy
registerPolicy dataNodeRegisterPolicy
unregisterPolicy dataNodeUnregisterPolicy
assginPolicy channelAssignPolicy
}
type clusterOption struct {
apply func(c *cluster)
}
func withStartupPolicy(p clusterStartupPolicy) clusterOption {
return clusterOption{
apply: func(c *cluster) { c.startupPolicy = p },
}
}
func withRegisterPolicy(p dataNodeRegisterPolicy) clusterOption {
return clusterOption{
apply: func(c *cluster) { c.registerPolicy = p },
}
}
func withUnregistorPolicy(p dataNodeUnregisterPolicy) clusterOption {
return clusterOption{
apply: func(c *cluster) { c.unregisterPolicy = p },
}
}
func withAssignPolicy(p channelAssignPolicy) clusterOption {
return clusterOption{
apply: func(c *cluster) { c.assginPolicy = p },
}
}
func defaultStartupPolicy() clusterStartupPolicy {
return newReWatchOnRestartsStartupPolicy()
}
func defaultRegisterPolicy() dataNodeRegisterPolicy {
return newDoNothingRegisterPolicy()
}
func defaultUnregisterPolicy() dataNodeUnregisterPolicy {
return newDoNothingUnregisterPolicy()
}
func defaultAssignPolicy() channelAssignPolicy {
return newAllAssignPolicy()
}
func newCluster(dataManager *clusterNodeManager, sessionManager sessionManager, opts ...clusterOption) *cluster {
c := &cluster{
dataManager: dataManager,
sessionManager: sessionManager,
}
c.startupPolicy = defaultStartupPolicy()
c.registerPolicy = defaultRegisterPolicy()
c.unregisterPolicy = defaultUnregisterPolicy()
c.assginPolicy = defaultAssignPolicy()
for _, opt := range opts {
opt.apply(c)
}
return c
}
func (c *cluster) startup(dataNodes []*datapb.DataNodeInfo) error {
deltaChange := c.dataManager.updateCluster(dataNodes)
nodes := c.dataManager.getDataNodes(false)
rets := c.startupPolicy.apply(nodes, deltaChange)
c.dataManager.updateDataNodes(rets)
rets = c.watch(rets)
c.dataManager.updateDataNodes(rets)
return nil
}
func (c *cluster) watch(nodes []*datapb.DataNodeInfo) []*datapb.DataNodeInfo {
for _, n := range nodes {
uncompletes := make([]string, 0)
for _, ch := range n.Channels {
if ch.State == datapb.ChannelWatchState_Uncomplete {
uncompletes = append(uncompletes, ch.Name)
}
}
executor := func(cli types.DataNode) error {
req := &datapb.WatchDmChannelsRequest{
Base: &commonpb.MsgBase{
SourceID: Params.NodeID,
},
ChannelNames: uncompletes,
}
resp, err := cli.WatchDmChannels(context.Background(), req)
if err != nil {
return err
}
if resp.ErrorCode != commonpb.ErrorCode_Success {
log.Warn("watch channels failed", zap.String("address", n.Address), zap.Error(err))
return nil
}
for _, ch := range n.Channels {
if ch.State == datapb.ChannelWatchState_Uncomplete {
ch.State = datapb.ChannelWatchState_Complete
}
}
return nil
}
if err := c.sessionManager.sendRequest(n.Address, executor); err != nil {
log.Warn("watch channels failed", zap.String("address", n.Address), zap.Error(err))
}
}
return nodes
}
func (c *cluster) register(n *datapb.DataNodeInfo) {
c.mu.Lock()
defer c.mu.Unlock()
c.dataManager.register(n)
cNodes := c.dataManager.getDataNodes(true)
rets := c.registerPolicy.apply(cNodes, n)
c.dataManager.updateDataNodes(rets)
rets = c.watch(rets)
c.dataManager.updateDataNodes(rets)
}
func (c *cluster) unregister(n *datapb.DataNodeInfo) {
c.mu.Lock()
defer c.mu.Unlock()
c.dataManager.unregister(n)
cNodes := c.dataManager.getDataNodes(true)
rets := c.unregisterPolicy.apply(cNodes, n)
c.dataManager.updateDataNodes(rets)
rets = c.watch(rets)
c.dataManager.updateDataNodes(rets)
}
func (c *cluster) watchIfNeeded(channel string) {
c.mu.Lock()
defer c.mu.Unlock()
cNodes := c.dataManager.getDataNodes(true)
rets := c.assginPolicy.apply(cNodes, channel)
c.dataManager.updateDataNodes(rets)
rets = c.watch(rets)
c.dataManager.updateDataNodes(rets)
}
func (c *cluster) flush(segments []*datapb.SegmentInfo) {
c.mu.Lock()
defer c.mu.Unlock()
m := make(map[string]map[UniqueID][]UniqueID) // channel-> map[collectionID]segmentIDs
for _, seg := range segments {
if _, ok := m[seg.InsertChannel]; !ok {
m[seg.InsertChannel] = make(map[UniqueID][]UniqueID)
}
m[seg.InsertChannel][seg.CollectionID] = append(m[seg.InsertChannel][seg.CollectionID], seg.ID)
}
dataNodes := c.dataManager.getDataNodes(true)
channel2Node := make(map[string]string)
for _, node := range dataNodes {
for _, chstatus := range node.Channels {
channel2Node[chstatus.Name] = node.Address
}
}
for ch, coll2seg := range m {
node, ok := channel2Node[ch]
if !ok {
continue
}
for coll, segs := range coll2seg {
executor := func(cli types.DataNode) error {
req := &datapb.FlushSegmentsRequest{
Base: &commonpb.MsgBase{
MsgType: commonpb.MsgType_Flush,
SourceID: Params.NodeID,
},
CollectionID: coll,
SegmentIDs: segs,
}
resp, err := cli.FlushSegments(context.Background(), req)
if err != nil {
return err
}
if resp.ErrorCode != commonpb.ErrorCode_Success {
log.Warn("flush segment error", zap.String("dataNode", node), zap.Error(err))
}
return nil
}
if err := c.sessionManager.sendRequest(node, executor); err != nil {
log.Warn("flush segment error", zap.String("dataNode", node), zap.Error(err))
}
}
}
}

View File

@ -0,0 +1,142 @@
// Copyright (C) 2019-2020 Zilliz. All rights reserved.
//
// Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software distributed under the License
// is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
// or implied. See the License for the specific language governing permissions and limitations under the License.
package dataservice
import (
"testing"
memkv "github.com/milvus-io/milvus/internal/kv/mem"
"github.com/milvus-io/milvus/internal/proto/datapb"
"github.com/stretchr/testify/assert"
)
func TestClusterCreate(t *testing.T) {
cPolicy := newMockStartupPolicy()
cluster := createCluster(t, withStartupPolicy(cPolicy))
addr := "localhost:8080"
nodes := []*datapb.DataNodeInfo{
{
Address: addr,
Version: 1,
Channels: []*datapb.ChannelStatus{},
},
}
err := cluster.startup(nodes)
assert.Nil(t, err)
dataNodes := cluster.dataManager.getDataNodes(true)
assert.EqualValues(t, 1, len(dataNodes))
assert.EqualValues(t, "localhost:8080", dataNodes[addr].Address)
}
func TestRegister(t *testing.T) {
cPolicy := newMockStartupPolicy()
registerPolicy := newDoNothingRegisterPolicy()
cluster := createCluster(t, withStartupPolicy(cPolicy), withRegisterPolicy(registerPolicy))
addr := "localhost:8080"
err := cluster.startup(nil)
assert.Nil(t, err)
cluster.register(&datapb.DataNodeInfo{
Address: addr,
Version: 1,
Channels: []*datapb.ChannelStatus{},
})
dataNodes := cluster.dataManager.getDataNodes(true)
assert.EqualValues(t, 1, len(dataNodes))
assert.EqualValues(t, "localhost:8080", dataNodes[addr].Address)
}
func TestUnregister(t *testing.T) {
cPolicy := newMockStartupPolicy()
unregisterPolicy := newDoNothingUnregisterPolicy()
cluster := createCluster(t, withStartupPolicy(cPolicy), withUnregistorPolicy(unregisterPolicy))
addr := "localhost:8080"
nodes := []*datapb.DataNodeInfo{
{
Address: addr,
Version: 1,
Channels: []*datapb.ChannelStatus{},
},
}
err := cluster.startup(nodes)
assert.Nil(t, err)
dataNodes := cluster.dataManager.getDataNodes(true)
assert.EqualValues(t, 1, len(dataNodes))
assert.EqualValues(t, "localhost:8080", dataNodes[addr].Address)
cluster.unregister(&datapb.DataNodeInfo{
Address: addr,
Version: 1,
Channels: []*datapb.ChannelStatus{},
})
dataNodes = cluster.dataManager.getDataNodes(false)
assert.EqualValues(t, 1, len(dataNodes))
assert.EqualValues(t, offline, cluster.dataManager.dataNodes[addr].status)
assert.EqualValues(t, "localhost:8080", dataNodes[addr].Address)
}
func TestWatchIfNeeded(t *testing.T) {
cPolicy := newMockStartupPolicy()
cluster := createCluster(t, withStartupPolicy(cPolicy))
addr := "localhost:8080"
nodes := []*datapb.DataNodeInfo{
{
Address: addr,
Version: 1,
Channels: []*datapb.ChannelStatus{},
},
}
err := cluster.startup(nodes)
assert.Nil(t, err)
dataNodes := cluster.dataManager.getDataNodes(true)
assert.EqualValues(t, 1, len(dataNodes))
assert.EqualValues(t, "localhost:8080", dataNodes[addr].Address)
chName := "ch1"
cluster.watchIfNeeded(chName)
dataNodes = cluster.dataManager.getDataNodes(true)
assert.EqualValues(t, 1, len(dataNodes[addr].Channels))
assert.EqualValues(t, chName, dataNodes[addr].Channels[0].Name)
cluster.watchIfNeeded(chName)
assert.EqualValues(t, 1, len(dataNodes[addr].Channels))
assert.EqualValues(t, chName, dataNodes[addr].Channels[0].Name)
}
func TestFlushSegments(t *testing.T) {
cPolicy := newMockStartupPolicy()
cluster := createCluster(t, withStartupPolicy(cPolicy))
addr := "localhost:8080"
nodes := []*datapb.DataNodeInfo{
{
Address: addr,
Version: 1,
Channels: []*datapb.ChannelStatus{},
},
}
err := cluster.startup(nodes)
assert.Nil(t, err)
segments := []*datapb.SegmentInfo{
{
ID: 0,
CollectionID: 0,
InsertChannel: "ch1",
},
}
cluster.flush(segments)
}
func createCluster(t *testing.T, options ...clusterOption) *cluster {
kv := memkv.NewMemoryKV()
sessionManager := newMockSessionManager()
dataManager, err := newClusterNodeManager(kv)
assert.Nil(t, err)
return newCluster(dataManager, sessionManager, options...)
}

View File

@ -16,6 +16,7 @@ import (
"time"
memkv "github.com/milvus-io/milvus/internal/kv/mem"
"github.com/milvus-io/milvus/internal/types"
"github.com/milvus-io/milvus/internal/util/tsoutil"
"github.com/milvus-io/milvus/internal/proto/commonpb"
@ -287,3 +288,25 @@ func (m *mockMasterService) GetDdChannel(ctx context.Context) (*milvuspb.StringR
func (m *mockMasterService) UpdateChannelTimeTick(ctx context.Context, req *internalpb.ChannelTimeTickMsg) (*commonpb.Status, error) {
panic("not implemented") // TODO: Implement
}
type mockStartupPolicy struct {
}
func newMockStartupPolicy() clusterStartupPolicy {
return &mockStartupPolicy{}
}
func (p *mockStartupPolicy) apply(oldCluster map[string]*datapb.DataNodeInfo, delta *clusterDeltaChange) []*datapb.DataNodeInfo {
return nil
}
type mockSessionManager struct {
}
func newMockSessionManager() sessionManager {
return &mockSessionManager{}
}
func (m *mockSessionManager) sendRequest(addr string, executor func(node types.DataNode) error) error {
return nil
}

View File

@ -0,0 +1,111 @@
// Copyright (C) 2019-2020 Zilliz. All rights reserved.
//
// Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software distributed under the License
// is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
// or implied. See the License for the specific language governing permissions and limitations under the License.
package dataservice
import (
"fmt"
"github.com/milvus-io/milvus/internal/proto/datapb"
)
type clusterDeltaChange struct {
newNodes []string
offlines []string
restarts []string
}
type clusterStartupPolicy interface {
apply(oldCluster map[string]*datapb.DataNodeInfo, delta *clusterDeltaChange) []*datapb.DataNodeInfo
}
type reWatchOnRestartsStartupPolicy struct {
}
func newReWatchOnRestartsStartupPolicy() clusterStartupPolicy {
return &reWatchOnRestartsStartupPolicy{}
}
func (p *reWatchOnRestartsStartupPolicy) apply(cluster map[string]*datapb.DataNodeInfo, delta *clusterDeltaChange) []*datapb.DataNodeInfo {
ret := make([]*datapb.DataNodeInfo, 0)
for _, addr := range delta.restarts {
node := cluster[addr]
for _, ch := range node.Channels {
ch.State = datapb.ChannelWatchState_Uncomplete
}
ret = append(ret, node)
}
return ret
}
type dataNodeRegisterPolicy interface {
apply(cluster map[string]*datapb.DataNodeInfo, session *datapb.DataNodeInfo) []*datapb.DataNodeInfo
}
type doNothingRegisterPolicy struct {
}
func newDoNothingRegisterPolicy() dataNodeRegisterPolicy {
return &doNothingRegisterPolicy{}
}
func (p *doNothingRegisterPolicy) apply(cluster map[string]*datapb.DataNodeInfo, session *datapb.DataNodeInfo) []*datapb.DataNodeInfo {
return []*datapb.DataNodeInfo{session}
}
type dataNodeUnregisterPolicy interface {
apply(cluster map[string]*datapb.DataNodeInfo, session *datapb.DataNodeInfo) []*datapb.DataNodeInfo
}
type doNothingUnregisterPolicy struct {
}
func newDoNothingUnregisterPolicy() dataNodeUnregisterPolicy {
return &doNothingUnregisterPolicy{}
}
func (p *doNothingUnregisterPolicy) apply(cluster map[string]*datapb.DataNodeInfo, session *datapb.DataNodeInfo) []*datapb.DataNodeInfo {
return nil
}
type channelAssignPolicy interface {
apply(cluster map[string]*datapb.DataNodeInfo, channel string) []*datapb.DataNodeInfo
}
type allAssignPolicy struct {
}
func newAllAssignPolicy() channelAssignPolicy {
return &allAssignPolicy{}
}
func (p *allAssignPolicy) apply(cluster map[string]*datapb.DataNodeInfo, channel string) []*datapb.DataNodeInfo {
ret := make([]*datapb.DataNodeInfo, 0)
for _, node := range cluster {
fmt.Printf("xxxxnode: %v\n", node.Address)
has := false
for _, ch := range node.Channels {
if ch.Name == channel {
has = true
break
}
}
if has {
continue
}
node.Channels = append(node.Channels, &datapb.ChannelStatus{
Name: channel,
State: datapb.ChannelWatchState_Uncomplete,
})
fmt.Printf("channelxxxx: %v\n", node.Channels)
ret = append(ret, node)
}
return ret
}

View File

@ -323,6 +323,7 @@ func (s *Server) startStatsChannel(ctx context.Context) {
log.Debug("DataService AsConsumer: " + Params.StatisticsChannelName + " : " + Params.DataServiceSubscriptionName)
// try to restore last processed pos
pos, err := s.loadStreamLastPos(streamTypeStats)
log.Debug("load last pos of stats channel", zap.Any("pos", pos))
if err == nil {
err = statsStream.Seek([]*internalpb.MsgPosition{pos})
if err != nil {

View File

@ -18,6 +18,7 @@ import (
"testing"
"time"
"github.com/milvus-io/milvus/internal/log"
"github.com/milvus-io/milvus/internal/msgstream"
"github.com/milvus-io/milvus/internal/proto/commonpb"
"github.com/milvus-io/milvus/internal/proto/datapb"
@ -28,6 +29,7 @@ import (
"github.com/milvus-io/milvus/internal/util/typeutil"
"github.com/stretchr/testify/assert"
"go.etcd.io/etcd/clientv3"
"go.uber.org/zap"
)
func TestRegisterNode(t *testing.T) {
@ -796,6 +798,7 @@ func TestResumeChannel(t *testing.T) {
defer svr.meta.RUnlock()
for _, segID := range segmentIDs {
seg, has := svr.meta.segments[segID]
log.Debug("check segment in meta", zap.Any("id", seg.ID), zap.Any("has", has))
assert.True(t, has)
if has {
assert.Equal(t, segRows, seg.NumRows)

View File

@ -276,6 +276,22 @@ message FieldBinlog{
repeated string binlogs = 2;
}
enum ChannelWatchState {
Uncomplete = 0;
Complete = 1;
}
message ChannelStatus {
string name = 1;
ChannelWatchState state=2;
}
message DataNodeInfo {
string address = 1;
int64 version = 2;
repeated ChannelStatus channels = 3;
}
message GetRecoveryInfoResponse {
common.MsgBase base = 1;
repeated VchannelInfo channels = 2;

View File

@ -28,6 +28,31 @@ var _ = math.Inf
// proto package needs to be updated.
const _ = proto.ProtoPackageIsVersion3 // please upgrade the proto package
type ChannelWatchState int32
const (
ChannelWatchState_Uncomplete ChannelWatchState = 0
ChannelWatchState_Complete ChannelWatchState = 1
)
var ChannelWatchState_name = map[int32]string{
0: "Uncomplete",
1: "Complete",
}
var ChannelWatchState_value = map[string]int32{
"Uncomplete": 0,
"Complete": 1,
}
func (x ChannelWatchState) String() string {
return proto.EnumName(ChannelWatchState_name, int32(x))
}
func (ChannelWatchState) EnumDescriptor() ([]byte, []int) {
return fileDescriptor_3385cd32ad6cfe64, []int{0}
}
type RegisterNodeRequest struct {
Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"`
Address *commonpb.Address `protobuf:"bytes,2,opt,name=address,proto3" json:"address,omitempty"`
@ -2167,6 +2192,108 @@ func (m *FieldBinlog) GetBinlogs() []string {
return nil
}
type ChannelStatus struct {
Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"`
State ChannelWatchState `protobuf:"varint,2,opt,name=state,proto3,enum=milvus.proto.data.ChannelWatchState" json:"state,omitempty"`
XXX_NoUnkeyedLiteral struct{} `json:"-"`
XXX_unrecognized []byte `json:"-"`
XXX_sizecache int32 `json:"-"`
}
func (m *ChannelStatus) Reset() { *m = ChannelStatus{} }
func (m *ChannelStatus) String() string { return proto.CompactTextString(m) }
func (*ChannelStatus) ProtoMessage() {}
func (*ChannelStatus) Descriptor() ([]byte, []int) {
return fileDescriptor_3385cd32ad6cfe64, []int{39}
}
func (m *ChannelStatus) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_ChannelStatus.Unmarshal(m, b)
}
func (m *ChannelStatus) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
return xxx_messageInfo_ChannelStatus.Marshal(b, m, deterministic)
}
func (m *ChannelStatus) XXX_Merge(src proto.Message) {
xxx_messageInfo_ChannelStatus.Merge(m, src)
}
func (m *ChannelStatus) XXX_Size() int {
return xxx_messageInfo_ChannelStatus.Size(m)
}
func (m *ChannelStatus) XXX_DiscardUnknown() {
xxx_messageInfo_ChannelStatus.DiscardUnknown(m)
}
var xxx_messageInfo_ChannelStatus proto.InternalMessageInfo
func (m *ChannelStatus) GetName() string {
if m != nil {
return m.Name
}
return ""
}
func (m *ChannelStatus) GetState() ChannelWatchState {
if m != nil {
return m.State
}
return ChannelWatchState_Uncomplete
}
type DataNodeInfo struct {
Address string `protobuf:"bytes,1,opt,name=address,proto3" json:"address,omitempty"`
Version int64 `protobuf:"varint,2,opt,name=version,proto3" json:"version,omitempty"`
Channels []*ChannelStatus `protobuf:"bytes,3,rep,name=channels,proto3" json:"channels,omitempty"`
XXX_NoUnkeyedLiteral struct{} `json:"-"`
XXX_unrecognized []byte `json:"-"`
XXX_sizecache int32 `json:"-"`
}
func (m *DataNodeInfo) Reset() { *m = DataNodeInfo{} }
func (m *DataNodeInfo) String() string { return proto.CompactTextString(m) }
func (*DataNodeInfo) ProtoMessage() {}
func (*DataNodeInfo) Descriptor() ([]byte, []int) {
return fileDescriptor_3385cd32ad6cfe64, []int{40}
}
func (m *DataNodeInfo) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_DataNodeInfo.Unmarshal(m, b)
}
func (m *DataNodeInfo) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
return xxx_messageInfo_DataNodeInfo.Marshal(b, m, deterministic)
}
func (m *DataNodeInfo) XXX_Merge(src proto.Message) {
xxx_messageInfo_DataNodeInfo.Merge(m, src)
}
func (m *DataNodeInfo) XXX_Size() int {
return xxx_messageInfo_DataNodeInfo.Size(m)
}
func (m *DataNodeInfo) XXX_DiscardUnknown() {
xxx_messageInfo_DataNodeInfo.DiscardUnknown(m)
}
var xxx_messageInfo_DataNodeInfo proto.InternalMessageInfo
func (m *DataNodeInfo) GetAddress() string {
if m != nil {
return m.Address
}
return ""
}
func (m *DataNodeInfo) GetVersion() int64 {
if m != nil {
return m.Version
}
return 0
}
func (m *DataNodeInfo) GetChannels() []*ChannelStatus {
if m != nil {
return m.Channels
}
return nil
}
type GetRecoveryInfoResponse struct {
Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"`
Channels []*VchannelInfo `protobuf:"bytes,2,rep,name=channels,proto3" json:"channels,omitempty"`
@ -2180,7 +2307,7 @@ func (m *GetRecoveryInfoResponse) Reset() { *m = GetRecoveryInfoResponse
func (m *GetRecoveryInfoResponse) String() string { return proto.CompactTextString(m) }
func (*GetRecoveryInfoResponse) ProtoMessage() {}
func (*GetRecoveryInfoResponse) Descriptor() ([]byte, []int) {
return fileDescriptor_3385cd32ad6cfe64, []int{39}
return fileDescriptor_3385cd32ad6cfe64, []int{41}
}
func (m *GetRecoveryInfoResponse) XXX_Unmarshal(b []byte) error {
@ -2235,7 +2362,7 @@ func (m *GetRecoveryInfoRequest) Reset() { *m = GetRecoveryInfoRequest{}
func (m *GetRecoveryInfoRequest) String() string { return proto.CompactTextString(m) }
func (*GetRecoveryInfoRequest) ProtoMessage() {}
func (*GetRecoveryInfoRequest) Descriptor() ([]byte, []int) {
return fileDescriptor_3385cd32ad6cfe64, []int{40}
return fileDescriptor_3385cd32ad6cfe64, []int{42}
}
func (m *GetRecoveryInfoRequest) XXX_Unmarshal(b []byte) error {
@ -2290,7 +2417,7 @@ func (m *DataNodeTtMsg) Reset() { *m = DataNodeTtMsg{} }
func (m *DataNodeTtMsg) String() string { return proto.CompactTextString(m) }
func (*DataNodeTtMsg) ProtoMessage() {}
func (*DataNodeTtMsg) Descriptor() ([]byte, []int) {
return fileDescriptor_3385cd32ad6cfe64, []int{41}
return fileDescriptor_3385cd32ad6cfe64, []int{43}
}
func (m *DataNodeTtMsg) XXX_Unmarshal(b []byte) error {
@ -2333,6 +2460,7 @@ func (m *DataNodeTtMsg) GetTimestamp() uint64 {
}
func init() {
proto.RegisterEnum("milvus.proto.data.ChannelWatchState", ChannelWatchState_name, ChannelWatchState_value)
proto.RegisterType((*RegisterNodeRequest)(nil), "milvus.proto.data.RegisterNodeRequest")
proto.RegisterType((*RegisterNodeResponse)(nil), "milvus.proto.data.RegisterNodeResponse")
proto.RegisterType((*FlushRequest)(nil), "milvus.proto.data.FlushRequest")
@ -2372,6 +2500,8 @@ func init() {
proto.RegisterType((*VchannelInfo)(nil), "milvus.proto.data.VchannelInfo")
proto.RegisterType((*SegmentBinlogs)(nil), "milvus.proto.data.SegmentBinlogs")
proto.RegisterType((*FieldBinlog)(nil), "milvus.proto.data.FieldBinlog")
proto.RegisterType((*ChannelStatus)(nil), "milvus.proto.data.ChannelStatus")
proto.RegisterType((*DataNodeInfo)(nil), "milvus.proto.data.DataNodeInfo")
proto.RegisterType((*GetRecoveryInfoResponse)(nil), "milvus.proto.data.GetRecoveryInfoResponse")
proto.RegisterType((*GetRecoveryInfoRequest)(nil), "milvus.proto.data.GetRecoveryInfoRequest")
proto.RegisterType((*DataNodeTtMsg)(nil), "milvus.proto.data.DataNodeTtMsg")
@ -2380,133 +2510,139 @@ func init() {
func init() { proto.RegisterFile("data_service.proto", fileDescriptor_3385cd32ad6cfe64) }
var fileDescriptor_3385cd32ad6cfe64 = []byte{
// 2003 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xc4, 0x59, 0xdb, 0x6f, 0x1b, 0x59,
0x19, 0xcf, 0xd8, 0xb9, 0xf9, 0xf3, 0xd8, 0x69, 0x4f, 0x43, 0xd6, 0xb8, 0x6d, 0x9a, 0xce, 0xb2,
0xdd, 0x6c, 0x57, 0x24, 0x5b, 0x17, 0x71, 0x51, 0x05, 0xa8, 0xa9, 0xb7, 0xc1, 0xa2, 0x29, 0xe1,
0xa4, 0xec, 0x4a, 0xac, 0x90, 0x35, 0xf1, 0x9c, 0x38, 0x43, 0xe6, 0xe2, 0x9d, 0x33, 0x6e, 0xd3,
0xa7, 0xa2, 0x85, 0x07, 0x84, 0x10, 0x85, 0xff, 0x00, 0x90, 0x90, 0x90, 0xe0, 0x81, 0x67, 0xde,
0xf9, 0x17, 0xf8, 0x7b, 0x56, 0xe7, 0x32, 0xf7, 0x63, 0x7b, 0xea, 0xb4, 0xca, 0x9b, 0xcf, 0xf1,
0x77, 0x3b, 0xdf, 0xe5, 0x77, 0xbe, 0xf3, 0x0d, 0x20, 0xcb, 0x0c, 0xcd, 0x3e, 0x25, 0xc1, 0x73,
0x7b, 0x40, 0x76, 0x46, 0x81, 0x1f, 0xfa, 0xe8, 0xaa, 0x6b, 0x3b, 0xcf, 0xc7, 0x54, 0xac, 0x76,
0x18, 0x41, 0x5b, 0x1f, 0xf8, 0xae, 0xeb, 0x7b, 0x62, 0xab, 0xdd, 0xb4, 0xbd, 0x90, 0x04, 0x9e,
0xe9, 0xc8, 0xb5, 0x9e, 0x66, 0x68, 0xeb, 0x74, 0x70, 0x4a, 0x5c, 0x53, 0xac, 0x8c, 0x57, 0x70,
0x0d, 0x93, 0xa1, 0x4d, 0x43, 0x12, 0x3c, 0xf5, 0x2d, 0x82, 0xc9, 0x97, 0x63, 0x42, 0x43, 0xf4,
0x09, 0x2c, 0x1e, 0x9b, 0x94, 0xb4, 0xb4, 0x2d, 0x6d, 0xbb, 0xde, 0xb9, 0xb1, 0x93, 0x51, 0x29,
0x95, 0x1d, 0xd0, 0xe1, 0x9e, 0x49, 0x09, 0xe6, 0x94, 0xe8, 0xbb, 0xb0, 0x62, 0x5a, 0x56, 0x40,
0x28, 0x6d, 0x55, 0xa6, 0x30, 0x3d, 0x14, 0x34, 0x38, 0x22, 0x36, 0x5e, 0x6b, 0xb0, 0x9e, 0xb5,
0x80, 0x8e, 0x7c, 0x8f, 0x12, 0xb4, 0x07, 0x75, 0xdb, 0xb3, 0xc3, 0xfe, 0xc8, 0x0c, 0x4c, 0x97,
0x4a, 0x4b, 0x6e, 0x67, 0x85, 0xc6, 0x07, 0xed, 0x79, 0x76, 0x78, 0xc8, 0x09, 0x31, 0xd8, 0xf1,
0x6f, 0x74, 0x1f, 0x96, 0x69, 0x68, 0x86, 0xe3, 0xc8, 0xa6, 0xeb, 0x4a, 0x9b, 0x8e, 0x38, 0x09,
0x96, 0xa4, 0xc6, 0x39, 0xe8, 0x8f, 0x9d, 0x31, 0x3d, 0x9d, 0xdf, 0x17, 0x08, 0x16, 0xad, 0xe3,
0x5e, 0x97, 0x2b, 0xad, 0x62, 0xfe, 0x1b, 0x19, 0xa0, 0x0f, 0x7c, 0xc7, 0x21, 0x83, 0xd0, 0xf6,
0xbd, 0x5e, 0xb7, 0xb5, 0xc8, 0xff, 0xcb, 0xec, 0x19, 0x7f, 0xd1, 0xe0, 0xca, 0x11, 0x19, 0xba,
0xc4, 0x0b, 0x7b, 0xdd, 0x48, 0xfd, 0x3a, 0x2c, 0x0d, 0xfc, 0xb1, 0x17, 0x72, 0xfd, 0x0d, 0x2c,
0x16, 0xe8, 0x36, 0xe8, 0x83, 0x53, 0xd3, 0xf3, 0x88, 0xd3, 0xf7, 0x4c, 0x97, 0x70, 0x55, 0x35,
0x5c, 0x97, 0x7b, 0x4f, 0x4d, 0x97, 0x14, 0x34, 0x56, 0x8b, 0x1a, 0xd1, 0x16, 0xd4, 0x47, 0x66,
0x10, 0xda, 0x19, 0xa3, 0xd2, 0x5b, 0xc6, 0x5f, 0x35, 0xd8, 0x78, 0x48, 0xa9, 0x3d, 0xf4, 0x0a,
0x96, 0x6d, 0xc0, 0xb2, 0xe7, 0x5b, 0xa4, 0xd7, 0xe5, 0xa6, 0x55, 0xb1, 0x5c, 0xa1, 0xeb, 0x50,
0x1b, 0x11, 0x12, 0xf4, 0x03, 0xdf, 0x89, 0x0c, 0x5b, 0x65, 0x1b, 0xd8, 0x77, 0x08, 0xfa, 0x39,
0x5c, 0xa5, 0x39, 0x41, 0xb4, 0x55, 0xdd, 0xaa, 0x6e, 0xd7, 0x3b, 0xef, 0xef, 0x14, 0x32, 0x7b,
0x27, 0xaf, 0x14, 0x17, 0xb9, 0x8d, 0xdf, 0x54, 0xe0, 0x5a, 0x4c, 0x27, 0x6c, 0x65, 0xbf, 0x99,
0xe7, 0x28, 0x19, 0xc6, 0xe6, 0x89, 0x45, 0x19, 0xcf, 0xc5, 0x2e, 0xaf, 0xa6, 0x5d, 0x5e, 0x22,
0x82, 0x79, 0x7f, 0x2e, 0x15, 0xfc, 0x89, 0x6e, 0x41, 0x9d, 0x9c, 0x8f, 0xec, 0x80, 0xf4, 0x43,
0xdb, 0x25, 0xad, 0xe5, 0x2d, 0x6d, 0x7b, 0x11, 0x83, 0xd8, 0x7a, 0x66, 0xbb, 0x24, 0x95, 0xb3,
0x2b, 0xe5, 0x73, 0xf6, 0xef, 0x1a, 0xbc, 0x57, 0x88, 0x92, 0x2c, 0x24, 0x0c, 0x57, 0xf8, 0xc9,
0x13, 0xcf, 0xb0, 0x6a, 0x62, 0x0e, 0xbf, 0x33, 0xcd, 0xe1, 0x09, 0x39, 0x2e, 0xf0, 0xcf, 0x57,
0x58, 0x7f, 0xd3, 0xe0, 0xda, 0xd1, 0xa9, 0xff, 0x42, 0xaa, 0xa0, 0xf3, 0x17, 0x58, 0x3e, 0x14,
0x95, 0xd9, 0xa1, 0xa8, 0x16, 0x43, 0x11, 0x95, 0xe9, 0x62, 0x52, 0xa6, 0xc6, 0x19, 0xac, 0x67,
0x4d, 0x94, 0x4e, 0xdc, 0x04, 0x88, 0x13, 0x4f, 0xb8, 0xaf, 0x8a, 0x53, 0x3b, 0xf3, 0x39, 0xe4,
0x0c, 0xde, 0xdb, 0x27, 0xa1, 0xd4, 0xc5, 0xfe, 0x23, 0x17, 0xf0, 0x49, 0xd6, 0xc2, 0x4a, 0xde,
0x42, 0xe3, 0x3f, 0x95, 0x18, 0x5c, 0xb8, 0xaa, 0x9e, 0x77, 0xe2, 0xa3, 0x1b, 0x50, 0x8b, 0x49,
0x64, 0x99, 0x24, 0x1b, 0xe8, 0x7b, 0xb0, 0xc4, 0x2c, 0x15, 0x35, 0xd2, 0xcc, 0x83, 0x6f, 0x74,
0xa6, 0x94, 0x4c, 0x2c, 0xe8, 0x51, 0x0f, 0x9a, 0x34, 0x34, 0x83, 0xb0, 0x3f, 0xf2, 0x29, 0xf7,
0x36, 0x77, 0x7f, 0xbd, 0x63, 0x4c, 0x80, 0xef, 0x03, 0x3a, 0x3c, 0x94, 0x94, 0xb8, 0xc1, 0x39,
0xa3, 0x25, 0xfa, 0x14, 0x74, 0xe2, 0x59, 0x89, 0xa0, 0xc5, 0xd2, 0x82, 0xea, 0xc4, 0xb3, 0x62,
0x31, 0x49, 0x7c, 0x96, 0xca, 0xc7, 0xe7, 0x8f, 0x1a, 0xb4, 0x8a, 0x01, 0x92, 0x19, 0x91, 0x48,
0xd4, 0x4a, 0x4b, 0x44, 0x0f, 0x04, 0x13, 0x11, 0x01, 0x9a, 0x0a, 0x79, 0x71, 0x90, 0xb0, 0x64,
0x31, 0x6c, 0xf8, 0x46, 0x62, 0x0d, 0xff, 0xe7, 0x9d, 0x25, 0xcb, 0x6f, 0x35, 0xd8, 0xc8, 0xeb,
0xba, 0xc8, 0xb9, 0xbf, 0x03, 0x4b, 0xb6, 0x77, 0xe2, 0x47, 0xc7, 0xde, 0x9c, 0x02, 0x3c, 0x4c,
0x97, 0x20, 0x36, 0x5c, 0xb8, 0xbe, 0x4f, 0xc2, 0x9e, 0x47, 0x49, 0x10, 0xee, 0xd9, 0x9e, 0xe3,
0x0f, 0x0f, 0xcd, 0xf0, 0xf4, 0x02, 0x35, 0x92, 0x49, 0xf7, 0x4a, 0x2e, 0xdd, 0x8d, 0x7f, 0x6a,
0x70, 0x43, 0xad, 0x4f, 0x1e, 0xbd, 0x0d, 0xab, 0x27, 0x36, 0x71, 0xac, 0x04, 0x02, 0xe2, 0x35,
0xab, 0x95, 0x11, 0x23, 0x96, 0x27, 0x9c, 0xd4, 0xa8, 0x1c, 0x85, 0x81, 0xed, 0x0d, 0x9f, 0xd8,
0x34, 0xc4, 0x82, 0x3e, 0xe5, 0xcf, 0x6a, 0xf9, 0xcc, 0xfc, 0x9d, 0xc8, 0x4c, 0x61, 0xea, 0x23,
0x71, 0x75, 0xd1, 0x77, 0xdb, 0xb0, 0x28, 0xda, 0x07, 0xe3, 0x0f, 0x1a, 0x6c, 0xee, 0x93, 0xf0,
0x51, 0xbc, 0xc7, 0xcc, 0xb4, 0x69, 0x68, 0x0f, 0x2e, 0xc1, 0x98, 0xd7, 0x1a, 0xdc, 0x9a, 0x68,
0x8c, 0x8c, 0xa0, 0x44, 0xb4, 0xe8, 0x02, 0x54, 0x23, 0xda, 0x4f, 0xc9, 0xcb, 0xcf, 0x4c, 0x67,
0x4c, 0x0e, 0x4d, 0x3b, 0x10, 0x88, 0x36, 0x27, 0xbe, 0xff, 0x4b, 0x83, 0x9b, 0xfb, 0x84, 0x35,
0xa3, 0xe2, 0xce, 0xb9, 0x44, 0xef, 0x94, 0xe8, 0xf4, 0xfe, 0x24, 0x82, 0xa9, 0xb4, 0xf6, 0x52,
0xdc, 0xb7, 0xc9, 0xcb, 0x31, 0x85, 0x0b, 0x32, 0xd1, 0xa5, 0xf3, 0x0c, 0x1f, 0x36, 0x3e, 0x37,
0xc3, 0xc1, 0x69, 0xd7, 0xbd, 0x78, 0x05, 0xbc, 0x0f, 0x8d, 0x74, 0x57, 0x28, 0xca, 0xb8, 0x86,
0xf5, 0x54, 0x5b, 0x48, 0x59, 0x2f, 0xbc, 0xce, 0x9f, 0x06, 0x17, 0xef, 0x60, 0xe6, 0x0d, 0x63,
0x16, 0xb8, 0x17, 0x0b, 0xc0, 0x7d, 0x0e, 0x20, 0x8d, 0x3b, 0xa0, 0xc3, 0x39, 0xec, 0xfa, 0x3e,
0xac, 0x48, 0x69, 0x32, 0x52, 0xb3, 0xa0, 0x3a, 0x22, 0x37, 0x8e, 0x60, 0x43, 0xee, 0x3f, 0x66,
0x98, 0x28, 0xf0, 0xf3, 0x80, 0x84, 0x26, 0x6a, 0xc1, 0x8a, 0x84, 0x49, 0xd9, 0x62, 0x44, 0x4b,
0xd6, 0x0c, 0x1f, 0x73, 0xba, 0x3e, 0xc3, 0x42, 0xd9, 0x8a, 0xc3, 0x71, 0x0c, 0xbd, 0xc6, 0xaf,
0xa0, 0xd1, 0xed, 0x3e, 0x49, 0xc9, 0xba, 0x03, 0x6b, 0x96, 0xe5, 0xf4, 0xd3, 0x5c, 0x1a, 0xe7,
0x6a, 0x58, 0x96, 0x93, 0x60, 0x36, 0xfa, 0x16, 0x34, 0x43, 0xda, 0x2f, 0x0a, 0xd7, 0x43, 0x9a,
0x50, 0x19, 0x07, 0xd0, 0xe4, 0xc6, 0xf2, 0xa0, 0xce, 0xb0, 0xf5, 0x36, 0xe8, 0x29, 0x71, 0x51,
0x82, 0xd4, 0x13, 0x63, 0x29, 0x83, 0xc3, 0xa8, 0xc5, 0x4a, 0x24, 0x4e, 0x6f, 0xb1, 0x6e, 0x02,
0xd8, 0xb4, 0x7f, 0xc2, 0xa8, 0x89, 0xc5, 0x6d, 0x5c, 0xc5, 0x35, 0x9b, 0x3e, 0x16, 0x1b, 0xe8,
0x07, 0xb0, 0xcc, 0xf5, 0xb3, 0xb6, 0x45, 0x51, 0x71, 0x3c, 0x1a, 0xd9, 0x13, 0x60, 0xc9, 0x60,
0xfc, 0x02, 0xf4, 0x6e, 0xf7, 0x49, 0x62, 0x47, 0x3e, 0xbb, 0x34, 0x45, 0x76, 0x95, 0x38, 0xe3,
0x2b, 0x68, 0x26, 0x08, 0xcb, 0x7b, 0xc8, 0x26, 0x54, 0x62, 0x71, 0x95, 0x5e, 0x17, 0xfd, 0x10,
0x96, 0xc5, 0x88, 0x41, 0x66, 0xd0, 0x07, 0x59, 0x9b, 0xe5, 0xf8, 0x21, 0x05, 0xd3, 0x7c, 0x03,
0x4b, 0x26, 0x96, 0xe1, 0x31, 0x2a, 0x89, 0x97, 0x61, 0x15, 0xa7, 0x76, 0x8c, 0xff, 0x56, 0xa1,
0x9e, 0x4a, 0xc0, 0x82, 0xfa, 0xb7, 0xf3, 0x36, 0xf8, 0x00, 0x9a, 0x36, 0xbf, 0x5c, 0xfb, 0x12,
0x01, 0x38, 0x62, 0xd6, 0x70, 0xc3, 0x4e, 0x5f, 0xb9, 0xe8, 0x9b, 0xb0, 0xea, 0x8d, 0xdd, 0x7e,
0xe0, 0xbf, 0xa0, 0xf2, 0xb1, 0xb7, 0xe2, 0x8d, 0x5d, 0xec, 0xbf, 0xa0, 0x49, 0xf3, 0xbc, 0x7c,
0xe1, 0xe6, 0x79, 0xe5, 0x6d, 0x35, 0xcf, 0xab, 0xf3, 0x35, 0xcf, 0x9b, 0x50, 0x77, 0xcd, 0x73,
0x76, 0xca, 0xbe, 0x37, 0x76, 0x5b, 0x35, 0x91, 0xc4, 0xae, 0x79, 0x8e, 0xfd, 0x17, 0x4f, 0xc7,
0x2e, 0xda, 0x86, 0x2b, 0x8e, 0x49, 0xc3, 0x7e, 0xfa, 0x61, 0x0b, 0xfc, 0x61, 0xdb, 0x64, 0xfb,
0x9f, 0xc6, 0x8f, 0x5b, 0xe3, 0x3e, 0xd4, 0x7b, 0xdd, 0x0e, 0xcb, 0x24, 0xd6, 0x02, 0x15, 0x62,
0xb7, 0x0e, 0x4b, 0x87, 0xa9, 0xc4, 0x13, 0x0b, 0x06, 0xbb, 0x7a, 0x64, 0x0b, 0xbb, 0x54, 0x14,
0x1e, 0xd2, 0xde, 0x96, 0x87, 0x2a, 0x73, 0x79, 0xc8, 0xf8, 0x77, 0x15, 0x36, 0x8e, 0xcc, 0xe7,
0xe4, 0xdd, 0x77, 0xa9, 0xa5, 0x6e, 0x89, 0x27, 0x70, 0x95, 0xa3, 0x40, 0x27, 0x65, 0x8f, 0x7c,
0x39, 0xa9, 0xf0, 0x3c, 0x15, 0x12, 0x5c, 0x64, 0x44, 0x3f, 0x81, 0x66, 0x06, 0x5c, 0x23, 0x30,
0xda, 0x52, 0x88, 0xca, 0xa0, 0x35, 0xce, 0xf1, 0xa1, 0x3d, 0xd0, 0x2d, 0xd7, 0x49, 0xbc, 0xbd,
0xcc, 0x4d, 0xba, 0xa5, 0x90, 0x93, 0x8e, 0x37, 0xae, 0x5b, 0xae, 0x13, 0x47, 0x8c, 0xc9, 0xb0,
0x9c, 0x7c, 0x71, 0x94, 0x90, 0x61, 0xc5, 0x32, 0x18, 0x50, 0xc3, 0xa3, 0x53, 0x32, 0x38, 0x3b,
0xf4, 0x6d, 0x2f, 0x9c, 0x01, 0xd1, 0x3f, 0x82, 0xd5, 0x39, 0xd2, 0x23, 0xe6, 0x61, 0xd5, 0xc3,
0x30, 0xc2, 0x3f, 0x11, 0x30, 0x21, 0xe2, 0x55, 0xf3, 0xc6, 0xee, 0xcf, 0x4e, 0x18, 0x50, 0x18,
0xbf, 0xaf, 0x80, 0xfe, 0x99, 0x44, 0x19, 0x8e, 0x68, 0x65, 0x90, 0x7a, 0x0b, 0xd2, 0x13, 0x2b,
0xd5, 0x10, 0x6b, 0x1f, 0x1a, 0x94, 0x90, 0xb3, 0x79, 0x9e, 0xe0, 0x3a, 0x63, 0x8c, 0x1d, 0xfe,
0x63, 0xa6, 0x2a, 0xf2, 0x95, 0xe8, 0x39, 0xea, 0x9d, 0x9b, 0x0a, 0x7f, 0x27, 0x1e, 0xc5, 0x69,
0x0e, 0xb4, 0x0d, 0x6b, 0xf2, 0x82, 0x8b, 0xfa, 0x26, 0x9e, 0x40, 0x55, 0x9c, 0xdf, 0x36, 0x02,
0x68, 0xca, 0xdf, 0x22, 0x6b, 0xe8, 0x8c, 0xd0, 0xec, 0x81, 0x7e, 0x92, 0x34, 0x1b, 0xd3, 0x5e,
0x97, 0xa9, 0x9e, 0x04, 0x67, 0x78, 0x8c, 0x87, 0x50, 0x4f, 0xfd, 0x39, 0xa5, 0x01, 0x68, 0xc1,
0xca, 0x71, 0x4a, 0x4f, 0x0d, 0x47, 0x4b, 0xe3, 0x7f, 0x1a, 0x1f, 0xe4, 0x60, 0x32, 0xf0, 0x9f,
0x93, 0xe0, 0x65, 0xe6, 0xb9, 0xfc, 0xe6, 0xe5, 0xff, 0x00, 0x56, 0x65, 0x1c, 0xa3, 0x03, 0xa9,
0x92, 0x3b, 0x9d, 0x31, 0x38, 0x66, 0x40, 0x0f, 0x12, 0x23, 0xab, 0x13, 0x3b, 0x86, 0xac, 0x8f,
0x93, 0x73, 0xfc, 0x59, 0xbc, 0xfa, 0xb3, 0xe7, 0xb8, 0xd4, 0x19, 0x9d, 0xf1, 0x95, 0x06, 0x8d,
0xae, 0x19, 0x9a, 0x4f, 0x7d, 0x8b, 0x3c, 0x9b, 0xb3, 0xa7, 0x2d, 0x31, 0xf1, 0xbd, 0x01, 0x35,
0x76, 0x6b, 0xd1, 0xd0, 0x74, 0x47, 0xdc, 0x8c, 0x45, 0x9c, 0x6c, 0x74, 0xfe, 0xdf, 0x80, 0x3a,
0x33, 0xe2, 0x48, 0x7c, 0x87, 0x41, 0x23, 0x40, 0xfc, 0xa1, 0xe9, 0x8e, 0x7c, 0x2f, 0x1e, 0x0c,
0xa1, 0x4f, 0x26, 0x54, 0x56, 0x91, 0x54, 0x3a, 0xb5, 0x7d, 0x67, 0x02, 0x47, 0x8e, 0xdc, 0x58,
0x40, 0x2e, 0xd7, 0xc8, 0xae, 0xd0, 0x67, 0xf6, 0xe0, 0x2c, 0xea, 0x3e, 0xa6, 0x68, 0xcc, 0x91,
0x46, 0x1a, 0x73, 0xf3, 0x26, 0xb9, 0x10, 0x43, 0x89, 0x28, 0x65, 0x8d, 0x05, 0xf4, 0x25, 0xac,
0xb3, 0x97, 0x57, 0xfc, 0x00, 0x8c, 0x14, 0x76, 0x26, 0x2b, 0x2c, 0x10, 0xbf, 0xa1, 0x4a, 0x13,
0xf4, 0xf4, 0x67, 0x20, 0xa4, 0x9a, 0x4d, 0x2b, 0xbe, 0x54, 0xb5, 0x3f, 0x9c, 0x49, 0x17, 0xab,
0xd8, 0x87, 0x25, 0xde, 0x0e, 0x23, 0x55, 0x3d, 0xa5, 0x3f, 0xf9, 0xb4, 0xa7, 0x3d, 0x4f, 0x8d,
0x05, 0xf4, 0x6b, 0x58, 0xcb, 0x0d, 0xdb, 0xd1, 0x47, 0x0a, 0x91, 0xea, 0xcf, 0x26, 0xed, 0xbb,
0x65, 0x48, 0xd3, 0x7e, 0x49, 0x0f, 0xa4, 0x95, 0x7e, 0x51, 0x0c, 0xd5, 0x95, 0x7e, 0x51, 0x4d,
0xb6, 0x8d, 0x05, 0x34, 0x84, 0x66, 0xf6, 0x9d, 0x8d, 0xb6, 0x15, 0xcc, 0xca, 0xd1, 0x63, 0xfb,
0xa3, 0x12, 0x94, 0xb1, 0x22, 0x17, 0xae, 0xe4, 0xc7, 0xa9, 0xe8, 0xee, 0x54, 0x01, 0xd9, 0x7a,
0xf9, 0xb8, 0x14, 0x6d, 0xac, 0xee, 0x25, 0xcf, 0xe2, 0xc2, 0x38, 0x0f, 0xed, 0xa8, 0xc5, 0x4c,
0x9a, 0x33, 0xb6, 0x77, 0x4b, 0xd3, 0xc7, 0xaa, 0x09, 0x5c, 0x2d, 0x8c, 0xe7, 0xd0, 0xc7, 0xd3,
0xe4, 0xe4, 0x46, 0x18, 0xed, 0xd9, 0x03, 0x44, 0x63, 0x01, 0x7d, 0x25, 0x2e, 0x1e, 0xd5, 0xc8,
0x0b, 0xdd, 0x53, 0x6b, 0x9b, 0x32, 0xab, 0x6b, 0x77, 0xde, 0x84, 0x25, 0x3e, 0xeb, 0x2b, 0x7e,
0x69, 0x28, 0xc6, 0x46, 0x79, 0x7c, 0x8a, 0xe4, 0x4d, 0x9e, 0x87, 0xb5, 0xef, 0xbd, 0x01, 0x47,
0x6c, 0x80, 0x9f, 0x9f, 0x8b, 0x47, 0x70, 0xb5, 0x3b, 0x33, 0x39, 0xe7, 0xc3, 0xaa, 0x2f, 0x60,
0x2d, 0xd7, 0xec, 0x2b, 0xeb, 0x5f, 0xfd, 0x20, 0x28, 0x01, 0x2e, 0xb9, 0x3b, 0x18, 0x4d, 0x28,
0x32, 0xc5, 0x3d, 0xdd, 0xbe, 0x5b, 0x86, 0x34, 0x3a, 0x48, 0xe7, 0x1f, 0x55, 0x58, 0x8d, 0x6e,
0xd7, 0x4b, 0xb8, 0xd5, 0x2e, 0xe1, 0x9a, 0xf9, 0x02, 0xd6, 0x72, 0x33, 0x43, 0xa5, 0x77, 0xd5,
0x73, 0xc5, 0x59, 0xa1, 0xfb, 0x1c, 0x1a, 0x99, 0xf1, 0x20, 0xfa, 0x70, 0xd2, 0x45, 0x93, 0x47,
0xeb, 0xe9, 0x82, 0xf7, 0xee, 0xff, 0xf2, 0xde, 0xd0, 0x0e, 0x4f, 0xc7, 0xc7, 0xec, 0x9f, 0x5d,
0x41, 0xfa, 0x6d, 0xdb, 0x97, 0xbf, 0x76, 0x23, 0x07, 0xed, 0x72, 0xee, 0x5d, 0xa6, 0x66, 0x74,
0x7c, 0xbc, 0xcc, 0x57, 0xf7, 0xbf, 0x0e, 0x00, 0x00, 0xff, 0xff, 0xbd, 0xf2, 0x86, 0xb3, 0x44,
0x22, 0x00, 0x00,
// 2101 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xc4, 0x59, 0x5b, 0x6f, 0x1b, 0xc7,
0x15, 0xd6, 0x92, 0xba, 0xf1, 0x70, 0x49, 0x49, 0x63, 0x55, 0x61, 0x69, 0x5b, 0x96, 0x37, 0x89,
0xa3, 0x38, 0xa8, 0x14, 0xd3, 0x45, 0x6f, 0xee, 0x05, 0x96, 0x19, 0xab, 0x44, 0x2d, 0x57, 0x1d,
0x39, 0x09, 0xd0, 0xa0, 0x20, 0x56, 0xdc, 0x11, 0xb5, 0x15, 0x77, 0x97, 0xd9, 0x59, 0xca, 0xf2,
0x93, 0x83, 0xb4, 0x0f, 0x45, 0x51, 0xd4, 0xed, 0x3f, 0x68, 0x0b, 0x14, 0x28, 0xd0, 0x3e, 0xf4,
0xb9, 0xef, 0xfd, 0x0b, 0xfd, 0x3d, 0xc5, 0x5c, 0xf6, 0x3e, 0x24, 0xd7, 0x94, 0x0d, 0xbd, 0x71,
0x86, 0xe7, 0x36, 0xe7, 0xf2, 0xcd, 0x99, 0xb3, 0x80, 0x2c, 0x33, 0x30, 0xbb, 0x94, 0xf8, 0xe7,
0x76, 0x8f, 0xec, 0x0c, 0x7d, 0x2f, 0xf0, 0xd0, 0x9a, 0x63, 0x0f, 0xce, 0x47, 0x54, 0xac, 0x76,
0x18, 0x41, 0x53, 0xef, 0x79, 0x8e, 0xe3, 0xb9, 0x62, 0xab, 0x59, 0xb7, 0xdd, 0x80, 0xf8, 0xae,
0x39, 0x90, 0x6b, 0x3d, 0xc9, 0xd0, 0xd4, 0x69, 0xef, 0x94, 0x38, 0xa6, 0x58, 0x19, 0x2f, 0xe1,
0x1a, 0x26, 0x7d, 0x9b, 0x06, 0xc4, 0x7f, 0xea, 0x59, 0x04, 0x93, 0x2f, 0x47, 0x84, 0x06, 0xe8,
0x63, 0x98, 0x3f, 0x36, 0x29, 0x69, 0x68, 0x5b, 0xda, 0x76, 0xb5, 0x75, 0x63, 0x27, 0xa5, 0x52,
0x2a, 0x3b, 0xa0, 0xfd, 0x3d, 0x93, 0x12, 0xcc, 0x29, 0xd1, 0x77, 0x60, 0xc9, 0xb4, 0x2c, 0x9f,
0x50, 0xda, 0x28, 0x4d, 0x60, 0x7a, 0x28, 0x68, 0x70, 0x48, 0x6c, 0xbc, 0xd2, 0x60, 0x3d, 0x6d,
0x01, 0x1d, 0x7a, 0x2e, 0x25, 0x68, 0x0f, 0xaa, 0xb6, 0x6b, 0x07, 0xdd, 0xa1, 0xe9, 0x9b, 0x0e,
0x95, 0x96, 0xdc, 0x4e, 0x0b, 0x8d, 0x0e, 0xda, 0x71, 0xed, 0xe0, 0x90, 0x13, 0x62, 0xb0, 0xa3,
0xdf, 0xe8, 0x3e, 0x2c, 0xd2, 0xc0, 0x0c, 0x46, 0xa1, 0x4d, 0xd7, 0x95, 0x36, 0x1d, 0x71, 0x12,
0x2c, 0x49, 0x8d, 0x0b, 0xd0, 0x1f, 0x0f, 0x46, 0xf4, 0x74, 0x76, 0x5f, 0x20, 0x98, 0xb7, 0x8e,
0x3b, 0x6d, 0xae, 0xb4, 0x8c, 0xf9, 0x6f, 0x64, 0x80, 0xde, 0xf3, 0x06, 0x03, 0xd2, 0x0b, 0x6c,
0xcf, 0xed, 0xb4, 0x1b, 0xf3, 0xfc, 0xbf, 0xd4, 0x9e, 0xf1, 0x67, 0x0d, 0x56, 0x8f, 0x48, 0xdf,
0x21, 0x6e, 0xd0, 0x69, 0x87, 0xea, 0xd7, 0x61, 0xa1, 0xe7, 0x8d, 0xdc, 0x80, 0xeb, 0xaf, 0x61,
0xb1, 0x40, 0xb7, 0x41, 0xef, 0x9d, 0x9a, 0xae, 0x4b, 0x06, 0x5d, 0xd7, 0x74, 0x08, 0x57, 0x55,
0xc1, 0x55, 0xb9, 0xf7, 0xd4, 0x74, 0x48, 0x4e, 0x63, 0x39, 0xaf, 0x11, 0x6d, 0x41, 0x75, 0x68,
0xfa, 0x81, 0x9d, 0x32, 0x2a, 0xb9, 0x65, 0xfc, 0x45, 0x83, 0x8d, 0x87, 0x94, 0xda, 0x7d, 0x37,
0x67, 0xd9, 0x06, 0x2c, 0xba, 0x9e, 0x45, 0x3a, 0x6d, 0x6e, 0x5a, 0x19, 0xcb, 0x15, 0xba, 0x0e,
0x95, 0x21, 0x21, 0x7e, 0xd7, 0xf7, 0x06, 0xa1, 0x61, 0xcb, 0x6c, 0x03, 0x7b, 0x03, 0x82, 0x7e,
0x01, 0x6b, 0x34, 0x23, 0x88, 0x36, 0xca, 0x5b, 0xe5, 0xed, 0x6a, 0xeb, 0xdd, 0x9d, 0x5c, 0x66,
0xef, 0x64, 0x95, 0xe2, 0x3c, 0xb7, 0xf1, 0x55, 0x09, 0xae, 0x45, 0x74, 0xc2, 0x56, 0xf6, 0x9b,
0x79, 0x8e, 0x92, 0x7e, 0x64, 0x9e, 0x58, 0x14, 0xf1, 0x5c, 0xe4, 0xf2, 0x72, 0xd2, 0xe5, 0x05,
0x22, 0x98, 0xf5, 0xe7, 0x42, 0xce, 0x9f, 0xe8, 0x16, 0x54, 0xc9, 0xc5, 0xd0, 0xf6, 0x49, 0x37,
0xb0, 0x1d, 0xd2, 0x58, 0xdc, 0xd2, 0xb6, 0xe7, 0x31, 0x88, 0xad, 0x67, 0xb6, 0x43, 0x12, 0x39,
0xbb, 0x54, 0x3c, 0x67, 0xff, 0xa6, 0xc1, 0x3b, 0xb9, 0x28, 0xc9, 0x42, 0xc2, 0xb0, 0xca, 0x4f,
0x1e, 0x7b, 0x86, 0x55, 0x13, 0x73, 0xf8, 0x9d, 0x49, 0x0e, 0x8f, 0xc9, 0x71, 0x8e, 0x7f, 0xb6,
0xc2, 0xfa, 0xab, 0x06, 0xd7, 0x8e, 0x4e, 0xbd, 0xe7, 0x52, 0x05, 0x9d, 0xbd, 0xc0, 0xb2, 0xa1,
0x28, 0x4d, 0x0f, 0x45, 0x39, 0x1f, 0x8a, 0xb0, 0x4c, 0xe7, 0xe3, 0x32, 0x35, 0xce, 0x60, 0x3d,
0x6d, 0xa2, 0x74, 0xe2, 0x26, 0x40, 0x94, 0x78, 0xc2, 0x7d, 0x65, 0x9c, 0xd8, 0x99, 0xcd, 0x21,
0x67, 0xf0, 0xce, 0x3e, 0x09, 0xa4, 0x2e, 0xf6, 0x1f, 0xb9, 0x84, 0x4f, 0xd2, 0x16, 0x96, 0xb2,
0x16, 0x1a, 0xff, 0x2e, 0x45, 0xe0, 0xc2, 0x55, 0x75, 0xdc, 0x13, 0x0f, 0xdd, 0x80, 0x4a, 0x44,
0x22, 0xcb, 0x24, 0xde, 0x40, 0xdf, 0x85, 0x05, 0x66, 0xa9, 0xa8, 0x91, 0x7a, 0x16, 0x7c, 0xc3,
0x33, 0x25, 0x64, 0x62, 0x41, 0x8f, 0x3a, 0x50, 0xa7, 0x81, 0xe9, 0x07, 0xdd, 0xa1, 0x47, 0xb9,
0xb7, 0xb9, 0xfb, 0xab, 0x2d, 0x63, 0x0c, 0x7c, 0x1f, 0xd0, 0xfe, 0xa1, 0xa4, 0xc4, 0x35, 0xce,
0x19, 0x2e, 0xd1, 0x27, 0xa0, 0x13, 0xd7, 0x8a, 0x05, 0xcd, 0x17, 0x16, 0x54, 0x25, 0xae, 0x15,
0x89, 0x89, 0xe3, 0xb3, 0x50, 0x3c, 0x3e, 0x7f, 0xd0, 0xa0, 0x91, 0x0f, 0x90, 0xcc, 0x88, 0x58,
0xa2, 0x56, 0x58, 0x22, 0x7a, 0x20, 0x98, 0x88, 0x08, 0xd0, 0x44, 0xc8, 0x8b, 0x82, 0x84, 0x25,
0x8b, 0x61, 0xc3, 0x37, 0x62, 0x6b, 0xf8, 0x3f, 0x6f, 0x2d, 0x59, 0x7e, 0xa3, 0xc1, 0x46, 0x56,
0xd7, 0x65, 0xce, 0xfd, 0x6d, 0x58, 0xb0, 0xdd, 0x13, 0x2f, 0x3c, 0xf6, 0xe6, 0x04, 0xe0, 0x61,
0xba, 0x04, 0xb1, 0xe1, 0xc0, 0xf5, 0x7d, 0x12, 0x74, 0x5c, 0x4a, 0xfc, 0x60, 0xcf, 0x76, 0x07,
0x5e, 0xff, 0xd0, 0x0c, 0x4e, 0x2f, 0x51, 0x23, 0xa9, 0x74, 0x2f, 0x65, 0xd2, 0xdd, 0xf8, 0x87,
0x06, 0x37, 0xd4, 0xfa, 0xe4, 0xd1, 0x9b, 0xb0, 0x7c, 0x62, 0x93, 0x81, 0x15, 0x43, 0x40, 0xb4,
0x66, 0xb5, 0x32, 0x64, 0xc4, 0xf2, 0x84, 0xe3, 0x1a, 0x95, 0xa3, 0xc0, 0xb7, 0xdd, 0xfe, 0x13,
0x9b, 0x06, 0x58, 0xd0, 0x27, 0xfc, 0x59, 0x2e, 0x9e, 0x99, 0xbf, 0x15, 0x99, 0x29, 0x4c, 0x7d,
0x24, 0xae, 0x2e, 0xfa, 0x76, 0x1b, 0x16, 0x45, 0xfb, 0x60, 0xfc, 0x5e, 0x83, 0xcd, 0x7d, 0x12,
0x3c, 0x8a, 0xf6, 0x98, 0x99, 0x36, 0x0d, 0xec, 0xde, 0x15, 0x18, 0xf3, 0x4a, 0x83, 0x5b, 0x63,
0x8d, 0x91, 0x11, 0x94, 0x88, 0x16, 0x5e, 0x80, 0x6a, 0x44, 0xfb, 0x19, 0x79, 0xf1, 0x99, 0x39,
0x18, 0x91, 0x43, 0xd3, 0xf6, 0x05, 0xa2, 0xcd, 0x88, 0xef, 0xff, 0xd4, 0xe0, 0xe6, 0x3e, 0x61,
0xcd, 0xa8, 0xb8, 0x73, 0xae, 0xd0, 0x3b, 0x05, 0x3a, 0xbd, 0x3f, 0x8a, 0x60, 0x2a, 0xad, 0xbd,
0x12, 0xf7, 0x6d, 0xf2, 0x72, 0x4c, 0xe0, 0x82, 0x4c, 0x74, 0xe9, 0x3c, 0xc3, 0x83, 0x8d, 0xcf,
0xcd, 0xa0, 0x77, 0xda, 0x76, 0x2e, 0x5f, 0x01, 0xef, 0x42, 0x2d, 0xd9, 0x15, 0x8a, 0x32, 0xae,
0x60, 0x3d, 0xd1, 0x16, 0x52, 0xd6, 0x0b, 0xaf, 0xf3, 0xa7, 0xc1, 0xe5, 0x3b, 0x98, 0x59, 0xc3,
0x98, 0x06, 0xee, 0xf9, 0x1c, 0x70, 0x5f, 0x00, 0x48, 0xe3, 0x0e, 0x68, 0x7f, 0x06, 0xbb, 0xbe,
0x07, 0x4b, 0x52, 0x9a, 0x8c, 0xd4, 0x34, 0xa8, 0x0e, 0xc9, 0x8d, 0x23, 0xd8, 0x90, 0xfb, 0x8f,
0x19, 0x26, 0x0a, 0xfc, 0x3c, 0x20, 0x81, 0x89, 0x1a, 0xb0, 0x24, 0x61, 0x52, 0xb6, 0x18, 0xe1,
0x92, 0x35, 0xc3, 0xc7, 0x9c, 0xae, 0xcb, 0xb0, 0x50, 0xb6, 0xe2, 0x70, 0x1c, 0x41, 0xaf, 0xf1,
0x2b, 0xa8, 0xb5, 0xdb, 0x4f, 0x12, 0xb2, 0xee, 0xc0, 0x8a, 0x65, 0x0d, 0xba, 0x49, 0x2e, 0x8d,
0x73, 0xd5, 0x2c, 0x6b, 0x10, 0x63, 0x36, 0x7a, 0x0f, 0xea, 0x01, 0xed, 0xe6, 0x85, 0xeb, 0x01,
0x8d, 0xa9, 0x8c, 0x03, 0xa8, 0x73, 0x63, 0x79, 0x50, 0xa7, 0xd8, 0x7a, 0x1b, 0xf4, 0x84, 0xb8,
0x30, 0x41, 0xaa, 0xb1, 0xb1, 0x94, 0xc1, 0x61, 0xd8, 0x62, 0xc5, 0x12, 0x27, 0xb7, 0x58, 0x37,
0x01, 0x6c, 0xda, 0x3d, 0x61, 0xd4, 0xc4, 0xe2, 0x36, 0x2e, 0xe3, 0x8a, 0x4d, 0x1f, 0x8b, 0x0d,
0xf4, 0x7d, 0x58, 0xe4, 0xfa, 0x59, 0xdb, 0xa2, 0xa8, 0x38, 0x1e, 0x8d, 0xf4, 0x09, 0xb0, 0x64,
0x30, 0x3e, 0x05, 0xbd, 0xdd, 0x7e, 0x12, 0xdb, 0x91, 0xcd, 0x2e, 0x4d, 0x91, 0x5d, 0x05, 0xce,
0xf8, 0x12, 0xea, 0x31, 0xc2, 0xf2, 0x1e, 0xb2, 0x0e, 0xa5, 0x48, 0x5c, 0xa9, 0xd3, 0x46, 0x3f,
0x82, 0x45, 0x31, 0x62, 0x90, 0x19, 0xf4, 0x7e, 0xda, 0x66, 0x39, 0x7e, 0x48, 0xc0, 0x34, 0xdf,
0xc0, 0x92, 0x89, 0x65, 0x78, 0x84, 0x4a, 0xe2, 0x65, 0x58, 0xc6, 0x89, 0x1d, 0xe3, 0x3f, 0x65,
0xa8, 0x26, 0x12, 0x30, 0xa7, 0xfe, 0xcd, 0xbc, 0x0d, 0xde, 0x87, 0xba, 0xcd, 0x2f, 0xd7, 0xae,
0x44, 0x00, 0x8e, 0x98, 0x15, 0x5c, 0xb3, 0x93, 0x57, 0x2e, 0xfa, 0x26, 0x2c, 0xbb, 0x23, 0xa7,
0xeb, 0x7b, 0xcf, 0xa9, 0x7c, 0xec, 0x2d, 0xb9, 0x23, 0x07, 0x7b, 0xcf, 0x69, 0xdc, 0x3c, 0x2f,
0x5e, 0xba, 0x79, 0x5e, 0x7a, 0x53, 0xcd, 0xf3, 0xf2, 0x6c, 0xcd, 0xf3, 0x26, 0x54, 0x1d, 0xf3,
0x82, 0x9d, 0xb2, 0xeb, 0x8e, 0x9c, 0x46, 0x45, 0x24, 0xb1, 0x63, 0x5e, 0x60, 0xef, 0xf9, 0xd3,
0x91, 0x83, 0xb6, 0x61, 0x75, 0x60, 0xd2, 0xa0, 0x9b, 0x7c, 0xd8, 0x02, 0x7f, 0xd8, 0xd6, 0xd9,
0xfe, 0x27, 0xd1, 0xe3, 0xd6, 0xb8, 0x0f, 0xd5, 0x4e, 0xbb, 0xc5, 0x32, 0x89, 0xb5, 0x40, 0xb9,
0xd8, 0xad, 0xc3, 0xc2, 0x61, 0x22, 0xf1, 0xc4, 0x82, 0xc1, 0xae, 0x1e, 0xda, 0xc2, 0x2e, 0x15,
0x85, 0x87, 0xb4, 0x37, 0xe5, 0xa1, 0xd2, 0x4c, 0x1e, 0x32, 0xfe, 0x55, 0x86, 0x8d, 0x23, 0xf3,
0x9c, 0xbc, 0xfd, 0x2e, 0xb5, 0xd0, 0x2d, 0xf1, 0x04, 0xd6, 0x38, 0x0a, 0xb4, 0x12, 0xf6, 0xc8,
0x97, 0x93, 0x0a, 0xcf, 0x13, 0x21, 0xc1, 0x79, 0x46, 0xf4, 0x53, 0xa8, 0xa7, 0xc0, 0x35, 0x04,
0xa3, 0x2d, 0x85, 0xa8, 0x14, 0x5a, 0xe3, 0x0c, 0x1f, 0xda, 0x03, 0xdd, 0x72, 0x06, 0xb1, 0xb7,
0x17, 0xb9, 0x49, 0xb7, 0x14, 0x72, 0x92, 0xf1, 0xc6, 0x55, 0xcb, 0x19, 0x44, 0x11, 0x63, 0x32,
0xac, 0x41, 0xb6, 0x38, 0x0a, 0xc8, 0xb0, 0x22, 0x19, 0x0c, 0xa8, 0xe1, 0xd1, 0x29, 0xe9, 0x9d,
0x1d, 0x7a, 0xb6, 0x1b, 0x4c, 0x81, 0xe8, 0x1f, 0xc3, 0xf2, 0x0c, 0xe9, 0x11, 0xf1, 0xb0, 0xea,
0x61, 0x18, 0xe1, 0x9d, 0x08, 0x98, 0x10, 0xf1, 0xaa, 0xb8, 0x23, 0xe7, 0xe7, 0x27, 0x0c, 0x28,
0x8c, 0xdf, 0x95, 0x40, 0xff, 0x4c, 0xa2, 0x0c, 0x47, 0xb4, 0x22, 0x48, 0xbd, 0x05, 0xc9, 0x89,
0x95, 0x6a, 0x88, 0xb5, 0x0f, 0x35, 0x4a, 0xc8, 0xd9, 0x2c, 0x4f, 0x70, 0x9d, 0x31, 0x46, 0x0e,
0xff, 0x09, 0x53, 0x15, 0xfa, 0x4a, 0xf4, 0x1c, 0xd5, 0xd6, 0x4d, 0x85, 0xbf, 0x63, 0x8f, 0xe2,
0x24, 0x07, 0xda, 0x86, 0x15, 0x79, 0xc1, 0x85, 0x7d, 0x13, 0x4f, 0xa0, 0x32, 0xce, 0x6e, 0x1b,
0x3e, 0xd4, 0xe5, 0x6f, 0x91, 0x35, 0x74, 0x4a, 0x68, 0xf6, 0x40, 0x3f, 0x89, 0x9b, 0x8d, 0x49,
0xaf, 0xcb, 0x44, 0x4f, 0x82, 0x53, 0x3c, 0xc6, 0x43, 0xa8, 0x26, 0xfe, 0x9c, 0xd0, 0x00, 0x34,
0x60, 0xe9, 0x38, 0xa1, 0xa7, 0x82, 0xc3, 0xa5, 0xd1, 0x85, 0x9a, 0xbc, 0x10, 0x44, 0x07, 0xcb,
0xba, 0x3b, 0x3e, 0x5b, 0x14, 0xad, 0x09, 0xff, 0x8d, 0x7e, 0x90, 0x1e, 0xa6, 0xbc, 0xa7, 0x74,
0x20, 0x17, 0xc2, 0x9b, 0xda, 0xe4, 0x95, 0x60, 0x7c, 0xa5, 0x81, 0xde, 0x36, 0x03, 0xf3, 0xa9,
0x67, 0x89, 0xb9, 0x4d, 0x23, 0x9e, 0xb6, 0x0b, 0x1d, 0xe1, 0x92, 0xfd, 0x73, 0x4e, 0x7c, 0x1a,
0x26, 0x6b, 0x19, 0x87, 0x4b, 0xf4, 0x43, 0x58, 0x96, 0xf9, 0x11, 0x0e, 0x5c, 0xb7, 0xc6, 0xdb,
0x20, 0x5b, 0xf1, 0x88, 0xc3, 0xf8, 0xaf, 0xc6, 0x87, 0x55, 0x98, 0xf4, 0xbc, 0x73, 0xe2, 0xbf,
0x48, 0x8d, 0x04, 0x5e, 0x1f, 0xe2, 0x1e, 0x24, 0x6c, 0x11, 0x41, 0x53, 0x15, 0x70, 0xb2, 0x2a,
0x62, 0x53, 0xd0, 0x83, 0x38, 0x10, 0xe5, 0xb1, 0x5d, 0x51, 0x3a, 0x8f, 0xe2, 0x58, 0xfd, 0x49,
0x4c, 0x36, 0xd2, 0xe7, 0xb8, 0xd2, 0x39, 0xa4, 0xf1, 0xb5, 0x06, 0xb5, 0x30, 0xba, 0xcf, 0x66,
0xec, 0xdb, 0x0b, 0x4c, 0xb5, 0x6f, 0x40, 0x85, 0xdd, 0xcc, 0x34, 0x30, 0x9d, 0x21, 0x37, 0x63,
0x1e, 0xc7, 0x1b, 0x77, 0xef, 0xc1, 0x5a, 0x2e, 0xfd, 0x50, 0x1d, 0xe0, 0x53, 0xb7, 0xe7, 0x39,
0xc3, 0x01, 0x09, 0xc8, 0xea, 0x1c, 0xd2, 0x61, 0xf9, 0x51, 0xb8, 0xd2, 0x5a, 0xff, 0xab, 0x41,
0x95, 0xd9, 0x7d, 0x24, 0x3e, 0x4f, 0xa1, 0x21, 0x20, 0xfe, 0xfe, 0x76, 0x86, 0x9e, 0x1b, 0xcd,
0xcb, 0xd0, 0xc7, 0x63, 0x00, 0x27, 0x4f, 0x2a, 0xe3, 0xd0, 0xbc, 0x33, 0x86, 0x23, 0x43, 0x6e,
0xcc, 0x21, 0x87, 0x6b, 0x64, 0x9d, 0xc5, 0x33, 0xbb, 0x77, 0x16, 0x36, 0x65, 0x13, 0x34, 0x66,
0x48, 0x43, 0x8d, 0x99, 0x31, 0x9c, 0x5c, 0x88, 0x59, 0x4d, 0x98, 0xe5, 0xc6, 0x1c, 0xfa, 0x12,
0xd6, 0xd9, 0x83, 0x34, 0x7a, 0x17, 0x87, 0x0a, 0x5b, 0xe3, 0x15, 0xe6, 0x88, 0x5f, 0x53, 0xa5,
0x09, 0x7a, 0xf2, 0xeb, 0x18, 0x52, 0x8d, 0xec, 0x15, 0x1f, 0xf0, 0x9a, 0x1f, 0x4c, 0xa5, 0x8b,
0x54, 0xec, 0xc3, 0x02, 0x7f, 0x25, 0x20, 0x55, 0x09, 0x26, 0xbf, 0x84, 0x35, 0x27, 0xbd, 0xda,
0x8d, 0x39, 0xf4, 0x6b, 0x58, 0xc9, 0x7c, 0x83, 0x40, 0x1f, 0x2a, 0x44, 0xaa, 0xbf, 0x26, 0x35,
0xef, 0x16, 0x21, 0x4d, 0xfa, 0x25, 0x39, 0xa7, 0x57, 0xfa, 0x45, 0xf1, 0xad, 0x41, 0xe9, 0x17,
0xd5, 0xc0, 0xdf, 0x98, 0x43, 0x7d, 0xa8, 0xa7, 0xc7, 0x0f, 0x68, 0x5b, 0xc1, 0xac, 0x9c, 0xc8,
0x36, 0x3f, 0x2c, 0x40, 0x19, 0x29, 0x72, 0x60, 0x35, 0x3b, 0x65, 0x46, 0x77, 0x27, 0x0a, 0x48,
0xd7, 0xcb, 0x47, 0x85, 0x68, 0x23, 0x75, 0x2f, 0x78, 0x16, 0xe7, 0xa6, 0x9c, 0x68, 0x47, 0x2d,
0x66, 0xdc, 0xf8, 0xb5, 0xb9, 0x5b, 0x98, 0x3e, 0x52, 0x4d, 0x60, 0x2d, 0x37, 0xb5, 0x44, 0x1f,
0x4d, 0x92, 0x93, 0x99, 0xec, 0x34, 0xa7, 0xcf, 0x55, 0x8d, 0x39, 0xf4, 0xb5, 0xb8, 0xab, 0x54,
0x93, 0x40, 0x74, 0x4f, 0xad, 0x6d, 0xc2, 0x08, 0xb3, 0xd9, 0x7a, 0x1d, 0x96, 0xe8, 0xac, 0x2f,
0xf9, 0x3d, 0xa3, 0x98, 0xa6, 0x65, 0xf1, 0x29, 0x94, 0x37, 0x7e, 0x4c, 0xd8, 0xbc, 0xf7, 0x1a,
0x1c, 0x91, 0x01, 0x5e, 0xf6, 0x73, 0x41, 0x08, 0x57, 0xbb, 0x53, 0x93, 0x73, 0x36, 0xac, 0xfa,
0x02, 0x56, 0x32, 0x6f, 0x20, 0x65, 0xfd, 0xab, 0xdf, 0x49, 0x05, 0xc0, 0x25, 0x73, 0x6d, 0xa3,
0x31, 0x45, 0xa6, 0xb8, 0xda, 0x9b, 0x77, 0x8b, 0x90, 0x86, 0x07, 0x69, 0xfd, 0xbd, 0x0c, 0xcb,
0xe1, 0x85, 0x7c, 0x05, 0xb7, 0xda, 0x15, 0x5c, 0x33, 0x5f, 0xc0, 0x4a, 0x66, 0x94, 0xaa, 0xf4,
0xae, 0x7a, 0xdc, 0x3a, 0x2d, 0x74, 0x9f, 0x43, 0x2d, 0x35, 0x35, 0x45, 0x1f, 0x8c, 0xbb, 0x68,
0xb2, 0x68, 0x3d, 0x59, 0xf0, 0xde, 0xfd, 0x5f, 0xde, 0xeb, 0xdb, 0xc1, 0xe9, 0xe8, 0x98, 0xfd,
0xb3, 0x2b, 0x48, 0xbf, 0x65, 0x7b, 0xf2, 0xd7, 0x6e, 0xe8, 0xa0, 0x5d, 0xce, 0xbd, 0xcb, 0xd4,
0x0c, 0x8f, 0x8f, 0x17, 0xf9, 0xea, 0xfe, 0xff, 0x03, 0x00, 0x00, 0xff, 0xff, 0xa8, 0x93, 0x0b,
0x43, 0x5b, 0x23, 0x00, 0x00,
}
// Reference imports to suppress errors if they are not otherwise used.