From d1ee0bb43b50366ae33eef5e007e5b7f1b4c0cf1 Mon Sep 17 00:00:00 2001 From: bigsheeper Date: Tue, 15 Nov 2022 14:07:07 +0800 Subject: [PATCH] Add channel level checkpoint [#20350] (#20542) Signed-off-by: bigsheeper Signed-off-by: bigsheeper --- internal/datacoord/channel_manager.go | 13 +- internal/datacoord/cluster.go | 7 +- internal/datacoord/cluster_test.go | 8 +- internal/datacoord/handler.go | 128 ++-- internal/datacoord/meta.go | 66 +- internal/datacoord/meta_test.go | 56 ++ internal/datacoord/mock_test.go | 4 +- internal/datacoord/server.go | 42 +- internal/datacoord/server_test.go | 121 +++- internal/datacoord/services.go | 23 + internal/datanode/buffer.go | 95 ++- internal/datanode/buffer_test.go | 29 +- internal/datanode/channel_meta.go | 223 +++++- internal/datanode/channel_meta_test.go | 141 +++- internal/datanode/data_node.go | 11 +- internal/datanode/data_node_test.go | 9 +- internal/datanode/data_sync_service.go | 18 +- internal/datanode/flow_graph_delete_node.go | 26 +- .../datanode/flow_graph_delete_node_test.go | 82 +-- .../datanode/flow_graph_insert_buffer_node.go | 100 ++- .../flow_graph_insert_buffer_node_test.go | 18 +- .../datanode/flow_graph_time_tick_node.go | 125 ++++ internal/datanode/flush_manager.go | 3 + internal/datanode/flush_task.go | 5 +- internal/datanode/mock_test.go | 6 + internal/datanode/segment.go | 68 +- internal/datanode/segment_sync_policy.go | 39 ++ internal/datanode/segment_sync_policy_test.go | 56 ++ .../distributed/datacoord/client/client.go | 19 + internal/distributed/datacoord/service.go | 5 + .../distributed/datacoord/service_test.go | 16 + internal/distributed/proxy/service_test.go | 4 + internal/metastore/catalog.go | 7 +- internal/metastore/kv/datacoord/constant.go | 1 + internal/metastore/kv/datacoord/kv_catalog.go | 52 +- .../metastore/kv/datacoord/kv_catalog_test.go | 83 ++- internal/proto/data_coord.proto | 8 +- internal/proto/datapb/data_coord.pb.go | 662 ++++++++++-------- internal/proxy/data_coord_mock_test.go | 7 + internal/types/types.go | 2 + internal/util/mock/grpc_datacoord_client.go | 4 + 41 files changed, 1696 insertions(+), 696 deletions(-) create mode 100644 internal/datanode/flow_graph_time_tick_node.go create mode 100644 internal/datanode/segment_sync_policy.go create mode 100644 internal/datanode/segment_sync_policy_test.go diff --git a/internal/datacoord/channel_manager.go b/internal/datacoord/channel_manager.go index 63fefc31e4..63e7f19929 100644 --- a/internal/datacoord/channel_manager.go +++ b/internal/datacoord/channel_manager.go @@ -240,7 +240,10 @@ func (c *ChannelManager) unwatchDroppedChannels() { log.Warn("unable to remove channel", zap.String("channel", ch.Name), zap.Error(err)) continue } - c.h.FinishDropChannel(ch.Name) + err = c.h.FinishDropChannel(ch.Name) + if err != nil { + log.Warn("FinishDropChannel failed when unwatchDroppedChannels", zap.String("channel", ch.Name), zap.Error(err)) + } } } } @@ -752,7 +755,9 @@ func (c *ChannelManager) Reassign(nodeID UniqueID, channelName string) error { if err := c.remove(nodeID, ch); err != nil { return fmt.Errorf("failed to remove watch info: %v,%s", ch, err.Error()) } - c.h.FinishDropChannel(channelName) + if err := c.h.FinishDropChannel(channelName); err != nil { + return fmt.Errorf("FinishDropChannel failed, err=%s", err) + } log.Info("removed channel assignment", zap.String("channel name", channelName)) return nil } @@ -799,7 +804,9 @@ func (c *ChannelManager) CleanupAndReassign(nodeID UniqueID, channelName string) } log.Info("try to cleanup removal flag ", zap.String("channel name", channelName)) - c.h.FinishDropChannel(channelName) + if err := c.h.FinishDropChannel(channelName); err != nil { + return fmt.Errorf("FinishDropChannel failed, err=%s", err) + } log.Info("removed channel assignment", zap.Any("channel", chToCleanUp)) return nil diff --git a/internal/datacoord/cluster.go b/internal/datacoord/cluster.go index b4457d8fa6..83c9aa7abb 100644 --- a/internal/datacoord/cluster.go +++ b/internal/datacoord/cluster.go @@ -85,7 +85,7 @@ func (c *Cluster) Watch(ch string, collectionID UniqueID) error { // Flush sends flush requests to dataNodes specified // which also according to channels where segments are assigned to. func (c *Cluster) Flush(ctx context.Context, nodeID int64, channel string, - segments []*datapb.SegmentInfo, markSegments []*datapb.SegmentInfo) error { + segments []*datapb.SegmentInfo) error { if !c.channelManager.Match(nodeID, channel) { log.Warn("node is not matched with channel", zap.String("channel", channel), @@ -106,9 +106,8 @@ func (c *Cluster) Flush(ctx context.Context, nodeID int64, channel string, commonpbutil.WithSourceID(Params.DataCoordCfg.GetNodeID()), commonpbutil.WithTargetID(nodeID), ), - CollectionID: ch.CollectionID, - SegmentIDs: lo.Map(segments, getSegmentID), - MarkSegmentIDs: lo.Map(markSegments, getSegmentID), + CollectionID: ch.CollectionID, + SegmentIDs: lo.Map(segments, getSegmentID), } c.sessionManager.Flush(ctx, nodeID, req) diff --git a/internal/datacoord/cluster_test.go b/internal/datacoord/cluster_test.go index 57e4e5c4c7..601c2b9e6b 100644 --- a/internal/datacoord/cluster_test.go +++ b/internal/datacoord/cluster_test.go @@ -535,21 +535,19 @@ func TestCluster_Flush(t *testing.T) { // flush empty should impact nothing assert.NotPanics(t, func() { - err := cluster.Flush(context.Background(), 1, "chan-1", []*datapb.SegmentInfo{}, []*datapb.SegmentInfo{}) + err := cluster.Flush(context.Background(), 1, "chan-1", []*datapb.SegmentInfo{}) assert.NoError(t, err) }) // flush not watched channel assert.NotPanics(t, func() { - err := cluster.Flush(context.Background(), 1, "chan-2", []*datapb.SegmentInfo{{ID: 1}}, - []*datapb.SegmentInfo{{ID: 2}}) + err := cluster.Flush(context.Background(), 1, "chan-2", []*datapb.SegmentInfo{{ID: 1}}) assert.Error(t, err) }) // flush from wrong datanode assert.NotPanics(t, func() { - err := cluster.Flush(context.Background(), 2, "chan-1", []*datapb.SegmentInfo{{ID: 1}}, - []*datapb.SegmentInfo{{ID: 3}}) + err := cluster.Flush(context.Background(), 2, "chan-1", []*datapb.SegmentInfo{{ID: 1}}) assert.Error(t, err) }) diff --git a/internal/datacoord/handler.go b/internal/datacoord/handler.go index 9983b99bfb..7f10632edb 100644 --- a/internal/datacoord/handler.go +++ b/internal/datacoord/handler.go @@ -36,7 +36,7 @@ type Handler interface { // GetDataVChanPositions gets the information recovery needed of a channel for DataNode GetDataVChanPositions(channel *channel, partitionID UniqueID) *datapb.VchannelInfo CheckShouldDropChannel(channel string) bool - FinishDropChannel(channel string) + FinishDropChannel(channel string) error GetCollection(ctx context.Context, collectionID UniqueID) (*collectionInfo, error) } @@ -64,10 +64,7 @@ func (h *ServerHandler) GetDataVChanPositions(channel *channel, partitionID Uniq flushedIDs = make(typeutil.UniqueSet) unflushedIDs = make(typeutil.UniqueSet) droppedIDs = make(typeutil.UniqueSet) - seekPosition *internalpb.MsgPosition ) - var minPosSegID int64 - var minPosTs uint64 for _, s := range segments { if (partitionID > allPartitionID && s.PartitionID != partitionID) || (s.GetStartPosition() == nil && s.GetDmlPosition() == nil) { @@ -86,51 +83,12 @@ func (h *ServerHandler) GetDataVChanPositions(channel *channel, partitionID Uniq } else { unflushedIDs.Insert(s.GetID()) } - - var segmentPosition *internalpb.MsgPosition - if s.GetDmlPosition() != nil { - segmentPosition = s.GetDmlPosition() - } else { - segmentPosition = s.GetStartPosition() - } - if seekPosition == nil || segmentPosition.Timestamp < seekPosition.Timestamp { - minPosSegID = s.GetID() - minPosTs = segmentPosition.GetTimestamp() - seekPosition = segmentPosition - } - } - - if seekPosition != nil { - log.Info("channel seek position set as the minimal segment position", - zap.Int64("segment ID", minPosSegID), - zap.Uint64("position timestamp", minPosTs), - zap.String("realworld position timestamp", tsoutil.ParseAndFormatHybridTs(minPosTs)), - ) - } else { - // use collection start position when segment position is not found - if channel.StartPositions == nil { - collection, err := h.GetCollection(h.s.ctx, channel.CollectionID) - if collection != nil && err == nil { - seekPosition = getCollectionStartPosition(channel.Name, collection) - } - log.Info("NEITHER segment position or channel start position are found, setting channel seek position to collection start position", - zap.Uint64("position timestamp", seekPosition.GetTimestamp()), - zap.String("realworld position timestamp", tsoutil.ParseAndFormatHybridTs(seekPosition.GetTimestamp())), - ) - } else { - // use passed start positions, skip to ask RootCoord. - seekPosition = toMsgPosition(channel.Name, channel.StartPositions) - log.Info("segment position not found, setting channel seek position to channel start position", - zap.Uint64("position timestamp", seekPosition.GetTimestamp()), - zap.String("realworld position timestamp", tsoutil.ParseAndFormatHybridTs(seekPosition.GetTimestamp())), - ) - } } return &datapb.VchannelInfo{ CollectionID: channel.CollectionID, ChannelName: channel.Name, - SeekPosition: seekPosition, + SeekPosition: h.getChannelCheckpoint(channel), FlushedSegmentIds: flushedIDs.Collect(), UnflushedSegmentIds: unflushedIDs.Collect(), DroppedSegmentIds: droppedIDs.Collect(), @@ -160,7 +118,6 @@ func (h *ServerHandler) GetQueryVChanPositions(channel *channel, partitionID Uni indexedIDs = make(typeutil.UniqueSet) unIndexedIDs = make(typeutil.UniqueSet) droppedIDs = make(typeutil.UniqueSet) - seekPosition *internalpb.MsgPosition ) for _, s := range segments { if (partitionID > allPartitionID && s.PartitionID != partitionID) || @@ -191,54 +148,47 @@ func (h *ServerHandler) GetQueryVChanPositions(channel *channel, partitionID Uni } } - for id := range indexedIDs { - var segmentPosition *internalpb.MsgPosition - segment := segmentInfos[id] - if segment.GetDmlPosition() != nil { - segmentPosition = segment.GetDmlPosition() - } else { - segmentPosition = segment.GetStartPosition() - } - - if seekPosition == nil || segmentPosition.Timestamp < seekPosition.Timestamp { - seekPosition = segmentPosition - } + return &datapb.VchannelInfo{ + CollectionID: channel.CollectionID, + ChannelName: channel.Name, + SeekPosition: h.getChannelCheckpoint(channel), + FlushedSegmentIds: indexedIDs.Collect(), + UnflushedSegmentIds: unIndexedIDs.Collect(), + DroppedSegmentIds: droppedIDs.Collect(), } - for id := range unIndexedIDs { - var segmentPosition *internalpb.MsgPosition - segment := segmentInfos[id] - if segment.GetDmlPosition() != nil { - segmentPosition = segment.GetDmlPosition() - } else { - segmentPosition = segment.GetStartPosition() - } +} - if seekPosition == nil || segmentPosition.Timestamp < seekPosition.Timestamp { - seekPosition = segmentPosition - } - } - - // use collection start position when segment position is not found - if seekPosition == nil { +func (h *ServerHandler) getChannelCheckpoint(channel *channel) *internalpb.MsgPosition { + seekPosition := h.s.meta.GetChannelCheckpoint(channel.Name) + if seekPosition != nil { + log.Info("channel seek position set from ChannelCP", + zap.String("channel", channel.Name), + zap.Uint64("position timestamp", seekPosition.Timestamp), + zap.Time("realworld position timestamp", tsoutil.PhysicalTime(seekPosition.GetTimestamp())), + ) + } else { + // use collection start position when segment position is not found if channel.StartPositions == nil { collection, err := h.GetCollection(h.s.ctx, channel.CollectionID) if collection != nil && err == nil { seekPosition = getCollectionStartPosition(channel.Name, collection) } + log.Info("NEITHER segment position or channel start position are found, setting channel seek position to collection start position", + zap.String("channel", channel.Name), + zap.Uint64("position timestamp", seekPosition.GetTimestamp()), + zap.Time("realworld position timestamp", tsoutil.PhysicalTime(seekPosition.GetTimestamp())), + ) } else { - // use passed start positions, skip to ask rootcoord. + // use passed start positions, skip to ask RootCoord. seekPosition = toMsgPosition(channel.Name, channel.StartPositions) + log.Info("segment position not found, setting channel seek position to channel start position", + zap.String("channel", channel.Name), + zap.Uint64("position timestamp", seekPosition.GetTimestamp()), + zap.Time("realworld position timestamp", tsoutil.PhysicalTime(seekPosition.GetTimestamp())), + ) } } - - return &datapb.VchannelInfo{ - CollectionID: channel.CollectionID, - ChannelName: channel.Name, - SeekPosition: seekPosition, - FlushedSegmentIds: indexedIDs.Collect(), - UnflushedSegmentIds: unIndexedIDs.Collect(), - DroppedSegmentIds: droppedIDs.Collect(), - } + return seekPosition } func getCollectionStartPosition(channel string, collectionInfo *collectionInfo) *internalpb.MsgPosition { @@ -311,6 +261,16 @@ func (h *ServerHandler) CheckShouldDropChannel(channel string) bool { // FinishDropChannel cleans up the remove flag for channels // this function is a wrapper of server.meta.FinishDropChannel -func (h *ServerHandler) FinishDropChannel(channel string) { - h.s.meta.catalog.DropChannel(h.s.ctx, channel) +func (h *ServerHandler) FinishDropChannel(channel string) error { + err := h.s.meta.catalog.DropChannel(h.s.ctx, channel) + if err != nil { + log.Warn("DropChannel failed", zap.String("vChannel", channel), zap.Error(err)) + return err + } + err = h.s.meta.DropChannelCheckpoint(channel) + if err != nil { + log.Warn("DropChannelCheckpoint failed", zap.String("vChannel", channel), zap.Error(err)) + return err + } + return nil } diff --git a/internal/datacoord/meta.go b/internal/datacoord/meta.go index 16c3ad824a..fd91120ade 100644 --- a/internal/datacoord/meta.go +++ b/internal/datacoord/meta.go @@ -23,16 +23,13 @@ import ( "sync" "time" - "golang.org/x/exp/maps" - "github.com/golang/protobuf/proto" - - "github.com/milvus-io/milvus/internal/common" - "github.com/milvus-io/milvus/internal/util/typeutil" "go.uber.org/zap" + "golang.org/x/exp/maps" "github.com/milvus-io/milvus-proto/go-api/commonpb" "github.com/milvus-io/milvus-proto/go-api/schemapb" + "github.com/milvus-io/milvus/internal/common" "github.com/milvus-io/milvus/internal/kv" "github.com/milvus-io/milvus/internal/log" "github.com/milvus-io/milvus/internal/metastore" @@ -40,14 +37,17 @@ import ( "github.com/milvus-io/milvus/internal/metrics" "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/internal/proto/internalpb" + "github.com/milvus-io/milvus/internal/util/tsoutil" + "github.com/milvus-io/milvus/internal/util/typeutil" ) type meta struct { sync.RWMutex ctx context.Context catalog metastore.DataCoordCatalog - collections map[UniqueID]*collectionInfo // collection id to collection info - segments *SegmentsInfo // segment id to segment info + collections map[UniqueID]*collectionInfo // collection id to collection info + segments *SegmentsInfo // segment id to segment info + channelCPs map[string]*internalpb.MsgPosition // vChannel -> channel checkpoint/see position } type collectionInfo struct { @@ -65,6 +65,7 @@ func newMeta(ctx context.Context, kv kv.TxnKV, chunkManagerRootPath string) (*me catalog: &datacoord.Catalog{Txn: kv, ChunkManagerRootPath: chunkManagerRootPath}, collections: make(map[UniqueID]*collectionInfo), segments: NewSegmentsInfo(), + channelCPs: make(map[string]*internalpb.MsgPosition), } err := mt.reloadFromKV() if err != nil { @@ -96,6 +97,14 @@ func (m *meta) reloadFromKV() error { } metrics.DataCoordNumStoredRows.WithLabelValues().Set(float64(numStoredRows)) metrics.DataCoordNumStoredRowsCounter.WithLabelValues().Add(float64(numStoredRows)) + + channelCPs, err := m.catalog.ListChannelCheckpoint(m.ctx) + if err != nil { + return err + } + for vChannel, pos := range channelCPs { + m.channelCPs[vChannel] = pos + } return nil } @@ -1130,3 +1139,46 @@ func (m *meta) GetCompactionTo(segmentID int64) *SegmentInfo { } return nil } + +// UpdateChannelCheckpoint updates and saves channel checkpoint. +func (m *meta) UpdateChannelCheckpoint(vChannel string, pos *internalpb.MsgPosition) error { + m.Lock() + defer m.Unlock() + + if pos == nil { + return fmt.Errorf("channelCP is nil, vChannel=%s", vChannel) + } + + oldPosition, ok := m.channelCPs[vChannel] + if !ok || oldPosition.Timestamp < pos.Timestamp { + err := m.catalog.SaveChannelCheckpoint(m.ctx, vChannel, pos) + if err != nil { + return err + } + m.channelCPs[vChannel] = pos + ts, _ := tsoutil.ParseTS(pos.Timestamp) + log.Debug("UpdateChannelCheckpoint done", zap.String("vChannel", vChannel), zap.Time("time", ts)) + } + return nil +} + +func (m *meta) GetChannelCheckpoint(vChannel string) *internalpb.MsgPosition { + m.RLock() + defer m.RUnlock() + if m.channelCPs[vChannel] == nil { + return nil + } + return proto.Clone(m.channelCPs[vChannel]).(*internalpb.MsgPosition) +} + +func (m *meta) DropChannelCheckpoint(vChannel string) error { + m.Lock() + defer m.Unlock() + err := m.catalog.DropChannelCheckpoint(m.ctx, vChannel) + if err != nil { + return err + } + delete(m.channelCPs, vChannel) + log.Debug("DropChannelCheckpoint done", zap.String("vChannel", vChannel)) + return nil +} diff --git a/internal/datacoord/meta_test.go b/internal/datacoord/meta_test.go index b1c880077a..14aeb01e5e 100644 --- a/internal/datacoord/meta_test.go +++ b/internal/datacoord/meta_test.go @@ -57,6 +57,11 @@ func (mek *mockEtcdKv) LoadWithPrefix(key string) ([]string, []string, error) { case strings.Contains(key, datacoord.SegmentStatslogPathPrefix): segInfo := getFieldBinlogPaths(1, "statslog1") val, _ = proto.Marshal(segInfo) + case strings.Contains(key, datacoord.ChannelCheckpointPrefix): + channelCP := &internalpb.MsgPosition{ + Timestamp: 1000, + } + val, _ = proto.Marshal(channelCP) default: return nil, nil, fmt.Errorf("invalid key") } @@ -1047,3 +1052,54 @@ func equalCollectionInfo(t *testing.T, a *collectionInfo, b *collectionInfo) { assert.Equal(t, a.Properties, b.Properties) assert.Equal(t, a.StartPositions, b.StartPositions) } + +func TestChannelCP(t *testing.T) { + mockVChannel := "fake-by-dev-rootcoord-dml-1-testchannelcp-v0" + mockPChannel := "fake-by-dev-rootcoord-dml-1" + + pos := &internalpb.MsgPosition{ + ChannelName: mockPChannel, + MsgID: []byte{}, + Timestamp: 1000, + } + + t.Run("UpdateChannelCheckpoint", func(t *testing.T) { + meta, err := newMeta(context.TODO(), memkv.NewMemoryKV(), "") + assert.NoError(t, err) + + // nil position + err = meta.UpdateChannelCheckpoint(mockVChannel, nil) + assert.Error(t, err) + + err = meta.UpdateChannelCheckpoint(mockVChannel, pos) + assert.NoError(t, err) + }) + + t.Run("GetChannelCheckpoint", func(t *testing.T) { + meta, err := newMeta(context.TODO(), memkv.NewMemoryKV(), "") + assert.NoError(t, err) + + position := meta.GetChannelCheckpoint(mockVChannel) + assert.Nil(t, position) + + err = meta.UpdateChannelCheckpoint(mockVChannel, pos) + assert.NoError(t, err) + position = meta.GetChannelCheckpoint(mockVChannel) + assert.NotNil(t, position) + assert.True(t, position.ChannelName == pos.ChannelName) + assert.True(t, position.Timestamp == pos.Timestamp) + }) + + t.Run("DropChannelCheckpoint", func(t *testing.T) { + meta, err := newMeta(context.TODO(), memkv.NewMemoryKV(), "") + assert.NoError(t, err) + + err = meta.DropChannelCheckpoint(mockVChannel) + assert.NoError(t, err) + + err = meta.UpdateChannelCheckpoint(mockVChannel, pos) + assert.NoError(t, err) + err = meta.DropChannelCheckpoint(mockVChannel) + assert.NoError(t, err) + }) +} diff --git a/internal/datacoord/mock_test.go b/internal/datacoord/mock_test.go index eac80c0f7d..61c3c76804 100644 --- a/internal/datacoord/mock_test.go +++ b/internal/datacoord/mock_test.go @@ -759,7 +759,9 @@ func (h *mockHandler) CheckShouldDropChannel(channel string) bool { return false } -func (h *mockHandler) FinishDropChannel(channel string) {} +func (h *mockHandler) FinishDropChannel(channel string) error { + return nil +} func (h *mockHandler) GetCollection(_ context.Context, collectionID UniqueID) (*collectionInfo, error) { // empty schema diff --git a/internal/datacoord/server.go b/internal/datacoord/server.go index b77cf780a0..2f64c5d63a 100644 --- a/internal/datacoord/server.go +++ b/internal/datacoord/server.go @@ -567,29 +567,21 @@ func (s *Server) handleTimetickMessage(ctx context.Context, ttMsg *msgstream.Dat } flushableSegments := s.getFlushableSegmentsInfo(flushableIDs) - staleSegments := s.getStaleSegmentsInfo(ch) - staleSegments = s.filterWithFlushableSegments(staleSegments, flushableIDs) - - if len(flushableSegments)+len(staleSegments) == 0 { + if len(flushableSegments) == 0 { return nil } log.Info("start flushing segments", - zap.Int64s("segment IDs", flushableIDs), - zap.Int("# of stale/mark segments", len(staleSegments))) + zap.Int64s("segment IDs", flushableIDs)) // update segment last update triggered time // it's ok to fail flushing, since next timetick after duration will re-trigger s.setLastFlushTime(flushableSegments) - s.setLastFlushTime(staleSegments) - finfo, minfo := make([]*datapb.SegmentInfo, 0, len(flushableSegments)), make([]*datapb.SegmentInfo, 0, len(staleSegments)) + finfo := make([]*datapb.SegmentInfo, 0, len(flushableSegments)) for _, info := range flushableSegments { finfo = append(finfo, info.SegmentInfo) } - for _, info := range staleSegments { - minfo = append(minfo, info.SegmentInfo) - } - err = s.cluster.Flush(s.ctx, ttMsg.GetBase().GetSourceID(), ch, finfo, minfo) + err = s.cluster.Flush(s.ctx, ttMsg.GetBase().GetSourceID(), ch, finfo) if err != nil { log.Warn("handle") return err @@ -626,32 +618,6 @@ func (s *Server) getFlushableSegmentsInfo(flushableIDs []int64) []*SegmentInfo { return res } -func (s *Server) getStaleSegmentsInfo(ch string) []*SegmentInfo { - return s.meta.SelectSegments(func(info *SegmentInfo) bool { - return isSegmentHealthy(info) && - info.GetInsertChannel() == ch && - !info.lastFlushTime.IsZero() && - time.Since(info.lastFlushTime).Minutes() >= segmentTimedFlushDuration && - info.GetNumOfRows() != 0 - }) -} - -func (s *Server) filterWithFlushableSegments(staleSegments []*SegmentInfo, flushableIDs []int64) []*SegmentInfo { - filter := map[int64]struct{}{} - for _, sid := range flushableIDs { - filter[sid] = struct{}{} - } - - res := make([]*SegmentInfo, 0, len(staleSegments)) - for _, sinfo := range staleSegments { - if _, ok := filter[sinfo.GetID()]; ok { - continue - } - res = append(res, sinfo) - } - return res -} - func (s *Server) setLastFlushTime(segments []*SegmentInfo) { for _, sinfo := range segments { s.meta.SetLastFlushTime(sinfo.GetID(), time.Now()) diff --git a/internal/datacoord/server_test.go b/internal/datacoord/server_test.go index 09ebf591ae..6676e39989 100644 --- a/internal/datacoord/server_test.go +++ b/internal/datacoord/server_test.go @@ -1644,6 +1644,63 @@ func TestDataNodeTtChannel(t *testing.T) { }) } +func TestGetChannelCheckpoint(t *testing.T) { + svr := newTestServer(t, nil) + defer closeTestServer(t, svr) + schema := newTestSchema() + svr.meta.AddCollection(&collectionInfo{ + ID: 0, + Schema: schema, + StartPositions: []*commonpb.KeyDataPair{ + { + Key: "ch1", + Data: []byte{8, 9, 10}, + }, + }, + }) + svr.meta.AddCollection(&collectionInfo{ + ID: 1, + Schema: schema, + StartPositions: []*commonpb.KeyDataPair{ + { + Key: "ch0", + Data: []byte{11, 12, 13}, + }, + }, + }) + + t.Run("get non-existent channel", func(t *testing.T) { + channelCP := svr.handler.(*ServerHandler).getChannelCheckpoint(&channel{Name: "chx1", CollectionID: 0}) + assert.Nil(t, channelCP) + }) + + t.Run("get no channelCP in meta", func(t *testing.T) { + channelCP := svr.handler.(*ServerHandler).getChannelCheckpoint(&channel{Name: "ch1", CollectionID: 0}) + assert.NotNil(t, channelCP) + assert.EqualValues(t, []byte{8, 9, 10}, channelCP.GetMsgID()) + channelCP = svr.handler.(*ServerHandler).getChannelCheckpoint(&channel{Name: "ch0", CollectionID: 1}) + assert.NotNil(t, channelCP) + assert.EqualValues(t, []byte{11, 12, 13}, channelCP.GetMsgID()) + }) + + t.Run("empty collection", func(t *testing.T) { + channelCP := svr.handler.(*ServerHandler).getChannelCheckpoint(&channel{Name: "ch0_suffix", CollectionID: 2}) + assert.Nil(t, channelCP) + }) + + t.Run("with channel cp", func(t *testing.T) { + err := svr.meta.UpdateChannelCheckpoint("ch1", &internalpb.MsgPosition{ + ChannelName: "ch1", + Timestamp: 100, + }) + assert.NoError(t, err) + channelCP := svr.handler.(*ServerHandler).getChannelCheckpoint(&channel{Name: "ch1", CollectionID: 1}) + assert.NotNil(t, channelCP) + assert.True(t, channelCP.ChannelName == "ch1") + assert.True(t, channelCP.Timestamp == 100) + }) +} + func TestGetDataVChanPositions(t *testing.T) { svr := newTestServer(t, nil) defer closeTestServer(t, svr) @@ -1731,7 +1788,6 @@ func TestGetDataVChanPositions(t *testing.T) { assert.EqualValues(t, 1, vchan.FlushedSegmentIds[0]) assert.EqualValues(t, 2, len(vchan.UnflushedSegmentIds)) assert.ElementsMatch(t, []int64{s2.ID, s3.ID}, vchan.UnflushedSegmentIds) - assert.EqualValues(t, []byte{1, 2, 3}, vchan.GetSeekPosition().GetMsgID()) }) t.Run("empty collection", func(t *testing.T) { @@ -1739,7 +1795,6 @@ func TestGetDataVChanPositions(t *testing.T) { assert.EqualValues(t, 1, infos.CollectionID) assert.EqualValues(t, 0, len(infos.FlushedSegmentIds)) assert.EqualValues(t, 0, len(infos.UnflushedSegmentIds)) - assert.EqualValues(t, []byte{8, 9, 10}, infos.SeekPosition.MsgID) }) t.Run("filter partition", func(t *testing.T) { @@ -1747,7 +1802,6 @@ func TestGetDataVChanPositions(t *testing.T) { assert.EqualValues(t, 0, infos.CollectionID) assert.EqualValues(t, 0, len(infos.FlushedSegmentIds)) assert.EqualValues(t, 1, len(infos.UnflushedSegmentIds)) - assert.EqualValues(t, []byte{11, 12, 13}, infos.SeekPosition.MsgID) }) t.Run("empty collection with passed positions", func(t *testing.T) { @@ -1760,7 +1814,6 @@ func TestGetDataVChanPositions(t *testing.T) { }, allPartitionID) assert.EqualValues(t, 0, infos.CollectionID) assert.EqualValues(t, vchannel, infos.ChannelName) - assert.EqualValues(t, []byte{14, 15, 16}, infos.SeekPosition.MsgID) }) } @@ -1875,7 +1928,6 @@ func TestGetQueryVChanPositions(t *testing.T) { assert.EqualValues(t, 1, vchan.FlushedSegmentIds[0]) assert.EqualValues(t, 2, len(vchan.UnflushedSegmentIds)) assert.ElementsMatch(t, []int64{s2.ID, s3.ID}, vchan.UnflushedSegmentIds) - assert.EqualValues(t, []byte{1, 2, 3}, vchan.GetSeekPosition().GetMsgID()) }) t.Run("empty collection", func(t *testing.T) { @@ -1883,7 +1935,6 @@ func TestGetQueryVChanPositions(t *testing.T) { assert.EqualValues(t, 1, infos.CollectionID) assert.EqualValues(t, 0, len(infos.FlushedSegmentIds)) assert.EqualValues(t, 0, len(infos.UnflushedSegmentIds)) - assert.EqualValues(t, []byte{8, 9, 10}, infos.SeekPosition.MsgID) }) t.Run("filter partition", func(t *testing.T) { @@ -1891,7 +1942,6 @@ func TestGetQueryVChanPositions(t *testing.T) { assert.EqualValues(t, 0, infos.CollectionID) assert.EqualValues(t, 0, len(infos.FlushedSegmentIds)) assert.EqualValues(t, 1, len(infos.UnflushedSegmentIds)) - assert.EqualValues(t, []byte{11, 12, 13}, infos.SeekPosition.MsgID) }) t.Run("empty collection with passed positions", func(t *testing.T) { @@ -1904,7 +1954,6 @@ func TestGetQueryVChanPositions(t *testing.T) { }, allPartitionID) assert.EqualValues(t, 0, infos.CollectionID) assert.EqualValues(t, vchannel, infos.ChannelName) - assert.EqualValues(t, []byte{14, 15, 16}, infos.SeekPosition.MsgID) }) t.Run("filter non indexed segments", func(t *testing.T) { @@ -1916,7 +1965,6 @@ func TestGetQueryVChanPositions(t *testing.T) { assert.EqualValues(t, 0, len(vchan.FlushedSegmentIds)) assert.EqualValues(t, 3, len(vchan.UnflushedSegmentIds)) assert.ElementsMatch(t, []int64{s1.ID, s2.ID, s3.ID}, vchan.UnflushedSegmentIds) - assert.EqualValues(t, []byte{1, 2, 3}, vchan.GetSeekPosition().GetMsgID()) }) } @@ -2123,9 +2171,16 @@ func TestGetRecoveryInfo(t *testing.T) { svr.meta.AddCollection(&collectionInfo{ Schema: newTestSchema(), }) + + err := svr.meta.UpdateChannelCheckpoint("vchan1", &internalpb.MsgPosition{ + ChannelName: "vchan1", + Timestamp: 10, + }) + assert.NoError(t, err) + seg1 := createSegment(0, 0, 0, 100, 10, "vchan1", commonpb.SegmentState_Flushed) seg2 := createSegment(1, 0, 0, 100, 20, "vchan1", commonpb.SegmentState_Flushed) - err := svr.meta.AddSegment(NewSegmentInfo(seg1)) + err = svr.meta.AddSegment(NewSegmentInfo(seg1)) assert.Nil(t, err) err = svr.meta.AddSegment(NewSegmentInfo(seg2)) assert.Nil(t, err) @@ -2185,9 +2240,15 @@ func TestGetRecoveryInfo(t *testing.T) { Schema: newTestSchema(), }) + err := svr.meta.UpdateChannelCheckpoint("vchan1", &internalpb.MsgPosition{ + ChannelName: "vchan1", + Timestamp: 0, + }) + assert.NoError(t, err) + seg1 := createSegment(3, 0, 0, 100, 30, "vchan1", commonpb.SegmentState_Growing) seg2 := createSegment(4, 0, 0, 100, 40, "vchan1", commonpb.SegmentState_Growing) - err := svr.meta.AddSegment(NewSegmentInfo(seg1)) + err = svr.meta.AddSegment(NewSegmentInfo(seg1)) assert.Nil(t, err) err = svr.meta.AddSegment(NewSegmentInfo(seg2)) assert.Nil(t, err) @@ -2319,9 +2380,15 @@ func TestGetRecoveryInfo(t *testing.T) { Schema: newTestSchema(), }) + err := svr.meta.UpdateChannelCheckpoint("vchan1", &internalpb.MsgPosition{ + ChannelName: "vchan1", + Timestamp: 0, + }) + assert.NoError(t, err) + seg1 := createSegment(7, 0, 0, 100, 30, "vchan1", commonpb.SegmentState_Growing) seg2 := createSegment(8, 0, 0, 100, 40, "vchan1", commonpb.SegmentState_Dropped) - err := svr.meta.AddSegment(NewSegmentInfo(seg1)) + err = svr.meta.AddSegment(NewSegmentInfo(seg1)) assert.Nil(t, err) err = svr.meta.AddSegment(NewSegmentInfo(seg2)) assert.Nil(t, err) @@ -3136,6 +3203,36 @@ func TestDataCoord_MarkSegmentsDropped(t *testing.T) { }) } +func TestDataCoordServer_UpdateChannelCheckpoint(t *testing.T) { + mockVChannel := "fake-by-dev-rootcoord-dml-1-testchannelcp-v0" + mockPChannel := "fake-by-dev-rootcoord-dml-1" + + t.Run("UpdateChannelCheckpoint", func(t *testing.T) { + svr := newTestServer(t, nil) + defer closeTestServer(t, svr) + + req := &datapb.UpdateChannelCheckpointRequest{ + Base: &commonpb.MsgBase{ + SourceID: Params.DataCoordCfg.GetNodeID(), + }, + VChannel: mockVChannel, + Position: &internalpb.MsgPosition{ + ChannelName: mockPChannel, + Timestamp: 1000, + }, + } + + resp, err := svr.UpdateChannelCheckpoint(context.TODO(), req) + assert.NoError(t, err) + assert.EqualValues(t, commonpb.ErrorCode_Success, resp.ErrorCode) + + req.Position = nil + resp, err = svr.UpdateChannelCheckpoint(context.TODO(), req) + assert.NoError(t, err) + assert.EqualValues(t, commonpb.ErrorCode_UnexpectedError, resp.ErrorCode) + }) +} + // https://github.com/milvus-io/milvus/issues/15659 func TestIssue15659(t *testing.T) { ctx, cancel := context.WithCancel(context.Background()) diff --git a/internal/datacoord/services.go b/internal/datacoord/services.go index d5cd1979ec..eebee839c1 100644 --- a/internal/datacoord/services.go +++ b/internal/datacoord/services.go @@ -1171,6 +1171,29 @@ func (s *Server) UpdateSegmentStatistics(ctx context.Context, req *datapb.Update }, nil } +// UpdateChannelCheckpoint updates channel checkpoint in dataCoord. +func (s *Server) UpdateChannelCheckpoint(ctx context.Context, req *datapb.UpdateChannelCheckpointRequest) (*commonpb.Status, error) { + resp := &commonpb.Status{ + ErrorCode: commonpb.ErrorCode_UnexpectedError, + } + if s.isClosed() { + log.Warn("failed to update channel position for closed server") + resp.Reason = msgDataCoordIsUnhealthy(Params.DataCoordCfg.GetNodeID()) + return resp, nil + } + + err := s.meta.UpdateChannelCheckpoint(req.GetVChannel(), req.GetPosition()) + if err != nil { + log.Warn("failed to UpdateChannelCheckpoint", zap.String("vChannel", req.GetVChannel()), zap.Error(err)) + resp.Reason = err.Error() + return resp, nil + } + + return &commonpb.Status{ + ErrorCode: commonpb.ErrorCode_Success, + }, nil +} + // getDiff returns the difference of base and remove. i.e. all items that are in `base` but not in `remove`. func getDiff(base, remove []int64) []int64 { mb := make(map[int64]struct{}, len(remove)) diff --git a/internal/datanode/buffer.go b/internal/datanode/buffer.go index 87780bb813..5fdcbc99e1 100644 --- a/internal/datanode/buffer.go +++ b/internal/datanode/buffer.go @@ -21,14 +21,15 @@ import ( "errors" "fmt" "math" - "sync" + + "go.uber.org/zap" "github.com/milvus-io/milvus-proto/go-api/schemapb" "github.com/milvus-io/milvus/internal/log" "github.com/milvus-io/milvus/internal/metrics" "github.com/milvus-io/milvus/internal/proto/datapb" + "github.com/milvus-io/milvus/internal/proto/internalpb" "github.com/milvus-io/milvus/internal/storage" - "go.uber.org/zap" ) // DelBufferManager is in charge of managing insertBuf and delBuf from an overall prospect @@ -36,37 +37,37 @@ import ( // insert/delete flush when the memory usage of the whole manager reach a certain level. // but at the first stage, this struct is only used for delete buff type DelBufferManager struct { - delBufMap sync.Map // map[segmentID]*DelDataBuf + channel Channel delMemorySize int64 delBufHeap *PriorityQueue } func (bm *DelBufferManager) GetSegDelBufMemSize(segID UniqueID) int64 { - if delDataBuf, ok := bm.delBufMap.Load(segID); ok { - return delDataBuf.(*DelDataBuf).item.memorySize + if delDataBuf, ok := bm.channel.getCurDeleteBuffer(segID); ok { + return delDataBuf.item.memorySize } return 0 } func (bm *DelBufferManager) GetEntriesNum(segID UniqueID) int64 { - if delDataBuf, ok := bm.delBufMap.Load(segID); ok { - return delDataBuf.(*DelDataBuf).GetEntriesNum() + if delDataBuf, ok := bm.channel.getCurDeleteBuffer(segID); ok { + return delDataBuf.GetEntriesNum() } return 0 } // Store :the method only for unit test func (bm *DelBufferManager) Store(segID UniqueID, delDataBuf *DelDataBuf) { - bm.delBufMap.Store(segID, delDataBuf) + bm.channel.setCurDeleteBuffer(segID, delDataBuf) } func (bm *DelBufferManager) StoreNewDeletes(segID UniqueID, pks []primaryKey, - tss []Timestamp, tr TimeRange) { + tss []Timestamp, tr TimeRange, startPos, endPos *internalpb.MsgPosition) { //1. load or create delDataBuf var delDataBuf *DelDataBuf - value, loaded := bm.delBufMap.Load(segID) + buffer, loaded := bm.channel.getCurDeleteBuffer(segID) if loaded { - delDataBuf = value.(*DelDataBuf) + delDataBuf = buffer } else { delDataBuf = newDelDataBuf() } @@ -91,6 +92,7 @@ func (bm *DelBufferManager) StoreNewDeletes(segID UniqueID, pks []primaryKey, //3. update statistics of del data delDataBuf.accumulateEntriesNum(int64(rowCount)) delDataBuf.updateTimeRange(tr) + delDataBuf.updateStartAndEndPosition(startPos, endPos) //4. update and sync memory size with priority queue if !loaded { @@ -100,7 +102,7 @@ func (bm *DelBufferManager) StoreNewDeletes(segID UniqueID, pks []primaryKey, } else { bm.delBufHeap.update(delDataBuf.item, delDataBuf.item.memorySize+bufSize) } - bm.delBufMap.Store(segID, delDataBuf) + bm.channel.setCurDeleteBuffer(segID, delDataBuf) bm.delMemorySize += bufSize //4. sync metrics metrics.DataNodeConsumeMsgRowsCount.WithLabelValues( @@ -108,19 +110,19 @@ func (bm *DelBufferManager) StoreNewDeletes(segID UniqueID, pks []primaryKey, } func (bm *DelBufferManager) Load(segID UniqueID) (delDataBuf *DelDataBuf, ok bool) { - value, ok := bm.delBufMap.Load(segID) + buffer, ok := bm.channel.getCurDeleteBuffer(segID) if ok { - return value.(*DelDataBuf), ok + return buffer, ok } return nil, ok } func (bm *DelBufferManager) Delete(segID UniqueID) { - if buf, ok := bm.delBufMap.Load(segID); ok { - item := buf.(*DelDataBuf).item + if buf, ok := bm.channel.getCurDeleteBuffer(segID); ok { + item := buf.item bm.delMemorySize -= item.memorySize heap.Remove(bm.delBufHeap, item.index) - bm.delBufMap.Delete(segID) + bm.channel.rollDeleteBuffer(segID) } } @@ -154,7 +156,7 @@ func (bm *DelBufferManager) CompactSegBuf(compactedToSegID UniqueID, compactedFr //note that when compacting segment in del buffer manager //there is no need to modify the general memory size as there is no new //added del into the memory - bm.delBufMap.Store(compactedToSegID, compactToDelBuff) + bm.channel.setCurDeleteBuffer(compactedToSegID, compactToDelBuff) } } @@ -236,11 +238,13 @@ func (pq *PriorityQueue) update(item *Item, memorySize int64) { // BufferData buffers insert data, monitoring buffer size and limit // size and limit both indicate numOfRows type BufferData struct { - buffer *InsertData - size int64 - limit int64 - tsFrom Timestamp - tsTo Timestamp + buffer *InsertData + size int64 + limit int64 + tsFrom Timestamp + tsTo Timestamp + startPos *internalpb.MsgPosition + endPos *internalpb.MsgPosition } func (bd *BufferData) effectiveCap() int64 { @@ -261,12 +265,23 @@ func (bd *BufferData) updateTimeRange(tr TimeRange) { } } +func (bd *BufferData) updateStartAndEndPosition(startPos *internalpb.MsgPosition, endPos *internalpb.MsgPosition) { + if bd.startPos == nil || startPos.Timestamp < bd.startPos.Timestamp { + bd.startPos = startPos + } + if bd.endPos == nil || endPos.Timestamp > bd.endPos.Timestamp { + bd.endPos = endPos + } +} + // DelDataBuf buffers delete data, monitoring buffer size and limit // size and limit both indicate numOfRows type DelDataBuf struct { datapb.Binlog - delData *DeleteData - item *Item + delData *DeleteData + item *Item + startPos *internalpb.MsgPosition + endPos *internalpb.MsgPosition } func (ddb *DelDataBuf) accumulateEntriesNum(entryNum int64) { @@ -293,6 +308,15 @@ func (ddb *DelDataBuf) mergeDelDataBuf(buf *DelDataBuf) { ddb.item.memorySize += buf.item.memorySize } +func (ddb *DelDataBuf) updateStartAndEndPosition(startPos *internalpb.MsgPosition, endPos *internalpb.MsgPosition) { + if ddb.startPos == nil || startPos.Timestamp < ddb.startPos.Timestamp { + ddb.startPos = startPos + } + if ddb.endPos == nil || endPos.Timestamp > ddb.endPos.Timestamp { + ddb.endPos = endPos + } +} + // newBufferData needs an input dimension to calculate the limit of this buffer // // `limit` is the segment numOfRows a buffer can buffer at most. @@ -311,12 +335,29 @@ func (ddb *DelDataBuf) mergeDelDataBuf(buf *DelDataBuf) { // to fit in both types of vector fields // // * This need to change for string field support and multi-vector fields support. -func newBufferData(dimension int64) (*BufferData, error) { +func newBufferData(collSchema *schemapb.CollectionSchema) (*BufferData, error) { + // Get Dimension + // TODO GOOSE: under assumption that there's only 1 Vector field in one collection schema + var dimension int + var err error + for _, field := range collSchema.Fields { + if field.DataType == schemapb.DataType_FloatVector || + field.DataType == schemapb.DataType_BinaryVector { + + dimension, err = storage.GetDimFromParams(field.TypeParams) + if err != nil { + log.Error("failed to get dim from field", zap.Error(err)) + return nil, err + } + break + } + } + if dimension == 0 { return nil, errors.New("Invalid dimension") } - limit := Params.DataNodeCfg.FlushInsertBufferSize / (dimension * 4) + limit := Params.DataNodeCfg.FlushInsertBufferSize / (int64(dimension) * 4) //TODO::xige-16 eval vec and string field return &BufferData{ diff --git a/internal/datanode/buffer_test.go b/internal/datanode/buffer_test.go index 96db0a5144..4034809cef 100644 --- a/internal/datanode/buffer_test.go +++ b/internal/datanode/buffer_test.go @@ -17,13 +17,38 @@ package datanode import ( + "fmt" "math" "testing" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" + + "github.com/milvus-io/milvus-proto/go-api/commonpb" + "github.com/milvus-io/milvus-proto/go-api/schemapb" ) +func genTestCollectionSchema(dim int64) *schemapb.CollectionSchema { + floatVecFieldSchema := &schemapb.FieldSchema{ + FieldID: 100, + Name: "vec", + DataType: schemapb.DataType_FloatVector, + TypeParams: []*commonpb.KeyValuePair{ + { + Key: "dim", + Value: fmt.Sprintf("%d", dim), + }, + }, + } + schema := &schemapb.CollectionSchema{ + Name: "collection-0", + Fields: []*schemapb.FieldSchema{ + floatVecFieldSchema, + }, + } + return schema +} + func TestBufferData(t *testing.T) { Params.DataNodeCfg.FlushInsertBufferSize = 16 * (1 << 20) // 16 MB @@ -43,7 +68,7 @@ func TestBufferData(t *testing.T) { for _, test := range tests { t.Run(test.description, func(t *testing.T) { - idata, err := newBufferData(test.indim) + idata, err := newBufferData(genTestCollectionSchema(test.indim)) if test.isValid { assert.NoError(t, err) @@ -101,7 +126,7 @@ func TestBufferData_updateTimeRange(t *testing.T) { for _, tc := range cases { t.Run(tc.tag, func(t *testing.T) { - bd, err := newBufferData(16) + bd, err := newBufferData(genTestCollectionSchema(16)) require.NoError(t, err) for _, tr := range tc.trs { bd.updateTimeRange(tr) diff --git a/internal/datanode/channel_meta.go b/internal/datanode/channel_meta.go index bf920a9ef7..fd727fbe73 100644 --- a/internal/datanode/channel_meta.go +++ b/internal/datanode/channel_meta.go @@ -19,9 +19,12 @@ package datanode import ( "context" "fmt" + "math" "sync" "time" + "go.uber.org/zap" + "github.com/milvus-io/milvus-proto/go-api/schemapb" "github.com/milvus-io/milvus/internal/common" "github.com/milvus-io/milvus/internal/log" @@ -31,7 +34,6 @@ import ( "github.com/milvus-io/milvus/internal/storage" "github.com/milvus-io/milvus/internal/types" "github.com/milvus-io/milvus/internal/util/typeutil" - "go.uber.org/zap" ) type ( @@ -59,18 +61,31 @@ type Channel interface { filterSegments(partitionID UniqueID) []*Segment listNewSegmentsStartPositions() []*datapb.SegmentStartPosition transferNewSegments(segmentIDs []UniqueID) - updateSegmentEndPosition(segID UniqueID, endPos *internalpb.MsgPosition) updateSegmentPKRange(segID UniqueID, ids storage.FieldData) mergeFlushedSegments(seg *Segment, planID UniqueID, compactedFrom []UniqueID) error hasSegment(segID UniqueID, countFlushed bool) bool removeSegments(segID ...UniqueID) listCompactedSegmentIDs() map[UniqueID][]UniqueID + listSegmentIDsToSync(ts Timestamp) []UniqueID + setSegmentLastSyncTs(segID UniqueID, ts Timestamp) updateStatistics(segID UniqueID, numRows int64) InitPKstats(ctx context.Context, s *Segment, statsBinlogs []*datapb.FieldBinlog, ts Timestamp) error RollPKstats(segID UniqueID, stats []*storage.PrimaryKeyStats) getSegmentStatisticsUpdates(segID UniqueID) (*datapb.SegmentStats, error) segmentFlushed(segID UniqueID) + + getChannelCheckpoint(ttPos *internalpb.MsgPosition) *internalpb.MsgPosition + + getCurInsertBuffer(segmentID UniqueID) (*BufferData, bool) + setCurInsertBuffer(segmentID UniqueID, buf *BufferData) + rollInsertBuffer(segmentID UniqueID) + evictHistoryInsertBuffer(segmentID UniqueID, endPos *internalpb.MsgPosition) + + getCurDeleteBuffer(segmentID UniqueID) (*DelDataBuf, bool) + setCurDeleteBuffer(segmentID UniqueID, buf *DelDataBuf) + rollDeleteBuffer(segmentID UniqueID) + evictHistoryDeleteBuffer(segmentID UniqueID, endPos *internalpb.MsgPosition) } // ChannelMeta contains channel meta and the latest segments infos of the channel. @@ -83,6 +98,8 @@ type ChannelMeta struct { segMu sync.RWMutex segments map[UniqueID]*Segment + syncPolicies []segmentSyncPolicy + metaService *metaService chunkManager storage.ChunkManager } @@ -99,6 +116,10 @@ func newChannel(channelName string, collID UniqueID, schema *schemapb.Collection segments: make(map[UniqueID]*Segment), + syncPolicies: []segmentSyncPolicy{ + syncPeriodically(), + }, + metaService: metaService, chunkManager: cm, } @@ -178,14 +199,15 @@ func (c *ChannelMeta) addSegment(req addSegmentReq) error { zap.Bool("importing", req.importing), ) seg := &Segment{ - collectionID: req.collID, - partitionID: req.partitionID, - segmentID: req.segID, - numRows: req.numOfRows, // 0 if segType == NEW - startPos: req.startPos, - endPos: req.endPos, + collectionID: req.collID, + partitionID: req.partitionID, + segmentID: req.segID, + numRows: req.numOfRows, // 0 if segType == NEW + historyInsertBuf: make([]*BufferData, 0), + historyDeleteBuf: make([]*DelDataBuf, 0), + startPos: req.startPos, } - seg.sType.Store(req.segType) + seg.setType(req.segType) // Set up pk stats err := c.InitPKstats(context.TODO(), seg, req.statsBinLogs, req.recoverTs) if err != nil { @@ -218,6 +240,33 @@ func (c *ChannelMeta) listCompactedSegmentIDs() map[UniqueID][]UniqueID { return compactedTo2From } +func (c *ChannelMeta) listSegmentIDsToSync(ts Timestamp) []UniqueID { + c.segMu.RLock() + defer c.segMu.RUnlock() + + segIDsToSync := make([]UniqueID, 0) + for segID, seg := range c.segments { + if !seg.isValid() { + continue + } + for _, policy := range c.syncPolicies { + if policy(seg, ts) { + segIDsToSync = append(segIDsToSync, segID) + break + } + } + } + return segIDsToSync +} + +func (c *ChannelMeta) setSegmentLastSyncTs(segID UniqueID, ts Timestamp) { + c.segMu.Lock() + defer c.segMu.Unlock() + if _, ok := c.segments[segID]; ok { + c.segments[segID].lastSyncTs = ts + } +} + // filterSegments return segments with same partitionID for all segments // get all segments func (c *ChannelMeta) filterSegments(partitionID UniqueID) []*Segment { @@ -355,20 +404,6 @@ func (c *ChannelMeta) transferNewSegments(segmentIDs []UniqueID) { } } -// updateSegmentEndPosition updates *New* or *Normal* segment's end position. -func (c *ChannelMeta) updateSegmentEndPosition(segID UniqueID, endPos *internalpb.MsgPosition) { - c.segMu.Lock() - defer c.segMu.Unlock() - - seg, ok := c.segments[segID] - if ok && seg.notFlushed() { - seg.endPos = endPos - return - } - - log.Warn("No match segment", zap.Int64("ID", segID)) -} - func (c *ChannelMeta) updateSegmentPKRange(segID UniqueID, ids storage.FieldData) { c.segMu.Lock() defer c.segMu.Unlock() @@ -389,10 +424,15 @@ func (c *ChannelMeta) removeSegments(segIDs ...UniqueID) { log.Info("remove segments if exist", zap.Int64s("segmentIDs", segIDs)) cnt := 0 for _, segID := range segIDs { - seg, ok := c.segments[segID] - if ok && - (seg.getType() == datapb.SegmentType_New || seg.getType() == datapb.SegmentType_Normal) { - cnt++ + if seg, ok := c.segments[segID]; ok { + if seg.notFlushed() { + cnt++ + } + // free memory + seg.curInsertBuf = nil + seg.curDeleteBuf = nil + seg.historyInsertBuf = nil + seg.historyDeleteBuf = nil } delete(c.segments, segID) @@ -605,3 +645,132 @@ func (c *ChannelMeta) listNotFlushedSegmentIDs() []UniqueID { return segIDs } + +func (c *ChannelMeta) getChannelCheckpoint(ttPos *internalpb.MsgPosition) *internalpb.MsgPosition { + c.segMu.RLock() + defer c.segMu.RUnlock() + channelCP := &internalpb.MsgPosition{Timestamp: math.MaxUint64} + // 1. find the earliest startPos in current buffer and history buffer + for _, seg := range c.segments { + if seg.curInsertBuf != nil && seg.curInsertBuf.startPos != nil && seg.curInsertBuf.startPos.Timestamp < channelCP.Timestamp { + channelCP = seg.curInsertBuf.startPos + } + if seg.curDeleteBuf != nil && seg.curDeleteBuf.startPos != nil && seg.curDeleteBuf.startPos.Timestamp < channelCP.Timestamp { + channelCP = seg.curDeleteBuf.startPos + } + for _, ib := range seg.historyInsertBuf { + if ib != nil && ib.startPos != nil && ib.startPos.Timestamp < channelCP.Timestamp { + channelCP = ib.startPos + } + } + for _, db := range seg.historyDeleteBuf { + if db != nil && db.startPos != nil && db.startPos.Timestamp < channelCP.Timestamp { + channelCP = db.startPos + } + } + // TODO: maybe too many logs would print + log.Debug("getChannelCheckpoint for segment", zap.Int64("segmentID", seg.segmentID), + zap.Bool("isCurIBEmpty", seg.curInsertBuf == nil), + zap.Bool("isCurDBEmpty", seg.curDeleteBuf == nil), + zap.Int("len(hisIB)", len(seg.historyInsertBuf)), + zap.Int("len(hisDB)", len(seg.historyDeleteBuf))) + } + // 2. if no data in buffer, use the current tt as channelCP + if channelCP.MsgID == nil { + channelCP = ttPos + } + return channelCP +} + +func (c *ChannelMeta) getCurInsertBuffer(segmentID UniqueID) (*BufferData, bool) { + c.segMu.RLock() + defer c.segMu.RUnlock() + seg, ok := c.segments[segmentID] + if ok { + return seg.curInsertBuf, seg.curInsertBuf != nil + } + return nil, false +} + +func (c *ChannelMeta) setCurInsertBuffer(segmentID UniqueID, buf *BufferData) { + c.segMu.Lock() + defer c.segMu.Unlock() + + seg, ok := c.segments[segmentID] + if ok { + seg.curInsertBuf = buf + return + } + log.Warn("cannot find segment when setCurInsertBuffer", zap.Int64("segmentID", segmentID)) +} + +func (c *ChannelMeta) rollInsertBuffer(segmentID UniqueID) { + c.segMu.Lock() + defer c.segMu.Unlock() + + seg, ok := c.segments[segmentID] + if ok { + seg.rollInsertBuffer() + return + } + log.Warn("cannot find segment when rollInsertBuffer", zap.Int64("segmentID", segmentID)) +} + +func (c *ChannelMeta) evictHistoryInsertBuffer(segmentID UniqueID, endPos *internalpb.MsgPosition) { + c.segMu.Lock() + defer c.segMu.Unlock() + + seg, ok := c.segments[segmentID] + if ok { + seg.evictHistoryInsertBuffer(endPos) + return + } + log.Warn("cannot find segment when evictHistoryInsertBuffer", zap.Int64("segmentID", segmentID)) +} + +func (c *ChannelMeta) getCurDeleteBuffer(segmentID UniqueID) (*DelDataBuf, bool) { + c.segMu.RLock() + defer c.segMu.RUnlock() + + seg, ok := c.segments[segmentID] + if ok { + return seg.curDeleteBuf, seg.curDeleteBuf != nil + } + return nil, false +} + +func (c *ChannelMeta) setCurDeleteBuffer(segmentID UniqueID, buf *DelDataBuf) { + c.segMu.Lock() + defer c.segMu.Unlock() + + seg, ok := c.segments[segmentID] + if ok { + seg.curDeleteBuf = buf + return + } + log.Warn("cannot find segment when setCurDeleteBuffer", zap.Int64("segmentID", segmentID)) +} + +func (c *ChannelMeta) rollDeleteBuffer(segmentID UniqueID) { + c.segMu.Lock() + defer c.segMu.Unlock() + + seg, ok := c.segments[segmentID] + if ok { + seg.rollDeleteBuffer() + return + } + log.Warn("cannot find segment when rollDeleteBuffer", zap.Int64("segmentID", segmentID)) +} + +func (c *ChannelMeta) evictHistoryDeleteBuffer(segmentID UniqueID, endPos *internalpb.MsgPosition) { + c.segMu.Lock() + defer c.segMu.Unlock() + + seg, ok := c.segments[segmentID] + if ok { + seg.evictHistoryDeleteBuffer(endPos) + return + } + log.Warn("cannot find segment when evictHistoryDeleteBuffer", zap.Int64("segmentID", segmentID)) +} diff --git a/internal/datanode/channel_meta_test.go b/internal/datanode/channel_meta_test.go index 92deccbb2d..45b19c2f10 100644 --- a/internal/datanode/channel_meta_test.go +++ b/internal/datanode/channel_meta_test.go @@ -142,7 +142,6 @@ func TestChannelMeta_InnerFunction(t *testing.T) { assert.Equal(t, UniqueID(1), seg.collectionID) assert.Equal(t, UniqueID(2), seg.partitionID) assert.Equal(t, Timestamp(100), seg.startPos.Timestamp) - assert.Equal(t, Timestamp(200), seg.endPos.Timestamp) assert.Equal(t, int64(0), seg.numRows) assert.Equal(t, datapb.SegmentType_New, seg.getType()) @@ -425,41 +424,6 @@ func TestChannelMeta_InterfaceMethod(t *testing.T) { }) }) - t.Run("Test_updateSegmentEndPosition", func(t *testing.T) { - segs := []struct { - segID UniqueID - segType datapb.SegmentType - }{ - {100, datapb.SegmentType_New}, - {200, datapb.SegmentType_Normal}, - {300, datapb.SegmentType_Flushed}, - } - - channel := ChannelMeta{segments: make(map[UniqueID]*Segment)} - for _, seg := range segs { - s := Segment{segmentID: seg.segID} - s.setType(seg.segType) - channel.segMu.Lock() - channel.segments[seg.segID] = &s - channel.segMu.Unlock() - } - - tests := []struct { - inSegID UniqueID - description string - }{ - {100, "seg 100 is type New"}, - {200, "seg 200 is type Normal"}, - {300, "seg 300 is type Flushed"}, - } - - for _, test := range tests { - t.Run(test.description, func(t *testing.T) { - channel.updateSegmentEndPosition(test.inSegID, new(internalpb.MsgPosition)) - }) - } - }) - t.Run("Test_getCollectionSchema", func(t *testing.T) { tests := []struct { isValid bool @@ -787,6 +751,111 @@ func TestChannelMeta_UpdatePKRange(t *testing.T) { } +func TestChannelMeta_ChannelCP(t *testing.T) { + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + rc := &RootCoordFactory{ + pkType: schemapb.DataType_Int64, + } + + mockVChannel := "fake-by-dev-rootcoord-dml-1-testchannelcp-v0" + mockPChannel := "fake-by-dev-rootcoord-dml-1" + + collID := UniqueID(1) + cm := storage.NewLocalChunkManager(storage.RootPath(channelMetaNodeTestDir)) + defer func() { + err := cm.RemoveWithPrefix(ctx, "") + assert.NoError(t, err) + }() + + t.Run("get and set", func(t *testing.T) { + pos := &internalpb.MsgPosition{ + ChannelName: mockPChannel, + Timestamp: 1000, + } + channel := newChannel(mockVChannel, collID, nil, rc, cm) + channel.chunkManager = &mockDataCM{} + position := channel.getChannelCheckpoint(pos) + assert.NotNil(t, position) + assert.True(t, position.ChannelName == pos.ChannelName) + assert.True(t, position.Timestamp == pos.Timestamp) + }) + + t.Run("set insertBuffer&deleteBuffer then get", func(t *testing.T) { + run := func(curInsertPos, curDeletePos *internalpb.MsgPosition, + hisInsertPoss, hisDeletePoss []*internalpb.MsgPosition, + ttPos, expectedPos *internalpb.MsgPosition) { + segmentID := UniqueID(1) + channel := newChannel(mockVChannel, collID, nil, rc, cm) + channel.chunkManager = &mockDataCM{} + err := channel.addSegment( + addSegmentReq{ + segType: datapb.SegmentType_New, + segID: segmentID, + collID: collID, + }) + assert.NoError(t, err) + // set history insert buffers + for _, pos := range hisInsertPoss { + pos.MsgID = []byte{1} + channel.setCurInsertBuffer(segmentID, &BufferData{ + startPos: pos, + }) + channel.rollInsertBuffer(segmentID) + } + // set history delete buffers + for _, pos := range hisDeletePoss { + pos.MsgID = []byte{1} + channel.setCurDeleteBuffer(segmentID, &DelDataBuf{ + startPos: pos, + }) + channel.rollDeleteBuffer(segmentID) + } + // set cur buffers + if curInsertPos != nil { + curInsertPos.MsgID = []byte{1} + channel.setCurInsertBuffer(segmentID, &BufferData{ + startPos: curInsertPos, + }) + } + if curDeletePos != nil { + curDeletePos.MsgID = []byte{1} + channel.setCurDeleteBuffer(segmentID, &DelDataBuf{ + startPos: curDeletePos, + }) + } + // set channelCP + resPos := channel.getChannelCheckpoint(ttPos) + assert.NotNil(t, resPos) + assert.True(t, resPos.ChannelName == expectedPos.ChannelName) + assert.True(t, resPos.Timestamp == expectedPos.Timestamp) + } + + run(&internalpb.MsgPosition{Timestamp: 50}, &internalpb.MsgPosition{Timestamp: 60}, + []*internalpb.MsgPosition{{Timestamp: 70}}, []*internalpb.MsgPosition{{Timestamp: 120}}, + &internalpb.MsgPosition{Timestamp: 120}, &internalpb.MsgPosition{Timestamp: 50}) + + run(&internalpb.MsgPosition{Timestamp: 50}, &internalpb.MsgPosition{Timestamp: 60}, + []*internalpb.MsgPosition{{Timestamp: 70}}, []*internalpb.MsgPosition{{Timestamp: 120}}, + &internalpb.MsgPosition{Timestamp: 30}, &internalpb.MsgPosition{Timestamp: 50}) + + // nil cur buffer + run(nil, nil, + []*internalpb.MsgPosition{{Timestamp: 120}}, []*internalpb.MsgPosition{{Timestamp: 110}}, + &internalpb.MsgPosition{Timestamp: 130}, &internalpb.MsgPosition{Timestamp: 110}) + + // nil history buffer + run(&internalpb.MsgPosition{Timestamp: 50}, &internalpb.MsgPosition{Timestamp: 100}, + nil, nil, + &internalpb.MsgPosition{Timestamp: 100}, &internalpb.MsgPosition{Timestamp: 50}) + + // nil buffer + run(nil, nil, + nil, nil, + &internalpb.MsgPosition{Timestamp: 100}, &internalpb.MsgPosition{Timestamp: 100}) + }) +} + // ChannelMetaSuite setup test suite for ChannelMeta type ChannelMetaSuite struct { suite.Suite diff --git a/internal/datanode/data_node.go b/internal/datanode/data_node.go index 73d7f8350d..923f2aa0c4 100644 --- a/internal/datanode/data_node.go +++ b/internal/datanode/data_node.go @@ -609,7 +609,6 @@ func (node *DataNode) FlushSegments(ctx context.Context, req *datapb.FlushSegmen log.Info("receiving FlushSegments request", zap.Int64("collection ID", req.GetCollectionID()), zap.Int64s("segments", req.GetSegmentIDs()), - zap.Int64s("stale segments", req.GetMarkSegmentIDs()), ) // TODO: Here and in other places, replace `flushed` param with a more meaningful name. @@ -658,15 +657,13 @@ func (node *DataNode) FlushSegments(ctx context.Context, req *datapb.FlushSegmen } seg, noErr1 := processSegments(req.GetSegmentIDs(), true) - staleSeg, noErr2 := processSegments(req.GetMarkSegmentIDs(), false) // Log success flushed segments. - if len(seg)+len(staleSeg) > 0 { + if len(seg) > 0 { log.Info("sending segments to flush channel", - zap.Any("newly sealed segment IDs", seg), - zap.Any("stale segment IDs", staleSeg)) + zap.Any("newly sealed segment IDs", seg)) } - // Fail FlushSegments call if at least one segment (no matter stale or not) fails to get flushed. - if !noErr1 || !noErr2 { + // Fail FlushSegments call if at least one segment fails to get flushed. + if !noErr1 { return errStatus, nil } diff --git a/internal/datanode/data_node_test.go b/internal/datanode/data_node_test.go index 950249712b..962322e15d 100644 --- a/internal/datanode/data_node_test.go +++ b/internal/datanode/data_node_test.go @@ -318,15 +318,14 @@ func TestDataNode(t *testing.T) { Base: &commonpb.MsgBase{ TargetID: node1.session.ServerID, }, - DbID: 0, - CollectionID: 1, - SegmentIDs: []int64{}, - MarkSegmentIDs: []int64{2}, + DbID: 0, + CollectionID: 1, + SegmentIDs: []int64{}, } status, err = node1.FlushSegments(node1.ctx, req) assert.NoError(t, err) - assert.Equal(t, commonpb.ErrorCode_UnexpectedError, status.ErrorCode) + assert.Equal(t, commonpb.ErrorCode_Success, status.ErrorCode) }) t.Run("Test GetTimeTickChannel", func(t *testing.T) { diff --git a/internal/datanode/data_sync_service.go b/internal/datanode/data_sync_service.go index a07aa38eb0..d847f719bd 100644 --- a/internal/datanode/data_sync_service.go +++ b/internal/datanode/data_sync_service.go @@ -301,10 +301,17 @@ func (dsService *dataSyncService) initNodes(vchanInfo *datapb.VchannelInfo) erro return err } + var ttNode Node + ttNode, err = newTTNode(c, dsService.dataCoord) + if err != nil { + return err + } + dsService.fg.AddNode(dmStreamNode) dsService.fg.AddNode(ddNode) dsService.fg.AddNode(insertBufferNode) dsService.fg.AddNode(deleteNode) + dsService.fg.AddNode(ttNode) // ddStreamNode err = dsService.fg.SetEdges(dmStreamNode.Name(), @@ -335,12 +342,21 @@ func (dsService *dataSyncService) initNodes(vchanInfo *datapb.VchannelInfo) erro //deleteNode err = dsService.fg.SetEdges(deleteNode.Name(), - []string{}, + []string{ttNode.Name()}, ) if err != nil { log.Error("set edges failed in node", zap.String("name", deleteNode.Name()), zap.Error(err)) return err } + + // ttNode + err = dsService.fg.SetEdges(ttNode.Name(), + []string{}, + ) + if err != nil { + log.Error("set edges failed in node", zap.String("name", ttNode.Name()), zap.Error(err)) + return err + } return nil } diff --git a/internal/datanode/flow_graph_delete_node.go b/internal/datanode/flow_graph_delete_node.go index ca99b23b0e..8b327b4859 100644 --- a/internal/datanode/flow_graph_delete_node.go +++ b/internal/datanode/flow_graph_delete_node.go @@ -20,16 +20,17 @@ import ( "context" "fmt" "reflect" - "sync" - - "github.com/milvus-io/milvus/internal/log" - "github.com/milvus-io/milvus/internal/mq/msgstream" - "github.com/milvus-io/milvus/internal/storage" - "github.com/milvus-io/milvus/internal/util/retry" - "github.com/milvus-io/milvus/internal/util/trace" "github.com/opentracing/opentracing-go" "go.uber.org/zap" + + "github.com/milvus-io/milvus/internal/log" + "github.com/milvus-io/milvus/internal/mq/msgstream" + "github.com/milvus-io/milvus/internal/proto/internalpb" + "github.com/milvus-io/milvus/internal/storage" + "github.com/milvus-io/milvus/internal/util/retry" + "github.com/milvus-io/milvus/internal/util/trace" + "github.com/milvus-io/milvus/internal/util/tsoutil" ) // DeleteNode is to process delete msg, flush delete info into storage. @@ -101,7 +102,7 @@ func (dn *deleteNode) Operate(in []Msg) []Msg { for i, msg := range fgMsg.deleteMessages { traceID, _, _ := trace.InfoFromSpan(spans[i]) log.Debug("Buffer delete request in DataNode", zap.String("traceID", traceID)) - tmpSegIDs, err := dn.bufferDeleteMsg(msg, fgMsg.timeRange) + tmpSegIDs, err := dn.bufferDeleteMsg(msg, fgMsg.timeRange, fgMsg.startPositions[0], fgMsg.endPositions[0]) if err != nil { // error occurs only when deleteMsg is misaligned, should not happen err = fmt.Errorf("buffer delete msg failed, err = %s", err) @@ -140,7 +141,8 @@ func (dn *deleteNode) Operate(in []Msg) []Msg { if len(segmentsToFlush) > 0 { log.Debug("DeleteNode receives flush message", zap.Int64s("segIDs", segmentsToFlush), - zap.String("vChannelName", dn.channelName)) + zap.String("vChannelName", dn.channelName), + zap.Time("posTime", tsoutil.PhysicalTime(fgMsg.endPositions[0].Timestamp))) for _, segmentToFlush := range segmentsToFlush { buf, ok := dn.delBufferManager.Load(segmentToFlush) if !ok { @@ -197,7 +199,7 @@ func (dn *deleteNode) updateCompactedSegments() { } } -func (dn *deleteNode) bufferDeleteMsg(msg *msgstream.DeleteMsg, tr TimeRange) ([]UniqueID, error) { +func (dn *deleteNode) bufferDeleteMsg(msg *msgstream.DeleteMsg, tr TimeRange, startPos, endPos *internalpb.MsgPosition) ([]UniqueID, error) { log.Debug("bufferDeleteMsg", zap.Any("primary keys", msg.PrimaryKeys), zap.String("vChannelName", dn.channelName)) primaryKeys := storage.ParseIDs2PrimaryKeys(msg.PrimaryKeys) @@ -211,7 +213,7 @@ func (dn *deleteNode) bufferDeleteMsg(msg *msgstream.DeleteMsg, tr TimeRange) ([ if !ok || len(pks) != len(tss) { return nil, fmt.Errorf("primary keys and timestamp's element num mis-match, segmentID = %d", segID) } - dn.delBufferManager.StoreNewDeletes(segID, pks, tss, tr) + dn.delBufferManager.StoreNewDeletes(segID, pks, tss, tr, startPos, endPos) } return segIDs, nil @@ -247,7 +249,7 @@ func newDeleteNode(ctx context.Context, fm flushManager, sig chan<- string, conf ctx: ctx, BaseNode: baseNode, delBufferManager: &DelBufferManager{ - delBufMap: sync.Map{}, + channel: config.channel, delMemorySize: 0, delBufHeap: &PriorityQueue{}, }, diff --git a/internal/datanode/flow_graph_delete_node_test.go b/internal/datanode/flow_graph_delete_node_test.go index aaa08783a2..87fdfb3192 100644 --- a/internal/datanode/flow_graph_delete_node_test.go +++ b/internal/datanode/flow_graph_delete_node_test.go @@ -326,40 +326,6 @@ func TestFlowGraphDeleteNode_Operate(t *testing.T) { } }) - t.Run("Test deleteNode Operate flushDelData failed", func(te *testing.T) { - ctx, cancel := context.WithTimeout(context.Background(), 2*time.Second) - defer cancel() - - chanName := "datanode-test-FlowGraphDeletenode-operate" - testPath := "/test/datanode/root/meta" - assert.NoError(t, clearEtcd(testPath)) - Params.EtcdCfg.MetaRootPath = testPath - - c := &nodeConfig{ - channel: nil, - allocator: NewAllocatorFactory(), - vChannelName: chanName, - } - delNode, err := newDeleteNode(ctx, fm, make(chan string, 1), c) - assert.Nil(te, err) - - msg := genFlowGraphDeleteMsg(int64Pks, chanName) - msg.segmentsToSync = []UniqueID{-1} - delDataBuf := newDelDataBuf() - delNode.delBufferManager.Store(UniqueID(-1), delDataBuf) - heap.Push(delNode.delBufferManager.delBufHeap, delDataBuf.item) - delNode.flushManager = &mockFlushManager{ - returnError: true, - } - - var fgMsg flowgraph.Msg = &msg - - setFlowGraphRetryOpt(retry.Attempts(1)) - assert.Panics(te, func() { - delNode.Operate([]flowgraph.Msg{fgMsg}) - }) - }) - t.Run("Test issue#18565", func(t *testing.T) { ctx, cancel := context.WithTimeout(context.Background(), 2*time.Second) defer cancel() @@ -510,8 +476,11 @@ func TestFlowGraphDeleteNode_showDelBuf(t *testing.T) { assert.NoError(t, clearEtcd(testPath)) Params.EtcdCfg.MetaRootPath = testPath + channel := &ChannelMeta{ + segments: make(map[UniqueID]*Segment), + } c := &nodeConfig{ - channel: nil, + channel: channel, allocator: NewAllocatorFactory(), vChannelName: chanName, } @@ -566,7 +535,6 @@ func TestFlowGraphDeleteNode_updateCompactedSegments(t *testing.T) { tests := []struct { description string compactToExist bool - segIDsInBuffer []UniqueID compactedToIDs []UniqueID compactedFromIDs []UniqueID @@ -574,35 +542,28 @@ func TestFlowGraphDeleteNode_updateCompactedSegments(t *testing.T) { expectedSegsRemain []UniqueID }{ {"zero segments", false, - []UniqueID{}, []UniqueID{}, []UniqueID{}, []UniqueID{}}, + []UniqueID{}, []UniqueID{}, []UniqueID{}}, {"segment no compaction", false, - []UniqueID{100, 101}, []UniqueID{}, []UniqueID{}, []UniqueID{100, 101}}, - {"segment compacted not in buffer", true, - []UniqueID{100, 101}, []UniqueID{200}, []UniqueID{103}, []UniqueID{100, 101}}, - {"segment compacted in buffer 100>201", true, - []UniqueID{100, 101}, []UniqueID{201}, []UniqueID{100}, []UniqueID{101, 201}}, - {"segment compacted in buffer 100+101>201", true, - []UniqueID{100, 101}, []UniqueID{201, 201}, []UniqueID{100, 101}, []UniqueID{201}}, - {"segment compacted in buffer 100>201, 101>202", true, - []UniqueID{100, 101}, []UniqueID{201, 202}, []UniqueID{100, 101}, []UniqueID{201, 202}}, + []UniqueID{}, []UniqueID{}, []UniqueID{100, 101}}, + {"segment compacted", true, + []UniqueID{200}, []UniqueID{103}, []UniqueID{100, 101}}, + {"segment compacted 100>201", true, + []UniqueID{201}, []UniqueID{100}, []UniqueID{101, 201}}, + {"segment compacted 100+101>201", true, + []UniqueID{201, 201}, []UniqueID{100, 101}, []UniqueID{201}}, + {"segment compacted 100>201, 101>202", true, + []UniqueID{201, 202}, []UniqueID{100, 101}, []UniqueID{201, 202}}, // false - {"segment compacted in buffer 100>201", false, - []UniqueID{100, 101}, []UniqueID{201}, []UniqueID{100}, []UniqueID{101}}, - {"segment compacted in buffer 100+101>201", false, - []UniqueID{100, 101}, []UniqueID{201, 201}, []UniqueID{100, 101}, []UniqueID{}}, - {"segment compacted in buffer 100>201, 101>202", false, - []UniqueID{100, 101}, []UniqueID{201, 202}, []UniqueID{100, 101}, []UniqueID{}}, + {"segment compacted 100>201", false, + []UniqueID{201}, []UniqueID{100}, []UniqueID{101}}, + {"segment compacted 100+101>201", false, + []UniqueID{201, 201}, []UniqueID{100, 101}, []UniqueID{}}, + {"segment compacted 100>201, 101>202", false, + []UniqueID{201, 202}, []UniqueID{100, 101}, []UniqueID{}}, } for _, test := range tests { t.Run(test.description, func(t *testing.T) { - for _, seg := range test.segIDsInBuffer { - delBuf := newDelDataBuf() - delBuf.accumulateEntriesNum(100) - heap.Push(delNode.delBufferManager.delBufHeap, delBuf.item) - delNode.delBufferManager.Store(seg, delBuf) - } - if test.compactToExist { for _, segID := range test.compactedToIDs { seg := Segment{ @@ -628,8 +589,7 @@ func TestFlowGraphDeleteNode_updateCompactedSegments(t *testing.T) { delNode.updateCompactedSegments() for _, remain := range test.expectedSegsRemain { - _, ok := delNode.delBufferManager.Load(remain) - assert.True(t, ok) + delNode.channel.hasSegment(remain, true) } }) } diff --git a/internal/datanode/flow_graph_insert_buffer_node.go b/internal/datanode/flow_graph_insert_buffer_node.go index fb39f371f1..04546ac719 100644 --- a/internal/datanode/flow_graph_insert_buffer_node.go +++ b/internal/datanode/flow_graph_insert_buffer_node.go @@ -29,7 +29,6 @@ import ( "go.uber.org/zap" "github.com/milvus-io/milvus-proto/go-api/commonpb" - "github.com/milvus-io/milvus-proto/go-api/schemapb" "github.com/milvus-io/milvus/internal/log" "github.com/milvus-io/milvus/internal/metrics" "github.com/milvus-io/milvus/internal/mq/msgstream" @@ -46,11 +45,10 @@ import ( type insertBufferNode struct { BaseNode - ctx context.Context - channelName string - insertBuffer sync.Map // SegmentID to BufferData - channel Channel - idAllocator allocatorInterface + ctx context.Context + channelName string + channel Channel + idAllocator allocatorInterface flushMap sync.Map flushChan <-chan flushMsg @@ -62,6 +60,7 @@ type insertBufferNode struct { ttLogger *timeTickLogger ttMerger *mergedTimeTickerSender + syncPolicies []segmentSyncPolicy lastTimestamp Timestamp } @@ -126,12 +125,14 @@ func (ibNode *insertBufferNode) Operate(in []Msg) []Msg { pos.ChannelName = ibNode.channelName startPositions = append(startPositions, pos) } + fgMsg.startPositions = startPositions endPositions := make([]*internalpb.MsgPosition, 0, len(fgMsg.endPositions)) for idx := range fgMsg.endPositions { pos := proto.Clone(fgMsg.endPositions[idx]).(*internalpb.MsgPosition) pos.ChannelName = ibNode.channelName endPositions = append(endPositions, pos) } + fgMsg.endPositions = endPositions if startPositions[0].Timestamp < ibNode.lastTimestamp { // message stream should guarantee that this should not happen @@ -154,7 +155,7 @@ func (ibNode *insertBufferNode) Operate(in []Msg) []Msg { // insert messages -> buffer for _, msg := range fgMsg.insertMessages { - err := ibNode.bufferInsertMsg(msg, endPositions[0]) + err := ibNode.bufferInsertMsg(msg, startPositions[0], endPositions[0]) if err != nil { // error occurs when missing schema info or data is misaligned, should not happen err = fmt.Errorf("insertBufferNode msg to buffer failed, err = %s", err) @@ -189,7 +190,7 @@ func (ibNode *insertBufferNode) Operate(in []Msg) []Msg { func (ibNode *insertBufferNode) verifyInMsg(in []Msg) (*flowGraphMsg, bool) { // while closing if in == nil { - log.Debug("type assertion failed for flowGraphMsg because it's nil") + log.Warn("type assertion failed for flowGraphMsg because it's nil") return nil, false } @@ -206,8 +207,8 @@ func (ibNode *insertBufferNode) verifyInMsg(in []Msg) (*flowGraphMsg, bool) { } func (ibNode *insertBufferNode) GetBufferIfFull(segID UniqueID) (*BufferData, bool) { - if bd, ok := ibNode.insertBuffer.Load(segID); ok && bd.(*BufferData).effectiveCap() <= 0 { - return bd.(*BufferData), true + if bd, ok := ibNode.channel.getCurInsertBuffer(segID); ok && bd.effectiveCap() <= 0 { + return bd, true } return nil, false @@ -216,8 +217,8 @@ func (ibNode *insertBufferNode) GetBufferIfFull(segID UniqueID) (*BufferData, bo // GetBuffer returns buffer data for a segment, returns nil if segment's not in buffer func (ibNode *insertBufferNode) GetBuffer(segID UniqueID) *BufferData { var buf *BufferData - if bd, ok := ibNode.insertBuffer.Load(segID); ok { - buf = bd.(*BufferData) + if bd, ok := ibNode.channel.getCurInsertBuffer(segID); ok { + buf = bd } return buf } @@ -271,13 +272,13 @@ func (ibNode *insertBufferNode) DisplayStatistics(seg2Upload []UniqueID) { displaySize := min(10, len(seg2Upload)) for k, segID := range seg2Upload[:displaySize] { - if bd, ok := ibNode.insertBuffer.Load(segID); ok { + if bd, ok := ibNode.channel.getCurInsertBuffer(segID); ok { log.Info("segment buffer status", zap.Int("no.", k), zap.Int64("segmentID", segID), zap.String("channel", ibNode.channelName), - zap.Int64("size", bd.(*BufferData).size), - zap.Int64("limit", bd.(*BufferData).limit)) + zap.Int64("size", bd.size), + zap.Int64("limit", bd.limit)) } } } @@ -314,7 +315,7 @@ func (ibNode *insertBufferNode) FillInSyncTasks(fgMsg *flowGraphMsg, seg2Upload return syncTasks } - // Auto Sync + // Auto Sync // TODO: move to segment_sync_policy for _, segID := range seg2Upload { if ibuffer, ok := ibNode.GetBufferIfFull(segID); ok { log.Info("(Auto Sync)", @@ -333,6 +334,19 @@ func (ibNode *insertBufferNode) FillInSyncTasks(fgMsg *flowGraphMsg, seg2Upload } } + syncSegmentIDs := ibNode.channel.listSegmentIDsToSync(fgMsg.endPositions[0].Timestamp) + for _, segID := range syncSegmentIDs { + buf := ibNode.GetBuffer(segID) + syncTasks[segID] = &syncTask{ + buffer: buf, // nil is valid + segmentID: segID, + } + } + if len(syncSegmentIDs) > 0 { + log.Debug("sync segments", zap.String("vChannel", ibNode.channelName), + zap.Int64s("segIDs", syncSegmentIDs)) // TODO: maybe too many prints here + } + mergeSyncTask := func(segmentIDs []UniqueID, syncTasks map[UniqueID]*syncTask, setupTask func(task *syncTask)) { // Merge auto & manual sync tasks with the same segment ID. for _, segmentID := range segmentIDs { @@ -361,6 +375,7 @@ func (ibNode *insertBufferNode) FillInSyncTasks(fgMsg *flowGraphMsg, seg2Upload mergeSyncTask(flushedSegments, syncTasks, func(task *syncTask) { task.flushed = true }) + mergeSyncTask(syncSegmentIDs, syncTasks, func(task *syncTask) {}) // process drop partition for _, partitionDrop := range fgMsg.dropPartitions { @@ -422,7 +437,7 @@ func (ibNode *insertBufferNode) Sync(fgMsg *flowGraphMsg, seg2Upload []UniqueID, panic(err) } segmentsToSync = append(segmentsToSync, task.segmentID) - ibNode.insertBuffer.Delete(task.segmentID) + ibNode.channel.rollInsertBuffer(task.segmentID) ibNode.channel.RollPKstats(task.segmentID, pkStats) metrics.DataNodeFlushBufferCount.WithLabelValues(fmt.Sprint(Params.DataNodeCfg.GetNodeID()), metrics.SuccessLabel).Inc() metrics.DataNodeFlushBufferCount.WithLabelValues(fmt.Sprint(Params.DataNodeCfg.GetNodeID()), metrics.TotalLabel).Inc() @@ -457,7 +472,7 @@ func (ibNode *insertBufferNode) updateSegmentStates(insertMsgs []*msgstream.Inse endPos: endPos, }) if err != nil { - log.Error("add segment wrong", + log.Warn("add segment wrong", zap.Int64("segID", currentSegID), zap.Int64("collID", collID), zap.Int64("partID", partitionID), @@ -486,7 +501,7 @@ func (ibNode *insertBufferNode) updateSegmentStates(insertMsgs []*msgstream.Inse // 1.2 Get buffer data and put data into each field buffer // 1.3 Put back into buffer // 1.4 Update related statistics -func (ibNode *insertBufferNode) bufferInsertMsg(msg *msgstream.InsertMsg, endPos *internalpb.MsgPosition) error { +func (ibNode *insertBufferNode) bufferInsertMsg(msg *msgstream.InsertMsg, startPos, endPos *internalpb.MsgPosition) error { if err := msg.CheckAligned(); err != nil { return err } @@ -495,38 +510,24 @@ func (ibNode *insertBufferNode) bufferInsertMsg(msg *msgstream.InsertMsg, endPos collSchema, err := ibNode.channel.getCollectionSchema(collectionID, msg.EndTs()) if err != nil { - log.Error("Get schema wrong:", zap.Error(err)) + log.Warn("Get schema wrong:", zap.Error(err)) return err } - // Get Dimension - // TODO GOOSE: under assumption that there's only 1 Vector field in one collection schema - var dimension int - for _, field := range collSchema.Fields { - if field.DataType == schemapb.DataType_FloatVector || - field.DataType == schemapb.DataType_BinaryVector { - - dimension, err = storage.GetDimFromParams(field.TypeParams) - if err != nil { - log.Error("failed to get dim from field", zap.Error(err)) - return err - } - break + // load or store insertBuffer + var buffer *BufferData + var loaded bool + buffer, loaded = ibNode.channel.getCurInsertBuffer(currentSegID) + if !loaded { + buffer, err = newBufferData(collSchema) + if err != nil { + return fmt.Errorf("newBufferData failed, segment=%d, channel=%s, err=%s", currentSegID, ibNode.channelName, err) } } - newbd, err := newBufferData(int64(dimension)) - if err != nil { - return err - } - bd, _ := ibNode.insertBuffer.LoadOrStore(currentSegID, newbd) - - buffer := bd.(*BufferData) - // idata := buffer.buffer - addedBuffer, err := storage.InsertMsgToInsertData(msg, collSchema) if err != nil { - log.Error("failed to transfer insert msg to insert data", zap.Error(err)) + log.Warn("failed to transfer insert msg to insert data", zap.Error(err)) return err } @@ -548,16 +549,14 @@ func (ibNode *insertBufferNode) bufferInsertMsg(msg *msgstream.InsertMsg, endPos // update buffer size buffer.updateSize(int64(msg.NRows())) - // update timestamp range + // update timestamp range and start-end position buffer.updateTimeRange(ibNode.getTimestampRange(tsData)) + buffer.updateStartAndEndPosition(startPos, endPos) metrics.DataNodeConsumeMsgRowsCount.WithLabelValues(fmt.Sprint(Params.DataNodeCfg.GetNodeID()), metrics.InsertLabel).Add(float64(len(msg.RowData))) // store in buffer - ibNode.insertBuffer.Store(currentSegID, buffer) - - // store current endPositions as Segment->EndPostion - ibNode.channel.updateSegmentEndPosition(currentSegID, endPos) + ibNode.channel.setCurInsertBuffer(currentSegID, buffer) return nil } @@ -654,9 +653,8 @@ func newInsertBufferNode(ctx context.Context, collID UniqueID, flushCh <-chan fl }) return &insertBufferNode{ - ctx: ctx, - BaseNode: baseNode, - insertBuffer: sync.Map{}, + ctx: ctx, + BaseNode: baseNode, timeTickStream: wTtMsgStream, flushMap: sync.Map{}, diff --git a/internal/datanode/flow_graph_insert_buffer_node_test.go b/internal/datanode/flow_graph_insert_buffer_node_test.go index 6d07383d2d..34f6cc489b 100644 --- a/internal/datanode/flow_graph_insert_buffer_node_test.go +++ b/internal/datanode/flow_graph_insert_buffer_node_test.go @@ -249,7 +249,7 @@ func TestFlowGraphInsertBufferNode_Operate(t *testing.T) { setFlowGraphRetryOpt(retry.Attempts(1)) inMsg = genFlowGraphInsertMsg(insertChannelName) iBNode.flushManager = &mockFlushManager{returnError: true} - iBNode.insertBuffer.Store(inMsg.insertMessages[0].SegmentID, &BufferData{}) + iBNode.channel.setCurInsertBuffer(inMsg.insertMessages[0].SegmentID, &BufferData{}) assert.Panics(t, func() { iBNode.Operate([]flowgraph.Msg{&inMsg}) }) iBNode.flushManager = fm } @@ -637,7 +637,6 @@ func TestRollBF(t *testing.T) { assert.Equal(t, datapb.SegmentType_New, seg.getType()) assert.Equal(t, int64(1), seg.numRows) assert.Equal(t, uint64(100), seg.startPos.GetTimestamp()) - assert.Equal(t, uint64(123), seg.endPos.GetTimestamp()) // because this is the origincal assert.True(t, seg.currentStat.PkFilter.Cap() > uint(1000000)) @@ -666,7 +665,6 @@ func TestRollBF(t *testing.T) { assert.Equal(t, datapb.SegmentType_Normal, seg.getType()) assert.Equal(t, int64(2), seg.numRows) assert.Equal(t, uint64(100), seg.startPos.GetTimestamp()) - assert.Equal(t, uint64(234), seg.endPos.GetTimestamp()) // filter should be rolled assert.Nil(t, seg.currentStat) @@ -769,9 +767,9 @@ func (s *InsertBufferNodeSuit) TestFillInSyncTasks() { size: 2, limit: 2, } - node.insertBuffer.Store(UniqueID(1), &buffer) + node.channel.setCurInsertBuffer(UniqueID(1), &buffer) - syncTasks := node.FillInSyncTasks(new(flowGraphMsg), segToFlush) + syncTasks := node.FillInSyncTasks(&flowGraphMsg{endPositions: []*internalpb.MsgPosition{{Timestamp: 100}}}, segToFlush) s.Assert().NotEmpty(syncTasks) s.Assert().Equal(1, len(syncTasks)) @@ -784,7 +782,7 @@ func (s *InsertBufferNodeSuit) TestFillInSyncTasks() { }) s.Run("drop partition", func() { - fgMsg := flowGraphMsg{dropPartitions: []UniqueID{s.partID}} + fgMsg := flowGraphMsg{dropPartitions: []UniqueID{s.partID}, endPositions: []*internalpb.MsgPosition{{Timestamp: 100}}} node := &insertBufferNode{ channelName: s.channel.channelName, channel: s.channel, @@ -820,7 +818,7 @@ func (s *InsertBufferNodeSuit) TestFillInSyncTasks() { flushCh <- msg } - syncTasks := node.FillInSyncTasks(new(flowGraphMsg), nil) + syncTasks := node.FillInSyncTasks(&flowGraphMsg{endPositions: []*internalpb.MsgPosition{{Timestamp: 100}}}, nil) s.Assert().NotEmpty(syncTasks) for segID, task := range syncTasks { @@ -857,7 +855,7 @@ func (s *InsertBufferNodeSuit) TestFillInSyncTasks() { flushCh <- msg } - syncTasks := node.FillInSyncTasks(new(flowGraphMsg), nil) + syncTasks := node.FillInSyncTasks(&flowGraphMsg{endPositions: []*internalpb.MsgPosition{{Timestamp: 100}}}, nil) s.Assert().NotEmpty(syncTasks) s.Assert().Equal(10, len(syncTasks)) // 10 is max batch @@ -960,14 +958,14 @@ func TestInsertBufferNode_bufferInsertMsg(t *testing.T) { inMsg := genFlowGraphInsertMsg(insertChannelName) for _, msg := range inMsg.insertMessages { msg.EndTimestamp = 101 // ts valid - err = iBNode.bufferInsertMsg(msg, &internalpb.MsgPosition{}) + err = iBNode.bufferInsertMsg(msg, &internalpb.MsgPosition{}, &internalpb.MsgPosition{}) assert.Nil(t, err) } for _, msg := range inMsg.insertMessages { msg.EndTimestamp = 101 // ts valid msg.RowIDs = []int64{} //misaligned data - err = iBNode.bufferInsertMsg(msg, &internalpb.MsgPosition{}) + err = iBNode.bufferInsertMsg(msg, &internalpb.MsgPosition{}, &internalpb.MsgPosition{}) assert.NotNil(t, err) } } diff --git a/internal/datanode/flow_graph_time_tick_node.go b/internal/datanode/flow_graph_time_tick_node.go new file mode 100644 index 0000000000..8c31e8b668 --- /dev/null +++ b/internal/datanode/flow_graph_time_tick_node.go @@ -0,0 +1,125 @@ +// Licensed to the LF AI & Data foundation under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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 datanode + +import ( + "context" + "fmt" + "reflect" + "time" + + "go.uber.org/zap" + + "github.com/milvus-io/milvus/internal/log" + "github.com/milvus-io/milvus/internal/proto/datapb" + "github.com/milvus-io/milvus/internal/proto/internalpb" + "github.com/milvus-io/milvus/internal/types" + "github.com/milvus-io/milvus/internal/util/commonpbutil" + "github.com/milvus-io/milvus/internal/util/flowgraph" + "github.com/milvus-io/milvus/internal/util/funcutil" + "github.com/milvus-io/milvus/internal/util/tsoutil" +) + +const ( + updateChanCPInterval = 1 * time.Minute + updateChanCPTimeout = 10 * time.Second +) + +// make sure ttNode implements flowgraph.Node +var _ flowgraph.Node = (*ttNode)(nil) + +type ttNode struct { + BaseNode + vChannelName string + channel Channel + lastUpdateTime time.Time + dataCoord types.DataCoord +} + +// Name returns node name, implementing flowgraph.Node +func (ttn *ttNode) Name() string { + return fmt.Sprintf("ttNode-%s", ttn.vChannelName) +} + +// Operate handles input messages, implementing flowgraph.Node +func (ttn *ttNode) Operate(in []Msg) []Msg { + if in == nil { + log.Debug("type assertion failed for flowGraphMsg because it's nil") + return []Msg{} + } + + if len(in) != 1 { + log.Warn("Invalid operate message input in ttNode", zap.Int("input length", len(in))) + return []Msg{} + } + + fgMsg, ok := in[0].(*flowGraphMsg) + if !ok { + log.Warn("type assertion failed for flowGraphMsg", zap.String("name", reflect.TypeOf(in[0]).Name())) + return []Msg{} + } + + curTs, _ := tsoutil.ParseTS(fgMsg.timeRange.timestampMax) + if curTs.Sub(ttn.lastUpdateTime) >= updateChanCPInterval { + ttn.updateChannelCP(fgMsg.endPositions[0]) + ttn.lastUpdateTime = curTs + } + + return []Msg{} +} + +func (ttn *ttNode) updateChannelCP(ttPos *internalpb.MsgPosition) { + channelPos := ttn.channel.getChannelCheckpoint(ttPos) + if channelPos == nil || channelPos.MsgID == nil { + log.Warn("updateChannelCP failed, get nil check point", zap.String("vChannel", ttn.vChannelName)) + return + } + channelCPTs, _ := tsoutil.ParseTS(channelPos.Timestamp) + + ctx, cancel := context.WithTimeout(context.Background(), updateChanCPTimeout) + defer cancel() + resp, err := ttn.dataCoord.UpdateChannelCheckpoint(ctx, &datapb.UpdateChannelCheckpointRequest{ + Base: commonpbutil.NewMsgBase( + commonpbutil.WithSourceID(Params.DataNodeCfg.GetNodeID()), + ), + VChannel: ttn.vChannelName, + Position: channelPos, + }) + if err = funcutil.VerifyResponse(resp, err); err != nil { + log.Warn("UpdateChannelCheckpoint failed", zap.String("channel", ttn.vChannelName), + zap.Time("channelCPTs", channelCPTs), zap.Error(err)) + return + } + + log.Info("UpdateChannelCheckpoint success", zap.String("channel", ttn.vChannelName), zap.Time("channelCPTs", channelCPTs)) +} + +func newTTNode(config *nodeConfig, dc types.DataCoord) (*ttNode, error) { + baseNode := BaseNode{} + baseNode.SetMaxQueueLength(Params.DataNodeCfg.FlowGraphMaxQueueLength) + baseNode.SetMaxParallelism(Params.DataNodeCfg.FlowGraphMaxParallelism) + + tt := &ttNode{ + BaseNode: baseNode, + vChannelName: config.vChannelName, + channel: config.channel, + lastUpdateTime: time.Time{}, // set to Zero to update channel checkpoint immediately after fg started + dataCoord: dc, + } + + return tt, nil +} diff --git a/internal/datanode/flush_manager.go b/internal/datanode/flush_manager.go index dd139c5631..db894401e9 100644 --- a/internal/datanode/flush_manager.go +++ b/internal/datanode/flush_manager.go @@ -851,5 +851,8 @@ func flushNotifyFunc(dsService *dataSyncService, opts ...retry.Option) notifyMet dsService.channel.segmentFlushed(pack.segmentID) } dsService.flushingSegCache.Remove(req.GetSegmentID()) + dsService.channel.evictHistoryInsertBuffer(req.GetSegmentID(), pack.pos) + dsService.channel.evictHistoryDeleteBuffer(req.GetSegmentID(), pack.pos) + dsService.channel.setSegmentLastSyncTs(req.GetSegmentID(), pack.pos.GetTimestamp()) } } diff --git a/internal/datanode/flush_task.go b/internal/datanode/flush_task.go index 7c2fecf377..9ded3cfa96 100644 --- a/internal/datanode/flush_task.go +++ b/internal/datanode/flush_task.go @@ -21,12 +21,14 @@ import ( "errors" "sync" + "go.uber.org/zap" + "github.com/milvus-io/milvus/internal/kv" "github.com/milvus-io/milvus/internal/log" "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/internal/proto/internalpb" "github.com/milvus-io/milvus/internal/util/retry" - "go.uber.org/zap" + "github.com/milvus-io/milvus/internal/util/tsoutil" ) // errStart used for retry start @@ -134,6 +136,7 @@ func (t *flushTaskRunner) runFlushInsert(task flushInsertTask, zap.Bool("flushed", flushed), zap.Bool("dropped", dropped), zap.Any("position", pos), + zap.Time("PosTime", tsoutil.PhysicalTime(pos.GetTimestamp())), ) go func() { err := retry.Do(context.Background(), func() error { diff --git a/internal/datanode/mock_test.go b/internal/datanode/mock_test.go index 573d88590d..7c4bd952e9 100644 --- a/internal/datanode/mock_test.go +++ b/internal/datanode/mock_test.go @@ -253,6 +253,12 @@ func (ds *DataCoordFactory) UpdateSegmentStatistics(ctx context.Context, req *da }, nil } +func (ds *DataCoordFactory) UpdateChannelCheckpoint(ctx context.Context, req *datapb.UpdateChannelCheckpointRequest) (*commonpb.Status, error) { + return &commonpb.Status{ + ErrorCode: commonpb.ErrorCode_Success, + }, nil +} + func (ds *DataCoordFactory) SaveImportSegment(ctx context.Context, req *datapb.SaveImportSegmentRequest) (*commonpb.Status, error) { return &commonpb.Status{ ErrorCode: commonpb.ErrorCode_Success, diff --git a/internal/datanode/segment.go b/internal/datanode/segment.go index 56a895dd27..8496fa13e6 100644 --- a/internal/datanode/segment.go +++ b/internal/datanode/segment.go @@ -21,9 +21,13 @@ import ( "sync/atomic" "github.com/bits-and-blooms/bloom/v3" + "go.uber.org/zap" + + "github.com/milvus-io/milvus/internal/log" "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/internal/proto/internalpb" "github.com/milvus-io/milvus/internal/storage" + "github.com/milvus-io/milvus/internal/util/tsoutil" ) // Segment contains the latest segment infos from channel. @@ -37,12 +41,17 @@ type Segment struct { memorySize int64 compactedTo UniqueID - statLock sync.Mutex + curInsertBuf *BufferData + curDeleteBuf *DelDataBuf + historyInsertBuf []*BufferData + historyDeleteBuf []*DelDataBuf + + statLock sync.RWMutex currentStat *storage.PkStatistics historyStats []*storage.PkStatistics - startPos *internalpb.MsgPosition // TODO readonly - endPos *internalpb.MsgPosition + lastSyncTs Timestamp + startPos *internalpb.MsgPosition // TODO readonly } type addSegmentReq struct { @@ -104,3 +113,56 @@ func (s *Segment) isPKExist(pk primaryKey) bool { } return false } + +// rollInsertBuffer moves curInsertBuf to historyInsertBuf, and then sets curInsertBuf to nil. +func (s *Segment) rollInsertBuffer() { + if s.curInsertBuf == nil { + return + } + s.curInsertBuf.buffer = nil // free buffer memory, only keep meta infos in historyInsertBuf + s.historyInsertBuf = append(s.historyInsertBuf, s.curInsertBuf) + s.curInsertBuf = nil +} + +// evictHistoryInsertBuffer removes flushed buffer from historyInsertBuf after saveBinlogPath. +func (s *Segment) evictHistoryInsertBuffer(endPos *internalpb.MsgPosition) { + tmpBuffers := make([]*BufferData, 0) + for _, buf := range s.historyInsertBuf { + if buf.endPos.Timestamp > endPos.Timestamp { + tmpBuffers = append(tmpBuffers, buf) + } + } + s.historyInsertBuf = tmpBuffers + ts, _ := tsoutil.ParseTS(endPos.Timestamp) + log.Debug("evictHistoryInsertBuffer done", zap.Int64("segmentID", s.segmentID), zap.Time("ts", ts), zap.String("channel", endPos.ChannelName)) +} + +// rollDeleteBuffer moves curDeleteBuf to historyDeleteBuf, and then sets curDeleteBuf to nil. +func (s *Segment) rollDeleteBuffer() { + if s.curDeleteBuf == nil { + return + } + s.curDeleteBuf.delData = nil // free buffer memory, only keep meta infos in historyDeleteBuf + s.historyDeleteBuf = append(s.historyDeleteBuf, s.curDeleteBuf) + s.curDeleteBuf = nil +} + +// evictHistoryDeleteBuffer removes flushed buffer from historyDeleteBuf after saveBinlogPath. +func (s *Segment) evictHistoryDeleteBuffer(endPos *internalpb.MsgPosition) { + tmpBuffers := make([]*DelDataBuf, 0) + for _, buf := range s.historyDeleteBuf { + if buf.endPos.Timestamp > endPos.Timestamp { + tmpBuffers = append(tmpBuffers, buf) + } + } + s.historyDeleteBuf = tmpBuffers + ts, _ := tsoutil.ParseTS(endPos.Timestamp) + log.Debug("evictHistoryDeleteBuffer done", zap.Int64("segmentID", s.segmentID), zap.Time("ts", ts), zap.String("channel", endPos.ChannelName)) +} + +func (s *Segment) isBufferEmpty() bool { + return s.curInsertBuf == nil && + s.curDeleteBuf == nil && + len(s.historyInsertBuf) == 0 && + len(s.historyDeleteBuf) == 0 +} diff --git a/internal/datanode/segment_sync_policy.go b/internal/datanode/segment_sync_policy.go new file mode 100644 index 0000000000..204fe1a851 --- /dev/null +++ b/internal/datanode/segment_sync_policy.go @@ -0,0 +1,39 @@ +// Licensed to the LF AI & Data foundation under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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 datanode + +import ( + "time" + + "github.com/milvus-io/milvus/internal/util/tsoutil" +) + +const ( + syncPeriod = 10 * time.Minute // TODO: move to config? +) + +// segmentSyncPolicy sync policy applies to segment +type segmentSyncPolicy func(segment *Segment, ts Timestamp) bool + +// syncPeriodically get segmentSyncPolicy with segment sync periodically. +func syncPeriodically() segmentSyncPolicy { + return func(segment *Segment, ts Timestamp) bool { + endTime := tsoutil.PhysicalTime(ts) + lastSyncTime := tsoutil.PhysicalTime(segment.lastSyncTs) + return endTime.Sub(lastSyncTime) >= syncPeriod && !segment.isBufferEmpty() + } +} diff --git a/internal/datanode/segment_sync_policy_test.go b/internal/datanode/segment_sync_policy_test.go new file mode 100644 index 0000000000..d5ede55150 --- /dev/null +++ b/internal/datanode/segment_sync_policy_test.go @@ -0,0 +1,56 @@ +// Licensed to the LF AI & Data foundation under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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 datanode + +import ( + "testing" + "time" + + "github.com/stretchr/testify/assert" + + "github.com/milvus-io/milvus/internal/util/tsoutil" +) + +func TestSyncPeriodically(t *testing.T) { + t0 := time.Now() + + tests := []struct { + testName string + lastTs time.Time + ts time.Time + isBufferEmpty bool + shouldSync bool + }{ + {"test buffer empty and stale", t0, t0.Add(syncPeriod), true, false}, + {"test buffer empty and not stale", t0, t0.Add(syncPeriod / 2), true, false}, + {"test buffer not empty and stale", t0, t0.Add(syncPeriod), false, true}, + {"test buffer not empty and not stale", t0, t0.Add(syncPeriod / 2), false, false}, + } + + for _, test := range tests { + t.Run(test.testName, func(t *testing.T) { + policy := syncPeriodically() + segment := &Segment{} + segment.lastSyncTs = tsoutil.ComposeTSByTime(test.lastTs, 0) + if !test.isBufferEmpty { + segment.curInsertBuf = &BufferData{} + } + res := policy(segment, tsoutil.ComposeTSByTime(test.ts, 0)) + assert.Equal(t, test.shouldSync, res) + }) + } +} diff --git a/internal/distributed/datacoord/client/client.go b/internal/distributed/datacoord/client/client.go index 6f87909804..52e23fb1b1 100644 --- a/internal/distributed/datacoord/client/client.go +++ b/internal/distributed/datacoord/client/client.go @@ -657,6 +657,25 @@ func (c *Client) UpdateSegmentStatistics(ctx context.Context, req *datapb.Update return ret.(*commonpb.Status), err } +// UpdateChannelCheckpoint updates channel checkpoint in dataCoord. +func (c *Client) UpdateChannelCheckpoint(ctx context.Context, req *datapb.UpdateChannelCheckpointRequest) (*commonpb.Status, error) { + req = typeutil.Clone(req) + commonpbutil.UpdateMsgBase( + req.GetBase(), + commonpbutil.FillMsgBaseFromClient(Params.DataCoordCfg.GetNodeID(), commonpbutil.WithTargetID(c.sess.ServerID)), + ) + ret, err := c.grpcClient.ReCall(ctx, func(client datapb.DataCoordClient) (any, error) { + if !funcutil.CheckCtxValid(ctx) { + return nil, ctx.Err() + } + return client.UpdateChannelCheckpoint(ctx, req) + }) + if err != nil || ret == nil { + return nil, err + } + return ret.(*commonpb.Status), err +} + // AcquireSegmentLock acquire the reference lock of the segments. func (c *Client) AcquireSegmentLock(ctx context.Context, req *datapb.AcquireSegmentLockRequest) (*commonpb.Status, error) { req = typeutil.Clone(req) diff --git a/internal/distributed/datacoord/service.go b/internal/distributed/datacoord/service.go index 7d020b99fe..29f2d088f8 100644 --- a/internal/distributed/datacoord/service.go +++ b/internal/distributed/datacoord/service.go @@ -372,6 +372,11 @@ func (s *Server) UpdateSegmentStatistics(ctx context.Context, req *datapb.Update return s.dataCoord.UpdateSegmentStatistics(ctx, req) } +// UpdateChannelCheckpoint updates channel checkpoint in dataCoord. +func (s *Server) UpdateChannelCheckpoint(ctx context.Context, req *datapb.UpdateChannelCheckpointRequest) (*commonpb.Status, error) { + return s.dataCoord.UpdateChannelCheckpoint(ctx, req) +} + // AcquireSegmentLock acquire the reference lock of the segments. func (s *Server) AcquireSegmentLock(ctx context.Context, req *datapb.AcquireSegmentLockRequest) (*commonpb.Status, error) { return s.dataCoord.AcquireSegmentLock(ctx, req) diff --git a/internal/distributed/datacoord/service_test.go b/internal/distributed/datacoord/service_test.go index b1800c26f2..be88b66fe4 100644 --- a/internal/distributed/datacoord/service_test.go +++ b/internal/distributed/datacoord/service_test.go @@ -62,6 +62,7 @@ type MockDataCoord struct { setSegmentStateResp *datapb.SetSegmentStateResponse importResp *datapb.ImportTaskResponse updateSegStatResp *commonpb.Status + updateChanPos *commonpb.Status acquireSegLockResp *commonpb.Status releaseSegLockResp *commonpb.Status addSegmentResp *commonpb.Status @@ -200,6 +201,10 @@ func (m *MockDataCoord) UpdateSegmentStatistics(ctx context.Context, req *datapb return m.updateSegStatResp, m.err } +func (m *MockDataCoord) UpdateChannelCheckpoint(ctx context.Context, req *datapb.UpdateChannelCheckpointRequest) (*commonpb.Status, error) { + return m.updateChanPos, m.err +} + func (m *MockDataCoord) AcquireSegmentLock(ctx context.Context, req *datapb.AcquireSegmentLockRequest) (*commonpb.Status, error) { return m.acquireSegLockResp, m.err } @@ -475,6 +480,17 @@ func Test_NewServer(t *testing.T) { assert.NotNil(t, resp) }) + t.Run("UpdateChannelCheckpoint", func(t *testing.T) { + server.dataCoord = &MockDataCoord{ + updateChanPos: &commonpb.Status{ + ErrorCode: commonpb.ErrorCode_Success, + }, + } + resp, err := server.UpdateChannelCheckpoint(ctx, nil) + assert.Nil(t, err) + assert.NotNil(t, resp) + }) + t.Run("acquire segment reference lock", func(t *testing.T) { server.dataCoord = &MockDataCoord{ acquireSegLockResp: &commonpb.Status{ diff --git a/internal/distributed/proxy/service_test.go b/internal/distributed/proxy/service_test.go index 0a915101fa..cd74ae4c7a 100644 --- a/internal/distributed/proxy/service_test.go +++ b/internal/distributed/proxy/service_test.go @@ -592,6 +592,10 @@ func (m *MockDataCoord) UpdateSegmentStatistics(ctx context.Context, req *datapb return nil, nil } +func (m *MockDataCoord) UpdateChannelCheckpoint(ctx context.Context, req *datapb.UpdateChannelCheckpointRequest) (*commonpb.Status, error) { + return nil, nil +} + func (m *MockDataCoord) AcquireSegmentLock(ctx context.Context, req *datapb.AcquireSegmentLockRequest) (*commonpb.Status, error) { return nil, nil } diff --git a/internal/metastore/catalog.go b/internal/metastore/catalog.go index 4de579d22c..553e53fc6a 100644 --- a/internal/metastore/catalog.go +++ b/internal/metastore/catalog.go @@ -6,6 +6,7 @@ import ( "github.com/milvus-io/milvus-proto/go-api/milvuspb" "github.com/milvus-io/milvus/internal/metastore/model" "github.com/milvus-io/milvus/internal/proto/datapb" + "github.com/milvus-io/milvus/internal/proto/internalpb" "github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/internal/util/typeutil" ) @@ -79,11 +80,15 @@ type DataCoordCatalog interface { AlterSegment(ctx context.Context, newSegment *datapb.SegmentInfo, oldSegment *datapb.SegmentInfo) error SaveDroppedSegmentsInBatch(ctx context.Context, segments []*datapb.SegmentInfo) error DropSegment(ctx context.Context, segment *datapb.SegmentInfo) error + RevertAlterSegmentsAndAddNewSegment(ctx context.Context, segments []*datapb.SegmentInfo, removalSegment *datapb.SegmentInfo) error + MarkChannelDeleted(ctx context.Context, channel string) error IsChannelDropped(ctx context.Context, channel string) bool DropChannel(ctx context.Context, channel string) error - RevertAlterSegmentsAndAddNewSegment(ctx context.Context, segments []*datapb.SegmentInfo, removalSegment *datapb.SegmentInfo) error + ListChannelCheckpoint(ctx context.Context) (map[string]*internalpb.MsgPosition, error) + SaveChannelCheckpoint(ctx context.Context, vChannel string, pos *internalpb.MsgPosition) error + DropChannelCheckpoint(ctx context.Context, vChannel string) error } type IndexCoordCatalog interface { diff --git a/internal/metastore/kv/datacoord/constant.go b/internal/metastore/kv/datacoord/constant.go index 5ad63cc030..077432b22b 100644 --- a/internal/metastore/kv/datacoord/constant.go +++ b/internal/metastore/kv/datacoord/constant.go @@ -23,6 +23,7 @@ const ( SegmentDeltalogPathPrefix = MetaPrefix + "/deltalog" SegmentStatslogPathPrefix = MetaPrefix + "/statslog" ChannelRemovePrefix = MetaPrefix + "/channel-removal" + ChannelCheckpointPrefix = MetaPrefix + "/channel-cp" RemoveFlagTomestone = "removed" ) diff --git a/internal/metastore/kv/datacoord/kv_catalog.go b/internal/metastore/kv/datacoord/kv_catalog.go index d64665f598..a814c7139c 100644 --- a/internal/metastore/kv/datacoord/kv_catalog.go +++ b/internal/metastore/kv/datacoord/kv_catalog.go @@ -22,21 +22,20 @@ import ( "strconv" "strings" - "github.com/milvus-io/milvus/internal/util/metautil" - - "github.com/milvus-io/milvus/internal/util/etcd" - + "github.com/golang/protobuf/proto" + "go.uber.org/zap" "golang.org/x/exp/maps" - "github.com/golang/protobuf/proto" "github.com/milvus-io/milvus-proto/go-api/commonpb" "github.com/milvus-io/milvus/internal/kv" "github.com/milvus-io/milvus/internal/log" "github.com/milvus-io/milvus/internal/proto/datapb" + "github.com/milvus-io/milvus/internal/proto/internalpb" "github.com/milvus-io/milvus/internal/storage" "github.com/milvus-io/milvus/internal/util" + "github.com/milvus-io/milvus/internal/util/etcd" + "github.com/milvus-io/milvus/internal/util/metautil" "github.com/milvus-io/milvus/internal/util/typeutil" - "go.uber.org/zap" ) type Catalog struct { @@ -312,6 +311,43 @@ func (kc *Catalog) DropChannel(ctx context.Context, channel string) error { return kc.Txn.Remove(key) } +func (kc *Catalog) ListChannelCheckpoint(ctx context.Context) (map[string]*internalpb.MsgPosition, error) { + keys, values, err := kc.Txn.LoadWithPrefix(ChannelCheckpointPrefix) + if err != nil { + return nil, err + } + + channelCPs := make(map[string]*internalpb.MsgPosition) + for i, key := range keys { + value := values[i] + channelCP := &internalpb.MsgPosition{} + err = proto.Unmarshal([]byte(value), channelCP) + if err != nil { + log.Error("unmarshal channelCP failed when ListChannelCheckpoint", zap.Error(err)) + return nil, err + } + ss := strings.Split(key, "/") + vChannel := ss[len(ss)-1] + channelCPs[vChannel] = channelCP + } + + return channelCPs, nil +} + +func (kc *Catalog) SaveChannelCheckpoint(ctx context.Context, vChannel string, pos *internalpb.MsgPosition) error { + k := buildChannelCPKey(vChannel) + v, err := proto.Marshal(pos) + if err != nil { + return err + } + return kc.Txn.Save(k, string(v)) +} + +func (kc *Catalog) DropChannelCheckpoint(ctx context.Context, vChannel string) error { + k := buildChannelCPKey(vChannel) + return kc.Txn.Remove(k) +} + func (kc *Catalog) getBinlogsWithPrefix(binlogType storage.BinlogType, collectionID, partitionID, segmentID typeutil.UniqueID) ([]string, []string, error) { var binlogPrefix string @@ -567,3 +603,7 @@ func buildFieldStatslogPathPrefix(collectionID typeutil.UniqueID, partitionID ty func buildChannelRemovePath(channel string) string { return fmt.Sprintf("%s/%s", ChannelRemovePrefix, channel) } + +func buildChannelCPKey(vChannel string) string { + return fmt.Sprintf("%s/%s", ChannelCheckpointPrefix, vChannel) +} diff --git a/internal/metastore/kv/datacoord/kv_catalog_test.go b/internal/metastore/kv/datacoord/kv_catalog_test.go index 58ea0f524a..021c87d5f4 100644 --- a/internal/metastore/kv/datacoord/kv_catalog_test.go +++ b/internal/metastore/kv/datacoord/kv_catalog_test.go @@ -7,14 +7,16 @@ import ( "testing" "github.com/golang/protobuf/proto" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/mock" + "golang.org/x/exp/maps" + "github.com/milvus-io/milvus-proto/go-api/commonpb" "github.com/milvus-io/milvus/internal/kv" "github.com/milvus-io/milvus/internal/kv/mocks" "github.com/milvus-io/milvus/internal/proto/datapb" + "github.com/milvus-io/milvus/internal/proto/internalpb" "github.com/milvus-io/milvus/internal/util/metautil" - "github.com/stretchr/testify/assert" - "github.com/stretchr/testify/mock" - "golang.org/x/exp/maps" ) type MockedTxnKV struct { @@ -509,6 +511,81 @@ func TestCatalog_RevertAlterSegmentsAndAddNewSegment(t *testing.T) { }) } +func TestChannelCP(t *testing.T) { + mockVChannel := "fake-by-dev-rootcoord-dml-1-testchannelcp-v0" + mockPChannel := "fake-by-dev-rootcoord-dml-1" + + pos := &internalpb.MsgPosition{ + ChannelName: mockPChannel, + MsgID: []byte{}, + Timestamp: 1000, + } + k := buildChannelCPKey(mockVChannel) + v, err := proto.Marshal(pos) + assert.NoError(t, err) + + t.Run("ListChannelCheckpoint", func(t *testing.T) { + txn := &mocks.TxnKV{} + txn.EXPECT().Save(mock.Anything, mock.Anything).Return(nil) + catalog := &Catalog{txn, ""} + err := catalog.SaveChannelCheckpoint(context.TODO(), mockVChannel, pos) + assert.NoError(t, err) + + txn.EXPECT().LoadWithPrefix(mock.Anything).Return([]string{k}, []string{string(v)}, nil) + res, err := catalog.ListChannelCheckpoint(context.TODO()) + assert.NoError(t, err) + assert.True(t, len(res) > 0) + }) + + t.Run("ListChannelCheckpoint failed", func(t *testing.T) { + txn := &mocks.TxnKV{} + catalog := &Catalog{txn, ""} + txn.EXPECT().LoadWithPrefix(mock.Anything).Return(nil, nil, errors.New("mock error")) + _, err = catalog.ListChannelCheckpoint(context.TODO()) + assert.Error(t, err) + }) + + t.Run("SaveChannelCheckpoint", func(t *testing.T) { + txn := &mocks.TxnKV{} + txn.EXPECT().Save(mock.Anything, mock.Anything).Return(nil) + catalog := &Catalog{txn, ""} + err := catalog.SaveChannelCheckpoint(context.TODO(), mockVChannel, pos) + assert.NoError(t, err) + }) + + t.Run("SaveChannelCheckpoint failed", func(t *testing.T) { + txn := &mocks.TxnKV{} + catalog := &Catalog{txn, ""} + txn.EXPECT().Save(mock.Anything, mock.Anything).Return(errors.New("mock error")) + err = catalog.SaveChannelCheckpoint(context.TODO(), mockVChannel, &internalpb.MsgPosition{}) + assert.Error(t, err) + }) + + t.Run("DropChannelCheckpoint", func(t *testing.T) { + txn := &mocks.TxnKV{} + txn.EXPECT().Save(mock.Anything, mock.Anything).Return(nil) + catalog := &Catalog{txn, ""} + err := catalog.SaveChannelCheckpoint(context.TODO(), mockVChannel, pos) + assert.NoError(t, err) + + txn.EXPECT().Remove(mock.Anything).Return(nil) + txn.EXPECT().LoadWithPrefix(mock.Anything).Return(nil, nil, nil) + err = catalog.DropChannelCheckpoint(context.TODO(), mockVChannel) + assert.NoError(t, err) + res, err := catalog.ListChannelCheckpoint(context.TODO()) + assert.NoError(t, err) + assert.True(t, len(res) == 0) + }) + + t.Run("DropChannelCheckpoint failed", func(t *testing.T) { + txn := &mocks.TxnKV{} + catalog := &Catalog{txn, ""} + txn.EXPECT().Remove(mock.Anything).Return(errors.New("mock error")) + err = catalog.DropChannelCheckpoint(context.TODO(), mockVChannel) + assert.Error(t, err) + }) +} + func Test_MarkChannelDeleted_SaveError(t *testing.T) { txn := &mocks.TxnKV{} txn.EXPECT(). diff --git a/internal/proto/data_coord.proto b/internal/proto/data_coord.proto index bbb1b40a7c..badd69d955 100644 --- a/internal/proto/data_coord.proto +++ b/internal/proto/data_coord.proto @@ -57,6 +57,7 @@ service DataCoord { // https://wiki.lfaidata.foundation/display/MIL/MEP+24+--+Support+bulk+load rpc Import(ImportTaskRequest) returns (ImportTaskResponse) {} rpc UpdateSegmentStatistics(UpdateSegmentStatisticsRequest) returns (common.Status) {} + rpc UpdateChannelCheckpoint(UpdateChannelCheckpointRequest) returns (common.Status) {} rpc AcquireSegmentLock(AcquireSegmentLockRequest) returns (common.Status) {} rpc ReleaseSegmentLock(ReleaseSegmentLockRequest) returns (common.Status) {} @@ -241,7 +242,6 @@ message FlushSegmentsRequest { int64 dbID = 2; int64 collectionID = 3; repeated int64 segmentIDs = 4; // segments to flush - repeated int64 markSegmentIDs = 5; // segments to clean buffer and mark segment position, but NOT flushed } message SegmentMsg{ @@ -579,6 +579,12 @@ message UpdateSegmentStatisticsRequest { repeated SegmentStats stats = 2; } +message UpdateChannelCheckpointRequest { + common.MsgBase base = 1; + string vChannel = 2; + internal.MsgPosition position = 3; +} + message ResendSegmentStatsRequest { common.MsgBase base = 1; } diff --git a/internal/proto/datapb/data_coord.pb.go b/internal/proto/datapb/data_coord.pb.go index 668cd5b32f..6f96d484eb 100644 --- a/internal/proto/datapb/data_coord.pb.go +++ b/internal/proto/datapb/data_coord.pb.go @@ -1465,7 +1465,6 @@ type FlushSegmentsRequest struct { DbID int64 `protobuf:"varint,2,opt,name=dbID,proto3" json:"dbID,omitempty"` CollectionID int64 `protobuf:"varint,3,opt,name=collectionID,proto3" json:"collectionID,omitempty"` SegmentIDs []int64 `protobuf:"varint,4,rep,packed,name=segmentIDs,proto3" json:"segmentIDs,omitempty"` - MarkSegmentIDs []int64 `protobuf:"varint,5,rep,packed,name=markSegmentIDs,proto3" json:"markSegmentIDs,omitempty"` XXX_NoUnkeyedLiteral struct{} `json:"-"` XXX_unrecognized []byte `json:"-"` XXX_sizecache int32 `json:"-"` @@ -1524,13 +1523,6 @@ func (m *FlushSegmentsRequest) GetSegmentIDs() []int64 { return nil } -func (m *FlushSegmentsRequest) GetMarkSegmentIDs() []int64 { - if m != nil { - return m.MarkSegmentIDs - } - return nil -} - type SegmentMsg struct { Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"` Segment *SegmentInfo `protobuf:"bytes,2,opt,name=segment,proto3" json:"segment,omitempty"` @@ -4292,6 +4284,61 @@ func (m *UpdateSegmentStatisticsRequest) GetStats() []*SegmentStats { return nil } +type UpdateChannelCheckpointRequest struct { + Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"` + VChannel string `protobuf:"bytes,2,opt,name=vChannel,proto3" json:"vChannel,omitempty"` + Position *internalpb.MsgPosition `protobuf:"bytes,3,opt,name=position,proto3" json:"position,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *UpdateChannelCheckpointRequest) Reset() { *m = UpdateChannelCheckpointRequest{} } +func (m *UpdateChannelCheckpointRequest) String() string { return proto.CompactTextString(m) } +func (*UpdateChannelCheckpointRequest) ProtoMessage() {} +func (*UpdateChannelCheckpointRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_82cd95f524594f49, []int{66} +} + +func (m *UpdateChannelCheckpointRequest) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_UpdateChannelCheckpointRequest.Unmarshal(m, b) +} +func (m *UpdateChannelCheckpointRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_UpdateChannelCheckpointRequest.Marshal(b, m, deterministic) +} +func (m *UpdateChannelCheckpointRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_UpdateChannelCheckpointRequest.Merge(m, src) +} +func (m *UpdateChannelCheckpointRequest) XXX_Size() int { + return xxx_messageInfo_UpdateChannelCheckpointRequest.Size(m) +} +func (m *UpdateChannelCheckpointRequest) XXX_DiscardUnknown() { + xxx_messageInfo_UpdateChannelCheckpointRequest.DiscardUnknown(m) +} + +var xxx_messageInfo_UpdateChannelCheckpointRequest proto.InternalMessageInfo + +func (m *UpdateChannelCheckpointRequest) GetBase() *commonpb.MsgBase { + if m != nil { + return m.Base + } + return nil +} + +func (m *UpdateChannelCheckpointRequest) GetVChannel() string { + if m != nil { + return m.VChannel + } + return "" +} + +func (m *UpdateChannelCheckpointRequest) GetPosition() *internalpb.MsgPosition { + if m != nil { + return m.Position + } + return nil +} + type ResendSegmentStatsRequest struct { Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"` XXX_NoUnkeyedLiteral struct{} `json:"-"` @@ -4303,7 +4350,7 @@ func (m *ResendSegmentStatsRequest) Reset() { *m = ResendSegmentStatsReq func (m *ResendSegmentStatsRequest) String() string { return proto.CompactTextString(m) } func (*ResendSegmentStatsRequest) ProtoMessage() {} func (*ResendSegmentStatsRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_82cd95f524594f49, []int{66} + return fileDescriptor_82cd95f524594f49, []int{67} } func (m *ResendSegmentStatsRequest) XXX_Unmarshal(b []byte) error { @@ -4343,7 +4390,7 @@ func (m *ResendSegmentStatsResponse) Reset() { *m = ResendSegmentStatsRe func (m *ResendSegmentStatsResponse) String() string { return proto.CompactTextString(m) } func (*ResendSegmentStatsResponse) ProtoMessage() {} func (*ResendSegmentStatsResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_82cd95f524594f49, []int{67} + return fileDescriptor_82cd95f524594f49, []int{68} } func (m *ResendSegmentStatsResponse) XXX_Unmarshal(b []byte) error { @@ -4395,7 +4442,7 @@ func (m *AddImportSegmentRequest) Reset() { *m = AddImportSegmentRequest func (m *AddImportSegmentRequest) String() string { return proto.CompactTextString(m) } func (*AddImportSegmentRequest) ProtoMessage() {} func (*AddImportSegmentRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_82cd95f524594f49, []int{68} + return fileDescriptor_82cd95f524594f49, []int{69} } func (m *AddImportSegmentRequest) XXX_Unmarshal(b []byte) error { @@ -4477,7 +4524,7 @@ func (m *AddImportSegmentResponse) Reset() { *m = AddImportSegmentRespon func (m *AddImportSegmentResponse) String() string { return proto.CompactTextString(m) } func (*AddImportSegmentResponse) ProtoMessage() {} func (*AddImportSegmentResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_82cd95f524594f49, []int{69} + return fileDescriptor_82cd95f524594f49, []int{70} } func (m *AddImportSegmentResponse) XXX_Unmarshal(b []byte) error { @@ -4530,7 +4577,7 @@ func (m *SaveImportSegmentRequest) Reset() { *m = SaveImportSegmentReque func (m *SaveImportSegmentRequest) String() string { return proto.CompactTextString(m) } func (*SaveImportSegmentRequest) ProtoMessage() {} func (*SaveImportSegmentRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_82cd95f524594f49, []int{70} + return fileDescriptor_82cd95f524594f49, []int{71} } func (m *SaveImportSegmentRequest) XXX_Unmarshal(b []byte) error { @@ -4619,7 +4666,7 @@ func (m *UnsetIsImportingStateRequest) Reset() { *m = UnsetIsImportingSt func (m *UnsetIsImportingStateRequest) String() string { return proto.CompactTextString(m) } func (*UnsetIsImportingStateRequest) ProtoMessage() {} func (*UnsetIsImportingStateRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_82cd95f524594f49, []int{71} + return fileDescriptor_82cd95f524594f49, []int{72} } func (m *UnsetIsImportingStateRequest) XXX_Unmarshal(b []byte) error { @@ -4666,7 +4713,7 @@ func (m *MarkSegmentsDroppedRequest) Reset() { *m = MarkSegmentsDroppedR func (m *MarkSegmentsDroppedRequest) String() string { return proto.CompactTextString(m) } func (*MarkSegmentsDroppedRequest) ProtoMessage() {} func (*MarkSegmentsDroppedRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_82cd95f524594f49, []int{72} + return fileDescriptor_82cd95f524594f49, []int{73} } func (m *MarkSegmentsDroppedRequest) XXX_Unmarshal(b []byte) error { @@ -4714,7 +4761,7 @@ func (m *SegmentReferenceLock) Reset() { *m = SegmentReferenceLock{} } func (m *SegmentReferenceLock) String() string { return proto.CompactTextString(m) } func (*SegmentReferenceLock) ProtoMessage() {} func (*SegmentReferenceLock) Descriptor() ([]byte, []int) { - return fileDescriptor_82cd95f524594f49, []int{73} + return fileDescriptor_82cd95f524594f49, []int{74} } func (m *SegmentReferenceLock) XXX_Unmarshal(b []byte) error { @@ -4826,6 +4873,7 @@ func init() { proto.RegisterType((*ImportTaskResponse)(nil), "milvus.proto.data.ImportTaskResponse") proto.RegisterType((*ImportTaskRequest)(nil), "milvus.proto.data.ImportTaskRequest") proto.RegisterType((*UpdateSegmentStatisticsRequest)(nil), "milvus.proto.data.UpdateSegmentStatisticsRequest") + proto.RegisterType((*UpdateChannelCheckpointRequest)(nil), "milvus.proto.data.UpdateChannelCheckpointRequest") proto.RegisterType((*ResendSegmentStatsRequest)(nil), "milvus.proto.data.ResendSegmentStatsRequest") proto.RegisterType((*ResendSegmentStatsResponse)(nil), "milvus.proto.data.ResendSegmentStatsResponse") proto.RegisterType((*AddImportSegmentRequest)(nil), "milvus.proto.data.AddImportSegmentRequest") @@ -4839,278 +4887,280 @@ func init() { func init() { proto.RegisterFile("data_coord.proto", fileDescriptor_82cd95f524594f49) } var fileDescriptor_82cd95f524594f49 = []byte{ - // 4328 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe4, 0x3c, 0x5b, 0x6f, 0x1c, 0x59, - 0x5a, 0xa9, 0xbe, 0xb9, 0xfb, 0xeb, 0x8b, 0xdb, 0x27, 0x19, 0xbb, 0xd3, 0xb9, 0x4e, 0x65, 0x32, - 0x93, 0xc9, 0x66, 0x92, 0x19, 0x0f, 0xa3, 0x1d, 0xc8, 0xce, 0xac, 0x62, 0x7b, 0xec, 0x69, 0xb0, - 0xbd, 0xde, 0xb2, 0x33, 0x91, 0x76, 0x91, 0x5a, 0xe5, 0xae, 0xe3, 0x76, 0xad, 0xeb, 0xd2, 0xa9, - 0xaa, 0xb6, 0xe3, 0xe5, 0x61, 0x47, 0xac, 0x84, 0xb4, 0x08, 0xb1, 0x08, 0x09, 0x01, 0x12, 0x48, - 0x88, 0xa7, 0x65, 0x11, 0x12, 0xd2, 0x8a, 0x17, 0x5e, 0x78, 0x1d, 0xc1, 0xc3, 0x0a, 0x21, 0xf1, - 0x03, 0x78, 0x00, 0xde, 0x79, 0xe5, 0x01, 0x9d, 0x4b, 0x9d, 0xba, 0x77, 0x97, 0xbb, 0x93, 0x09, - 0x82, 0xb7, 0x3e, 0x5f, 0x7d, 0xe7, 0xf6, 0x9d, 0xef, 0x7e, 0xbe, 0xd3, 0xd0, 0xd6, 0x54, 0x4f, - 0xed, 0x0f, 0x6c, 0xdb, 0xd1, 0x1e, 0x8e, 0x1c, 0xdb, 0xb3, 0xd1, 0x92, 0xa9, 0x1b, 0xa7, 0x63, - 0x97, 0xb5, 0x1e, 0x92, 0xcf, 0xdd, 0xc6, 0xc0, 0x36, 0x4d, 0xdb, 0x62, 0xa0, 0x6e, 0x4b, 0xb7, - 0x3c, 0xec, 0x58, 0xaa, 0xc1, 0xdb, 0x8d, 0x70, 0x87, 0x6e, 0xc3, 0x1d, 0x1c, 0x63, 0x53, 0x65, - 0x2d, 0x79, 0x01, 0xca, 0x9f, 0x99, 0x23, 0xef, 0x5c, 0xfe, 0x13, 0x09, 0x1a, 0x9b, 0xc6, 0xd8, - 0x3d, 0x56, 0xf0, 0xf3, 0x31, 0x76, 0x3d, 0xf4, 0x3e, 0x94, 0x0e, 0x55, 0x17, 0x77, 0xa4, 0xdb, - 0xd2, 0xbd, 0xfa, 0xea, 0xf5, 0x87, 0x91, 0x59, 0xf9, 0x7c, 0x3b, 0xee, 0x70, 0x4d, 0x75, 0xb1, - 0x42, 0x31, 0x11, 0x82, 0x92, 0x76, 0xd8, 0xdb, 0xe8, 0x14, 0x6e, 0x4b, 0xf7, 0x8a, 0x0a, 0xfd, - 0x8d, 0x6e, 0x02, 0xb8, 0x78, 0x68, 0x62, 0xcb, 0xeb, 0x6d, 0xb8, 0x9d, 0xe2, 0xed, 0xe2, 0xbd, - 0xa2, 0x12, 0x82, 0x20, 0x19, 0x1a, 0x03, 0xdb, 0x30, 0xf0, 0xc0, 0xd3, 0x6d, 0xab, 0xb7, 0xd1, - 0x29, 0xd1, 0xbe, 0x11, 0x98, 0xfc, 0xef, 0x12, 0x34, 0xf9, 0xd2, 0xdc, 0x91, 0x6d, 0xb9, 0x18, - 0x7d, 0x08, 0x15, 0xd7, 0x53, 0xbd, 0xb1, 0xcb, 0x57, 0x77, 0x2d, 0x75, 0x75, 0xfb, 0x14, 0x45, - 0xe1, 0xa8, 0xa9, 0xcb, 0x8b, 0x4f, 0x5f, 0x4c, 0x4e, 0x1f, 0xdb, 0x42, 0x29, 0xb1, 0x85, 0x7b, - 0xb0, 0x78, 0x44, 0x56, 0xb7, 0x1f, 0x20, 0x95, 0x29, 0x52, 0x1c, 0x4c, 0x46, 0xf2, 0x74, 0x13, - 0x7f, 0xe7, 0x68, 0x1f, 0xab, 0x46, 0xa7, 0x42, 0xe7, 0x0a, 0x41, 0xe4, 0x7f, 0x96, 0xa0, 0x2d, - 0xd0, 0xfd, 0x73, 0xb8, 0x02, 0xe5, 0x81, 0x3d, 0xb6, 0x3c, 0xba, 0xd5, 0xa6, 0xc2, 0x1a, 0xe8, - 0x4d, 0x68, 0x0c, 0x8e, 0x55, 0xcb, 0xc2, 0x46, 0xdf, 0x52, 0x4d, 0x4c, 0x37, 0x55, 0x53, 0xea, - 0x1c, 0xb6, 0xab, 0x9a, 0x38, 0xd7, 0xde, 0x6e, 0x43, 0x7d, 0xa4, 0x3a, 0x9e, 0x1e, 0xa1, 0x7e, - 0x18, 0x84, 0xba, 0x50, 0xd5, 0xdd, 0x9e, 0x39, 0xb2, 0x1d, 0xaf, 0x53, 0xbe, 0x2d, 0xdd, 0xab, - 0x2a, 0xa2, 0x4d, 0x66, 0xd0, 0xe9, 0xaf, 0x03, 0xd5, 0x3d, 0xe9, 0x6d, 0xf0, 0x1d, 0x45, 0x60, - 0xf2, 0x5f, 0x48, 0xb0, 0xfc, 0xc4, 0x75, 0xf5, 0xa1, 0x95, 0xd8, 0xd9, 0x32, 0x54, 0x2c, 0x5b, - 0xc3, 0xbd, 0x0d, 0xba, 0xb5, 0xa2, 0xc2, 0x5b, 0xe8, 0x1a, 0xd4, 0x46, 0x18, 0x3b, 0x7d, 0xc7, - 0x36, 0xfc, 0x8d, 0x55, 0x09, 0x40, 0xb1, 0x0d, 0x8c, 0xbe, 0x0b, 0x4b, 0x6e, 0x6c, 0x20, 0xc6, - 0x57, 0xf5, 0xd5, 0x3b, 0x0f, 0x13, 0x92, 0xf1, 0x30, 0x3e, 0xa9, 0x92, 0xec, 0x2d, 0x7f, 0x59, - 0x80, 0xcb, 0x02, 0x8f, 0xad, 0x95, 0xfc, 0x26, 0x94, 0x77, 0xf1, 0x50, 0x2c, 0x8f, 0x35, 0xf2, - 0x50, 0x5e, 0x1c, 0x59, 0x31, 0x7c, 0x64, 0x39, 0x58, 0x3d, 0x7e, 0x1e, 0xe5, 0xe4, 0x79, 0xdc, - 0x82, 0x3a, 0x7e, 0x31, 0xd2, 0x1d, 0xdc, 0x27, 0x8c, 0x43, 0x49, 0x5e, 0x52, 0x80, 0x81, 0x0e, - 0x74, 0x33, 0x2c, 0x1b, 0x0b, 0xb9, 0x65, 0x43, 0xfe, 0x4b, 0x09, 0x56, 0x12, 0xa7, 0xc4, 0x85, - 0x4d, 0x81, 0x36, 0xdd, 0x79, 0x40, 0x19, 0x22, 0x76, 0x84, 0xe0, 0x6f, 0x4f, 0x22, 0x78, 0x80, - 0xae, 0x24, 0xfa, 0x87, 0x16, 0x59, 0xc8, 0xbf, 0xc8, 0x13, 0x58, 0xd9, 0xc2, 0x1e, 0x9f, 0x80, - 0x7c, 0xc3, 0xee, 0xec, 0xca, 0x2a, 0x2a, 0xd5, 0x85, 0xb8, 0x54, 0xcb, 0x7f, 0x5b, 0x10, 0xb2, - 0x48, 0xa7, 0xea, 0x59, 0x47, 0x36, 0xba, 0x0e, 0x35, 0x81, 0xc2, 0xb9, 0x22, 0x00, 0xa0, 0x6f, - 0x42, 0x99, 0xac, 0x94, 0xb1, 0x44, 0x6b, 0xf5, 0xcd, 0xf4, 0x3d, 0x85, 0xc6, 0x54, 0x18, 0x3e, - 0xea, 0x41, 0xcb, 0xf5, 0x54, 0xc7, 0xeb, 0x8f, 0x6c, 0x97, 0x9e, 0x33, 0x65, 0x9c, 0xfa, 0xaa, - 0x1c, 0x1d, 0x41, 0xa8, 0xf5, 0x1d, 0x77, 0xb8, 0xc7, 0x31, 0x95, 0x26, 0xed, 0xe9, 0x37, 0xd1, - 0x67, 0xd0, 0xc0, 0x96, 0x16, 0x0c, 0x54, 0xca, 0x3d, 0x50, 0x1d, 0x5b, 0x9a, 0x18, 0x26, 0x38, - 0x9f, 0x72, 0xfe, 0xf3, 0xf9, 0x3d, 0x09, 0x3a, 0xc9, 0x03, 0x9a, 0x47, 0x65, 0x3f, 0x66, 0x9d, - 0x30, 0x3b, 0xa0, 0x89, 0x12, 0x2e, 0x0e, 0x49, 0xe1, 0x5d, 0xe4, 0x3f, 0x92, 0xe0, 0x8d, 0x60, - 0x39, 0xf4, 0xd3, 0xab, 0xe2, 0x16, 0x74, 0x1f, 0xda, 0xba, 0x35, 0x30, 0xc6, 0x1a, 0x7e, 0x6a, - 0x7d, 0x8e, 0x55, 0xc3, 0x3b, 0x3e, 0xa7, 0x67, 0x58, 0x55, 0x12, 0x70, 0xf9, 0xc7, 0x12, 0x2c, - 0xc7, 0xd7, 0x35, 0x0f, 0x91, 0x7e, 0x05, 0xca, 0xba, 0x75, 0x64, 0xfb, 0x34, 0xba, 0x39, 0x41, - 0x28, 0xc9, 0x5c, 0x0c, 0x59, 0x36, 0xe1, 0xda, 0x16, 0xf6, 0x7a, 0x96, 0x8b, 0x1d, 0x6f, 0x4d, - 0xb7, 0x0c, 0x7b, 0xb8, 0xa7, 0x7a, 0xc7, 0x73, 0x08, 0x54, 0x44, 0x36, 0x0a, 0x31, 0xd9, 0x90, - 0x7f, 0x26, 0xc1, 0xf5, 0xf4, 0xf9, 0xf8, 0xd6, 0xbb, 0x50, 0x3d, 0xd2, 0xb1, 0xa1, 0x11, 0xfa, - 0x4a, 0x94, 0xbe, 0xa2, 0x4d, 0x04, 0x6b, 0x44, 0x90, 0xf9, 0x0e, 0xdf, 0xcc, 0xe0, 0xe6, 0x7d, - 0xcf, 0xd1, 0xad, 0xe1, 0xb6, 0xee, 0x7a, 0x0a, 0xc3, 0x0f, 0xd1, 0xb3, 0x98, 0x9f, 0x8d, 0x7f, - 0x57, 0x82, 0x9b, 0x5b, 0xd8, 0x5b, 0x17, 0x7a, 0x99, 0x7c, 0xd7, 0x5d, 0x4f, 0x1f, 0xb8, 0x2f, - 0xd7, 0x37, 0xca, 0x61, 0xa0, 0xe5, 0x9f, 0x4a, 0x70, 0x2b, 0x73, 0x31, 0x9c, 0x74, 0x5c, 0xef, - 0xf8, 0x5a, 0x39, 0x5d, 0xef, 0xfc, 0x06, 0x3e, 0xff, 0x42, 0x35, 0xc6, 0x78, 0x4f, 0xd5, 0x1d, - 0xa6, 0x77, 0x66, 0xd4, 0xc2, 0x7f, 0x23, 0xc1, 0x8d, 0x2d, 0xec, 0xed, 0xf9, 0x36, 0xe9, 0x35, - 0x52, 0x87, 0xe0, 0x84, 0x6c, 0xa3, 0xef, 0x9c, 0x45, 0x60, 0xf2, 0xef, 0xb3, 0xe3, 0x4c, 0x5d, - 0xef, 0x6b, 0x21, 0xe0, 0x4d, 0x2a, 0x09, 0x21, 0x91, 0x5c, 0x67, 0xae, 0x03, 0x27, 0x9f, 0xfc, - 0xe7, 0x12, 0x5c, 0x7d, 0x32, 0x78, 0x3e, 0xd6, 0x1d, 0xcc, 0x91, 0xb6, 0xed, 0xc1, 0xc9, 0xec, - 0xc4, 0x0d, 0xdc, 0xac, 0x42, 0xc4, 0xcd, 0x9a, 0xe6, 0x9a, 0x2f, 0x43, 0xc5, 0x63, 0x7e, 0x1d, - 0xf3, 0x54, 0x78, 0x8b, 0xae, 0x4f, 0xc1, 0x06, 0x56, 0xdd, 0xff, 0x9d, 0xeb, 0xfb, 0x69, 0x09, - 0x1a, 0x5f, 0x70, 0x77, 0x8c, 0x5a, 0xed, 0x38, 0x27, 0x49, 0xe9, 0x8e, 0x57, 0xc8, 0x83, 0x4b, - 0x73, 0xea, 0xb6, 0xa0, 0xe9, 0x62, 0x7c, 0x32, 0x8b, 0x8d, 0x6e, 0x90, 0x8e, 0xc2, 0xb6, 0x6e, - 0xc3, 0xd2, 0xd8, 0xa2, 0xa1, 0x01, 0xd6, 0x38, 0x01, 0x19, 0xe7, 0x4e, 0xd7, 0xdd, 0xc9, 0x8e, - 0xe8, 0x73, 0x1e, 0x7d, 0x84, 0xc6, 0x2a, 0xe7, 0x1a, 0x2b, 0xde, 0x0d, 0xf5, 0xa0, 0xad, 0x39, - 0xf6, 0x68, 0x84, 0xb5, 0xbe, 0xeb, 0x0f, 0x55, 0xc9, 0x37, 0x14, 0xef, 0x27, 0x86, 0x7a, 0x1f, - 0x2e, 0xc7, 0x57, 0xda, 0xd3, 0x88, 0x43, 0x4a, 0xce, 0x30, 0xed, 0x13, 0x7a, 0x00, 0x4b, 0x49, - 0xfc, 0x2a, 0xc5, 0x4f, 0x7e, 0x40, 0xef, 0x01, 0x8a, 0x2d, 0x95, 0xa0, 0xd7, 0x18, 0x7a, 0x74, - 0x31, 0x3d, 0xcd, 0x95, 0x7f, 0x22, 0xc1, 0xf2, 0x33, 0xd5, 0x1b, 0x1c, 0x6f, 0x98, 0x5c, 0xd6, - 0xe6, 0xd0, 0x55, 0x9f, 0x40, 0xed, 0x94, 0xf3, 0x85, 0x6f, 0x90, 0x6e, 0xa5, 0xd0, 0x27, 0xcc, - 0x81, 0x4a, 0xd0, 0x43, 0xfe, 0x4a, 0x82, 0x2b, 0x9b, 0xa1, 0xb8, 0xf0, 0x35, 0x68, 0xcd, 0x69, - 0x01, 0xed, 0xdb, 0xd0, 0x32, 0x55, 0xe7, 0x24, 0x11, 0xcf, 0xc6, 0xa0, 0xf2, 0x0b, 0x00, 0xde, - 0xda, 0x71, 0x87, 0x33, 0xac, 0xff, 0x63, 0x58, 0xe0, 0xb3, 0x72, 0xf5, 0x39, 0x8d, 0xcf, 0x7c, - 0x74, 0xf9, 0xe7, 0x15, 0xa8, 0x87, 0x3e, 0xa0, 0x16, 0x14, 0x84, 0x5c, 0x17, 0x52, 0xa8, 0x50, - 0x98, 0x1e, 0x6a, 0x15, 0x93, 0xa1, 0xd6, 0x5d, 0x68, 0xe9, 0xd4, 0x5f, 0xe9, 0xf3, 0xd3, 0xa3, - 0x8a, 0xa6, 0xa6, 0x34, 0x19, 0x94, 0xb3, 0x12, 0xba, 0x09, 0x75, 0x6b, 0x6c, 0xf6, 0xed, 0xa3, - 0xbe, 0x63, 0x9f, 0xb9, 0x3c, 0x66, 0xab, 0x59, 0x63, 0xf3, 0x3b, 0x47, 0x8a, 0x7d, 0xe6, 0x06, - 0x61, 0x41, 0xe5, 0x82, 0x61, 0xc1, 0x4d, 0xa8, 0x9b, 0xea, 0x0b, 0x32, 0x6a, 0xdf, 0x1a, 0x9b, - 0x34, 0x9c, 0x2b, 0x2a, 0x35, 0x53, 0x7d, 0xa1, 0xd8, 0x67, 0xbb, 0x63, 0x13, 0xdd, 0x83, 0xb6, - 0xa1, 0xba, 0x5e, 0x3f, 0x1c, 0x0f, 0x56, 0x69, 0x3c, 0xd8, 0x22, 0xf0, 0xcf, 0x82, 0x98, 0x30, - 0x19, 0x60, 0xd4, 0xe6, 0x08, 0x30, 0x34, 0xd3, 0x08, 0x06, 0x82, 0xfc, 0x01, 0x86, 0x66, 0x1a, - 0x62, 0x98, 0x8f, 0x61, 0xe1, 0x90, 0x7a, 0x81, 0x6e, 0xa7, 0x9e, 0xa9, 0x63, 0x36, 0x89, 0x03, - 0xc8, 0x9c, 0x45, 0xc5, 0x47, 0x47, 0xdf, 0x82, 0x1a, 0x35, 0xbe, 0xb4, 0x6f, 0x23, 0x57, 0xdf, - 0xa0, 0x03, 0xe9, 0xad, 0x61, 0xc3, 0x53, 0x69, 0xef, 0x66, 0xbe, 0xde, 0xa2, 0x03, 0xd1, 0x6b, - 0x03, 0x07, 0xab, 0x1e, 0xd6, 0xd6, 0xce, 0xd7, 0x6d, 0x73, 0xa4, 0x52, 0x66, 0xea, 0xb4, 0xa8, - 0xa7, 0x9f, 0xf6, 0x89, 0xc8, 0xd2, 0x40, 0xb4, 0x36, 0x1d, 0xdb, 0xec, 0x2c, 0x32, 0x59, 0x8a, - 0x42, 0xd1, 0x0d, 0x00, 0x5f, 0xa3, 0xa9, 0x5e, 0xa7, 0x4d, 0x4f, 0xb1, 0xc6, 0x21, 0x4f, 0x68, - 0xba, 0x47, 0x77, 0xfb, 0x2c, 0xb1, 0xa2, 0x5b, 0xc3, 0xce, 0x12, 0x9d, 0xb1, 0xee, 0x67, 0x62, - 0x74, 0x6b, 0x88, 0x56, 0x60, 0x41, 0x77, 0xfb, 0x47, 0xea, 0x09, 0xee, 0x20, 0xfa, 0xb5, 0xa2, - 0xbb, 0x9b, 0xea, 0x09, 0x96, 0x7f, 0x04, 0x57, 0x02, 0xee, 0x0a, 0x9d, 0x64, 0x92, 0x29, 0xa4, - 0x59, 0x99, 0x62, 0xb2, 0xef, 0xff, 0xcb, 0x12, 0x2c, 0xef, 0xab, 0xa7, 0xf8, 0xd5, 0x87, 0x19, - 0xb9, 0xd4, 0xdf, 0x36, 0x2c, 0xd1, 0xc8, 0x62, 0x35, 0xb4, 0x9e, 0x09, 0xf6, 0x37, 0xcc, 0x0a, - 0xc9, 0x8e, 0xe8, 0xdb, 0xc4, 0x71, 0xc0, 0x83, 0x93, 0x3d, 0x5b, 0x0f, 0x6c, 0xef, 0x8d, 0x94, - 0x71, 0xd6, 0x05, 0x96, 0x12, 0xee, 0x81, 0xf6, 0x60, 0x31, 0x7a, 0x0c, 0xbe, 0xd5, 0x7d, 0x67, - 0x62, 0xb0, 0x1b, 0x50, 0x5f, 0x69, 0x45, 0x0e, 0xc3, 0x45, 0x1d, 0x58, 0xe0, 0x26, 0x93, 0xea, - 0x8c, 0xaa, 0xe2, 0x37, 0xd1, 0x1e, 0x5c, 0x66, 0x3b, 0xd8, 0xe7, 0x02, 0xc1, 0x36, 0x5f, 0xcd, - 0xb5, 0xf9, 0xb4, 0xae, 0x51, 0x79, 0xaa, 0x5d, 0x54, 0x9e, 0x3a, 0xb0, 0xc0, 0x79, 0x9c, 0xea, - 0x91, 0xaa, 0xe2, 0x37, 0xc9, 0x31, 0x07, 0xdc, 0x5e, 0xa7, 0xdf, 0x02, 0x00, 0x09, 0xd1, 0x20, - 0xa0, 0xe7, 0x94, 0xb4, 0xcc, 0xa7, 0x50, 0x15, 0x1c, 0x5e, 0xc8, 0xcd, 0xe1, 0xa2, 0x4f, 0x5c, - 0xbf, 0x17, 0x63, 0xfa, 0x5d, 0xfe, 0x27, 0x09, 0x1a, 0x1b, 0x64, 0x4b, 0xdb, 0xf6, 0x90, 0x5a, - 0xa3, 0xbb, 0xd0, 0x72, 0xf0, 0xc0, 0x76, 0xb4, 0x3e, 0xb6, 0x3c, 0x47, 0xc7, 0x2c, 0x9a, 0x2f, - 0x29, 0x4d, 0x06, 0xfd, 0x8c, 0x01, 0x09, 0x1a, 0x51, 0xd9, 0xae, 0xa7, 0x9a, 0xa3, 0xfe, 0x11, - 0x51, 0x0d, 0x05, 0x86, 0x26, 0xa0, 0x54, 0x33, 0xbc, 0x09, 0x8d, 0x00, 0xcd, 0xb3, 0xe9, 0xfc, - 0x25, 0xa5, 0x2e, 0x60, 0x07, 0x36, 0x7a, 0x0b, 0x5a, 0x94, 0xa6, 0x7d, 0xc3, 0x1e, 0xf6, 0x49, - 0xe4, 0xcb, 0x0d, 0x55, 0x43, 0xe3, 0xcb, 0x22, 0x67, 0x15, 0xc5, 0x72, 0xf5, 0x1f, 0x62, 0x6e, - 0xaa, 0x04, 0xd6, 0xbe, 0xfe, 0x43, 0x2c, 0xff, 0xa3, 0x04, 0xcd, 0x0d, 0xd5, 0x53, 0x77, 0x6d, - 0x0d, 0x1f, 0xcc, 0x68, 0xd8, 0x73, 0xa4, 0x48, 0xaf, 0x43, 0x4d, 0xec, 0x80, 0x6f, 0x29, 0x00, - 0xa0, 0x4d, 0x68, 0xf9, 0x2e, 0x68, 0x9f, 0x45, 0x66, 0xa5, 0x4c, 0x47, 0x2b, 0x64, 0x39, 0x5d, - 0xa5, 0xe9, 0x77, 0xa3, 0x4d, 0x79, 0x13, 0x1a, 0xe1, 0xcf, 0x64, 0xd6, 0xfd, 0x38, 0xa3, 0x08, - 0x00, 0xe1, 0xc6, 0xdd, 0xb1, 0x49, 0xce, 0x94, 0x2b, 0x16, 0xbf, 0x29, 0xff, 0x58, 0x82, 0x26, - 0x37, 0xf7, 0xfb, 0xe2, 0x32, 0x81, 0x6e, 0x4d, 0xa2, 0x5b, 0xa3, 0xbf, 0xd1, 0xaf, 0x45, 0xf3, - 0x7f, 0x6f, 0xa5, 0x2a, 0x01, 0x3a, 0x08, 0x75, 0x46, 0x23, 0xb6, 0x3e, 0x4f, 0x2e, 0xe0, 0x4b, - 0xc2, 0x68, 0xfc, 0x68, 0x28, 0xa3, 0x75, 0x60, 0x41, 0xd5, 0x34, 0x07, 0xbb, 0x2e, 0x5f, 0x87, - 0xdf, 0x24, 0x5f, 0x4e, 0xb1, 0xe3, 0xfa, 0x2c, 0x5f, 0x54, 0xfc, 0x26, 0xfa, 0x16, 0x54, 0x85, - 0xf7, 0xca, 0xd2, 0xe6, 0xb7, 0xb3, 0xd7, 0xc9, 0x23, 0x57, 0xd1, 0x43, 0xfe, 0xbb, 0x02, 0xb4, - 0x38, 0xc1, 0xd6, 0xb8, 0x3d, 0x9e, 0x2c, 0x7c, 0x6b, 0xd0, 0x38, 0x0a, 0x64, 0x7f, 0x52, 0x8e, - 0x2a, 0xac, 0x22, 0x22, 0x7d, 0xa6, 0x09, 0x60, 0xd4, 0x23, 0x28, 0xcd, 0xe5, 0x11, 0x94, 0x2f, - 0xaa, 0xc1, 0x92, 0x3e, 0x62, 0x25, 0xc5, 0x47, 0x94, 0x7f, 0x13, 0xea, 0xa1, 0x01, 0xa8, 0x86, - 0x66, 0xc9, 0x2d, 0x4e, 0x31, 0xbf, 0x89, 0x3e, 0x0c, 0xfc, 0x22, 0x46, 0xaa, 0xab, 0x29, 0x6b, - 0x89, 0xb9, 0x44, 0xf2, 0x3f, 0x48, 0x50, 0xe1, 0x23, 0xdf, 0x82, 0x3a, 0x57, 0x3a, 0xd4, 0x67, - 0x64, 0xa3, 0x03, 0x07, 0x11, 0xa7, 0xf1, 0xe5, 0x69, 0x9d, 0xab, 0x50, 0x8d, 0xe9, 0x9b, 0x05, - 0x6e, 0x16, 0xfc, 0x4f, 0x21, 0x25, 0x43, 0x3e, 0x11, 0xfd, 0x82, 0xae, 0x40, 0xd9, 0xb0, 0x87, - 0xe2, 0xb2, 0x88, 0x35, 0x48, 0x54, 0xb4, 0xb2, 0x85, 0x3d, 0x05, 0x0f, 0xec, 0x53, 0xec, 0x9c, - 0xcf, 0x9f, 0x14, 0x7d, 0x1c, 0x62, 0xf3, 0x9c, 0x41, 0x9a, 0xe8, 0x80, 0x1e, 0x07, 0x87, 0x50, - 0x4c, 0xcb, 0x08, 0x85, 0xf5, 0x0e, 0x67, 0xd2, 0xe0, 0x30, 0xfe, 0x80, 0xa5, 0x77, 0xa3, 0x5b, - 0x99, 0xd5, 0xdb, 0x79, 0x29, 0x81, 0x8c, 0xfc, 0x4b, 0x09, 0xba, 0x41, 0xca, 0xc9, 0x5d, 0x3b, - 0x9f, 0xf7, 0xf2, 0xe4, 0xe5, 0xc4, 0x57, 0xbf, 0x2a, 0xb2, 0xfb, 0x44, 0x68, 0x73, 0x45, 0x46, - 0x7e, 0x6e, 0xdf, 0xa2, 0xd9, 0xeb, 0xe4, 0x86, 0xe6, 0x61, 0x99, 0x2e, 0x54, 0x45, 0xde, 0x83, - 0x65, 0xf8, 0x45, 0x9b, 0x48, 0xd8, 0xd5, 0x2d, 0xec, 0x6d, 0x46, 0x53, 0x26, 0xaf, 0x9b, 0x80, - 0xe1, 0x5b, 0x87, 0x63, 0x7e, 0xeb, 0x50, 0x8a, 0xdd, 0x3a, 0x70, 0xb8, 0x6c, 0x52, 0x16, 0x48, - 0x6c, 0xe0, 0x55, 0x11, 0xec, 0x77, 0x24, 0xe8, 0xf0, 0x59, 0xe8, 0x9c, 0x24, 0x24, 0x32, 0xb0, - 0x87, 0xb5, 0xaf, 0x3b, 0x55, 0xf0, 0xdf, 0x12, 0xb4, 0xc3, 0x56, 0x97, 0x1a, 0xce, 0x8f, 0xa0, - 0x4c, 0x33, 0x32, 0x7c, 0x05, 0x53, 0x55, 0x03, 0xc3, 0x26, 0x6a, 0x9b, 0xba, 0xda, 0x07, 0xc2, - 0x41, 0xe0, 0xcd, 0xc0, 0xf4, 0x17, 0x2f, 0x6e, 0xfa, 0xb9, 0x2b, 0x64, 0x8f, 0xc9, 0xb8, 0x2c, - 0x95, 0x19, 0x00, 0xd0, 0x27, 0x50, 0x61, 0x05, 0x1b, 0xfc, 0x26, 0xee, 0x6e, 0x74, 0x68, 0x5e, - 0xcc, 0x11, 0xba, 0x1f, 0xa0, 0x00, 0x85, 0x77, 0x92, 0x7f, 0x1d, 0x96, 0x83, 0x68, 0x94, 0x4d, - 0x3b, 0x2b, 0xd3, 0xca, 0xff, 0x2a, 0xc1, 0xe5, 0xfd, 0x73, 0x6b, 0x10, 0x67, 0xff, 0x65, 0xa8, - 0x8c, 0x0c, 0x35, 0xc8, 0xac, 0xf2, 0x16, 0x75, 0x03, 0xd9, 0xdc, 0x58, 0x23, 0x36, 0x84, 0xd1, - 0xac, 0x2e, 0x60, 0x07, 0xf6, 0x54, 0xd3, 0x7e, 0x57, 0x84, 0xcf, 0x58, 0x63, 0xd6, 0x8a, 0xa5, - 0xab, 0x9a, 0x02, 0x4a, 0xad, 0xd5, 0x27, 0x00, 0xd4, 0xa0, 0xf7, 0x2f, 0x62, 0xc4, 0x69, 0x8f, - 0x6d, 0xa2, 0xb2, 0x7f, 0x51, 0x80, 0x4e, 0x88, 0x4a, 0x5f, 0xb7, 0x7f, 0x93, 0x11, 0x95, 0x15, - 0x5f, 0x52, 0x54, 0x56, 0x9a, 0xdf, 0xa7, 0x29, 0xa7, 0xf9, 0x34, 0xff, 0x56, 0x80, 0x56, 0x40, - 0xb5, 0x3d, 0x43, 0xb5, 0x32, 0x39, 0x61, 0x5f, 0xf8, 0xf3, 0x51, 0x3a, 0x7d, 0x23, 0x4d, 0x4e, - 0x32, 0x0e, 0x42, 0x89, 0x0d, 0x81, 0x6e, 0xd0, 0x43, 0x77, 0x3c, 0x96, 0xf8, 0xe2, 0x31, 0x04, - 0x13, 0x48, 0xdd, 0xc4, 0xe8, 0x01, 0x20, 0x2e, 0x45, 0x7d, 0xdd, 0xea, 0xbb, 0x78, 0x60, 0x5b, - 0x1a, 0x93, 0xaf, 0xb2, 0xd2, 0xe6, 0x5f, 0x7a, 0xd6, 0x3e, 0x83, 0xa3, 0x8f, 0xa0, 0xe4, 0x9d, - 0x8f, 0x98, 0xb7, 0xd2, 0x4a, 0xb5, 0xf7, 0xc1, 0xba, 0x0e, 0xce, 0x47, 0x58, 0xa1, 0xe8, 0x7e, - 0x45, 0x8f, 0xe7, 0xa8, 0xa7, 0xdc, 0xf5, 0x2b, 0x29, 0x21, 0x08, 0xd1, 0x18, 0x3e, 0x0d, 0x17, - 0x98, 0x8b, 0xc4, 0x9b, 0x8c, 0xb3, 0x7d, 0xa1, 0xed, 0x7b, 0x9e, 0x41, 0x53, 0x77, 0x94, 0xb3, - 0x7d, 0xe8, 0x81, 0x67, 0xc8, 0xff, 0x52, 0x80, 0x76, 0x30, 0xb3, 0x82, 0xdd, 0xb1, 0x91, 0x2d, - 0x70, 0x93, 0x73, 0x23, 0xd3, 0x64, 0xed, 0xdb, 0x50, 0xe7, 0xc7, 0x7e, 0x01, 0xb6, 0x01, 0xd6, - 0x65, 0x7b, 0x02, 0x1f, 0x97, 0x5f, 0x12, 0x1f, 0x57, 0x66, 0xc8, 0x2e, 0xa4, 0x13, 0x5f, 0xfe, - 0x99, 0x04, 0x6f, 0x24, 0xd4, 0xe2, 0x44, 0xd2, 0x4e, 0x8e, 0xed, 0xb8, 0xba, 0x8c, 0x0f, 0xc9, - 0x15, 0xfc, 0x63, 0xa8, 0x38, 0x74, 0x74, 0x7e, 0x65, 0x74, 0x67, 0x22, 0x77, 0xb1, 0x85, 0x28, - 0xbc, 0x8b, 0xfc, 0x87, 0x12, 0xac, 0x24, 0x97, 0x3a, 0x87, 0xd5, 0x5e, 0x83, 0x05, 0x36, 0xb4, - 0x2f, 0x84, 0xf7, 0x26, 0x0b, 0x61, 0x40, 0x1c, 0xc5, 0xef, 0x28, 0xef, 0xc3, 0xb2, 0x6f, 0xdc, - 0x03, 0xd2, 0xef, 0x60, 0x4f, 0x9d, 0x10, 0xd9, 0xdc, 0x82, 0x3a, 0x73, 0x91, 0x59, 0xc4, 0xc0, - 0x72, 0x02, 0x70, 0x28, 0x52, 0x69, 0xf2, 0x7f, 0x4a, 0x70, 0x85, 0x5a, 0xc7, 0xf8, 0x1d, 0x4d, - 0x9e, 0xfb, 0x3b, 0x59, 0xa4, 0x1c, 0x76, 0x55, 0x93, 0xd7, 0x8b, 0xd4, 0x94, 0x08, 0x0c, 0xf5, - 0x92, 0x99, 0xb6, 0xd4, 0x08, 0x38, 0xb8, 0xf0, 0x25, 0xd1, 0x36, 0xbd, 0xef, 0x8d, 0xa7, 0xd8, - 0x02, 0xab, 0x5c, 0x9a, 0xc5, 0x2a, 0x6f, 0xc3, 0x1b, 0xb1, 0x9d, 0xce, 0x71, 0xa2, 0xf2, 0x5f, - 0x49, 0xe4, 0x38, 0x22, 0x75, 0x37, 0xb3, 0x7b, 0xa6, 0x37, 0xc4, 0xe5, 0x50, 0x5f, 0xd7, 0xe2, - 0x4a, 0x44, 0x43, 0x9f, 0x42, 0xcd, 0xc2, 0x67, 0xfd, 0xb0, 0xb3, 0x93, 0xc3, 0x6d, 0xaf, 0x5a, - 0xf8, 0x8c, 0xfe, 0x92, 0x77, 0x61, 0x25, 0xb1, 0xd4, 0x79, 0xf6, 0xfe, 0xf7, 0x12, 0x5c, 0xdd, - 0x70, 0xec, 0xd1, 0x17, 0xba, 0xe3, 0x8d, 0x55, 0x23, 0x7a, 0x95, 0xfe, 0x6a, 0x52, 0x57, 0x9f, - 0x87, 0xdc, 0x5e, 0xc6, 0x3f, 0x0f, 0x52, 0x24, 0x28, 0xb9, 0x28, 0xbe, 0xe9, 0x90, 0x93, 0xfc, - 0x1f, 0xc5, 0xb4, 0xc5, 0x73, 0xbc, 0x29, 0x8e, 0x47, 0x9e, 0x08, 0x22, 0x35, 0xd3, 0x5d, 0x9c, - 0x35, 0xd3, 0x9d, 0xa1, 0xde, 0x4b, 0x2f, 0x49, 0xbd, 0x5f, 0x38, 0xf5, 0xf2, 0x39, 0x44, 0x6f, - 0x21, 0xa8, 0xf9, 0x9d, 0xe9, 0xfa, 0x62, 0x0d, 0x20, 0xc8, 0xc8, 0xf3, 0xb2, 0xc9, 0x3c, 0xc3, - 0x84, 0x7a, 0x91, 0xd3, 0x12, 0xa6, 0x94, 0x9b, 0xf2, 0x50, 0x8e, 0xf8, 0xbb, 0xd0, 0x4d, 0xe3, - 0xd2, 0x79, 0x38, 0xff, 0x17, 0x05, 0x80, 0x9e, 0xa8, 0xb4, 0x9d, 0xcd, 0x16, 0xdc, 0x81, 0x90, - 0xbb, 0x11, 0xc8, 0x7b, 0x98, 0x8b, 0x34, 0x22, 0x12, 0x22, 0xe8, 0x24, 0x38, 0x89, 0x40, 0x54, - 0xa3, 0xe3, 0x84, 0xa4, 0x86, 0x31, 0x45, 0x5c, 0xfd, 0x5e, 0x83, 0x9a, 0x63, 0x9f, 0xf5, 0x89, - 0x98, 0x69, 0x7e, 0x29, 0xb1, 0x63, 0x9f, 0x11, 0xe1, 0xd3, 0xd0, 0x0a, 0x2c, 0x78, 0xaa, 0x7b, - 0x42, 0xc6, 0xaf, 0x84, 0xaa, 0x39, 0x34, 0x74, 0x05, 0xca, 0x47, 0xba, 0x81, 0x59, 0xf1, 0x40, - 0x4d, 0x61, 0x0d, 0xf4, 0x4d, 0xbf, 0xe6, 0xad, 0x9a, 0xbb, 0x62, 0x87, 0x95, 0xbd, 0x7d, 0x25, - 0xc1, 0x62, 0x40, 0x35, 0xaa, 0x80, 0x88, 0x4e, 0xa3, 0xfa, 0x6c, 0xdd, 0xd6, 0x98, 0xaa, 0x68, - 0x65, 0x58, 0x04, 0xd6, 0x91, 0x69, 0xad, 0xa0, 0xcb, 0xa4, 0x38, 0x98, 0xec, 0x8b, 0x6c, 0x5a, - 0xd7, 0xfc, 0x0a, 0x96, 0x8a, 0x63, 0x9f, 0xf5, 0x34, 0x41, 0x0d, 0x56, 0x27, 0xcc, 0xa2, 0x3e, - 0x42, 0x8d, 0x75, 0x5a, 0x2a, 0x7c, 0x07, 0x9a, 0xd8, 0x71, 0x6c, 0xa7, 0x6f, 0x62, 0xd7, 0x55, - 0x87, 0x98, 0x3b, 0xe0, 0x0d, 0x0a, 0xdc, 0x61, 0x30, 0xf9, 0x8f, 0x4b, 0xd0, 0x0a, 0xb6, 0xe2, - 0xdf, 0x83, 0xeb, 0x9a, 0x7f, 0x0f, 0xae, 0x93, 0xa3, 0x03, 0x87, 0xa9, 0x42, 0x71, 0xb8, 0x6b, - 0x85, 0x8e, 0xa4, 0xd4, 0x38, 0xb4, 0xa7, 0x11, 0xb3, 0x4c, 0x84, 0xcc, 0xb2, 0x35, 0x1c, 0x1c, - 0x2e, 0xf8, 0x20, 0x7e, 0xb6, 0x11, 0x1e, 0x29, 0xe5, 0xe0, 0x91, 0x72, 0x0e, 0x1e, 0xa9, 0xa4, - 0xf0, 0xc8, 0x32, 0x54, 0x0e, 0xc7, 0x83, 0x13, 0xec, 0x71, 0x8f, 0x8d, 0xb7, 0xa2, 0xbc, 0x53, - 0x8d, 0xf1, 0x8e, 0x60, 0x91, 0x5a, 0x98, 0x45, 0xae, 0x41, 0x8d, 0x5d, 0xc8, 0xf6, 0x3d, 0x97, - 0xde, 0x2e, 0x15, 0x95, 0x2a, 0x03, 0x1c, 0xb8, 0xe8, 0x63, 0xdf, 0x9d, 0xab, 0xa7, 0x09, 0x3b, - 0xd5, 0x3a, 0x31, 0x2e, 0xf1, 0x9d, 0xb9, 0x77, 0x60, 0x31, 0x44, 0x0e, 0x6a, 0x23, 0x1a, 0x74, - 0xa9, 0x21, 0x77, 0x9e, 0x9a, 0x89, 0xbb, 0xd0, 0x0a, 0x48, 0x42, 0xf1, 0x9a, 0x2c, 0x8a, 0x12, - 0x50, 0x8a, 0x26, 0x38, 0xb9, 0x75, 0x31, 0x4e, 0x46, 0x57, 0xa1, 0xca, 0xc3, 0x1f, 0xb7, 0xb3, - 0x18, 0xc9, 0x46, 0xc8, 0x3f, 0x00, 0x14, 0xac, 0x7e, 0x3e, 0x6f, 0x31, 0xc6, 0x1e, 0x85, 0x38, - 0x7b, 0xc8, 0x3f, 0x97, 0x60, 0x29, 0x3c, 0xd9, 0xac, 0x86, 0xf7, 0x53, 0xa8, 0xb3, 0xfb, 0xbd, - 0x3e, 0x11, 0x7c, 0x9e, 0xe5, 0xb9, 0x31, 0xf1, 0x5c, 0x14, 0x08, 0x5e, 0x1a, 0x10, 0xf6, 0x3a, - 0xb3, 0x9d, 0x13, 0xdd, 0x1a, 0xf6, 0xc9, 0xca, 0x7c, 0x71, 0x6b, 0x70, 0xe0, 0x2e, 0x81, 0xc9, - 0x3f, 0x91, 0xe0, 0xe6, 0xd3, 0x91, 0xa6, 0x7a, 0x38, 0xe4, 0x81, 0xcc, 0x5b, 0xbc, 0xf8, 0x91, - 0x5f, 0x3d, 0x58, 0xc8, 0x77, 0x47, 0xc5, 0xb0, 0xe5, 0x1d, 0xb8, 0xaa, 0x60, 0x17, 0x5b, 0x5a, - 0xe4, 0xe3, 0xcc, 0xc9, 0x99, 0x11, 0x74, 0xd3, 0x86, 0x9b, 0xe7, 0xec, 0x99, 0x2b, 0xd8, 0x77, - 0xc8, 0xb0, 0x1e, 0xd7, 0x6c, 0xc4, 0x03, 0xa1, 0xf3, 0x78, 0xf2, 0x5f, 0x17, 0x60, 0xe5, 0x89, - 0xa6, 0x71, 0xa5, 0xc8, 0x9d, 0x9b, 0x57, 0xe5, 0x77, 0xc6, 0xfd, 0xb2, 0x62, 0xd2, 0x2f, 0x7b, - 0x59, 0x8a, 0x8a, 0xab, 0x6c, 0x6b, 0x6c, 0xfa, 0xa6, 0xc8, 0x61, 0xf5, 0x36, 0x8f, 0xf9, 0x3d, - 0x13, 0x89, 0x8f, 0xa9, 0x39, 0x9a, 0xee, 0xae, 0x54, 0xfd, 0x24, 0x93, 0x3c, 0x82, 0x4e, 0x92, - 0x58, 0x73, 0x4a, 0xa6, 0x4f, 0x91, 0x91, 0xcd, 0x12, 0x92, 0x0d, 0xe2, 0x91, 0x50, 0xd0, 0x9e, - 0xed, 0xca, 0xff, 0x55, 0x80, 0xce, 0xbe, 0x7a, 0x8a, 0xff, 0xff, 0x1c, 0xd0, 0xf7, 0xe0, 0x8a, - 0xab, 0x9e, 0xe2, 0x7e, 0x28, 0xce, 0xec, 0x3b, 0xf8, 0x39, 0xf7, 0xe8, 0xde, 0x4d, 0x13, 0xcc, - 0xd4, 0xb2, 0x14, 0x65, 0xc9, 0x8d, 0xc0, 0x15, 0xfc, 0x1c, 0xbd, 0x0d, 0x8b, 0xe1, 0xba, 0x27, - 0xb2, 0xb4, 0x2a, 0x25, 0x79, 0x33, 0x54, 0xd6, 0xd4, 0xd3, 0xe4, 0xe7, 0x70, 0xfd, 0xa9, 0xe5, - 0x62, 0xaf, 0x17, 0x94, 0xe6, 0xcc, 0x19, 0x91, 0xdd, 0x82, 0x7a, 0x40, 0xf8, 0xc4, 0xe3, 0x03, - 0xcd, 0x95, 0x6d, 0xe8, 0xee, 0x04, 0x95, 0x79, 0xee, 0x06, 0x2b, 0xa1, 0x78, 0x85, 0x13, 0x1e, - 0x89, 0x8a, 0x22, 0x05, 0x1f, 0x61, 0x07, 0x5b, 0x03, 0xbc, 0x6d, 0x0f, 0x4e, 0x42, 0x15, 0xb9, - 0x52, 0xb8, 0x22, 0x77, 0xd6, 0x0a, 0xdf, 0xfb, 0x9f, 0x8a, 0x2a, 0xbf, 0x83, 0xf3, 0x11, 0x46, - 0x0b, 0x50, 0xdc, 0xc5, 0x67, 0xed, 0x4b, 0x08, 0xa0, 0xb2, 0x6b, 0x3b, 0xa6, 0x6a, 0xb4, 0x25, - 0x54, 0x87, 0x05, 0x7e, 0xa9, 0xd1, 0x2e, 0xa0, 0x26, 0xd4, 0xd6, 0xfd, 0xc4, 0x70, 0xbb, 0x78, - 0xff, 0x4f, 0x25, 0x58, 0x4a, 0xa4, 0xdd, 0x51, 0x0b, 0xe0, 0xa9, 0x35, 0xe0, 0xf7, 0x11, 0xed, - 0x4b, 0xa8, 0x01, 0x55, 0xff, 0x76, 0x82, 0x8d, 0x77, 0x60, 0x53, 0xec, 0x76, 0x01, 0xb5, 0xa1, - 0xc1, 0x3a, 0x8e, 0x07, 0x03, 0xec, 0xba, 0xed, 0xa2, 0x80, 0x6c, 0xaa, 0xba, 0x31, 0x76, 0x70, - 0xbb, 0x44, 0xe6, 0x3c, 0xb0, 0x79, 0x3d, 0x74, 0xbb, 0x8c, 0x10, 0xb4, 0xfc, 0xe2, 0x68, 0xde, - 0xa9, 0x12, 0x82, 0xf9, 0xdd, 0x16, 0xee, 0x3f, 0x0b, 0x27, 0x4f, 0xe9, 0xf6, 0x56, 0xe0, 0xf2, - 0x53, 0x4b, 0xc3, 0x47, 0xba, 0x85, 0xb5, 0xe0, 0x53, 0xfb, 0x12, 0xba, 0x0c, 0x8b, 0x3b, 0xd8, - 0x19, 0xe2, 0x10, 0xb0, 0x80, 0x96, 0xa0, 0xb9, 0xa3, 0xbf, 0x08, 0x81, 0x8a, 0x72, 0xa9, 0x2a, - 0xb5, 0xa5, 0xd5, 0x3f, 0xbb, 0x0e, 0xb5, 0x0d, 0xd5, 0x53, 0xd7, 0x6d, 0xdb, 0xd1, 0x90, 0x01, - 0x88, 0x3e, 0x1f, 0x30, 0x47, 0xb6, 0x25, 0x1e, 0xe5, 0xa0, 0x87, 0x51, 0x2e, 0xe0, 0x8d, 0x24, - 0x22, 0xe7, 0xa1, 0xee, 0x5b, 0xa9, 0xf8, 0x31, 0x64, 0xf9, 0x12, 0x32, 0xe9, 0x6c, 0x07, 0xba, - 0x89, 0x0f, 0xf4, 0xc1, 0x89, 0x5f, 0x20, 0xf9, 0x7e, 0x46, 0x28, 0x95, 0x44, 0xf5, 0xe7, 0xbb, - 0x93, 0x3a, 0x1f, 0x7b, 0xdf, 0xe1, 0x6b, 0x4d, 0xf9, 0x12, 0x7a, 0x0e, 0x57, 0xb6, 0x70, 0xc8, - 0x86, 0xfb, 0x13, 0xae, 0x66, 0x4f, 0x98, 0x40, 0xbe, 0xe0, 0x94, 0xdb, 0x50, 0xa6, 0xec, 0x86, - 0xd2, 0xcc, 0x7c, 0xf8, 0xfd, 0x6c, 0xf7, 0x76, 0x36, 0x82, 0x18, 0xed, 0x07, 0xb0, 0x18, 0x7b, - 0x75, 0x87, 0xd2, 0xb4, 0x54, 0xfa, 0xfb, 0xc9, 0xee, 0xfd, 0x3c, 0xa8, 0x62, 0xae, 0x21, 0xb4, - 0xa2, 0xcf, 0x0e, 0x50, 0x5a, 0xe2, 0x2f, 0xf5, 0xc1, 0x54, 0xf7, 0xdd, 0x1c, 0x98, 0x62, 0x22, - 0x13, 0xda, 0xf1, 0x57, 0x60, 0xe8, 0xfe, 0xc4, 0x01, 0xa2, 0xcc, 0xf6, 0x8d, 0x5c, 0xb8, 0x62, - 0xba, 0x73, 0xca, 0x04, 0x89, 0x87, 0x45, 0x71, 0x1e, 0xf7, 0x87, 0xc9, 0x7a, 0xf1, 0xd4, 0x7d, - 0x94, 0x1b, 0x5f, 0x4c, 0xfd, 0xdb, 0xac, 0x6a, 0x21, 0xed, 0x71, 0x0e, 0xfa, 0x20, 0x7d, 0xb8, - 0x09, 0xaf, 0x8a, 0xba, 0xab, 0x17, 0xe9, 0x22, 0x16, 0xf1, 0x23, 0x5a, 0x6e, 0x90, 0xf2, 0xbc, - 0x25, 0x2e, 0x77, 0xfe, 0x78, 0xd9, 0x2f, 0x77, 0xba, 0x1f, 0x5c, 0xa0, 0x87, 0x58, 0x80, 0x1d, - 0x7f, 0x66, 0xe7, 0x8b, 0xe1, 0xa3, 0xa9, 0x5c, 0x33, 0x9b, 0x0c, 0x7e, 0x1f, 0x16, 0x63, 0x76, - 0x1b, 0xe5, 0xb7, 0xed, 0xdd, 0x49, 0xce, 0x15, 0x13, 0xc9, 0x58, 0xf5, 0x06, 0xca, 0xe0, 0xfe, - 0x94, 0x0a, 0x8f, 0xee, 0xfd, 0x3c, 0xa8, 0x62, 0x23, 0x2e, 0x55, 0x97, 0xb1, 0x3b, 0x79, 0xf4, - 0x20, 0x7d, 0x8c, 0xf4, 0xda, 0x83, 0xee, 0x7b, 0x39, 0xb1, 0xc5, 0xa4, 0xa7, 0x70, 0x39, 0xa5, - 0x74, 0x02, 0xbd, 0x37, 0xf1, 0xb0, 0xe2, 0x35, 0x23, 0xdd, 0x87, 0x79, 0xd1, 0xc5, 0xbc, 0xbf, - 0x05, 0x68, 0xff, 0xd8, 0x3e, 0x5b, 0xb7, 0xad, 0x23, 0x7d, 0x38, 0x76, 0x54, 0x96, 0x48, 0xcf, - 0xb2, 0x0d, 0x49, 0xd4, 0x0c, 0x1e, 0x9d, 0xd8, 0x43, 0x4c, 0xde, 0x07, 0xd8, 0xc2, 0xde, 0x0e, - 0xf6, 0x1c, 0x22, 0x18, 0x6f, 0x67, 0x99, 0x3f, 0x8e, 0xe0, 0x4f, 0xf5, 0xce, 0x54, 0xbc, 0x90, - 0x29, 0x6a, 0xef, 0xa8, 0xd6, 0x58, 0x35, 0x42, 0xb5, 0xdf, 0x0f, 0x52, 0xbb, 0xc7, 0xd1, 0x32, - 0x0e, 0x32, 0x13, 0x5b, 0x4c, 0x79, 0x26, 0x4c, 0x7b, 0xe8, 0xa6, 0x66, 0xb2, 0x69, 0x4f, 0x96, - 0x01, 0xc4, 0xd5, 0xde, 0x04, 0x7c, 0x31, 0xf1, 0x97, 0x12, 0xad, 0xbe, 0x89, 0x21, 0x3c, 0xd3, - 0xbd, 0xe3, 0x3d, 0x43, 0xb5, 0xdc, 0x3c, 0x4b, 0xa0, 0x88, 0x17, 0x58, 0x02, 0xc7, 0x17, 0x4b, - 0xd0, 0xa0, 0x19, 0xb9, 0x40, 0x41, 0x69, 0xc5, 0xd2, 0x69, 0x97, 0x49, 0xdd, 0x7b, 0xd3, 0x11, - 0xc5, 0x2c, 0xc7, 0xd0, 0xf4, 0x45, 0x89, 0x11, 0xf7, 0xdd, 0xac, 0x95, 0x06, 0x38, 0x19, 0x9a, - 0x20, 0x1d, 0x35, 0xac, 0x09, 0x92, 0xf9, 0x61, 0x94, 0xef, 0x5e, 0x61, 0x92, 0x26, 0xc8, 0x4e, - 0x3a, 0x33, 0x55, 0x17, 0xbb, 0x8b, 0x49, 0xd7, 0xa3, 0xa9, 0x57, 0x4b, 0xa9, 0xaa, 0x2e, 0xe3, - 0x6a, 0x47, 0xbe, 0x84, 0x9e, 0x41, 0x85, 0xff, 0x69, 0xc4, 0x5b, 0x93, 0x73, 0x3a, 0x7c, 0xf4, - 0xbb, 0x53, 0xb0, 0xc4, 0xc0, 0x27, 0xb0, 0x92, 0x91, 0xd1, 0x49, 0x35, 0xc1, 0x93, 0xb3, 0x3f, - 0xd3, 0x8c, 0x83, 0x0a, 0x28, 0xf9, 0x32, 0x33, 0xf5, 0x98, 0x32, 0x1f, 0x70, 0xe6, 0x98, 0x22, - 0xf9, 0xb8, 0x32, 0x75, 0x8a, 0xcc, 0x37, 0x98, 0xd3, 0xa6, 0xe8, 0xc3, 0x52, 0x22, 0x2f, 0x80, - 0xbe, 0x91, 0x61, 0x41, 0xd3, 0xb2, 0x07, 0xd3, 0x26, 0x18, 0xc2, 0x1b, 0xa9, 0x31, 0x70, 0xaa, - 0x47, 0x30, 0x29, 0x5a, 0x9e, 0x36, 0xd1, 0x00, 0x2e, 0xa7, 0x44, 0xbe, 0xa9, 0xb6, 0x2c, 0x3b, - 0x42, 0x9e, 0x36, 0xc9, 0x31, 0x74, 0xd7, 0x1c, 0x5b, 0xd5, 0x06, 0xaa, 0xeb, 0x3d, 0x31, 0x3c, - 0xec, 0x90, 0xf0, 0xcc, 0x77, 0xc9, 0xe2, 0x74, 0xe3, 0x0d, 0x8a, 0x17, 0x60, 0xe5, 0x9c, 0xe9, - 0x10, 0xea, 0xf4, 0x55, 0x03, 0xfb, 0xa7, 0x00, 0x94, 0x6e, 0x7e, 0x42, 0x18, 0x19, 0x3a, 0x2d, - 0x0d, 0xd1, 0x97, 0x97, 0xd5, 0xaf, 0x6a, 0x50, 0xf5, 0x8b, 0xc8, 0xbf, 0xe6, 0xe8, 0xf0, 0x35, - 0x84, 0x6b, 0xdf, 0x87, 0xc5, 0xd8, 0xc3, 0xcf, 0x54, 0x15, 0x97, 0xfe, 0x38, 0x74, 0xda, 0x71, - 0x3d, 0xe3, 0x7f, 0x4b, 0x24, 0x3c, 0xb7, 0x77, 0xb2, 0x42, 0xbe, 0xb8, 0xd3, 0x36, 0x65, 0xe0, - 0xff, 0xdb, 0xae, 0xd2, 0x2e, 0x40, 0xc8, 0x49, 0x9a, 0x5c, 0x6a, 0x45, 0xec, 0xfe, 0x34, 0x6a, - 0x99, 0xa9, 0x7e, 0xd0, 0xbb, 0x79, 0xaa, 0x5a, 0xb2, 0x2d, 0x59, 0xb6, 0xf7, 0xf3, 0x14, 0x1a, - 0xe1, 0x22, 0x48, 0x94, 0xfa, 0x27, 0x38, 0xc9, 0x2a, 0xc9, 0x69, 0xbb, 0xd8, 0xb9, 0xa0, 0x81, - 0x9c, 0x32, 0x9c, 0x4b, 0xcc, 0x48, 0xfc, 0x3a, 0x20, 0xc3, 0x8c, 0x64, 0x5c, 0x42, 0xa4, 0x3a, - 0x14, 0xd9, 0x77, 0x0c, 0x2c, 0xf2, 0x8f, 0xe7, 0xb8, 0x53, 0x23, 0xff, 0x8c, 0x5b, 0x83, 0xd4, - 0xc8, 0x3f, 0x2b, 0x69, 0x2e, 0x5f, 0x5a, 0xfb, 0xf0, 0x7b, 0x1f, 0x0c, 0x75, 0xef, 0x78, 0x7c, - 0x48, 0x76, 0xff, 0x88, 0x75, 0x7d, 0x4f, 0xb7, 0xf9, 0xaf, 0x47, 0x3e, 0xbb, 0x3f, 0xa2, 0xa3, - 0x3d, 0x22, 0xa3, 0x8d, 0x0e, 0x0f, 0x2b, 0xb4, 0xf5, 0xe1, 0xff, 0x04, 0x00, 0x00, 0xff, 0xff, - 0x51, 0x6c, 0x4e, 0xfb, 0x58, 0x4d, 0x00, 0x00, + // 4364 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe4, 0x3c, 0x4b, 0x6f, 0x24, 0x49, + 0x5a, 0x9d, 0xf5, 0x72, 0xd5, 0x57, 0x0f, 0x97, 0xa3, 0x7b, 0xec, 0xea, 0xea, 0xe7, 0x64, 0x4f, + 0xcf, 0xf4, 0xf4, 0xf4, 0x74, 0xcf, 0x78, 0x18, 0xed, 0x40, 0xef, 0xcc, 0xaa, 0x6d, 0x8f, 0xbb, + 0x0b, 0x6c, 0xaf, 0x37, 0xed, 0x9e, 0x96, 0x76, 0x91, 0x4a, 0xe9, 0xca, 0x70, 0x39, 0xd7, 0xf9, + 0xa8, 0xce, 0xcc, 0xb2, 0xdb, 0xcb, 0x61, 0x47, 0xac, 0x84, 0x34, 0x08, 0xb1, 0x08, 0x09, 0x01, + 0x07, 0x24, 0xc4, 0x69, 0x59, 0x04, 0x42, 0x5a, 0x71, 0xe1, 0xc2, 0x75, 0x04, 0x87, 0x15, 0x42, + 0xe2, 0x07, 0x70, 0x00, 0xee, 0x5c, 0x39, 0xa0, 0x78, 0x64, 0xe4, 0xbb, 0x2a, 0x5d, 0xd5, 0x3d, + 0x8d, 0xe0, 0x56, 0x11, 0xf9, 0x45, 0x7c, 0x11, 0x5f, 0x7c, 0xef, 0xf8, 0xa2, 0xa0, 0xad, 0xa9, + 0x9e, 0xda, 0x1f, 0xd8, 0xb6, 0xa3, 0xdd, 0x1f, 0x39, 0xb6, 0x67, 0xa3, 0x25, 0x53, 0x37, 0x4e, + 0xc6, 0x2e, 0x6b, 0xdd, 0x27, 0x9f, 0xbb, 0x8d, 0x81, 0x6d, 0x9a, 0xb6, 0xc5, 0xba, 0xba, 0x2d, + 0xdd, 0xf2, 0xb0, 0x63, 0xa9, 0x06, 0x6f, 0x37, 0xc2, 0x03, 0xba, 0x0d, 0x77, 0x70, 0x84, 0x4d, + 0x95, 0xb5, 0xe4, 0x05, 0x28, 0x7f, 0x6e, 0x8e, 0xbc, 0x33, 0xf9, 0x4f, 0x24, 0x68, 0x6c, 0x1a, + 0x63, 0xf7, 0x48, 0xc1, 0xcf, 0xc7, 0xd8, 0xf5, 0xd0, 0x07, 0x50, 0x3a, 0x50, 0x5d, 0xdc, 0x91, + 0x6e, 0x4a, 0x77, 0xea, 0xab, 0x57, 0xef, 0x47, 0xb0, 0x72, 0x7c, 0xdb, 0xee, 0x70, 0x4d, 0x75, + 0xb1, 0x42, 0x21, 0x11, 0x82, 0x92, 0x76, 0xd0, 0xdb, 0xe8, 0x14, 0x6e, 0x4a, 0x77, 0x8a, 0x0a, + 0xfd, 0x8d, 0xae, 0x03, 0xb8, 0x78, 0x68, 0x62, 0xcb, 0xeb, 0x6d, 0xb8, 0x9d, 0xe2, 0xcd, 0xe2, + 0x9d, 0xa2, 0x12, 0xea, 0x41, 0x32, 0x34, 0x06, 0xb6, 0x61, 0xe0, 0x81, 0xa7, 0xdb, 0x56, 0x6f, + 0xa3, 0x53, 0xa2, 0x63, 0x23, 0x7d, 0xf2, 0xbf, 0x4b, 0xd0, 0xe4, 0x4b, 0x73, 0x47, 0xb6, 0xe5, + 0x62, 0xf4, 0x11, 0x54, 0x5c, 0x4f, 0xf5, 0xc6, 0x2e, 0x5f, 0xdd, 0x95, 0xd4, 0xd5, 0xed, 0x51, + 0x10, 0x85, 0x83, 0xa6, 0x2e, 0x2f, 0x8e, 0xbe, 0x98, 0x44, 0x1f, 0xdb, 0x42, 0x29, 0xb1, 0x85, + 0x3b, 0xb0, 0x78, 0x48, 0x56, 0xb7, 0x17, 0x00, 0x95, 0x29, 0x50, 0xbc, 0x9b, 0xcc, 0xe4, 0xe9, + 0x26, 0xfe, 0xee, 0xe1, 0x1e, 0x56, 0x8d, 0x4e, 0x85, 0xe2, 0x0a, 0xf5, 0xc8, 0xff, 0x2c, 0x41, + 0x5b, 0x80, 0xfb, 0xe7, 0x70, 0x09, 0xca, 0x03, 0x7b, 0x6c, 0x79, 0x74, 0xab, 0x4d, 0x85, 0x35, + 0xd0, 0x9b, 0xd0, 0x18, 0x1c, 0xa9, 0x96, 0x85, 0x8d, 0xbe, 0xa5, 0x9a, 0x98, 0x6e, 0xaa, 0xa6, + 0xd4, 0x79, 0xdf, 0x8e, 0x6a, 0xe2, 0x5c, 0x7b, 0xbb, 0x09, 0xf5, 0x91, 0xea, 0x78, 0x7a, 0x84, + 0xfa, 0xe1, 0x2e, 0xd4, 0x85, 0xaa, 0xee, 0xf6, 0xcc, 0x91, 0xed, 0x78, 0x9d, 0xf2, 0x4d, 0xe9, + 0x4e, 0x55, 0x11, 0x6d, 0x82, 0x41, 0xa7, 0xbf, 0xf6, 0x55, 0xf7, 0xb8, 0xb7, 0xc1, 0x77, 0x14, + 0xe9, 0x93, 0xff, 0x5c, 0x82, 0xe5, 0x47, 0xae, 0xab, 0x0f, 0xad, 0xc4, 0xce, 0x96, 0xa1, 0x62, + 0xd9, 0x1a, 0xee, 0x6d, 0xd0, 0xad, 0x15, 0x15, 0xde, 0x42, 0x57, 0xa0, 0x36, 0xc2, 0xd8, 0xe9, + 0x3b, 0xb6, 0xe1, 0x6f, 0xac, 0x4a, 0x3a, 0x14, 0xdb, 0xc0, 0xe8, 0x7b, 0xb0, 0xe4, 0xc6, 0x26, + 0x62, 0x7c, 0x55, 0x5f, 0xbd, 0x75, 0x3f, 0x21, 0x19, 0xf7, 0xe3, 0x48, 0x95, 0xe4, 0x68, 0xf9, + 0xcb, 0x02, 0x5c, 0x14, 0x70, 0x6c, 0xad, 0xe4, 0x37, 0xa1, 0xbc, 0x8b, 0x87, 0x62, 0x79, 0xac, + 0x91, 0x87, 0xf2, 0xe2, 0xc8, 0x8a, 0xe1, 0x23, 0xcb, 0xc1, 0xea, 0xf1, 0xf3, 0x28, 0x27, 0xcf, + 0xe3, 0x06, 0xd4, 0xf1, 0x8b, 0x91, 0xee, 0xe0, 0x3e, 0x61, 0x1c, 0x4a, 0xf2, 0x92, 0x02, 0xac, + 0x6b, 0x5f, 0x37, 0xc3, 0xb2, 0xb1, 0x90, 0x5b, 0x36, 0xe4, 0xbf, 0x90, 0x60, 0x25, 0x71, 0x4a, + 0x5c, 0xd8, 0x14, 0x68, 0xd3, 0x9d, 0x07, 0x94, 0x21, 0x62, 0x47, 0x08, 0xfe, 0xf6, 0x24, 0x82, + 0x07, 0xe0, 0x4a, 0x62, 0x7c, 0x68, 0x91, 0x85, 0xfc, 0x8b, 0x3c, 0x86, 0x95, 0xc7, 0xd8, 0xe3, + 0x08, 0xc8, 0x37, 0xec, 0xce, 0xae, 0xac, 0xa2, 0x52, 0x5d, 0x88, 0x4b, 0xb5, 0xfc, 0xb7, 0x05, + 0x21, 0x8b, 0x14, 0x55, 0xcf, 0x3a, 0xb4, 0xd1, 0x55, 0xa8, 0x09, 0x10, 0xce, 0x15, 0x41, 0x07, + 0xfa, 0x16, 0x94, 0xc9, 0x4a, 0x19, 0x4b, 0xb4, 0x56, 0xdf, 0x4c, 0xdf, 0x53, 0x68, 0x4e, 0x85, + 0xc1, 0xa3, 0x1e, 0xb4, 0x5c, 0x4f, 0x75, 0xbc, 0xfe, 0xc8, 0x76, 0xe9, 0x39, 0x53, 0xc6, 0xa9, + 0xaf, 0xca, 0xd1, 0x19, 0x84, 0x5a, 0xdf, 0x76, 0x87, 0xbb, 0x1c, 0x52, 0x69, 0xd2, 0x91, 0x7e, + 0x13, 0x7d, 0x0e, 0x0d, 0x6c, 0x69, 0xc1, 0x44, 0xa5, 0xdc, 0x13, 0xd5, 0xb1, 0xa5, 0x89, 0x69, + 0x82, 0xf3, 0x29, 0xe7, 0x3f, 0x9f, 0xdf, 0x93, 0xa0, 0x93, 0x3c, 0xa0, 0x79, 0x54, 0xf6, 0x43, + 0x36, 0x08, 0xb3, 0x03, 0x9a, 0x28, 0xe1, 0xe2, 0x90, 0x14, 0x3e, 0x44, 0xfe, 0x23, 0x09, 0xde, + 0x08, 0x96, 0x43, 0x3f, 0xbd, 0x2a, 0x6e, 0x41, 0x77, 0xa1, 0xad, 0x5b, 0x03, 0x63, 0xac, 0xe1, + 0xa7, 0xd6, 0x13, 0xac, 0x1a, 0xde, 0xd1, 0x19, 0x3d, 0xc3, 0xaa, 0x92, 0xe8, 0x97, 0x7f, 0x22, + 0xc1, 0x72, 0x7c, 0x5d, 0xf3, 0x10, 0xe9, 0x57, 0xa0, 0xac, 0x5b, 0x87, 0xb6, 0x4f, 0xa3, 0xeb, + 0x13, 0x84, 0x92, 0xe0, 0x62, 0xc0, 0xb2, 0x09, 0x57, 0x1e, 0x63, 0xaf, 0x67, 0xb9, 0xd8, 0xf1, + 0xd6, 0x74, 0xcb, 0xb0, 0x87, 0xbb, 0xaa, 0x77, 0x34, 0x87, 0x40, 0x45, 0x64, 0xa3, 0x10, 0x93, + 0x0d, 0xf9, 0x67, 0x12, 0x5c, 0x4d, 0xc7, 0xc7, 0xb7, 0xde, 0x85, 0xea, 0xa1, 0x8e, 0x0d, 0x8d, + 0xd0, 0x57, 0xa2, 0xf4, 0x15, 0x6d, 0x22, 0x58, 0x23, 0x02, 0xcc, 0x77, 0xf8, 0x66, 0x06, 0x37, + 0xef, 0x79, 0x8e, 0x6e, 0x0d, 0xb7, 0x74, 0xd7, 0x53, 0x18, 0x7c, 0x88, 0x9e, 0xc5, 0xfc, 0x6c, + 0xfc, 0xbb, 0x12, 0x5c, 0x7f, 0x8c, 0xbd, 0x75, 0xa1, 0x97, 0xc9, 0x77, 0xdd, 0xf5, 0xf4, 0x81, + 0xfb, 0x72, 0x7d, 0xa3, 0x1c, 0x06, 0x5a, 0xfe, 0xa9, 0x04, 0x37, 0x32, 0x17, 0xc3, 0x49, 0xc7, + 0xf5, 0x8e, 0xaf, 0x95, 0xd3, 0xf5, 0xce, 0x6f, 0xe0, 0xb3, 0x2f, 0x54, 0x63, 0x8c, 0x77, 0x55, + 0xdd, 0x61, 0x7a, 0x67, 0x46, 0x2d, 0xfc, 0xd7, 0x12, 0x5c, 0x7b, 0x8c, 0xbd, 0x5d, 0xdf, 0x26, + 0xbd, 0x46, 0xea, 0x10, 0x98, 0x90, 0x6d, 0xf4, 0x9d, 0xb3, 0x48, 0x9f, 0xfc, 0xfb, 0xec, 0x38, + 0x53, 0xd7, 0xfb, 0x5a, 0x08, 0x78, 0x9d, 0x4a, 0x42, 0x48, 0x24, 0xd7, 0x99, 0xeb, 0xc0, 0xc9, + 0x27, 0xff, 0x99, 0x04, 0x97, 0x1f, 0x0d, 0x9e, 0x8f, 0x75, 0x07, 0x73, 0xa0, 0x2d, 0x7b, 0x70, + 0x3c, 0x3b, 0x71, 0x03, 0x37, 0xab, 0x10, 0x71, 0xb3, 0xa6, 0xb9, 0xe6, 0xcb, 0x50, 0xf1, 0x98, + 0x5f, 0xc7, 0x3c, 0x15, 0xde, 0xa2, 0xeb, 0x53, 0xb0, 0x81, 0x55, 0xf7, 0x7f, 0xe7, 0xfa, 0x7e, + 0x5a, 0x82, 0xc6, 0x17, 0xdc, 0x1d, 0xa3, 0x56, 0x3b, 0xce, 0x49, 0x52, 0xba, 0xe3, 0x15, 0xf2, + 0xe0, 0xd2, 0x9c, 0xba, 0xc7, 0xd0, 0x74, 0x31, 0x3e, 0x9e, 0xc5, 0x46, 0x37, 0xc8, 0x40, 0x61, + 0x5b, 0xb7, 0x60, 0x69, 0x6c, 0xd1, 0xd0, 0x00, 0x6b, 0x9c, 0x80, 0x8c, 0x73, 0xa7, 0xeb, 0xee, + 0xe4, 0x40, 0xf4, 0x84, 0x47, 0x1f, 0xa1, 0xb9, 0xca, 0xb9, 0xe6, 0x8a, 0x0f, 0x43, 0x3d, 0x68, + 0x6b, 0x8e, 0x3d, 0x1a, 0x61, 0xad, 0xef, 0xfa, 0x53, 0x55, 0xf2, 0x4d, 0xc5, 0xc7, 0x89, 0xa9, + 0x3e, 0x80, 0x8b, 0xf1, 0x95, 0xf6, 0x34, 0xe2, 0x90, 0x92, 0x33, 0x4c, 0xfb, 0x84, 0xee, 0xc1, + 0x52, 0x12, 0xbe, 0x4a, 0xe1, 0x93, 0x1f, 0xd0, 0xfb, 0x80, 0x62, 0x4b, 0x25, 0xe0, 0x35, 0x06, + 0x1e, 0x5d, 0x4c, 0x4f, 0x73, 0xe5, 0xaf, 0x24, 0x58, 0x7e, 0xa6, 0x7a, 0x83, 0xa3, 0x0d, 0x93, + 0xcb, 0xda, 0x1c, 0xba, 0xea, 0x53, 0xa8, 0x9d, 0x70, 0xbe, 0xf0, 0x0d, 0xd2, 0x8d, 0x14, 0xfa, + 0x84, 0x39, 0x50, 0x09, 0x46, 0x90, 0x78, 0xe8, 0xd2, 0x66, 0x28, 0x2e, 0x7c, 0x0d, 0x5a, 0x73, + 0x4a, 0x40, 0x2b, 0xbf, 0x00, 0xe0, 0x8b, 0xdb, 0x76, 0x87, 0x33, 0xac, 0xeb, 0x13, 0x58, 0xe0, + 0xb3, 0x71, 0xb5, 0x38, 0x8d, 0x7f, 0x7c, 0x70, 0xf9, 0xe7, 0x15, 0xa8, 0x87, 0x3e, 0xa0, 0x16, + 0x14, 0x84, 0xbc, 0x16, 0x52, 0x76, 0x57, 0x98, 0x1e, 0x42, 0x15, 0x93, 0x21, 0xd4, 0x6d, 0x68, + 0xe9, 0xd4, 0x0f, 0xe9, 0xf3, 0x53, 0xa1, 0x0a, 0xa4, 0xa6, 0x34, 0x59, 0x2f, 0x67, 0x11, 0x74, + 0x1d, 0xea, 0xd6, 0xd8, 0xec, 0xdb, 0x87, 0x7d, 0xc7, 0x3e, 0x75, 0x79, 0x2c, 0x56, 0xb3, 0xc6, + 0xe6, 0x77, 0x0f, 0x15, 0xfb, 0xd4, 0x0d, 0xdc, 0xfd, 0xca, 0x39, 0xdd, 0xfd, 0xeb, 0x50, 0x37, + 0xd5, 0x17, 0x64, 0xd6, 0xbe, 0x35, 0x36, 0x69, 0x98, 0x56, 0x54, 0x6a, 0xa6, 0xfa, 0x42, 0xb1, + 0x4f, 0x77, 0xc6, 0x26, 0xba, 0x03, 0x6d, 0x43, 0x75, 0xbd, 0x7e, 0x38, 0xce, 0xab, 0xd2, 0x38, + 0xaf, 0x45, 0xfa, 0x3f, 0x0f, 0x62, 0xbd, 0x64, 0xe0, 0x50, 0x9b, 0x23, 0x70, 0xd0, 0x4c, 0x23, + 0x98, 0x08, 0xf2, 0x07, 0x0e, 0x9a, 0x69, 0x88, 0x69, 0x3e, 0x81, 0x85, 0x03, 0xea, 0xdd, 0xb9, + 0x9d, 0x7a, 0xa6, 0xee, 0xd8, 0x24, 0x8e, 0x1d, 0x73, 0x02, 0x15, 0x1f, 0x1c, 0x7d, 0x1b, 0x6a, + 0xd4, 0xa8, 0xd2, 0xb1, 0x8d, 0x5c, 0x63, 0x83, 0x01, 0x64, 0xb4, 0x86, 0x0d, 0x4f, 0xa5, 0xa3, + 0x9b, 0xf9, 0x46, 0x8b, 0x01, 0x44, 0x5f, 0x0d, 0x1c, 0xac, 0x7a, 0x58, 0x5b, 0x3b, 0x5b, 0xb7, + 0xcd, 0x91, 0x4a, 0x99, 0xa9, 0xd3, 0xa2, 0x1e, 0x7c, 0xda, 0x27, 0xf4, 0x36, 0xb4, 0x06, 0xa2, + 0xb5, 0xe9, 0xd8, 0x66, 0x67, 0x91, 0xca, 0x51, 0xac, 0x17, 0x5d, 0x03, 0xf0, 0x35, 0x95, 0xea, + 0x75, 0xda, 0xf4, 0x14, 0x6b, 0xbc, 0xe7, 0x11, 0x4d, 0xe3, 0xe8, 0x6e, 0x9f, 0x25, 0x4c, 0x74, + 0x6b, 0xd8, 0x59, 0xa2, 0x18, 0xeb, 0x7e, 0x86, 0x45, 0xb7, 0x86, 0x68, 0x05, 0x16, 0x74, 0xb7, + 0x7f, 0xa8, 0x1e, 0xe3, 0x0e, 0xa2, 0x5f, 0x2b, 0xba, 0xbb, 0xa9, 0x1e, 0x63, 0xf9, 0xc7, 0x70, + 0x29, 0xe0, 0xae, 0xd0, 0x49, 0x26, 0x99, 0x42, 0x9a, 0x95, 0x29, 0x26, 0xfb, 0xf4, 0xbf, 0x2c, + 0xc1, 0xf2, 0x9e, 0x7a, 0x82, 0x5f, 0x7d, 0xf8, 0x90, 0x4b, 0xad, 0x6d, 0xc1, 0x12, 0x8d, 0x18, + 0x56, 0x43, 0xeb, 0x99, 0x60, 0x57, 0xc3, 0xac, 0x90, 0x1c, 0x88, 0xbe, 0x43, 0x1c, 0x02, 0x3c, + 0x38, 0xde, 0xb5, 0xf5, 0xc0, 0xa6, 0x5e, 0x4b, 0x99, 0x67, 0x5d, 0x40, 0x29, 0xe1, 0x11, 0x68, + 0x17, 0x16, 0xa3, 0xc7, 0xe0, 0x5b, 0xd3, 0x77, 0x26, 0x06, 0xb1, 0x01, 0xf5, 0x95, 0x56, 0xe4, + 0x30, 0x5c, 0xd4, 0x81, 0x05, 0x6e, 0x0a, 0xa9, 0xce, 0xa8, 0x2a, 0x7e, 0x13, 0xed, 0xc2, 0x45, + 0xb6, 0x83, 0x3d, 0x2e, 0x10, 0x6c, 0xf3, 0xd5, 0x5c, 0x9b, 0x4f, 0x1b, 0x1a, 0x95, 0xa7, 0xda, + 0x79, 0xe5, 0xa9, 0x03, 0x0b, 0x9c, 0xc7, 0xa9, 0x1e, 0xa9, 0x2a, 0x7e, 0x93, 0x1c, 0x73, 0xc0, + 0xed, 0x75, 0xfa, 0x2d, 0xe8, 0x20, 0xa1, 0x17, 0x04, 0xf4, 0x9c, 0x92, 0x6e, 0xf9, 0x0c, 0xaa, + 0x82, 0xc3, 0x0b, 0xb9, 0x39, 0x5c, 0x8c, 0x89, 0xeb, 0xf7, 0x62, 0x4c, 0xbf, 0xcb, 0xff, 0x24, + 0x41, 0x63, 0x83, 0x6c, 0x69, 0xcb, 0x1e, 0x52, 0x6b, 0x74, 0x1b, 0x5a, 0x0e, 0x1e, 0xd8, 0x8e, + 0xd6, 0xc7, 0x96, 0xe7, 0xe8, 0x98, 0x45, 0xe9, 0x25, 0xa5, 0xc9, 0x7a, 0x3f, 0x67, 0x9d, 0x04, + 0x8c, 0xa8, 0x6c, 0xd7, 0x53, 0xcd, 0x51, 0xff, 0x90, 0xa8, 0x86, 0x02, 0x03, 0x13, 0xbd, 0x54, + 0x33, 0xbc, 0x09, 0x8d, 0x00, 0xcc, 0xb3, 0x29, 0xfe, 0x92, 0x52, 0x17, 0x7d, 0xfb, 0x36, 0x7a, + 0x0b, 0x5a, 0x94, 0xa6, 0x7d, 0xc3, 0x1e, 0xf6, 0x49, 0x44, 0xcb, 0x0d, 0x55, 0x43, 0xe3, 0xcb, + 0x22, 0x67, 0x15, 0x85, 0x72, 0xf5, 0x1f, 0x61, 0x6e, 0xaa, 0x04, 0xd4, 0x9e, 0xfe, 0x23, 0x2c, + 0xff, 0xa3, 0x04, 0xcd, 0x0d, 0xd5, 0x53, 0x77, 0x6c, 0x0d, 0xef, 0xcf, 0x68, 0xd8, 0x73, 0xa4, + 0x3e, 0xaf, 0x42, 0x4d, 0xec, 0x80, 0x6f, 0x29, 0xe8, 0x40, 0x9b, 0xd0, 0xf2, 0x5d, 0xcb, 0x3e, + 0x8b, 0xb8, 0x4a, 0x99, 0x0e, 0x54, 0xc8, 0x72, 0xba, 0x4a, 0xd3, 0x1f, 0x46, 0x9b, 0xf2, 0x26, + 0x34, 0xc2, 0x9f, 0x09, 0xd6, 0xbd, 0x38, 0xa3, 0x88, 0x0e, 0xc2, 0x8d, 0x3b, 0x63, 0x93, 0x9c, + 0x29, 0x57, 0x2c, 0x7e, 0x53, 0xfe, 0x89, 0x04, 0x4d, 0x6e, 0xee, 0xf7, 0xc4, 0x25, 0x01, 0xdd, + 0x9a, 0x44, 0xb7, 0x46, 0x7f, 0xa3, 0x5f, 0x8b, 0xe6, 0xf5, 0xde, 0x4a, 0x55, 0x02, 0x74, 0x12, + 0xea, 0x64, 0x46, 0x6c, 0x7d, 0x9e, 0x18, 0xff, 0x4b, 0xc2, 0x68, 0xfc, 0x68, 0x28, 0xa3, 0x75, + 0x60, 0x41, 0xd5, 0x34, 0x07, 0xbb, 0x2e, 0x5f, 0x87, 0xdf, 0x24, 0x5f, 0x4e, 0xb0, 0xe3, 0xfa, + 0x2c, 0x5f, 0x54, 0xfc, 0x26, 0xfa, 0x36, 0x54, 0x85, 0x57, 0xca, 0xd2, 0xe1, 0x37, 0xb3, 0xd7, + 0xc9, 0x23, 0x52, 0x31, 0x42, 0xfe, 0xbb, 0x02, 0xb4, 0x38, 0xc1, 0xd6, 0xb8, 0x3d, 0x9e, 0x2c, + 0x7c, 0x6b, 0xd0, 0x38, 0x0c, 0x64, 0x7f, 0x52, 0xee, 0x29, 0xac, 0x22, 0x22, 0x63, 0xa6, 0x09, + 0x60, 0xd4, 0x23, 0x28, 0xcd, 0xe5, 0x11, 0x94, 0xcf, 0xab, 0xc1, 0x92, 0x3e, 0x62, 0x25, 0xc5, + 0x47, 0x94, 0x7f, 0x13, 0xea, 0xa1, 0x09, 0xa8, 0x86, 0x66, 0x49, 0x2b, 0x4e, 0x31, 0xbf, 0x89, + 0x3e, 0x0a, 0xfc, 0x22, 0x46, 0xaa, 0xcb, 0x29, 0x6b, 0x89, 0xb9, 0x44, 0xf2, 0x3f, 0x48, 0x50, + 0xe1, 0x33, 0xdf, 0x80, 0x3a, 0x57, 0x3a, 0xd4, 0x67, 0x64, 0xb3, 0x03, 0xef, 0x22, 0x4e, 0xe3, + 0xcb, 0xd3, 0x3a, 0x97, 0xa1, 0x1a, 0xd3, 0x37, 0x0b, 0xdc, 0x2c, 0xf8, 0x9f, 0x42, 0x4a, 0x86, + 0x7c, 0x22, 0xfa, 0x05, 0x5d, 0x82, 0xb2, 0x61, 0x0f, 0xc5, 0x25, 0x10, 0x6b, 0xc8, 0x5f, 0x4b, + 0x34, 0x67, 0xaf, 0xe0, 0x81, 0x7d, 0x82, 0x9d, 0xb3, 0xf9, 0x93, 0x9d, 0x0f, 0x43, 0x6c, 0x9e, + 0x33, 0xf8, 0x12, 0x03, 0xd0, 0xc3, 0xe0, 0x10, 0x8a, 0x69, 0x99, 0x9e, 0xb0, 0xde, 0xe1, 0x4c, + 0x1a, 0x1c, 0xc6, 0x1f, 0xb0, 0xb4, 0x6d, 0x74, 0x2b, 0xb3, 0x7a, 0x3b, 0x2f, 0x25, 0x90, 0x91, + 0x7f, 0x29, 0x41, 0x37, 0x48, 0x25, 0xb9, 0x6b, 0x67, 0xf3, 0x5e, 0x8a, 0xbc, 0x9c, 0xf8, 0xea, + 0x57, 0x45, 0xd6, 0x9e, 0x08, 0x6d, 0xae, 0xc8, 0xc8, 0xcf, 0xd9, 0x5b, 0x34, 0x2b, 0x9d, 0xdc, + 0xd0, 0x3c, 0x2c, 0xd3, 0x85, 0xaa, 0xc8, 0x67, 0xb0, 0xcc, 0xbd, 0x68, 0x13, 0x09, 0xbb, 0xfc, + 0x18, 0x7b, 0x9b, 0xd1, 0x54, 0xc8, 0xeb, 0x26, 0x60, 0xf8, 0x36, 0xe1, 0x88, 0xdf, 0x26, 0x94, + 0x62, 0xb7, 0x09, 0xbc, 0x5f, 0x36, 0x29, 0x0b, 0x24, 0x36, 0xf0, 0xaa, 0x08, 0xf6, 0x3b, 0x12, + 0x74, 0x38, 0x16, 0x8a, 0x93, 0x84, 0x44, 0x06, 0xf6, 0xb0, 0xf6, 0x4d, 0xa7, 0x0a, 0xfe, 0x5b, + 0x82, 0x76, 0xd8, 0xea, 0x52, 0xc3, 0xf9, 0x31, 0x94, 0x69, 0xa6, 0x85, 0xaf, 0x60, 0xaa, 0x6a, + 0x60, 0xd0, 0x44, 0x6d, 0x53, 0x57, 0x7b, 0x5f, 0x38, 0x08, 0xbc, 0x19, 0x98, 0xfe, 0xe2, 0xf9, + 0x4d, 0x3f, 0x77, 0x85, 0xec, 0x31, 0x99, 0x97, 0xa5, 0x28, 0x83, 0x0e, 0xf4, 0x29, 0x54, 0x58, + 0x21, 0x06, 0xbf, 0x61, 0xbb, 0x1d, 0x9d, 0x9a, 0x17, 0x69, 0x84, 0xf2, 0xfe, 0xb4, 0x43, 0xe1, + 0x83, 0xe4, 0x5f, 0x87, 0xe5, 0x20, 0x1a, 0x65, 0x68, 0x67, 0x65, 0x5a, 0xf9, 0x5f, 0x25, 0xb8, + 0xb8, 0x77, 0x66, 0x0d, 0xe2, 0xec, 0xbf, 0x0c, 0x95, 0x91, 0xa1, 0x06, 0x19, 0x53, 0xde, 0xa2, + 0x6e, 0x20, 0xc3, 0x8d, 0x35, 0x62, 0x43, 0x18, 0xcd, 0xea, 0xa2, 0x6f, 0xdf, 0x9e, 0x6a, 0xda, + 0x6f, 0x8b, 0xf0, 0x19, 0x6b, 0xcc, 0x5a, 0xb1, 0x34, 0x54, 0x53, 0xf4, 0x52, 0x6b, 0xf5, 0x29, + 0x00, 0x35, 0xe8, 0xfd, 0xf3, 0x18, 0x71, 0x3a, 0x62, 0x8b, 0xa8, 0xec, 0x5f, 0x14, 0xa0, 0x13, + 0xa2, 0xd2, 0x37, 0xed, 0xdf, 0x64, 0x44, 0x65, 0xc5, 0x97, 0x14, 0x95, 0x95, 0xe6, 0xf7, 0x69, + 0xca, 0x69, 0x3e, 0xcd, 0xbf, 0x15, 0xa0, 0x15, 0x50, 0x6d, 0xd7, 0x50, 0xad, 0x4c, 0x4e, 0xd8, + 0x13, 0xfe, 0x7c, 0x94, 0x4e, 0xef, 0xa5, 0xc9, 0x49, 0xc6, 0x41, 0x28, 0xb1, 0x29, 0xd0, 0x35, + 0x7a, 0xe8, 0x8e, 0xc7, 0x12, 0x5f, 0x3c, 0x86, 0x60, 0x02, 0xa9, 0x9b, 0x18, 0xdd, 0x03, 0xc4, + 0xa5, 0xa8, 0xaf, 0x5b, 0x7d, 0x17, 0x0f, 0x6c, 0x4b, 0x63, 0xf2, 0x55, 0x56, 0xda, 0xfc, 0x4b, + 0xcf, 0xda, 0x63, 0xfd, 0xe8, 0x63, 0x28, 0x79, 0x67, 0x23, 0xe6, 0xad, 0xb4, 0x52, 0xed, 0x7d, + 0xb0, 0xae, 0xfd, 0xb3, 0x11, 0x56, 0x28, 0xb8, 0x5f, 0xa9, 0xe3, 0x39, 0xea, 0x09, 0x77, 0xfd, + 0x4a, 0x4a, 0xa8, 0x87, 0x68, 0x0c, 0x9f, 0x86, 0x0b, 0xcc, 0x45, 0xe2, 0x4d, 0xc6, 0xd9, 0xbe, + 0xd0, 0xf6, 0x3d, 0xcf, 0xa0, 0xa9, 0x3b, 0xca, 0xd9, 0x7e, 0xef, 0xbe, 0x67, 0xc8, 0xff, 0x52, + 0x80, 0x76, 0x80, 0x59, 0xc1, 0xee, 0xd8, 0xc8, 0x16, 0xb8, 0xc9, 0xb9, 0x91, 0x69, 0xb2, 0xf6, + 0x1d, 0xa8, 0xf3, 0x63, 0x3f, 0x07, 0xdb, 0x00, 0x1b, 0xb2, 0x35, 0x81, 0x8f, 0xcb, 0x2f, 0x89, + 0x8f, 0x2b, 0x33, 0x64, 0x17, 0xd2, 0x89, 0x2f, 0xff, 0x4c, 0x82, 0x37, 0x12, 0x6a, 0x71, 0x22, + 0x69, 0x27, 0xc7, 0x76, 0x5c, 0x5d, 0xc6, 0xa7, 0xe4, 0x0a, 0xfe, 0x21, 0x54, 0x1c, 0x3a, 0x3b, + 0xbf, 0x0a, 0xba, 0x35, 0x91, 0xbb, 0xd8, 0x42, 0x14, 0x3e, 0x44, 0xfe, 0x43, 0x09, 0x56, 0x92, + 0x4b, 0x9d, 0xc3, 0x6a, 0xaf, 0xc1, 0x02, 0x9b, 0xda, 0x17, 0xc2, 0x3b, 0x93, 0x85, 0x30, 0x20, + 0x8e, 0xe2, 0x0f, 0x94, 0xf7, 0x60, 0xd9, 0x37, 0xee, 0x01, 0xe9, 0xb7, 0xb1, 0xa7, 0x4e, 0x88, + 0x6c, 0x6e, 0x40, 0x9d, 0xb9, 0xc8, 0x2c, 0x62, 0x60, 0x39, 0x01, 0x38, 0x10, 0xa9, 0x34, 0xf9, + 0x3f, 0x25, 0xb8, 0x44, 0xad, 0x63, 0xfc, 0xee, 0x25, 0xcf, 0xbd, 0x9c, 0x2c, 0x52, 0x0e, 0x3b, + 0xaa, 0xc9, 0xeb, 0x40, 0x6a, 0x4a, 0xa4, 0x0f, 0xf5, 0x92, 0x99, 0xb6, 0xd4, 0x08, 0x38, 0xb8, + 0xc8, 0x25, 0xd1, 0x36, 0xbd, 0xc7, 0x8d, 0xa7, 0xd8, 0x02, 0xab, 0x5c, 0x9a, 0xc5, 0x2a, 0x6f, + 0xc1, 0x1b, 0xb1, 0x9d, 0xce, 0x71, 0xa2, 0xf2, 0x5f, 0x4a, 0xe4, 0x38, 0x22, 0xf5, 0x34, 0xb3, + 0x7b, 0xa6, 0xd7, 0xc4, 0xa5, 0x4f, 0x5f, 0xd7, 0xe2, 0x4a, 0x44, 0x43, 0x9f, 0x41, 0xcd, 0xc2, + 0xa7, 0xfd, 0xb0, 0xb3, 0x93, 0xc3, 0x6d, 0xaf, 0x5a, 0xf8, 0x94, 0xfe, 0x92, 0x77, 0x60, 0x25, + 0xb1, 0xd4, 0x79, 0xf6, 0xfe, 0xf7, 0x12, 0x5c, 0xde, 0x70, 0xec, 0xd1, 0x17, 0xba, 0xe3, 0x8d, + 0x55, 0x23, 0x7a, 0x45, 0xfe, 0x6a, 0x52, 0x57, 0x4f, 0x42, 0x6e, 0x2f, 0xe3, 0x9f, 0x7b, 0x29, + 0x12, 0x94, 0x5c, 0x14, 0xdf, 0x74, 0xc8, 0x49, 0xfe, 0x8f, 0x62, 0xda, 0xe2, 0x39, 0xdc, 0x14, + 0xc7, 0x23, 0x4f, 0x04, 0x91, 0x9a, 0xe9, 0x2e, 0xce, 0x9a, 0xe9, 0xce, 0x50, 0xef, 0xa5, 0x97, + 0xa4, 0xde, 0xcf, 0x9d, 0x7a, 0x79, 0x02, 0xd1, 0x5b, 0x08, 0x6a, 0x7e, 0x67, 0xba, 0xbe, 0x58, + 0x03, 0x08, 0x32, 0xf2, 0xbc, 0x1c, 0x32, 0xcf, 0x34, 0xa1, 0x51, 0xe4, 0xb4, 0x84, 0x29, 0xe5, + 0xa6, 0x3c, 0x94, 0x23, 0xfe, 0x1e, 0x74, 0xd3, 0xb8, 0x74, 0x1e, 0xce, 0xff, 0x45, 0x01, 0xa0, + 0x27, 0x2a, 0x68, 0x67, 0xb3, 0x05, 0xb7, 0x20, 0xe4, 0x6e, 0x04, 0xf2, 0x1e, 0xe6, 0x22, 0x8d, + 0x88, 0x84, 0x08, 0x3a, 0x09, 0x4c, 0x22, 0x10, 0xd5, 0xe8, 0x3c, 0x21, 0xa9, 0x61, 0x4c, 0x11, + 0x57, 0xbf, 0x57, 0xa0, 0xe6, 0xd8, 0xa7, 0x7d, 0x22, 0x66, 0x9a, 0x5f, 0x22, 0xec, 0xd8, 0xa7, + 0x44, 0xf8, 0x34, 0xb4, 0x02, 0x0b, 0x9e, 0xea, 0x1e, 0x93, 0xf9, 0x2b, 0xa1, 0x2a, 0x0d, 0x0d, + 0x5d, 0x82, 0xf2, 0xa1, 0x6e, 0x60, 0x56, 0x14, 0x50, 0x53, 0x58, 0x03, 0x7d, 0xcb, 0xaf, 0x65, + 0xab, 0xe6, 0xae, 0xc4, 0x61, 0xe5, 0x6c, 0x5f, 0x4b, 0xb0, 0x18, 0x50, 0x8d, 0x2a, 0x20, 0xa2, + 0xd3, 0xa8, 0x3e, 0x5b, 0xb7, 0x35, 0xa6, 0x2a, 0x5a, 0x19, 0x16, 0x81, 0x0d, 0x64, 0x5a, 0x2b, + 0x18, 0x32, 0x29, 0x0e, 0x26, 0xfb, 0x22, 0x9b, 0xd6, 0x35, 0xbf, 0x32, 0xa5, 0xe2, 0xd8, 0xa7, + 0x3d, 0x4d, 0x50, 0x83, 0xd5, 0xff, 0xb2, 0xa8, 0x8f, 0x50, 0x63, 0x9d, 0x96, 0x00, 0xdf, 0x82, + 0x26, 0x76, 0x1c, 0xdb, 0xe9, 0x9b, 0xd8, 0x75, 0xd5, 0x21, 0xe6, 0x0e, 0x78, 0x83, 0x76, 0x6e, + 0xb3, 0x3e, 0xf9, 0x8f, 0x4b, 0xd0, 0x0a, 0xb6, 0xe2, 0xdf, 0x83, 0xeb, 0x9a, 0x7f, 0x0f, 0xae, + 0x93, 0xa3, 0x03, 0x87, 0xa9, 0x42, 0x71, 0xb8, 0x6b, 0x85, 0x8e, 0xa4, 0xd4, 0x78, 0x6f, 0x4f, + 0x23, 0x66, 0x99, 0x08, 0x99, 0x65, 0x6b, 0x38, 0x38, 0x5c, 0xf0, 0xbb, 0xf8, 0xd9, 0x46, 0x78, + 0xa4, 0x94, 0x83, 0x47, 0xca, 0x39, 0x78, 0xa4, 0x92, 0xc2, 0x23, 0xcb, 0x50, 0x39, 0x18, 0x0f, + 0x8e, 0xb1, 0xc7, 0x3d, 0x36, 0xde, 0x8a, 0xf2, 0x4e, 0x35, 0xc6, 0x3b, 0x82, 0x45, 0x6a, 0x61, + 0x16, 0xb9, 0x02, 0x35, 0x76, 0x21, 0xdb, 0xf7, 0x5c, 0x7a, 0xbb, 0x54, 0x54, 0xaa, 0xac, 0x63, + 0xdf, 0x45, 0x9f, 0xf8, 0xee, 0x5c, 0x3d, 0x4d, 0xd8, 0xa9, 0xd6, 0x89, 0x71, 0x89, 0xef, 0xcc, + 0xbd, 0x03, 0x8b, 0x21, 0x72, 0x50, 0x1b, 0xd1, 0xa0, 0x4b, 0x0d, 0xb9, 0xf3, 0xd4, 0x4c, 0xdc, + 0x86, 0x56, 0x40, 0x12, 0x0a, 0xd7, 0x64, 0x51, 0x94, 0xe8, 0xa5, 0x60, 0x82, 0x93, 0x5b, 0xe7, + 0xe3, 0x64, 0x74, 0x19, 0xaa, 0x3c, 0xfc, 0x71, 0x3b, 0x8b, 0x91, 0x6c, 0x84, 0xfc, 0x43, 0x40, + 0xc1, 0xea, 0xe7, 0xf3, 0x16, 0x63, 0xec, 0x51, 0x88, 0xb3, 0x87, 0xfc, 0x73, 0x09, 0x96, 0xc2, + 0xc8, 0x66, 0x35, 0xbc, 0x9f, 0x41, 0x9d, 0xdd, 0xef, 0xf5, 0x89, 0xe0, 0xf3, 0x2c, 0xcf, 0xb5, + 0x89, 0xe7, 0xa2, 0x40, 0xf0, 0x82, 0x80, 0xb0, 0xd7, 0xa9, 0xed, 0x1c, 0xeb, 0xd6, 0xb0, 0x4f, + 0x56, 0xe6, 0x8b, 0x5b, 0x83, 0x77, 0xee, 0x90, 0x3e, 0xf9, 0x2b, 0x09, 0xae, 0x3f, 0x1d, 0x69, + 0xaa, 0x87, 0x43, 0x1e, 0xc8, 0xbc, 0x45, 0x89, 0x1f, 0xfb, 0x55, 0x81, 0x85, 0x7c, 0x77, 0x54, + 0x0c, 0x5a, 0xfe, 0x1b, 0xb1, 0x16, 0x6e, 0x0e, 0xe8, 0x85, 0xe6, 0x88, 0x5e, 0x10, 0xcf, 0xbc, + 0x96, 0x2e, 0x54, 0x4f, 0xf8, 0x74, 0xfe, 0x8b, 0x08, 0xbf, 0x1d, 0xb9, 0x07, 0x2d, 0x9e, 0xff, + 0x1e, 0x54, 0xde, 0x86, 0xcb, 0x0a, 0x76, 0xb1, 0xa5, 0x45, 0x76, 0x33, 0x73, 0x36, 0x69, 0x04, + 0xdd, 0xb4, 0xe9, 0xe6, 0x61, 0x56, 0xe6, 0xbb, 0xf6, 0x1d, 0x32, 0xad, 0xc7, 0x55, 0x31, 0x71, + 0x99, 0x28, 0x1e, 0x4f, 0xfe, 0xab, 0x02, 0xac, 0x3c, 0xd2, 0x34, 0xae, 0xc5, 0xb9, 0x37, 0xf6, + 0xaa, 0x1c, 0xe5, 0xb8, 0x23, 0x59, 0x4c, 0x3a, 0x92, 0x2f, 0x4b, 0xb3, 0x72, 0x1b, 0x63, 0x8d, + 0x4d, 0xdf, 0x76, 0x3a, 0xac, 0x40, 0xe8, 0x21, 0xbf, 0x18, 0x23, 0x01, 0x3d, 0xb5, 0x9f, 0xd3, + 0xfd, 0xab, 0xaa, 0x9f, 0x15, 0x93, 0x47, 0xd0, 0x49, 0x12, 0x6b, 0x4e, 0x55, 0xe2, 0x53, 0x64, + 0x64, 0xb3, 0x0c, 0x6a, 0x83, 0xb8, 0x50, 0xb4, 0x6b, 0xd7, 0x76, 0xe5, 0xff, 0x2a, 0x40, 0x67, + 0x4f, 0x3d, 0xc1, 0xff, 0x7f, 0x0e, 0xe8, 0xfb, 0x70, 0xc9, 0x55, 0x4f, 0x70, 0x3f, 0x14, 0x18, + 0xf7, 0x1d, 0xfc, 0x9c, 0xbb, 0xa0, 0xef, 0xa6, 0x69, 0x92, 0xd4, 0x3a, 0x1a, 0x65, 0xc9, 0x8d, + 0xf4, 0x2b, 0xf8, 0x39, 0x7a, 0x1b, 0x16, 0xc3, 0x85, 0x5a, 0x64, 0x69, 0x55, 0x4a, 0xf2, 0x66, + 0xa8, 0x0e, 0xab, 0xa7, 0xc9, 0xcf, 0xe1, 0xea, 0x53, 0xcb, 0xc5, 0x5e, 0x2f, 0xa8, 0x25, 0x9a, + 0x33, 0x84, 0xbc, 0x01, 0xf5, 0x80, 0xf0, 0x89, 0x57, 0x10, 0x9a, 0x2b, 0xdb, 0xd0, 0xdd, 0x56, + 0x9d, 0x63, 0x3f, 0x8f, 0xbc, 0xc1, 0x6a, 0x3e, 0x5e, 0x21, 0xc2, 0x43, 0x51, 0x02, 0xa5, 0xe0, + 0x43, 0xec, 0x60, 0x6b, 0x80, 0xb7, 0xec, 0xc1, 0x71, 0xa8, 0x34, 0x58, 0x0a, 0x97, 0x06, 0xcf, + 0x5a, 0x6a, 0x7c, 0xf7, 0x33, 0x51, 0x96, 0xb8, 0x7f, 0x36, 0xc2, 0x68, 0x01, 0x8a, 0x3b, 0xf8, + 0xb4, 0x7d, 0x01, 0x01, 0x54, 0x76, 0x6c, 0xc7, 0x54, 0x8d, 0xb6, 0x84, 0xea, 0xb0, 0xc0, 0x6f, + 0x61, 0xda, 0x05, 0xd4, 0x84, 0xda, 0xba, 0x9f, 0xc9, 0x6e, 0x17, 0xef, 0xfe, 0xa9, 0x04, 0x4b, + 0x89, 0x7b, 0x02, 0xd4, 0x02, 0x78, 0x6a, 0x0d, 0xf8, 0x05, 0x4a, 0xfb, 0x02, 0x6a, 0x40, 0xd5, + 0xbf, 0x4e, 0x61, 0xf3, 0xed, 0xdb, 0x14, 0xba, 0x5d, 0x40, 0x6d, 0x68, 0xb0, 0x81, 0xe3, 0xc1, + 0x00, 0xbb, 0x6e, 0xbb, 0x28, 0x7a, 0x36, 0x55, 0xdd, 0x18, 0x3b, 0xb8, 0x5d, 0x22, 0x38, 0xf7, + 0x6d, 0x5e, 0x98, 0xdd, 0x2e, 0x23, 0x04, 0x2d, 0xbf, 0x4a, 0x9b, 0x0f, 0xaa, 0x84, 0xfa, 0xfc, + 0x61, 0x0b, 0x77, 0x9f, 0x85, 0xb3, 0xbd, 0x74, 0x7b, 0x2b, 0x70, 0xf1, 0xa9, 0xa5, 0xe1, 0x43, + 0xdd, 0xc2, 0x5a, 0xf0, 0xa9, 0x7d, 0x01, 0x5d, 0x84, 0xc5, 0x6d, 0xec, 0x0c, 0x71, 0xa8, 0xb3, + 0x80, 0x96, 0xa0, 0xb9, 0xad, 0xbf, 0x08, 0x75, 0x15, 0xe5, 0x52, 0x55, 0x6a, 0x4b, 0xab, 0x5f, + 0x5d, 0x83, 0xda, 0x86, 0xea, 0xa9, 0xeb, 0xb6, 0xed, 0x68, 0xc8, 0x00, 0x44, 0xdf, 0x31, 0x98, + 0x23, 0xdb, 0x12, 0xaf, 0x83, 0xd0, 0xfd, 0x28, 0x17, 0xf0, 0x46, 0x12, 0x90, 0xf3, 0x50, 0xf7, + 0xad, 0x54, 0xf8, 0x18, 0xb0, 0x7c, 0x01, 0x99, 0x14, 0xdb, 0xbe, 0x6e, 0xe2, 0x7d, 0x7d, 0x70, + 0xec, 0x5b, 0xca, 0x0f, 0x32, 0xec, 0x62, 0x12, 0xd4, 0xc7, 0x77, 0x2b, 0x15, 0x1f, 0x7b, 0x68, + 0xe2, 0x6b, 0x4d, 0xf9, 0x02, 0x7a, 0x0e, 0x97, 0x1e, 0xe3, 0x90, 0xd3, 0xe1, 0x23, 0x5c, 0xcd, + 0x46, 0x98, 0x00, 0x3e, 0x27, 0xca, 0x2d, 0x28, 0x53, 0x76, 0x43, 0x69, 0x7e, 0x49, 0xf8, 0x21, + 0x6f, 0xf7, 0x66, 0x36, 0x80, 0x98, 0xed, 0x87, 0xb0, 0x18, 0x7b, 0xfe, 0x87, 0xd2, 0xb4, 0x54, + 0xfa, 0x43, 0xce, 0xee, 0xdd, 0x3c, 0xa0, 0x02, 0xd7, 0x10, 0x5a, 0xd1, 0xf7, 0x0f, 0x28, 0x2d, + 0x53, 0x99, 0xfa, 0x72, 0xab, 0xfb, 0x6e, 0x0e, 0x48, 0x81, 0xc8, 0x84, 0x76, 0xfc, 0x39, 0x1a, + 0xba, 0x3b, 0x71, 0x82, 0x28, 0xb3, 0xbd, 0x97, 0x0b, 0x56, 0xa0, 0x3b, 0xa3, 0x4c, 0x90, 0x78, + 0xe1, 0x14, 0xe7, 0x71, 0x7f, 0x9a, 0xac, 0xa7, 0x57, 0xdd, 0x07, 0xb9, 0xe1, 0x05, 0xea, 0xdf, + 0x66, 0x65, 0x16, 0x69, 0xaf, 0x84, 0xd0, 0x87, 0xe9, 0xd3, 0x4d, 0x78, 0xde, 0xd4, 0x5d, 0x3d, + 0xcf, 0x10, 0xb1, 0x88, 0x1f, 0xd3, 0xfa, 0x88, 0x94, 0x77, 0x36, 0x71, 0xb9, 0xf3, 0xe7, 0xcb, + 0x7e, 0x42, 0xd4, 0xfd, 0xf0, 0x1c, 0x23, 0xc4, 0x02, 0xec, 0xf8, 0x7b, 0x3f, 0x5f, 0x0c, 0x1f, + 0x4c, 0xe5, 0x9a, 0xd9, 0x64, 0xf0, 0x07, 0xb0, 0x18, 0xb3, 0xdb, 0x28, 0xbf, 0x6d, 0xef, 0x4e, + 0x72, 0xae, 0x98, 0x48, 0xc6, 0xca, 0x4d, 0x50, 0x06, 0xf7, 0xa7, 0x94, 0xa4, 0x74, 0xef, 0xe6, + 0x01, 0x15, 0x1b, 0x71, 0xa9, 0xba, 0x8c, 0x15, 0x11, 0xa0, 0x7b, 0xe9, 0x73, 0xa4, 0x17, 0x4b, + 0x74, 0xdf, 0xcf, 0x09, 0x2d, 0x90, 0x9e, 0xc0, 0xc5, 0x94, 0x5a, 0x0f, 0xf4, 0xfe, 0xc4, 0xc3, + 0x8a, 0x17, 0xb9, 0x74, 0xef, 0xe7, 0x05, 0x17, 0x78, 0x7f, 0x0b, 0xd0, 0xde, 0x91, 0x7d, 0xba, + 0x6e, 0x5b, 0x87, 0xfa, 0x70, 0xec, 0xa8, 0x2c, 0xf3, 0x9f, 0x65, 0x1b, 0x92, 0xa0, 0x19, 0x3c, + 0x3a, 0x71, 0x84, 0x40, 0xde, 0x07, 0x78, 0x8c, 0xbd, 0x6d, 0xec, 0x39, 0x44, 0x30, 0xde, 0xce, + 0x32, 0x7f, 0x1c, 0xc0, 0x47, 0xf5, 0xce, 0x54, 0xb8, 0x90, 0x29, 0x6a, 0x6f, 0xab, 0xd6, 0x58, + 0x35, 0x42, 0xc5, 0xea, 0xf7, 0x52, 0x87, 0xc7, 0xc1, 0x32, 0x0e, 0x32, 0x13, 0x5a, 0xa0, 0x3c, + 0x15, 0xa6, 0x3d, 0x74, 0xb5, 0x34, 0xd9, 0xb4, 0x27, 0xeb, 0x16, 0xe2, 0x6a, 0x6f, 0x02, 0xbc, + 0x40, 0xfc, 0xa5, 0x44, 0xcb, 0x85, 0x62, 0x00, 0xcf, 0x74, 0xef, 0x68, 0xd7, 0x50, 0x2d, 0x37, + 0xcf, 0x12, 0x28, 0xe0, 0x39, 0x96, 0xc0, 0xe1, 0xc5, 0x12, 0x34, 0x68, 0x46, 0x6e, 0x7c, 0x50, + 0x5a, 0x75, 0x77, 0xda, 0xed, 0x57, 0xf7, 0xce, 0x74, 0x40, 0x81, 0xe5, 0x08, 0x9a, 0xbe, 0x28, + 0x31, 0xe2, 0xbe, 0x9b, 0xb5, 0xd2, 0x00, 0x26, 0x43, 0x13, 0xa4, 0x83, 0x86, 0x35, 0x41, 0x32, + 0xa1, 0x8d, 0xf2, 0x5d, 0x84, 0x4c, 0xd2, 0x04, 0xd9, 0x59, 0x72, 0xa6, 0xea, 0x62, 0x97, 0x47, + 0xe9, 0x7a, 0x34, 0xf5, 0x2e, 0x2c, 0x55, 0xd5, 0x65, 0xdc, 0x45, 0xc9, 0x17, 0xd0, 0x33, 0xa8, + 0xf0, 0x7f, 0xaf, 0x78, 0x6b, 0x72, 0x12, 0x8a, 0xcf, 0x7e, 0x7b, 0x0a, 0x94, 0x98, 0xf8, 0x18, + 0x56, 0x32, 0x52, 0x50, 0xa9, 0x26, 0x78, 0x72, 0xba, 0x6a, 0x9a, 0x71, 0x10, 0xc8, 0x12, 0x39, + 0xa6, 0x09, 0xc8, 0xb2, 0xf2, 0x51, 0xd3, 0x90, 0xa9, 0x80, 0x92, 0xef, 0x51, 0x53, 0x79, 0x22, + 0xf3, 0xd9, 0x6a, 0x0e, 0x14, 0xc9, 0x27, 0xa5, 0xa9, 0x28, 0x32, 0x5f, 0x9e, 0x4e, 0x43, 0xd1, + 0x87, 0xa5, 0x44, 0x12, 0x02, 0xbd, 0x97, 0x61, 0xae, 0xd3, 0x52, 0x15, 0xd3, 0x10, 0x0c, 0xe1, + 0x8d, 0xd4, 0x80, 0x3b, 0xd5, 0xfd, 0x98, 0x14, 0x9a, 0x4f, 0x43, 0x34, 0x80, 0x8b, 0x29, 0x61, + 0x76, 0xaa, 0xe1, 0xcc, 0x0e, 0xc7, 0xa7, 0x21, 0x39, 0x82, 0xee, 0x9a, 0x63, 0xab, 0xda, 0x40, + 0x75, 0xbd, 0x47, 0x86, 0x87, 0x1d, 0x12, 0x0b, 0xfa, 0xfe, 0x5f, 0x9c, 0x6e, 0xbc, 0x41, 0xe1, + 0x02, 0xa8, 0x9c, 0x98, 0x0e, 0xa0, 0x4e, 0x59, 0x92, 0xfd, 0x3f, 0x02, 0x4a, 0xb7, 0x75, 0x21, + 0x88, 0x0c, 0x05, 0x9a, 0x06, 0xe8, 0x0b, 0xe7, 0xea, 0xd7, 0x35, 0xa8, 0xfa, 0x25, 0xf6, 0xdf, + 0x70, 0x28, 0xfa, 0x1a, 0x62, 0xc3, 0x1f, 0xc0, 0x62, 0xec, 0xb9, 0x6b, 0xaa, 0x3e, 0x4d, 0x7f, + 0x12, 0x3b, 0xed, 0xb8, 0x9e, 0xf1, 0x3f, 0x63, 0x12, 0x6e, 0xe2, 0x3b, 0x59, 0xf1, 0x65, 0xdc, + 0x43, 0x9c, 0x32, 0xf1, 0xff, 0x6d, 0xbf, 0x6c, 0x07, 0x20, 0xe4, 0x91, 0x4d, 0x2e, 0x44, 0x23, + 0x4e, 0xc6, 0x34, 0x6a, 0x99, 0xa9, 0x4e, 0xd7, 0xbb, 0x79, 0x6a, 0x7e, 0xb2, 0xcd, 0x66, 0xb6, + 0xab, 0xf5, 0x14, 0x1a, 0xe1, 0x12, 0x51, 0x94, 0xfa, 0xd7, 0x3f, 0xc9, 0x1a, 0xd2, 0x69, 0xbb, + 0xd8, 0x3e, 0xa7, 0x35, 0x9e, 0x32, 0x9d, 0x4b, 0xcc, 0x48, 0xfc, 0xee, 0x21, 0xc3, 0x8c, 0x64, + 0xdc, 0x78, 0xa4, 0x7a, 0x2f, 0xd9, 0x17, 0x1a, 0x2c, 0xcd, 0x10, 0x4f, 0xa8, 0xa7, 0xa6, 0x19, + 0x32, 0xae, 0x28, 0x52, 0xd3, 0x0c, 0x59, 0x19, 0x7a, 0xf9, 0xc2, 0xda, 0x47, 0xdf, 0xff, 0x70, + 0xa8, 0x7b, 0x47, 0xe3, 0x03, 0xb2, 0xfb, 0x07, 0x6c, 0xe8, 0xfb, 0xba, 0xcd, 0x7f, 0x3d, 0xf0, + 0xd9, 0xfd, 0x01, 0x9d, 0xed, 0x01, 0x99, 0x6d, 0x74, 0x70, 0x50, 0xa1, 0xad, 0x8f, 0xfe, 0x27, + 0x00, 0x00, 0xff, 0xff, 0x36, 0x4c, 0xc5, 0x32, 0x4e, 0x4e, 0x00, 0x00, } // Reference imports to suppress errors if they are not otherwise used. @@ -5153,6 +5203,7 @@ type DataCoordClient interface { // https://wiki.lfaidata.foundation/display/MIL/MEP+24+--+Support+bulk+load Import(ctx context.Context, in *ImportTaskRequest, opts ...grpc.CallOption) (*ImportTaskResponse, error) UpdateSegmentStatistics(ctx context.Context, in *UpdateSegmentStatisticsRequest, opts ...grpc.CallOption) (*commonpb.Status, error) + UpdateChannelCheckpoint(ctx context.Context, in *UpdateChannelCheckpointRequest, opts ...grpc.CallOption) (*commonpb.Status, error) AcquireSegmentLock(ctx context.Context, in *AcquireSegmentLockRequest, opts ...grpc.CallOption) (*commonpb.Status, error) ReleaseSegmentLock(ctx context.Context, in *ReleaseSegmentLockRequest, opts ...grpc.CallOption) (*commonpb.Status, error) SaveImportSegment(ctx context.Context, in *SaveImportSegmentRequest, opts ...grpc.CallOption) (*commonpb.Status, error) @@ -5404,6 +5455,15 @@ func (c *dataCoordClient) UpdateSegmentStatistics(ctx context.Context, in *Updat return out, nil } +func (c *dataCoordClient) UpdateChannelCheckpoint(ctx context.Context, in *UpdateChannelCheckpointRequest, opts ...grpc.CallOption) (*commonpb.Status, error) { + out := new(commonpb.Status) + err := c.cc.Invoke(ctx, "/milvus.proto.data.DataCoord/UpdateChannelCheckpoint", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + func (c *dataCoordClient) AcquireSegmentLock(ctx context.Context, in *AcquireSegmentLockRequest, opts ...grpc.CallOption) (*commonpb.Status, error) { out := new(commonpb.Status) err := c.cc.Invoke(ctx, "/milvus.proto.data.DataCoord/AcquireSegmentLock", in, out, opts...) @@ -5497,6 +5557,7 @@ type DataCoordServer interface { // https://wiki.lfaidata.foundation/display/MIL/MEP+24+--+Support+bulk+load Import(context.Context, *ImportTaskRequest) (*ImportTaskResponse, error) UpdateSegmentStatistics(context.Context, *UpdateSegmentStatisticsRequest) (*commonpb.Status, error) + UpdateChannelCheckpoint(context.Context, *UpdateChannelCheckpointRequest) (*commonpb.Status, error) AcquireSegmentLock(context.Context, *AcquireSegmentLockRequest) (*commonpb.Status, error) ReleaseSegmentLock(context.Context, *ReleaseSegmentLockRequest) (*commonpb.Status, error) SaveImportSegment(context.Context, *SaveImportSegmentRequest) (*commonpb.Status, error) @@ -5588,6 +5649,9 @@ func (*UnimplementedDataCoordServer) Import(ctx context.Context, req *ImportTask func (*UnimplementedDataCoordServer) UpdateSegmentStatistics(ctx context.Context, req *UpdateSegmentStatisticsRequest) (*commonpb.Status, error) { return nil, status.Errorf(codes.Unimplemented, "method UpdateSegmentStatistics not implemented") } +func (*UnimplementedDataCoordServer) UpdateChannelCheckpoint(ctx context.Context, req *UpdateChannelCheckpointRequest) (*commonpb.Status, error) { + return nil, status.Errorf(codes.Unimplemented, "method UpdateChannelCheckpoint not implemented") +} func (*UnimplementedDataCoordServer) AcquireSegmentLock(ctx context.Context, req *AcquireSegmentLockRequest) (*commonpb.Status, error) { return nil, status.Errorf(codes.Unimplemented, "method AcquireSegmentLock not implemented") } @@ -6082,6 +6146,24 @@ func _DataCoord_UpdateSegmentStatistics_Handler(srv interface{}, ctx context.Con return interceptor(ctx, in, info, handler) } +func _DataCoord_UpdateChannelCheckpoint_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(UpdateChannelCheckpointRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(DataCoordServer).UpdateChannelCheckpoint(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/milvus.proto.data.DataCoord/UpdateChannelCheckpoint", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(DataCoordServer).UpdateChannelCheckpoint(ctx, req.(*UpdateChannelCheckpointRequest)) + } + return interceptor(ctx, in, info, handler) +} + func _DataCoord_AcquireSegmentLock_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { in := new(AcquireSegmentLockRequest) if err := dec(in); err != nil { @@ -6316,6 +6398,10 @@ var _DataCoord_serviceDesc = grpc.ServiceDesc{ MethodName: "UpdateSegmentStatistics", Handler: _DataCoord_UpdateSegmentStatistics_Handler, }, + { + MethodName: "UpdateChannelCheckpoint", + Handler: _DataCoord_UpdateChannelCheckpoint_Handler, + }, { MethodName: "AcquireSegmentLock", Handler: _DataCoord_AcquireSegmentLock_Handler, diff --git a/internal/proxy/data_coord_mock_test.go b/internal/proxy/data_coord_mock_test.go index e7be3a5cd0..d78f07fa65 100644 --- a/internal/proxy/data_coord_mock_test.go +++ b/internal/proxy/data_coord_mock_test.go @@ -271,6 +271,13 @@ func (coord *DataCoordMock) UpdateSegmentStatistics(ctx context.Context, req *da }, nil } +func (coord *DataCoordMock) UpdateChannelCheckpoint(ctx context.Context, req *datapb.UpdateChannelCheckpointRequest) (*commonpb.Status, error) { + return &commonpb.Status{ + ErrorCode: commonpb.ErrorCode_Success, + Reason: "", + }, nil +} + func (coord *DataCoordMock) AcquireSegmentLock(ctx context.Context, req *datapb.AcquireSegmentLockRequest) (*commonpb.Status, error) { return &commonpb.Status{ ErrorCode: commonpb.ErrorCode_Success, diff --git a/internal/types/types.go b/internal/types/types.go index 954c18cbf8..baef16bfca 100644 --- a/internal/types/types.go +++ b/internal/types/types.go @@ -311,6 +311,8 @@ type DataCoord interface { // UpdateSegmentStatistics updates a segment's stats. UpdateSegmentStatistics(ctx context.Context, req *datapb.UpdateSegmentStatisticsRequest) (*commonpb.Status, error) + // UpdateChannelCheckpoint updates channel checkpoint in dataCoord. + UpdateChannelCheckpoint(ctx context.Context, req *datapb.UpdateChannelCheckpointRequest) (*commonpb.Status, error) AcquireSegmentLock(ctx context.Context, req *datapb.AcquireSegmentLockRequest) (*commonpb.Status, error) ReleaseSegmentLock(ctx context.Context, req *datapb.ReleaseSegmentLockRequest) (*commonpb.Status, error) diff --git a/internal/util/mock/grpc_datacoord_client.go b/internal/util/mock/grpc_datacoord_client.go index e017c308b3..4e28f220b7 100644 --- a/internal/util/mock/grpc_datacoord_client.go +++ b/internal/util/mock/grpc_datacoord_client.go @@ -145,6 +145,10 @@ func (m *GrpcDataCoordClient) UpdateSegmentStatistics(ctx context.Context, req * return &commonpb.Status{}, m.Err } +func (m *GrpcDataCoordClient) UpdateChannelCheckpoint(ctx context.Context, req *datapb.UpdateChannelCheckpointRequest, opts ...grpc.CallOption) (*commonpb.Status, error) { + return &commonpb.Status{}, m.Err +} + func (m *GrpcDataCoordClient) AcquireSegmentLock(ctx context.Context, req *datapb.AcquireSegmentLockRequest, opts ...grpc.CallOption) (*commonpb.Status, error) { return &commonpb.Status{}, m.Err }