From 81ac20143bdea467d6661e9610eb4a7918f38fb6 Mon Sep 17 00:00:00 2001 From: sunby Date: Tue, 25 May 2021 16:22:02 +0800 Subject: [PATCH] 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 --- internal/dataservice/cluster_data_manager.go | 158 +++++++ .../dataservice/cluster_session_manager.go | 71 ++++ internal/dataservice/cluster_v2.go | 229 ++++++++++ internal/dataservice/cluster_v2_test.go | 142 +++++++ internal/dataservice/mock_test.go | 23 + internal/dataservice/policy.go | 111 +++++ internal/dataservice/server.go | 1 + internal/dataservice/server_test.go | 3 + internal/proto/data_service.proto | 16 + internal/proto/datapb/data_service.pb.go | 396 ++++++++++++------ 10 files changed, 1020 insertions(+), 130 deletions(-) create mode 100644 internal/dataservice/cluster_data_manager.go create mode 100644 internal/dataservice/cluster_session_manager.go create mode 100644 internal/dataservice/cluster_v2.go create mode 100644 internal/dataservice/cluster_v2_test.go create mode 100644 internal/dataservice/policy.go diff --git a/internal/dataservice/cluster_data_manager.go b/internal/dataservice/cluster_data_manager.go new file mode 100644 index 000000000..d08fde76f --- /dev/null +++ b/internal/dataservice/cluster_data_manager.go @@ -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) +} diff --git a/internal/dataservice/cluster_session_manager.go b/internal/dataservice/cluster_session_manager.go new file mode 100644 index 000000000..a025a0636 --- /dev/null +++ b/internal/dataservice/cluster_session_manager.go @@ -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 +} diff --git a/internal/dataservice/cluster_v2.go b/internal/dataservice/cluster_v2.go new file mode 100644 index 000000000..13d6d2bda --- /dev/null +++ b/internal/dataservice/cluster_v2.go @@ -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)) + } + } + } +} diff --git a/internal/dataservice/cluster_v2_test.go b/internal/dataservice/cluster_v2_test.go new file mode 100644 index 000000000..e1e2156e5 --- /dev/null +++ b/internal/dataservice/cluster_v2_test.go @@ -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...) +} diff --git a/internal/dataservice/mock_test.go b/internal/dataservice/mock_test.go index 4d63fde38..2a49ce94e 100644 --- a/internal/dataservice/mock_test.go +++ b/internal/dataservice/mock_test.go @@ -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 +} diff --git a/internal/dataservice/policy.go b/internal/dataservice/policy.go new file mode 100644 index 000000000..bd250689b --- /dev/null +++ b/internal/dataservice/policy.go @@ -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 +} diff --git a/internal/dataservice/server.go b/internal/dataservice/server.go index 1c5f06f57..42935d188 100644 --- a/internal/dataservice/server.go +++ b/internal/dataservice/server.go @@ -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 { diff --git a/internal/dataservice/server_test.go b/internal/dataservice/server_test.go index cff29be7d..e56d4cad9 100644 --- a/internal/dataservice/server_test.go +++ b/internal/dataservice/server_test.go @@ -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) diff --git a/internal/proto/data_service.proto b/internal/proto/data_service.proto index 5cb1f5496..ec42e312e 100644 --- a/internal/proto/data_service.proto +++ b/internal/proto/data_service.proto @@ -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; diff --git a/internal/proto/datapb/data_service.pb.go b/internal/proto/datapb/data_service.pb.go index b240225e4..df274d4ed 100644 --- a/internal/proto/datapb/data_service.pb.go +++ b/internal/proto/datapb/data_service.pb.go @@ -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. -- GitLab