mirror of
https://gitee.com/milvus-io/milvus.git
synced 2025-12-07 17:48:29 +08:00
Add channel level checkpoint (#20350)
Signed-off-by: bigsheeper <yihao.dai@zilliz.com> Signed-off-by: bigsheeper <yihao.dai@zilliz.com>
This commit is contained in:
parent
663bdf3399
commit
cd19d99ad7
@ -240,7 +240,10 @@ func (c *ChannelManager) unwatchDroppedChannels() {
|
|||||||
log.Warn("unable to remove channel", zap.String("channel", ch.Name), zap.Error(err))
|
log.Warn("unable to remove channel", zap.String("channel", ch.Name), zap.Error(err))
|
||||||
continue
|
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 {
|
if err := c.remove(nodeID, ch); err != nil {
|
||||||
return fmt.Errorf("failed to remove watch info: %v,%s", ch, err.Error())
|
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))
|
log.Info("removed channel assignment", zap.String("channel name", channelName))
|
||||||
return nil
|
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))
|
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))
|
log.Info("removed channel assignment", zap.Any("channel", chToCleanUp))
|
||||||
return nil
|
return nil
|
||||||
|
|||||||
@ -86,7 +86,7 @@ func (c *Cluster) Watch(ch string, collectionID UniqueID) error {
|
|||||||
// Flush sends flush requests to dataNodes specified
|
// Flush sends flush requests to dataNodes specified
|
||||||
// which also according to channels where segments are assigned to.
|
// which also according to channels where segments are assigned to.
|
||||||
func (c *Cluster) Flush(ctx context.Context, nodeID int64, channel string,
|
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) {
|
if !c.channelManager.Match(nodeID, channel) {
|
||||||
log.Warn("node is not matched with channel",
|
log.Warn("node is not matched with channel",
|
||||||
zap.String("channel", channel),
|
zap.String("channel", channel),
|
||||||
@ -107,9 +107,8 @@ func (c *Cluster) Flush(ctx context.Context, nodeID int64, channel string,
|
|||||||
commonpbutil.WithSourceID(paramtable.GetNodeID()),
|
commonpbutil.WithSourceID(paramtable.GetNodeID()),
|
||||||
commonpbutil.WithTargetID(nodeID),
|
commonpbutil.WithTargetID(nodeID),
|
||||||
),
|
),
|
||||||
CollectionID: ch.CollectionID,
|
CollectionID: ch.CollectionID,
|
||||||
SegmentIDs: lo.Map(segments, getSegmentID),
|
SegmentIDs: lo.Map(segments, getSegmentID),
|
||||||
MarkSegmentIDs: lo.Map(markSegments, getSegmentID),
|
|
||||||
}
|
}
|
||||||
|
|
||||||
c.sessionManager.Flush(ctx, nodeID, req)
|
c.sessionManager.Flush(ctx, nodeID, req)
|
||||||
|
|||||||
@ -534,21 +534,19 @@ func TestCluster_Flush(t *testing.T) {
|
|||||||
|
|
||||||
// flush empty should impact nothing
|
// flush empty should impact nothing
|
||||||
assert.NotPanics(t, func() {
|
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)
|
assert.NoError(t, err)
|
||||||
})
|
})
|
||||||
|
|
||||||
// flush not watched channel
|
// flush not watched channel
|
||||||
assert.NotPanics(t, func() {
|
assert.NotPanics(t, func() {
|
||||||
err := cluster.Flush(context.Background(), 1, "chan-2", []*datapb.SegmentInfo{{ID: 1}},
|
err := cluster.Flush(context.Background(), 1, "chan-2", []*datapb.SegmentInfo{{ID: 1}})
|
||||||
[]*datapb.SegmentInfo{{ID: 2}})
|
|
||||||
assert.Error(t, err)
|
assert.Error(t, err)
|
||||||
})
|
})
|
||||||
|
|
||||||
// flush from wrong datanode
|
// flush from wrong datanode
|
||||||
assert.NotPanics(t, func() {
|
assert.NotPanics(t, func() {
|
||||||
err := cluster.Flush(context.Background(), 2, "chan-1", []*datapb.SegmentInfo{{ID: 1}},
|
err := cluster.Flush(context.Background(), 2, "chan-1", []*datapb.SegmentInfo{{ID: 1}})
|
||||||
[]*datapb.SegmentInfo{{ID: 3}})
|
|
||||||
assert.Error(t, err)
|
assert.Error(t, err)
|
||||||
})
|
})
|
||||||
|
|
||||||
|
|||||||
@ -36,7 +36,7 @@ type Handler interface {
|
|||||||
// GetDataVChanPositions gets the information recovery needed of a channel for DataNode
|
// GetDataVChanPositions gets the information recovery needed of a channel for DataNode
|
||||||
GetDataVChanPositions(channel *channel, partitionID UniqueID) *datapb.VchannelInfo
|
GetDataVChanPositions(channel *channel, partitionID UniqueID) *datapb.VchannelInfo
|
||||||
CheckShouldDropChannel(channel string) bool
|
CheckShouldDropChannel(channel string) bool
|
||||||
FinishDropChannel(channel string)
|
FinishDropChannel(channel string) error
|
||||||
GetCollection(ctx context.Context, collectionID UniqueID) (*collectionInfo, 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)
|
flushedIDs = make(typeutil.UniqueSet)
|
||||||
unflushedIDs = make(typeutil.UniqueSet)
|
unflushedIDs = make(typeutil.UniqueSet)
|
||||||
droppedIDs = make(typeutil.UniqueSet)
|
droppedIDs = make(typeutil.UniqueSet)
|
||||||
seekPosition *internalpb.MsgPosition
|
|
||||||
)
|
)
|
||||||
var minPosSegID int64
|
|
||||||
var minPosTs uint64
|
|
||||||
for _, s := range segments {
|
for _, s := range segments {
|
||||||
if (partitionID > allPartitionID && s.PartitionID != partitionID) ||
|
if (partitionID > allPartitionID && s.PartitionID != partitionID) ||
|
||||||
(s.GetStartPosition() == nil && s.GetDmlPosition() == nil) {
|
(s.GetStartPosition() == nil && s.GetDmlPosition() == nil) {
|
||||||
@ -86,51 +83,12 @@ func (h *ServerHandler) GetDataVChanPositions(channel *channel, partitionID Uniq
|
|||||||
} else {
|
} else {
|
||||||
unflushedIDs.Insert(s.GetID())
|
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{
|
return &datapb.VchannelInfo{
|
||||||
CollectionID: channel.CollectionID,
|
CollectionID: channel.CollectionID,
|
||||||
ChannelName: channel.Name,
|
ChannelName: channel.Name,
|
||||||
SeekPosition: seekPosition,
|
SeekPosition: h.getChannelCheckpoint(channel),
|
||||||
FlushedSegmentIds: flushedIDs.Collect(),
|
FlushedSegmentIds: flushedIDs.Collect(),
|
||||||
UnflushedSegmentIds: unflushedIDs.Collect(),
|
UnflushedSegmentIds: unflushedIDs.Collect(),
|
||||||
DroppedSegmentIds: droppedIDs.Collect(),
|
DroppedSegmentIds: droppedIDs.Collect(),
|
||||||
@ -160,7 +118,6 @@ func (h *ServerHandler) GetQueryVChanPositions(channel *channel, partitionID Uni
|
|||||||
indexedIDs = make(typeutil.UniqueSet)
|
indexedIDs = make(typeutil.UniqueSet)
|
||||||
unIndexedIDs = make(typeutil.UniqueSet)
|
unIndexedIDs = make(typeutil.UniqueSet)
|
||||||
droppedIDs = make(typeutil.UniqueSet)
|
droppedIDs = make(typeutil.UniqueSet)
|
||||||
seekPosition *internalpb.MsgPosition
|
|
||||||
)
|
)
|
||||||
for _, s := range segments {
|
for _, s := range segments {
|
||||||
if (partitionID > allPartitionID && s.PartitionID != partitionID) ||
|
if (partitionID > allPartitionID && s.PartitionID != partitionID) ||
|
||||||
@ -191,54 +148,47 @@ func (h *ServerHandler) GetQueryVChanPositions(channel *channel, partitionID Uni
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
for id := range indexedIDs {
|
return &datapb.VchannelInfo{
|
||||||
var segmentPosition *internalpb.MsgPosition
|
CollectionID: channel.CollectionID,
|
||||||
segment := segmentInfos[id]
|
ChannelName: channel.Name,
|
||||||
if segment.GetDmlPosition() != nil {
|
SeekPosition: h.getChannelCheckpoint(channel),
|
||||||
segmentPosition = segment.GetDmlPosition()
|
FlushedSegmentIds: indexedIDs.Collect(),
|
||||||
} else {
|
UnflushedSegmentIds: unIndexedIDs.Collect(),
|
||||||
segmentPosition = segment.GetStartPosition()
|
DroppedSegmentIds: droppedIDs.Collect(),
|
||||||
}
|
|
||||||
|
|
||||||
if seekPosition == nil || segmentPosition.Timestamp < seekPosition.Timestamp {
|
|
||||||
seekPosition = segmentPosition
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
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 {
|
func (h *ServerHandler) getChannelCheckpoint(channel *channel) *internalpb.MsgPosition {
|
||||||
seekPosition = segmentPosition
|
seekPosition := h.s.meta.GetChannelCheckpoint(channel.Name)
|
||||||
}
|
if seekPosition != nil {
|
||||||
}
|
log.Info("channel seek position set from ChannelCP",
|
||||||
|
zap.String("channel", channel.Name),
|
||||||
// use collection start position when segment position is not found
|
zap.Uint64("position timestamp", seekPosition.Timestamp),
|
||||||
if seekPosition == nil {
|
zap.Time("realworld position timestamp", tsoutil.PhysicalTime(seekPosition.GetTimestamp())),
|
||||||
|
)
|
||||||
|
} else {
|
||||||
|
// use collection start position when segment position is not found
|
||||||
if channel.StartPositions == nil {
|
if channel.StartPositions == nil {
|
||||||
collection, err := h.GetCollection(h.s.ctx, channel.CollectionID)
|
collection, err := h.GetCollection(h.s.ctx, channel.CollectionID)
|
||||||
if collection != nil && err == nil {
|
if collection != nil && err == nil {
|
||||||
seekPosition = getCollectionStartPosition(channel.Name, collection)
|
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 {
|
} else {
|
||||||
// use passed start positions, skip to ask rootcoord.
|
// use passed start positions, skip to ask RootCoord.
|
||||||
seekPosition = toMsgPosition(channel.Name, channel.StartPositions)
|
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 seekPosition
|
||||||
return &datapb.VchannelInfo{
|
|
||||||
CollectionID: channel.CollectionID,
|
|
||||||
ChannelName: channel.Name,
|
|
||||||
SeekPosition: seekPosition,
|
|
||||||
FlushedSegmentIds: indexedIDs.Collect(),
|
|
||||||
UnflushedSegmentIds: unIndexedIDs.Collect(),
|
|
||||||
DroppedSegmentIds: droppedIDs.Collect(),
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
func getCollectionStartPosition(channel string, collectionInfo *collectionInfo) *internalpb.MsgPosition {
|
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
|
// FinishDropChannel cleans up the remove flag for channels
|
||||||
// this function is a wrapper of server.meta.FinishDropChannel
|
// this function is a wrapper of server.meta.FinishDropChannel
|
||||||
func (h *ServerHandler) FinishDropChannel(channel string) {
|
func (h *ServerHandler) FinishDropChannel(channel string) error {
|
||||||
h.s.meta.catalog.DropChannel(h.s.ctx, channel)
|
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
|
||||||
}
|
}
|
||||||
|
|||||||
@ -23,18 +23,12 @@ import (
|
|||||||
"sync"
|
"sync"
|
||||||
"time"
|
"time"
|
||||||
|
|
||||||
"github.com/milvus-io/milvus/internal/util/timerecord"
|
|
||||||
|
|
||||||
"golang.org/x/exp/maps"
|
|
||||||
|
|
||||||
"github.com/golang/protobuf/proto"
|
"github.com/golang/protobuf/proto"
|
||||||
|
"golang.org/x/exp/maps"
|
||||||
"github.com/milvus-io/milvus/internal/common"
|
|
||||||
"github.com/milvus-io/milvus/internal/util/typeutil"
|
|
||||||
"go.uber.org/zap"
|
|
||||||
|
|
||||||
"github.com/milvus-io/milvus-proto/go-api/commonpb"
|
"github.com/milvus-io/milvus-proto/go-api/commonpb"
|
||||||
"github.com/milvus-io/milvus-proto/go-api/schemapb"
|
"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/kv"
|
||||||
"github.com/milvus-io/milvus/internal/log"
|
"github.com/milvus-io/milvus/internal/log"
|
||||||
"github.com/milvus-io/milvus/internal/metastore"
|
"github.com/milvus-io/milvus/internal/metastore"
|
||||||
@ -42,14 +36,19 @@ import (
|
|||||||
"github.com/milvus-io/milvus/internal/metrics"
|
"github.com/milvus-io/milvus/internal/metrics"
|
||||||
"github.com/milvus-io/milvus/internal/proto/datapb"
|
"github.com/milvus-io/milvus/internal/proto/datapb"
|
||||||
"github.com/milvus-io/milvus/internal/proto/internalpb"
|
"github.com/milvus-io/milvus/internal/proto/internalpb"
|
||||||
|
"github.com/milvus-io/milvus/internal/util/timerecord"
|
||||||
|
"github.com/milvus-io/milvus/internal/util/tsoutil"
|
||||||
|
"github.com/milvus-io/milvus/internal/util/typeutil"
|
||||||
|
"go.uber.org/zap"
|
||||||
)
|
)
|
||||||
|
|
||||||
type meta struct {
|
type meta struct {
|
||||||
sync.RWMutex
|
sync.RWMutex
|
||||||
ctx context.Context
|
ctx context.Context
|
||||||
catalog metastore.DataCoordCatalog
|
catalog metastore.DataCoordCatalog
|
||||||
collections map[UniqueID]*collectionInfo // collection id to collection info
|
collections map[UniqueID]*collectionInfo // collection id to collection info
|
||||||
segments *SegmentsInfo // segment id to segment info
|
segments *SegmentsInfo // segment id to segment info
|
||||||
|
channelCPs map[string]*internalpb.MsgPosition // vChannel -> channel checkpoint/see position
|
||||||
}
|
}
|
||||||
|
|
||||||
type collectionInfo struct {
|
type collectionInfo struct {
|
||||||
@ -67,6 +66,7 @@ func newMeta(ctx context.Context, kv kv.TxnKV, chunkManagerRootPath string) (*me
|
|||||||
catalog: &datacoord.Catalog{Txn: kv, ChunkManagerRootPath: chunkManagerRootPath},
|
catalog: &datacoord.Catalog{Txn: kv, ChunkManagerRootPath: chunkManagerRootPath},
|
||||||
collections: make(map[UniqueID]*collectionInfo),
|
collections: make(map[UniqueID]*collectionInfo),
|
||||||
segments: NewSegmentsInfo(),
|
segments: NewSegmentsInfo(),
|
||||||
|
channelCPs: make(map[string]*internalpb.MsgPosition),
|
||||||
}
|
}
|
||||||
err := mt.reloadFromKV()
|
err := mt.reloadFromKV()
|
||||||
if err != nil {
|
if err != nil {
|
||||||
@ -99,6 +99,14 @@ func (m *meta) reloadFromKV() error {
|
|||||||
}
|
}
|
||||||
metrics.DataCoordNumStoredRows.WithLabelValues().Set(float64(numStoredRows))
|
metrics.DataCoordNumStoredRows.WithLabelValues().Set(float64(numStoredRows))
|
||||||
metrics.DataCoordNumStoredRowsCounter.WithLabelValues().Add(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
|
||||||
|
}
|
||||||
record.Record("meta reloadFromKV")
|
record.Record("meta reloadFromKV")
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
@ -1128,3 +1136,46 @@ func (m *meta) GetCompactionTo(segmentID int64) *SegmentInfo {
|
|||||||
}
|
}
|
||||||
return nil
|
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
|
||||||
|
}
|
||||||
|
|||||||
@ -57,6 +57,11 @@ func (mek *mockEtcdKv) LoadWithPrefix(key string) ([]string, []string, error) {
|
|||||||
case strings.Contains(key, datacoord.SegmentStatslogPathPrefix):
|
case strings.Contains(key, datacoord.SegmentStatslogPathPrefix):
|
||||||
segInfo := getFieldBinlogPaths(1, "statslog1")
|
segInfo := getFieldBinlogPaths(1, "statslog1")
|
||||||
val, _ = proto.Marshal(segInfo)
|
val, _ = proto.Marshal(segInfo)
|
||||||
|
case strings.Contains(key, datacoord.ChannelCheckpointPrefix):
|
||||||
|
channelCP := &internalpb.MsgPosition{
|
||||||
|
Timestamp: 1000,
|
||||||
|
}
|
||||||
|
val, _ = proto.Marshal(channelCP)
|
||||||
default:
|
default:
|
||||||
return nil, nil, fmt.Errorf("invalid key")
|
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.Properties, b.Properties)
|
||||||
assert.Equal(t, a.StartPositions, b.StartPositions)
|
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)
|
||||||
|
})
|
||||||
|
}
|
||||||
|
|||||||
@ -759,7 +759,9 @@ func (h *mockHandler) CheckShouldDropChannel(channel string) bool {
|
|||||||
return false
|
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) {
|
func (h *mockHandler) GetCollection(_ context.Context, collectionID UniqueID) (*collectionInfo, error) {
|
||||||
// empty schema
|
// empty schema
|
||||||
|
|||||||
@ -573,29 +573,21 @@ func (s *Server) handleTimetickMessage(ctx context.Context, ttMsg *msgstream.Dat
|
|||||||
}
|
}
|
||||||
flushableSegments := s.getFlushableSegmentsInfo(flushableIDs)
|
flushableSegments := s.getFlushableSegmentsInfo(flushableIDs)
|
||||||
|
|
||||||
staleSegments := s.getStaleSegmentsInfo(ch)
|
if len(flushableSegments) == 0 {
|
||||||
staleSegments = s.filterWithFlushableSegments(staleSegments, flushableIDs)
|
|
||||||
|
|
||||||
if len(flushableSegments)+len(staleSegments) == 0 {
|
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
|
||||||
log.Info("start flushing segments",
|
log.Info("start flushing segments",
|
||||||
zap.Int64s("segment IDs", flushableIDs),
|
zap.Int64s("segment IDs", flushableIDs))
|
||||||
zap.Int("# of stale/mark segments", len(staleSegments)))
|
|
||||||
// update segment last update triggered time
|
// update segment last update triggered time
|
||||||
// it's ok to fail flushing, since next timetick after duration will re-trigger
|
// it's ok to fail flushing, since next timetick after duration will re-trigger
|
||||||
s.setLastFlushTime(flushableSegments)
|
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 {
|
for _, info := range flushableSegments {
|
||||||
finfo = append(finfo, info.SegmentInfo)
|
finfo = append(finfo, info.SegmentInfo)
|
||||||
}
|
}
|
||||||
for _, info := range staleSegments {
|
err = s.cluster.Flush(s.ctx, ttMsg.GetBase().GetSourceID(), ch, finfo)
|
||||||
minfo = append(minfo, info.SegmentInfo)
|
|
||||||
}
|
|
||||||
err = s.cluster.Flush(s.ctx, ttMsg.GetBase().GetSourceID(), ch, finfo, minfo)
|
|
||||||
if err != nil {
|
if err != nil {
|
||||||
log.Warn("handle")
|
log.Warn("handle")
|
||||||
return err
|
return err
|
||||||
@ -632,32 +624,6 @@ func (s *Server) getFlushableSegmentsInfo(flushableIDs []int64) []*SegmentInfo {
|
|||||||
return res
|
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) {
|
func (s *Server) setLastFlushTime(segments []*SegmentInfo) {
|
||||||
for _, sinfo := range segments {
|
for _, sinfo := range segments {
|
||||||
s.meta.SetLastFlushTime(sinfo.GetID(), time.Now())
|
s.meta.SetLastFlushTime(sinfo.GetID(), time.Now())
|
||||||
|
|||||||
@ -1646,6 +1646,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) {
|
func TestGetDataVChanPositions(t *testing.T) {
|
||||||
svr := newTestServer(t, nil)
|
svr := newTestServer(t, nil)
|
||||||
defer closeTestServer(t, svr)
|
defer closeTestServer(t, svr)
|
||||||
@ -1733,7 +1790,6 @@ func TestGetDataVChanPositions(t *testing.T) {
|
|||||||
assert.EqualValues(t, 1, vchan.FlushedSegmentIds[0])
|
assert.EqualValues(t, 1, vchan.FlushedSegmentIds[0])
|
||||||
assert.EqualValues(t, 2, len(vchan.UnflushedSegmentIds))
|
assert.EqualValues(t, 2, len(vchan.UnflushedSegmentIds))
|
||||||
assert.ElementsMatch(t, []int64{s2.ID, s3.ID}, 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) {
|
t.Run("empty collection", func(t *testing.T) {
|
||||||
@ -1741,7 +1797,6 @@ func TestGetDataVChanPositions(t *testing.T) {
|
|||||||
assert.EqualValues(t, 1, infos.CollectionID)
|
assert.EqualValues(t, 1, infos.CollectionID)
|
||||||
assert.EqualValues(t, 0, len(infos.FlushedSegmentIds))
|
assert.EqualValues(t, 0, len(infos.FlushedSegmentIds))
|
||||||
assert.EqualValues(t, 0, len(infos.UnflushedSegmentIds))
|
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) {
|
t.Run("filter partition", func(t *testing.T) {
|
||||||
@ -1749,7 +1804,6 @@ func TestGetDataVChanPositions(t *testing.T) {
|
|||||||
assert.EqualValues(t, 0, infos.CollectionID)
|
assert.EqualValues(t, 0, infos.CollectionID)
|
||||||
assert.EqualValues(t, 0, len(infos.FlushedSegmentIds))
|
assert.EqualValues(t, 0, len(infos.FlushedSegmentIds))
|
||||||
assert.EqualValues(t, 1, len(infos.UnflushedSegmentIds))
|
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) {
|
t.Run("empty collection with passed positions", func(t *testing.T) {
|
||||||
@ -1762,7 +1816,6 @@ func TestGetDataVChanPositions(t *testing.T) {
|
|||||||
}, allPartitionID)
|
}, allPartitionID)
|
||||||
assert.EqualValues(t, 0, infos.CollectionID)
|
assert.EqualValues(t, 0, infos.CollectionID)
|
||||||
assert.EqualValues(t, vchannel, infos.ChannelName)
|
assert.EqualValues(t, vchannel, infos.ChannelName)
|
||||||
assert.EqualValues(t, []byte{14, 15, 16}, infos.SeekPosition.MsgID)
|
|
||||||
})
|
})
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -1877,7 +1930,6 @@ func TestGetQueryVChanPositions(t *testing.T) {
|
|||||||
assert.EqualValues(t, 1, vchan.FlushedSegmentIds[0])
|
assert.EqualValues(t, 1, vchan.FlushedSegmentIds[0])
|
||||||
assert.EqualValues(t, 2, len(vchan.UnflushedSegmentIds))
|
assert.EqualValues(t, 2, len(vchan.UnflushedSegmentIds))
|
||||||
assert.ElementsMatch(t, []int64{s2.ID, s3.ID}, 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) {
|
t.Run("empty collection", func(t *testing.T) {
|
||||||
@ -1885,7 +1937,6 @@ func TestGetQueryVChanPositions(t *testing.T) {
|
|||||||
assert.EqualValues(t, 1, infos.CollectionID)
|
assert.EqualValues(t, 1, infos.CollectionID)
|
||||||
assert.EqualValues(t, 0, len(infos.FlushedSegmentIds))
|
assert.EqualValues(t, 0, len(infos.FlushedSegmentIds))
|
||||||
assert.EqualValues(t, 0, len(infos.UnflushedSegmentIds))
|
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) {
|
t.Run("filter partition", func(t *testing.T) {
|
||||||
@ -1893,7 +1944,6 @@ func TestGetQueryVChanPositions(t *testing.T) {
|
|||||||
assert.EqualValues(t, 0, infos.CollectionID)
|
assert.EqualValues(t, 0, infos.CollectionID)
|
||||||
assert.EqualValues(t, 0, len(infos.FlushedSegmentIds))
|
assert.EqualValues(t, 0, len(infos.FlushedSegmentIds))
|
||||||
assert.EqualValues(t, 1, len(infos.UnflushedSegmentIds))
|
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) {
|
t.Run("empty collection with passed positions", func(t *testing.T) {
|
||||||
@ -1906,7 +1956,6 @@ func TestGetQueryVChanPositions(t *testing.T) {
|
|||||||
}, allPartitionID)
|
}, allPartitionID)
|
||||||
assert.EqualValues(t, 0, infos.CollectionID)
|
assert.EqualValues(t, 0, infos.CollectionID)
|
||||||
assert.EqualValues(t, vchannel, infos.ChannelName)
|
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) {
|
t.Run("filter non indexed segments", func(t *testing.T) {
|
||||||
@ -1918,7 +1967,6 @@ func TestGetQueryVChanPositions(t *testing.T) {
|
|||||||
assert.EqualValues(t, 0, len(vchan.FlushedSegmentIds))
|
assert.EqualValues(t, 0, len(vchan.FlushedSegmentIds))
|
||||||
assert.EqualValues(t, 3, len(vchan.UnflushedSegmentIds))
|
assert.EqualValues(t, 3, len(vchan.UnflushedSegmentIds))
|
||||||
assert.ElementsMatch(t, []int64{s1.ID, s2.ID, s3.ID}, vchan.UnflushedSegmentIds)
|
assert.ElementsMatch(t, []int64{s1.ID, s2.ID, s3.ID}, vchan.UnflushedSegmentIds)
|
||||||
assert.EqualValues(t, []byte{1, 2, 3}, vchan.GetSeekPosition().GetMsgID())
|
|
||||||
})
|
})
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -2125,9 +2173,16 @@ func TestGetRecoveryInfo(t *testing.T) {
|
|||||||
svr.meta.AddCollection(&collectionInfo{
|
svr.meta.AddCollection(&collectionInfo{
|
||||||
Schema: newTestSchema(),
|
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)
|
seg1 := createSegment(0, 0, 0, 100, 10, "vchan1", commonpb.SegmentState_Flushed)
|
||||||
seg2 := createSegment(1, 0, 0, 100, 20, "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)
|
assert.Nil(t, err)
|
||||||
err = svr.meta.AddSegment(NewSegmentInfo(seg2))
|
err = svr.meta.AddSegment(NewSegmentInfo(seg2))
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
@ -2187,9 +2242,15 @@ func TestGetRecoveryInfo(t *testing.T) {
|
|||||||
Schema: newTestSchema(),
|
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)
|
seg1 := createSegment(3, 0, 0, 100, 30, "vchan1", commonpb.SegmentState_Growing)
|
||||||
seg2 := createSegment(4, 0, 0, 100, 40, "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)
|
assert.Nil(t, err)
|
||||||
err = svr.meta.AddSegment(NewSegmentInfo(seg2))
|
err = svr.meta.AddSegment(NewSegmentInfo(seg2))
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
@ -2321,9 +2382,15 @@ func TestGetRecoveryInfo(t *testing.T) {
|
|||||||
Schema: newTestSchema(),
|
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)
|
seg1 := createSegment(7, 0, 0, 100, 30, "vchan1", commonpb.SegmentState_Growing)
|
||||||
seg2 := createSegment(8, 0, 0, 100, 40, "vchan1", commonpb.SegmentState_Dropped)
|
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)
|
assert.Nil(t, err)
|
||||||
err = svr.meta.AddSegment(NewSegmentInfo(seg2))
|
err = svr.meta.AddSegment(NewSegmentInfo(seg2))
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
@ -3138,6 +3205,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: paramtable.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
|
// https://github.com/milvus-io/milvus/issues/15659
|
||||||
func TestIssue15659(t *testing.T) {
|
func TestIssue15659(t *testing.T) {
|
||||||
ctx, cancel := context.WithCancel(context.Background())
|
ctx, cancel := context.WithCancel(context.Background())
|
||||||
|
|||||||
@ -1170,6 +1170,29 @@ func (s *Server) UpdateSegmentStatistics(ctx context.Context, req *datapb.Update
|
|||||||
}, nil
|
}, 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(paramtable.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`.
|
// 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 {
|
func getDiff(base, remove []int64) []int64 {
|
||||||
mb := make(map[int64]struct{}, len(remove))
|
mb := make(map[int64]struct{}, len(remove))
|
||||||
|
|||||||
@ -21,16 +21,16 @@ import (
|
|||||||
"errors"
|
"errors"
|
||||||
"fmt"
|
"fmt"
|
||||||
"math"
|
"math"
|
||||||
"sync"
|
|
||||||
|
"go.uber.org/zap"
|
||||||
|
|
||||||
"github.com/milvus-io/milvus-proto/go-api/schemapb"
|
"github.com/milvus-io/milvus-proto/go-api/schemapb"
|
||||||
"github.com/milvus-io/milvus/internal/log"
|
"github.com/milvus-io/milvus/internal/log"
|
||||||
"github.com/milvus-io/milvus/internal/metrics"
|
"github.com/milvus-io/milvus/internal/metrics"
|
||||||
"github.com/milvus-io/milvus/internal/proto/datapb"
|
"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/storage"
|
||||||
"github.com/milvus-io/milvus/internal/util/paramtable"
|
"github.com/milvus-io/milvus/internal/util/paramtable"
|
||||||
|
|
||||||
"go.uber.org/zap"
|
|
||||||
)
|
)
|
||||||
|
|
||||||
// DelBufferManager is in charge of managing insertBuf and delBuf from an overall prospect
|
// DelBufferManager is in charge of managing insertBuf and delBuf from an overall prospect
|
||||||
@ -38,37 +38,37 @@ import (
|
|||||||
// insert/delete flush when the memory usage of the whole manager reach a certain level.
|
// 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
|
// but at the first stage, this struct is only used for delete buff
|
||||||
type DelBufferManager struct {
|
type DelBufferManager struct {
|
||||||
delBufMap sync.Map // map[segmentID]*DelDataBuf
|
channel Channel
|
||||||
delMemorySize int64
|
delMemorySize int64
|
||||||
delBufHeap *PriorityQueue
|
delBufHeap *PriorityQueue
|
||||||
}
|
}
|
||||||
|
|
||||||
func (bm *DelBufferManager) GetSegDelBufMemSize(segID UniqueID) int64 {
|
func (bm *DelBufferManager) GetSegDelBufMemSize(segID UniqueID) int64 {
|
||||||
if delDataBuf, ok := bm.delBufMap.Load(segID); ok {
|
if delDataBuf, ok := bm.channel.getCurDeleteBuffer(segID); ok {
|
||||||
return delDataBuf.(*DelDataBuf).item.memorySize
|
return delDataBuf.item.memorySize
|
||||||
}
|
}
|
||||||
return 0
|
return 0
|
||||||
}
|
}
|
||||||
|
|
||||||
func (bm *DelBufferManager) GetEntriesNum(segID UniqueID) int64 {
|
func (bm *DelBufferManager) GetEntriesNum(segID UniqueID) int64 {
|
||||||
if delDataBuf, ok := bm.delBufMap.Load(segID); ok {
|
if delDataBuf, ok := bm.channel.getCurDeleteBuffer(segID); ok {
|
||||||
return delDataBuf.(*DelDataBuf).GetEntriesNum()
|
return delDataBuf.GetEntriesNum()
|
||||||
}
|
}
|
||||||
return 0
|
return 0
|
||||||
}
|
}
|
||||||
|
|
||||||
// Store :the method only for unit test
|
// Store :the method only for unit test
|
||||||
func (bm *DelBufferManager) Store(segID UniqueID, delDataBuf *DelDataBuf) {
|
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,
|
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
|
//1. load or create delDataBuf
|
||||||
var delDataBuf *DelDataBuf
|
var delDataBuf *DelDataBuf
|
||||||
value, loaded := bm.delBufMap.Load(segID)
|
buffer, loaded := bm.channel.getCurDeleteBuffer(segID)
|
||||||
if loaded {
|
if loaded {
|
||||||
delDataBuf = value.(*DelDataBuf)
|
delDataBuf = buffer
|
||||||
} else {
|
} else {
|
||||||
delDataBuf = newDelDataBuf()
|
delDataBuf = newDelDataBuf()
|
||||||
}
|
}
|
||||||
@ -93,6 +93,7 @@ func (bm *DelBufferManager) StoreNewDeletes(segID UniqueID, pks []primaryKey,
|
|||||||
//3. update statistics of del data
|
//3. update statistics of del data
|
||||||
delDataBuf.accumulateEntriesNum(int64(rowCount))
|
delDataBuf.accumulateEntriesNum(int64(rowCount))
|
||||||
delDataBuf.updateTimeRange(tr)
|
delDataBuf.updateTimeRange(tr)
|
||||||
|
delDataBuf.updateStartAndEndPosition(startPos, endPos)
|
||||||
|
|
||||||
//4. update and sync memory size with priority queue
|
//4. update and sync memory size with priority queue
|
||||||
if !loaded {
|
if !loaded {
|
||||||
@ -102,7 +103,7 @@ func (bm *DelBufferManager) StoreNewDeletes(segID UniqueID, pks []primaryKey,
|
|||||||
} else {
|
} else {
|
||||||
bm.delBufHeap.update(delDataBuf.item, delDataBuf.item.memorySize+bufSize)
|
bm.delBufHeap.update(delDataBuf.item, delDataBuf.item.memorySize+bufSize)
|
||||||
}
|
}
|
||||||
bm.delBufMap.Store(segID, delDataBuf)
|
bm.channel.setCurDeleteBuffer(segID, delDataBuf)
|
||||||
bm.delMemorySize += bufSize
|
bm.delMemorySize += bufSize
|
||||||
//4. sync metrics
|
//4. sync metrics
|
||||||
metrics.DataNodeConsumeMsgRowsCount.WithLabelValues(
|
metrics.DataNodeConsumeMsgRowsCount.WithLabelValues(
|
||||||
@ -110,19 +111,19 @@ func (bm *DelBufferManager) StoreNewDeletes(segID UniqueID, pks []primaryKey,
|
|||||||
}
|
}
|
||||||
|
|
||||||
func (bm *DelBufferManager) Load(segID UniqueID) (delDataBuf *DelDataBuf, ok bool) {
|
func (bm *DelBufferManager) Load(segID UniqueID) (delDataBuf *DelDataBuf, ok bool) {
|
||||||
value, ok := bm.delBufMap.Load(segID)
|
buffer, ok := bm.channel.getCurDeleteBuffer(segID)
|
||||||
if ok {
|
if ok {
|
||||||
return value.(*DelDataBuf), ok
|
return buffer, ok
|
||||||
}
|
}
|
||||||
return nil, ok
|
return nil, ok
|
||||||
}
|
}
|
||||||
|
|
||||||
func (bm *DelBufferManager) Delete(segID UniqueID) {
|
func (bm *DelBufferManager) Delete(segID UniqueID) {
|
||||||
if buf, ok := bm.delBufMap.Load(segID); ok {
|
if buf, ok := bm.channel.getCurDeleteBuffer(segID); ok {
|
||||||
item := buf.(*DelDataBuf).item
|
item := buf.item
|
||||||
bm.delMemorySize -= item.memorySize
|
bm.delMemorySize -= item.memorySize
|
||||||
heap.Remove(bm.delBufHeap, item.index)
|
heap.Remove(bm.delBufHeap, item.index)
|
||||||
bm.delBufMap.Delete(segID)
|
bm.channel.rollDeleteBuffer(segID)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -156,7 +157,7 @@ func (bm *DelBufferManager) CompactSegBuf(compactedToSegID UniqueID, compactedFr
|
|||||||
//note that when compacting segment in del buffer manager
|
//note that when compacting segment in del buffer manager
|
||||||
//there is no need to modify the general memory size as there is no new
|
//there is no need to modify the general memory size as there is no new
|
||||||
//added del into the memory
|
//added del into the memory
|
||||||
bm.delBufMap.Store(compactedToSegID, compactToDelBuff)
|
bm.channel.setCurDeleteBuffer(compactedToSegID, compactToDelBuff)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -238,11 +239,13 @@ func (pq *PriorityQueue) update(item *Item, memorySize int64) {
|
|||||||
// BufferData buffers insert data, monitoring buffer size and limit
|
// BufferData buffers insert data, monitoring buffer size and limit
|
||||||
// size and limit both indicate numOfRows
|
// size and limit both indicate numOfRows
|
||||||
type BufferData struct {
|
type BufferData struct {
|
||||||
buffer *InsertData
|
buffer *InsertData
|
||||||
size int64
|
size int64
|
||||||
limit int64
|
limit int64
|
||||||
tsFrom Timestamp
|
tsFrom Timestamp
|
||||||
tsTo Timestamp
|
tsTo Timestamp
|
||||||
|
startPos *internalpb.MsgPosition
|
||||||
|
endPos *internalpb.MsgPosition
|
||||||
}
|
}
|
||||||
|
|
||||||
func (bd *BufferData) effectiveCap() int64 {
|
func (bd *BufferData) effectiveCap() int64 {
|
||||||
@ -263,12 +266,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
|
// DelDataBuf buffers delete data, monitoring buffer size and limit
|
||||||
// size and limit both indicate numOfRows
|
// size and limit both indicate numOfRows
|
||||||
type DelDataBuf struct {
|
type DelDataBuf struct {
|
||||||
datapb.Binlog
|
datapb.Binlog
|
||||||
delData *DeleteData
|
delData *DeleteData
|
||||||
item *Item
|
item *Item
|
||||||
|
startPos *internalpb.MsgPosition
|
||||||
|
endPos *internalpb.MsgPosition
|
||||||
}
|
}
|
||||||
|
|
||||||
func (ddb *DelDataBuf) accumulateEntriesNum(entryNum int64) {
|
func (ddb *DelDataBuf) accumulateEntriesNum(entryNum int64) {
|
||||||
@ -295,6 +309,15 @@ func (ddb *DelDataBuf) mergeDelDataBuf(buf *DelDataBuf) {
|
|||||||
ddb.item.memorySize += buf.item.memorySize
|
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
|
// newBufferData needs an input dimension to calculate the limit of this buffer
|
||||||
//
|
//
|
||||||
// `limit` is the segment numOfRows a buffer can buffer at most.
|
// `limit` is the segment numOfRows a buffer can buffer at most.
|
||||||
@ -313,12 +336,29 @@ func (ddb *DelDataBuf) mergeDelDataBuf(buf *DelDataBuf) {
|
|||||||
// to fit in both types of vector fields
|
// to fit in both types of vector fields
|
||||||
//
|
//
|
||||||
// * This need to change for string field support and multi-vector fields support.
|
// * 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 {
|
if dimension == 0 {
|
||||||
return nil, errors.New("Invalid dimension")
|
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
|
//TODO::xige-16 eval vec and string field
|
||||||
return &BufferData{
|
return &BufferData{
|
||||||
|
|||||||
@ -17,13 +17,38 @@
|
|||||||
package datanode
|
package datanode
|
||||||
|
|
||||||
import (
|
import (
|
||||||
|
"fmt"
|
||||||
"math"
|
"math"
|
||||||
"testing"
|
"testing"
|
||||||
|
|
||||||
"github.com/stretchr/testify/assert"
|
"github.com/stretchr/testify/assert"
|
||||||
"github.com/stretchr/testify/require"
|
"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) {
|
func TestBufferData(t *testing.T) {
|
||||||
Params.DataNodeCfg.FlushInsertBufferSize = 16 * (1 << 20) // 16 MB
|
Params.DataNodeCfg.FlushInsertBufferSize = 16 * (1 << 20) // 16 MB
|
||||||
|
|
||||||
@ -43,7 +68,7 @@ func TestBufferData(t *testing.T) {
|
|||||||
|
|
||||||
for _, test := range tests {
|
for _, test := range tests {
|
||||||
t.Run(test.description, func(t *testing.T) {
|
t.Run(test.description, func(t *testing.T) {
|
||||||
idata, err := newBufferData(test.indim)
|
idata, err := newBufferData(genTestCollectionSchema(test.indim))
|
||||||
|
|
||||||
if test.isValid {
|
if test.isValid {
|
||||||
assert.NoError(t, err)
|
assert.NoError(t, err)
|
||||||
@ -101,7 +126,7 @@ func TestBufferData_updateTimeRange(t *testing.T) {
|
|||||||
|
|
||||||
for _, tc := range cases {
|
for _, tc := range cases {
|
||||||
t.Run(tc.tag, func(t *testing.T) {
|
t.Run(tc.tag, func(t *testing.T) {
|
||||||
bd, err := newBufferData(16)
|
bd, err := newBufferData(genTestCollectionSchema(16))
|
||||||
require.NoError(t, err)
|
require.NoError(t, err)
|
||||||
for _, tr := range tc.trs {
|
for _, tr := range tc.trs {
|
||||||
bd.updateTimeRange(tr)
|
bd.updateTimeRange(tr)
|
||||||
|
|||||||
@ -19,9 +19,12 @@ package datanode
|
|||||||
import (
|
import (
|
||||||
"context"
|
"context"
|
||||||
"fmt"
|
"fmt"
|
||||||
|
"math"
|
||||||
"sync"
|
"sync"
|
||||||
"time"
|
"time"
|
||||||
|
|
||||||
|
"go.uber.org/zap"
|
||||||
|
|
||||||
"github.com/milvus-io/milvus-proto/go-api/schemapb"
|
"github.com/milvus-io/milvus-proto/go-api/schemapb"
|
||||||
"github.com/milvus-io/milvus/internal/common"
|
"github.com/milvus-io/milvus/internal/common"
|
||||||
"github.com/milvus-io/milvus/internal/log"
|
"github.com/milvus-io/milvus/internal/log"
|
||||||
@ -32,7 +35,6 @@ import (
|
|||||||
"github.com/milvus-io/milvus/internal/types"
|
"github.com/milvus-io/milvus/internal/types"
|
||||||
"github.com/milvus-io/milvus/internal/util/paramtable"
|
"github.com/milvus-io/milvus/internal/util/paramtable"
|
||||||
"github.com/milvus-io/milvus/internal/util/typeutil"
|
"github.com/milvus-io/milvus/internal/util/typeutil"
|
||||||
"go.uber.org/zap"
|
|
||||||
)
|
)
|
||||||
|
|
||||||
type (
|
type (
|
||||||
@ -60,18 +62,31 @@ type Channel interface {
|
|||||||
filterSegments(partitionID UniqueID) []*Segment
|
filterSegments(partitionID UniqueID) []*Segment
|
||||||
listNewSegmentsStartPositions() []*datapb.SegmentStartPosition
|
listNewSegmentsStartPositions() []*datapb.SegmentStartPosition
|
||||||
transferNewSegments(segmentIDs []UniqueID)
|
transferNewSegments(segmentIDs []UniqueID)
|
||||||
updateSegmentEndPosition(segID UniqueID, endPos *internalpb.MsgPosition)
|
|
||||||
updateSegmentPKRange(segID UniqueID, ids storage.FieldData)
|
updateSegmentPKRange(segID UniqueID, ids storage.FieldData)
|
||||||
mergeFlushedSegments(seg *Segment, planID UniqueID, compactedFrom []UniqueID) error
|
mergeFlushedSegments(seg *Segment, planID UniqueID, compactedFrom []UniqueID) error
|
||||||
hasSegment(segID UniqueID, countFlushed bool) bool
|
hasSegment(segID UniqueID, countFlushed bool) bool
|
||||||
removeSegments(segID ...UniqueID)
|
removeSegments(segID ...UniqueID)
|
||||||
listCompactedSegmentIDs() map[UniqueID][]UniqueID
|
listCompactedSegmentIDs() map[UniqueID][]UniqueID
|
||||||
|
listSegmentIDsToSync(ts Timestamp) []UniqueID
|
||||||
|
setSegmentLastSyncTs(segID UniqueID, ts Timestamp)
|
||||||
|
|
||||||
updateStatistics(segID UniqueID, numRows int64)
|
updateStatistics(segID UniqueID, numRows int64)
|
||||||
InitPKstats(ctx context.Context, s *Segment, statsBinlogs []*datapb.FieldBinlog, ts Timestamp) error
|
InitPKstats(ctx context.Context, s *Segment, statsBinlogs []*datapb.FieldBinlog, ts Timestamp) error
|
||||||
RollPKstats(segID UniqueID, stats []*storage.PrimaryKeyStats)
|
RollPKstats(segID UniqueID, stats []*storage.PrimaryKeyStats)
|
||||||
getSegmentStatisticsUpdates(segID UniqueID) (*datapb.SegmentStats, error)
|
getSegmentStatisticsUpdates(segID UniqueID) (*datapb.SegmentStats, error)
|
||||||
segmentFlushed(segID UniqueID)
|
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.
|
// ChannelMeta contains channel meta and the latest segments infos of the channel.
|
||||||
@ -84,6 +99,8 @@ type ChannelMeta struct {
|
|||||||
segMu sync.RWMutex
|
segMu sync.RWMutex
|
||||||
segments map[UniqueID]*Segment
|
segments map[UniqueID]*Segment
|
||||||
|
|
||||||
|
syncPolicies []segmentSyncPolicy
|
||||||
|
|
||||||
metaService *metaService
|
metaService *metaService
|
||||||
chunkManager storage.ChunkManager
|
chunkManager storage.ChunkManager
|
||||||
}
|
}
|
||||||
@ -100,6 +117,10 @@ func newChannel(channelName string, collID UniqueID, schema *schemapb.Collection
|
|||||||
|
|
||||||
segments: make(map[UniqueID]*Segment),
|
segments: make(map[UniqueID]*Segment),
|
||||||
|
|
||||||
|
syncPolicies: []segmentSyncPolicy{
|
||||||
|
syncPeriodically(),
|
||||||
|
},
|
||||||
|
|
||||||
metaService: metaService,
|
metaService: metaService,
|
||||||
chunkManager: cm,
|
chunkManager: cm,
|
||||||
}
|
}
|
||||||
@ -179,14 +200,15 @@ func (c *ChannelMeta) addSegment(req addSegmentReq) error {
|
|||||||
zap.Bool("importing", req.importing),
|
zap.Bool("importing", req.importing),
|
||||||
)
|
)
|
||||||
seg := &Segment{
|
seg := &Segment{
|
||||||
collectionID: req.collID,
|
collectionID: req.collID,
|
||||||
partitionID: req.partitionID,
|
partitionID: req.partitionID,
|
||||||
segmentID: req.segID,
|
segmentID: req.segID,
|
||||||
numRows: req.numOfRows, // 0 if segType == NEW
|
numRows: req.numOfRows, // 0 if segType == NEW
|
||||||
startPos: req.startPos,
|
historyInsertBuf: make([]*BufferData, 0),
|
||||||
endPos: req.endPos,
|
historyDeleteBuf: make([]*DelDataBuf, 0),
|
||||||
|
startPos: req.startPos,
|
||||||
}
|
}
|
||||||
seg.sType.Store(req.segType)
|
seg.setType(req.segType)
|
||||||
// Set up pk stats
|
// Set up pk stats
|
||||||
err := c.InitPKstats(context.TODO(), seg, req.statsBinLogs, req.recoverTs)
|
err := c.InitPKstats(context.TODO(), seg, req.statsBinLogs, req.recoverTs)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
@ -219,6 +241,33 @@ func (c *ChannelMeta) listCompactedSegmentIDs() map[UniqueID][]UniqueID {
|
|||||||
return compactedTo2From
|
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
|
// filterSegments return segments with same partitionID for all segments
|
||||||
// get all segments
|
// get all segments
|
||||||
func (c *ChannelMeta) filterSegments(partitionID UniqueID) []*Segment {
|
func (c *ChannelMeta) filterSegments(partitionID UniqueID) []*Segment {
|
||||||
@ -356,20 +405,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) {
|
func (c *ChannelMeta) updateSegmentPKRange(segID UniqueID, ids storage.FieldData) {
|
||||||
c.segMu.Lock()
|
c.segMu.Lock()
|
||||||
defer c.segMu.Unlock()
|
defer c.segMu.Unlock()
|
||||||
@ -390,10 +425,15 @@ func (c *ChannelMeta) removeSegments(segIDs ...UniqueID) {
|
|||||||
log.Info("remove segments if exist", zap.Int64s("segmentIDs", segIDs))
|
log.Info("remove segments if exist", zap.Int64s("segmentIDs", segIDs))
|
||||||
cnt := 0
|
cnt := 0
|
||||||
for _, segID := range segIDs {
|
for _, segID := range segIDs {
|
||||||
seg, ok := c.segments[segID]
|
if seg, ok := c.segments[segID]; ok {
|
||||||
if ok &&
|
if seg.notFlushed() {
|
||||||
(seg.getType() == datapb.SegmentType_New || seg.getType() == datapb.SegmentType_Normal) {
|
cnt++
|
||||||
cnt++
|
}
|
||||||
|
// free memory
|
||||||
|
seg.curInsertBuf = nil
|
||||||
|
seg.curDeleteBuf = nil
|
||||||
|
seg.historyInsertBuf = nil
|
||||||
|
seg.historyDeleteBuf = nil
|
||||||
}
|
}
|
||||||
|
|
||||||
delete(c.segments, segID)
|
delete(c.segments, segID)
|
||||||
@ -606,3 +646,132 @@ func (c *ChannelMeta) listNotFlushedSegmentIDs() []UniqueID {
|
|||||||
|
|
||||||
return segIDs
|
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))
|
||||||
|
}
|
||||||
|
|||||||
@ -142,7 +142,6 @@ func TestChannelMeta_InnerFunction(t *testing.T) {
|
|||||||
assert.Equal(t, UniqueID(1), seg.collectionID)
|
assert.Equal(t, UniqueID(1), seg.collectionID)
|
||||||
assert.Equal(t, UniqueID(2), seg.partitionID)
|
assert.Equal(t, UniqueID(2), seg.partitionID)
|
||||||
assert.Equal(t, Timestamp(100), seg.startPos.Timestamp)
|
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, int64(0), seg.numRows)
|
||||||
assert.Equal(t, datapb.SegmentType_New, seg.getType())
|
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) {
|
t.Run("Test_getCollectionSchema", func(t *testing.T) {
|
||||||
tests := []struct {
|
tests := []struct {
|
||||||
isValid bool
|
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
|
// ChannelMetaSuite setup test suite for ChannelMeta
|
||||||
type ChannelMetaSuite struct {
|
type ChannelMetaSuite struct {
|
||||||
suite.Suite
|
suite.Suite
|
||||||
|
|||||||
@ -613,7 +613,6 @@ func (node *DataNode) FlushSegments(ctx context.Context, req *datapb.FlushSegmen
|
|||||||
log.Info("receiving FlushSegments request",
|
log.Info("receiving FlushSegments request",
|
||||||
zap.Int64("collection ID", req.GetCollectionID()),
|
zap.Int64("collection ID", req.GetCollectionID()),
|
||||||
zap.Int64s("segments", req.GetSegmentIDs()),
|
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.
|
// TODO: Here and in other places, replace `flushed` param with a more meaningful name.
|
||||||
@ -662,15 +661,13 @@ func (node *DataNode) FlushSegments(ctx context.Context, req *datapb.FlushSegmen
|
|||||||
}
|
}
|
||||||
|
|
||||||
seg, noErr1 := processSegments(req.GetSegmentIDs(), true)
|
seg, noErr1 := processSegments(req.GetSegmentIDs(), true)
|
||||||
staleSeg, noErr2 := processSegments(req.GetMarkSegmentIDs(), false)
|
|
||||||
// Log success flushed segments.
|
// Log success flushed segments.
|
||||||
if len(seg)+len(staleSeg) > 0 {
|
if len(seg) > 0 {
|
||||||
log.Info("sending segments to flush channel",
|
log.Info("sending segments to flush channel",
|
||||||
zap.Any("newly sealed segment IDs", seg),
|
zap.Any("newly sealed segment IDs", seg))
|
||||||
zap.Any("stale segment IDs", staleSeg))
|
|
||||||
}
|
}
|
||||||
// Fail FlushSegments call if at least one segment (no matter stale or not) fails to get flushed.
|
// Fail FlushSegments call if at least one segment fails to get flushed.
|
||||||
if !noErr1 || !noErr2 {
|
if !noErr1 {
|
||||||
return errStatus, nil
|
return errStatus, nil
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|||||||
@ -319,15 +319,14 @@ func TestDataNode(t *testing.T) {
|
|||||||
Base: &commonpb.MsgBase{
|
Base: &commonpb.MsgBase{
|
||||||
TargetID: node1.session.ServerID,
|
TargetID: node1.session.ServerID,
|
||||||
},
|
},
|
||||||
DbID: 0,
|
DbID: 0,
|
||||||
CollectionID: 1,
|
CollectionID: 1,
|
||||||
SegmentIDs: []int64{},
|
SegmentIDs: []int64{},
|
||||||
MarkSegmentIDs: []int64{2},
|
|
||||||
}
|
}
|
||||||
|
|
||||||
status, err = node1.FlushSegments(node1.ctx, req)
|
status, err = node1.FlushSegments(node1.ctx, req)
|
||||||
assert.NoError(t, err)
|
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) {
|
t.Run("Test GetTimeTickChannel", func(t *testing.T) {
|
||||||
|
|||||||
@ -302,10 +302,17 @@ func (dsService *dataSyncService) initNodes(vchanInfo *datapb.VchannelInfo) erro
|
|||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
|
|
||||||
|
var ttNode Node
|
||||||
|
ttNode, err = newTTNode(c, dsService.dataCoord)
|
||||||
|
if err != nil {
|
||||||
|
return err
|
||||||
|
}
|
||||||
|
|
||||||
dsService.fg.AddNode(dmStreamNode)
|
dsService.fg.AddNode(dmStreamNode)
|
||||||
dsService.fg.AddNode(ddNode)
|
dsService.fg.AddNode(ddNode)
|
||||||
dsService.fg.AddNode(insertBufferNode)
|
dsService.fg.AddNode(insertBufferNode)
|
||||||
dsService.fg.AddNode(deleteNode)
|
dsService.fg.AddNode(deleteNode)
|
||||||
|
dsService.fg.AddNode(ttNode)
|
||||||
|
|
||||||
// ddStreamNode
|
// ddStreamNode
|
||||||
err = dsService.fg.SetEdges(dmStreamNode.Name(),
|
err = dsService.fg.SetEdges(dmStreamNode.Name(),
|
||||||
@ -336,12 +343,21 @@ func (dsService *dataSyncService) initNodes(vchanInfo *datapb.VchannelInfo) erro
|
|||||||
|
|
||||||
//deleteNode
|
//deleteNode
|
||||||
err = dsService.fg.SetEdges(deleteNode.Name(),
|
err = dsService.fg.SetEdges(deleteNode.Name(),
|
||||||
[]string{},
|
[]string{ttNode.Name()},
|
||||||
)
|
)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
log.Error("set edges failed in node", zap.String("name", deleteNode.Name()), zap.Error(err))
|
log.Error("set edges failed in node", zap.String("name", deleteNode.Name()), zap.Error(err))
|
||||||
return 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
|
return nil
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|||||||
@ -20,16 +20,17 @@ import (
|
|||||||
"context"
|
"context"
|
||||||
"fmt"
|
"fmt"
|
||||||
"reflect"
|
"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"
|
"github.com/opentracing/opentracing-go"
|
||||||
"go.uber.org/zap"
|
"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.
|
// 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 {
|
for i, msg := range fgMsg.deleteMessages {
|
||||||
traceID, _, _ := trace.InfoFromSpan(spans[i])
|
traceID, _, _ := trace.InfoFromSpan(spans[i])
|
||||||
log.Debug("Buffer delete request in DataNode", zap.String("traceID", traceID))
|
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 {
|
if err != nil {
|
||||||
// error occurs only when deleteMsg is misaligned, should not happen
|
// error occurs only when deleteMsg is misaligned, should not happen
|
||||||
err = fmt.Errorf("buffer delete msg failed, err = %s", err)
|
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 {
|
if len(segmentsToFlush) > 0 {
|
||||||
log.Debug("DeleteNode receives flush message",
|
log.Debug("DeleteNode receives flush message",
|
||||||
zap.Int64s("segIDs", segmentsToFlush),
|
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 {
|
for _, segmentToFlush := range segmentsToFlush {
|
||||||
buf, ok := dn.delBufferManager.Load(segmentToFlush)
|
buf, ok := dn.delBufferManager.Load(segmentToFlush)
|
||||||
if !ok {
|
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))
|
log.Debug("bufferDeleteMsg", zap.Any("primary keys", msg.PrimaryKeys), zap.String("vChannelName", dn.channelName))
|
||||||
|
|
||||||
primaryKeys := storage.ParseIDs2PrimaryKeys(msg.PrimaryKeys)
|
primaryKeys := storage.ParseIDs2PrimaryKeys(msg.PrimaryKeys)
|
||||||
@ -211,7 +213,7 @@ func (dn *deleteNode) bufferDeleteMsg(msg *msgstream.DeleteMsg, tr TimeRange) ([
|
|||||||
if !ok || len(pks) != len(tss) {
|
if !ok || len(pks) != len(tss) {
|
||||||
return nil, fmt.Errorf("primary keys and timestamp's element num mis-match, segmentID = %d", segID)
|
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
|
return segIDs, nil
|
||||||
@ -247,7 +249,7 @@ func newDeleteNode(ctx context.Context, fm flushManager, sig chan<- string, conf
|
|||||||
ctx: ctx,
|
ctx: ctx,
|
||||||
BaseNode: baseNode,
|
BaseNode: baseNode,
|
||||||
delBufferManager: &DelBufferManager{
|
delBufferManager: &DelBufferManager{
|
||||||
delBufMap: sync.Map{},
|
channel: config.channel,
|
||||||
delMemorySize: 0,
|
delMemorySize: 0,
|
||||||
delBufHeap: &PriorityQueue{},
|
delBufHeap: &PriorityQueue{},
|
||||||
},
|
},
|
||||||
|
|||||||
@ -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) {
|
t.Run("Test issue#18565", func(t *testing.T) {
|
||||||
ctx, cancel := context.WithTimeout(context.Background(), 2*time.Second)
|
ctx, cancel := context.WithTimeout(context.Background(), 2*time.Second)
|
||||||
defer cancel()
|
defer cancel()
|
||||||
@ -510,8 +476,11 @@ func TestFlowGraphDeleteNode_showDelBuf(t *testing.T) {
|
|||||||
assert.NoError(t, clearEtcd(testPath))
|
assert.NoError(t, clearEtcd(testPath))
|
||||||
Params.EtcdCfg.MetaRootPath = testPath
|
Params.EtcdCfg.MetaRootPath = testPath
|
||||||
|
|
||||||
|
channel := &ChannelMeta{
|
||||||
|
segments: make(map[UniqueID]*Segment),
|
||||||
|
}
|
||||||
c := &nodeConfig{
|
c := &nodeConfig{
|
||||||
channel: nil,
|
channel: channel,
|
||||||
allocator: NewAllocatorFactory(),
|
allocator: NewAllocatorFactory(),
|
||||||
vChannelName: chanName,
|
vChannelName: chanName,
|
||||||
}
|
}
|
||||||
@ -566,7 +535,6 @@ func TestFlowGraphDeleteNode_updateCompactedSegments(t *testing.T) {
|
|||||||
tests := []struct {
|
tests := []struct {
|
||||||
description string
|
description string
|
||||||
compactToExist bool
|
compactToExist bool
|
||||||
segIDsInBuffer []UniqueID
|
|
||||||
|
|
||||||
compactedToIDs []UniqueID
|
compactedToIDs []UniqueID
|
||||||
compactedFromIDs []UniqueID
|
compactedFromIDs []UniqueID
|
||||||
@ -574,35 +542,28 @@ func TestFlowGraphDeleteNode_updateCompactedSegments(t *testing.T) {
|
|||||||
expectedSegsRemain []UniqueID
|
expectedSegsRemain []UniqueID
|
||||||
}{
|
}{
|
||||||
{"zero segments", false,
|
{"zero segments", false,
|
||||||
[]UniqueID{}, []UniqueID{}, []UniqueID{}, []UniqueID{}},
|
[]UniqueID{}, []UniqueID{}, []UniqueID{}},
|
||||||
{"segment no compaction", false,
|
{"segment no compaction", false,
|
||||||
[]UniqueID{100, 101}, []UniqueID{}, []UniqueID{}, []UniqueID{100, 101}},
|
[]UniqueID{}, []UniqueID{}, []UniqueID{100, 101}},
|
||||||
{"segment compacted not in buffer", true,
|
{"segment compacted", true,
|
||||||
[]UniqueID{100, 101}, []UniqueID{200}, []UniqueID{103}, []UniqueID{100, 101}},
|
[]UniqueID{200}, []UniqueID{103}, []UniqueID{100, 101}},
|
||||||
{"segment compacted in buffer 100>201", true,
|
{"segment compacted 100>201", true,
|
||||||
[]UniqueID{100, 101}, []UniqueID{201}, []UniqueID{100}, []UniqueID{101, 201}},
|
[]UniqueID{201}, []UniqueID{100}, []UniqueID{101, 201}},
|
||||||
{"segment compacted in buffer 100+101>201", true,
|
{"segment compacted 100+101>201", true,
|
||||||
[]UniqueID{100, 101}, []UniqueID{201, 201}, []UniqueID{100, 101}, []UniqueID{201}},
|
[]UniqueID{201, 201}, []UniqueID{100, 101}, []UniqueID{201}},
|
||||||
{"segment compacted in buffer 100>201, 101>202", true,
|
{"segment compacted 100>201, 101>202", true,
|
||||||
[]UniqueID{100, 101}, []UniqueID{201, 202}, []UniqueID{100, 101}, []UniqueID{201, 202}},
|
[]UniqueID{201, 202}, []UniqueID{100, 101}, []UniqueID{201, 202}},
|
||||||
// false
|
// false
|
||||||
{"segment compacted in buffer 100>201", false,
|
{"segment compacted 100>201", false,
|
||||||
[]UniqueID{100, 101}, []UniqueID{201}, []UniqueID{100}, []UniqueID{101}},
|
[]UniqueID{201}, []UniqueID{100}, []UniqueID{101}},
|
||||||
{"segment compacted in buffer 100+101>201", false,
|
{"segment compacted 100+101>201", false,
|
||||||
[]UniqueID{100, 101}, []UniqueID{201, 201}, []UniqueID{100, 101}, []UniqueID{}},
|
[]UniqueID{201, 201}, []UniqueID{100, 101}, []UniqueID{}},
|
||||||
{"segment compacted in buffer 100>201, 101>202", false,
|
{"segment compacted 100>201, 101>202", false,
|
||||||
[]UniqueID{100, 101}, []UniqueID{201, 202}, []UniqueID{100, 101}, []UniqueID{}},
|
[]UniqueID{201, 202}, []UniqueID{100, 101}, []UniqueID{}},
|
||||||
}
|
}
|
||||||
|
|
||||||
for _, test := range tests {
|
for _, test := range tests {
|
||||||
t.Run(test.description, func(t *testing.T) {
|
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 {
|
if test.compactToExist {
|
||||||
for _, segID := range test.compactedToIDs {
|
for _, segID := range test.compactedToIDs {
|
||||||
seg := Segment{
|
seg := Segment{
|
||||||
@ -628,8 +589,7 @@ func TestFlowGraphDeleteNode_updateCompactedSegments(t *testing.T) {
|
|||||||
delNode.updateCompactedSegments()
|
delNode.updateCompactedSegments()
|
||||||
|
|
||||||
for _, remain := range test.expectedSegsRemain {
|
for _, remain := range test.expectedSegsRemain {
|
||||||
_, ok := delNode.delBufferManager.Load(remain)
|
delNode.channel.hasSegment(remain, true)
|
||||||
assert.True(t, ok)
|
|
||||||
}
|
}
|
||||||
})
|
})
|
||||||
}
|
}
|
||||||
|
|||||||
@ -29,7 +29,6 @@ import (
|
|||||||
"go.uber.org/zap"
|
"go.uber.org/zap"
|
||||||
|
|
||||||
"github.com/milvus-io/milvus-proto/go-api/commonpb"
|
"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/log"
|
||||||
"github.com/milvus-io/milvus/internal/metrics"
|
"github.com/milvus-io/milvus/internal/metrics"
|
||||||
"github.com/milvus-io/milvus/internal/mq/msgstream"
|
"github.com/milvus-io/milvus/internal/mq/msgstream"
|
||||||
@ -47,11 +46,10 @@ import (
|
|||||||
type insertBufferNode struct {
|
type insertBufferNode struct {
|
||||||
BaseNode
|
BaseNode
|
||||||
|
|
||||||
ctx context.Context
|
ctx context.Context
|
||||||
channelName string
|
channelName string
|
||||||
insertBuffer sync.Map // SegmentID to BufferData
|
channel Channel
|
||||||
channel Channel
|
idAllocator allocatorInterface
|
||||||
idAllocator allocatorInterface
|
|
||||||
|
|
||||||
flushMap sync.Map
|
flushMap sync.Map
|
||||||
flushChan <-chan flushMsg
|
flushChan <-chan flushMsg
|
||||||
@ -63,6 +61,7 @@ type insertBufferNode struct {
|
|||||||
ttLogger *timeTickLogger
|
ttLogger *timeTickLogger
|
||||||
ttMerger *mergedTimeTickerSender
|
ttMerger *mergedTimeTickerSender
|
||||||
|
|
||||||
|
syncPolicies []segmentSyncPolicy
|
||||||
lastTimestamp Timestamp
|
lastTimestamp Timestamp
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -155,7 +154,7 @@ func (ibNode *insertBufferNode) Operate(in []Msg) []Msg {
|
|||||||
|
|
||||||
// insert messages -> buffer
|
// insert messages -> buffer
|
||||||
for _, msg := range fgMsg.insertMessages {
|
for _, msg := range fgMsg.insertMessages {
|
||||||
err := ibNode.bufferInsertMsg(msg, endPositions[0])
|
err := ibNode.bufferInsertMsg(msg, startPositions[0], endPositions[0])
|
||||||
if err != nil {
|
if err != nil {
|
||||||
// error occurs when missing schema info or data is misaligned, should not happen
|
// error occurs when missing schema info or data is misaligned, should not happen
|
||||||
err = fmt.Errorf("insertBufferNode msg to buffer failed, err = %s", err)
|
err = fmt.Errorf("insertBufferNode msg to buffer failed, err = %s", err)
|
||||||
@ -190,7 +189,7 @@ func (ibNode *insertBufferNode) Operate(in []Msg) []Msg {
|
|||||||
func (ibNode *insertBufferNode) verifyInMsg(in []Msg) (*flowGraphMsg, bool) {
|
func (ibNode *insertBufferNode) verifyInMsg(in []Msg) (*flowGraphMsg, bool) {
|
||||||
// while closing
|
// while closing
|
||||||
if in == nil {
|
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
|
return nil, false
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -207,8 +206,8 @@ func (ibNode *insertBufferNode) verifyInMsg(in []Msg) (*flowGraphMsg, bool) {
|
|||||||
}
|
}
|
||||||
|
|
||||||
func (ibNode *insertBufferNode) GetBufferIfFull(segID UniqueID) (*BufferData, bool) {
|
func (ibNode *insertBufferNode) GetBufferIfFull(segID UniqueID) (*BufferData, bool) {
|
||||||
if bd, ok := ibNode.insertBuffer.Load(segID); ok && bd.(*BufferData).effectiveCap() <= 0 {
|
if bd, ok := ibNode.channel.getCurInsertBuffer(segID); ok && bd.effectiveCap() <= 0 {
|
||||||
return bd.(*BufferData), true
|
return bd, true
|
||||||
}
|
}
|
||||||
|
|
||||||
return nil, false
|
return nil, false
|
||||||
@ -217,8 +216,8 @@ func (ibNode *insertBufferNode) GetBufferIfFull(segID UniqueID) (*BufferData, bo
|
|||||||
// GetBuffer returns buffer data for a segment, returns nil if segment's not in buffer
|
// GetBuffer returns buffer data for a segment, returns nil if segment's not in buffer
|
||||||
func (ibNode *insertBufferNode) GetBuffer(segID UniqueID) *BufferData {
|
func (ibNode *insertBufferNode) GetBuffer(segID UniqueID) *BufferData {
|
||||||
var buf *BufferData
|
var buf *BufferData
|
||||||
if bd, ok := ibNode.insertBuffer.Load(segID); ok {
|
if bd, ok := ibNode.channel.getCurInsertBuffer(segID); ok {
|
||||||
buf = bd.(*BufferData)
|
buf = bd
|
||||||
}
|
}
|
||||||
return buf
|
return buf
|
||||||
}
|
}
|
||||||
@ -272,13 +271,13 @@ func (ibNode *insertBufferNode) DisplayStatistics(seg2Upload []UniqueID) {
|
|||||||
displaySize := min(10, len(seg2Upload))
|
displaySize := min(10, len(seg2Upload))
|
||||||
|
|
||||||
for k, segID := range seg2Upload[:displaySize] {
|
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",
|
log.Info("segment buffer status",
|
||||||
zap.Int("no.", k),
|
zap.Int("no.", k),
|
||||||
zap.Int64("segmentID", segID),
|
zap.Int64("segmentID", segID),
|
||||||
zap.String("channel", ibNode.channelName),
|
zap.String("channel", ibNode.channelName),
|
||||||
zap.Int64("size", bd.(*BufferData).size),
|
zap.Int64("size", bd.size),
|
||||||
zap.Int64("limit", bd.(*BufferData).limit))
|
zap.Int64("limit", bd.limit))
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
@ -315,7 +314,7 @@ func (ibNode *insertBufferNode) FillInSyncTasks(fgMsg *flowGraphMsg, seg2Upload
|
|||||||
return syncTasks
|
return syncTasks
|
||||||
}
|
}
|
||||||
|
|
||||||
// Auto Sync
|
// Auto Sync // TODO: move to segment_sync_policy
|
||||||
for _, segID := range seg2Upload {
|
for _, segID := range seg2Upload {
|
||||||
if ibuffer, ok := ibNode.GetBufferIfFull(segID); ok {
|
if ibuffer, ok := ibNode.GetBufferIfFull(segID); ok {
|
||||||
log.Info("(Auto Sync)",
|
log.Info("(Auto Sync)",
|
||||||
@ -334,6 +333,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)) {
|
mergeSyncTask := func(segmentIDs []UniqueID, syncTasks map[UniqueID]*syncTask, setupTask func(task *syncTask)) {
|
||||||
// Merge auto & manual sync tasks with the same segment ID.
|
// Merge auto & manual sync tasks with the same segment ID.
|
||||||
for _, segmentID := range segmentIDs {
|
for _, segmentID := range segmentIDs {
|
||||||
@ -362,6 +374,7 @@ func (ibNode *insertBufferNode) FillInSyncTasks(fgMsg *flowGraphMsg, seg2Upload
|
|||||||
mergeSyncTask(flushedSegments, syncTasks, func(task *syncTask) {
|
mergeSyncTask(flushedSegments, syncTasks, func(task *syncTask) {
|
||||||
task.flushed = true
|
task.flushed = true
|
||||||
})
|
})
|
||||||
|
mergeSyncTask(syncSegmentIDs, syncTasks, func(task *syncTask) {})
|
||||||
|
|
||||||
// process drop partition
|
// process drop partition
|
||||||
for _, partitionDrop := range fgMsg.dropPartitions {
|
for _, partitionDrop := range fgMsg.dropPartitions {
|
||||||
@ -423,7 +436,7 @@ func (ibNode *insertBufferNode) Sync(fgMsg *flowGraphMsg, seg2Upload []UniqueID,
|
|||||||
panic(err)
|
panic(err)
|
||||||
}
|
}
|
||||||
segmentsToSync = append(segmentsToSync, task.segmentID)
|
segmentsToSync = append(segmentsToSync, task.segmentID)
|
||||||
ibNode.insertBuffer.Delete(task.segmentID)
|
ibNode.channel.rollInsertBuffer(task.segmentID)
|
||||||
ibNode.channel.RollPKstats(task.segmentID, pkStats)
|
ibNode.channel.RollPKstats(task.segmentID, pkStats)
|
||||||
metrics.DataNodeFlushBufferCount.WithLabelValues(fmt.Sprint(paramtable.GetNodeID()), metrics.SuccessLabel).Inc()
|
metrics.DataNodeFlushBufferCount.WithLabelValues(fmt.Sprint(paramtable.GetNodeID()), metrics.SuccessLabel).Inc()
|
||||||
metrics.DataNodeFlushBufferCount.WithLabelValues(fmt.Sprint(paramtable.GetNodeID()), metrics.TotalLabel).Inc()
|
metrics.DataNodeFlushBufferCount.WithLabelValues(fmt.Sprint(paramtable.GetNodeID()), metrics.TotalLabel).Inc()
|
||||||
@ -458,7 +471,7 @@ func (ibNode *insertBufferNode) updateSegmentStates(insertMsgs []*msgstream.Inse
|
|||||||
endPos: endPos,
|
endPos: endPos,
|
||||||
})
|
})
|
||||||
if err != nil {
|
if err != nil {
|
||||||
log.Error("add segment wrong",
|
log.Warn("add segment wrong",
|
||||||
zap.Int64("segID", currentSegID),
|
zap.Int64("segID", currentSegID),
|
||||||
zap.Int64("collID", collID),
|
zap.Int64("collID", collID),
|
||||||
zap.Int64("partID", partitionID),
|
zap.Int64("partID", partitionID),
|
||||||
@ -487,7 +500,7 @@ func (ibNode *insertBufferNode) updateSegmentStates(insertMsgs []*msgstream.Inse
|
|||||||
// 1.2 Get buffer data and put data into each field buffer
|
// 1.2 Get buffer data and put data into each field buffer
|
||||||
// 1.3 Put back into buffer
|
// 1.3 Put back into buffer
|
||||||
// 1.4 Update related statistics
|
// 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 {
|
if err := msg.CheckAligned(); err != nil {
|
||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
@ -496,38 +509,24 @@ func (ibNode *insertBufferNode) bufferInsertMsg(msg *msgstream.InsertMsg, endPos
|
|||||||
|
|
||||||
collSchema, err := ibNode.channel.getCollectionSchema(collectionID, msg.EndTs())
|
collSchema, err := ibNode.channel.getCollectionSchema(collectionID, msg.EndTs())
|
||||||
if err != nil {
|
if err != nil {
|
||||||
log.Error("Get schema wrong:", zap.Error(err))
|
log.Warn("Get schema wrong:", zap.Error(err))
|
||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
|
|
||||||
// Get Dimension
|
// load or store insertBuffer
|
||||||
// TODO GOOSE: under assumption that there's only 1 Vector field in one collection schema
|
var buffer *BufferData
|
||||||
var dimension int
|
var loaded bool
|
||||||
for _, field := range collSchema.Fields {
|
buffer, loaded = ibNode.channel.getCurInsertBuffer(currentSegID)
|
||||||
if field.DataType == schemapb.DataType_FloatVector ||
|
if !loaded {
|
||||||
field.DataType == schemapb.DataType_BinaryVector {
|
buffer, err = newBufferData(collSchema)
|
||||||
|
if err != nil {
|
||||||
dimension, err = storage.GetDimFromParams(field.TypeParams)
|
return fmt.Errorf("newBufferData failed, segment=%d, channel=%s, err=%s", currentSegID, ibNode.channelName, err)
|
||||||
if err != nil {
|
|
||||||
log.Error("failed to get dim from field", zap.Error(err))
|
|
||||||
return err
|
|
||||||
}
|
|
||||||
break
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
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)
|
addedBuffer, err := storage.InsertMsgToInsertData(msg, collSchema)
|
||||||
if err != nil {
|
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
|
return err
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -549,16 +548,14 @@ func (ibNode *insertBufferNode) bufferInsertMsg(msg *msgstream.InsertMsg, endPos
|
|||||||
|
|
||||||
// update buffer size
|
// update buffer size
|
||||||
buffer.updateSize(int64(msg.NRows()))
|
buffer.updateSize(int64(msg.NRows()))
|
||||||
// update timestamp range
|
// update timestamp range and start-end position
|
||||||
buffer.updateTimeRange(ibNode.getTimestampRange(tsData))
|
buffer.updateTimeRange(ibNode.getTimestampRange(tsData))
|
||||||
|
buffer.updateStartAndEndPosition(startPos, endPos)
|
||||||
|
|
||||||
metrics.DataNodeConsumeMsgRowsCount.WithLabelValues(fmt.Sprint(paramtable.GetNodeID()), metrics.InsertLabel).Add(float64(len(msg.RowData)))
|
metrics.DataNodeConsumeMsgRowsCount.WithLabelValues(fmt.Sprint(paramtable.GetNodeID()), metrics.InsertLabel).Add(float64(len(msg.RowData)))
|
||||||
|
|
||||||
// store in buffer
|
// store in buffer
|
||||||
ibNode.insertBuffer.Store(currentSegID, buffer)
|
ibNode.channel.setCurInsertBuffer(currentSegID, buffer)
|
||||||
|
|
||||||
// store current endPositions as Segment->EndPostion
|
|
||||||
ibNode.channel.updateSegmentEndPosition(currentSegID, endPos)
|
|
||||||
|
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
@ -657,9 +654,8 @@ func newInsertBufferNode(ctx context.Context, collID UniqueID, flushCh <-chan fl
|
|||||||
})
|
})
|
||||||
|
|
||||||
return &insertBufferNode{
|
return &insertBufferNode{
|
||||||
ctx: ctx,
|
ctx: ctx,
|
||||||
BaseNode: baseNode,
|
BaseNode: baseNode,
|
||||||
insertBuffer: sync.Map{},
|
|
||||||
|
|
||||||
timeTickStream: wTtMsgStream,
|
timeTickStream: wTtMsgStream,
|
||||||
flushMap: sync.Map{},
|
flushMap: sync.Map{},
|
||||||
|
|||||||
@ -249,7 +249,7 @@ func TestFlowGraphInsertBufferNode_Operate(t *testing.T) {
|
|||||||
setFlowGraphRetryOpt(retry.Attempts(1))
|
setFlowGraphRetryOpt(retry.Attempts(1))
|
||||||
inMsg = genFlowGraphInsertMsg(insertChannelName)
|
inMsg = genFlowGraphInsertMsg(insertChannelName)
|
||||||
iBNode.flushManager = &mockFlushManager{returnError: true}
|
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}) })
|
assert.Panics(t, func() { iBNode.Operate([]flowgraph.Msg{&inMsg}) })
|
||||||
iBNode.flushManager = fm
|
iBNode.flushManager = fm
|
||||||
}
|
}
|
||||||
@ -637,7 +637,6 @@ func TestRollBF(t *testing.T) {
|
|||||||
assert.Equal(t, datapb.SegmentType_New, seg.getType())
|
assert.Equal(t, datapb.SegmentType_New, seg.getType())
|
||||||
assert.Equal(t, int64(1), seg.numRows)
|
assert.Equal(t, int64(1), seg.numRows)
|
||||||
assert.Equal(t, uint64(100), seg.startPos.GetTimestamp())
|
assert.Equal(t, uint64(100), seg.startPos.GetTimestamp())
|
||||||
assert.Equal(t, uint64(123), seg.endPos.GetTimestamp())
|
|
||||||
// because this is the origincal
|
// because this is the origincal
|
||||||
assert.True(t, seg.currentStat.PkFilter.Cap() > uint(1000000))
|
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, datapb.SegmentType_Normal, seg.getType())
|
||||||
assert.Equal(t, int64(2), seg.numRows)
|
assert.Equal(t, int64(2), seg.numRows)
|
||||||
assert.Equal(t, uint64(100), seg.startPos.GetTimestamp())
|
assert.Equal(t, uint64(100), seg.startPos.GetTimestamp())
|
||||||
assert.Equal(t, uint64(234), seg.endPos.GetTimestamp())
|
|
||||||
// filter should be rolled
|
// filter should be rolled
|
||||||
|
|
||||||
assert.Nil(t, seg.currentStat)
|
assert.Nil(t, seg.currentStat)
|
||||||
@ -769,9 +767,9 @@ func (s *InsertBufferNodeSuit) TestFillInSyncTasks() {
|
|||||||
size: 2,
|
size: 2,
|
||||||
limit: 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().NotEmpty(syncTasks)
|
||||||
s.Assert().Equal(1, len(syncTasks))
|
s.Assert().Equal(1, len(syncTasks))
|
||||||
|
|
||||||
@ -784,7 +782,7 @@ func (s *InsertBufferNodeSuit) TestFillInSyncTasks() {
|
|||||||
})
|
})
|
||||||
|
|
||||||
s.Run("drop partition", func() {
|
s.Run("drop partition", func() {
|
||||||
fgMsg := flowGraphMsg{dropPartitions: []UniqueID{s.partID}}
|
fgMsg := flowGraphMsg{dropPartitions: []UniqueID{s.partID}, endPositions: []*internalpb.MsgPosition{{Timestamp: 100}}}
|
||||||
node := &insertBufferNode{
|
node := &insertBufferNode{
|
||||||
channelName: s.channel.channelName,
|
channelName: s.channel.channelName,
|
||||||
channel: s.channel,
|
channel: s.channel,
|
||||||
@ -820,7 +818,7 @@ func (s *InsertBufferNodeSuit) TestFillInSyncTasks() {
|
|||||||
flushCh <- msg
|
flushCh <- msg
|
||||||
}
|
}
|
||||||
|
|
||||||
syncTasks := node.FillInSyncTasks(new(flowGraphMsg), nil)
|
syncTasks := node.FillInSyncTasks(&flowGraphMsg{endPositions: []*internalpb.MsgPosition{{Timestamp: 100}}}, nil)
|
||||||
s.Assert().NotEmpty(syncTasks)
|
s.Assert().NotEmpty(syncTasks)
|
||||||
|
|
||||||
for segID, task := range syncTasks {
|
for segID, task := range syncTasks {
|
||||||
@ -857,7 +855,7 @@ func (s *InsertBufferNodeSuit) TestFillInSyncTasks() {
|
|||||||
flushCh <- msg
|
flushCh <- msg
|
||||||
}
|
}
|
||||||
|
|
||||||
syncTasks := node.FillInSyncTasks(new(flowGraphMsg), nil)
|
syncTasks := node.FillInSyncTasks(&flowGraphMsg{endPositions: []*internalpb.MsgPosition{{Timestamp: 100}}}, nil)
|
||||||
s.Assert().NotEmpty(syncTasks)
|
s.Assert().NotEmpty(syncTasks)
|
||||||
s.Assert().Equal(10, len(syncTasks)) // 10 is max batch
|
s.Assert().Equal(10, len(syncTasks)) // 10 is max batch
|
||||||
|
|
||||||
@ -960,14 +958,14 @@ func TestInsertBufferNode_bufferInsertMsg(t *testing.T) {
|
|||||||
inMsg := genFlowGraphInsertMsg(insertChannelName)
|
inMsg := genFlowGraphInsertMsg(insertChannelName)
|
||||||
for _, msg := range inMsg.insertMessages {
|
for _, msg := range inMsg.insertMessages {
|
||||||
msg.EndTimestamp = 101 // ts valid
|
msg.EndTimestamp = 101 // ts valid
|
||||||
err = iBNode.bufferInsertMsg(msg, &internalpb.MsgPosition{})
|
err = iBNode.bufferInsertMsg(msg, &internalpb.MsgPosition{}, &internalpb.MsgPosition{})
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
}
|
}
|
||||||
|
|
||||||
for _, msg := range inMsg.insertMessages {
|
for _, msg := range inMsg.insertMessages {
|
||||||
msg.EndTimestamp = 101 // ts valid
|
msg.EndTimestamp = 101 // ts valid
|
||||||
msg.RowIDs = []int64{} //misaligned data
|
msg.RowIDs = []int64{} //misaligned data
|
||||||
err = iBNode.bufferInsertMsg(msg, &internalpb.MsgPosition{})
|
err = iBNode.bufferInsertMsg(msg, &internalpb.MsgPosition{}, &internalpb.MsgPosition{})
|
||||||
assert.NotNil(t, err)
|
assert.NotNil(t, err)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|||||||
126
internal/datanode/flow_graph_time_tick_node.go
Normal file
126
internal/datanode/flow_graph_time_tick_node.go
Normal file
@ -0,0 +1,126 @@
|
|||||||
|
// 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/paramtable"
|
||||||
|
"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(paramtable.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
|
||||||
|
}
|
||||||
@ -852,5 +852,8 @@ func flushNotifyFunc(dsService *dataSyncService, opts ...retry.Option) notifyMet
|
|||||||
dsService.channel.segmentFlushed(pack.segmentID)
|
dsService.channel.segmentFlushed(pack.segmentID)
|
||||||
}
|
}
|
||||||
dsService.flushingSegCache.Remove(req.GetSegmentID())
|
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())
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|||||||
@ -21,12 +21,14 @@ import (
|
|||||||
"errors"
|
"errors"
|
||||||
"sync"
|
"sync"
|
||||||
|
|
||||||
|
"go.uber.org/zap"
|
||||||
|
|
||||||
"github.com/milvus-io/milvus/internal/kv"
|
"github.com/milvus-io/milvus/internal/kv"
|
||||||
"github.com/milvus-io/milvus/internal/log"
|
"github.com/milvus-io/milvus/internal/log"
|
||||||
"github.com/milvus-io/milvus/internal/proto/datapb"
|
"github.com/milvus-io/milvus/internal/proto/datapb"
|
||||||
"github.com/milvus-io/milvus/internal/proto/internalpb"
|
"github.com/milvus-io/milvus/internal/proto/internalpb"
|
||||||
"github.com/milvus-io/milvus/internal/util/retry"
|
"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
|
// errStart used for retry start
|
||||||
@ -134,6 +136,7 @@ func (t *flushTaskRunner) runFlushInsert(task flushInsertTask,
|
|||||||
zap.Bool("flushed", flushed),
|
zap.Bool("flushed", flushed),
|
||||||
zap.Bool("dropped", dropped),
|
zap.Bool("dropped", dropped),
|
||||||
zap.Any("position", pos),
|
zap.Any("position", pos),
|
||||||
|
zap.Time("PosTime", tsoutil.PhysicalTime(pos.GetTimestamp())),
|
||||||
)
|
)
|
||||||
go func() {
|
go func() {
|
||||||
err := retry.Do(context.Background(), func() error {
|
err := retry.Do(context.Background(), func() error {
|
||||||
|
|||||||
@ -253,6 +253,12 @@ func (ds *DataCoordFactory) UpdateSegmentStatistics(ctx context.Context, req *da
|
|||||||
}, nil
|
}, 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) {
|
func (ds *DataCoordFactory) SaveImportSegment(ctx context.Context, req *datapb.SaveImportSegmentRequest) (*commonpb.Status, error) {
|
||||||
return &commonpb.Status{
|
return &commonpb.Status{
|
||||||
ErrorCode: commonpb.ErrorCode_Success,
|
ErrorCode: commonpb.ErrorCode_Success,
|
||||||
|
|||||||
@ -21,9 +21,13 @@ import (
|
|||||||
"sync/atomic"
|
"sync/atomic"
|
||||||
|
|
||||||
"github.com/bits-and-blooms/bloom/v3"
|
"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/datapb"
|
||||||
"github.com/milvus-io/milvus/internal/proto/internalpb"
|
"github.com/milvus-io/milvus/internal/proto/internalpb"
|
||||||
"github.com/milvus-io/milvus/internal/storage"
|
"github.com/milvus-io/milvus/internal/storage"
|
||||||
|
"github.com/milvus-io/milvus/internal/util/tsoutil"
|
||||||
)
|
)
|
||||||
|
|
||||||
// Segment contains the latest segment infos from channel.
|
// Segment contains the latest segment infos from channel.
|
||||||
@ -37,12 +41,17 @@ type Segment struct {
|
|||||||
memorySize int64
|
memorySize int64
|
||||||
compactedTo UniqueID
|
compactedTo UniqueID
|
||||||
|
|
||||||
statLock sync.Mutex
|
curInsertBuf *BufferData
|
||||||
|
curDeleteBuf *DelDataBuf
|
||||||
|
historyInsertBuf []*BufferData
|
||||||
|
historyDeleteBuf []*DelDataBuf
|
||||||
|
|
||||||
|
statLock sync.RWMutex
|
||||||
currentStat *storage.PkStatistics
|
currentStat *storage.PkStatistics
|
||||||
historyStats []*storage.PkStatistics
|
historyStats []*storage.PkStatistics
|
||||||
|
|
||||||
startPos *internalpb.MsgPosition // TODO readonly
|
lastSyncTs Timestamp
|
||||||
endPos *internalpb.MsgPosition
|
startPos *internalpb.MsgPosition // TODO readonly
|
||||||
}
|
}
|
||||||
|
|
||||||
type addSegmentReq struct {
|
type addSegmentReq struct {
|
||||||
@ -104,3 +113,56 @@ func (s *Segment) isPKExist(pk primaryKey) bool {
|
|||||||
}
|
}
|
||||||
return false
|
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
|
||||||
|
}
|
||||||
|
|||||||
39
internal/datanode/segment_sync_policy.go
Normal file
39
internal/datanode/segment_sync_policy.go
Normal file
@ -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()
|
||||||
|
}
|
||||||
|
}
|
||||||
56
internal/datanode/segment_sync_policy_test.go
Normal file
56
internal/datanode/segment_sync_policy_test.go
Normal file
@ -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)
|
||||||
|
})
|
||||||
|
}
|
||||||
|
}
|
||||||
@ -657,6 +657,25 @@ func (c *Client) UpdateSegmentStatistics(ctx context.Context, req *datapb.Update
|
|||||||
return ret.(*commonpb.Status), err
|
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(paramtable.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.
|
// AcquireSegmentLock acquire the reference lock of the segments.
|
||||||
func (c *Client) AcquireSegmentLock(ctx context.Context, req *datapb.AcquireSegmentLockRequest) (*commonpb.Status, error) {
|
func (c *Client) AcquireSegmentLock(ctx context.Context, req *datapb.AcquireSegmentLockRequest) (*commonpb.Status, error) {
|
||||||
req = typeutil.Clone(req)
|
req = typeutil.Clone(req)
|
||||||
|
|||||||
@ -369,6 +369,11 @@ func (s *Server) UpdateSegmentStatistics(ctx context.Context, req *datapb.Update
|
|||||||
return s.dataCoord.UpdateSegmentStatistics(ctx, req)
|
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.
|
// AcquireSegmentLock acquire the reference lock of the segments.
|
||||||
func (s *Server) AcquireSegmentLock(ctx context.Context, req *datapb.AcquireSegmentLockRequest) (*commonpb.Status, error) {
|
func (s *Server) AcquireSegmentLock(ctx context.Context, req *datapb.AcquireSegmentLockRequest) (*commonpb.Status, error) {
|
||||||
return s.dataCoord.AcquireSegmentLock(ctx, req)
|
return s.dataCoord.AcquireSegmentLock(ctx, req)
|
||||||
|
|||||||
@ -63,6 +63,7 @@ type MockDataCoord struct {
|
|||||||
setSegmentStateResp *datapb.SetSegmentStateResponse
|
setSegmentStateResp *datapb.SetSegmentStateResponse
|
||||||
importResp *datapb.ImportTaskResponse
|
importResp *datapb.ImportTaskResponse
|
||||||
updateSegStatResp *commonpb.Status
|
updateSegStatResp *commonpb.Status
|
||||||
|
updateChanPos *commonpb.Status
|
||||||
acquireSegLockResp *commonpb.Status
|
acquireSegLockResp *commonpb.Status
|
||||||
releaseSegLockResp *commonpb.Status
|
releaseSegLockResp *commonpb.Status
|
||||||
addSegmentResp *commonpb.Status
|
addSegmentResp *commonpb.Status
|
||||||
@ -204,6 +205,10 @@ func (m *MockDataCoord) UpdateSegmentStatistics(ctx context.Context, req *datapb
|
|||||||
return m.updateSegStatResp, m.err
|
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) {
|
func (m *MockDataCoord) AcquireSegmentLock(ctx context.Context, req *datapb.AcquireSegmentLockRequest) (*commonpb.Status, error) {
|
||||||
return m.acquireSegLockResp, m.err
|
return m.acquireSegLockResp, m.err
|
||||||
}
|
}
|
||||||
@ -480,6 +485,17 @@ func Test_NewServer(t *testing.T) {
|
|||||||
assert.NotNil(t, resp)
|
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) {
|
t.Run("acquire segment reference lock", func(t *testing.T) {
|
||||||
server.dataCoord = &MockDataCoord{
|
server.dataCoord = &MockDataCoord{
|
||||||
acquireSegLockResp: &commonpb.Status{
|
acquireSegLockResp: &commonpb.Status{
|
||||||
|
|||||||
@ -593,6 +593,10 @@ func (m *MockDataCoord) UpdateSegmentStatistics(ctx context.Context, req *datapb
|
|||||||
return nil, nil
|
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) {
|
func (m *MockDataCoord) AcquireSegmentLock(ctx context.Context, req *datapb.AcquireSegmentLockRequest) (*commonpb.Status, error) {
|
||||||
return nil, nil
|
return nil, nil
|
||||||
}
|
}
|
||||||
|
|||||||
@ -6,6 +6,7 @@ import (
|
|||||||
"github.com/milvus-io/milvus-proto/go-api/milvuspb"
|
"github.com/milvus-io/milvus-proto/go-api/milvuspb"
|
||||||
"github.com/milvus-io/milvus/internal/metastore/model"
|
"github.com/milvus-io/milvus/internal/metastore/model"
|
||||||
"github.com/milvus-io/milvus/internal/proto/datapb"
|
"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/proto/querypb"
|
||||||
"github.com/milvus-io/milvus/internal/util/typeutil"
|
"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
|
AlterSegment(ctx context.Context, newSegment *datapb.SegmentInfo, oldSegment *datapb.SegmentInfo) error
|
||||||
SaveDroppedSegmentsInBatch(ctx context.Context, segments []*datapb.SegmentInfo) error
|
SaveDroppedSegmentsInBatch(ctx context.Context, segments []*datapb.SegmentInfo) error
|
||||||
DropSegment(ctx context.Context, segment *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
|
MarkChannelDeleted(ctx context.Context, channel string) error
|
||||||
IsChannelDropped(ctx context.Context, channel string) bool
|
IsChannelDropped(ctx context.Context, channel string) bool
|
||||||
DropChannel(ctx context.Context, channel string) error
|
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 {
|
type IndexCoordCatalog interface {
|
||||||
|
|||||||
@ -23,6 +23,7 @@ const (
|
|||||||
SegmentDeltalogPathPrefix = MetaPrefix + "/deltalog"
|
SegmentDeltalogPathPrefix = MetaPrefix + "/deltalog"
|
||||||
SegmentStatslogPathPrefix = MetaPrefix + "/statslog"
|
SegmentStatslogPathPrefix = MetaPrefix + "/statslog"
|
||||||
ChannelRemovePrefix = MetaPrefix + "/channel-removal"
|
ChannelRemovePrefix = MetaPrefix + "/channel-removal"
|
||||||
|
ChannelCheckpointPrefix = MetaPrefix + "/channel-cp"
|
||||||
|
|
||||||
RemoveFlagTomestone = "removed"
|
RemoveFlagTomestone = "removed"
|
||||||
)
|
)
|
||||||
|
|||||||
@ -22,21 +22,20 @@ import (
|
|||||||
"strconv"
|
"strconv"
|
||||||
"strings"
|
"strings"
|
||||||
|
|
||||||
"github.com/milvus-io/milvus/internal/util/metautil"
|
"github.com/golang/protobuf/proto"
|
||||||
|
"go.uber.org/zap"
|
||||||
"github.com/milvus-io/milvus/internal/util/etcd"
|
|
||||||
|
|
||||||
"golang.org/x/exp/maps"
|
"golang.org/x/exp/maps"
|
||||||
|
|
||||||
"github.com/golang/protobuf/proto"
|
|
||||||
"github.com/milvus-io/milvus-proto/go-api/commonpb"
|
"github.com/milvus-io/milvus-proto/go-api/commonpb"
|
||||||
"github.com/milvus-io/milvus/internal/kv"
|
"github.com/milvus-io/milvus/internal/kv"
|
||||||
"github.com/milvus-io/milvus/internal/log"
|
"github.com/milvus-io/milvus/internal/log"
|
||||||
"github.com/milvus-io/milvus/internal/proto/datapb"
|
"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/storage"
|
||||||
"github.com/milvus-io/milvus/internal/util"
|
"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"
|
"github.com/milvus-io/milvus/internal/util/typeutil"
|
||||||
"go.uber.org/zap"
|
|
||||||
)
|
)
|
||||||
|
|
||||||
type Catalog struct {
|
type Catalog struct {
|
||||||
@ -312,6 +311,43 @@ func (kc *Catalog) DropChannel(ctx context.Context, channel string) error {
|
|||||||
return kc.Txn.Remove(key)
|
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,
|
func (kc *Catalog) getBinlogsWithPrefix(binlogType storage.BinlogType, collectionID, partitionID,
|
||||||
segmentID typeutil.UniqueID) ([]string, []string, error) {
|
segmentID typeutil.UniqueID) ([]string, []string, error) {
|
||||||
var binlogPrefix string
|
var binlogPrefix string
|
||||||
@ -567,3 +603,7 @@ func buildFieldStatslogPathPrefix(collectionID typeutil.UniqueID, partitionID ty
|
|||||||
func buildChannelRemovePath(channel string) string {
|
func buildChannelRemovePath(channel string) string {
|
||||||
return fmt.Sprintf("%s/%s", ChannelRemovePrefix, channel)
|
return fmt.Sprintf("%s/%s", ChannelRemovePrefix, channel)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func buildChannelCPKey(vChannel string) string {
|
||||||
|
return fmt.Sprintf("%s/%s", ChannelCheckpointPrefix, vChannel)
|
||||||
|
}
|
||||||
|
|||||||
@ -7,14 +7,16 @@ import (
|
|||||||
"testing"
|
"testing"
|
||||||
|
|
||||||
"github.com/golang/protobuf/proto"
|
"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-proto/go-api/commonpb"
|
||||||
"github.com/milvus-io/milvus/internal/kv"
|
"github.com/milvus-io/milvus/internal/kv"
|
||||||
"github.com/milvus-io/milvus/internal/kv/mocks"
|
"github.com/milvus-io/milvus/internal/kv/mocks"
|
||||||
"github.com/milvus-io/milvus/internal/proto/datapb"
|
"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/milvus-io/milvus/internal/util/metautil"
|
||||||
"github.com/stretchr/testify/assert"
|
|
||||||
"github.com/stretchr/testify/mock"
|
|
||||||
"golang.org/x/exp/maps"
|
|
||||||
)
|
)
|
||||||
|
|
||||||
type MockedTxnKV struct {
|
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) {
|
func Test_MarkChannelDeleted_SaveError(t *testing.T) {
|
||||||
txn := &mocks.TxnKV{}
|
txn := &mocks.TxnKV{}
|
||||||
txn.EXPECT().
|
txn.EXPECT().
|
||||||
|
|||||||
@ -57,6 +57,7 @@ service DataCoord {
|
|||||||
// https://wiki.lfaidata.foundation/display/MIL/MEP+24+--+Support+bulk+load
|
// https://wiki.lfaidata.foundation/display/MIL/MEP+24+--+Support+bulk+load
|
||||||
rpc Import(ImportTaskRequest) returns (ImportTaskResponse) {}
|
rpc Import(ImportTaskRequest) returns (ImportTaskResponse) {}
|
||||||
rpc UpdateSegmentStatistics(UpdateSegmentStatisticsRequest) returns (common.Status) {}
|
rpc UpdateSegmentStatistics(UpdateSegmentStatisticsRequest) returns (common.Status) {}
|
||||||
|
rpc UpdateChannelCheckpoint(UpdateChannelCheckpointRequest) returns (common.Status) {}
|
||||||
|
|
||||||
rpc AcquireSegmentLock(AcquireSegmentLockRequest) returns (common.Status) {}
|
rpc AcquireSegmentLock(AcquireSegmentLockRequest) returns (common.Status) {}
|
||||||
rpc ReleaseSegmentLock(ReleaseSegmentLockRequest) returns (common.Status) {}
|
rpc ReleaseSegmentLock(ReleaseSegmentLockRequest) returns (common.Status) {}
|
||||||
@ -241,7 +242,6 @@ message FlushSegmentsRequest {
|
|||||||
int64 dbID = 2;
|
int64 dbID = 2;
|
||||||
int64 collectionID = 3;
|
int64 collectionID = 3;
|
||||||
repeated int64 segmentIDs = 4; // segments to flush
|
repeated int64 segmentIDs = 4; // segments to flush
|
||||||
repeated int64 markSegmentIDs = 5; // segments to clean buffer and mark segment position, but NOT flushed
|
|
||||||
}
|
}
|
||||||
|
|
||||||
message SegmentMsg{
|
message SegmentMsg{
|
||||||
@ -578,6 +578,12 @@ message UpdateSegmentStatisticsRequest {
|
|||||||
repeated SegmentStats stats = 2;
|
repeated SegmentStats stats = 2;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
message UpdateChannelCheckpointRequest {
|
||||||
|
common.MsgBase base = 1;
|
||||||
|
string vChannel = 2;
|
||||||
|
internal.MsgPosition position = 3;
|
||||||
|
}
|
||||||
|
|
||||||
message ResendSegmentStatsRequest {
|
message ResendSegmentStatsRequest {
|
||||||
common.MsgBase base = 1;
|
common.MsgBase base = 1;
|
||||||
}
|
}
|
||||||
|
|||||||
@ -1465,7 +1465,6 @@ type FlushSegmentsRequest struct {
|
|||||||
DbID int64 `protobuf:"varint,2,opt,name=dbID,proto3" json:"dbID,omitempty"`
|
DbID int64 `protobuf:"varint,2,opt,name=dbID,proto3" json:"dbID,omitempty"`
|
||||||
CollectionID int64 `protobuf:"varint,3,opt,name=collectionID,proto3" json:"collectionID,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"`
|
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_NoUnkeyedLiteral struct{} `json:"-"`
|
||||||
XXX_unrecognized []byte `json:"-"`
|
XXX_unrecognized []byte `json:"-"`
|
||||||
XXX_sizecache int32 `json:"-"`
|
XXX_sizecache int32 `json:"-"`
|
||||||
@ -1524,13 +1523,6 @@ func (m *FlushSegmentsRequest) GetSegmentIDs() []int64 {
|
|||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
|
||||||
func (m *FlushSegmentsRequest) GetMarkSegmentIDs() []int64 {
|
|
||||||
if m != nil {
|
|
||||||
return m.MarkSegmentIDs
|
|
||||||
}
|
|
||||||
return nil
|
|
||||||
}
|
|
||||||
|
|
||||||
type SegmentMsg struct {
|
type SegmentMsg struct {
|
||||||
Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"`
|
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"`
|
Segment *SegmentInfo `protobuf:"bytes,2,opt,name=segment,proto3" json:"segment,omitempty"`
|
||||||
@ -4284,6 +4276,61 @@ func (m *UpdateSegmentStatisticsRequest) GetStats() []*SegmentStats {
|
|||||||
return nil
|
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 {
|
type ResendSegmentStatsRequest struct {
|
||||||
Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"`
|
Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"`
|
||||||
XXX_NoUnkeyedLiteral struct{} `json:"-"`
|
XXX_NoUnkeyedLiteral struct{} `json:"-"`
|
||||||
@ -4295,7 +4342,7 @@ func (m *ResendSegmentStatsRequest) Reset() { *m = ResendSegmentStatsReq
|
|||||||
func (m *ResendSegmentStatsRequest) String() string { return proto.CompactTextString(m) }
|
func (m *ResendSegmentStatsRequest) String() string { return proto.CompactTextString(m) }
|
||||||
func (*ResendSegmentStatsRequest) ProtoMessage() {}
|
func (*ResendSegmentStatsRequest) ProtoMessage() {}
|
||||||
func (*ResendSegmentStatsRequest) Descriptor() ([]byte, []int) {
|
func (*ResendSegmentStatsRequest) Descriptor() ([]byte, []int) {
|
||||||
return fileDescriptor_82cd95f524594f49, []int{66}
|
return fileDescriptor_82cd95f524594f49, []int{67}
|
||||||
}
|
}
|
||||||
|
|
||||||
func (m *ResendSegmentStatsRequest) XXX_Unmarshal(b []byte) error {
|
func (m *ResendSegmentStatsRequest) XXX_Unmarshal(b []byte) error {
|
||||||
@ -4335,7 +4382,7 @@ func (m *ResendSegmentStatsResponse) Reset() { *m = ResendSegmentStatsRe
|
|||||||
func (m *ResendSegmentStatsResponse) String() string { return proto.CompactTextString(m) }
|
func (m *ResendSegmentStatsResponse) String() string { return proto.CompactTextString(m) }
|
||||||
func (*ResendSegmentStatsResponse) ProtoMessage() {}
|
func (*ResendSegmentStatsResponse) ProtoMessage() {}
|
||||||
func (*ResendSegmentStatsResponse) Descriptor() ([]byte, []int) {
|
func (*ResendSegmentStatsResponse) Descriptor() ([]byte, []int) {
|
||||||
return fileDescriptor_82cd95f524594f49, []int{67}
|
return fileDescriptor_82cd95f524594f49, []int{68}
|
||||||
}
|
}
|
||||||
|
|
||||||
func (m *ResendSegmentStatsResponse) XXX_Unmarshal(b []byte) error {
|
func (m *ResendSegmentStatsResponse) XXX_Unmarshal(b []byte) error {
|
||||||
@ -4387,7 +4434,7 @@ func (m *AddImportSegmentRequest) Reset() { *m = AddImportSegmentRequest
|
|||||||
func (m *AddImportSegmentRequest) String() string { return proto.CompactTextString(m) }
|
func (m *AddImportSegmentRequest) String() string { return proto.CompactTextString(m) }
|
||||||
func (*AddImportSegmentRequest) ProtoMessage() {}
|
func (*AddImportSegmentRequest) ProtoMessage() {}
|
||||||
func (*AddImportSegmentRequest) Descriptor() ([]byte, []int) {
|
func (*AddImportSegmentRequest) Descriptor() ([]byte, []int) {
|
||||||
return fileDescriptor_82cd95f524594f49, []int{68}
|
return fileDescriptor_82cd95f524594f49, []int{69}
|
||||||
}
|
}
|
||||||
|
|
||||||
func (m *AddImportSegmentRequest) XXX_Unmarshal(b []byte) error {
|
func (m *AddImportSegmentRequest) XXX_Unmarshal(b []byte) error {
|
||||||
@ -4469,7 +4516,7 @@ func (m *AddImportSegmentResponse) Reset() { *m = AddImportSegmentRespon
|
|||||||
func (m *AddImportSegmentResponse) String() string { return proto.CompactTextString(m) }
|
func (m *AddImportSegmentResponse) String() string { return proto.CompactTextString(m) }
|
||||||
func (*AddImportSegmentResponse) ProtoMessage() {}
|
func (*AddImportSegmentResponse) ProtoMessage() {}
|
||||||
func (*AddImportSegmentResponse) Descriptor() ([]byte, []int) {
|
func (*AddImportSegmentResponse) Descriptor() ([]byte, []int) {
|
||||||
return fileDescriptor_82cd95f524594f49, []int{69}
|
return fileDescriptor_82cd95f524594f49, []int{70}
|
||||||
}
|
}
|
||||||
|
|
||||||
func (m *AddImportSegmentResponse) XXX_Unmarshal(b []byte) error {
|
func (m *AddImportSegmentResponse) XXX_Unmarshal(b []byte) error {
|
||||||
@ -4522,7 +4569,7 @@ func (m *SaveImportSegmentRequest) Reset() { *m = SaveImportSegmentReque
|
|||||||
func (m *SaveImportSegmentRequest) String() string { return proto.CompactTextString(m) }
|
func (m *SaveImportSegmentRequest) String() string { return proto.CompactTextString(m) }
|
||||||
func (*SaveImportSegmentRequest) ProtoMessage() {}
|
func (*SaveImportSegmentRequest) ProtoMessage() {}
|
||||||
func (*SaveImportSegmentRequest) Descriptor() ([]byte, []int) {
|
func (*SaveImportSegmentRequest) Descriptor() ([]byte, []int) {
|
||||||
return fileDescriptor_82cd95f524594f49, []int{70}
|
return fileDescriptor_82cd95f524594f49, []int{71}
|
||||||
}
|
}
|
||||||
|
|
||||||
func (m *SaveImportSegmentRequest) XXX_Unmarshal(b []byte) error {
|
func (m *SaveImportSegmentRequest) XXX_Unmarshal(b []byte) error {
|
||||||
@ -4611,7 +4658,7 @@ func (m *UnsetIsImportingStateRequest) Reset() { *m = UnsetIsImportingSt
|
|||||||
func (m *UnsetIsImportingStateRequest) String() string { return proto.CompactTextString(m) }
|
func (m *UnsetIsImportingStateRequest) String() string { return proto.CompactTextString(m) }
|
||||||
func (*UnsetIsImportingStateRequest) ProtoMessage() {}
|
func (*UnsetIsImportingStateRequest) ProtoMessage() {}
|
||||||
func (*UnsetIsImportingStateRequest) Descriptor() ([]byte, []int) {
|
func (*UnsetIsImportingStateRequest) Descriptor() ([]byte, []int) {
|
||||||
return fileDescriptor_82cd95f524594f49, []int{71}
|
return fileDescriptor_82cd95f524594f49, []int{72}
|
||||||
}
|
}
|
||||||
|
|
||||||
func (m *UnsetIsImportingStateRequest) XXX_Unmarshal(b []byte) error {
|
func (m *UnsetIsImportingStateRequest) XXX_Unmarshal(b []byte) error {
|
||||||
@ -4658,7 +4705,7 @@ func (m *MarkSegmentsDroppedRequest) Reset() { *m = MarkSegmentsDroppedR
|
|||||||
func (m *MarkSegmentsDroppedRequest) String() string { return proto.CompactTextString(m) }
|
func (m *MarkSegmentsDroppedRequest) String() string { return proto.CompactTextString(m) }
|
||||||
func (*MarkSegmentsDroppedRequest) ProtoMessage() {}
|
func (*MarkSegmentsDroppedRequest) ProtoMessage() {}
|
||||||
func (*MarkSegmentsDroppedRequest) Descriptor() ([]byte, []int) {
|
func (*MarkSegmentsDroppedRequest) Descriptor() ([]byte, []int) {
|
||||||
return fileDescriptor_82cd95f524594f49, []int{72}
|
return fileDescriptor_82cd95f524594f49, []int{73}
|
||||||
}
|
}
|
||||||
|
|
||||||
func (m *MarkSegmentsDroppedRequest) XXX_Unmarshal(b []byte) error {
|
func (m *MarkSegmentsDroppedRequest) XXX_Unmarshal(b []byte) error {
|
||||||
@ -4706,7 +4753,7 @@ func (m *SegmentReferenceLock) Reset() { *m = SegmentReferenceLock{} }
|
|||||||
func (m *SegmentReferenceLock) String() string { return proto.CompactTextString(m) }
|
func (m *SegmentReferenceLock) String() string { return proto.CompactTextString(m) }
|
||||||
func (*SegmentReferenceLock) ProtoMessage() {}
|
func (*SegmentReferenceLock) ProtoMessage() {}
|
||||||
func (*SegmentReferenceLock) Descriptor() ([]byte, []int) {
|
func (*SegmentReferenceLock) Descriptor() ([]byte, []int) {
|
||||||
return fileDescriptor_82cd95f524594f49, []int{73}
|
return fileDescriptor_82cd95f524594f49, []int{74}
|
||||||
}
|
}
|
||||||
|
|
||||||
func (m *SegmentReferenceLock) XXX_Unmarshal(b []byte) error {
|
func (m *SegmentReferenceLock) XXX_Unmarshal(b []byte) error {
|
||||||
@ -4818,6 +4865,7 @@ func init() {
|
|||||||
proto.RegisterType((*ImportTaskResponse)(nil), "milvus.proto.data.ImportTaskResponse")
|
proto.RegisterType((*ImportTaskResponse)(nil), "milvus.proto.data.ImportTaskResponse")
|
||||||
proto.RegisterType((*ImportTaskRequest)(nil), "milvus.proto.data.ImportTaskRequest")
|
proto.RegisterType((*ImportTaskRequest)(nil), "milvus.proto.data.ImportTaskRequest")
|
||||||
proto.RegisterType((*UpdateSegmentStatisticsRequest)(nil), "milvus.proto.data.UpdateSegmentStatisticsRequest")
|
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((*ResendSegmentStatsRequest)(nil), "milvus.proto.data.ResendSegmentStatsRequest")
|
||||||
proto.RegisterType((*ResendSegmentStatsResponse)(nil), "milvus.proto.data.ResendSegmentStatsResponse")
|
proto.RegisterType((*ResendSegmentStatsResponse)(nil), "milvus.proto.data.ResendSegmentStatsResponse")
|
||||||
proto.RegisterType((*AddImportSegmentRequest)(nil), "milvus.proto.data.AddImportSegmentRequest")
|
proto.RegisterType((*AddImportSegmentRequest)(nil), "milvus.proto.data.AddImportSegmentRequest")
|
||||||
@ -4831,277 +4879,280 @@ func init() {
|
|||||||
func init() { proto.RegisterFile("data_coord.proto", fileDescriptor_82cd95f524594f49) }
|
func init() { proto.RegisterFile("data_coord.proto", fileDescriptor_82cd95f524594f49) }
|
||||||
|
|
||||||
var fileDescriptor_82cd95f524594f49 = []byte{
|
var fileDescriptor_82cd95f524594f49 = []byte{
|
||||||
// 4318 bytes of a gzipped FileDescriptorProto
|
// 4353 bytes of a gzipped FileDescriptorProto
|
||||||
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe4, 0x3c, 0x4b, 0x6f, 0x24, 0x49,
|
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe4, 0x3c, 0x4b, 0x6f, 0x1c, 0x47,
|
||||||
0x5a, 0x9d, 0xf5, 0x72, 0xd5, 0x57, 0x0f, 0x97, 0xa3, 0x7b, 0xec, 0xea, 0x9a, 0x7e, 0x4d, 0xf6,
|
0x7a, 0xea, 0x79, 0x71, 0xe6, 0x9b, 0x07, 0x87, 0x25, 0x99, 0x1c, 0x8d, 0x9e, 0x6e, 0x59, 0xb6,
|
||||||
0xf4, 0x4c, 0x4f, 0x4f, 0x4f, 0xf7, 0x8c, 0x87, 0xd1, 0x0e, 0xf4, 0xce, 0xac, 0xda, 0xf6, 0xd8,
|
0x2c, 0xcb, 0x92, 0x4d, 0xc7, 0x58, 0x27, 0x5a, 0x7b, 0x21, 0x92, 0xa6, 0x3c, 0x09, 0xc9, 0xe5,
|
||||||
0x53, 0x60, 0x7b, 0xbd, 0x69, 0xf7, 0xb4, 0xb4, 0x8b, 0x54, 0x4a, 0x57, 0x86, 0xcb, 0xb9, 0xce,
|
0x36, 0x29, 0x0b, 0xd8, 0x0d, 0x30, 0x68, 0x4e, 0x17, 0x87, 0xbd, 0xec, 0xc7, 0xa8, 0xbb, 0x87,
|
||||||
0x47, 0x75, 0x66, 0x96, 0xdd, 0x5e, 0x0e, 0x3b, 0x62, 0x25, 0xa4, 0x45, 0x88, 0x45, 0x20, 0x04,
|
0x14, 0x37, 0x87, 0x35, 0xb2, 0x40, 0x00, 0x07, 0x41, 0x36, 0x48, 0x10, 0x24, 0x39, 0x04, 0x08,
|
||||||
0x48, 0x20, 0x21, 0x4e, 0xcb, 0x22, 0x24, 0xa4, 0x15, 0x17, 0x2e, 0x5c, 0x47, 0x70, 0x58, 0x21,
|
0x72, 0xda, 0x6c, 0x90, 0x20, 0xc0, 0x22, 0x97, 0x5c, 0x72, 0x35, 0x92, 0x83, 0x11, 0x04, 0xc8,
|
||||||
0x24, 0x7e, 0x00, 0x07, 0xe0, 0xce, 0x95, 0x03, 0x8a, 0x47, 0x46, 0xbe, 0xab, 0xd2, 0x55, 0xdd,
|
0x0f, 0xc8, 0x21, 0xc9, 0x3d, 0xd7, 0x1c, 0x82, 0x7a, 0x74, 0xf5, 0x7b, 0xa6, 0x39, 0x23, 0x59,
|
||||||
0xd3, 0x88, 0xbd, 0x55, 0x44, 0x7e, 0x11, 0x5f, 0xc4, 0x17, 0xdf, 0x3b, 0xbe, 0x28, 0x68, 0x6b,
|
0x41, 0xf6, 0x36, 0x55, 0xfd, 0x55, 0x7d, 0x55, 0x5f, 0x7d, 0xef, 0xfa, 0x6a, 0xa0, 0xad, 0xa9,
|
||||||
0xaa, 0xa7, 0xf6, 0x07, 0xb6, 0xed, 0x68, 0x0f, 0x46, 0x8e, 0xed, 0xd9, 0x68, 0xc9, 0xd4, 0x8d,
|
0x9e, 0xda, 0x1f, 0xd8, 0xb6, 0xa3, 0xdd, 0x1f, 0x39, 0xb6, 0x67, 0xa3, 0x25, 0x53, 0x37, 0x4e,
|
||||||
0xd3, 0xb1, 0xcb, 0x5a, 0x0f, 0xc8, 0xe7, 0x6e, 0x63, 0x60, 0x9b, 0xa6, 0x6d, 0xb1, 0xae, 0x6e,
|
0xc6, 0x2e, 0x6b, 0xdd, 0x27, 0x9f, 0xbb, 0x8d, 0x81, 0x6d, 0x9a, 0xb6, 0xc5, 0xba, 0xba, 0x2d,
|
||||||
0x4b, 0xb7, 0x3c, 0xec, 0x58, 0xaa, 0xc1, 0xdb, 0x8d, 0xf0, 0x80, 0x6e, 0xc3, 0x1d, 0x1c, 0x63,
|
0xdd, 0xf2, 0xb0, 0x63, 0xa9, 0x06, 0x6f, 0x37, 0xc2, 0x03, 0xba, 0x0d, 0x77, 0x70, 0x84, 0x4d,
|
||||||
0x53, 0x65, 0x2d, 0x79, 0x01, 0xca, 0x9f, 0x99, 0x23, 0xef, 0x5c, 0xfe, 0x53, 0x09, 0x1a, 0x9b,
|
0x95, 0xb5, 0xe4, 0x05, 0x28, 0x7f, 0x6a, 0x8e, 0xbc, 0x33, 0xf9, 0x4f, 0x25, 0x68, 0x6c, 0x1a,
|
||||||
0xc6, 0xd8, 0x3d, 0x56, 0xf0, 0xb3, 0x31, 0x76, 0x3d, 0xf4, 0x3e, 0x94, 0x0e, 0x55, 0x17, 0x77,
|
0x63, 0xf7, 0x48, 0xc1, 0xcf, 0xc6, 0xd8, 0xf5, 0xd0, 0x7b, 0x50, 0x3a, 0x50, 0x5d, 0xdc, 0x91,
|
||||||
0xa4, 0x5b, 0xd2, 0xdd, 0xfa, 0xea, 0xb5, 0x07, 0x11, 0xac, 0x1c, 0xdf, 0x8e, 0x3b, 0x5c, 0x53,
|
0x6e, 0x4a, 0x77, 0xea, 0xab, 0x57, 0xef, 0x47, 0xb0, 0x72, 0x7c, 0xdb, 0xee, 0x70, 0x4d, 0x75,
|
||||||
0x5d, 0xac, 0x50, 0x48, 0x84, 0xa0, 0xa4, 0x1d, 0xf6, 0x36, 0x3a, 0x85, 0x5b, 0xd2, 0xdd, 0xa2,
|
0xb1, 0x42, 0x21, 0x11, 0x82, 0x92, 0x76, 0xd0, 0xdb, 0xe8, 0x14, 0x6e, 0x4a, 0x77, 0x8a, 0x0a,
|
||||||
0x42, 0x7f, 0xa3, 0x1b, 0x00, 0x2e, 0x1e, 0x9a, 0xd8, 0xf2, 0x7a, 0x1b, 0x6e, 0xa7, 0x78, 0xab,
|
0xfd, 0x8d, 0xae, 0x03, 0xb8, 0x78, 0x68, 0x62, 0xcb, 0xeb, 0x6d, 0xb8, 0x9d, 0xe2, 0xcd, 0xe2,
|
||||||
0x78, 0xb7, 0xa8, 0x84, 0x7a, 0x90, 0x0c, 0x8d, 0x81, 0x6d, 0x18, 0x78, 0xe0, 0xe9, 0xb6, 0xd5,
|
0x9d, 0xa2, 0x12, 0xea, 0x41, 0x32, 0x34, 0x06, 0xb6, 0x61, 0xe0, 0x81, 0xa7, 0xdb, 0x56, 0x6f,
|
||||||
0xdb, 0xe8, 0x94, 0xe8, 0xd8, 0x48, 0x9f, 0xfc, 0x1f, 0x12, 0x34, 0xf9, 0xd2, 0xdc, 0x91, 0x6d,
|
0xa3, 0x53, 0xa2, 0x63, 0x23, 0x7d, 0xf2, 0x7f, 0x48, 0xd0, 0xe4, 0x4b, 0x73, 0x47, 0xb6, 0xe5,
|
||||||
0xb9, 0x18, 0x7d, 0x08, 0x15, 0xd7, 0x53, 0xbd, 0xb1, 0xcb, 0x57, 0xf7, 0x7a, 0xea, 0xea, 0xf6,
|
0x62, 0xf4, 0x01, 0x54, 0x5c, 0x4f, 0xf5, 0xc6, 0x2e, 0x5f, 0xdd, 0x95, 0xd4, 0xd5, 0xed, 0x51,
|
||||||
0x29, 0x88, 0xc2, 0x41, 0x53, 0x97, 0x17, 0x47, 0x5f, 0x4c, 0xa2, 0x8f, 0x6d, 0xa1, 0x94, 0xd8,
|
0x10, 0x85, 0x83, 0xa6, 0x2e, 0x2f, 0x8e, 0xbe, 0x98, 0x44, 0x1f, 0xdb, 0x42, 0x29, 0xb1, 0x85,
|
||||||
0xc2, 0x5d, 0x58, 0x3c, 0x22, 0xab, 0xdb, 0x0f, 0x80, 0xca, 0x14, 0x28, 0xde, 0x4d, 0x66, 0xf2,
|
0x3b, 0xb0, 0x78, 0x48, 0x56, 0xb7, 0x17, 0x00, 0x95, 0x29, 0x50, 0xbc, 0x9b, 0xcc, 0xe4, 0xe9,
|
||||||
0x74, 0x13, 0x7f, 0xfb, 0x68, 0x1f, 0xab, 0x46, 0xa7, 0x42, 0x71, 0x85, 0x7a, 0xe4, 0x7f, 0x91,
|
0x26, 0xfe, 0xee, 0xe1, 0x1e, 0x56, 0x8d, 0x4e, 0x85, 0xe2, 0x0a, 0xf5, 0xc8, 0xff, 0x22, 0x41,
|
||||||
0xa0, 0x2d, 0xc0, 0xfd, 0x73, 0xb8, 0x02, 0xe5, 0x81, 0x3d, 0xb6, 0x3c, 0xba, 0xd5, 0xa6, 0xc2,
|
0x5b, 0x80, 0xfb, 0xe7, 0x70, 0x09, 0xca, 0x03, 0x7b, 0x6c, 0x79, 0x74, 0xab, 0x4d, 0x85, 0x35,
|
||||||
0x1a, 0xe8, 0x0d, 0x68, 0x0c, 0x8e, 0x55, 0xcb, 0xc2, 0x46, 0xdf, 0x52, 0x4d, 0x4c, 0x37, 0x55,
|
0xd0, 0xeb, 0xd0, 0x18, 0x1c, 0xa9, 0x96, 0x85, 0x8d, 0xbe, 0xa5, 0x9a, 0x98, 0x6e, 0xaa, 0xa6,
|
||||||
0x53, 0xea, 0xbc, 0x6f, 0x57, 0x35, 0x71, 0xae, 0xbd, 0xdd, 0x82, 0xfa, 0x48, 0x75, 0x3c, 0x3d,
|
0xd4, 0x79, 0xdf, 0x8e, 0x6a, 0xe2, 0x5c, 0x7b, 0xbb, 0x09, 0xf5, 0x91, 0xea, 0x78, 0x7a, 0x84,
|
||||||
0x42, 0xfd, 0x70, 0x17, 0xea, 0x42, 0x55, 0x77, 0x7b, 0xe6, 0xc8, 0x76, 0xbc, 0x4e, 0xf9, 0x96,
|
0xfa, 0xe1, 0x2e, 0xd4, 0x85, 0xaa, 0xee, 0xf6, 0xcc, 0x91, 0xed, 0x78, 0x9d, 0xf2, 0x4d, 0xe9,
|
||||||
0x74, 0xb7, 0xaa, 0x88, 0x36, 0xc1, 0xa0, 0xd3, 0x5f, 0x07, 0xaa, 0x7b, 0xd2, 0xdb, 0xe0, 0x3b,
|
0x4e, 0x55, 0x11, 0x6d, 0x82, 0x41, 0xa7, 0xbf, 0xf6, 0x55, 0xf7, 0xb8, 0xb7, 0xc1, 0x77, 0x14,
|
||||||
0x8a, 0xf4, 0xc9, 0x7f, 0x29, 0xc1, 0xf2, 0x63, 0xd7, 0xd5, 0x87, 0x56, 0x62, 0x67, 0xcb, 0x50,
|
0xe9, 0x93, 0xff, 0x42, 0x82, 0xe5, 0x47, 0xae, 0xab, 0x0f, 0xad, 0xc4, 0xce, 0x96, 0xa1, 0x62,
|
||||||
0xb1, 0x6c, 0x0d, 0xf7, 0x36, 0xe8, 0xd6, 0x8a, 0x0a, 0x6f, 0xa1, 0xd7, 0xa1, 0x36, 0xc2, 0xd8,
|
0xd9, 0x1a, 0xee, 0x6d, 0xd0, 0xad, 0x15, 0x15, 0xde, 0x42, 0x57, 0xa0, 0x36, 0xc2, 0xd8, 0xe9,
|
||||||
0xe9, 0x3b, 0xb6, 0xe1, 0x6f, 0xac, 0x4a, 0x3a, 0x14, 0xdb, 0xc0, 0xe8, 0x3b, 0xb0, 0xe4, 0xc6,
|
0x3b, 0xb6, 0xe1, 0x6f, 0xac, 0x4a, 0x3a, 0x14, 0xdb, 0xc0, 0xe8, 0x7b, 0xb0, 0xe4, 0xc6, 0x26,
|
||||||
0x26, 0x62, 0x7c, 0x55, 0x5f, 0xbd, 0xfd, 0x20, 0x21, 0x19, 0x0f, 0xe2, 0x48, 0x95, 0xe4, 0x68,
|
0x62, 0x7c, 0x55, 0x5f, 0xbd, 0x75, 0x3f, 0x21, 0x19, 0xf7, 0xe3, 0x48, 0x95, 0xe4, 0x68, 0xf9,
|
||||||
0xf9, 0xcb, 0x02, 0x5c, 0x16, 0x70, 0x6c, 0xad, 0xe4, 0x37, 0xa1, 0xbc, 0x8b, 0x87, 0x62, 0x79,
|
0x8b, 0x02, 0x5c, 0x14, 0x70, 0x6c, 0xad, 0xe4, 0x37, 0xa1, 0xbc, 0x8b, 0x87, 0x62, 0x79, 0xac,
|
||||||
0xac, 0x91, 0x87, 0xf2, 0xe2, 0xc8, 0x8a, 0xe1, 0x23, 0xcb, 0xc1, 0xea, 0xf1, 0xf3, 0x28, 0x27,
|
0x91, 0x87, 0xf2, 0xe2, 0xc8, 0x8a, 0xe1, 0x23, 0xcb, 0xc1, 0xea, 0xf1, 0xf3, 0x28, 0x27, 0xcf,
|
||||||
0xcf, 0xe3, 0x26, 0xd4, 0xf1, 0xf3, 0x91, 0xee, 0xe0, 0x3e, 0x61, 0x1c, 0x4a, 0xf2, 0x92, 0x02,
|
0xe3, 0x06, 0xd4, 0xf1, 0xf3, 0x91, 0xee, 0xe0, 0x3e, 0x61, 0x1c, 0x4a, 0xf2, 0x92, 0x02, 0xac,
|
||||||
0xac, 0xeb, 0x40, 0x37, 0xc3, 0xb2, 0xb1, 0x90, 0x5b, 0x36, 0xe4, 0xbf, 0x92, 0x60, 0x25, 0x71,
|
0x6b, 0x5f, 0x37, 0xc3, 0xb2, 0xb1, 0x90, 0x5b, 0x36, 0xe4, 0xbf, 0x94, 0x60, 0x25, 0x71, 0x4a,
|
||||||
0x4a, 0x5c, 0xd8, 0x14, 0x68, 0xd3, 0x9d, 0x07, 0x94, 0x21, 0x62, 0x47, 0x08, 0xfe, 0xd6, 0x24,
|
0x5c, 0xd8, 0x14, 0x68, 0xd3, 0x9d, 0x07, 0x94, 0x21, 0x62, 0x47, 0x08, 0xfe, 0xe6, 0x24, 0x82,
|
||||||
0x82, 0x07, 0xe0, 0x4a, 0x62, 0x7c, 0x68, 0x91, 0x85, 0xfc, 0x8b, 0x3c, 0x81, 0x95, 0x2d, 0xec,
|
0x07, 0xe0, 0x4a, 0x62, 0x7c, 0x68, 0x91, 0x85, 0xfc, 0x8b, 0x3c, 0x86, 0x95, 0xc7, 0xd8, 0xe3,
|
||||||
0x71, 0x04, 0xe4, 0x1b, 0x76, 0x67, 0x57, 0x56, 0x51, 0xa9, 0x2e, 0xc4, 0xa5, 0x5a, 0xfe, 0xbb,
|
0x08, 0xc8, 0x37, 0xec, 0xce, 0xae, 0xac, 0xa2, 0x52, 0x5d, 0x88, 0x4b, 0xb5, 0xfc, 0x77, 0x05,
|
||||||
0x82, 0x90, 0x45, 0x8a, 0xaa, 0x67, 0x1d, 0xd9, 0xe8, 0x1a, 0xd4, 0x04, 0x08, 0xe7, 0x8a, 0xa0,
|
0x21, 0x8b, 0x14, 0x55, 0xcf, 0x3a, 0xb4, 0xd1, 0x55, 0xa8, 0x09, 0x10, 0xce, 0x15, 0x41, 0x07,
|
||||||
0x03, 0x7d, 0x03, 0xca, 0x64, 0xa5, 0x8c, 0x25, 0x5a, 0xab, 0x6f, 0xa4, 0xef, 0x29, 0x34, 0xa7,
|
0xfa, 0x16, 0x94, 0xc9, 0x4a, 0x19, 0x4b, 0xb4, 0x56, 0x5f, 0x4f, 0xdf, 0x53, 0x68, 0x4e, 0x85,
|
||||||
0xc2, 0xe0, 0x51, 0x0f, 0x5a, 0xae, 0xa7, 0x3a, 0x5e, 0x7f, 0x64, 0xbb, 0xf4, 0x9c, 0x29, 0xe3,
|
0xc1, 0xa3, 0x1e, 0xb4, 0x5c, 0x4f, 0x75, 0xbc, 0xfe, 0xc8, 0x76, 0xe9, 0x39, 0x53, 0xc6, 0xa9,
|
||||||
0xd4, 0x57, 0xe5, 0xe8, 0x0c, 0x42, 0xad, 0xef, 0xb8, 0xc3, 0x3d, 0x0e, 0xa9, 0x34, 0xe9, 0x48,
|
0xaf, 0xca, 0xd1, 0x19, 0x84, 0x5a, 0xdf, 0x76, 0x87, 0xbb, 0x1c, 0x52, 0x69, 0xd2, 0x91, 0x7e,
|
||||||
0xbf, 0x89, 0x3e, 0x83, 0x06, 0xb6, 0xb4, 0x60, 0xa2, 0x52, 0xee, 0x89, 0xea, 0xd8, 0xd2, 0xc4,
|
0x13, 0x7d, 0x0a, 0x0d, 0x6c, 0x69, 0xc1, 0x44, 0xa5, 0xdc, 0x13, 0xd5, 0xb1, 0xa5, 0x89, 0x69,
|
||||||
0x34, 0xc1, 0xf9, 0x94, 0xf3, 0x9f, 0xcf, 0xef, 0x49, 0xd0, 0x49, 0x1e, 0xd0, 0x3c, 0x2a, 0xfb,
|
0x82, 0xf3, 0x29, 0xe7, 0x3f, 0x9f, 0xdf, 0x93, 0xa0, 0x93, 0x3c, 0xa0, 0x79, 0x54, 0xf6, 0x43,
|
||||||
0x11, 0x1b, 0x84, 0xd9, 0x01, 0x4d, 0x94, 0x70, 0x71, 0x48, 0x0a, 0x1f, 0x22, 0xff, 0xb1, 0x04,
|
0x36, 0x08, 0xb3, 0x03, 0x9a, 0x28, 0xe1, 0xe2, 0x90, 0x14, 0x3e, 0x44, 0xfe, 0x63, 0x09, 0x5e,
|
||||||
0xaf, 0x05, 0xcb, 0xa1, 0x9f, 0x5e, 0x16, 0xb7, 0xa0, 0x7b, 0xd0, 0xd6, 0xad, 0x81, 0x31, 0xd6,
|
0x0b, 0x96, 0x43, 0x3f, 0xbd, 0x2c, 0x6e, 0x41, 0x77, 0xa1, 0xad, 0x5b, 0x03, 0x63, 0xac, 0xe1,
|
||||||
0xf0, 0x13, 0xeb, 0x73, 0xac, 0x1a, 0xde, 0xf1, 0x39, 0x3d, 0xc3, 0xaa, 0x92, 0xe8, 0x97, 0x7f,
|
0x27, 0xd6, 0x67, 0x58, 0x35, 0xbc, 0xa3, 0x33, 0x7a, 0x86, 0x55, 0x25, 0xd1, 0x2f, 0xff, 0x44,
|
||||||
0x24, 0xc1, 0x72, 0x7c, 0x5d, 0xf3, 0x10, 0xe9, 0x57, 0xa0, 0xac, 0x5b, 0x47, 0xb6, 0x4f, 0xa3,
|
0x82, 0xe5, 0xf8, 0xba, 0xe6, 0x21, 0xd2, 0xaf, 0x40, 0x59, 0xb7, 0x0e, 0x6d, 0x9f, 0x46, 0xd7,
|
||||||
0x1b, 0x13, 0x84, 0x92, 0xe0, 0x62, 0xc0, 0xb2, 0x09, 0xaf, 0x6f, 0x61, 0xaf, 0x67, 0xb9, 0xd8,
|
0x27, 0x08, 0x25, 0xc1, 0xc5, 0x80, 0x65, 0x13, 0xae, 0x3c, 0xc6, 0x5e, 0xcf, 0x72, 0xb1, 0xe3,
|
||||||
0xf1, 0xd6, 0x74, 0xcb, 0xb0, 0x87, 0x7b, 0xaa, 0x77, 0x3c, 0x87, 0x40, 0x45, 0x64, 0xa3, 0x10,
|
0xad, 0xe9, 0x96, 0x61, 0x0f, 0x77, 0x55, 0xef, 0x68, 0x0e, 0x81, 0x8a, 0xc8, 0x46, 0x21, 0x26,
|
||||||
0x93, 0x0d, 0xf9, 0xa7, 0x12, 0x5c, 0x4b, 0xc7, 0xc7, 0xb7, 0xde, 0x85, 0xea, 0x91, 0x8e, 0x0d,
|
0x1b, 0xf2, 0xcf, 0x24, 0xb8, 0x9a, 0x8e, 0x8f, 0x6f, 0xbd, 0x0b, 0xd5, 0x43, 0x1d, 0x1b, 0x1a,
|
||||||
0x8d, 0xd0, 0x57, 0xa2, 0xf4, 0x15, 0x6d, 0x22, 0x58, 0x23, 0x02, 0xcc, 0x77, 0xf8, 0x46, 0x06,
|
0xa1, 0xaf, 0x44, 0xe9, 0x2b, 0xda, 0x44, 0xb0, 0x46, 0x04, 0x98, 0xef, 0xf0, 0xf5, 0x0c, 0x6e,
|
||||||
0x37, 0xef, 0x7b, 0x8e, 0x6e, 0x0d, 0xb7, 0x75, 0xd7, 0x53, 0x18, 0x7c, 0x88, 0x9e, 0xc5, 0xfc,
|
0xde, 0xf3, 0x1c, 0xdd, 0x1a, 0x6e, 0xe9, 0xae, 0xa7, 0x30, 0xf8, 0x10, 0x3d, 0x8b, 0xf9, 0xd9,
|
||||||
0x6c, 0xfc, 0xbb, 0x12, 0xdc, 0xd8, 0xc2, 0xde, 0xba, 0xd0, 0xcb, 0xe4, 0xbb, 0xee, 0x7a, 0xfa,
|
0xf8, 0x77, 0x25, 0xb8, 0xfe, 0x18, 0x7b, 0xeb, 0x42, 0x2f, 0x93, 0xef, 0xba, 0xeb, 0xe9, 0x03,
|
||||||
0xc0, 0x7d, 0xb1, 0xbe, 0x51, 0x0e, 0x03, 0x2d, 0xff, 0x44, 0x82, 0x9b, 0x99, 0x8b, 0xe1, 0xa4,
|
0xf7, 0xc5, 0xfa, 0x46, 0x39, 0x0c, 0xb4, 0xfc, 0x53, 0x09, 0x6e, 0x64, 0x2e, 0x86, 0x93, 0x8e,
|
||||||
0xe3, 0x7a, 0xc7, 0xd7, 0xca, 0xe9, 0x7a, 0xe7, 0x37, 0xf0, 0xf9, 0x17, 0xaa, 0x31, 0xc6, 0x7b,
|
0xeb, 0x1d, 0x5f, 0x2b, 0xa7, 0xeb, 0x9d, 0xdf, 0xc0, 0x67, 0x9f, 0xab, 0xc6, 0x18, 0xef, 0xaa,
|
||||||
0xaa, 0xee, 0x30, 0xbd, 0x33, 0xa3, 0x16, 0xfe, 0x5b, 0x09, 0xae, 0x6f, 0x61, 0x6f, 0xcf, 0xb7,
|
0xba, 0xc3, 0xf4, 0xce, 0x8c, 0x5a, 0xf8, 0x6f, 0x24, 0xb8, 0xf6, 0x18, 0x7b, 0xbb, 0xbe, 0x4d,
|
||||||
0x49, 0xaf, 0x90, 0x3a, 0x04, 0x26, 0x64, 0x1b, 0x7d, 0xe7, 0x2c, 0xd2, 0x27, 0xff, 0x3e, 0x3b,
|
0x7a, 0x85, 0xd4, 0x21, 0x30, 0x21, 0xdb, 0xe8, 0x3b, 0x67, 0x91, 0x3e, 0xf9, 0xf7, 0xd9, 0x71,
|
||||||
0xce, 0xd4, 0xf5, 0xbe, 0x12, 0x02, 0xde, 0xa0, 0x92, 0x10, 0x12, 0xc9, 0x75, 0xe6, 0x3a, 0x70,
|
0xa6, 0xae, 0xf7, 0x95, 0x10, 0xf0, 0x3a, 0x95, 0x84, 0x90, 0x48, 0xae, 0x33, 0xd7, 0x81, 0x93,
|
||||||
0xf2, 0xc9, 0x7f, 0x21, 0xc1, 0xd5, 0xc7, 0x83, 0x67, 0x63, 0xdd, 0xc1, 0x1c, 0x68, 0xdb, 0x1e,
|
0x4f, 0xfe, 0x73, 0x09, 0x2e, 0x3f, 0x1a, 0x3c, 0x1b, 0xeb, 0x0e, 0xe6, 0x40, 0x5b, 0xf6, 0xe0,
|
||||||
0x9c, 0xcc, 0x4e, 0xdc, 0xc0, 0xcd, 0x2a, 0x44, 0xdc, 0xac, 0x69, 0xae, 0xf9, 0x32, 0x54, 0x3c,
|
0x78, 0x76, 0xe2, 0x06, 0x6e, 0x56, 0x21, 0xe2, 0x66, 0x4d, 0x73, 0xcd, 0x97, 0xa1, 0xe2, 0x31,
|
||||||
0xe6, 0xd7, 0x31, 0x4f, 0x85, 0xb7, 0xe8, 0xfa, 0x14, 0x6c, 0x60, 0xd5, 0xfd, 0xbf, 0xb9, 0xbe,
|
0xbf, 0x8e, 0x79, 0x2a, 0xbc, 0x45, 0xd7, 0xa7, 0x60, 0x03, 0xab, 0xee, 0xff, 0xcd, 0xf5, 0xfd,
|
||||||
0x9f, 0x94, 0xa0, 0xf1, 0x05, 0x77, 0xc7, 0xa8, 0xd5, 0x8e, 0x73, 0x92, 0x94, 0xee, 0x78, 0x85,
|
0xb4, 0x04, 0x8d, 0xcf, 0xb9, 0x3b, 0x46, 0xad, 0x76, 0x9c, 0x93, 0xa4, 0x74, 0xc7, 0x2b, 0xe4,
|
||||||
0x3c, 0xb8, 0x34, 0xa7, 0x6e, 0x0b, 0x9a, 0x2e, 0xc6, 0x27, 0xb3, 0xd8, 0xe8, 0x06, 0x19, 0x28,
|
0xc1, 0xa5, 0x39, 0x75, 0x8f, 0xa1, 0xe9, 0x62, 0x7c, 0x3c, 0x8b, 0x8d, 0x6e, 0x90, 0x81, 0xc2,
|
||||||
0x6c, 0xeb, 0x36, 0x2c, 0x8d, 0x2d, 0x1a, 0x1a, 0x60, 0x8d, 0x13, 0x90, 0x71, 0xee, 0x74, 0xdd,
|
0xb6, 0x6e, 0xc1, 0xd2, 0xd8, 0xa2, 0xa1, 0x01, 0xd6, 0x38, 0x01, 0x19, 0xe7, 0x4e, 0xd7, 0xdd,
|
||||||
0x9d, 0x1c, 0x88, 0x3e, 0xe7, 0xd1, 0x47, 0x68, 0xae, 0x72, 0xae, 0xb9, 0xe2, 0xc3, 0x50, 0x0f,
|
0xc9, 0x81, 0xe8, 0x33, 0x1e, 0x7d, 0x84, 0xe6, 0x2a, 0xe7, 0x9a, 0x2b, 0x3e, 0x0c, 0xf5, 0xa0,
|
||||||
0xda, 0x9a, 0x63, 0x8f, 0x46, 0x58, 0xeb, 0xbb, 0xfe, 0x54, 0x95, 0x7c, 0x53, 0xf1, 0x71, 0x62,
|
0xad, 0x39, 0xf6, 0x68, 0x84, 0xb5, 0xbe, 0xeb, 0x4f, 0x55, 0xc9, 0x37, 0x15, 0x1f, 0x27, 0xa6,
|
||||||
0xaa, 0xf7, 0xe1, 0x72, 0x7c, 0xa5, 0x3d, 0x8d, 0x38, 0xa4, 0xe4, 0x0c, 0xd3, 0x3e, 0xa1, 0xfb,
|
0x7a, 0x0f, 0x2e, 0xc6, 0x57, 0xda, 0xd3, 0x88, 0x43, 0x4a, 0xce, 0x30, 0xed, 0x13, 0xba, 0x07,
|
||||||
0xb0, 0x94, 0x84, 0xaf, 0x52, 0xf8, 0xe4, 0x07, 0xf4, 0x1e, 0xa0, 0xd8, 0x52, 0x09, 0x78, 0x8d,
|
0x4b, 0x49, 0xf8, 0x2a, 0x85, 0x4f, 0x7e, 0x40, 0xef, 0x02, 0x8a, 0x2d, 0x95, 0x80, 0xd7, 0x18,
|
||||||
0x81, 0x47, 0x17, 0xd3, 0xd3, 0x5c, 0xf9, 0xc7, 0x12, 0x2c, 0x3f, 0x55, 0xbd, 0xc1, 0xf1, 0x86,
|
0x78, 0x74, 0x31, 0x3d, 0xcd, 0x95, 0xbf, 0x94, 0x60, 0xf9, 0xa9, 0xea, 0x0d, 0x8e, 0x36, 0x4c,
|
||||||
0xc9, 0x65, 0x6d, 0x0e, 0x5d, 0xf5, 0x09, 0xd4, 0x4e, 0x39, 0x5f, 0xf8, 0x06, 0xe9, 0x66, 0x0a,
|
0x2e, 0x6b, 0x73, 0xe8, 0xaa, 0x8f, 0xa1, 0x76, 0xc2, 0xf9, 0xc2, 0x37, 0x48, 0x37, 0x52, 0xe8,
|
||||||
0x7d, 0xc2, 0x1c, 0xa8, 0x04, 0x23, 0xe4, 0xaf, 0x24, 0xb8, 0xb2, 0x19, 0x8a, 0x0b, 0x5f, 0x81,
|
0x13, 0xe6, 0x40, 0x25, 0x18, 0x41, 0xe2, 0xa1, 0x4b, 0x9b, 0xa1, 0xb8, 0xf0, 0x15, 0x68, 0xcd,
|
||||||
0xd6, 0x9c, 0x16, 0xd0, 0xbe, 0x05, 0x2d, 0x53, 0x75, 0x4e, 0x12, 0xf1, 0x6c, 0xac, 0x57, 0x7e,
|
0x29, 0x01, 0xad, 0xfc, 0x1c, 0x80, 0x2f, 0x6e, 0xdb, 0x1d, 0xce, 0xb0, 0xae, 0x8f, 0x60, 0x81,
|
||||||
0x0e, 0xc0, 0x5b, 0x3b, 0xee, 0x70, 0x86, 0xf5, 0x7f, 0x0c, 0x0b, 0x1c, 0x2b, 0x57, 0x9f, 0xd3,
|
0xcf, 0xc6, 0xd5, 0xe2, 0x34, 0xfe, 0xf1, 0xc1, 0xe5, 0x9f, 0x57, 0xa0, 0x1e, 0xfa, 0x80, 0x5a,
|
||||||
0xf8, 0xcc, 0x07, 0x97, 0x7f, 0x56, 0x81, 0x7a, 0xe8, 0x03, 0x6a, 0x41, 0x41, 0xc8, 0x75, 0x21,
|
0x50, 0x10, 0xf2, 0x5a, 0x48, 0xd9, 0x5d, 0x61, 0x7a, 0x08, 0x55, 0x4c, 0x86, 0x50, 0xb7, 0xa1,
|
||||||
0x85, 0x0a, 0x85, 0xe9, 0xa1, 0x56, 0x31, 0x19, 0x6a, 0xdd, 0x81, 0x96, 0x4e, 0xfd, 0x95, 0x3e,
|
0xa5, 0x53, 0x3f, 0xa4, 0xcf, 0x4f, 0x85, 0x2a, 0x90, 0x9a, 0xd2, 0x64, 0xbd, 0x9c, 0x45, 0xd0,
|
||||||
0x3f, 0x3d, 0xaa, 0x68, 0x6a, 0x4a, 0x93, 0xf5, 0x72, 0x56, 0x42, 0x37, 0xa0, 0x6e, 0x8d, 0xcd,
|
0x75, 0xa8, 0x5b, 0x63, 0xb3, 0x6f, 0x1f, 0xf6, 0x1d, 0xfb, 0xd4, 0xe5, 0xb1, 0x58, 0xcd, 0x1a,
|
||||||
0xbe, 0x7d, 0xd4, 0x77, 0xec, 0x33, 0x97, 0xc7, 0x6c, 0x35, 0x6b, 0x6c, 0x7e, 0xfb, 0x48, 0xb1,
|
0x9b, 0xdf, 0x3d, 0x54, 0xec, 0x53, 0x37, 0x70, 0xf7, 0x2b, 0xe7, 0x74, 0xf7, 0xaf, 0x43, 0xdd,
|
||||||
0xcf, 0xdc, 0x20, 0x2c, 0xa8, 0x5c, 0x30, 0x2c, 0xb8, 0x01, 0x75, 0x53, 0x7d, 0x4e, 0x66, 0xed,
|
0x54, 0x9f, 0x93, 0x59, 0xfb, 0xd6, 0xd8, 0xa4, 0x61, 0x5a, 0x51, 0xa9, 0x99, 0xea, 0x73, 0xc5,
|
||||||
0x5b, 0x63, 0x93, 0x86, 0x73, 0x45, 0xa5, 0x66, 0xaa, 0xcf, 0x15, 0xfb, 0x6c, 0x77, 0x6c, 0xa2,
|
0x3e, 0xdd, 0x19, 0x9b, 0xe8, 0x0e, 0xb4, 0x0d, 0xd5, 0xf5, 0xfa, 0xe1, 0x38, 0xaf, 0x4a, 0xe3,
|
||||||
0xbb, 0xd0, 0x36, 0x54, 0xd7, 0xeb, 0x87, 0xe3, 0xc1, 0x2a, 0x8d, 0x07, 0x5b, 0xa4, 0xff, 0xb3,
|
0xbc, 0x16, 0xe9, 0xff, 0x34, 0x88, 0xf5, 0x92, 0x81, 0x43, 0x6d, 0x8e, 0xc0, 0x41, 0x33, 0x8d,
|
||||||
0x20, 0x26, 0x4c, 0x06, 0x18, 0xb5, 0x39, 0x02, 0x0c, 0xcd, 0x34, 0x82, 0x89, 0x20, 0x7f, 0x80,
|
0x60, 0x22, 0xc8, 0x1f, 0x38, 0x68, 0xa6, 0x21, 0xa6, 0xf9, 0x08, 0x16, 0x0e, 0xa8, 0x77, 0xe7,
|
||||||
0xa1, 0x99, 0x86, 0x98, 0xe6, 0x63, 0x58, 0x38, 0xa4, 0x5e, 0xa0, 0xdb, 0xa9, 0x67, 0xea, 0x98,
|
0x76, 0xea, 0x99, 0xba, 0x63, 0x93, 0x38, 0x76, 0xcc, 0x09, 0x54, 0x7c, 0x70, 0xf4, 0x6d, 0xa8,
|
||||||
0x4d, 0xe2, 0x00, 0x32, 0x67, 0x51, 0xf1, 0xc1, 0xd1, 0x37, 0xa1, 0x46, 0x8d, 0x2f, 0x1d, 0xdb,
|
0x51, 0xa3, 0x4a, 0xc7, 0x36, 0x72, 0x8d, 0x0d, 0x06, 0x90, 0xd1, 0x1a, 0x36, 0x3c, 0x95, 0x8e,
|
||||||
0xc8, 0x35, 0x36, 0x18, 0x40, 0x46, 0x6b, 0xd8, 0xf0, 0x54, 0x3a, 0xba, 0x99, 0x6f, 0xb4, 0x18,
|
0x6e, 0xe6, 0x1b, 0x2d, 0x06, 0x10, 0x7d, 0x35, 0x70, 0xb0, 0xea, 0x61, 0x6d, 0xed, 0x6c, 0xdd,
|
||||||
0x40, 0xf4, 0xda, 0xc0, 0xc1, 0xaa, 0x87, 0xb5, 0xb5, 0xf3, 0x75, 0xdb, 0x1c, 0xa9, 0x94, 0x99,
|
0x36, 0x47, 0x2a, 0x65, 0xa6, 0x4e, 0x8b, 0x7a, 0xf0, 0x69, 0x9f, 0xd0, 0x9b, 0xd0, 0x1a, 0x88,
|
||||||
0x3a, 0x2d, 0xea, 0xe9, 0xa7, 0x7d, 0x22, 0xb2, 0x34, 0x10, 0xad, 0x4d, 0xc7, 0x36, 0x3b, 0x8b,
|
0xd6, 0xa6, 0x63, 0x9b, 0x9d, 0x45, 0x2a, 0x47, 0xb1, 0x5e, 0x74, 0x0d, 0xc0, 0xd7, 0x54, 0xaa,
|
||||||
0x4c, 0x96, 0xa2, 0xbd, 0xe8, 0x3a, 0x80, 0xaf, 0xd1, 0x54, 0xaf, 0xd3, 0xa6, 0xa7, 0x58, 0xe3,
|
0xd7, 0x69, 0xd3, 0x53, 0xac, 0xf1, 0x9e, 0x47, 0x34, 0x8d, 0xa3, 0xbb, 0x7d, 0x96, 0x30, 0xd1,
|
||||||
0x3d, 0x8f, 0x69, 0xba, 0x47, 0x77, 0xfb, 0x2c, 0xb1, 0xa2, 0x5b, 0xc3, 0xce, 0x12, 0xc5, 0x58,
|
0xad, 0x61, 0x67, 0x89, 0x62, 0xac, 0xfb, 0x19, 0x16, 0xdd, 0x1a, 0xa2, 0x15, 0x58, 0xd0, 0xdd,
|
||||||
0xf7, 0x33, 0x31, 0xba, 0x35, 0x44, 0x2b, 0xb0, 0xa0, 0xbb, 0xfd, 0x23, 0xf5, 0x04, 0x77, 0x10,
|
0xfe, 0xa1, 0x7a, 0x8c, 0x3b, 0x88, 0x7e, 0xad, 0xe8, 0xee, 0xa6, 0x7a, 0x8c, 0xe5, 0x1f, 0xc3,
|
||||||
0xfd, 0x5a, 0xd1, 0xdd, 0x4d, 0xf5, 0x04, 0xcb, 0x3f, 0x84, 0x2b, 0x01, 0x77, 0x85, 0x4e, 0x32,
|
0xa5, 0x80, 0xbb, 0x42, 0x27, 0x99, 0x64, 0x0a, 0x69, 0x56, 0xa6, 0x98, 0xec, 0xd3, 0x7f, 0x5d,
|
||||||
0xc9, 0x14, 0xd2, 0xac, 0x4c, 0x31, 0xd9, 0xf7, 0xff, 0x45, 0x09, 0x96, 0xf7, 0xd5, 0x53, 0xfc,
|
0x82, 0xe5, 0x3d, 0xf5, 0x04, 0xbf, 0xfc, 0xf0, 0x21, 0x97, 0x5a, 0xdb, 0x82, 0x25, 0x1a, 0x31,
|
||||||
0xf2, 0xc3, 0x8c, 0x5c, 0xea, 0x6f, 0x1b, 0x96, 0x68, 0x64, 0xb1, 0x1a, 0x5a, 0xcf, 0x04, 0xfb,
|
0xac, 0x86, 0xd6, 0x33, 0xc1, 0xae, 0x86, 0x59, 0x21, 0x39, 0x10, 0x7d, 0x87, 0x38, 0x04, 0x78,
|
||||||
0x1b, 0x66, 0x85, 0xe4, 0x40, 0xf4, 0x2d, 0xe2, 0x38, 0xe0, 0xc1, 0xc9, 0x9e, 0xad, 0x07, 0xb6,
|
0x70, 0xbc, 0x6b, 0xeb, 0x81, 0x4d, 0xbd, 0x96, 0x32, 0xcf, 0xba, 0x80, 0x52, 0xc2, 0x23, 0xd0,
|
||||||
0xf7, 0x7a, 0xca, 0x3c, 0xeb, 0x02, 0x4a, 0x09, 0x8f, 0x40, 0x7b, 0xb0, 0x18, 0x3d, 0x06, 0xdf,
|
0x2e, 0x2c, 0x46, 0x8f, 0xc1, 0xb7, 0xa6, 0x6f, 0x4d, 0x0c, 0x62, 0x03, 0xea, 0x2b, 0xad, 0xc8,
|
||||||
0xea, 0xbe, 0x3d, 0x31, 0xd8, 0x0d, 0xa8, 0xaf, 0xb4, 0x22, 0x87, 0xe1, 0xa2, 0x0e, 0x2c, 0x70,
|
0x61, 0xb8, 0xa8, 0x03, 0x0b, 0xdc, 0x14, 0x52, 0x9d, 0x51, 0x55, 0xfc, 0x26, 0xda, 0x85, 0x8b,
|
||||||
0x93, 0x49, 0x75, 0x46, 0x55, 0xf1, 0x9b, 0x68, 0x0f, 0x2e, 0xb3, 0x1d, 0xec, 0x73, 0x81, 0x60,
|
0x6c, 0x07, 0x7b, 0x5c, 0x20, 0xd8, 0xe6, 0xab, 0xb9, 0x36, 0x9f, 0x36, 0x34, 0x2a, 0x4f, 0xb5,
|
||||||
0x9b, 0xaf, 0xe6, 0xda, 0x7c, 0xda, 0xd0, 0xa8, 0x3c, 0xd5, 0x2e, 0x2a, 0x4f, 0x1d, 0x58, 0xe0,
|
0xf3, 0xca, 0x53, 0x07, 0x16, 0x38, 0x8f, 0x53, 0x3d, 0x52, 0x55, 0xfc, 0x26, 0x39, 0xe6, 0x80,
|
||||||
0x3c, 0x4e, 0xf5, 0x48, 0x55, 0xf1, 0x9b, 0xe4, 0x98, 0x03, 0x6e, 0xaf, 0xd3, 0x6f, 0x41, 0x07,
|
0xdb, 0xeb, 0xf4, 0x5b, 0xd0, 0x41, 0x42, 0x2f, 0x08, 0xe8, 0x39, 0x25, 0xdd, 0xf2, 0x09, 0x54,
|
||||||
0x09, 0xd1, 0x20, 0xa0, 0xe7, 0x94, 0xb4, 0xcc, 0xa7, 0x50, 0x15, 0x1c, 0x5e, 0xc8, 0xcd, 0xe1,
|
0x05, 0x87, 0x17, 0x72, 0x73, 0xb8, 0x18, 0x13, 0xd7, 0xef, 0xc5, 0x98, 0x7e, 0x97, 0xff, 0x59,
|
||||||
0x62, 0x4c, 0x5c, 0xbf, 0x17, 0x63, 0xfa, 0x5d, 0xfe, 0x67, 0x09, 0x1a, 0x1b, 0x64, 0x4b, 0xdb,
|
0x82, 0xc6, 0x06, 0xd9, 0xd2, 0x96, 0x3d, 0xa4, 0xd6, 0xe8, 0x36, 0xb4, 0x1c, 0x3c, 0xb0, 0x1d,
|
||||||
0xf6, 0x90, 0x5a, 0xa3, 0x3b, 0xd0, 0x72, 0xf0, 0xc0, 0x76, 0xb4, 0x3e, 0xb6, 0x3c, 0x47, 0xc7,
|
0xad, 0x8f, 0x2d, 0xcf, 0xd1, 0x31, 0x8b, 0xd2, 0x4b, 0x4a, 0x93, 0xf5, 0x7e, 0xca, 0x3a, 0x09,
|
||||||
0x2c, 0x9a, 0x2f, 0x29, 0x4d, 0xd6, 0xfb, 0x19, 0xeb, 0x24, 0x60, 0x44, 0x65, 0xbb, 0x9e, 0x6a,
|
0x18, 0x51, 0xd9, 0xae, 0xa7, 0x9a, 0xa3, 0xfe, 0x21, 0x51, 0x0d, 0x05, 0x06, 0x26, 0x7a, 0xa9,
|
||||||
0x8e, 0xfa, 0x47, 0x44, 0x35, 0x14, 0x18, 0x98, 0xe8, 0xa5, 0x9a, 0xe1, 0x0d, 0x68, 0x04, 0x60,
|
0x66, 0x78, 0x1d, 0x1a, 0x01, 0x98, 0x67, 0x53, 0xfc, 0x25, 0xa5, 0x2e, 0xfa, 0xf6, 0x6d, 0xf4,
|
||||||
0x9e, 0x4d, 0xf1, 0x97, 0x94, 0xba, 0xe8, 0x3b, 0xb0, 0xd1, 0x9b, 0xd0, 0xa2, 0x34, 0xed, 0x1b,
|
0x06, 0xb4, 0x28, 0x4d, 0xfb, 0x86, 0x3d, 0xec, 0x93, 0x88, 0x96, 0x1b, 0xaa, 0x86, 0xc6, 0x97,
|
||||||
0xf6, 0xb0, 0x4f, 0x22, 0x5f, 0x6e, 0xa8, 0x1a, 0x1a, 0x5f, 0x16, 0x39, 0xab, 0x28, 0x94, 0xab,
|
0x45, 0xce, 0x2a, 0x0a, 0xe5, 0xea, 0x3f, 0xc2, 0xdc, 0x54, 0x09, 0xa8, 0x3d, 0xfd, 0x47, 0x58,
|
||||||
0xff, 0x00, 0x73, 0x53, 0x25, 0xa0, 0xf6, 0xf5, 0x1f, 0x60, 0xf9, 0x9f, 0x24, 0x68, 0x6e, 0xa8,
|
0xfe, 0x27, 0x09, 0x9a, 0x1b, 0xaa, 0xa7, 0xee, 0xd8, 0x1a, 0xde, 0x9f, 0xd1, 0xb0, 0xe7, 0x48,
|
||||||
0x9e, 0xba, 0x6b, 0x6b, 0xf8, 0x60, 0x46, 0xc3, 0x9e, 0x23, 0x45, 0x7a, 0x0d, 0x6a, 0x62, 0x07,
|
0x7d, 0x5e, 0x85, 0x9a, 0xd8, 0x01, 0xdf, 0x52, 0xd0, 0x81, 0x36, 0xa1, 0xe5, 0xbb, 0x96, 0x7d,
|
||||||
0x7c, 0x4b, 0x41, 0x07, 0xda, 0x84, 0x96, 0xef, 0x82, 0xf6, 0x59, 0x64, 0x56, 0xca, 0x74, 0xb4,
|
0x16, 0x71, 0x95, 0x32, 0x1d, 0xa8, 0x90, 0xe5, 0x74, 0x95, 0xa6, 0x3f, 0x8c, 0x36, 0xe5, 0x4d,
|
||||||
0x42, 0x96, 0xd3, 0x55, 0x9a, 0xfe, 0x30, 0xda, 0x94, 0x37, 0xa1, 0x11, 0xfe, 0x4c, 0xb0, 0xee,
|
0x68, 0x84, 0x3f, 0x13, 0xac, 0x7b, 0x71, 0x46, 0x11, 0x1d, 0x84, 0x1b, 0x77, 0xc6, 0x26, 0x39,
|
||||||
0xc7, 0x19, 0x45, 0x74, 0x10, 0x6e, 0xdc, 0x1d, 0x9b, 0xe4, 0x4c, 0xb9, 0x62, 0xf1, 0x9b, 0xf2,
|
0x53, 0xae, 0x58, 0xfc, 0xa6, 0xfc, 0x13, 0x09, 0x9a, 0xdc, 0xdc, 0xef, 0x89, 0x4b, 0x02, 0xba,
|
||||||
0x8f, 0x24, 0x68, 0x72, 0x73, 0xbf, 0x2f, 0x2e, 0x13, 0xe8, 0xd6, 0x24, 0xba, 0x35, 0xfa, 0x1b,
|
0x35, 0x89, 0x6e, 0x8d, 0xfe, 0x46, 0xbf, 0x16, 0xcd, 0xeb, 0xbd, 0x91, 0xaa, 0x04, 0xe8, 0x24,
|
||||||
0xfd, 0x5a, 0x34, 0xff, 0xf7, 0x66, 0xaa, 0x12, 0xa0, 0x93, 0x50, 0x67, 0x34, 0x62, 0xeb, 0xf3,
|
0xd4, 0xc9, 0x8c, 0xd8, 0xfa, 0x3c, 0x31, 0xfe, 0x17, 0x84, 0xd1, 0xf8, 0xd1, 0x50, 0x46, 0xeb,
|
||||||
0xe4, 0x02, 0xbe, 0x24, 0x8c, 0xc6, 0x8f, 0x86, 0x32, 0x5a, 0x07, 0x16, 0x54, 0x4d, 0x73, 0xb0,
|
0xc0, 0x82, 0xaa, 0x69, 0x0e, 0x76, 0x5d, 0xbe, 0x0e, 0xbf, 0x49, 0xbe, 0x9c, 0x60, 0xc7, 0xf5,
|
||||||
0xeb, 0xf2, 0x75, 0xf8, 0x4d, 0xf2, 0xe5, 0x14, 0x3b, 0xae, 0xcf, 0xf2, 0x45, 0xc5, 0x6f, 0xa2,
|
0x59, 0xbe, 0xa8, 0xf8, 0x4d, 0xf4, 0x6d, 0xa8, 0x0a, 0xaf, 0x94, 0xa5, 0xc3, 0x6f, 0x66, 0xaf,
|
||||||
0x6f, 0x42, 0x55, 0x78, 0xaf, 0x2c, 0x6d, 0x7e, 0x2b, 0x7b, 0x9d, 0x3c, 0x72, 0x15, 0x23, 0xe4,
|
0x93, 0x47, 0xa4, 0x62, 0x84, 0xfc, 0xf7, 0x05, 0x68, 0x71, 0x82, 0xad, 0x71, 0x7b, 0x3c, 0x59,
|
||||||
0xbf, 0x2f, 0x40, 0x8b, 0x13, 0x6c, 0x8d, 0xdb, 0xe3, 0xc9, 0xc2, 0xb7, 0x06, 0x8d, 0xa3, 0x40,
|
0xf8, 0xd6, 0xa0, 0x71, 0x18, 0xc8, 0xfe, 0xa4, 0xdc, 0x53, 0x58, 0x45, 0x44, 0xc6, 0x4c, 0x13,
|
||||||
0xf6, 0x27, 0xe5, 0xa8, 0xc2, 0x2a, 0x22, 0x32, 0x66, 0x9a, 0x00, 0x46, 0x3d, 0x82, 0xd2, 0x5c,
|
0xc0, 0xa8, 0x47, 0x50, 0x9a, 0xcb, 0x23, 0x28, 0x9f, 0x57, 0x83, 0x25, 0x7d, 0xc4, 0x4a, 0x8a,
|
||||||
0x1e, 0x41, 0xf9, 0xa2, 0x1a, 0x2c, 0xe9, 0x23, 0x56, 0x52, 0x7c, 0x44, 0xf9, 0x37, 0xa1, 0x1e,
|
0x8f, 0x28, 0xff, 0x26, 0xd4, 0x43, 0x13, 0x50, 0x0d, 0xcd, 0x92, 0x56, 0x9c, 0x62, 0x7e, 0x13,
|
||||||
0x9a, 0x80, 0x6a, 0x68, 0x96, 0xdc, 0xe2, 0x14, 0xf3, 0x9b, 0xe8, 0xc3, 0xc0, 0x2f, 0x62, 0xa4,
|
0x7d, 0x10, 0xf8, 0x45, 0x8c, 0x54, 0x97, 0x53, 0xd6, 0x12, 0x73, 0x89, 0xe4, 0x7f, 0x94, 0xa0,
|
||||||
0xba, 0x9a, 0xb2, 0x96, 0x98, 0x4b, 0x24, 0xff, 0xa3, 0x04, 0x15, 0x3e, 0xf3, 0x4d, 0xa8, 0x73,
|
0xc2, 0x67, 0xbe, 0x01, 0x75, 0xae, 0x74, 0xa8, 0xcf, 0xc8, 0x66, 0x07, 0xde, 0x45, 0x9c, 0xc6,
|
||||||
0xa5, 0x43, 0x7d, 0x46, 0x36, 0x3b, 0xf0, 0x2e, 0xe2, 0x34, 0xbe, 0x38, 0xad, 0x73, 0x15, 0xaa,
|
0x17, 0xa7, 0x75, 0x2e, 0x43, 0x35, 0xa6, 0x6f, 0x16, 0xb8, 0x59, 0xf0, 0x3f, 0x85, 0x94, 0x0c,
|
||||||
0x31, 0x7d, 0xb3, 0xc0, 0xcd, 0x82, 0xff, 0x29, 0xa4, 0x64, 0xc8, 0x27, 0xa2, 0x5f, 0xd0, 0x15,
|
0xf9, 0x44, 0xf4, 0x0b, 0xba, 0x04, 0x65, 0xc3, 0x1e, 0x8a, 0x4b, 0x20, 0xd6, 0x90, 0xbf, 0x92,
|
||||||
0x28, 0x1b, 0xf6, 0x50, 0x5c, 0x16, 0xb1, 0x06, 0x89, 0x8a, 0x56, 0xb6, 0xb0, 0xa7, 0xe0, 0x81,
|
0x68, 0xce, 0x5e, 0xc1, 0x03, 0xfb, 0x04, 0x3b, 0x67, 0xf3, 0x27, 0x3b, 0x1f, 0x86, 0xd8, 0x3c,
|
||||||
0x7d, 0x8a, 0x9d, 0xf3, 0xf9, 0x93, 0xa2, 0x8f, 0x42, 0x6c, 0x9e, 0x33, 0x48, 0x13, 0x03, 0xd0,
|
0x67, 0xf0, 0x25, 0x06, 0xa0, 0x87, 0xc1, 0x21, 0x14, 0xd3, 0x32, 0x3d, 0x61, 0xbd, 0xc3, 0x99,
|
||||||
0xa3, 0xe0, 0x10, 0x8a, 0x69, 0x19, 0xa1, 0xb0, 0xde, 0xe1, 0x4c, 0x1a, 0x1c, 0xc6, 0x1f, 0xb0,
|
0x34, 0x38, 0x8c, 0x3f, 0x60, 0x69, 0xdb, 0xe8, 0x56, 0x66, 0xf5, 0x76, 0x5e, 0x48, 0x20, 0x23,
|
||||||
0xf4, 0x6e, 0x74, 0x2b, 0xb3, 0x7a, 0x3b, 0x2f, 0x24, 0x90, 0x91, 0x7f, 0x21, 0x41, 0x37, 0x48,
|
0x7f, 0x2d, 0x41, 0x37, 0x48, 0x25, 0xb9, 0x6b, 0x67, 0xf3, 0x5e, 0x8a, 0xbc, 0x98, 0xf8, 0xea,
|
||||||
0x39, 0xb9, 0x6b, 0xe7, 0xf3, 0x5e, 0x9e, 0xbc, 0x98, 0xf8, 0xea, 0x57, 0x45, 0x76, 0x9f, 0x08,
|
0x57, 0x45, 0xd6, 0x9e, 0x08, 0x6d, 0xae, 0xc8, 0xc8, 0xcf, 0xd9, 0x5b, 0x34, 0x2b, 0x9d, 0xdc,
|
||||||
0x6d, 0xae, 0xc8, 0xc8, 0xcf, 0xed, 0x5b, 0x34, 0x7b, 0x9d, 0xdc, 0xd0, 0x3c, 0x2c, 0xd3, 0x85,
|
0xd0, 0x3c, 0x2c, 0xd3, 0x85, 0xaa, 0xc8, 0x67, 0xb0, 0xcc, 0xbd, 0x68, 0xcb, 0x7f, 0x24, 0xc1,
|
||||||
0xaa, 0xc8, 0x7b, 0xb0, 0x0c, 0xbf, 0x68, 0xcb, 0x7f, 0x24, 0xc1, 0xd5, 0x2d, 0xec, 0x6d, 0x46,
|
0xe5, 0xc7, 0xd8, 0xdb, 0x8c, 0xa6, 0x42, 0x5e, 0xf5, 0xb9, 0x9a, 0xf4, 0x58, 0x13, 0x8b, 0x7a,
|
||||||
0x53, 0x26, 0xaf, 0xfa, 0x5c, 0x4d, 0x7a, 0xac, 0x89, 0x45, 0xbd, 0x2c, 0x22, 0xfc, 0x8e, 0x04,
|
0x59, 0x44, 0xf8, 0x1d, 0x09, 0x3a, 0x1c, 0x0b, 0xc5, 0x49, 0xc2, 0x1c, 0x03, 0x7b, 0x58, 0xfb,
|
||||||
0x1d, 0x8e, 0x85, 0xe2, 0x24, 0x61, 0x8e, 0x81, 0x3d, 0xac, 0x7d, 0xdd, 0xe1, 0xff, 0xff, 0x48,
|
0xa6, 0xc3, 0xff, 0xff, 0x91, 0xa0, 0x1d, 0xb6, 0xa4, 0xd4, 0x18, 0x7e, 0x08, 0x65, 0x9a, 0x3d,
|
||||||
0xd0, 0x0e, 0x5b, 0x52, 0x6a, 0x0c, 0x3f, 0x82, 0x32, 0xcd, 0xb2, 0xf0, 0x15, 0x4c, 0x15, 0x77,
|
0xe1, 0x2b, 0x98, 0x2a, 0xee, 0x0c, 0x9a, 0xa8, 0x62, 0xea, 0x3e, 0xef, 0x0b, 0xa3, 0xcf, 0x9b,
|
||||||
0x06, 0x4d, 0x54, 0x31, 0x75, 0x9f, 0x0f, 0x84, 0xd1, 0xe7, 0xcd, 0xc0, 0x9c, 0x17, 0x2f, 0x6e,
|
0x81, 0x39, 0x2f, 0x9e, 0xdf, 0x9c, 0x73, 0xf7, 0xc6, 0x1e, 0x93, 0x79, 0x59, 0xda, 0x31, 0xe8,
|
||||||
0xce, 0xb9, 0x7b, 0x63, 0x8f, 0xc9, 0xbc, 0x2c, 0x3d, 0x19, 0x74, 0xa0, 0x4f, 0xa0, 0xc2, 0x8a,
|
0x40, 0x1f, 0x43, 0x85, 0x15, 0x57, 0xf0, 0x5b, 0xb3, 0xdb, 0xd1, 0xa9, 0x79, 0xe1, 0x45, 0x28,
|
||||||
0x30, 0xf8, 0xed, 0xda, 0x9d, 0xe8, 0xd4, 0xbc, 0x40, 0x23, 0x94, 0xf3, 0xa7, 0x1d, 0x0a, 0x1f,
|
0x97, 0x4f, 0x3b, 0x14, 0x3e, 0x48, 0xfe, 0x75, 0x58, 0x0e, 0x22, 0x4c, 0x86, 0x76, 0x56, 0x46,
|
||||||
0x24, 0xff, 0x3a, 0x2c, 0x07, 0x11, 0x26, 0x43, 0x3b, 0x2b, 0x23, 0xca, 0xff, 0x26, 0xc1, 0xe5,
|
0x94, 0xff, 0x4d, 0x82, 0x8b, 0x7b, 0x67, 0xd6, 0x20, 0xce, 0xd2, 0xcb, 0x50, 0x19, 0x19, 0x6a,
|
||||||
0xfd, 0x73, 0x6b, 0x10, 0x67, 0xe9, 0x65, 0xa8, 0x8c, 0x0c, 0x35, 0xc8, 0x96, 0xf2, 0x16, 0x75,
|
0x90, 0x05, 0xe5, 0x2d, 0xea, 0xda, 0x31, 0xdc, 0x58, 0x23, 0x76, 0x81, 0xd1, 0xac, 0x2e, 0xfa,
|
||||||
0xed, 0x18, 0x6e, 0xac, 0x11, 0xbb, 0xc0, 0x68, 0x56, 0x17, 0x7d, 0x07, 0xf6, 0x54, 0x73, 0x7d,
|
0xf6, 0xed, 0xa9, 0xe6, 0xfa, 0xb6, 0x08, 0x89, 0xb1, 0xc6, 0x2c, 0x10, 0x4b, 0x2d, 0x35, 0x45,
|
||||||
0x47, 0x84, 0xc4, 0x58, 0x63, 0x16, 0x88, 0xa5, 0xa0, 0x9a, 0xa2, 0x97, 0x5a, 0xa0, 0x4f, 0x00,
|
0x2f, 0xb5, 0x40, 0x1f, 0x03, 0x50, 0x23, 0xdd, 0x3f, 0x8f, 0x61, 0xa6, 0x23, 0xb6, 0x88, 0x1a,
|
||||||
0xa8, 0x91, 0xee, 0x5f, 0xc4, 0x30, 0xd3, 0x11, 0xdb, 0x44, 0x0d, 0xff, 0xbc, 0x00, 0x9d, 0x10,
|
0xfe, 0x45, 0x01, 0x3a, 0x21, 0x2a, 0x7d, 0xd3, 0x3e, 0x4b, 0x46, 0xa4, 0x55, 0x7c, 0x41, 0x91,
|
||||||
0x95, 0xbe, 0x6e, 0x9f, 0x25, 0x23, 0xd2, 0x2a, 0xbe, 0xa0, 0x48, 0xab, 0x34, 0xbf, 0x9f, 0x52,
|
0x56, 0x69, 0x7e, 0x3f, 0xa5, 0x9c, 0xe6, 0xa7, 0xfc, 0x7b, 0x01, 0x5a, 0x01, 0xd5, 0x76, 0x0d,
|
||||||
0x4e, 0xf3, 0x53, 0xfe, 0xbd, 0x00, 0xad, 0x80, 0x6a, 0x7b, 0x86, 0x6a, 0x65, 0x72, 0xc2, 0xbe,
|
0xd5, 0xca, 0xe4, 0x84, 0x3d, 0xe1, 0xa3, 0x47, 0xe9, 0xf4, 0x4e, 0x9a, 0x9c, 0x64, 0x1c, 0x84,
|
||||||
0xf0, 0xd1, 0xa3, 0x74, 0x7a, 0x37, 0x4d, 0x4e, 0x32, 0x0e, 0x42, 0x89, 0x4d, 0x81, 0xae, 0xd3,
|
0x12, 0x9b, 0x02, 0x5d, 0xa3, 0x87, 0xee, 0x78, 0x2c, 0x99, 0xc5, 0xe3, 0x02, 0x26, 0x90, 0xba,
|
||||||
0x43, 0x77, 0x3c, 0x96, 0xcc, 0xe2, 0x71, 0x01, 0x13, 0x48, 0xdd, 0xc4, 0xe8, 0x3e, 0x20, 0x2e,
|
0x89, 0xd1, 0x3d, 0x40, 0x5c, 0x8a, 0xfa, 0xba, 0xd5, 0x77, 0xf1, 0xc0, 0xb6, 0x34, 0x26, 0x5f,
|
||||||
0x45, 0x7d, 0xdd, 0xea, 0xbb, 0x78, 0x60, 0x5b, 0x1a, 0x93, 0xaf, 0xb2, 0xd2, 0xe6, 0x5f, 0x7a,
|
0x65, 0xa5, 0xcd, 0xbf, 0xf4, 0xac, 0x3d, 0xd6, 0x8f, 0x3e, 0x84, 0x92, 0x77, 0x36, 0x62, 0x1e,
|
||||||
0xd6, 0x3e, 0xeb, 0x47, 0x1f, 0x41, 0xc9, 0x3b, 0x1f, 0x31, 0x0f, 0xa4, 0x95, 0x6a, 0xc3, 0x83,
|
0x48, 0x2b, 0xd5, 0x86, 0x07, 0xeb, 0xda, 0x3f, 0x1b, 0x61, 0x85, 0x82, 0xfb, 0xd5, 0x37, 0x9e,
|
||||||
0x75, 0x1d, 0x9c, 0x8f, 0xb0, 0x42, 0xc1, 0xfd, 0x2a, 0x1d, 0xcf, 0x51, 0x4f, 0xb9, 0x3b, 0x57,
|
0xa3, 0x9e, 0x70, 0x77, 0xae, 0xa4, 0x84, 0x7a, 0x88, 0xc6, 0xf0, 0x69, 0xb8, 0xc0, 0xdc, 0x1e,
|
||||||
0x52, 0x42, 0x3d, 0x44, 0x63, 0xf8, 0x34, 0x5c, 0x60, 0x6e, 0x0f, 0x6f, 0x32, 0xce, 0xf6, 0x85,
|
0xde, 0x64, 0x9c, 0xed, 0x0b, 0x6d, 0xdf, 0xf3, 0x0c, 0x9a, 0x8e, 0xa3, 0x9c, 0xed, 0xf7, 0xee,
|
||||||
0xb6, 0xef, 0x79, 0x06, 0x4d, 0xc7, 0x51, 0xce, 0xf6, 0x7b, 0x0f, 0x3c, 0x43, 0xfe, 0xd7, 0x02,
|
0x7b, 0x86, 0xfc, 0xaf, 0x05, 0x68, 0x07, 0x98, 0x15, 0xec, 0x8e, 0x8d, 0x6c, 0x81, 0x9b, 0x9c,
|
||||||
0xb4, 0x03, 0xcc, 0x0a, 0x76, 0xc7, 0x46, 0xb6, 0xc0, 0x4d, 0xce, 0x77, 0x4c, 0x93, 0xb5, 0x6f,
|
0xef, 0x98, 0x26, 0x6b, 0xdf, 0x81, 0x3a, 0x3f, 0xf6, 0x73, 0xb0, 0x0d, 0xb0, 0x21, 0x5b, 0x13,
|
||||||
0x41, 0x9d, 0x1f, 0xfb, 0x05, 0xd8, 0x06, 0xd8, 0x90, 0xed, 0x09, 0x7c, 0x5c, 0x7e, 0x41, 0x7c,
|
0xf8, 0xb8, 0xfc, 0x82, 0xf8, 0xb8, 0x32, 0x43, 0xc6, 0x20, 0x9d, 0xf8, 0xf2, 0xcf, 0x24, 0x78,
|
||||||
0x5c, 0x99, 0x21, 0x63, 0x90, 0x4e, 0x7c, 0xf9, 0xa7, 0x12, 0xbc, 0x96, 0x50, 0x8b, 0x13, 0x49,
|
0x2d, 0xa1, 0x16, 0x27, 0x92, 0x76, 0x72, 0xbc, 0xc6, 0xd5, 0x65, 0x7c, 0x4a, 0xae, 0xe0, 0x1f,
|
||||||
0x3b, 0x39, 0x5e, 0xe3, 0xea, 0x32, 0x3e, 0x25, 0x57, 0xf0, 0x8f, 0xa0, 0xe2, 0xd0, 0xd9, 0xf9,
|
0x42, 0xc5, 0xa1, 0xb3, 0xf3, 0xeb, 0x9d, 0x5b, 0x13, 0xb9, 0x8b, 0x2d, 0x44, 0xe1, 0x43, 0xe4,
|
||||||
0x35, 0xd0, 0xed, 0x89, 0xdc, 0xc5, 0x16, 0xa2, 0xf0, 0x21, 0xf2, 0x1f, 0x4a, 0xb0, 0x92, 0x5c,
|
0x3f, 0x94, 0x60, 0x25, 0xb9, 0xd4, 0x39, 0xac, 0xf6, 0x1a, 0x2c, 0xb0, 0xa9, 0x7d, 0x21, 0xbc,
|
||||||
0xea, 0x1c, 0x56, 0x7b, 0x0d, 0x16, 0xd8, 0xd4, 0xbe, 0x10, 0xde, 0x9d, 0x2c, 0x84, 0x01, 0x71,
|
0x33, 0x59, 0x08, 0x03, 0xe2, 0x28, 0xfe, 0x40, 0x79, 0x0f, 0x96, 0x7d, 0xe3, 0x1e, 0x90, 0x7e,
|
||||||
0x14, 0x7f, 0xa0, 0xbc, 0x0f, 0xcb, 0xbe, 0x71, 0x0f, 0x48, 0xbf, 0x83, 0x3d, 0x75, 0x42, 0xb4,
|
0x1b, 0x7b, 0xea, 0x84, 0x68, 0xe5, 0x06, 0xd4, 0x99, 0xdb, 0xcb, 0xa2, 0x00, 0x16, 0xe7, 0xc3,
|
||||||
0x72, 0x13, 0xea, 0xcc, 0xed, 0x65, 0x51, 0x00, 0x8b, 0xf3, 0xe1, 0x50, 0xa4, 0xc7, 0xe4, 0xff,
|
0x81, 0x48, 0x8f, 0xc9, 0xff, 0x25, 0xc1, 0x25, 0x6a, 0x1d, 0xe3, 0xf7, 0x29, 0x79, 0xee, 0xda,
|
||||||
0x92, 0xe0, 0x0a, 0xb5, 0x8e, 0xf1, 0x7b, 0x97, 0x3c, 0x77, 0x72, 0xb2, 0x48, 0x23, 0xec, 0xaa,
|
0x64, 0x91, 0x46, 0xd8, 0x51, 0x4d, 0x5e, 0xdb, 0x51, 0x53, 0x22, 0x7d, 0xa8, 0x97, 0xcc, 0x9e,
|
||||||
0x26, 0xaf, 0x01, 0xa9, 0x29, 0x91, 0x3e, 0xd4, 0x4b, 0x66, 0xcf, 0x52, 0xa3, 0xda, 0xe0, 0x12,
|
0xa5, 0x46, 0xb5, 0xc1, 0xe5, 0x2c, 0x89, 0xa0, 0xe9, 0xdd, 0x6c, 0x3c, 0x6d, 0x16, 0x58, 0xe5,
|
||||||
0x97, 0x44, 0xd0, 0xf4, 0x0e, 0x37, 0x9e, 0x36, 0x0b, 0xac, 0x72, 0x69, 0x16, 0xab, 0xbc, 0x0d,
|
0xd2, 0x2c, 0x56, 0x79, 0x0b, 0x5e, 0x8b, 0xed, 0x74, 0x8e, 0x13, 0x95, 0xff, 0x4a, 0x22, 0xc7,
|
||||||
0xaf, 0xc5, 0x76, 0x3a, 0xc7, 0x89, 0xca, 0x7f, 0x2d, 0x91, 0xe3, 0x88, 0xd4, 0xd2, 0xcc, 0xee,
|
0x11, 0xa9, 0x91, 0x99, 0xdd, 0xdb, 0xbc, 0x26, 0x2e, 0x72, 0xfa, 0xba, 0x16, 0x57, 0x22, 0x1a,
|
||||||
0x6d, 0x5e, 0x17, 0x17, 0x3e, 0x7d, 0x5d, 0x8b, 0x2b, 0x11, 0x0d, 0x7d, 0x0a, 0x35, 0x0b, 0x9f,
|
0xfa, 0x04, 0x6a, 0x16, 0x3e, 0xed, 0x87, 0x9d, 0x9d, 0x1c, 0xae, 0x78, 0xd5, 0xc2, 0xa7, 0xf4,
|
||||||
0xf5, 0xc3, 0xce, 0x4e, 0x0e, 0x57, 0xbc, 0x6a, 0xe1, 0x33, 0xfa, 0x4b, 0xde, 0x85, 0x95, 0xc4,
|
0x97, 0xbc, 0x03, 0x2b, 0x89, 0xa5, 0xce, 0xb3, 0xf7, 0x7f, 0x90, 0xe0, 0xf2, 0x86, 0x63, 0x8f,
|
||||||
0x52, 0xe7, 0xd9, 0xfb, 0x3f, 0x48, 0x70, 0x75, 0xc3, 0xb1, 0x47, 0x5f, 0xe8, 0x8e, 0x37, 0x56,
|
0x3e, 0xd7, 0x1d, 0x6f, 0xac, 0x1a, 0xd1, 0x6b, 0xef, 0x97, 0x93, 0x8e, 0xfa, 0x2c, 0xe4, 0xf6,
|
||||||
0x8d, 0xe8, 0xf5, 0xf8, 0xcb, 0x49, 0x47, 0x7d, 0x1e, 0x72, 0x7b, 0x19, 0xff, 0xdc, 0x4f, 0x91,
|
0x32, 0xfe, 0xb9, 0x97, 0x22, 0x41, 0xc9, 0x45, 0xf1, 0x4d, 0x87, 0x9c, 0xe4, 0xff, 0x2c, 0xa6,
|
||||||
0xa0, 0xe4, 0xa2, 0xf8, 0xa6, 0x43, 0x4e, 0xf2, 0x7f, 0x16, 0xd3, 0x16, 0xcf, 0xe1, 0xa6, 0x38,
|
0x2d, 0x9e, 0xc3, 0x4d, 0x71, 0x3c, 0xf2, 0x44, 0x05, 0xa9, 0xd9, 0xeb, 0xe2, 0xac, 0xd9, 0xeb,
|
||||||
0x1e, 0x79, 0xa2, 0x82, 0xd4, 0xec, 0x75, 0x71, 0xd6, 0xec, 0x75, 0x86, 0x7a, 0x2f, 0xbd, 0x20,
|
0x0c, 0xf5, 0x5e, 0x7a, 0x41, 0xea, 0xfd, 0xdc, 0xe9, 0x94, 0xcf, 0x20, 0x7a, 0xb3, 0x40, 0xcd,
|
||||||
0xf5, 0x7e, 0xe1, 0x74, 0xca, 0xe7, 0x10, 0xbd, 0x59, 0xa0, 0xe6, 0x77, 0xa6, 0x2b, 0x89, 0x35,
|
0xef, 0x4c, 0x57, 0x12, 0x6b, 0x00, 0x41, 0x96, 0x9d, 0x97, 0x38, 0xe6, 0x99, 0x26, 0x34, 0x8a,
|
||||||
0x80, 0x20, 0xcb, 0xce, 0x4b, 0x21, 0xf3, 0x4c, 0x13, 0x1a, 0x45, 0x4e, 0x4b, 0x98, 0x52, 0x6e,
|
0x9c, 0x96, 0x30, 0xa5, 0xdc, 0x94, 0x87, 0xf2, 0xbe, 0xdf, 0x83, 0x6e, 0x1a, 0x97, 0xce, 0xc3,
|
||||||
0xca, 0x43, 0x79, 0xdf, 0xef, 0x40, 0x37, 0x8d, 0x4b, 0xe7, 0xe1, 0xfc, 0x9f, 0x17, 0x00, 0x7a,
|
0xf9, 0xbf, 0x28, 0x00, 0xf4, 0x44, 0x55, 0xec, 0x6c, 0xb6, 0xe0, 0x16, 0x84, 0xdc, 0x8d, 0x40,
|
||||||
0xa2, 0x7a, 0x76, 0x36, 0x5b, 0x70, 0x1b, 0x42, 0xee, 0x46, 0x20, 0xef, 0x61, 0x2e, 0xd2, 0x88,
|
0xde, 0xc3, 0x5c, 0xa4, 0x11, 0x91, 0x10, 0x81, 0x24, 0x81, 0x49, 0x04, 0x97, 0x1a, 0x9d, 0x27,
|
||||||
0x48, 0x88, 0x40, 0x92, 0xc0, 0x24, 0x82, 0x4b, 0x8d, 0xce, 0x13, 0x92, 0x1a, 0xc6, 0x14, 0x71,
|
0x24, 0x35, 0x8c, 0x29, 0xe2, 0xea, 0xf7, 0x0a, 0xd4, 0x1c, 0xfb, 0xb4, 0x4f, 0xc4, 0x4c, 0xf3,
|
||||||
0xf5, 0xfb, 0x3a, 0xd4, 0x1c, 0xfb, 0xac, 0x4f, 0xc4, 0x4c, 0xf3, 0xcb, 0x83, 0x1d, 0xfb, 0x8c,
|
0xcb, 0x7e, 0x1d, 0xfb, 0x94, 0x08, 0x9f, 0x86, 0x56, 0x60, 0xc1, 0x53, 0xdd, 0x63, 0x32, 0x7f,
|
||||||
0x08, 0x9f, 0x86, 0x56, 0x60, 0xc1, 0x53, 0xdd, 0x13, 0x32, 0x7f, 0x25, 0x54, 0xa1, 0xa1, 0xa1,
|
0x25, 0x54, 0x79, 0xa1, 0xa1, 0x4b, 0x50, 0x3e, 0xd4, 0x0d, 0xcc, 0x2e, 0xfa, 0x6b, 0x0a, 0x6b,
|
||||||
0x2b, 0x50, 0x3e, 0xd2, 0x0d, 0xcc, 0x0a, 0x02, 0x6a, 0x0a, 0x6b, 0xa0, 0x6f, 0xf8, 0x75, 0x6c,
|
0xa0, 0x6f, 0xf9, 0xf5, 0x69, 0xd5, 0xdc, 0xd5, 0x35, 0xac, 0x44, 0xed, 0x2b, 0x09, 0x16, 0x03,
|
||||||
0xd5, 0xdc, 0x55, 0x38, 0xac, 0x94, 0xed, 0x2b, 0x09, 0x16, 0x03, 0xaa, 0x51, 0x05, 0x44, 0x74,
|
0xaa, 0x51, 0x05, 0x44, 0x74, 0x1a, 0xd5, 0x67, 0xeb, 0xb6, 0xc6, 0x54, 0x45, 0x2b, 0xc3, 0x22,
|
||||||
0x1a, 0xd5, 0x67, 0xeb, 0xb6, 0xc6, 0x54, 0x45, 0x2b, 0xc3, 0x22, 0xb0, 0x81, 0x4c, 0x6b, 0x05,
|
0xb0, 0x81, 0x4c, 0x6b, 0x05, 0x43, 0x26, 0xc5, 0xc1, 0x64, 0x5f, 0x64, 0xd3, 0xba, 0xe6, 0x57,
|
||||||
0x43, 0x26, 0xc5, 0xc1, 0x64, 0x5f, 0x64, 0xd3, 0xba, 0xe6, 0x57, 0xa5, 0x54, 0x1c, 0xfb, 0xac,
|
0x9b, 0x54, 0x1c, 0xfb, 0xb4, 0xa7, 0x09, 0x6a, 0xb0, 0x9a, 0x5e, 0x16, 0xf5, 0x11, 0x6a, 0xac,
|
||||||
0xa7, 0x09, 0x6a, 0xb0, 0xda, 0x5f, 0x16, 0xf5, 0x11, 0x6a, 0xac, 0xd3, 0xf2, 0xdf, 0xdb, 0xd0,
|
0xd3, 0xb2, 0xde, 0x5b, 0xd0, 0xc4, 0x8e, 0x63, 0x3b, 0x7d, 0x13, 0xbb, 0xae, 0x3a, 0xc4, 0xdc,
|
||||||
0xc4, 0x8e, 0x63, 0x3b, 0x7d, 0x13, 0xbb, 0xae, 0x3a, 0xc4, 0xdc, 0x01, 0x6f, 0xd0, 0xce, 0x1d,
|
0x01, 0x6f, 0xd0, 0xce, 0x6d, 0xd6, 0x27, 0xff, 0x49, 0x09, 0x5a, 0xc1, 0x56, 0xfc, 0xbb, 0x6d,
|
||||||
0xd6, 0x27, 0xff, 0x49, 0x09, 0x5a, 0xc1, 0x56, 0xfc, 0xbb, 0x6d, 0x5d, 0xf3, 0xef, 0xb6, 0x75,
|
0x5d, 0xf3, 0xef, 0xb6, 0x75, 0x72, 0x74, 0xe0, 0x30, 0x55, 0x28, 0x0e, 0x77, 0xad, 0xd0, 0x91,
|
||||||
0x72, 0x74, 0xe0, 0x30, 0x55, 0x28, 0x0e, 0x77, 0xad, 0xd0, 0x91, 0x94, 0x1a, 0xef, 0xed, 0x69,
|
0x94, 0x1a, 0xef, 0xed, 0x69, 0xc4, 0x2c, 0x13, 0x21, 0xb3, 0x6c, 0x0d, 0x07, 0x87, 0x0b, 0x7e,
|
||||||
0xc4, 0x2c, 0x13, 0x21, 0xb3, 0x6c, 0x0d, 0x07, 0x87, 0x0b, 0x7e, 0x17, 0x3f, 0xdb, 0x08, 0x8f,
|
0x17, 0x3f, 0xdb, 0x08, 0x8f, 0x94, 0x72, 0xf0, 0x48, 0x39, 0x07, 0x8f, 0x54, 0x52, 0x78, 0x64,
|
||||||
0x94, 0x72, 0xf0, 0x48, 0x39, 0x07, 0x8f, 0x54, 0x52, 0x78, 0x64, 0x19, 0x2a, 0x87, 0xe3, 0xc1,
|
0x19, 0x2a, 0x07, 0xe3, 0xc1, 0x31, 0xf6, 0xb8, 0xc7, 0xc6, 0x5b, 0x51, 0xde, 0xa9, 0xc6, 0x78,
|
||||||
0x09, 0xf6, 0xb8, 0xc7, 0xc6, 0x5b, 0x51, 0xde, 0xa9, 0xc6, 0x78, 0x47, 0xb0, 0x48, 0x2d, 0xcc,
|
0x47, 0xb0, 0x48, 0x2d, 0xcc, 0x22, 0x57, 0xa0, 0xc6, 0x2e, 0x59, 0xfb, 0x9e, 0x4b, 0x6f, 0x8c,
|
||||||
0x22, 0xaf, 0x43, 0x8d, 0x5d, 0xb2, 0xf6, 0x3d, 0x97, 0xde, 0x18, 0x15, 0x95, 0x2a, 0xeb, 0x38,
|
0x8a, 0x4a, 0x95, 0x75, 0xec, 0xbb, 0xe8, 0x23, 0xdf, 0x9d, 0xab, 0xa7, 0x09, 0x3b, 0xd5, 0x3a,
|
||||||
0x70, 0xd1, 0xc7, 0xbe, 0x3b, 0x57, 0x4f, 0x13, 0x76, 0xaa, 0x75, 0x62, 0x5c, 0xe2, 0x3b, 0x73,
|
0x31, 0x2e, 0xf1, 0x9d, 0xb9, 0xb7, 0x60, 0x31, 0x44, 0x0e, 0x6a, 0x23, 0x1a, 0x74, 0xa9, 0x21,
|
||||||
0x6f, 0xc3, 0x62, 0x88, 0x1c, 0xd4, 0x46, 0x34, 0xe8, 0x52, 0x43, 0xee, 0x3c, 0x35, 0x13, 0x77,
|
0x77, 0x9e, 0x9a, 0x89, 0xdb, 0xd0, 0x0a, 0x48, 0x42, 0xe1, 0x9a, 0x2c, 0x8a, 0x12, 0xbd, 0x14,
|
||||||
0xa0, 0x15, 0x90, 0x84, 0xc2, 0x35, 0x59, 0x14, 0x25, 0x7a, 0x29, 0x98, 0xe0, 0xe4, 0xd6, 0xc5,
|
0x4c, 0x70, 0x72, 0xeb, 0x7c, 0x9c, 0x8c, 0x2e, 0x43, 0x95, 0x87, 0x3f, 0x6e, 0x67, 0x31, 0x92,
|
||||||
0x38, 0x19, 0x5d, 0x85, 0x2a, 0x0f, 0x7f, 0xdc, 0xce, 0x62, 0x24, 0x1b, 0x21, 0x7f, 0x1f, 0x50,
|
0x8d, 0x90, 0x7f, 0x08, 0x28, 0x58, 0xfd, 0x7c, 0xde, 0x62, 0x8c, 0x3d, 0x0a, 0x71, 0xf6, 0x90,
|
||||||
0xb0, 0xfa, 0xf9, 0xbc, 0xc5, 0x18, 0x7b, 0x14, 0xe2, 0xec, 0x21, 0xff, 0x4c, 0x82, 0xa5, 0x30,
|
0x7f, 0x2e, 0xc1, 0x52, 0x18, 0xd9, 0xac, 0x86, 0xf7, 0x13, 0xa8, 0xb3, 0x3b, 0xbb, 0x3e, 0x11,
|
||||||
0xb2, 0x59, 0x0d, 0xef, 0xa7, 0x50, 0x67, 0x77, 0x76, 0x7d, 0x22, 0xf8, 0x3c, 0xcb, 0x73, 0x7d,
|
0x7c, 0x9e, 0xe5, 0xb9, 0x36, 0xf1, 0x5c, 0x14, 0x08, 0x5e, 0x05, 0x10, 0xf6, 0x3a, 0xb5, 0x9d,
|
||||||
0xe2, 0xb9, 0x28, 0x10, 0xbc, 0x1e, 0x20, 0xec, 0x75, 0x66, 0x3b, 0x27, 0xba, 0x35, 0xec, 0x93,
|
0x63, 0xdd, 0x1a, 0xf6, 0xc9, 0xca, 0x7c, 0x71, 0x6b, 0xf0, 0xce, 0x1d, 0xd2, 0x27, 0x7f, 0x29,
|
||||||
0x95, 0xf9, 0xe2, 0xd6, 0xe0, 0x9d, 0xbb, 0xa4, 0x4f, 0xfe, 0xb1, 0x04, 0x37, 0x9e, 0x8c, 0x34,
|
0xc1, 0xf5, 0x27, 0x23, 0x4d, 0xf5, 0x70, 0xc8, 0x03, 0x99, 0xb7, 0xd0, 0xf0, 0x43, 0xbf, 0xd2,
|
||||||
0xd5, 0xc3, 0x21, 0x0f, 0x64, 0xde, 0x82, 0xc4, 0x8f, 0xfc, 0x8a, 0xc0, 0x42, 0xbe, 0x7b, 0x27,
|
0xaf, 0x90, 0xef, 0xde, 0x89, 0x41, 0xcb, 0x7f, 0x2b, 0xd6, 0xc2, 0xcd, 0x01, 0xbd, 0xa4, 0x1c,
|
||||||
0x06, 0x2d, 0xef, 0xc0, 0x55, 0x05, 0xbb, 0xd8, 0xd2, 0x22, 0x1f, 0x67, 0x4e, 0xce, 0x8c, 0xa0,
|
0xd1, 0x4b, 0xdf, 0x99, 0xd7, 0xd2, 0x85, 0xea, 0x09, 0x9f, 0xce, 0x7f, 0xe5, 0xe0, 0xb7, 0x23,
|
||||||
0x9b, 0x36, 0xdd, 0x3c, 0x67, 0xcf, 0x5c, 0xc1, 0xbe, 0x43, 0xa6, 0xf5, 0xb8, 0x66, 0x23, 0x1e,
|
0x77, 0x9b, 0xc5, 0xf3, 0xdf, 0x6d, 0xca, 0xdb, 0x70, 0x59, 0xc1, 0x2e, 0xb6, 0xb4, 0xc8, 0x6e,
|
||||||
0x08, 0xc5, 0xe3, 0xc9, 0x7f, 0x53, 0x80, 0x95, 0xc7, 0x9a, 0xc6, 0x95, 0x22, 0x77, 0x6e, 0x5e,
|
0x66, 0xce, 0x26, 0x8d, 0xa0, 0x9b, 0x36, 0xdd, 0x3c, 0xcc, 0xca, 0x7c, 0xd7, 0xbe, 0x43, 0xa6,
|
||||||
0x96, 0xdf, 0x19, 0xf7, 0xcb, 0x8a, 0x49, 0xbf, 0xec, 0x45, 0x29, 0x2a, 0xae, 0xb2, 0xad, 0xb1,
|
0xf5, 0xb8, 0x2a, 0x26, 0x2e, 0x13, 0xc5, 0xe3, 0xc9, 0x7f, 0x5d, 0x80, 0x95, 0x47, 0x9a, 0xc6,
|
||||||
0xe9, 0x9b, 0x22, 0x87, 0xd5, 0xd0, 0x3c, 0xe2, 0x77, 0x47, 0x24, 0x3e, 0xa6, 0xe6, 0x68, 0xba,
|
0xb5, 0x38, 0xf7, 0xc6, 0x5e, 0x96, 0xa3, 0x1c, 0x77, 0x24, 0x8b, 0x49, 0x47, 0xf2, 0x45, 0x69,
|
||||||
0xbb, 0x52, 0xf5, 0x93, 0x4c, 0xf2, 0x08, 0x3a, 0x49, 0x62, 0xcd, 0x29, 0x99, 0x3e, 0x45, 0x46,
|
0x56, 0x6e, 0x63, 0xac, 0xb1, 0xe9, 0xdb, 0x4e, 0x87, 0x15, 0xfd, 0x3c, 0xe4, 0x97, 0x5d, 0x24,
|
||||||
0x36, 0x4b, 0x48, 0x36, 0x88, 0x47, 0x42, 0xbb, 0xf6, 0x6c, 0x57, 0xfe, 0xef, 0x02, 0x74, 0xf6,
|
0xa0, 0xa7, 0xf6, 0x73, 0xba, 0x7f, 0x55, 0xf5, 0xb3, 0x62, 0xf2, 0x08, 0x3a, 0x49, 0x62, 0xcd,
|
||||||
0xd5, 0x53, 0xfc, 0xcb, 0x73, 0x40, 0xdf, 0x85, 0x2b, 0xae, 0x7a, 0x8a, 0xfb, 0xa1, 0x38, 0xb3,
|
0xa9, 0x4a, 0x7c, 0x8a, 0x8c, 0x6c, 0x96, 0x41, 0x6d, 0x10, 0x17, 0x8a, 0x76, 0xed, 0xda, 0xae,
|
||||||
0xef, 0xe0, 0x67, 0xdc, 0xa3, 0x7b, 0x27, 0x4d, 0x30, 0x53, 0x4b, 0x4d, 0x94, 0x25, 0x37, 0xd2,
|
0xfc, 0xdf, 0x05, 0xe8, 0xec, 0xa9, 0x27, 0xf8, 0x97, 0xe7, 0x80, 0xbe, 0x0f, 0x97, 0x5c, 0xf5,
|
||||||
0xaf, 0xe0, 0x67, 0xe8, 0x2d, 0x58, 0x0c, 0xd7, 0x32, 0x91, 0xa5, 0x55, 0x29, 0xc9, 0x9b, 0xa1,
|
0x04, 0xf7, 0x43, 0x81, 0x71, 0xdf, 0xc1, 0xcf, 0xb8, 0x0b, 0xfa, 0x76, 0x9a, 0x26, 0x49, 0xad,
|
||||||
0x52, 0xa5, 0x9e, 0x26, 0x3f, 0x83, 0x6b, 0x4f, 0x2c, 0x17, 0x7b, 0xbd, 0xa0, 0xdc, 0x66, 0xce,
|
0x8d, 0x51, 0x96, 0xdc, 0x48, 0xbf, 0x82, 0x9f, 0xa1, 0x37, 0x61, 0x31, 0x5c, 0x7c, 0x45, 0x96,
|
||||||
0x88, 0xec, 0x26, 0xd4, 0x03, 0xc2, 0x27, 0x1e, 0x14, 0x68, 0xae, 0x6c, 0x43, 0x77, 0x27, 0xa8,
|
0x56, 0xa5, 0x24, 0x6f, 0x86, 0x6a, 0xab, 0x7a, 0x9a, 0xfc, 0x0c, 0xae, 0x3e, 0xb1, 0x5c, 0xec,
|
||||||
0xb6, 0x73, 0x37, 0x58, 0x59, 0xc4, 0x4b, 0x44, 0x78, 0x24, 0xaa, 0x84, 0x14, 0x7c, 0x84, 0x1d,
|
0xf5, 0x82, 0xfa, 0xa0, 0x39, 0x43, 0xc8, 0x1b, 0x50, 0x0f, 0x08, 0x9f, 0x78, 0xd9, 0xa0, 0xb9,
|
||||||
0x6c, 0x0d, 0xf0, 0xb6, 0x3d, 0x38, 0x09, 0x55, 0xd9, 0x4a, 0xe1, 0x2a, 0xdb, 0x59, 0xab, 0x76,
|
0xb2, 0x0d, 0xdd, 0x6d, 0xd5, 0x39, 0xf6, 0xf3, 0xc8, 0x1b, 0xac, 0x8e, 0xe3, 0x25, 0x22, 0x3c,
|
||||||
0xef, 0x7d, 0x2a, 0x2a, 0xf7, 0x0e, 0xce, 0x47, 0x18, 0x2d, 0x40, 0x71, 0x17, 0x9f, 0xb5, 0x2f,
|
0x14, 0x65, 0x4d, 0x0a, 0x3e, 0xc4, 0x0e, 0xb6, 0x06, 0x78, 0xcb, 0x1e, 0x1c, 0x87, 0xca, 0x7d,
|
||||||
0x21, 0x80, 0xca, 0xae, 0xed, 0x98, 0xaa, 0xd1, 0x96, 0x50, 0x1d, 0x16, 0xf8, 0xa5, 0x46, 0xbb,
|
0xa5, 0x70, 0xb9, 0xef, 0xac, 0xe5, 0xc3, 0x77, 0x3f, 0x11, 0xa5, 0x86, 0xfb, 0x67, 0x23, 0x8c,
|
||||||
0x80, 0x9a, 0x50, 0x5b, 0xf7, 0x13, 0xc3, 0xed, 0xe2, 0xbd, 0x3f, 0x93, 0x60, 0x29, 0x91, 0x76,
|
0x16, 0xa0, 0xb8, 0x83, 0x4f, 0xdb, 0x17, 0x10, 0x40, 0x65, 0xc7, 0x76, 0x4c, 0xd5, 0x68, 0x4b,
|
||||||
0x47, 0x2d, 0x80, 0x27, 0xd6, 0x80, 0xdf, 0x47, 0xb4, 0x2f, 0xa1, 0x06, 0x54, 0xfd, 0xdb, 0x09,
|
0xa8, 0x0e, 0x0b, 0xfc, 0x16, 0xa6, 0x5d, 0x40, 0x4d, 0xa8, 0xad, 0xfb, 0x99, 0xec, 0x76, 0xf1,
|
||||||
0x36, 0xdf, 0x81, 0x4d, 0xa1, 0xdb, 0x05, 0xd4, 0x86, 0x06, 0x1b, 0x38, 0x1e, 0x0c, 0xb0, 0xeb,
|
0xee, 0x9f, 0x49, 0xb0, 0x94, 0xb8, 0x27, 0x40, 0x2d, 0x80, 0x27, 0xd6, 0x80, 0x5f, 0xa0, 0xb4,
|
||||||
0xb6, 0x8b, 0xa2, 0x67, 0x53, 0xd5, 0x8d, 0xb1, 0x83, 0xdb, 0x25, 0x82, 0xf3, 0xc0, 0xe6, 0x35,
|
0x2f, 0xa0, 0x06, 0x54, 0xfd, 0xeb, 0x14, 0x36, 0xdf, 0xbe, 0x4d, 0xa1, 0xdb, 0x05, 0xd4, 0x86,
|
||||||
0xce, 0xed, 0x32, 0x42, 0xd0, 0xf2, 0x0b, 0x9e, 0xf9, 0xa0, 0x4a, 0xa8, 0xcf, 0x1f, 0xb6, 0x70,
|
0x06, 0x1b, 0x38, 0x1e, 0x0c, 0xb0, 0xeb, 0xb6, 0x8b, 0xa2, 0x67, 0x53, 0xd5, 0x8d, 0xb1, 0x83,
|
||||||
0xef, 0x69, 0x38, 0x79, 0x4a, 0xb7, 0xb7, 0x02, 0x97, 0x9f, 0x58, 0x1a, 0x3e, 0xd2, 0x2d, 0xac,
|
0xdb, 0x25, 0x82, 0x73, 0xdf, 0xe6, 0xc5, 0xd6, 0xed, 0x32, 0x42, 0xd0, 0xf2, 0x2b, 0xaf, 0xf9,
|
||||||
0x05, 0x9f, 0xda, 0x97, 0xd0, 0x65, 0x58, 0xdc, 0xc1, 0xce, 0x10, 0x87, 0x3a, 0x0b, 0x68, 0x09,
|
0xa0, 0x4a, 0xa8, 0xcf, 0x1f, 0xb6, 0x70, 0xf7, 0x69, 0x38, 0xdb, 0x4b, 0xb7, 0xb7, 0x02, 0x17,
|
||||||
0x9a, 0x3b, 0xfa, 0xf3, 0x50, 0x57, 0x51, 0x2e, 0x55, 0xa5, 0xb6, 0xb4, 0xfa, 0xe7, 0xd7, 0xa0,
|
0x9f, 0x58, 0x1a, 0x3e, 0xd4, 0x2d, 0xac, 0x05, 0x9f, 0xda, 0x17, 0xd0, 0x45, 0x58, 0xdc, 0xc6,
|
||||||
0xb6, 0xa1, 0x7a, 0xea, 0xba, 0x6d, 0x3b, 0x1a, 0x32, 0x00, 0xd1, 0x27, 0x01, 0xe6, 0xc8, 0xb6,
|
0xce, 0x10, 0x87, 0x3a, 0x0b, 0x68, 0x09, 0x9a, 0xdb, 0xfa, 0xf3, 0x50, 0x57, 0x51, 0x2e, 0x55,
|
||||||
0xc4, 0x43, 0x1b, 0xf4, 0x20, 0xca, 0x05, 0xbc, 0x91, 0x04, 0xe4, 0x3c, 0xd4, 0x7d, 0x33, 0x15,
|
0xa5, 0xb6, 0xb4, 0xfa, 0xe5, 0x35, 0xa8, 0x6d, 0xa8, 0x9e, 0xba, 0x6e, 0xdb, 0x8e, 0x86, 0x0c,
|
||||||
0x3e, 0x06, 0x2c, 0x5f, 0x42, 0x26, 0xc5, 0x76, 0xa0, 0x9b, 0xf8, 0x40, 0x1f, 0x9c, 0xf8, 0x45,
|
0x40, 0xf4, 0x6d, 0x82, 0x39, 0xb2, 0x2d, 0xf1, 0xe2, 0x07, 0xdd, 0x8f, 0x72, 0x01, 0x6f, 0x24,
|
||||||
0x8f, 0xef, 0x67, 0x84, 0x52, 0x49, 0x50, 0x1f, 0xdf, 0xed, 0x54, 0x7c, 0xec, 0xcd, 0x86, 0xaf,
|
0x01, 0x39, 0x0f, 0x75, 0xdf, 0x48, 0x85, 0x8f, 0x01, 0xcb, 0x17, 0x90, 0x49, 0xb1, 0xed, 0xeb,
|
||||||
0x35, 0xe5, 0x4b, 0xe8, 0x19, 0x5c, 0xd9, 0xc2, 0x21, 0x1b, 0xee, 0x23, 0x5c, 0xcd, 0x46, 0x98,
|
0x26, 0xde, 0xd7, 0x07, 0xc7, 0xbe, 0xa5, 0x7c, 0x2f, 0xc3, 0x2e, 0x26, 0x41, 0x7d, 0x7c, 0xb7,
|
||||||
0x00, 0xbe, 0x20, 0xca, 0x6d, 0x28, 0x53, 0x76, 0x43, 0x69, 0x66, 0x3e, 0xfc, 0x26, 0xb6, 0x7b,
|
0x52, 0xf1, 0xb1, 0xc7, 0x23, 0xbe, 0xd6, 0x94, 0x2f, 0xa0, 0x67, 0x70, 0xe9, 0x31, 0x0e, 0x39,
|
||||||
0x2b, 0x1b, 0x40, 0xcc, 0xf6, 0x7d, 0x58, 0x8c, 0xbd, 0xa4, 0x43, 0x69, 0x5a, 0x2a, 0xfd, 0x4d,
|
0x1d, 0x3e, 0xc2, 0xd5, 0x6c, 0x84, 0x09, 0xe0, 0x73, 0xa2, 0xdc, 0x82, 0x32, 0x65, 0x37, 0x94,
|
||||||
0x64, 0xf7, 0x5e, 0x1e, 0x50, 0x81, 0x6b, 0x08, 0xad, 0xe8, 0x53, 0x02, 0x94, 0x96, 0xf8, 0x4b,
|
0xe6, 0x97, 0x84, 0x1f, 0xe7, 0x76, 0x6f, 0x66, 0x03, 0x88, 0xd9, 0x7e, 0x08, 0x8b, 0xb1, 0x27,
|
||||||
0x7d, 0x04, 0xd5, 0x7d, 0x27, 0x07, 0xa4, 0x40, 0x64, 0x42, 0x3b, 0xfe, 0xb2, 0x0b, 0xdd, 0x9b,
|
0x7d, 0x28, 0x4d, 0x4b, 0xa5, 0x3f, 0xce, 0xec, 0xde, 0xcd, 0x03, 0x2a, 0x70, 0x0d, 0xa1, 0x15,
|
||||||
0x38, 0x41, 0x94, 0xd9, 0xde, 0xcd, 0x05, 0x2b, 0xd0, 0x9d, 0x53, 0x26, 0x48, 0x3c, 0x16, 0x8a,
|
0x7d, 0xd3, 0x80, 0xd2, 0x32, 0x95, 0xa9, 0xaf, 0xb1, 0xba, 0x6f, 0xe7, 0x80, 0x14, 0x88, 0x4c,
|
||||||
0xf3, 0xb8, 0x3f, 0x4d, 0xd6, 0x2b, 0xa6, 0xee, 0xc3, 0xdc, 0xf0, 0x02, 0xf5, 0x6f, 0xb3, 0x4a,
|
0x68, 0xc7, 0x9f, 0x98, 0xa1, 0xbb, 0x13, 0x27, 0x88, 0x32, 0xdb, 0x3b, 0xb9, 0x60, 0x05, 0xba,
|
||||||
0x84, 0xb4, 0x07, 0x37, 0xe8, 0x83, 0xf4, 0xe9, 0x26, 0xbc, 0x14, 0xea, 0xae, 0x5e, 0x64, 0x88,
|
0x33, 0xca, 0x04, 0x89, 0x57, 0x4b, 0x71, 0x1e, 0xf7, 0xa7, 0xc9, 0x7a, 0x4e, 0xd5, 0x7d, 0x90,
|
||||||
0x58, 0xc4, 0x0f, 0x69, 0x09, 0x41, 0xca, 0x93, 0x95, 0xb8, 0xdc, 0xf9, 0xf3, 0x65, 0xbf, 0xc6,
|
0x1b, 0x5e, 0xa0, 0xfe, 0x6d, 0x56, 0x3a, 0x91, 0xf6, 0xf2, 0x07, 0xbd, 0x9f, 0x3e, 0xdd, 0x84,
|
||||||
0xe9, 0x7e, 0x70, 0x81, 0x11, 0x62, 0x01, 0x76, 0xfc, 0xe9, 0x9c, 0x2f, 0x86, 0x0f, 0xa7, 0x72,
|
0x27, 0x4b, 0xdd, 0xd5, 0xf3, 0x0c, 0x11, 0x8b, 0xf8, 0x31, 0xad, 0x79, 0x48, 0x79, 0x3b, 0x13,
|
||||||
0xcd, 0x6c, 0x32, 0xf8, 0x3d, 0x58, 0x8c, 0xd9, 0x6d, 0x94, 0xdf, 0xb6, 0x77, 0x27, 0x39, 0x57,
|
0x97, 0x3b, 0x7f, 0xbe, 0xec, 0x67, 0x41, 0xdd, 0xf7, 0xcf, 0x31, 0x42, 0x2c, 0xc0, 0x8e, 0xbf,
|
||||||
0x4c, 0x24, 0x63, 0x15, 0x19, 0x28, 0x83, 0xfb, 0x53, 0xaa, 0x36, 0xba, 0xf7, 0xf2, 0x80, 0x8a,
|
0xe1, 0xf3, 0xc5, 0xf0, 0xc1, 0x54, 0xae, 0x99, 0x4d, 0x06, 0x7f, 0x00, 0x8b, 0x31, 0xbb, 0x8d,
|
||||||
0x8d, 0xb8, 0x54, 0x5d, 0xc6, 0xee, 0xe4, 0xd1, 0xfd, 0xf4, 0x39, 0xd2, 0xeb, 0x09, 0xba, 0xef,
|
0xf2, 0xdb, 0xf6, 0xee, 0x24, 0xe7, 0x8a, 0x89, 0x64, 0xac, 0x84, 0x04, 0x65, 0x70, 0x7f, 0x4a,
|
||||||
0xe5, 0x84, 0x16, 0x48, 0x4f, 0xe1, 0x72, 0x4a, 0x39, 0x04, 0x7a, 0x6f, 0xe2, 0x61, 0xc5, 0xeb,
|
0x99, 0x49, 0xf7, 0x6e, 0x1e, 0x50, 0xb1, 0x11, 0x97, 0xaa, 0xcb, 0x58, 0x11, 0x01, 0xba, 0x97,
|
||||||
0x40, 0xba, 0x0f, 0xf2, 0x82, 0x0b, 0xbc, 0xbf, 0x05, 0x68, 0xff, 0xd8, 0x3e, 0x5b, 0xb7, 0xad,
|
0x3e, 0x47, 0x7a, 0x01, 0x44, 0xf7, 0xdd, 0x9c, 0xd0, 0x02, 0xe9, 0x09, 0x5c, 0x4c, 0xa9, 0xdf,
|
||||||
0x23, 0x7d, 0x38, 0x76, 0x54, 0x96, 0x48, 0xcf, 0xb2, 0x0d, 0x49, 0xd0, 0x0c, 0x1e, 0x9d, 0x38,
|
0x40, 0xef, 0x4e, 0x3c, 0xac, 0x78, 0xe1, 0x4a, 0xf7, 0x7e, 0x5e, 0x70, 0x81, 0xf7, 0xb7, 0x00,
|
||||||
0x42, 0x20, 0xef, 0x03, 0x6c, 0x61, 0x6f, 0x07, 0x7b, 0x0e, 0x11, 0x8c, 0xb7, 0xb2, 0xcc, 0x1f,
|
0xed, 0x1d, 0xd9, 0xa7, 0xeb, 0xb6, 0x75, 0xa8, 0x0f, 0xc7, 0x8e, 0xca, 0x32, 0xff, 0x59, 0xb6,
|
||||||
0x07, 0xf0, 0x51, 0xbd, 0x3d, 0x15, 0x2e, 0x64, 0x8a, 0xda, 0x3b, 0xaa, 0x35, 0x56, 0x8d, 0x50,
|
0x21, 0x09, 0x9a, 0xc1, 0xa3, 0x13, 0x47, 0x08, 0xe4, 0x7d, 0x80, 0xc7, 0xd8, 0xdb, 0xc6, 0x9e,
|
||||||
0x3d, 0xf7, 0xfd, 0xd4, 0xe1, 0x71, 0xb0, 0x8c, 0x83, 0xcc, 0x84, 0x16, 0x28, 0xcf, 0x84, 0x69,
|
0x43, 0x04, 0xe3, 0xcd, 0x2c, 0xf3, 0xc7, 0x01, 0x7c, 0x54, 0x6f, 0x4d, 0x85, 0x0b, 0x99, 0xa2,
|
||||||
0x0f, 0xdd, 0xd4, 0x4c, 0x36, 0xed, 0xc9, 0x32, 0x80, 0xb8, 0xda, 0x9b, 0x00, 0x2f, 0x10, 0x7f,
|
0xf6, 0xb6, 0x6a, 0x8d, 0x55, 0x23, 0x54, 0x80, 0x7e, 0x2f, 0x75, 0x78, 0x1c, 0x2c, 0xe3, 0x20,
|
||||||
0x29, 0xd1, 0x8a, 0x9a, 0x18, 0xc0, 0x53, 0xdd, 0x3b, 0xde, 0x33, 0x54, 0xcb, 0xcd, 0xb3, 0x04,
|
0x33, 0xa1, 0x05, 0xca, 0x53, 0x61, 0xda, 0x43, 0x57, 0x4b, 0x93, 0x4d, 0x7b, 0xb2, 0x6e, 0x21,
|
||||||
0x0a, 0x78, 0x81, 0x25, 0x70, 0x78, 0xb1, 0x04, 0x0d, 0x9a, 0x91, 0x0b, 0x14, 0x94, 0x56, 0x00,
|
0xae, 0xf6, 0x26, 0xc0, 0x0b, 0xc4, 0x5f, 0x48, 0xb4, 0x04, 0x28, 0x06, 0xf0, 0x54, 0xf7, 0x8e,
|
||||||
0x9d, 0x76, 0x99, 0xd4, 0xbd, 0x3b, 0x1d, 0x50, 0x60, 0x39, 0x86, 0xa6, 0x2f, 0x4a, 0x8c, 0xb8,
|
0x76, 0x0d, 0xd5, 0x72, 0xf3, 0x2c, 0x81, 0x02, 0x9e, 0x63, 0x09, 0x1c, 0x5e, 0x2c, 0x41, 0x83,
|
||||||
0xef, 0x64, 0xad, 0x34, 0x80, 0xc9, 0xd0, 0x04, 0xe9, 0xa0, 0x61, 0x4d, 0x90, 0xcc, 0x0f, 0xa3,
|
0x66, 0xe4, 0xc6, 0x07, 0xa5, 0x55, 0x6c, 0xa7, 0xdd, 0x7e, 0x75, 0xef, 0x4c, 0x07, 0x14, 0x58,
|
||||||
0x7c, 0xf7, 0x0a, 0x93, 0x34, 0x41, 0x76, 0xd2, 0x99, 0xa9, 0xba, 0xd8, 0x5d, 0x4c, 0xba, 0x1e,
|
0x8e, 0xa0, 0xe9, 0x8b, 0x12, 0x23, 0xee, 0xdb, 0x59, 0x2b, 0x0d, 0x60, 0x32, 0x34, 0x41, 0x3a,
|
||||||
0x4d, 0xbd, 0x5a, 0x4a, 0x55, 0x75, 0x19, 0x57, 0x3b, 0xf2, 0x25, 0xf4, 0x14, 0x2a, 0xfc, 0x8f,
|
0x68, 0x58, 0x13, 0x24, 0x13, 0xda, 0x28, 0xdf, 0x45, 0xc8, 0x24, 0x4d, 0x90, 0x9d, 0x25, 0x67,
|
||||||
0x20, 0xde, 0x9c, 0x9c, 0xd3, 0xe1, 0xb3, 0xdf, 0x99, 0x02, 0x25, 0x26, 0x3e, 0x81, 0x95, 0x8c,
|
0xaa, 0x2e, 0x76, 0x79, 0x94, 0xae, 0x47, 0x53, 0xef, 0xc2, 0x52, 0x55, 0x5d, 0xc6, 0x5d, 0x94,
|
||||||
0x8c, 0x4e, 0xaa, 0x09, 0x9e, 0x9c, 0xfd, 0x99, 0x66, 0x1c, 0x54, 0x40, 0xc9, 0xd7, 0x96, 0xa9,
|
0x7c, 0x01, 0x3d, 0x85, 0x0a, 0xff, 0x47, 0x8a, 0x37, 0x26, 0x27, 0xa1, 0xf8, 0xec, 0xb7, 0xa7,
|
||||||
0xc7, 0x94, 0xf9, 0x28, 0x33, 0x07, 0x8a, 0xe4, 0x83, 0xc9, 0x54, 0x14, 0x99, 0xef, 0x2a, 0xa7,
|
0x40, 0x89, 0x89, 0x8f, 0x61, 0x25, 0x23, 0x05, 0x95, 0x6a, 0x82, 0x27, 0xa7, 0xab, 0xa6, 0x19,
|
||||||
0xa1, 0xe8, 0xc3, 0x52, 0x22, 0x2f, 0x80, 0xde, 0xcd, 0xb0, 0xa0, 0x69, 0xd9, 0x83, 0x69, 0x08,
|
0x07, 0x81, 0x2c, 0x91, 0x63, 0x9a, 0x80, 0x2c, 0x2b, 0x1f, 0x35, 0x0d, 0x99, 0x0a, 0x28, 0xf9,
|
||||||
0x86, 0xf0, 0x5a, 0x6a, 0x0c, 0x9c, 0xea, 0x11, 0x4c, 0x8a, 0x96, 0xa7, 0x21, 0x1a, 0xc0, 0xe5,
|
0xc6, 0x34, 0x95, 0x27, 0x32, 0x9f, 0xa2, 0xe6, 0x40, 0x91, 0x7c, 0x26, 0x9a, 0x8a, 0x22, 0xf3,
|
||||||
0x94, 0xc8, 0x37, 0xd5, 0x96, 0x65, 0x47, 0xc8, 0xd3, 0x90, 0x1c, 0x43, 0x77, 0xcd, 0xb1, 0x55,
|
0x35, 0xe9, 0x34, 0x14, 0x7d, 0x58, 0x4a, 0x24, 0x21, 0xd0, 0x3b, 0x19, 0xe6, 0x3a, 0x2d, 0x55,
|
||||||
0x6d, 0xa0, 0xba, 0xde, 0x63, 0xc3, 0xc3, 0x0e, 0x09, 0xcf, 0x7c, 0x97, 0x2c, 0x4e, 0x37, 0xde,
|
0x31, 0x0d, 0xc1, 0x10, 0x5e, 0x4b, 0x0d, 0xb8, 0x53, 0xdd, 0x8f, 0x49, 0xa1, 0xf9, 0x34, 0x44,
|
||||||
0xa0, 0x70, 0x01, 0x54, 0x4e, 0x4c, 0x87, 0x50, 0xa7, 0x2f, 0x15, 0xd8, 0xeb, 0x7f, 0x94, 0x6e,
|
0x03, 0xb8, 0x98, 0x12, 0x66, 0xa7, 0x1a, 0xce, 0xec, 0x70, 0x7c, 0x1a, 0x92, 0x23, 0xe8, 0xae,
|
||||||
0x7e, 0x42, 0x10, 0x19, 0x3a, 0x2d, 0x0d, 0xd0, 0x97, 0x97, 0xd5, 0xaf, 0x6a, 0x50, 0xf5, 0x0b,
|
0x39, 0xb6, 0xaa, 0x0d, 0x54, 0xd7, 0x7b, 0x64, 0x78, 0xd8, 0x21, 0xb1, 0xa0, 0xef, 0xff, 0xc5,
|
||||||
0xc3, 0xbf, 0xe6, 0xe8, 0xf0, 0x15, 0x84, 0x6b, 0xdf, 0x83, 0xc5, 0xd8, 0x63, 0xce, 0x54, 0x15,
|
0xe9, 0xc6, 0x1b, 0x14, 0x2e, 0x80, 0xca, 0x89, 0xe9, 0x00, 0xea, 0x94, 0x25, 0xd9, 0x7f, 0x1e,
|
||||||
0x97, 0xfe, 0xe0, 0x73, 0xda, 0x71, 0x3d, 0xe5, 0x7f, 0x35, 0x24, 0x3c, 0xb7, 0xb7, 0xb3, 0x42,
|
0xa0, 0x74, 0x5b, 0x17, 0x82, 0xc8, 0x50, 0xa0, 0x69, 0x80, 0xbe, 0x70, 0xae, 0x7e, 0x55, 0x83,
|
||||||
0xbe, 0xb8, 0xd3, 0x36, 0x65, 0xe2, 0xff, 0xdf, 0xae, 0xd2, 0x2e, 0x40, 0xc8, 0x49, 0x9a, 0x5c,
|
0xaa, 0x5f, 0x36, 0xff, 0x0d, 0x87, 0xa2, 0xaf, 0x20, 0x36, 0xfc, 0x01, 0x2c, 0xc6, 0x9e, 0xb0,
|
||||||
0x6a, 0x45, 0xec, 0xfe, 0x34, 0x6a, 0x99, 0xa9, 0x7e, 0xd0, 0x3b, 0x79, 0xaa, 0x5a, 0xb2, 0x2d,
|
0xa6, 0xea, 0xd3, 0xf4, 0x67, 0xae, 0xd3, 0x8e, 0xeb, 0x29, 0xff, 0x83, 0x25, 0xe1, 0x26, 0xbe,
|
||||||
0x59, 0xb6, 0xf7, 0xf3, 0x04, 0x1a, 0xe1, 0x22, 0x48, 0x94, 0xfa, 0xc7, 0x36, 0xc9, 0x2a, 0xc9,
|
0x95, 0x15, 0x5f, 0xc6, 0x3d, 0xc4, 0x29, 0x13, 0xff, 0xff, 0xf6, 0xcb, 0x76, 0x00, 0x42, 0x1e,
|
||||||
0x69, 0xbb, 0xd8, 0xb9, 0xa0, 0x81, 0x9c, 0x32, 0x9d, 0x4b, 0xcc, 0x48, 0xfc, 0x3a, 0x20, 0xc3,
|
0xd9, 0xe4, 0x42, 0x34, 0xe2, 0x64, 0x4c, 0xa3, 0x96, 0x99, 0xea, 0x74, 0xbd, 0x9d, 0xa7, 0xe6,
|
||||||
0x8c, 0x64, 0x5c, 0x42, 0xa4, 0x3a, 0x14, 0xd9, 0x77, 0x0c, 0x2c, 0xf2, 0x8f, 0xe7, 0xb8, 0x53,
|
0x27, 0xdb, 0x6c, 0x66, 0xbb, 0x5a, 0x4f, 0xa0, 0x11, 0x2e, 0x11, 0x45, 0xa9, 0x7f, 0xe7, 0x93,
|
||||||
0x23, 0xff, 0x8c, 0x5b, 0x83, 0xd4, 0xc8, 0x3f, 0x2b, 0x69, 0x2e, 0x5f, 0x5a, 0xfb, 0xf0, 0xbb,
|
0xac, 0x21, 0x9d, 0xb6, 0x8b, 0xed, 0x73, 0x5a, 0xe3, 0x29, 0xd3, 0xb9, 0xc4, 0x8c, 0xc4, 0xef,
|
||||||
0x1f, 0x0c, 0x75, 0xef, 0x78, 0x7c, 0x48, 0x76, 0xff, 0x90, 0x0d, 0x7d, 0x4f, 0xb7, 0xf9, 0xaf,
|
0x1e, 0x32, 0xcc, 0x48, 0xc6, 0x8d, 0x47, 0xaa, 0xf7, 0x92, 0x7d, 0xa1, 0xc1, 0xd2, 0x0c, 0xf1,
|
||||||
0x87, 0x3e, 0xbb, 0x3f, 0xa4, 0xb3, 0x3d, 0x24, 0xb3, 0x8d, 0x0e, 0x0f, 0x2b, 0xb4, 0xf5, 0xe1,
|
0x84, 0x7a, 0x6a, 0x9a, 0x21, 0xe3, 0x8a, 0x22, 0x35, 0xcd, 0x90, 0x95, 0xa1, 0x97, 0x2f, 0xac,
|
||||||
0xff, 0x06, 0x00, 0x00, 0xff, 0xff, 0xaa, 0x2f, 0xbb, 0x90, 0x2c, 0x4d, 0x00, 0x00,
|
0x7d, 0xf0, 0xfd, 0xf7, 0x87, 0xba, 0x77, 0x34, 0x3e, 0x20, 0xbb, 0x7f, 0xc0, 0x86, 0xbe, 0xab,
|
||||||
|
0xdb, 0xfc, 0xd7, 0x03, 0x9f, 0xdd, 0x1f, 0xd0, 0xd9, 0x1e, 0x90, 0xd9, 0x46, 0x07, 0x07, 0x15,
|
||||||
|
0xda, 0xfa, 0xe0, 0x7f, 0x03, 0x00, 0x00, 0xff, 0xff, 0x6b, 0x3a, 0x03, 0xfa, 0x22, 0x4e, 0x00,
|
||||||
|
0x00,
|
||||||
}
|
}
|
||||||
|
|
||||||
// Reference imports to suppress errors if they are not otherwise used.
|
// Reference imports to suppress errors if they are not otherwise used.
|
||||||
@ -5144,6 +5195,7 @@ type DataCoordClient interface {
|
|||||||
// https://wiki.lfaidata.foundation/display/MIL/MEP+24+--+Support+bulk+load
|
// https://wiki.lfaidata.foundation/display/MIL/MEP+24+--+Support+bulk+load
|
||||||
Import(ctx context.Context, in *ImportTaskRequest, opts ...grpc.CallOption) (*ImportTaskResponse, error)
|
Import(ctx context.Context, in *ImportTaskRequest, opts ...grpc.CallOption) (*ImportTaskResponse, error)
|
||||||
UpdateSegmentStatistics(ctx context.Context, in *UpdateSegmentStatisticsRequest, opts ...grpc.CallOption) (*commonpb.Status, 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)
|
AcquireSegmentLock(ctx context.Context, in *AcquireSegmentLockRequest, opts ...grpc.CallOption) (*commonpb.Status, error)
|
||||||
ReleaseSegmentLock(ctx context.Context, in *ReleaseSegmentLockRequest, 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)
|
SaveImportSegment(ctx context.Context, in *SaveImportSegmentRequest, opts ...grpc.CallOption) (*commonpb.Status, error)
|
||||||
@ -5395,6 +5447,15 @@ func (c *dataCoordClient) UpdateSegmentStatistics(ctx context.Context, in *Updat
|
|||||||
return out, nil
|
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) {
|
func (c *dataCoordClient) AcquireSegmentLock(ctx context.Context, in *AcquireSegmentLockRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
|
||||||
out := new(commonpb.Status)
|
out := new(commonpb.Status)
|
||||||
err := c.cc.Invoke(ctx, "/milvus.proto.data.DataCoord/AcquireSegmentLock", in, out, opts...)
|
err := c.cc.Invoke(ctx, "/milvus.proto.data.DataCoord/AcquireSegmentLock", in, out, opts...)
|
||||||
@ -5488,6 +5549,7 @@ type DataCoordServer interface {
|
|||||||
// https://wiki.lfaidata.foundation/display/MIL/MEP+24+--+Support+bulk+load
|
// https://wiki.lfaidata.foundation/display/MIL/MEP+24+--+Support+bulk+load
|
||||||
Import(context.Context, *ImportTaskRequest) (*ImportTaskResponse, error)
|
Import(context.Context, *ImportTaskRequest) (*ImportTaskResponse, error)
|
||||||
UpdateSegmentStatistics(context.Context, *UpdateSegmentStatisticsRequest) (*commonpb.Status, error)
|
UpdateSegmentStatistics(context.Context, *UpdateSegmentStatisticsRequest) (*commonpb.Status, error)
|
||||||
|
UpdateChannelCheckpoint(context.Context, *UpdateChannelCheckpointRequest) (*commonpb.Status, error)
|
||||||
AcquireSegmentLock(context.Context, *AcquireSegmentLockRequest) (*commonpb.Status, error)
|
AcquireSegmentLock(context.Context, *AcquireSegmentLockRequest) (*commonpb.Status, error)
|
||||||
ReleaseSegmentLock(context.Context, *ReleaseSegmentLockRequest) (*commonpb.Status, error)
|
ReleaseSegmentLock(context.Context, *ReleaseSegmentLockRequest) (*commonpb.Status, error)
|
||||||
SaveImportSegment(context.Context, *SaveImportSegmentRequest) (*commonpb.Status, error)
|
SaveImportSegment(context.Context, *SaveImportSegmentRequest) (*commonpb.Status, error)
|
||||||
@ -5579,6 +5641,9 @@ func (*UnimplementedDataCoordServer) Import(ctx context.Context, req *ImportTask
|
|||||||
func (*UnimplementedDataCoordServer) UpdateSegmentStatistics(ctx context.Context, req *UpdateSegmentStatisticsRequest) (*commonpb.Status, error) {
|
func (*UnimplementedDataCoordServer) UpdateSegmentStatistics(ctx context.Context, req *UpdateSegmentStatisticsRequest) (*commonpb.Status, error) {
|
||||||
return nil, status.Errorf(codes.Unimplemented, "method UpdateSegmentStatistics not implemented")
|
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) {
|
func (*UnimplementedDataCoordServer) AcquireSegmentLock(ctx context.Context, req *AcquireSegmentLockRequest) (*commonpb.Status, error) {
|
||||||
return nil, status.Errorf(codes.Unimplemented, "method AcquireSegmentLock not implemented")
|
return nil, status.Errorf(codes.Unimplemented, "method AcquireSegmentLock not implemented")
|
||||||
}
|
}
|
||||||
@ -6073,6 +6138,24 @@ func _DataCoord_UpdateSegmentStatistics_Handler(srv interface{}, ctx context.Con
|
|||||||
return interceptor(ctx, in, info, handler)
|
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) {
|
func _DataCoord_AcquireSegmentLock_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
|
||||||
in := new(AcquireSegmentLockRequest)
|
in := new(AcquireSegmentLockRequest)
|
||||||
if err := dec(in); err != nil {
|
if err := dec(in); err != nil {
|
||||||
@ -6307,6 +6390,10 @@ var _DataCoord_serviceDesc = grpc.ServiceDesc{
|
|||||||
MethodName: "UpdateSegmentStatistics",
|
MethodName: "UpdateSegmentStatistics",
|
||||||
Handler: _DataCoord_UpdateSegmentStatistics_Handler,
|
Handler: _DataCoord_UpdateSegmentStatistics_Handler,
|
||||||
},
|
},
|
||||||
|
{
|
||||||
|
MethodName: "UpdateChannelCheckpoint",
|
||||||
|
Handler: _DataCoord_UpdateChannelCheckpoint_Handler,
|
||||||
|
},
|
||||||
{
|
{
|
||||||
MethodName: "AcquireSegmentLock",
|
MethodName: "AcquireSegmentLock",
|
||||||
Handler: _DataCoord_AcquireSegmentLock_Handler,
|
Handler: _DataCoord_AcquireSegmentLock_Handler,
|
||||||
|
|||||||
@ -271,6 +271,13 @@ func (coord *DataCoordMock) UpdateSegmentStatistics(ctx context.Context, req *da
|
|||||||
}, nil
|
}, 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) {
|
func (coord *DataCoordMock) AcquireSegmentLock(ctx context.Context, req *datapb.AcquireSegmentLockRequest) (*commonpb.Status, error) {
|
||||||
return &commonpb.Status{
|
return &commonpb.Status{
|
||||||
ErrorCode: commonpb.ErrorCode_Success,
|
ErrorCode: commonpb.ErrorCode_Success,
|
||||||
|
|||||||
@ -313,6 +313,8 @@ type DataCoord interface {
|
|||||||
|
|
||||||
// UpdateSegmentStatistics updates a segment's stats.
|
// UpdateSegmentStatistics updates a segment's stats.
|
||||||
UpdateSegmentStatistics(ctx context.Context, req *datapb.UpdateSegmentStatisticsRequest) (*commonpb.Status, error)
|
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)
|
AcquireSegmentLock(ctx context.Context, req *datapb.AcquireSegmentLockRequest) (*commonpb.Status, error)
|
||||||
ReleaseSegmentLock(ctx context.Context, req *datapb.ReleaseSegmentLockRequest) (*commonpb.Status, error)
|
ReleaseSegmentLock(ctx context.Context, req *datapb.ReleaseSegmentLockRequest) (*commonpb.Status, error)
|
||||||
|
|||||||
@ -145,6 +145,10 @@ func (m *GrpcDataCoordClient) UpdateSegmentStatistics(ctx context.Context, req *
|
|||||||
return &commonpb.Status{}, m.Err
|
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) {
|
func (m *GrpcDataCoordClient) AcquireSegmentLock(ctx context.Context, req *datapb.AcquireSegmentLockRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
|
||||||
return &commonpb.Status{}, m.Err
|
return &commonpb.Status{}, m.Err
|
||||||
}
|
}
|
||||||
|
|||||||
Loading…
x
Reference in New Issue
Block a user