diff --git a/internal/datanode/channel_meta.go b/internal/datanode/channel_meta.go new file mode 100644 index 0000000000..6164bde1a0 --- /dev/null +++ b/internal/datanode/channel_meta.go @@ -0,0 +1,637 @@ +// 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" + "sync" + + "github.com/bits-and-blooms/bloom/v3" + "github.com/milvus-io/milvus-proto/go-api/schemapb" + "go.uber.org/zap" + + "github.com/milvus-io/milvus/internal/common" + "github.com/milvus-io/milvus/internal/log" + "github.com/milvus-io/milvus/internal/metrics" + "github.com/milvus-io/milvus/internal/proto/datapb" + "github.com/milvus-io/milvus/internal/proto/internalpb" + "github.com/milvus-io/milvus/internal/storage" + "github.com/milvus-io/milvus/internal/types" + "github.com/milvus-io/milvus/internal/util/typeutil" +) + +const ( + maxBloomFalsePositive float64 = 0.005 +) + +type ( + primaryKey = storage.PrimaryKey + int64PrimaryKey = storage.Int64PrimaryKey + varCharPrimaryKey = storage.VarCharPrimaryKey +) + +var ( + newInt64PrimaryKey = storage.NewInt64PrimaryKey + newVarCharPrimaryKey = storage.NewVarCharPrimaryKey +) + +// Channel is DataNode unique replication +type Channel interface { + getCollectionID() UniqueID + getCollectionSchema(collectionID UniqueID, ts Timestamp) (*schemapb.CollectionSchema, error) + getCollectionAndPartitionID(segID UniqueID) (collID, partitionID UniqueID, err error) + getChannelName(segID UniqueID) string + + listAllSegmentIDs() []UniqueID + listNotFlushedSegmentIDs() []UniqueID + addSegment(req addSegmentReq) error + listPartitionSegments(partID UniqueID) []UniqueID + filterSegments(partitionID UniqueID) []*Segment + listNewSegmentsStartPositions() []*datapb.SegmentStartPosition + transferNewSegments(segmentIDs []UniqueID) + updateSegmentEndPosition(segID UniqueID, endPos *internalpb.MsgPosition) + updateSegmentPKRange(segID UniqueID, ids storage.FieldData) + mergeFlushedSegments(seg *Segment, planID UniqueID, compactedFrom []UniqueID) error + hasSegment(segID UniqueID, countFlushed bool) bool + removeSegments(segID ...UniqueID) + listCompactedSegmentIDs() map[UniqueID][]UniqueID + + updateStatistics(segID UniqueID, numRows int64) + getSegmentStatisticsUpdates(segID UniqueID) (*datapb.SegmentStats, error) + segmentFlushed(segID UniqueID) + getSegmentStatslog(segID UniqueID) ([]byte, error) + initSegmentBloomFilter(seg *Segment) error +} + +// ChannelMeta contains channel meta and the latest segments infos of the channel. +type ChannelMeta struct { + collectionID UniqueID + channelName string + collSchema *schemapb.CollectionSchema + schemaMut sync.RWMutex + + segMu sync.RWMutex + segments map[UniqueID]*Segment + + metaService *metaService + chunkManager storage.ChunkManager +} + +var _ Channel = &ChannelMeta{} + +func newChannel(channelName string, collID UniqueID, schema *schemapb.CollectionSchema, rc types.RootCoord, cm storage.ChunkManager) *ChannelMeta { + metaService := newMetaService(rc, collID) + + channel := ChannelMeta{ + collectionID: collID, + collSchema: schema, + channelName: channelName, + + segments: make(map[UniqueID]*Segment), + + metaService: metaService, + chunkManager: cm, + } + + return &channel +} + +// segmentFlushed transfers a segment from *New* or *Normal* into *Flushed*. +func (c *ChannelMeta) segmentFlushed(segID UniqueID) { + c.segMu.Lock() + defer c.segMu.Unlock() + + if seg, ok := c.segments[segID]; ok { + seg.setType(datapb.SegmentType_Flushed) + } + metrics.DataNodeNumUnflushedSegments.WithLabelValues(fmt.Sprint(Params.DataNodeCfg.GetNodeID())).Dec() +} + +// new2NormalSegment transfers a segment from *New* to *Normal*. +// make sure the segID is in the channel before call this func +func (c *ChannelMeta) new2NormalSegment(segID UniqueID) { + seg := c.segments[segID] + if seg.getType() == datapb.SegmentType_New { + seg.setType(datapb.SegmentType_Normal) + } +} + +func (c *ChannelMeta) getCollectionAndPartitionID(segID UniqueID) (collID, partitionID UniqueID, err error) { + c.segMu.RLock() + defer c.segMu.RUnlock() + + if seg, ok := c.segments[segID]; ok && seg.isValid() { + return seg.collectionID, seg.partitionID, nil + } + return 0, 0, fmt.Errorf("cannot find segment, id = %d", segID) +} + +func (c *ChannelMeta) getChannelName(segID UniqueID) string { + return c.channelName +} + +// maxRowCountPerSegment returns max row count for a segment based on estimation of row size. +func (c *ChannelMeta) maxRowCountPerSegment(ts Timestamp) (int64, error) { + log := log.With(zap.Int64("collectionID", c.collectionID), zap.Uint64("timpstamp", ts)) + schema, err := c.getCollectionSchema(c.collectionID, ts) + if err != nil { + log.Warn("failed to get collection schema", zap.Error(err)) + return 0, err + } + sizePerRecord, err := typeutil.EstimateSizePerRecord(schema) + if err != nil { + log.Warn("failed to estimate size per record", zap.Error(err)) + return 0, err + } + threshold := Params.DataCoordCfg.SegmentMaxSize * 1024 * 1024 + return int64(threshold / float64(sizePerRecord)), nil +} + +// initSegmentBloomFilter initialize segment pkFilter with a new bloom filter. +// this new BF will be initialized with estimated max rows and default false positive rate. +func (c *ChannelMeta) initSegmentBloomFilter(s *Segment) error { + var ts Timestamp + if s.startPos != nil { + ts = s.startPos.Timestamp + } + maxRowCount, err := c.maxRowCountPerSegment(ts) + if err != nil { + log.Warn("initSegmentBloomFilter failed, cannot estimate max row count", zap.Error(err)) + return err + } + + s.pkFilter = bloom.NewWithEstimates(uint(maxRowCount), maxBloomFalsePositive) + return nil +} + +// addSegment adds the segment to current channel. Segments can be added as *new*, *normal* or *flushed*. +// Make sure to verify `channel.hasSegment(segID)` == false before calling `channel.addSegment()`. +func (c *ChannelMeta) addSegment(req addSegmentReq) error { + if req.collID != c.collectionID { + log.Warn("collection mismatch", + zap.Int64("current collection ID", req.collID), + zap.Int64("expected collection ID", c.collectionID)) + return fmt.Errorf("mismatch collection, ID=%d", req.collID) + } + log.Info("adding segment", + zap.String("segment type", req.segType.String()), + zap.Int64("segment ID", req.segID), + zap.Int64("collection ID", req.collID), + zap.Int64("partition ID", req.partitionID), + zap.String("channel name", c.channelName), + zap.Any("start position", req.startPos), + zap.Any("end position", req.endPos), + zap.Uint64("recover ts", req.recoverTs), + zap.Bool("importing", req.importing), + ) + seg := &Segment{ + collectionID: req.collID, + partitionID: req.partitionID, + segmentID: req.segID, + numRows: req.numOfRows, // 0 if segType == NEW + startPos: req.startPos, + endPos: req.endPos, + } + seg.sType.Store(req.segType) + // Set up bloom filter. + err := c.initPKBloomFilter(context.TODO(), seg, req.statsBinLogs, req.recoverTs) + if err != nil { + log.Error("failed to init bloom filter", + zap.Int64("segment ID", req.segID), + zap.Error(err)) + return err + } + + c.segMu.Lock() + c.segments[req.segID] = seg + c.segMu.Unlock() + if req.segType == datapb.SegmentType_New || req.segType == datapb.SegmentType_Normal { + metrics.DataNodeNumUnflushedSegments.WithLabelValues(fmt.Sprint(Params.DataNodeCfg.GetNodeID())).Inc() + } + return nil +} + +func (c *ChannelMeta) listCompactedSegmentIDs() map[UniqueID][]UniqueID { + c.segMu.RLock() + defer c.segMu.RUnlock() + + compactedTo2From := make(map[UniqueID][]UniqueID) + + for segID, seg := range c.segments { + if !seg.isValid() { + compactedTo2From[seg.compactedTo] = append(compactedTo2From[seg.compactedTo], segID) + } + } + return compactedTo2From +} + +// filterSegments return segments with same partitionID for all segments +// get all segments +func (c *ChannelMeta) filterSegments(partitionID UniqueID) []*Segment { + c.segMu.RLock() + defer c.segMu.RUnlock() + + var results []*Segment + for _, seg := range c.segments { + if seg.isValid() && + partitionID == common.InvalidPartitionID || seg.partitionID == partitionID { + results = append(results, seg) + } + } + return results +} + +func (c *ChannelMeta) initPKBloomFilter(ctx context.Context, s *Segment, statsBinlogs []*datapb.FieldBinlog, ts Timestamp) error { + log := log.With(zap.Int64("segmentID", s.segmentID)) + log.Info("begin to init pk bloom filter", zap.Int("stats bin logs", len(statsBinlogs))) + schema, err := c.getCollectionSchema(s.collectionID, ts) + if err != nil { + log.Warn("failed to initPKBloomFilter, get schema return error", zap.Error(err)) + return err + } + + // get pkfield id + pkField := int64(-1) + for _, field := range schema.Fields { + if field.IsPrimaryKey { + pkField = field.FieldID + break + } + } + + // filter stats binlog files which is pk field stats log + var bloomFilterFiles []string + for _, binlog := range statsBinlogs { + if binlog.FieldID != pkField { + continue + } + for _, log := range binlog.GetBinlogs() { + bloomFilterFiles = append(bloomFilterFiles, log.GetLogPath()) + } + } + + // no stats log to parse, initialize a new BF + if len(bloomFilterFiles) == 0 { + log.Warn("no stats files to load, initializa a new one") + return c.initSegmentBloomFilter(s) + } + + values, err := c.chunkManager.MultiRead(ctx, bloomFilterFiles) + if err != nil { + log.Warn("failed to load bloom filter files", zap.Error(err)) + return err + } + blobs := make([]*Blob, 0) + for i := 0; i < len(values); i++ { + blobs = append(blobs, &Blob{Value: values[i]}) + } + + stats, err := storage.DeserializeStats(blobs) + if err != nil { + log.Warn("failed to deserialize bloom filter files", zap.Error(err)) + return err + } + for _, stat := range stats { + // use first BF to merge + if s.pkFilter == nil { + s.pkFilter = stat.BF + } else { + // for compatibility, statslog before 2.1.2 uses separated stats log which needs to be merged + // assuming all legacy BF has same attributes. + err = s.pkFilter.Merge(stat.BF) + if err != nil { + return err + } + } + + s.updatePk(stat.MinPk) + s.updatePk(stat.MaxPk) + } + + return nil +} + +// listNewSegmentsStartPositions gets all *New Segments* start positions and +// transfer segments states from *New* to *Normal*. +func (c *ChannelMeta) listNewSegmentsStartPositions() []*datapb.SegmentStartPosition { + c.segMu.Lock() + defer c.segMu.Unlock() + + var result []*datapb.SegmentStartPosition + for id, seg := range c.segments { + if seg.getType() == datapb.SegmentType_New { + result = append(result, &datapb.SegmentStartPosition{ + SegmentID: id, + StartPosition: seg.startPos, + }) + } + } + return result +} + +// transferNewSegments make new segment transfer to normal segments. +func (c *ChannelMeta) transferNewSegments(segmentIDs []UniqueID) { + c.segMu.Lock() + defer c.segMu.Unlock() + + for _, segmentID := range segmentIDs { + c.new2NormalSegment(segmentID) + } +} + +// updateSegmentEndPosition updates *New* or *Normal* segment's end position. +func (c *ChannelMeta) updateSegmentEndPosition(segID UniqueID, endPos *internalpb.MsgPosition) { + c.segMu.Lock() + defer c.segMu.Unlock() + + seg, ok := c.segments[segID] + if ok && seg.notFlushed() { + seg.endPos = endPos + return + } + + log.Warn("No match segment", zap.Int64("ID", segID)) +} + +func (c *ChannelMeta) updateSegmentPKRange(segID UniqueID, ids storage.FieldData) { + c.segMu.Lock() + defer c.segMu.Unlock() + + seg, ok := c.segments[segID] + if ok && seg.isValid() { + seg.updatePKRange(ids) + return + } + + log.Warn("No match segment to update PK range", zap.Int64("ID", segID)) +} + +func (c *ChannelMeta) removeSegments(segIDs ...UniqueID) { + c.segMu.Lock() + defer c.segMu.Unlock() + + log.Info("remove segments if exist", zap.Int64s("segmentIDs", segIDs)) + cnt := 0 + for _, segID := range segIDs { + seg, ok := c.segments[segID] + if ok && + (seg.getType() == datapb.SegmentType_New || seg.getType() == datapb.SegmentType_Normal) { + cnt++ + } + + delete(c.segments, segID) + } + metrics.DataNodeNumUnflushedSegments.WithLabelValues(fmt.Sprint(Params.DataNodeCfg.GetNodeID())).Sub(float64(cnt)) +} + +// hasSegment checks whether this channel has a segment according to segment ID. +func (c *ChannelMeta) hasSegment(segID UniqueID, countFlushed bool) bool { + c.segMu.RLock() + defer c.segMu.RUnlock() + + seg, ok := c.segments[segID] + if !ok { + return false + } + + if !seg.isValid() || + (!countFlushed && seg.getType() == datapb.SegmentType_Flushed) { + return false + } + + return true +} + +// updateStatistics updates the number of rows of a segment in channel. +func (c *ChannelMeta) updateStatistics(segID UniqueID, numRows int64) { + c.segMu.Lock() + defer c.segMu.Unlock() + + log.Info("updating segment", zap.Int64("Segment ID", segID), zap.Int64("numRows", numRows)) + seg, ok := c.segments[segID] + if ok && seg.notFlushed() { + seg.memorySize = 0 + seg.numRows += numRows + return + } + + log.Warn("update segment num row not exist", zap.Int64("segID", segID)) +} + +// getSegmentStatisticsUpdates gives current segment's statistics updates. +func (c *ChannelMeta) getSegmentStatisticsUpdates(segID UniqueID) (*datapb.SegmentStats, error) { + c.segMu.RLock() + defer c.segMu.RUnlock() + + if seg, ok := c.segments[segID]; ok && seg.isValid() { + return &datapb.SegmentStats{SegmentID: segID, NumRows: seg.numRows}, nil + } + + return nil, fmt.Errorf("error, there's no segment %d", segID) +} + +func (c *ChannelMeta) getCollectionID() UniqueID { + return c.collectionID +} + +// getCollectionSchema gets collection schema from rootcoord for a certain timestamp. +// If you want the latest collection schema, ts should be 0. +func (c *ChannelMeta) getCollectionSchema(collID UniqueID, ts Timestamp) (*schemapb.CollectionSchema, error) { + if !c.validCollection(collID) { + return nil, fmt.Errorf("mismatch collection, want %d, actual %d", c.collectionID, collID) + } + + c.schemaMut.RLock() + if c.collSchema == nil { + c.schemaMut.RUnlock() + + c.schemaMut.Lock() + defer c.schemaMut.Unlock() + if c.collSchema == nil { + sch, err := c.metaService.getCollectionSchema(context.Background(), collID, ts) + if err != nil { + return nil, err + } + c.collSchema = sch + } + } else { + defer c.schemaMut.RUnlock() + } + + return c.collSchema, nil +} + +func (c *ChannelMeta) validCollection(collID UniqueID) bool { + return collID == c.collectionID +} + +func (c *ChannelMeta) mergeFlushedSegments(seg *Segment, planID UniqueID, compactedFrom []UniqueID) error { + + log := log.With( + zap.Int64("segment ID", seg.segmentID), + zap.Int64("collection ID", seg.collectionID), + zap.Int64("partition ID", seg.partitionID), + zap.Int64s("compacted from", compactedFrom), + zap.Int64("planID", planID), + zap.String("channel name", c.channelName)) + + if seg.collectionID != c.collectionID { + log.Warn("Mismatch collection", + zap.Int64("expected collectionID", c.collectionID)) + return fmt.Errorf("mismatch collection, ID=%d", seg.collectionID) + } + + var inValidSegments []UniqueID + for _, ID := range compactedFrom { + // no such segments in channel or the segments are unflushed. + if !c.hasSegment(ID, true) || c.hasSegment(ID, false) { + inValidSegments = append(inValidSegments, ID) + } + } + + if len(inValidSegments) > 0 { + log.Warn("no match flushed segments to merge from", zap.Int64s("invalid segmentIDs", inValidSegments)) + return fmt.Errorf("invalid compactedFrom segments: %v", inValidSegments) + } + + log.Info("merge flushed segments") + c.segMu.Lock() + for _, ID := range compactedFrom { + // the existent of the segments are already checked + s := c.segments[ID] + s.compactedTo = seg.segmentID + s.setType(datapb.SegmentType_Compacted) + } + c.segMu.Unlock() + + // only store segments with numRows > 0 + if seg.numRows > 0 { + seg.setType(datapb.SegmentType_Flushed) + + c.segMu.Lock() + c.segments[seg.segmentID] = seg + c.segMu.Unlock() + } + + return nil +} + +// for tests only +func (c *ChannelMeta) addFlushedSegmentWithPKs(segID, collID, partID UniqueID, numOfRows int64, ids storage.FieldData) error { + if collID != c.collectionID { + log.Warn("Mismatch collection", + zap.Int64("input ID", collID), + zap.Int64("expected ID", c.collectionID)) + return fmt.Errorf("mismatch collection, ID=%d", collID) + } + + log.Info("Add Flushed segment", + zap.Int64("segment ID", segID), + zap.Int64("collection ID", collID), + zap.Int64("partition ID", partID), + zap.String("channel name", c.channelName), + ) + + seg := &Segment{ + collectionID: collID, + partitionID: partID, + segmentID: segID, + numRows: numOfRows, + } + + err := c.initSegmentBloomFilter(seg) + if err != nil { + return err + } + + seg.updatePKRange(ids) + seg.setType(datapb.SegmentType_Flushed) + + c.segMu.Lock() + c.segments[segID] = seg + c.segMu.Unlock() + + return nil +} + +func (c *ChannelMeta) listAllSegmentIDs() []UniqueID { + c.segMu.RLock() + defer c.segMu.RUnlock() + + var segIDs []UniqueID + for _, seg := range c.segments { + if seg.isValid() { + segIDs = append(segIDs, seg.segmentID) + } + } + return segIDs +} + +func (c *ChannelMeta) listPartitionSegments(partID UniqueID) []UniqueID { + c.segMu.RLock() + defer c.segMu.RUnlock() + + var segIDs []UniqueID + for _, seg := range c.segments { + if seg.isValid() && seg.partitionID == partID { + segIDs = append(segIDs, seg.segmentID) + } + } + return segIDs +} + +func (c *ChannelMeta) listNotFlushedSegmentIDs() []UniqueID { + c.segMu.RLock() + defer c.segMu.RUnlock() + + var segIDs []UniqueID + for sID, seg := range c.segments { + if seg.notFlushed() { + segIDs = append(segIDs, sID) + } + } + + return segIDs +} + +// getSegmentStatslog returns the segment statslog for the provided segment id. +func (c *ChannelMeta) getSegmentStatslog(segID UniqueID) ([]byte, error) { + c.segMu.RLock() + defer c.segMu.RUnlock() + colID := c.getCollectionID() + + schema, err := c.getCollectionSchema(colID, 0) + if err != nil { + return nil, err + } + + var pkID UniqueID + var pkType schemapb.DataType + for _, field := range schema.GetFields() { + if field.GetIsPrimaryKey() { + pkID = field.GetFieldID() + pkType = field.GetDataType() + } + } + + if seg, ok := c.segments[segID]; ok && seg.isValid() { + return seg.getSegmentStatslog(pkID, pkType) + } + + return nil, fmt.Errorf("segment not found: %d", segID) +} diff --git a/internal/datanode/channel_meta_test.go b/internal/datanode/channel_meta_test.go new file mode 100644 index 0000000000..3f81fac005 --- /dev/null +++ b/internal/datanode/channel_meta_test.go @@ -0,0 +1,959 @@ +// 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" + "encoding/json" + "errors" + "fmt" + "math/rand" + "testing" + + "github.com/bits-and-blooms/bloom/v3" + "github.com/samber/lo" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + "github.com/stretchr/testify/suite" + + "github.com/milvus-io/milvus-proto/go-api/schemapb" + "github.com/milvus-io/milvus/internal/common" + "github.com/milvus-io/milvus/internal/proto/datapb" + "github.com/milvus-io/milvus/internal/proto/internalpb" + "github.com/milvus-io/milvus/internal/storage" +) + +var channelMetaNodeTestDir = "/tmp/milvus_test/channel_meta" + +func TestNewChannel(t *testing.T) { + rc := &RootCoordFactory{} + cm := storage.NewLocalChunkManager(storage.RootPath(channelMetaNodeTestDir)) + defer cm.RemoveWithPrefix(context.Background(), "") + channel := newChannel("channel", 0, nil, rc, cm) + assert.NotNil(t, channel) +} + +type mockDataCM struct { + storage.ChunkManager +} + +func (kv *mockDataCM) MultiRead(ctx context.Context, keys []string) ([][]byte, error) { + stats := &storage.PrimaryKeyStats{ + FieldID: common.RowIDField, + Min: 0, + Max: 10, + BF: bloom.NewWithEstimates(100000, maxBloomFalsePositive), + } + buffer, _ := json.Marshal(stats) + return [][]byte{buffer}, nil +} + +type mockPkfilterMergeError struct { + storage.ChunkManager +} + +func (kv *mockPkfilterMergeError) MultiRead(ctx context.Context, keys []string) ([][]byte, error) { + /* + stats := &storage.PrimaryKeyStats{ + FieldID: common.RowIDField, + Min: 0, + Max: 10, + BF: bloom.NewWithEstimates(1, 0.0001), + } + buffer, _ := json.Marshal(stats) + return [][]byte{buffer}, nil*/ + return nil, errors.New("mocked multi read error") +} + +type mockDataCMError struct { + storage.ChunkManager +} + +func (kv *mockDataCMError) MultiRead(ctx context.Context, keys []string) ([][]byte, error) { + return nil, fmt.Errorf("mock error") +} + +type mockDataCMStatsError struct { + storage.ChunkManager +} + +func (kv *mockDataCMStatsError) MultiRead(ctx context.Context, keys []string) ([][]byte, error) { + return [][]byte{[]byte("3123123,error,test")}, nil +} + +func getSimpleFieldBinlog() *datapb.FieldBinlog { + return &datapb.FieldBinlog{ + FieldID: 106, + Binlogs: []*datapb.Binlog{{LogPath: "test"}}, + } +} + +func TestChannelMeta_InnerFunction(t *testing.T) { + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + rc := &RootCoordFactory{ + pkType: schemapb.DataType_Int64, + } + + var ( + collID = UniqueID(1) + cm = storage.NewLocalChunkManager(storage.RootPath(channelMetaNodeTestDir)) + channel = newChannel("insert-01", collID, nil, rc, cm) + ) + defer cm.RemoveWithPrefix(ctx, "") + + require.False(t, channel.hasSegment(0, true)) + require.False(t, channel.hasSegment(0, false)) + + var err error + + startPos := &internalpb.MsgPosition{ChannelName: "insert-01", Timestamp: Timestamp(100)} + endPos := &internalpb.MsgPosition{ChannelName: "insert-01", Timestamp: Timestamp(200)} + err = channel.addSegment( + addSegmentReq{ + segType: datapb.SegmentType_New, + segID: 0, + collID: 1, + partitionID: 2, + startPos: startPos, + endPos: endPos, + }) + assert.NoError(t, err) + assert.True(t, channel.hasSegment(0, true)) + + seg, ok := channel.segments[UniqueID(0)] + assert.True(t, ok) + require.NotNil(t, seg) + assert.Equal(t, UniqueID(0), seg.segmentID) + assert.Equal(t, UniqueID(1), seg.collectionID) + assert.Equal(t, UniqueID(2), seg.partitionID) + assert.Equal(t, Timestamp(100), seg.startPos.Timestamp) + assert.Equal(t, Timestamp(200), seg.endPos.Timestamp) + assert.Equal(t, int64(0), seg.numRows) + assert.Equal(t, datapb.SegmentType_New, seg.getType()) + + channel.updateStatistics(0, 10) + assert.Equal(t, int64(10), seg.numRows) + + segPos := channel.listNewSegmentsStartPositions() + assert.Equal(t, 1, len(segPos)) + assert.Equal(t, UniqueID(0), segPos[0].SegmentID) + assert.Equal(t, "insert-01", segPos[0].StartPosition.ChannelName) + assert.Equal(t, Timestamp(100), segPos[0].StartPosition.Timestamp) + + channel.transferNewSegments(lo.Map(segPos, func(pos *datapb.SegmentStartPosition, _ int) UniqueID { + return pos.GetSegmentID() + })) + + updates, err := channel.getSegmentStatisticsUpdates(0) + assert.NoError(t, err) + assert.Equal(t, int64(10), updates.NumRows) + + totalSegments := channel.filterSegments(common.InvalidPartitionID) + assert.Equal(t, len(totalSegments), 1) +} + +// TODO GOOSE +func TestChannelMeta_getChannelName(t *testing.T) { + t.Skip() +} + +func TestChannelMeta_getCollectionAndPartitionID(t *testing.T) { + tests := []struct { + segID UniqueID + segType datapb.SegmentType + + inCollID UniqueID + inParID UniqueID + description string + }{ + {100, datapb.SegmentType_New, 1, 10, "Segment 100 of type New"}, + {200, datapb.SegmentType_Normal, 2, 20, "Segment 200 of type Normal"}, + {300, datapb.SegmentType_Flushed, 3, 30, "Segment 300 of type Flushed"}, + } + + for _, test := range tests { + t.Run(test.description, func(t *testing.T) { + seg := Segment{ + collectionID: test.inCollID, + partitionID: test.inParID, + segmentID: test.segID, + } + seg.setType(test.segType) + channel := &ChannelMeta{ + segments: map[UniqueID]*Segment{ + test.segID: &seg}, + } + + collID, parID, err := channel.getCollectionAndPartitionID(test.segID) + assert.NoError(t, err) + assert.Equal(t, test.inCollID, collID) + assert.Equal(t, test.inParID, parID) + }) + } +} + +func TestChannelMeta_segmentFlushed(t *testing.T) { + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + rc := &RootCoordFactory{ + pkType: schemapb.DataType_Int64, + } + collID := UniqueID(1) + cm := storage.NewLocalChunkManager(storage.RootPath(channelMetaNodeTestDir)) + defer cm.RemoveWithPrefix(ctx, "") + + t.Run("Test coll mot match", func(t *testing.T) { + channel := newChannel("channel", collID, nil, rc, cm) + err := channel.addSegment( + addSegmentReq{ + segType: datapb.SegmentType_New, + segID: 1, + collID: collID + 1, + partitionID: 0, + startPos: nil, + endPos: nil, + }) + assert.NotNil(t, err) + }) + + t.Run("Test segmentFlushed", func(t *testing.T) { + channel := &ChannelMeta{ + segments: make(map[UniqueID]*Segment), + } + + type Test struct { + inSegType datapb.SegmentType + inSegID UniqueID + } + + tests := []Test{ + // new segment + {datapb.SegmentType_New, 1}, + {datapb.SegmentType_New, 2}, + {datapb.SegmentType_New, 3}, + // normal segment + {datapb.SegmentType_Normal, 10}, + {datapb.SegmentType_Normal, 20}, + {datapb.SegmentType_Normal, 30}, + // flushed segment + {datapb.SegmentType_Flushed, 100}, + {datapb.SegmentType_Flushed, 200}, + {datapb.SegmentType_Flushed, 300}, + } + + newSeg := func(channel *ChannelMeta, sType datapb.SegmentType, id UniqueID) { + s := Segment{segmentID: id} + s.setType(sType) + channel.segments[id] = &s + } + + for _, test := range tests { + // prepare case + newSeg(channel, test.inSegType, test.inSegID) + + channel.segmentFlushed(test.inSegID) + flushedSeg, ok := channel.segments[test.inSegID] + assert.True(t, ok) + assert.Equal(t, test.inSegID, flushedSeg.segmentID) + assert.Equal(t, datapb.SegmentType_Flushed, flushedSeg.getType()) + } + }) +} + +func TestChannelMeta_InterfaceMethod(t *testing.T) { + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + rc := &RootCoordFactory{ + pkType: schemapb.DataType_Int64, + } + cm := storage.NewLocalChunkManager(storage.RootPath(channelMetaNodeTestDir)) + defer cm.RemoveWithPrefix(ctx, "") + + t.Run("Test addFlushedSegmentWithPKs", func(t *testing.T) { + tests := []struct { + isvalid bool + + incollID UniqueID + channelCollID UniqueID + description string + }{ + {true, 1, 1, "valid input collection"}, + {false, 1, 2, "invalid input collection"}, + } + + primaryKeyData := &storage.Int64FieldData{ + Data: []int64{9}, + } + for _, test := range tests { + t.Run(test.description, func(t *testing.T) { + channel := newChannel("a", test.channelCollID, nil, rc, cm) + if test.isvalid { + channel.addFlushedSegmentWithPKs(100, test.incollID, 10, 1, primaryKeyData) + + assert.True(t, channel.hasSegment(100, true)) + assert.False(t, channel.hasSegment(100, false)) + } else { + channel.addFlushedSegmentWithPKs(100, test.incollID, 10, 1, primaryKeyData) + assert.False(t, channel.hasSegment(100, true)) + assert.False(t, channel.hasSegment(100, false)) + } + }) + } + }) + + t.Run("Test_addNewSegment", func(t *testing.T) { + tests := []struct { + isValidCase bool + channelCollID UniqueID + inCollID UniqueID + inSegID UniqueID + + instartPos *internalpb.MsgPosition + + expectedSegType datapb.SegmentType + + description string + }{ + {isValidCase: false, channelCollID: 1, inCollID: 2, inSegID: 300, description: "input CollID 2 mismatch with channel collID"}, + {true, 1, 1, 200, new(internalpb.MsgPosition), datapb.SegmentType_New, "nill address for startPos"}, + {true, 1, 1, 200, &internalpb.MsgPosition{}, datapb.SegmentType_New, "empty struct for startPos"}, + } + + for _, test := range tests { + t.Run(test.description, func(t *testing.T) { + channel := newChannel("a", test.channelCollID, nil, rc, cm) + require.False(t, channel.hasSegment(test.inSegID, true)) + err := channel.addSegment( + addSegmentReq{ + segType: datapb.SegmentType_New, + segID: test.inSegID, + collID: test.inCollID, + partitionID: 1, + startPos: test.instartPos, + endPos: &internalpb.MsgPosition{}, + }) + if test.isValidCase { + assert.NoError(t, err) + assert.True(t, channel.hasSegment(test.inSegID, true)) + + seg, ok := channel.segments[test.inSegID] + assert.True(t, ok) + assert.Equal(t, test.expectedSegType, seg.getType()) + } else { + assert.Error(t, err) + assert.False(t, channel.hasSegment(test.inSegID, true)) + } + }) + } + }) + + t.Run("Test_addNormalSegment", func(t *testing.T) { + tests := []struct { + isValidCase bool + channelCollID UniqueID + inCollID UniqueID + inSegID UniqueID + + expectedSegType datapb.SegmentType + + description string + }{ + {isValidCase: false, channelCollID: 1, inCollID: 2, inSegID: 300, description: "input CollID 2 mismatch with channel collID"}, + {true, 1, 1, 200, datapb.SegmentType_Normal, "normal case"}, + } + + for _, test := range tests { + t.Run(test.description, func(t *testing.T) { + channel := newChannel("a", test.channelCollID, nil, rc, &mockDataCM{}) + require.False(t, channel.hasSegment(test.inSegID, true)) + err := channel.addSegment( + addSegmentReq{ + segType: datapb.SegmentType_Normal, + segID: test.inSegID, + collID: test.inCollID, + partitionID: 1, + numOfRows: 0, + statsBinLogs: []*datapb.FieldBinlog{getSimpleFieldBinlog()}, + recoverTs: 0, + }) + if test.isValidCase { + assert.NoError(t, err) + assert.True(t, channel.hasSegment(test.inSegID, true)) + + seg, ok := channel.segments[test.inSegID] + assert.True(t, ok) + assert.Equal(t, test.expectedSegType, seg.getType()) + } else { + assert.Error(t, err) + assert.False(t, channel.hasSegment(test.inSegID, true)) + } + }) + } + }) + + t.Run("Test_addNormalSegmentWithNilDml", func(t *testing.T) { + channel := newChannel("a", 1, nil, rc, &mockDataCM{}) + segID := int64(101) + require.False(t, channel.hasSegment(segID, true)) + assert.NotPanics(t, func() { + err := channel.addSegment( + addSegmentReq{ + segType: datapb.SegmentType_Normal, + segID: segID, + collID: 1, + partitionID: 10, + numOfRows: 0, + statsBinLogs: []*datapb.FieldBinlog{}, + recoverTs: 0, + }) + assert.NoError(t, err) + }) + }) + + t.Run("Test_updateSegmentEndPosition", func(t *testing.T) { + segs := []struct { + segID UniqueID + segType datapb.SegmentType + }{ + {100, datapb.SegmentType_New}, + {200, datapb.SegmentType_Normal}, + {300, datapb.SegmentType_Flushed}, + } + + channel := ChannelMeta{segments: make(map[UniqueID]*Segment)} + for _, seg := range segs { + s := Segment{segmentID: seg.segID} + s.setType(seg.segType) + channel.segMu.Lock() + channel.segments[seg.segID] = &s + channel.segMu.Unlock() + } + + tests := []struct { + inSegID UniqueID + description string + }{ + {100, "seg 100 is type New"}, + {200, "seg 200 is type Normal"}, + {300, "seg 300 is type Flushed"}, + } + + for _, test := range tests { + t.Run(test.description, func(t *testing.T) { + channel.updateSegmentEndPosition(test.inSegID, new(internalpb.MsgPosition)) + }) + } + }) + + t.Run("Test_getCollectionSchema", func(t *testing.T) { + tests := []struct { + isValid bool + channelCollID UniqueID + inputCollID UniqueID + + metaServiceErr bool + description string + }{ + {true, 1, 1, false, "Normal case"}, + {false, 1, 2, false, "Input collID 2 mismatch with channel collID 1"}, + {false, 1, 1, true, "RPC call fails"}, + } + + for _, test := range tests { + t.Run(test.description, func(t *testing.T) { + channel := newChannel("a", test.channelCollID, nil, rc, cm) + + if test.metaServiceErr { + channel.collSchema = nil + rc.setCollectionID(-1) + } else { + rc.setCollectionID(1) + } + + s, err := channel.getCollectionSchema(test.inputCollID, Timestamp(0)) + if test.isValid { + assert.NoError(t, err) + assert.NotNil(t, s) + } else { + assert.Error(t, err) + assert.Nil(t, s) + } + }) + } + rc.setCollectionID(1) + }) + + t.Run("Test listAllSegmentIDs", func(t *testing.T) { + s1 := Segment{segmentID: 1} + s2 := Segment{segmentID: 2} + s3 := Segment{segmentID: 3} + + s1.setType(datapb.SegmentType_New) + s2.setType(datapb.SegmentType_Normal) + s3.setType(datapb.SegmentType_Flushed) + channel := &ChannelMeta{ + segments: map[UniqueID]*Segment{ + 1: &s1, + 2: &s2, + 3: &s3, + }, + } + + ids := channel.listAllSegmentIDs() + assert.ElementsMatch(t, []UniqueID{1, 2, 3}, ids) + }) + + t.Run("Test listPartitionSegments", func(t *testing.T) { + channel := &ChannelMeta{segments: make(map[UniqueID]*Segment)} + segs := []struct { + segID UniqueID + partID UniqueID + segType datapb.SegmentType + }{ + {1, 1, datapb.SegmentType_New}, + {2, 1, datapb.SegmentType_Normal}, + {3, 1, datapb.SegmentType_Flushed}, + {4, 2, datapb.SegmentType_New}, + {5, 2, datapb.SegmentType_Normal}, + {6, 2, datapb.SegmentType_Flushed}, + } + + for _, seg := range segs { + s := Segment{ + segmentID: seg.segID, + partitionID: seg.partID, + } + + s.setType(seg.segType) + channel.segments[seg.segID] = &s + } + + ids := channel.listPartitionSegments(1) + assert.ElementsMatch(t, []UniqueID{1, 2, 3}, ids) + }) + + t.Run("Test_addSegmentMinIOLoadError", func(t *testing.T) { + channel := newChannel("a", 1, nil, rc, cm) + channel.chunkManager = &mockDataCMError{} + + err := channel.addSegment( + addSegmentReq{ + segType: datapb.SegmentType_Normal, + segID: 1, + collID: 1, + partitionID: 2, + numOfRows: int64(10), + statsBinLogs: []*datapb.FieldBinlog{getSimpleFieldBinlog()}, + recoverTs: 0, + }) + assert.NotNil(t, err) + err = channel.addSegment( + addSegmentReq{ + segType: datapb.SegmentType_Flushed, + segID: 1, + collID: 1, + partitionID: 2, + numOfRows: int64(0), + statsBinLogs: []*datapb.FieldBinlog{getSimpleFieldBinlog()}, + recoverTs: 0, + }) + assert.NotNil(t, err) + }) + + t.Run("Test_addSegmentStatsError", func(t *testing.T) { + channel := newChannel("insert-01", 1, nil, rc, cm) + channel.chunkManager = &mockDataCMStatsError{} + var err error + + err = channel.addSegment( + addSegmentReq{ + segType: datapb.SegmentType_Normal, + segID: 1, + collID: 1, + partitionID: 2, + numOfRows: int64(10), + statsBinLogs: []*datapb.FieldBinlog{getSimpleFieldBinlog()}, + recoverTs: 0, + }) + assert.NotNil(t, err) + err = channel.addSegment( + addSegmentReq{ + segType: datapb.SegmentType_Flushed, + segID: 1, + collID: 1, + partitionID: 2, + numOfRows: int64(0), + statsBinLogs: []*datapb.FieldBinlog{getSimpleFieldBinlog()}, + recoverTs: 0, + }) + assert.NotNil(t, err) + }) + + t.Run("Test_addSegmentPkfilterError", func(t *testing.T) { + channel := newChannel("insert-01", 1, nil, rc, cm) + channel.chunkManager = &mockPkfilterMergeError{} + var err error + + err = channel.addSegment( + addSegmentReq{ + segType: datapb.SegmentType_Normal, + segID: 1, + collID: 1, + partitionID: 2, + numOfRows: int64(10), + statsBinLogs: []*datapb.FieldBinlog{getSimpleFieldBinlog()}, + recoverTs: 0, + }) + assert.NotNil(t, err) + err = channel.addSegment( + addSegmentReq{ + segType: datapb.SegmentType_Flushed, + segID: 1, + collID: 1, + partitionID: 2, + numOfRows: int64(0), + statsBinLogs: []*datapb.FieldBinlog{getSimpleFieldBinlog()}, + recoverTs: 0, + }) + assert.NotNil(t, err) + }) + + t.Run("Test_mergeFlushedSegments", func(t *testing.T) { + channel := newChannel("channel", 1, nil, rc, cm) + + primaryKeyData := &storage.Int64FieldData{ + Data: []UniqueID{1}, + } + tests := []struct { + description string + isValid bool + stored bool + + inCompactedFrom []UniqueID + inSeg *Segment + }{ + {"mismatch collection", false, false, []UniqueID{1, 2}, &Segment{ + segmentID: 3, + collectionID: -1, + }}, + {"no match flushed segment", false, false, []UniqueID{1, 6}, &Segment{ + segmentID: 3, + collectionID: 1, + }}, + {"numRows==0", true, false, []UniqueID{1, 2}, &Segment{ + segmentID: 3, + collectionID: 1, + numRows: 0, + }}, + {"numRows>0", true, true, []UniqueID{1, 2}, &Segment{ + segmentID: 3, + collectionID: 1, + numRows: 15, + }}, + {"segment exists but not flushed", false, false, []UniqueID{1, 4}, &Segment{ + segmentID: 3, + collectionID: 1, + numRows: 15, + }}, + } + + for _, test := range tests { + t.Run(test.description, func(t *testing.T) { + // prepare segment + if !channel.hasSegment(1, true) { + channel.addFlushedSegmentWithPKs(1, 1, 0, 10, primaryKeyData) + } + + if !channel.hasSegment(2, true) { + channel.addFlushedSegmentWithPKs(2, 1, 0, 10, primaryKeyData) + } + + if !channel.hasSegment(4, false) { + channel.removeSegments(4) + channel.addSegment(addSegmentReq{ + segType: datapb.SegmentType_Normal, + segID: 4, + collID: 1, + partitionID: 0, + }) + } + + if channel.hasSegment(3, true) { + channel.removeSegments(3) + } + + require.True(t, channel.hasSegment(1, true)) + require.True(t, channel.hasSegment(2, true)) + require.True(t, channel.hasSegment(4, false)) + require.False(t, channel.hasSegment(3, true)) + + // tests start + err := channel.mergeFlushedSegments(test.inSeg, 100, test.inCompactedFrom) + if test.isValid { + assert.NoError(t, err) + } else { + assert.Error(t, err) + } + + if test.stored { + assert.True(t, channel.hasSegment(3, true)) + + to2from := channel.listCompactedSegmentIDs() + assert.NotEmpty(t, to2from) + + from, ok := to2from[3] + assert.True(t, ok) + assert.ElementsMatch(t, []UniqueID{1, 2}, from) + } else { + assert.False(t, channel.hasSegment(3, true)) + } + + }) + } + }) + +} +func TestChannelMeta_UpdatePKRange(t *testing.T) { + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + rc := &RootCoordFactory{ + pkType: schemapb.DataType_Int64, + } + collID := UniqueID(1) + partID := UniqueID(2) + chanName := "insert-02" + startPos := &internalpb.MsgPosition{ChannelName: chanName, Timestamp: Timestamp(100)} + endPos := &internalpb.MsgPosition{ChannelName: chanName, Timestamp: Timestamp(200)} + + cm := storage.NewLocalChunkManager(storage.RootPath(channelMetaNodeTestDir)) + defer cm.RemoveWithPrefix(ctx, "") + channel := newChannel("chanName", collID, nil, rc, cm) + channel.chunkManager = &mockDataCM{} + + err := channel.addSegment( + addSegmentReq{ + segType: datapb.SegmentType_New, + segID: 1, + collID: collID, + partitionID: partID, + startPos: startPos, + endPos: endPos, + }) + assert.Nil(t, err) + err = channel.addSegment( + addSegmentReq{ + segType: datapb.SegmentType_Normal, + segID: 2, + collID: collID, + partitionID: partID, + numOfRows: 100, + statsBinLogs: []*datapb.FieldBinlog{getSimpleFieldBinlog()}, + recoverTs: 0, + }) + assert.Nil(t, err) + + segNew := channel.segments[1] + segNormal := channel.segments[2] + + cases := make([]int64, 0, 100) + for i := 0; i < 100; i++ { + cases = append(cases, rand.Int63()) + } + buf := make([]byte, 8) + for _, c := range cases { + channel.updateSegmentPKRange(1, &storage.Int64FieldData{Data: []int64{c}}) // new segment + channel.updateSegmentPKRange(2, &storage.Int64FieldData{Data: []int64{c}}) // normal segment + channel.updateSegmentPKRange(3, &storage.Int64FieldData{Data: []int64{c}}) // non-exist segment + + pk := newInt64PrimaryKey(c) + + assert.Equal(t, true, segNew.minPK.LE(pk)) + assert.Equal(t, true, segNew.maxPK.GE(pk)) + + assert.Equal(t, true, segNormal.minPK.LE(pk)) + assert.Equal(t, true, segNormal.maxPK.GE(pk)) + + common.Endian.PutUint64(buf, uint64(c)) + assert.True(t, segNew.pkFilter.Test(buf)) + assert.True(t, segNormal.pkFilter.Test(buf)) + + } + +} + +// ChannelMetaSuite setup test suite for ChannelMeta +type ChannelMetaSuite struct { + suite.Suite + channel *ChannelMeta + + collID UniqueID + partID UniqueID + vchanName string + cm *storage.LocalChunkManager +} + +func (s *ChannelMetaSuite) SetupSuite() { + rc := &RootCoordFactory{ + pkType: schemapb.DataType_Int64, + } + s.collID = 1 + s.cm = storage.NewLocalChunkManager(storage.RootPath(channelMetaNodeTestDir)) + s.channel = newChannel("channel", s.collID, nil, rc, s.cm) + s.vchanName = "channel" +} + +func (s *ChannelMetaSuite) TearDownSuite() { + s.cm.RemoveWithPrefix(context.Background(), "") +} + +func (s *ChannelMetaSuite) SetupTest() { + var err error + err = s.channel.addSegment(addSegmentReq{ + segType: datapb.SegmentType_New, + segID: 1, + collID: s.collID, + partitionID: s.partID, + startPos: &internalpb.MsgPosition{}, + endPos: nil, + }) + s.Require().NoError(err) + err = s.channel.addSegment(addSegmentReq{ + segType: datapb.SegmentType_Normal, + segID: 2, + collID: s.collID, + partitionID: s.partID, + numOfRows: 10, + statsBinLogs: nil, + recoverTs: 0, + }) + s.Require().NoError(err) + err = s.channel.addSegment(addSegmentReq{ + segType: datapb.SegmentType_Flushed, + segID: 3, + collID: s.collID, + partitionID: s.partID, + numOfRows: 10, + statsBinLogs: nil, + recoverTs: 0, + }) + s.Require().NoError(err) +} + +func (s *ChannelMetaSuite) TearDownTest() { + s.channel.removeSegments(1, 2, 3) +} + +func (s *ChannelMetaSuite) TestHasSegment() { + segs := []struct { + segID UniqueID + sType datapb.SegmentType + }{ + {100, datapb.SegmentType_New}, + {101, datapb.SegmentType_New}, + {200, datapb.SegmentType_Normal}, + {201, datapb.SegmentType_Normal}, + {300, datapb.SegmentType_Flushed}, + {301, datapb.SegmentType_Flushed}, + {400, datapb.SegmentType_Compacted}, + {401, datapb.SegmentType_Compacted}, + } + + channel := &ChannelMeta{ + segments: make(map[UniqueID]*Segment), + } + + for _, seg := range segs { + s := Segment{segmentID: seg.segID} + s.setType(seg.sType) + channel.segments[seg.segID] = &s + } + + tests := []struct { + description string + inSegID UniqueID + countFlushed bool + + expected bool + }{ + {"segment 100 exist/not countFlushed", 100, false, true}, + {"segment 101 exist/countFlushed", 101, true, true}, + {"segment 200 exist/not countFlushed", 200, false, true}, + {"segment 201 exist/countFlushed", 201, true, true}, + {"segment 300 not exist/not countFlushed", 300, false, false}, + {"segment 301 exist/countFlushed", 301, true, true}, + {"segment 400 not exist/not countFlushed", 400, false, false}, + {"segment 401 not exist/countFlushed", 401, true, false}, + {"segment 500 not exist/not countFlushed", 500, false, false}, + {"segment 501 not exist/countFlushed", 501, true, false}, + } + + for _, test := range tests { + s.Run(test.description, func() { + got := channel.hasSegment(test.inSegID, test.countFlushed) + s.Assert().Equal(test.expected, got) + }) + } +} + +func (s *ChannelMetaSuite) TestGetSegmentStatslog() { + bs, err := s.channel.getSegmentStatslog(1) + s.NoError(err) + + segment, ok := s.getSegmentByID(1) + s.Require().True(ok) + expected, err := segment.getSegmentStatslog(106, schemapb.DataType_Int64) + s.Require().NoError(err) + s.Equal(expected, bs) + + bs, err = s.channel.getSegmentStatslog(2) + s.NoError(err) + + segment, ok = s.getSegmentByID(2) + s.Require().True(ok) + expected, err = segment.getSegmentStatslog(106, schemapb.DataType_Int64) + s.Require().NoError(err) + s.Equal(expected, bs) + + bs, err = s.channel.getSegmentStatslog(3) + s.NoError(err) + + segment, ok = s.getSegmentByID(3) + s.Require().True(ok) + expected, err = segment.getSegmentStatslog(106, schemapb.DataType_Int64) + s.Require().NoError(err) + s.Equal(expected, bs) + + _, err = s.channel.getSegmentStatslog(4) + s.Error(err) +} + +func (s *ChannelMetaSuite) getSegmentByID(id UniqueID) (*Segment, bool) { + s.channel.segMu.RLock() + defer s.channel.segMu.RUnlock() + + seg, ok := s.channel.segments[id] + if ok && seg.isValid() { + return seg, true + } + + return nil, false +} + +func TestChannelMetaSuite(t *testing.T) { + suite.Run(t, new(ChannelMetaSuite)) +} diff --git a/internal/datanode/compactor.go b/internal/datanode/compactor.go index 3d9a25b728..0395bb5762 100644 --- a/internal/datanode/compactor.go +++ b/internal/datanode/compactor.go @@ -67,7 +67,7 @@ type compactionTask struct { downloader uploader compactor - Replica + Channel flushManager allocatorInterface @@ -87,7 +87,7 @@ func newCompactionTask( ctx context.Context, dl downloader, ul uploader, - replica Replica, + channel Channel, fm flushManager, alloc allocatorInterface, plan *datapb.CompactionPlan) *compactionTask { @@ -99,7 +99,7 @@ func newCompactionTask( downloader: dl, uploader: ul, - Replica: replica, + Channel: channel, flushManager: fm, allocatorInterface: alloc, plan: plan, @@ -261,7 +261,7 @@ func (t *compactionTask) merge( insertPaths = make([]*datapb.FieldBinlog, 0) ) - t.Replica.initSegmentBloomFilter(segment) + t.Channel.initSegmentBloomFilter(segment) isDeletedValue := func(v *storage.Value) bool { ts, ok := delta[v.PK.GetValue()] @@ -588,6 +588,7 @@ func (t *compactionTask) compact() (*datapb.CompactionResult, error) { log.Info("compaction done", zap.Int64("planID", t.plan.GetPlanID()), zap.Int64("targetSegmentID", targetSegID), + zap.Int64s("compactedFrom", segIDs), zap.Int("num of binlog paths", len(inPaths)), zap.Int("num of stats paths", len(statsPaths)), zap.Int("num of delta paths", len(deltaInfo)), diff --git a/internal/datanode/compactor_test.go b/internal/datanode/compactor_test.go index 2fef4505d7..7bd11b0136 100644 --- a/internal/datanode/compactor_test.go +++ b/internal/datanode/compactor_test.go @@ -50,22 +50,21 @@ func TestCompactionTaskInnerMethods(t *testing.T) { rc := &RootCoordFactory{ pkType: schemapb.DataType_Int64, } - replica, err := newReplica(context.TODO(), rc, cm, 1, nil) - require.NoError(t, err) + channel := newChannel("a", 1, nil, rc, cm) + var err error task := &compactionTask{ - Replica: replica, + Channel: channel, } _, _, _, err = task.getSegmentMeta(100) assert.Error(t, err) - err = replica.addSegment(addSegmentReq{ + err = channel.addSegment(addSegmentReq{ segType: datapb.SegmentType_New, segID: 100, collID: 1, partitionID: 10, - channelName: "a", startPos: new(internalpb.MsgPosition), endPos: nil, }) @@ -78,7 +77,7 @@ func TestCompactionTaskInnerMethods(t *testing.T) { assert.NotNil(t, meta) rc.setCollectionID(-2) - task.Replica.(*SegmentReplica).collSchema = nil + task.Channel.(*ChannelMeta).collSchema = nil _, _, _, err = task.getSegmentMeta(100) assert.Error(t, err) }) @@ -270,8 +269,7 @@ func TestCompactionTaskInnerMethods(t *testing.T) { Return(&milvuspb.DescribeCollectionResponse{ Schema: meta.GetSchema(), }, nil) - replica, err := newReplica(context.Background(), rc, nil, collectionID, meta.GetSchema()) - require.NoError(t, err) + channel := newChannel("a", collectionID, meta.GetSchema(), rc, nil) t.Run("Merge without expiration", func(t *testing.T) { alloc := NewAllocatorFactory(1) mockbIO := &binlogIO{cm, alloc} @@ -297,7 +295,7 @@ func TestCompactionTaskInnerMethods(t *testing.T) { 1: 10000, } - ct := &compactionTask{Replica: replica, downloader: mockbIO, uploader: mockbIO} + ct := &compactionTask{Channel: channel, downloader: mockbIO, uploader: mockbIO} inPaths, statsPaths, _, numOfRow, err := ct.merge(context.Background(), allPaths, 2, 0, meta, dm) assert.NoError(t, err) assert.Equal(t, int64(2), numOfRow) @@ -333,7 +331,7 @@ func TestCompactionTaskInnerMethods(t *testing.T) { dm := map[interface{}]Timestamp{} - ct := &compactionTask{Replica: replica, downloader: mockbIO, uploader: mockbIO} + ct := &compactionTask{Channel: channel, downloader: mockbIO, uploader: mockbIO} inPaths, statsPaths, _, numOfRow, err := ct.merge(context.Background(), allPaths, 2, 0, meta, dm) assert.NoError(t, err) assert.Equal(t, int64(2), numOfRow) @@ -370,7 +368,7 @@ func TestCompactionTaskInnerMethods(t *testing.T) { // 10 days in seconds ct := &compactionTask{ - Replica: replica, + Channel: channel, downloader: mockbIO, uploader: mockbIO, plan: &datapb.CompactionPlan{ @@ -410,7 +408,7 @@ func TestCompactionTaskInnerMethods(t *testing.T) { 1: 10000, } - ct := &compactionTask{Replica: replica, downloader: mockbIO, uploader: mockbIO} + ct := &compactionTask{Channel: channel, downloader: mockbIO, uploader: mockbIO} _, _, _, _, err = ct.merge(context.Background(), allPaths, 2, 0, &etcdpb.CollectionMeta{ Schema: &schemapb.CollectionSchema{Fields: []*schemapb.FieldSchema{ {DataType: schemapb.DataType_FloatVector, TypeParams: []*commonpb.KeyValuePair{ @@ -447,7 +445,7 @@ func TestCompactionTaskInnerMethods(t *testing.T) { 1: 10000, } - ct := &compactionTask{Replica: replica, downloader: mockbIO, uploader: mockbIO} + ct := &compactionTask{Channel: channel, downloader: mockbIO, uploader: mockbIO} _, _, _, _, err = ct.merge(context.Background(), allPaths, 2, 0, &etcdpb.CollectionMeta{ Schema: &schemapb.CollectionSchema{Fields: []*schemapb.FieldSchema{ @@ -640,13 +638,12 @@ func TestCompactorInterfaceMethods(t *testing.T) { mockfm := &mockFlushManager{} mockKv := memkv.NewMemoryKV() mockbIO := &binlogIO{cm, alloc} - replica, err := newReplica(context.TODO(), rc, cm, c.colID, nil) - require.NoError(t, err) + channel := newChannel("a", c.colID, nil, rc, cm) - replica.addFlushedSegmentWithPKs(c.segID1, c.colID, c.parID, "channelname", 2, c.iData1) - replica.addFlushedSegmentWithPKs(c.segID2, c.colID, c.parID, "channelname", 2, c.iData2) - require.True(t, replica.hasSegment(c.segID1, true)) - require.True(t, replica.hasSegment(c.segID2, true)) + channel.addFlushedSegmentWithPKs(c.segID1, c.colID, c.parID, 2, c.iData1) + channel.addFlushedSegmentWithPKs(c.segID2, c.colID, c.parID, 2, c.iData2) + require.True(t, channel.hasSegment(c.segID1, true)) + require.True(t, channel.hasSegment(c.segID2, true)) meta := NewMetaFactory().GetCollectionMeta(c.colID, "test_compact_coll_name", c.pkType) iData1 := genInsertDataWithPKs(c.pks1, c.pkType) @@ -694,7 +691,7 @@ func TestCompactorInterfaceMethods(t *testing.T) { } alloc.random = false // generated ID = 19530 - task := newCompactionTask(context.TODO(), mockbIO, mockbIO, replica, mockfm, alloc, plan) + task := newCompactionTask(context.TODO(), mockbIO, mockbIO, channel, mockfm, alloc, plan) result, err := task.compact() assert.NoError(t, err) assert.NotNil(t, result) @@ -712,12 +709,12 @@ func TestCompactorInterfaceMethods(t *testing.T) { plan.PlanID++ plan.Timetravel = Timestamp(25000) - replica.addFlushedSegmentWithPKs(c.segID1, c.colID, c.parID, "channelname", 2, c.iData1) - replica.addFlushedSegmentWithPKs(c.segID2, c.colID, c.parID, "channelname", 2, c.iData2) - replica.removeSegments(19530) - require.True(t, replica.hasSegment(c.segID1, true)) - require.True(t, replica.hasSegment(c.segID2, true)) - require.False(t, replica.hasSegment(19530, true)) + channel.addFlushedSegmentWithPKs(c.segID1, c.colID, c.parID, 2, c.iData1) + channel.addFlushedSegmentWithPKs(c.segID2, c.colID, c.parID, 2, c.iData2) + channel.removeSegments(19530) + require.True(t, channel.hasSegment(c.segID1, true)) + require.True(t, channel.hasSegment(c.segID2, true)) + require.False(t, channel.hasSegment(19530, true)) result, err = task.compact() assert.NoError(t, err) @@ -736,12 +733,12 @@ func TestCompactorInterfaceMethods(t *testing.T) { plan.PlanID++ plan.Timetravel = Timestamp(10000) - replica.addFlushedSegmentWithPKs(c.segID1, c.colID, c.parID, "channelname", 2, c.iData1) - replica.addFlushedSegmentWithPKs(c.segID2, c.colID, c.parID, "channelname", 2, c.iData2) - replica.removeSegments(19530) - require.True(t, replica.hasSegment(c.segID1, true)) - require.True(t, replica.hasSegment(c.segID2, true)) - require.False(t, replica.hasSegment(19530, true)) + channel.addFlushedSegmentWithPKs(c.segID1, c.colID, c.parID, 2, c.iData1) + channel.addFlushedSegmentWithPKs(c.segID2, c.colID, c.parID, 2, c.iData2) + channel.removeSegments(19530) + require.True(t, channel.hasSegment(c.segID1, true)) + require.True(t, channel.hasSegment(c.segID2, true)) + require.False(t, channel.hasSegment(19530, true)) result, err = task.compact() assert.NoError(t, err) @@ -767,13 +764,12 @@ func TestCompactorInterfaceMethods(t *testing.T) { } mockfm := &mockFlushManager{} mockbIO := &binlogIO{cm, alloc} - replica, err := newReplica(context.TODO(), rc, cm, collID, nil) - require.NoError(t, err) + channel := newChannel("channelname", collID, nil, rc, cm) - replica.addFlushedSegmentWithPKs(segID1, collID, partID, "channelname", 2, &storage.Int64FieldData{Data: []UniqueID{1}}) - replica.addFlushedSegmentWithPKs(segID2, collID, partID, "channelname", 2, &storage.Int64FieldData{Data: []UniqueID{1}}) - require.True(t, replica.hasSegment(segID1, true)) - require.True(t, replica.hasSegment(segID2, true)) + channel.addFlushedSegmentWithPKs(segID1, collID, partID, 2, &storage.Int64FieldData{Data: []UniqueID{1}}) + channel.addFlushedSegmentWithPKs(segID2, collID, partID, 2, &storage.Int64FieldData{Data: []UniqueID{1}}) + require.True(t, channel.hasSegment(segID1, true)) + require.True(t, channel.hasSegment(segID2, true)) meta := NewMetaFactory().GetCollectionMeta(collID, "test_compact_coll_name", schemapb.DataType_Int64) // the same pk for segmentI and segmentII @@ -826,7 +822,7 @@ func TestCompactorInterfaceMethods(t *testing.T) { } alloc.random = false // generated ID = 19530 - task := newCompactionTask(context.TODO(), mockbIO, mockbIO, replica, mockfm, alloc, plan) + task := newCompactionTask(context.TODO(), mockbIO, mockbIO, channel, mockfm, alloc, plan) result, err := task.compact() assert.NoError(t, err) assert.NotNil(t, result) diff --git a/internal/datanode/data_node.go b/internal/datanode/data_node.go index b50ce816cb..3509273db0 100644 --- a/internal/datanode/data_node.go +++ b/internal/datanode/data_node.go @@ -834,7 +834,7 @@ func (node *DataNode) Compaction(ctx context.Context, req *datapb.CompactionPlan task := newCompactionTask( node.ctx, binlogIO, binlogIO, - ds.replica, + ds.channel, ds.flushManager, ds.idAllocator, req, @@ -904,7 +904,7 @@ func (node *DataNode) SyncSegments(ctx context.Context, req *datapb.SyncSegments } oneSegment := req.GetCompactedFrom()[0] - replica, err := node.flowgraphManager.getReplica(oneSegment) + channel, err := node.flowgraphManager.getChannel(oneSegment) if err != nil { status.Reason = fmt.Sprintf("invalid request, err=%s", err.Error()) return status, nil @@ -913,29 +913,27 @@ func (node *DataNode) SyncSegments(ctx context.Context, req *datapb.SyncSegments // check if all compactedFrom segments are valid var invalidSegIDs []UniqueID for _, segID := range req.GetCompactedFrom() { - if !replica.hasSegment(segID, true) { + if !channel.hasSegment(segID, true) { invalidSegIDs = append(invalidSegIDs, segID) } } if len(invalidSegIDs) > 0 { - status.Reason = fmt.Sprintf("invalid request, some segments are not in the same replica: %v", invalidSegIDs) + status.Reason = fmt.Sprintf("invalid request, some segments are not in the same channel: %v", invalidSegIDs) return status, nil } - // oneSegment is definitely in the replica, guaranteed by the check before. - collID, partID, _ := replica.getCollectionAndPartitionID(oneSegment) - chanName, _ := replica.getChannelName(oneSegment) + // oneSegment is definitely in the channel, guaranteed by the check before. + collID, partID, _ := channel.getCollectionAndPartitionID(oneSegment) targetSeg := &Segment{ collectionID: collID, partitionID: partID, - channelName: chanName, segmentID: req.GetCompactedTo(), numRows: req.GetNumOfRows(), } - replica.(*SegmentReplica).initPKBloomFilter(ctx, targetSeg, req.GetStatsLogs(), tsoutil.GetCurrentTime()) + channel.(*ChannelMeta).initPKBloomFilter(ctx, targetSeg, req.GetStatsLogs(), tsoutil.GetCurrentTime()) - if err := replica.mergeFlushedSegments(targetSeg, req.GetPlanID(), req.GetCompactedFrom()); err != nil { + if err := channel.mergeFlushedSegments(targetSeg, req.GetPlanID(), req.GetCompactedFrom()); err != nil { status.Reason = err.Error() return status, nil } @@ -1116,19 +1114,18 @@ func (node *DataNode) AddImportSegment(ctx context.Context, req *datapb.AddImpor }, }, nil } - // Add the new segment to the replica. - if !ds.replica.hasSegment(req.GetSegmentId(), true) { - log.Info("adding a new segment to replica", + // Add the new segment to the channel. + if !ds.channel.hasSegment(req.GetSegmentId(), true) { + log.Info("adding a new segment to channel", zap.Int64("segment ID", req.GetSegmentId())) // Add segment as a flushed segment, but set `importing` to true to add extra information of the segment. // By 'extra information' we mean segment info while adding a `SegmentType_Flushed` typed segment. - if err := ds.replica.addSegment( + if err := ds.channel.addSegment( addSegmentReq{ segType: datapb.SegmentType_Flushed, segID: req.GetSegmentId(), collID: req.GetCollectionId(), partitionID: req.GetPartitionId(), - channelName: req.GetChannelName(), numOfRows: req.GetRowNum(), statsBinLogs: req.GetStatsLog(), startPos: &internalpb.MsgPosition{ diff --git a/internal/datanode/data_node_test.go b/internal/datanode/data_node_test.go index 532672aede..2c112850aa 100644 --- a/internal/datanode/data_node_test.go +++ b/internal/datanode/data_node_test.go @@ -217,12 +217,11 @@ func TestDataNode(t *testing.T) { fgservice, ok := node1.flowgraphManager.getFlowgraphService(dmChannelName) assert.True(t, ok) - err = fgservice.replica.addSegment(addSegmentReq{ + err = fgservice.channel.addSegment(addSegmentReq{ segType: datapb.SegmentType_New, segID: 0, collID: 1, partitionID: 1, - channelName: dmChannelName, startPos: &internalpb.MsgPosition{}, endPos: &internalpb.MsgPosition{}, }) @@ -681,16 +680,22 @@ func TestDataNode(t *testing.T) { fg, ok := node.flowgraphManager.getFlowgraphService(chanName) assert.True(t, ok) - fg.replica.(*SegmentReplica).flushedSegments = map[UniqueID]*Segment{ - 100: {channelName: chanName}, - 101: {channelName: chanName}, - 102: {channelName: chanName}, + s1 := Segment{segmentID: 100} + s2 := Segment{segmentID: 200} + s3 := Segment{segmentID: 300} + s1.setType(datapb.SegmentType_Flushed) + s2.setType(datapb.SegmentType_Flushed) + s3.setType(datapb.SegmentType_Flushed) + fg.channel.(*ChannelMeta).segments = map[UniqueID]*Segment{ + s1.segmentID: &s1, + s2.segmentID: &s2, + s3.segmentID: &s3, } t.Run("invalid compacted from", func(t *testing.T) { invalidCompactedFroms := [][]UniqueID{ {}, - {100, 200}, + {101, 201}, } req := &datapb.SyncSegmentsRequest{} @@ -704,38 +709,42 @@ func TestDataNode(t *testing.T) { t.Run("valid request numRows>0", func(t *testing.T) { req := &datapb.SyncSegmentsRequest{ - CompactedFrom: []int64{100, 101}, - CompactedTo: 200, + CompactedFrom: []int64{100, 200}, + CompactedTo: 101, NumOfRows: 100, } status, err := node.SyncSegments(ctx, req) assert.NoError(t, err) assert.Equal(t, commonpb.ErrorCode_Success, status.GetErrorCode()) - assert.True(t, fg.replica.hasSegment(req.CompactedTo, true)) - assert.False(t, fg.replica.hasSegment(req.CompactedFrom[0], true)) - assert.False(t, fg.replica.hasSegment(req.CompactedFrom[1], true)) + assert.True(t, fg.channel.hasSegment(req.CompactedTo, true)) + assert.False(t, fg.channel.hasSegment(req.CompactedFrom[0], true)) + assert.False(t, fg.channel.hasSegment(req.CompactedFrom[1], true)) }) t.Run("valid request numRows=0", func(t *testing.T) { - fg.replica.(*SegmentReplica).flushedSegments = map[UniqueID]*Segment{ - 100: {channelName: chanName}, - 101: {channelName: chanName}, - 102: {channelName: chanName}, + s1.setType(datapb.SegmentType_Flushed) + s2.setType(datapb.SegmentType_Flushed) + s3.setType(datapb.SegmentType_Flushed) + + fg.channel.(*ChannelMeta).segments = map[UniqueID]*Segment{ + s1.segmentID: &s1, + s2.segmentID: &s2, + s3.segmentID: &s3, } req := &datapb.SyncSegmentsRequest{ - CompactedFrom: []int64{100, 101}, - CompactedTo: 200, + CompactedFrom: []int64{s1.segmentID, s2.segmentID}, + CompactedTo: 101, NumOfRows: 0, } status, err := node.SyncSegments(ctx, req) assert.NoError(t, err) assert.Equal(t, commonpb.ErrorCode_Success, status.GetErrorCode()) - assert.False(t, fg.replica.hasSegment(req.CompactedTo, true)) - assert.False(t, fg.replica.hasSegment(req.CompactedFrom[0], true)) - assert.False(t, fg.replica.hasSegment(req.CompactedFrom[1], true)) + assert.False(t, fg.channel.hasSegment(req.CompactedTo, true)) + assert.False(t, fg.channel.hasSegment(req.CompactedFrom[0], true)) + assert.False(t, fg.channel.hasSegment(req.CompactedFrom[1], true)) }) }) } @@ -1114,32 +1123,29 @@ func TestDataNode_ResendSegmentStats(t *testing.T) { fgService, ok := node.flowgraphManager.getFlowgraphService(dmChannelName) assert.True(t, ok) - err = fgService.replica.addSegment(addSegmentReq{ + err = fgService.channel.addSegment(addSegmentReq{ segType: datapb.SegmentType_New, segID: 0, collID: 1, partitionID: 1, - channelName: dmChannelName, startPos: &internalpb.MsgPosition{}, endPos: &internalpb.MsgPosition{}, }) assert.Nil(t, err) - err = fgService.replica.addSegment(addSegmentReq{ + err = fgService.channel.addSegment(addSegmentReq{ segType: datapb.SegmentType_New, segID: 1, collID: 1, partitionID: 2, - channelName: dmChannelName, startPos: &internalpb.MsgPosition{}, endPos: &internalpb.MsgPosition{}, }) assert.Nil(t, err) - err = fgService.replica.addSegment(addSegmentReq{ + err = fgService.channel.addSegment(addSegmentReq{ segType: datapb.SegmentType_New, segID: 2, collID: 1, partitionID: 3, - channelName: dmChannelName, startPos: &internalpb.MsgPosition{}, endPos: &internalpb.MsgPosition{}, }) diff --git a/internal/datanode/data_sync_service.go b/internal/datanode/data_sync_service.go index 4a5d929bfb..c7cdcc7024 100644 --- a/internal/datanode/data_sync_service.go +++ b/internal/datanode/data_sync_service.go @@ -42,7 +42,7 @@ type dataSyncService struct { fg *flowgraph.TimeTickedFlowGraph // internal flowgraph processes insert/delta messages flushCh chan flushMsg // chan to notify flush resendTTCh chan resendTTMsg // chan to ask for resending DataNode time tick message. - replica Replica // segment replica stores meta + channel Channel // channel stores meta of channel idAllocator allocatorInterface // id/timestamp allocator msFactory msgstream.Factory collectionID UniqueID // collection id of vchan for which this data sync service serves @@ -62,7 +62,7 @@ type dataSyncService struct { func newDataSyncService(ctx context.Context, flushCh chan flushMsg, resendTTCh chan resendTTMsg, - replica Replica, + channel Channel, alloc allocatorInterface, factory msgstream.Factory, vchan *datapb.VchannelInfo, @@ -73,7 +73,7 @@ func newDataSyncService(ctx context.Context, compactor *compactionExecutor, ) (*dataSyncService, error) { - if replica == nil { + if channel == nil { return nil, errors.New("Nil input") } @@ -93,7 +93,7 @@ func newDataSyncService(ctx context.Context, fg: nil, flushCh: flushCh, resendTTCh: resendTTCh, - replica: replica, + channel: channel, idAllocator: alloc, msFactory: factory, collectionID: vchan.GetCollectionID(), @@ -121,7 +121,7 @@ type nodeConfig struct { msFactory msgstream.Factory // msgStream factory collectionID UniqueID vChannelName string - replica Replica // Segment replica + channel Channel // Channel info allocator allocatorInterface // defaults @@ -160,7 +160,7 @@ func (dsService *dataSyncService) close() { } func (dsService *dataSyncService) clearGlobalFlushingCache() { - segments := dsService.replica.listAllSegmentIDs() + segments := dsService.channel.listAllSegmentIDs() dsService.flushingSegCache.Remove(segments...) } @@ -168,7 +168,7 @@ func (dsService *dataSyncService) clearGlobalFlushingCache() { func (dsService *dataSyncService) initNodes(vchanInfo *datapb.VchannelInfo) error { dsService.fg = flowgraph.NewTimeTickedFlowGraph(dsService.ctx) // initialize flush manager for DataSync Service - dsService.flushManager = NewRendezvousFlushManager(dsService.idAllocator, dsService.chunkManager, dsService.replica, + dsService.flushManager = NewRendezvousFlushManager(dsService.idAllocator, dsService.chunkManager, dsService.channel, flushNotifyFunc(dsService), dropVirtualChannelFunc(dsService)) var err error @@ -201,25 +201,18 @@ func (dsService *dataSyncService) initNodes(vchanInfo *datapb.VchannelInfo) erro zap.Int64("segmentID", us.GetID()), zap.Int64("numRows", us.GetNumOfRows()), ) - var cp *segmentCheckPoint - if us.GetDmlPosition() != nil { - cp = &segmentCheckPoint{ - numRows: us.GetNumOfRows(), - pos: *us.GetDmlPosition(), - } - } + // avoid closure capture iteration variable segment := us future := dsService.ioPool.Submit(func() (interface{}, error) { - if err := dsService.replica.addSegment(addSegmentReq{ + if err := dsService.channel.addSegment(addSegmentReq{ segType: datapb.SegmentType_Normal, segID: segment.GetID(), collID: segment.CollectionID, partitionID: segment.PartitionID, - channelName: segment.GetInsertChannel(), numOfRows: segment.GetNumOfRows(), statsBinLogs: segment.Statslogs, - cp: cp, + endPos: segment.GetDmlPosition(), recoverTs: vchanInfo.GetSeekPosition().GetTimestamp()}); err != nil { return nil, err } @@ -247,12 +240,11 @@ func (dsService *dataSyncService) initNodes(vchanInfo *datapb.VchannelInfo) erro // avoid closure capture iteration variable segment := fs future := dsService.ioPool.Submit(func() (interface{}, error) { - if err := dsService.replica.addSegment(addSegmentReq{ + if err := dsService.channel.addSegment(addSegmentReq{ segType: datapb.SegmentType_Flushed, segID: segment.GetID(), collID: segment.CollectionID, partitionID: segment.PartitionID, - channelName: segment.GetInsertChannel(), numOfRows: segment.GetNumOfRows(), statsBinLogs: segment.Statslogs, recoverTs: vchanInfo.GetSeekPosition().GetTimestamp(), @@ -273,7 +265,7 @@ func (dsService *dataSyncService) initNodes(vchanInfo *datapb.VchannelInfo) erro msFactory: dsService.msFactory, collectionID: vchanInfo.GetCollectionID(), vChannelName: vchanInfo.GetChannelName(), - replica: dsService.replica, + channel: dsService.channel, allocator: dsService.idAllocator, parallelConfig: newParallelConfig(), diff --git a/internal/datanode/data_sync_service_test.go b/internal/datanode/data_sync_service_test.go index 073a9cb6ad..fa5019ffcd 100644 --- a/internal/datanode/data_sync_service_test.go +++ b/internal/datanode/data_sync_service_test.go @@ -25,7 +25,6 @@ import ( "time" "github.com/stretchr/testify/assert" - "github.com/stretchr/testify/require" "go.uber.org/zap" "github.com/milvus-io/milvus-proto/go-api/commonpb" @@ -85,7 +84,7 @@ func getVchanInfo(info *testInfo) *datapb.VchannelInfo { type testInfo struct { isValidCase bool - replicaNil bool + channelNil bool inMsgFactory msgstream.Factory collID UniqueID @@ -138,7 +137,7 @@ func TestDataSyncService_newDataSyncService(te *testing.T) { 0, "by-dev-rootcoord-dml-test_v0", 0, 0, "", 0, 0, 0, "", 0, - "replica nil"}, + "channel nil"}, {true, false, &mockMsgStreamFactory{true, true}, 1, "by-dev-rootcoord-dml-test_v1", 1, 1, "by-dev-rootcoord-dml-test_v1", 0, @@ -151,17 +150,17 @@ func TestDataSyncService_newDataSyncService(te *testing.T) { for _, test := range tests { te.Run(test.description, func(t *testing.T) { df := &DataCoordFactory{} + rc := &RootCoordFactory{pkType: schemapb.DataType_Int64} - replica, err := newReplica(context.Background(), &RootCoordFactory{pkType: schemapb.DataType_Int64}, cm, test.collID, nil) - assert.Nil(t, err) - if test.replicaNil { - replica = nil + channel := newChannel("channel", test.collID, nil, rc, cm) + if test.channelNil { + channel = nil } ds, err := newDataSyncService(ctx, make(chan flushMsg), make(chan resendTTMsg), - replica, + channel, NewAllocatorFactory(), test.inMsgFactory, getVchanInfo(test), @@ -199,6 +198,9 @@ func TestDataSyncService_Start(t *testing.T) { // init data node + insertChannelName := "data_sync_service_test_dml" + ddlChannelName := "data_sync_service_test_ddl" + Factory := &MetaFactory{} collMeta := Factory.GetCollectionMeta(UniqueID(0), "coll1", schemapb.DataType_Int64) mockRootCoord := &RootCoordFactory{ @@ -210,14 +212,11 @@ func TestDataSyncService_Start(t *testing.T) { resendTTChan := make(chan resendTTMsg, 100) cm := storage.NewLocalChunkManager(storage.RootPath(dataSyncServiceTestDir)) defer cm.RemoveWithPrefix(ctx, "") - replica, err := newReplica(context.Background(), mockRootCoord, cm, collectionID, collMeta.GetSchema()) - assert.Nil(t, err) + channel := newChannel(insertChannelName, collectionID, collMeta.GetSchema(), mockRootCoord, cm) allocFactory := NewAllocatorFactory(1) factory := dependency.NewDefaultFactory(true) - insertChannelName := "data_sync_service_test_dml" - ddlChannelName := "data_sync_service_test_ddl" Params.DataNodeCfg.FlushInsertBufferSize = 1 ufs := []*datapb.SegmentInfo{{ @@ -251,10 +250,10 @@ func TestDataSyncService_Start(t *testing.T) { } signalCh := make(chan string, 100) - sync, err := newDataSyncService(ctx, flushChan, resendTTChan, replica, allocFactory, factory, vchan, signalCh, &DataCoordFactory{}, newCache(), cm, newCompactionExecutor()) + sync, err := newDataSyncService(ctx, flushChan, resendTTChan, channel, allocFactory, factory, vchan, signalCh, &DataCoordFactory{}, newCache(), cm, newCompactionExecutor()) assert.Nil(t, err) - // sync.replica.addCollection(collMeta.ID, collMeta.Schema) + // sync.channel.addCollection(collMeta.ID, collMeta.Schema) sync.start() timeRange := TimeRange{ @@ -416,50 +415,46 @@ func TestClearGlobalFlushingCache(t *testing.T) { dataCoord := &DataCoordFactory{} cm := storage.NewLocalChunkManager(storage.RootPath(dataSyncServiceTestDir)) defer cm.RemoveWithPrefix(ctx, "") - replica, err := newReplica(context.Background(), &RootCoordFactory{pkType: schemapb.DataType_Int64}, cm, 1, nil) - require.NoError(t, err) + channel := newChannel("channel", 1, nil, &RootCoordFactory{pkType: schemapb.DataType_Int64}, cm) + var err error cache := newCache() dsService := &dataSyncService{ dataCoord: dataCoord, - replica: replica, + channel: channel, flushingSegCache: cache, } - err = replica.addSegment( + err = channel.addSegment( addSegmentReq{ segType: datapb.SegmentType_New, segID: 1, collID: 1, partitionID: 1, - channelName: "", startPos: &internalpb.MsgPosition{}, endPos: &internalpb.MsgPosition{}}) assert.NoError(t, err) - err = replica.addSegment( + err = channel.addSegment( addSegmentReq{ segType: datapb.SegmentType_Flushed, segID: 2, collID: 1, partitionID: 1, - channelName: "", numOfRows: 0, statsBinLogs: nil, recoverTs: 0, }) assert.NoError(t, err) - err = replica.addSegment( + err = channel.addSegment( addSegmentReq{ segType: datapb.SegmentType_Normal, segID: 3, collID: 1, partitionID: 1, - channelName: "", numOfRows: 0, statsBinLogs: nil, - cp: nil, recoverTs: 0, }) assert.NoError(t, err) diff --git a/internal/datanode/flow_graph_delete_node.go b/internal/datanode/flow_graph_delete_node.go index bb2891230f..dde049a6b7 100644 --- a/internal/datanode/flow_graph_delete_node.go +++ b/internal/datanode/flow_graph_delete_node.go @@ -49,7 +49,7 @@ type deleteNode struct { ctx context.Context channelName string delBuf sync.Map // map[segmentID]*DelDataBuf - replica Replica + channel Channel idAllocator allocatorInterface flushManager flushManager @@ -209,15 +209,15 @@ func (dn *deleteNode) Operate(in []Msg) []Msg { // update delBuf for compacted segments func (dn *deleteNode) updateCompactedSegments() { - compactedTo2From := dn.replica.listCompactedSegmentIDs() + compactedTo2From := dn.channel.listCompactedSegmentIDs() for compactedTo, compactedFrom := range compactedTo2From { // if the compactedTo segment has 0 numRows, remove all segments related - if !dn.replica.hasSegment(compactedTo, true) { + if !dn.channel.hasSegment(compactedTo, true) { for _, segID := range compactedFrom { dn.delBuf.Delete(segID) } - dn.replica.removeSegments(compactedFrom...) + dn.channel.removeSegments(compactedFrom...) continue } @@ -243,7 +243,7 @@ func (dn *deleteNode) updateCompactedSegments() { zap.Int64("compactedTo segmentID", compactedTo), zap.Int64s("compactedFrom segmentIDs", compactedFrom), ) - dn.replica.removeSegments(compactedFrom...) + dn.channel.removeSegments(compactedFrom...) } } @@ -301,7 +301,7 @@ func (dn *deleteNode) filterSegmentByPK(partID UniqueID, pks []primaryKey, tss [ segID2Pks := make(map[UniqueID][]primaryKey) segID2Tss := make(map[UniqueID][]uint64) buf := make([]byte, 8) - segments := dn.replica.filterSegments(dn.channelName, partID) + segments := dn.channel.filterSegments(partID) for index, pk := range pks { for _, segment := range segments { segmentID := segment.segmentID @@ -337,7 +337,7 @@ func newDeleteNode(ctx context.Context, fm flushManager, sig chan<- string, conf BaseNode: baseNode, delBuf: sync.Map{}, - replica: config.replica, + channel: config.channel, idAllocator: config.allocator, channelName: config.vChannelName, flushManager: fm, diff --git a/internal/datanode/flow_graph_delete_node_test.go b/internal/datanode/flow_graph_delete_node_test.go index efaa3edaf9..a8eb05d590 100644 --- a/internal/datanode/flow_graph_delete_node_test.go +++ b/internal/datanode/flow_graph_delete_node_test.go @@ -18,7 +18,6 @@ package datanode import ( "context" - "errors" "testing" "time" @@ -29,6 +28,7 @@ import ( "github.com/milvus-io/milvus-proto/go-api/schemapb" "github.com/milvus-io/milvus/internal/common" "github.com/milvus-io/milvus/internal/mq/msgstream" + "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/internal/storage" "github.com/milvus-io/milvus/internal/util/flowgraph" "github.com/milvus-io/milvus/internal/util/retry" @@ -36,80 +36,6 @@ import ( var deleteNodeTestDir = "/tmp/milvus_test/deleteNode" -type mockReplica struct { - Replica - - newSegments map[UniqueID]*Segment - normalSegments map[UniqueID]*Segment - flushedSegments map[UniqueID]*Segment - compactedSegments map[UniqueID]*Segment -} - -var _ Replica = (*mockReplica)(nil) - -func newMockReplica() *mockReplica { - return &mockReplica{ - newSegments: make(map[int64]*Segment), - normalSegments: make(map[int64]*Segment), - flushedSegments: make(map[int64]*Segment), - compactedSegments: make(map[int64]*Segment), - } -} - -func (replica *mockReplica) listCompactedSegmentIDs() map[UniqueID][]UniqueID { - return make(map[UniqueID][]UniqueID) -} - -func (replica *mockReplica) removeSegments(segIDs ...UniqueID) {} - -func (replica *mockReplica) filterSegments(channelName string, partitionID UniqueID) []*Segment { - results := make([]*Segment, 0) - for _, value := range replica.newSegments { - results = append(results, value) - } - for _, value := range replica.normalSegments { - results = append(results, value) - } - for _, value := range replica.flushedSegments { - results = append(results, value) - } - return results -} - -func (replica *mockReplica) getCollectionID() UniqueID { - return 0 -} - -func (replica *mockReplica) getCollectionSchema(collectionID UniqueID, ts Timestamp) (*schemapb.CollectionSchema, error) { - if ts == 0 { - return nil, errors.New("mocked error") - } - return &schemapb.CollectionSchema{}, nil -} - -func (replica *mockReplica) getCollectionAndPartitionID(segID UniqueID) (collID, partitionID UniqueID, err error) { - if segID == -1 { - return -1, -1, errors.New("mocked error") - } - return 0, 1, nil -} - -func (replica *mockReplica) hasSegment(segID UniqueID, countFlushed bool) bool { - _, has := replica.newSegments[segID] - if has { - return true - } - _, has = replica.normalSegments[segID] - if has { - return true - } - if !countFlushed { - return false - } - _, has = replica.flushedSegments[segID] - return has -} - func TestFlowGraphDeleteNode_newDeleteNode(te *testing.T) { tests := []struct { ctx context.Context @@ -117,7 +43,7 @@ func TestFlowGraphDeleteNode_newDeleteNode(te *testing.T) { description string }{ - {context.Background(), &nodeConfig{}, "pointer of SegmentReplica"}, + {context.Background(), &nodeConfig{}, "pointer of channel"}, } for _, test := range tests { @@ -132,7 +58,7 @@ func TestFlowGraphDeleteNode_newDeleteNode(te *testing.T) { } } -func genMockReplica(segIDs []int64, pks []primaryKey, chanName string) *mockReplica { +func genMockChannel(segIDs []int64, pks []primaryKey, chanName string) *ChannelMeta { buf := make([]byte, 8) filter0 := bloom.NewWithEstimates(1000000, 0.01) for i := 0; i < 3; i++ { @@ -156,46 +82,33 @@ func genMockReplica(segIDs []int64, pks []primaryKey, chanName string) *mockRepl } } - seg0 := &Segment{ - segmentID: segIDs[0], - channelName: chanName, - pkFilter: filter0, - } - seg1 := &Segment{ - segmentID: segIDs[1], - channelName: chanName, - pkFilter: filter0, - } - seg2 := &Segment{ - segmentID: segIDs[2], - channelName: chanName, - pkFilter: filter0, - } - seg3 := &Segment{ - segmentID: segIDs[3], - channelName: chanName, - pkFilter: filter1, - } - seg4 := &Segment{ - segmentID: segIDs[4], - channelName: chanName, - pkFilter: filter1, - } - seg5 := &Segment{ - segmentID: segIDs[4], - channelName: "test_error", - pkFilter: filter1, + segTypes := []datapb.SegmentType{ + datapb.SegmentType_New, + datapb.SegmentType_New, + datapb.SegmentType_Normal, + datapb.SegmentType_Normal, + datapb.SegmentType_Flushed, + datapb.SegmentType_Flushed, } - replica := newMockReplica() - replica.newSegments[segIDs[0]] = seg0 - replica.newSegments[segIDs[1]] = seg1 - replica.normalSegments[segIDs[2]] = seg2 - replica.normalSegments[segIDs[3]] = seg3 - replica.flushedSegments[segIDs[4]] = seg4 - replica.flushedSegments[segIDs[4]] = seg5 + channel := &ChannelMeta{ + channelName: chanName, + segments: make(map[UniqueID]*Segment), + } + for i := range segIDs { + seg := Segment{ + segmentID: segIDs[i], + } + seg.setType(segTypes[i]) + if i < 3 { + seg.pkFilter = filter0 + } else { + seg.pkFilter = filter1 + } + channel.segments[segIDs[i]] = &seg + } - return replica + return channel } func TestFlowGraphDeleteNode_Operate(t *testing.T) { @@ -247,10 +160,10 @@ func TestFlowGraphDeleteNode_Operate(t *testing.T) { defer cm.RemoveWithPrefix(ctx, "") t.Run("Test get segment by varChar primary keys", func(te *testing.T) { - replica := genMockReplica(segIDs, varCharPks, chanName) - fm := NewRendezvousFlushManager(NewAllocatorFactory(), cm, replica, func(*segmentFlushPack) {}, emptyFlushAndDropFunc) + channel := genMockChannel(segIDs, varCharPks, chanName) + fm := NewRendezvousFlushManager(NewAllocatorFactory(), cm, channel, func(*segmentFlushPack) {}, emptyFlushAndDropFunc) c := &nodeConfig{ - replica: replica, + channel: channel, allocator: &allocator{}, vChannelName: chanName, } @@ -275,11 +188,11 @@ func TestFlowGraphDeleteNode_Operate(t *testing.T) { } }) - replica := genMockReplica(segIDs, int64Pks, chanName) - fm := NewRendezvousFlushManager(NewAllocatorFactory(), cm, replica, func(*segmentFlushPack) {}, emptyFlushAndDropFunc) + channel := genMockChannel(segIDs, int64Pks, chanName) + fm := NewRendezvousFlushManager(NewAllocatorFactory(), cm, channel, func(*segmentFlushPack) {}, emptyFlushAndDropFunc) t.Run("Test get segment by int64 primary keys", func(te *testing.T) { c := &nodeConfig{ - replica: replica, + channel: channel, allocator: &allocator{}, vChannelName: chanName, } @@ -314,7 +227,7 @@ func TestFlowGraphDeleteNode_Operate(t *testing.T) { Params.EtcdCfg.MetaRootPath = testPath c := &nodeConfig{ - replica: replica, + channel: channel, allocator: NewAllocatorFactory(), vChannelName: chanName, } @@ -337,7 +250,7 @@ func TestFlowGraphDeleteNode_Operate(t *testing.T) { Params.EtcdCfg.MetaRootPath = testPath c := &nodeConfig{ - replica: replica, + channel: channel, allocator: NewAllocatorFactory(), vChannelName: chanName, } @@ -366,7 +279,7 @@ func TestFlowGraphDeleteNode_Operate(t *testing.T) { Params.EtcdCfg.MetaRootPath = testPath c := &nodeConfig{ - replica: replica, + channel: channel, allocator: NewAllocatorFactory(), vChannelName: chanName, } @@ -401,7 +314,7 @@ func TestFlowGraphDeleteNode_Operate(t *testing.T) { Params.EtcdCfg.MetaRootPath = testPath c := &nodeConfig{ - replica: &mockReplica{}, + channel: nil, allocator: NewAllocatorFactory(), vChannelName: chanName, } @@ -432,15 +345,12 @@ func TestFlowGraphDeleteNode_Operate(t *testing.T) { assert.NoError(t, clearEtcd(testPath)) Params.EtcdCfg.MetaRootPath = testPath - replica := &SegmentReplica{ - newSegments: make(map[UniqueID]*Segment), - normalSegments: make(map[UniqueID]*Segment), - flushedSegments: make(map[UniqueID]*Segment), - compactedSegments: make(map[UniqueID]*Segment), + channel := &ChannelMeta{ + segments: make(map[UniqueID]*Segment), } c := &nodeConfig{ - replica: replica, + channel: channel, allocator: NewAllocatorFactory(), vChannelName: chanName, } @@ -448,17 +358,19 @@ func TestFlowGraphDeleteNode_Operate(t *testing.T) { assert.Nil(t, err) compactedSegment := UniqueID(10020987) - replica.compactedSegments[compactedSegment] = &Segment{ + seg := Segment{ segmentID: compactedSegment, compactedTo: 100, } + seg.setType(datapb.SegmentType_Compacted) + channel.segments[compactedSegment] = &seg msg := genFlowGraphDeleteMsg(int64Pks, chanName) msg.deleteMessages = []*msgstream.DeleteMsg{} msg.segmentsToFlush = []UniqueID{compactedSegment} delNode.delBuf.Store(compactedSegment, &DelDataBuf{delData: &DeleteData{}}) - delNode.flushManager = NewRendezvousFlushManager(&allocator{}, cm, replica, func(*segmentFlushPack) {}, emptyFlushAndDropFunc) + delNode.flushManager = NewRendezvousFlushManager(&allocator{}, cm, channel, func(*segmentFlushPack) {}, emptyFlushAndDropFunc) var fgMsg flowgraph.Msg = &msg setFlowGraphRetryOpt(retry.Attempts(1)) @@ -480,7 +392,7 @@ func TestFlowGraphDeleteNode_showDelBuf(t *testing.T) { cm := storage.NewLocalChunkManager(storage.RootPath(deleteNodeTestDir)) defer cm.RemoveWithPrefix(ctx, "") - fm := NewRendezvousFlushManager(NewAllocatorFactory(), cm, &mockReplica{}, func(*segmentFlushPack) {}, emptyFlushAndDropFunc) + fm := NewRendezvousFlushManager(NewAllocatorFactory(), cm, nil, func(*segmentFlushPack) {}, emptyFlushAndDropFunc) chanName := "datanode-test-FlowGraphDeletenode-showDelBuf" testPath := "/test/datanode/root/meta" @@ -488,7 +400,7 @@ func TestFlowGraphDeleteNode_showDelBuf(t *testing.T) { Params.EtcdCfg.MetaRootPath = testPath c := &nodeConfig{ - replica: &mockReplica{}, + channel: nil, allocator: NewAllocatorFactory(), vChannelName: chanName, } @@ -520,22 +432,19 @@ func TestFlowGraphDeleteNode_updateCompactedSegments(t *testing.T) { cm := storage.NewLocalChunkManager(storage.RootPath(deleteNodeTestDir)) defer cm.RemoveWithPrefix(ctx, "") - fm := NewRendezvousFlushManager(NewAllocatorFactory(), cm, &mockReplica{}, func(*segmentFlushPack) {}, emptyFlushAndDropFunc) + fm := NewRendezvousFlushManager(NewAllocatorFactory(), cm, nil, func(*segmentFlushPack) {}, emptyFlushAndDropFunc) chanName := "datanode-test-FlowGraphDeletenode-showDelBuf" testPath := "/test/datanode/root/meta" assert.NoError(t, clearEtcd(testPath)) Params.EtcdCfg.MetaRootPath = testPath - replica := SegmentReplica{ - newSegments: make(map[UniqueID]*Segment), - normalSegments: make(map[UniqueID]*Segment), - flushedSegments: make(map[UniqueID]*Segment), - compactedSegments: make(map[UniqueID]*Segment), + channel := ChannelMeta{ + segments: make(map[UniqueID]*Segment), } c := &nodeConfig{ - replica: &replica, + channel: &channel, allocator: NewAllocatorFactory(), vChannelName: chanName, } @@ -581,22 +490,26 @@ func TestFlowGraphDeleteNode_updateCompactedSegments(t *testing.T) { delNode.delBuf.Store(seg, delBuf) } - for i, seg := range test.compactedFromIDs { - replica.compactedSegments[seg] = &Segment{ - segmentID: seg, - compactedTo: test.compactedToIDs[i], - } - } - if test.compactToExist { - for _, seg := range test.compactedToIDs { - replica.flushedSegments[seg] = &Segment{ - segmentID: seg, + for _, segID := range test.compactedToIDs { + seg := Segment{ + segmentID: segID, numRows: 10, } + seg.setType(datapb.SegmentType_Flushed) + channel.segments[segID] = &seg } - } else { - replica.flushedSegments = make(map[UniqueID]*Segment) + } else { // clear all segments in channel + channel.segments = make(map[UniqueID]*Segment) + } + + for i, segID := range test.compactedFromIDs { + seg := Segment{ + segmentID: segID, + compactedTo: test.compactedToIDs[i], + } + seg.setType(datapb.SegmentType_Compacted) + channel.segments[segID] = &seg } delNode.updateCompactedSegments() diff --git a/internal/datanode/flow_graph_insert_buffer_node.go b/internal/datanode/flow_graph_insert_buffer_node.go index 3ded0ed3a9..fc4bc9a89b 100644 --- a/internal/datanode/flow_graph_insert_buffer_node.go +++ b/internal/datanode/flow_graph_insert_buffer_node.go @@ -57,7 +57,7 @@ type insertBufferNode struct { ctx context.Context channelName string insertBuffer sync.Map // SegmentID to BufferData - replica Replica + channel Channel idAllocator allocatorInterface flushMap sync.Map @@ -226,11 +226,11 @@ func (ibNode *insertBufferNode) Operate(in []Msg) []Msg { ibNode.lastTimestamp = endPositions[0].Timestamp - // Updating segment statistics in replica - seg2Upload, err := ibNode.updateSegStatesInReplica(fgMsg.insertMessages, startPositions[0], endPositions[0]) + // Updating segment statistics in channel + seg2Upload, err := ibNode.updateSegmentStates(fgMsg.insertMessages, startPositions[0], endPositions[0]) if err != nil { // Occurs only if the collectionID is mismatch, should not happen - err = fmt.Errorf("update segment states in Replica wrong, err = %s", err) + err = fmt.Errorf("update segment states in channel meta wrong, err = %s", err) log.Error(err.Error()) panic(err) } @@ -285,7 +285,7 @@ func (ibNode *insertBufferNode) Operate(in []Msg) []Msg { ) if fgMsg.dropCollection { - segmentsToFlush := ibNode.replica.listAllSegmentIDs() + segmentsToFlush := ibNode.channel.listAllSegmentIDs() log.Info("Receive drop collection req and flushing all segments", zap.Any("segments", segmentsToFlush), zap.String("vchannel name", ibNode.channelName), @@ -378,9 +378,9 @@ func (ibNode *insertBufferNode) Operate(in []Msg) []Msg { // process drop partition for _, partitionDrop := range fgMsg.dropPartitions { - segmentIDs := ibNode.replica.listPartitionSegments(partitionDrop) + segmentIDs := ibNode.channel.listPartitionSegments(partitionDrop) log.Info("(Drop Partition) process drop partition", - zap.Int64("collectionID", ibNode.replica.getCollectionID()), + zap.Int64("collectionID", ibNode.channel.getCollectionID()), zap.Int64("partitionID", partitionDrop), zap.Int64s("segmentIDs", segmentIDs), zap.String("v-channel name", ibNode.channelName), @@ -402,7 +402,7 @@ func (ibNode *insertBufferNode) Operate(in []Msg) []Msg { zap.Any("pos", endPositions[0]), ) - segStats, err := ibNode.replica.getSegmentStatslog(task.segmentID) + segStats, err := ibNode.channel.getSegmentStatslog(task.segmentID) if err != nil { log.Error("failed to get segment stats log", zap.Int64("segmentID", task.segmentID), zap.Error(err)) panic(err) @@ -463,10 +463,10 @@ func (ibNode *insertBufferNode) Operate(in []Msg) []Msg { return []Msg{&res} } -// updateSegStatesInReplica updates statistics in replica for the segments in insertMsgs. +// updateSegmentStates updates statistics in channel meta for the segments in insertMsgs. // If the segment doesn't exist, a new segment will be created. // The segment number of rows will be updated in mem, waiting to be uploaded to DataCoord. -func (ibNode *insertBufferNode) updateSegStatesInReplica(insertMsgs []*msgstream.InsertMsg, startPos, endPos *internalpb.MsgPosition) (seg2Upload []UniqueID, err error) { +func (ibNode *insertBufferNode) updateSegmentStates(insertMsgs []*msgstream.InsertMsg, startPos, endPos *internalpb.MsgPosition) (seg2Upload []UniqueID, err error) { uniqueSeg := make(map[UniqueID]int64) for _, msg := range insertMsgs { @@ -474,14 +474,13 @@ func (ibNode *insertBufferNode) updateSegStatesInReplica(insertMsgs []*msgstream collID := msg.GetCollectionID() partitionID := msg.GetPartitionID() - if !ibNode.replica.hasSegment(currentSegID, true) { - err = ibNode.replica.addSegment( + if !ibNode.channel.hasSegment(currentSegID, true) { + err = ibNode.channel.addSegment( addSegmentReq{ segType: datapb.SegmentType_New, segID: currentSegID, collID: collID, partitionID: partitionID, - channelName: msg.GetShardName(), startPos: startPos, endPos: endPos, }) @@ -503,7 +502,7 @@ func (ibNode *insertBufferNode) updateSegStatesInReplica(insertMsgs []*msgstream seg2Upload = make([]UniqueID, 0, len(uniqueSeg)) for id, num := range uniqueSeg { seg2Upload = append(seg2Upload, id) - ibNode.replica.updateStatistics(id, num) + ibNode.channel.updateStatistics(id, num) } return @@ -511,7 +510,7 @@ func (ibNode *insertBufferNode) updateSegStatesInReplica(insertMsgs []*msgstream /* #nosec G103 */ // bufferInsertMsg put InsertMsg into buffer -// 1.1 fetch related schema from replica +// 1.1 fetch related schema from channel meta // 1.2 Get buffer data and put data into each field buffer // 1.3 Put back into buffer // 1.4 Update related statistics @@ -522,7 +521,7 @@ func (ibNode *insertBufferNode) bufferInsertMsg(msg *msgstream.InsertMsg, endPos currentSegID := msg.GetSegmentID() collectionID := msg.GetCollectionID() - collSchema, err := ibNode.replica.getCollectionSchema(collectionID, msg.EndTs()) + collSchema, err := ibNode.channel.getCollectionSchema(collectionID, msg.EndTs()) if err != nil { log.Error("Get schema wrong:", zap.Error(err)) return err @@ -563,7 +562,7 @@ func (ibNode *insertBufferNode) bufferInsertMsg(msg *msgstream.InsertMsg, endPos if err != nil { log.Warn("no primary field found in insert msg", zap.Error(err)) } else { - ibNode.replica.updateSegmentPKRange(currentSegID, addedPfData) + ibNode.channel.updateSegmentPKRange(currentSegID, addedPfData) } // Maybe there are large write zoom if frequent insert requests are met. @@ -586,7 +585,7 @@ func (ibNode *insertBufferNode) bufferInsertMsg(msg *msgstream.InsertMsg, endPos ibNode.insertBuffer.Store(currentSegID, buffer) // store current endPositions as Segment->EndPostion - ibNode.replica.updateSegmentEndPosition(currentSegID, endPos) + ibNode.channel.updateSegmentEndPosition(currentSegID, endPos) return nil } @@ -616,7 +615,7 @@ func (ibNode *insertBufferNode) writeHardTimeTick(ts Timestamp, segmentIDs []int } func (ibNode *insertBufferNode) getCollectionandPartitionIDbySegID(segmentID UniqueID) (collID, partitionID UniqueID, err error) { - return ibNode.replica.getCollectionAndPartitionID(segmentID) + return ibNode.channel.getCollectionAndPartitionID(segmentID) } func newInsertBufferNode(ctx context.Context, collID UniqueID, flushCh <-chan flushMsg, resendTTCh <-chan resendTTMsg, @@ -640,7 +639,7 @@ func newInsertBufferNode(ctx context.Context, collID UniqueID, flushCh <-chan fl mt := newMergedTimeTickerSender(func(ts Timestamp, segmentIDs []int64) error { stats := make([]*datapb.SegmentStats, 0, len(segmentIDs)) for _, sid := range segmentIDs { - stat, err := config.replica.getSegmentStatisticsUpdates(sid) + stat, err := config.channel.getSegmentStatisticsUpdates(sid) if err != nil { log.Warn("failed to get segment statistics info", zap.Int64("segmentID", sid), zap.Error(err)) continue @@ -684,7 +683,7 @@ func newInsertBufferNode(ctx context.Context, collID UniqueID, flushCh <-chan fl flushingSegCache: flushingSegCache, flushManager: fm, - replica: config.replica, + channel: config.channel, idAllocator: config.allocator, channelName: config.vChannelName, ttMerger: mt, diff --git a/internal/datanode/flow_graph_insert_buffer_node_test.go b/internal/datanode/flow_graph_insert_buffer_node_test.go index 1d788a72cd..2b1a7606ba 100644 --- a/internal/datanode/flow_graph_insert_buffer_node_test.go +++ b/internal/datanode/flow_graph_insert_buffer_node_test.go @@ -77,15 +77,13 @@ func TestFlowGraphInsertBufferNodeCreate(t *testing.T) { pkType: schemapb.DataType_Int64, } - replica, err := newReplica(ctx, mockRootCoord, cm, collMeta.ID, collMeta.GetSchema()) - assert.Nil(t, err) - err = replica.addSegment( + channel := newChannel(insertChannelName, collMeta.ID, collMeta.Schema, mockRootCoord, cm) + err = channel.addSegment( addSegmentReq{ segType: datapb.SegmentType_New, segID: 1, collID: collMeta.ID, partitionID: 0, - channelName: insertChannelName, startPos: &internalpb.MsgPosition{}, endPos: &internalpb.MsgPosition{}, }) @@ -93,13 +91,13 @@ func TestFlowGraphInsertBufferNodeCreate(t *testing.T) { factory := dependency.NewDefaultFactory(true) - fm := NewRendezvousFlushManager(&allocator{}, cm, replica, func(*segmentFlushPack) {}, emptyFlushAndDropFunc) + fm := NewRendezvousFlushManager(&allocator{}, cm, channel, func(*segmentFlushPack) {}, emptyFlushAndDropFunc) flushChan := make(chan flushMsg, 100) resendTTChan := make(chan resendTTMsg, 100) c := &nodeConfig{ - replica: replica, + channel: channel, msFactory: factory, allocator: NewAllocatorFactory(), vChannelName: "string", @@ -172,16 +170,14 @@ func TestFlowGraphInsertBufferNode_Operate(t *testing.T) { pkType: schemapb.DataType_Int64, } - replica, err := newReplica(ctx, mockRootCoord, cm, collMeta.ID, collMeta.GetSchema()) - assert.Nil(t, err) + channel := newChannel(insertChannelName, collMeta.ID, collMeta.Schema, mockRootCoord, cm) - err = replica.addSegment( + err = channel.addSegment( addSegmentReq{ segType: datapb.SegmentType_New, segID: 1, collID: collMeta.ID, partitionID: 0, - channelName: insertChannelName, startPos: &internalpb.MsgPosition{}, endPos: &internalpb.MsgPosition{}, }) @@ -189,12 +185,12 @@ func TestFlowGraphInsertBufferNode_Operate(t *testing.T) { factory := dependency.NewDefaultFactory(true) - fm := NewRendezvousFlushManager(NewAllocatorFactory(), cm, replica, func(*segmentFlushPack) {}, emptyFlushAndDropFunc) + fm := NewRendezvousFlushManager(NewAllocatorFactory(), cm, channel, func(*segmentFlushPack) {}, emptyFlushAndDropFunc) flushChan := make(chan flushMsg, 100) resendTTChan := make(chan resendTTMsg, 100) c := &nodeConfig{ - replica: replica, + channel: channel, msFactory: factory, allocator: NewAllocatorFactory(), vChannelName: "string", @@ -235,7 +231,7 @@ func TestFlowGraphInsertBufferNode_Operate(t *testing.T) { assert.Panics(t, func() { iBNode.Operate([]flowgraph.Msg{&inMsg}) }) iBNode.lastTimestamp = timestampBak - // test updateSegStatesInReplica failed + // test updateSegmentStates failed inMsg = genFlowGraphInsertMsg(insertChannelName) inMsg.insertMessages[0].CollectionID = UniqueID(-1) inMsg.insertMessages[0].SegmentID = UniqueID(-1) @@ -275,12 +271,11 @@ func TestFlushSegment(t *testing.T) { flushMap := sync.Map{} mockRootCoord := &RootCoordFactory{} - replica, err := newReplica(ctx, mockRootCoord, collMeta.ID) - assert.Nil(t, err) + channel := newChannel("channel", collMeta.ID, collMeta.Schema, mockRootCoord) - err = replica.addNewSegment(segmentID, collMeta.ID, 0, insertChannelName, &internalpb.MsgPosition{}, &internalpb.MsgPosition{}) + err = channel.addNewSegment(segmentID, collMeta.ID, 0, insertChannelName, &internalpb.MsgPosition{}, &internalpb.MsgPosition{}) require.NoError(t, err) - replica.updateSegmentEndPosition(segmentID, &internalpb.MsgPosition{ChannelName: "TestChannel"}) + channel.updateSegmentEndPosition(segmentID, &internalpb.MsgPosition{ChannelName: "TestChannel"}) finishCh := make(chan segmentFlushUnit, 1) @@ -312,12 +307,12 @@ func TestFlushSegment(t *testing.T) { memkv := memkv.NewMemoryKV() - fm := NewRendezvousFlushManager(&allocator{}, memkv, replica, func(*segmentFlushPack) error { + fm := NewRendezvousFlushManager(&allocator{}, memkv, channel, func(*segmentFlushPack) error { return nil }) c := &nodeConfig{ - replica: replica, + channel: channel, factory: factory, allocator: NewAllocatorFactory(), vChannelName: "string", @@ -414,14 +409,12 @@ func TestFlowGraphInsertBufferNode_AutoFlush(t *testing.T) { pkType: schemapb.DataType_Int64, } - colRep := &SegmentReplica{ - collectionID: collMeta.ID, - newSegments: make(map[UniqueID]*Segment), - normalSegments: make(map[UniqueID]*Segment), - flushedSegments: make(map[UniqueID]*Segment), + channel := &ChannelMeta{ + collectionID: collMeta.ID, + segments: make(map[UniqueID]*Segment), } - colRep.metaService = newMetaService(mockRootCoord, collMeta.ID) + channel.metaService = newMetaService(mockRootCoord, collMeta.ID) factory := dependency.NewDefaultFactory(true) @@ -431,17 +424,16 @@ func TestFlowGraphInsertBufferNode_AutoFlush(t *testing.T) { cm := storage.NewLocalChunkManager(storage.RootPath(insertNodeTestDir)) defer cm.RemoveWithPrefix(ctx, "") - fm := NewRendezvousFlushManager(NewAllocatorFactory(), cm, colRep, func(pack *segmentFlushPack) { + fm := NewRendezvousFlushManager(NewAllocatorFactory(), cm, channel, func(pack *segmentFlushPack) { fpMut.Lock() flushPacks = append(flushPacks, pack) fpMut.Unlock() - startPos := colRep.listNewSegmentsStartPositions() - colRep.transferNewSegments(lo.Map(startPos, func(pos *datapb.SegmentStartPosition, _ int) UniqueID { + startPos := channel.listNewSegmentsStartPositions() + channel.transferNewSegments(lo.Map(startPos, func(pos *datapb.SegmentStartPosition, _ int) UniqueID { return pos.GetSegmentID() })) - colRep.listSegmentsCheckPoints() if pack.flushed || pack.dropped { - colRep.segmentFlushed(pack.segmentID) + channel.segmentFlushed(pack.segmentID) } wg.Done() }, emptyFlushAndDropFunc) @@ -449,7 +441,7 @@ func TestFlowGraphInsertBufferNode_AutoFlush(t *testing.T) { flushChan := make(chan flushMsg, 100) resendTTChan := make(chan resendTTMsg, 100) c := &nodeConfig{ - replica: colRep, + channel: channel, msFactory: factory, allocator: NewAllocatorFactory(), vChannelName: "string", @@ -493,20 +485,17 @@ func TestFlowGraphInsertBufferNode_AutoFlush(t *testing.T) { } iBNode.Operate([]flowgraph.Msg{iMsg}) - require.Equal(t, 2, len(colRep.newSegments)) - require.Equal(t, 0, len(colRep.normalSegments)) assert.Equal(t, 0, len(flushPacks)) for i, test := range beforeAutoFlushTests { - colRep.segMu.Lock() - seg, ok := colRep.newSegments[UniqueID(i+1)] - colRep.segMu.Unlock() + channel.segMu.Lock() + seg, ok := channel.segments[UniqueID(i+1)] + channel.segMu.Unlock() assert.True(t, ok) + assert.Equal(t, datapb.SegmentType_New, seg.getType()) assert.Equal(t, test.expectedSegID, seg.segmentID) assert.Equal(t, test.expectedNumOfRows, seg.numRows) assert.Equal(t, test.expectedStartPosTs, seg.startPos.GetTimestamp()) - assert.Equal(t, test.expectedCpNumOfRows, seg.checkPoint.numRows) - assert.Equal(t, test.expectedCpPosTs, seg.checkPoint.pos.GetTimestamp()) } for i := range inMsg.insertMessages { @@ -528,11 +517,8 @@ func TestFlowGraphInsertBufferNode_AutoFlush(t *testing.T) { assert.NoError(t, err) } wg.Wait() - require.Equal(t, 0, len(colRep.newSegments)) - require.Equal(t, 3, len(colRep.normalSegments)) assert.Equal(t, 1, len(flushPacks)) - // assert.Equal(t, 3, len(flushUnit[0].checkPoint)) assert.Less(t, 0, len(flushPacks[0].insertLogs)) assert.False(t, flushPacks[0].flushed) @@ -544,16 +530,12 @@ func TestFlowGraphInsertBufferNode_AutoFlush(t *testing.T) { } for i, test := range afterAutoFlushTests { - seg, ok := colRep.normalSegments[UniqueID(i+1)] + seg, ok := channel.segments[UniqueID(i+1)] assert.True(t, ok) + assert.Equal(t, datapb.SegmentType_Normal, seg.getType()) assert.Equal(t, test.expectedSegID, seg.segmentID) assert.Equal(t, test.expectedNumOfRows, seg.numRows) assert.Equal(t, test.expectedStartPosTs, seg.startPos.GetTimestamp()) - assert.Equal(t, test.expectedCpNumOfRows, seg.checkPoint.numRows) - assert.Equal(t, test.expectedCpPosTs, seg.checkPoint.pos.GetTimestamp()) - - // assert.Equal(t, test.expectedCpNumOfRows, flushPacks[0].checkPoint[UniqueID(i+1)].numRows) - // assert.Equal(t, test.expectedCpPosTs, flushPacks[0].checkPoint[UniqueID(i+1)].pos.Timestamp) if i == 1 { assert.Equal(t, test.expectedSegID, flushPacks[0].segmentID) @@ -603,20 +585,17 @@ func TestFlowGraphInsertBufferNode_AutoFlush(t *testing.T) { } iBNode.Operate([]flowgraph.Msg{iMsg}) - require.Equal(t, 2, len(colRep.newSegments)) - require.Equal(t, 3, len(colRep.normalSegments)) assert.Equal(t, 0, len(flushPacks)) for _, test := range beforeAutoFlushTests { - colRep.segMu.Lock() - seg, ok := colRep.newSegments[test.expectedSegID] - colRep.segMu.Unlock() + channel.segMu.Lock() + seg, ok := channel.segments[test.expectedSegID] + channel.segMu.Unlock() assert.True(t, ok) + assert.Equal(t, datapb.SegmentType_New, seg.getType()) assert.Equal(t, test.expectedSegID, seg.segmentID) assert.Equal(t, test.expectedNumOfRows, seg.numRows) assert.Equal(t, test.expectedStartPosTs, seg.startPos.GetTimestamp()) - assert.Equal(t, test.expectedCpNumOfRows, seg.checkPoint.numRows) - assert.Equal(t, test.expectedCpPosTs, seg.checkPoint.pos.GetTimestamp()) } inMsg.startPositions = []*internalpb.MsgPosition{{Timestamp: 400}} @@ -638,9 +617,6 @@ func TestFlowGraphInsertBufferNode_AutoFlush(t *testing.T) { assert.NoError(t, err) } wg.Wait() - require.Equal(t, 0, len(colRep.newSegments)) - require.Equal(t, 4, len(colRep.normalSegments)) - require.Equal(t, 1, len(colRep.flushedSegments)) assert.Equal(t, 2, len(flushPacks)) for _, pack := range flushPacks { @@ -673,14 +649,12 @@ func TestFlowGraphInsertBufferNode_DropPartition(t *testing.T) { pkType: schemapb.DataType_Int64, } - colRep := &SegmentReplica{ - collectionID: collMeta.ID, - newSegments: make(map[UniqueID]*Segment), - normalSegments: make(map[UniqueID]*Segment), - flushedSegments: make(map[UniqueID]*Segment), + channel := &ChannelMeta{ + collectionID: collMeta.ID, + segments: make(map[UniqueID]*Segment), } - colRep.metaService = newMetaService(mockRootCoord, collMeta.ID) + channel.metaService = newMetaService(mockRootCoord, collMeta.ID) factory := dependency.NewDefaultFactory(true) @@ -690,14 +664,13 @@ func TestFlowGraphInsertBufferNode_DropPartition(t *testing.T) { cm := storage.NewLocalChunkManager(storage.RootPath(insertNodeTestDir)) defer cm.RemoveWithPrefix(ctx, "") - fm := NewRendezvousFlushManager(NewAllocatorFactory(), cm, colRep, func(pack *segmentFlushPack) { + fm := NewRendezvousFlushManager(NewAllocatorFactory(), cm, channel, func(pack *segmentFlushPack) { fpMut.Lock() flushPacks = append(flushPacks, pack) fpMut.Unlock() - colRep.listNewSegmentsStartPositions() - colRep.listSegmentsCheckPoints() + channel.listNewSegmentsStartPositions() if pack.flushed || pack.dropped { - colRep.segmentFlushed(pack.segmentID) + channel.segmentFlushed(pack.segmentID) } wg.Done() }, emptyFlushAndDropFunc) @@ -705,7 +678,7 @@ func TestFlowGraphInsertBufferNode_DropPartition(t *testing.T) { flushChan := make(chan flushMsg, 100) resendTTChan := make(chan resendTTMsg, 100) c := &nodeConfig{ - replica: colRep, + channel: channel, msFactory: factory, allocator: NewAllocatorFactory(), vChannelName: "string", @@ -750,21 +723,18 @@ func TestFlowGraphInsertBufferNode_DropPartition(t *testing.T) { } iBNode.Operate([]flowgraph.Msg{iMsg}) - require.Equal(t, 2, len(colRep.newSegments)) - require.Equal(t, 0, len(colRep.normalSegments)) assert.Equal(t, 0, len(flushPacks)) for i, test := range beforeAutoFlushTests { - colRep.segMu.Lock() - seg, ok := colRep.newSegments[UniqueID(i+1)] - colRep.segMu.Unlock() + channel.segMu.Lock() + seg, ok := channel.segments[UniqueID(i+1)] + channel.segMu.Unlock() assert.True(t, ok) + assert.Equal(t, datapb.SegmentType_New, seg.getType()) assert.Equal(t, partitionID, seg.partitionID) assert.Equal(t, test.expectedSegID, seg.segmentID) assert.Equal(t, test.expectedNumOfRows, seg.numRows) assert.Equal(t, test.expectedStartPosTs, seg.startPos.GetTimestamp()) - assert.Equal(t, test.expectedCpNumOfRows, seg.checkPoint.numRows) - assert.Equal(t, test.expectedCpPosTs, seg.checkPoint.pos.GetTimestamp()) } inMsg.insertMessages = nil @@ -785,9 +755,6 @@ func TestFlowGraphInsertBufferNode_DropPartition(t *testing.T) { assert.NoError(t, err) } wg.Wait() - require.Equal(t, 0, len(colRep.newSegments)) - require.Equal(t, 0, len(colRep.normalSegments)) - require.Equal(t, 2, len(colRep.flushedSegments)) assert.Equal(t, 2, len(flushPacks)) assert.Less(t, 0, len(flushPacks[0].insertLogs)) @@ -836,21 +803,18 @@ func TestFlowGraphInsertBufferNode_DropPartition(t *testing.T) { } iBNode.Operate([]flowgraph.Msg{iMsg}) - require.Equal(t, 2, len(colRep.newSegments)) - require.Equal(t, 0, len(colRep.normalSegments)) assert.Equal(t, 0, len(flushPacks)) for _, test := range beforeAutoFlushTests { - colRep.segMu.Lock() - seg, ok := colRep.newSegments[test.expectedSegID] - colRep.segMu.Unlock() + channel.segMu.Lock() + seg, ok := channel.segments[test.expectedSegID] + channel.segMu.Unlock() assert.True(t, ok) + assert.Equal(t, datapb.SegmentType_New, seg.getType()) assert.Equal(t, partitionID, seg.partitionID) assert.Equal(t, test.expectedSegID, seg.segmentID) assert.Equal(t, test.expectedNumOfRows, seg.numRows) assert.Equal(t, test.expectedStartPosTs, seg.startPos.GetTimestamp()) - assert.Equal(t, test.expectedCpNumOfRows, seg.checkPoint.numRows) - assert.Equal(t, test.expectedCpPosTs, seg.checkPoint.pos.GetTimestamp()) } inMsg.startPositions = []*internalpb.MsgPosition{{Timestamp: 400}} @@ -873,9 +837,6 @@ func TestFlowGraphInsertBufferNode_DropPartition(t *testing.T) { assert.NoError(t, err) } wg.Wait() - require.Equal(t, 0, len(colRep.newSegments)) - require.Equal(t, 0, len(colRep.normalSegments)) - require.Equal(t, 4, len(colRep.flushedSegments)) assert.Equal(t, 4, len(flushPacks)) for _, pack := range flushPacks { @@ -937,15 +898,13 @@ func TestInsertBufferNode_bufferInsertMsg(t *testing.T) { compactTs: 100, } - replica, err := newReplica(ctx, mockRootCoord, cm, collMeta.ID, collMeta.GetSchema()) - assert.Nil(t, err) - err = replica.addSegment( + channel := newChannel(insertChannelName, collMeta.ID, collMeta.Schema, mockRootCoord, cm) + err = channel.addSegment( addSegmentReq{ segType: datapb.SegmentType_New, segID: 1, collID: collMeta.ID, partitionID: 0, - channelName: insertChannelName, startPos: &internalpb.MsgPosition{}, endPos: &internalpb.MsgPosition{Timestamp: 101}, }) @@ -953,12 +912,12 @@ func TestInsertBufferNode_bufferInsertMsg(t *testing.T) { factory := dependency.NewDefaultFactory(true) - fm := NewRendezvousFlushManager(&allocator{}, cm, replica, func(*segmentFlushPack) {}, emptyFlushAndDropFunc) + fm := NewRendezvousFlushManager(&allocator{}, cm, channel, func(*segmentFlushPack) {}, emptyFlushAndDropFunc) flushChan := make(chan flushMsg, 100) resendTTChan := make(chan resendTTMsg, 100) c := &nodeConfig{ - replica: replica, + channel: channel, msFactory: factory, allocator: NewAllocatorFactory(), vChannelName: "string", @@ -982,13 +941,13 @@ func TestInsertBufferNode_bufferInsertMsg(t *testing.T) { } } -func TestInsertBufferNode_updateSegStatesInReplica(te *testing.T) { +func TestInsertBufferNode_updateSegmentStates(te *testing.T) { ctx, cancel := context.WithCancel(context.Background()) defer cancel() cm := storage.NewLocalChunkManager(storage.RootPath(insertNodeTestDir)) defer cm.RemoveWithPrefix(ctx, "") invalideTests := []struct { - replicaCollID UniqueID + channelCollID UniqueID inCollID UniqueID segID UniqueID @@ -998,11 +957,10 @@ func TestInsertBufferNode_updateSegStatesInReplica(te *testing.T) { } for _, test := range invalideTests { - replica, err := newReplica(context.Background(), &RootCoordFactory{pkType: schemapb.DataType_Int64}, cm, test.replicaCollID, nil) - assert.Nil(te, err) + channel := newChannel("channel", test.channelCollID, nil, &RootCoordFactory{pkType: schemapb.DataType_Int64}, cm) ibNode := &insertBufferNode{ - replica: replica, + channel: channel, } im := []*msgstream.InsertMsg{ @@ -1014,13 +972,11 @@ func TestInsertBufferNode_updateSegStatesInReplica(te *testing.T) { }, } - seg, err := ibNode.updateSegStatesInReplica(im, &internalpb.MsgPosition{}, &internalpb.MsgPosition{}) + seg, err := ibNode.updateSegmentStates(im, &internalpb.MsgPosition{}, &internalpb.MsgPosition{}) assert.Error(te, err) assert.Empty(te, seg) - } - } func TestInsertBufferNode_BufferData(te *testing.T) { diff --git a/internal/datanode/flow_graph_manager.go b/internal/datanode/flow_graph_manager.go index 7017cc6b99..cd7dd671b9 100644 --- a/internal/datanode/flow_graph_manager.go +++ b/internal/datanode/flow_graph_manager.go @@ -42,15 +42,11 @@ func (fm *flowgraphManager) addAndStart(dn *DataNode, vchan *datapb.VchannelInfo return nil } - replica, err := newReplica(dn.ctx, dn.rootCoord, dn.chunkManager, vchan.GetCollectionID(), schema) - if err != nil { - log.Warn("new replica failed", zap.String("vChannelName", vchan.GetChannelName()), zap.Error(err)) - return err - } + channel := newChannel(vchan.GetChannelName(), vchan.GetCollectionID(), schema, dn.rootCoord, dn.chunkManager) var alloc allocatorInterface = newAllocator(dn.rootCoord) - dataSyncService, err := newDataSyncService(dn.ctx, make(chan flushMsg, 100), make(chan resendTTMsg, 100), replica, + dataSyncService, err := newDataSyncService(dn.ctx, make(chan flushMsg, 100), make(chan resendTTMsg, 100), channel, alloc, dn.factory, vchan, dn.clearSignal, dn.dataCoord, dn.segmentCache, dn.chunkManager, dn.compactionExecutor) if err != nil { log.Warn("new data sync service fail", zap.String("vChannelName", vchan.GetChannelName()), zap.Error(err)) @@ -76,7 +72,7 @@ func (fm *flowgraphManager) getFlushCh(segID UniqueID) (chan<- flushMsg, error) fm.flowgraphs.Range(func(key, value interface{}) bool { fg := value.(*dataSyncService) - if fg.replica.hasSegment(segID, true) { + if fg.channel.hasSegment(segID, true) { flushCh = fg.flushCh return false } @@ -90,16 +86,16 @@ func (fm *flowgraphManager) getFlushCh(segID UniqueID) (chan<- flushMsg, error) return nil, fmt.Errorf("cannot find segment %d in all flowgraphs", segID) } -func (fm *flowgraphManager) getReplica(segID UniqueID) (Replica, error) { +func (fm *flowgraphManager) getChannel(segID UniqueID) (Channel, error) { var ( - rep Replica + rep Channel exists = false ) fm.flowgraphs.Range(func(key, value interface{}) bool { fg := value.(*dataSyncService) - if fg.replica.hasSegment(segID, true) { + if fg.channel.hasSegment(segID, true) { exists = true - rep = fg.replica + rep = fg.channel return false } return true @@ -118,7 +114,7 @@ func (fm *flowgraphManager) resendTT() []UniqueID { var unFlushedSegments []UniqueID fm.flowgraphs.Range(func(key, value interface{}) bool { fg := value.(*dataSyncService) - segIDs := fg.replica.listNotFlushedSegmentIDs() + segIDs := fg.channel.listNotFlushedSegmentIDs() if len(segIDs) > 0 { log.Info("un-flushed segments found, stats will be resend", zap.Int64s("segment IDs", segIDs)) diff --git a/internal/datanode/flow_graph_manager_test.go b/internal/datanode/flow_graph_manager_test.go index 9a104fa5cb..bd48f0559c 100644 --- a/internal/datanode/flow_graph_manager_test.go +++ b/internal/datanode/flow_graph_manager_test.go @@ -81,8 +81,8 @@ func TestFlowGraphManager(t *testing.T) { fm.dropAll() }) - t.Run("Test getReplica", func(t *testing.T) { - vchanName := "by-dev-rootcoord-dml-test-flowgraphmanager-getReplica" + t.Run("Test getChannel", func(t *testing.T) { + vchanName := "by-dev-rootcoord-dml-test-flowgraphmanager-getChannel" vchan := &datapb.VchannelInfo{ CollectionID: 1, ChannelName: vchanName, @@ -94,12 +94,11 @@ func TestFlowGraphManager(t *testing.T) { assert.True(t, fm.exist(vchanName)) fg, ok := fm.getFlowgraphService(vchanName) require.True(t, ok) - err = fg.replica.addSegment(addSegmentReq{ + err = fg.channel.addSegment(addSegmentReq{ segType: datapb.SegmentType_New, segID: 100, collID: 1, partitionID: 10, - channelName: vchanName, startPos: &internalpb.MsgPosition{}, endPos: &internalpb.MsgPosition{}, }) @@ -117,7 +116,7 @@ func TestFlowGraphManager(t *testing.T) { for _, test := range tests { t.Run(test.description, func(t *testing.T) { - rep, err := fm.getReplica(test.inSegID) + rep, err := fm.getChannel(test.inSegID) if test.isvalid { assert.NoError(t, err) @@ -144,12 +143,11 @@ func TestFlowGraphManager(t *testing.T) { fg, ok := fm.getFlowgraphService(vchanName) require.True(t, ok) - err = fg.replica.addSegment(addSegmentReq{ + err = fg.channel.addSegment(addSegmentReq{ segType: datapb.SegmentType_New, segID: 100, collID: 1, partitionID: 10, - channelName: vchanName, startPos: &internalpb.MsgPosition{}, endPos: &internalpb.MsgPosition{}, }) diff --git a/internal/datanode/flush_manager.go b/internal/datanode/flush_manager.go index e89332ac1d..b70263368b 100644 --- a/internal/datanode/flush_manager.go +++ b/internal/datanode/flush_manager.go @@ -262,7 +262,7 @@ type dropHandler struct { type rendezvousFlushManager struct { allocatorInterface storage.ChunkManager - Replica + Channel // segment id => flush queue dispatcher sync.Map @@ -422,7 +422,6 @@ func (m *rendezvousFlushManager) flushBufferData(data *BufferData, segStats []by LogSize: int64(len(segStats)), } - m.updateSegmentCheckPoint(segmentID) m.handleInsertTask(segmentID, &flushBufferInsertTask{ ChunkManager: m.ChunkManager, data: kvs, @@ -484,7 +483,7 @@ func (m *rendezvousFlushManager) injectFlush(injection *taskInjection, segments // fetch meta info for segment func (m *rendezvousFlushManager) getSegmentMeta(segmentID UniqueID, pos *internalpb.MsgPosition) (UniqueID, UniqueID, *etcdpb.CollectionMeta, error) { if !m.hasSegment(segmentID, true) { - return -1, -1, nil, fmt.Errorf("no such segment %d in the replica", segmentID) + return -1, -1, nil, fmt.Errorf("no such segment %d in the channel", segmentID) } // fetch meta information of segment @@ -608,12 +607,12 @@ func (t *flushBufferDeleteTask) flushDeleteData() error { } // NewRendezvousFlushManager create rendezvousFlushManager with provided allocator and kv -func NewRendezvousFlushManager(allocator allocatorInterface, cm storage.ChunkManager, replica Replica, f notifyMetaFunc, drop flushAndDropFunc) *rendezvousFlushManager { +func NewRendezvousFlushManager(allocator allocatorInterface, cm storage.ChunkManager, channel Channel, f notifyMetaFunc, drop flushAndDropFunc) *rendezvousFlushManager { fm := &rendezvousFlushManager{ allocatorInterface: allocator, ChunkManager: cm, notifyFunc: f, - Replica: replica, + Channel: channel, dropHandler: dropHandler{ flushAndDrop: drop, }, @@ -680,7 +679,7 @@ func dropVirtualChannelFunc(dsService *dataSyncService, opts ...retry.Option) fl segment.Deltalogs = append(segment.Deltalogs, &datapb.FieldBinlog{ Binlogs: pack.deltaLogs, }) - updates, _ := dsService.replica.getSegmentStatisticsUpdates(pack.segmentID) + updates, _ := dsService.channel.getSegmentStatisticsUpdates(pack.segmentID) segment.NumOfRows = updates.GetNumRows() if pack.pos != nil { if segment.CheckPoint == nil || pack.pos.Timestamp > segment.CheckPoint.Timestamp { @@ -689,7 +688,7 @@ func dropVirtualChannelFunc(dsService *dataSyncService, opts ...retry.Option) fl } } - startPos := dsService.replica.listNewSegmentsStartPositions() + startPos := dsService.channel.listNewSegmentsStartPositions() // start positions for all new segments for _, pos := range startPos { segment, has := segmentPack[pos.GetSegmentID()] @@ -728,7 +727,7 @@ func dropVirtualChannelFunc(dsService *dataSyncService, opts ...retry.Option) fl if rsp.GetStatus().GetErrorCode() != commonpb.ErrorCode_Success { return fmt.Errorf("data service DropVirtualChannel failed, reason = %s", rsp.GetStatus().GetReason()) } - dsService.replica.transferNewSegments(lo.Map(startPos, func(pos *datapb.SegmentStartPosition, _ int) UniqueID { + dsService.channel.transferNewSegments(lo.Map(startPos, func(pos *datapb.SegmentStartPosition, _ int) UniqueID { return pos.GetSegmentID() })) return nil @@ -738,7 +737,7 @@ func dropVirtualChannelFunc(dsService *dataSyncService, opts ...retry.Option) fl panic(err) } for segID := range segmentPack { - dsService.replica.segmentFlushed(segID) + dsService.channel.segmentFlushed(segID) dsService.flushingSegCache.Remove(segID) } } @@ -768,14 +767,14 @@ func flushNotifyFunc(dsService *dataSyncService, opts ...retry.Option) notifyMet deltaInfos[0] = &datapb.FieldBinlog{Binlogs: pack.deltaLogs} // only current segment checkpoint info, - updates, _ := dsService.replica.getSegmentStatisticsUpdates(pack.segmentID) + updates, _ := dsService.channel.getSegmentStatisticsUpdates(pack.segmentID) checkPoints = append(checkPoints, &datapb.CheckPoint{ SegmentID: pack.segmentID, NumOfRows: updates.GetNumRows(), Position: pack.pos, }) - startPos := dsService.replica.listNewSegmentsStartPositions() + startPos := dsService.channel.listNewSegmentsStartPositions() log.Info("SaveBinlogPath", zap.Int64("SegmentID", pack.segmentID), @@ -833,7 +832,7 @@ func flushNotifyFunc(dsService *dataSyncService, opts ...retry.Option) notifyMet return fmt.Errorf("data service save bin log path failed, reason = %s", rsp.Reason) } - dsService.replica.transferNewSegments(lo.Map(startPos, func(pos *datapb.SegmentStartPosition, _ int) UniqueID { + dsService.channel.transferNewSegments(lo.Map(startPos, func(pos *datapb.SegmentStartPosition, _ int) UniqueID { return pos.GetSegmentID() })) return nil @@ -846,7 +845,7 @@ func flushNotifyFunc(dsService *dataSyncService, opts ...retry.Option) notifyMet panic(err) } if pack.flushed || pack.dropped { - dsService.replica.segmentFlushed(pack.segmentID) + dsService.channel.segmentFlushed(pack.segmentID) } dsService.flushingSegCache.Remove(req.GetSegmentID()) } diff --git a/internal/datanode/flush_manager_test.go b/internal/datanode/flush_manager_test.go index 3429345be0..eb3413618e 100644 --- a/internal/datanode/flush_manager_test.go +++ b/internal/datanode/flush_manager_test.go @@ -140,6 +140,12 @@ func TestOrderFlushQueue_Order(t *testing.T) { } } +func newTestChannel() *ChannelMeta { + return &ChannelMeta{ + segments: make(map[UniqueID]*Segment), + } +} + func TestRendezvousFlushManager(t *testing.T) { ctx, cancel := context.WithCancel(context.Background()) defer cancel() @@ -150,7 +156,7 @@ func TestRendezvousFlushManager(t *testing.T) { var counter atomic.Int64 finish := sync.WaitGroup{} finish.Add(size) - m := NewRendezvousFlushManager(&allocator{}, cm, newMockReplica(), func(pack *segmentFlushPack) { + m := NewRendezvousFlushManager(&allocator{}, cm, newTestChannel(), func(pack *segmentFlushPack) { counter.Inc() finish.Done() }, emptyFlushAndDropFunc) @@ -191,7 +197,7 @@ func TestRendezvousFlushManager_Inject(t *testing.T) { finish.Add(size) var packMut sync.Mutex packs := make([]*segmentFlushPack, 0, size+3) - m := NewRendezvousFlushManager(&allocator{}, cm, newMockReplica(), func(pack *segmentFlushPack) { + m := NewRendezvousFlushManager(&allocator{}, cm, newTestChannel(), func(pack *segmentFlushPack) { packMut.Lock() packs = append(packs, pack) packMut.Unlock() @@ -283,23 +289,29 @@ func TestRendezvousFlushManager_Inject(t *testing.T) { func TestRendezvousFlushManager_getSegmentMeta(t *testing.T) { cm := storage.NewLocalChunkManager(storage.RootPath(flushTestDir)) - replica := newMockReplica() - fm := NewRendezvousFlushManager(NewAllocatorFactory(), cm, replica, func(*segmentFlushPack) { + channel := newTestChannel() + channel.collSchema = &schemapb.CollectionSchema{} + fm := NewRendezvousFlushManager(NewAllocatorFactory(), cm, channel, func(*segmentFlushPack) { }, emptyFlushAndDropFunc) // non exists segment _, _, _, err := fm.getSegmentMeta(-1, &internalpb.MsgPosition{}) assert.Error(t, err) - replica.newSegments[-1] = &Segment{} - replica.newSegments[1] = &Segment{} + seg0 := Segment{segmentID: -1} + seg1 := Segment{segmentID: 1} + seg0.setType(datapb.SegmentType_New) + seg1.setType(datapb.SegmentType_New) - // injected get part/coll id error - _, _, _, err = fm.getSegmentMeta(-1, &internalpb.MsgPosition{}) - assert.Error(t, err) - // injected get schema error - _, _, _, err = fm.getSegmentMeta(1, &internalpb.MsgPosition{}) - assert.Error(t, err) + channel.segments[-1] = &seg0 + channel.segments[1] = &seg1 + + // // injected get part/coll id error + // _, _, _, err = fm.getSegmentMeta(-1, &internalpb.MsgPosition{}) + // assert.Error(t, err) + // // injected get schema error + // _, _, _, err = fm.getSegmentMeta(1, &internalpb.MsgPosition{}) + // assert.Error(t, err) } func TestRendezvousFlushManager_waitForAllFlushQueue(t *testing.T) { @@ -309,7 +321,7 @@ func TestRendezvousFlushManager_waitForAllFlushQueue(t *testing.T) { var counter atomic.Int64 var finish sync.WaitGroup finish.Add(size) - m := NewRendezvousFlushManager(&allocator{}, cm, newMockReplica(), func(pack *segmentFlushPack) { + m := NewRendezvousFlushManager(&allocator{}, cm, newTestChannel(), func(pack *segmentFlushPack) { counter.Inc() finish.Done() }, emptyFlushAndDropFunc) @@ -379,7 +391,7 @@ func TestRendezvousFlushManager_dropMode(t *testing.T) { var result []*segmentFlushPack signal := make(chan struct{}) - m := NewRendezvousFlushManager(&allocator{}, cm, newMockReplica(), func(pack *segmentFlushPack) { + m := NewRendezvousFlushManager(&allocator{}, cm, newTestChannel(), func(pack *segmentFlushPack) { }, func(packs []*segmentFlushPack) { mut.Lock() result = packs @@ -431,7 +443,7 @@ func TestRendezvousFlushManager_dropMode(t *testing.T) { var result []*segmentFlushPack signal := make(chan struct{}) - m := NewRendezvousFlushManager(&allocator{}, cm, newMockReplica(), func(pack *segmentFlushPack) { + m := NewRendezvousFlushManager(&allocator{}, cm, newTestChannel(), func(pack *segmentFlushPack) { }, func(packs []*segmentFlushPack) { mut.Lock() result = packs @@ -490,7 +502,7 @@ func TestRendezvousFlushManager_close(t *testing.T) { var counter atomic.Int64 finish := sync.WaitGroup{} finish.Add(size) - m := NewRendezvousFlushManager(&allocator{}, cm, newMockReplica(), func(pack *segmentFlushPack) { + m := NewRendezvousFlushManager(&allocator{}, cm, newTestChannel(), func(pack *segmentFlushPack) { counter.Inc() finish.Done() }, emptyFlushAndDropFunc) @@ -521,20 +533,18 @@ func TestRendezvousFlushManager_close(t *testing.T) { } func TestFlushNotifyFunc(t *testing.T) { - ctx := context.Background() rcf := &RootCoordFactory{ pkType: schemapb.DataType_Int64, } cm := storage.NewLocalChunkManager(storage.RootPath(flushTestDir)) - replica, err := newReplica(ctx, rcf, cm, 1, nil) - require.NoError(t, err) + channel := newChannel("channel", 1, nil, rcf, cm) dataCoord := &DataCoordFactory{} flushingCache := newCache() dsService := &dataSyncService{ collectionID: 1, - replica: replica, + channel: channel, dataCoord: dataCoord, flushingSegCache: flushingCache, } @@ -598,34 +608,33 @@ func TestFlushNotifyFunc(t *testing.T) { } func TestDropVirtualChannelFunc(t *testing.T) { - ctx := context.Background() rcf := &RootCoordFactory{ pkType: schemapb.DataType_Int64, } + vchanName := "vchan_01" cm := storage.NewLocalChunkManager(storage.RootPath(flushTestDir)) - replica, err := newReplica(ctx, rcf, cm, 1, nil) - require.NoError(t, err) + channel := newChannel(vchanName, 1, nil, rcf, cm) dataCoord := &DataCoordFactory{} flushingCache := newCache() dsService := &dataSyncService{ collectionID: 1, - replica: replica, + channel: channel, dataCoord: dataCoord, flushingSegCache: flushingCache, - vchannelName: "vchan_01", + vchannelName: vchanName, } dropFunc := dropVirtualChannelFunc(dsService, retry.Attempts(1)) t.Run("normal run", func(t *testing.T) { - replica.addSegment( + channel.addSegment( addSegmentReq{ segType: datapb.SegmentType_New, segID: 2, collID: 1, partitionID: 10, - channelName: "vchan_01", startPos: &internalpb.MsgPosition{ - ChannelName: "vchan_01", + startPos: &internalpb.MsgPosition{ + ChannelName: vchanName, MsgID: []byte{1, 2, 3}, Timestamp: 10, }, endPos: nil}) @@ -637,7 +646,7 @@ func TestDropVirtualChannelFunc(t *testing.T) { statsLogs: map[UniqueID]*datapb.Binlog{1: {LogPath: "/dev/test/id-stats"}}, deltaLogs: []*datapb.Binlog{{LogPath: "/dev/test/del"}}, pos: &internalpb.MsgPosition{ - ChannelName: "vchan_01", + ChannelName: vchanName, MsgID: []byte{1, 2, 3}, Timestamp: 10, }, @@ -648,7 +657,7 @@ func TestDropVirtualChannelFunc(t *testing.T) { statsLogs: map[UniqueID]*datapb.Binlog{1: {LogPath: "/dev/test/id-stats-2"}}, deltaLogs: []*datapb.Binlog{{LogPath: "/dev/test/del-2"}}, pos: &internalpb.MsgPosition{ - ChannelName: "vchan_01", + ChannelName: vchanName, MsgID: []byte{1, 2, 3}, Timestamp: 30, }, diff --git a/internal/datanode/meta_service.go b/internal/datanode/meta_service.go index 8aa5f3632c..c492f85965 100644 --- a/internal/datanode/meta_service.go +++ b/internal/datanode/meta_service.go @@ -29,11 +29,11 @@ import ( "github.com/milvus-io/milvus/internal/proto/etcdpb" ) -// metaService initialize replica collections in data node from root coord. -// Initializing replica collections happens on data node starting. It depends on +// metaService initialize channel collection in data node from root coord. +// Initializing channel collection happens on data node starting. It depends on // a healthy root coord and a valid root coord grpc client. type metaService struct { - replica Replica + channel Channel collectionID UniqueID rootCoord types.RootCoord } diff --git a/internal/datanode/segment.go b/internal/datanode/segment.go new file mode 100644 index 0000000000..5ab70949a6 --- /dev/null +++ b/internal/datanode/segment.go @@ -0,0 +1,138 @@ +// 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 ( + "encoding/json" + "sync/atomic" + + "github.com/bits-and-blooms/bloom/v3" + "github.com/milvus-io/milvus-proto/go-api/schemapb" + "go.uber.org/zap" + + "github.com/milvus-io/milvus/internal/common" + "github.com/milvus-io/milvus/internal/log" + "github.com/milvus-io/milvus/internal/proto/datapb" + "github.com/milvus-io/milvus/internal/proto/internalpb" + "github.com/milvus-io/milvus/internal/storage" +) + +// Segment contains the latest segment infos from channel. +type Segment struct { + collectionID UniqueID + partitionID UniqueID + segmentID UniqueID + sType atomic.Value // datapb.SegmentType + + numRows int64 + memorySize int64 + compactedTo UniqueID + + startPos *internalpb.MsgPosition // TODO readonly + endPos *internalpb.MsgPosition + + pkFilter *bloom.BloomFilter // bloom filter of pk inside a segment + minPK primaryKey // minimal pk value, shortcut for checking whether a pk is inside this segment + maxPK primaryKey // maximal pk value, same above +} + +type addSegmentReq struct { + segType datapb.SegmentType + segID, collID, partitionID UniqueID + numOfRows int64 + startPos, endPos *internalpb.MsgPosition + statsBinLogs []*datapb.FieldBinlog + recoverTs Timestamp + importing bool +} + +func (s *Segment) updatePk(pk primaryKey) error { + if s.minPK == nil { + s.minPK = pk + } else if s.minPK.GT(pk) { + s.minPK = pk + } + + if s.maxPK == nil { + s.maxPK = pk + } else if s.maxPK.LT(pk) { + s.maxPK = pk + } + + return nil +} + +func (s *Segment) isValid() bool { + return s.getType() != datapb.SegmentType_Compacted +} + +func (s *Segment) notFlushed() bool { + return s.isValid() && s.getType() != datapb.SegmentType_Flushed +} + +func (s *Segment) getType() datapb.SegmentType { + return s.sType.Load().(datapb.SegmentType) +} + +func (s *Segment) setType(t datapb.SegmentType) { + s.sType.Store(t) +} + +func (s *Segment) updatePKRange(ids storage.FieldData) error { + switch pks := ids.(type) { + case *storage.Int64FieldData: + buf := make([]byte, 8) + for _, pk := range pks.Data { + id := storage.NewInt64PrimaryKey(pk) + err := s.updatePk(id) + if err != nil { + return err + } + common.Endian.PutUint64(buf, uint64(pk)) + s.pkFilter.Add(buf) + } + case *storage.StringFieldData: + for _, pk := range pks.Data { + id := storage.NewVarCharPrimaryKey(pk) + err := s.updatePk(id) + if err != nil { + return err + } + s.pkFilter.AddString(pk) + } + default: + //TODO:: + } + + log.Info("update pk range", + zap.Int64("collectionID", s.collectionID), zap.Int64("partitionID", s.partitionID), zap.Int64("segmentID", s.segmentID), + zap.Int64("num_rows", s.numRows), zap.Any("minPK", s.minPK), zap.Any("maxPK", s.maxPK)) + + return nil +} + +func (s *Segment) getSegmentStatslog(pkID UniqueID, pkType schemapb.DataType) ([]byte, error) { + pks := storage.PrimaryKeyStats{ + FieldID: pkID, + PkType: int64(pkType), + MaxPk: s.maxPK, + MinPk: s.minPK, + BF: s.pkFilter, + } + + return json.Marshal(pks) +} diff --git a/internal/datanode/segment_replica.go b/internal/datanode/segment_replica.go deleted file mode 100644 index 3afaf449fd..0000000000 --- a/internal/datanode/segment_replica.go +++ /dev/null @@ -1,953 +0,0 @@ -// 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" - "encoding/json" - "fmt" - "sync" - "sync/atomic" - - "github.com/bits-and-blooms/bloom/v3" - "go.uber.org/zap" - - "github.com/milvus-io/milvus-proto/go-api/schemapb" - "github.com/milvus-io/milvus/internal/common" - "github.com/milvus-io/milvus/internal/log" - "github.com/milvus-io/milvus/internal/metrics" - "github.com/milvus-io/milvus/internal/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/internalpb" - "github.com/milvus-io/milvus/internal/storage" - "github.com/milvus-io/milvus/internal/types" - "github.com/milvus-io/milvus/internal/util/typeutil" -) - -const ( - maxBloomFalsePositive float64 = 0.005 -) - -type ( - primaryKey = storage.PrimaryKey - int64PrimaryKey = storage.Int64PrimaryKey - varCharPrimaryKey = storage.VarCharPrimaryKey -) - -var ( - newInt64PrimaryKey = storage.NewInt64PrimaryKey - newVarCharPrimaryKey = storage.NewVarCharPrimaryKey -) - -// Replica is DataNode unique replication -type Replica interface { - getCollectionID() UniqueID - getCollectionSchema(collectionID UniqueID, ts Timestamp) (*schemapb.CollectionSchema, error) - getCollectionAndPartitionID(segID UniqueID) (collID, partitionID UniqueID, err error) - getChannelName(segID UniqueID) (string, error) - - listAllSegmentIDs() []UniqueID - listNotFlushedSegmentIDs() []UniqueID - addSegment(req addSegmentReq) error - listPartitionSegments(partID UniqueID) []UniqueID - filterSegments(channelName string, partitionID UniqueID) []*Segment - listNewSegmentsStartPositions() []*datapb.SegmentStartPosition - transferNewSegments(segmentIDs []UniqueID) - listSegmentsCheckPoints() map[UniqueID]segmentCheckPoint - updateSegmentEndPosition(segID UniqueID, endPos *internalpb.MsgPosition) - updateSegmentCheckPoint(segID UniqueID) - updateSegmentPKRange(segID UniqueID, ids storage.FieldData) - mergeFlushedSegments(seg *Segment, planID UniqueID, compactedFrom []UniqueID) error - hasSegment(segID UniqueID, countFlushed bool) bool - removeSegments(segID ...UniqueID) - listCompactedSegmentIDs() map[UniqueID][]UniqueID - - updateStatistics(segID UniqueID, numRows int64) - refreshFlushedSegStatistics(segID UniqueID, numRows int64) - getSegmentStatisticsUpdates(segID UniqueID) (*datapb.SegmentStats, error) - segmentFlushed(segID UniqueID) - getSegmentStatslog(segID UniqueID) ([]byte, error) - initSegmentBloomFilter(seg *Segment) error -} - -// Segment is the data structure of segments in data node replica. -type Segment struct { - collectionID UniqueID - partitionID UniqueID - segmentID UniqueID - numRows int64 - memorySize int64 - isNew atomic.Value // bool - isFlushed atomic.Value // bool - channelName string - compactedTo UniqueID - - checkPoint segmentCheckPoint - startPos *internalpb.MsgPosition // TODO readonly - endPos *internalpb.MsgPosition - - pkFilter *bloom.BloomFilter // bloom filter of pk inside a segment - minPK primaryKey // minimal pk value, shortcut for checking whether a pk is inside this segment - maxPK primaryKey // maximal pk value, same above -} - -// SegmentReplica is the data replication of persistent data in datanode. -// It implements `Replica` interface. -type SegmentReplica struct { - collectionID UniqueID - collSchema *schemapb.CollectionSchema - schemaMut sync.RWMutex - - segMu sync.RWMutex - newSegments map[UniqueID]*Segment - normalSegments map[UniqueID]*Segment - flushedSegments map[UniqueID]*Segment - compactedSegments map[UniqueID]*Segment - - metaService *metaService - chunkManager storage.ChunkManager -} - -type addSegmentReq struct { - segType datapb.SegmentType - segID, collID, partitionID UniqueID - channelName string - numOfRows int64 - startPos, endPos *internalpb.MsgPosition - statsBinLogs []*datapb.FieldBinlog - cp *segmentCheckPoint - recoverTs Timestamp - importing bool -} - -func (s *Segment) updatePk(pk primaryKey) error { - if s.minPK == nil { - s.minPK = pk - } else if s.minPK.GT(pk) { - s.minPK = pk - } - - if s.maxPK == nil { - s.maxPK = pk - } else if s.maxPK.LT(pk) { - s.maxPK = pk - } - - return nil -} - -func (s *Segment) updatePKRange(ids storage.FieldData) error { - switch pks := ids.(type) { - case *storage.Int64FieldData: - buf := make([]byte, 8) - for _, pk := range pks.Data { - id := newInt64PrimaryKey(pk) - err := s.updatePk(id) - if err != nil { - return err - } - common.Endian.PutUint64(buf, uint64(pk)) - s.pkFilter.Add(buf) - } - case *storage.StringFieldData: - for _, pk := range pks.Data { - id := newVarCharPrimaryKey(pk) - err := s.updatePk(id) - if err != nil { - return err - } - s.pkFilter.AddString(pk) - } - default: - //TODO:: - } - - log.Info("update pk range", - zap.Int64("collectionID", s.collectionID), zap.Int64("partitionID", s.partitionID), zap.Int64("segmentID", s.segmentID), - zap.String("channel", s.channelName), - zap.Int64("num_rows", s.numRows), zap.Any("minPK", s.minPK), zap.Any("maxPK", s.maxPK)) - - return nil -} - -func (s *Segment) getSegmentStatslog(pkID UniqueID, pkType schemapb.DataType) ([]byte, error) { - pks := storage.PrimaryKeyStats{ - FieldID: pkID, - PkType: int64(pkType), - MaxPk: s.maxPK, - MinPk: s.minPK, - BF: s.pkFilter, - } - - return json.Marshal(pks) -} - -var _ Replica = &SegmentReplica{} - -func newReplica(ctx context.Context, rc types.RootCoord, cm storage.ChunkManager, collID UniqueID, schema *schemapb.CollectionSchema) (*SegmentReplica, error) { - metaService := newMetaService(rc, collID) - - replica := &SegmentReplica{ - collectionID: collID, - collSchema: schema, - - newSegments: make(map[UniqueID]*Segment), - normalSegments: make(map[UniqueID]*Segment), - flushedSegments: make(map[UniqueID]*Segment), - compactedSegments: make(map[UniqueID]*Segment), - - metaService: metaService, - chunkManager: cm, - } - - return replica, nil -} - -// segmentFlushed transfers a segment from *New* or *Normal* into *Flushed*. -func (replica *SegmentReplica) segmentFlushed(segID UniqueID) { - replica.segMu.Lock() - defer replica.segMu.Unlock() - - if _, ok := replica.newSegments[segID]; ok { - replica.new2FlushedSegment(segID) - } - - if _, ok := replica.normalSegments[segID]; ok { - replica.normal2FlushedSegment(segID) - } -} - -func (replica *SegmentReplica) new2NormalSegment(segID UniqueID) { - var seg = *replica.newSegments[segID] - - seg.isNew.Store(false) - replica.normalSegments[segID] = &seg - - delete(replica.newSegments, segID) -} - -func (replica *SegmentReplica) new2FlushedSegment(segID UniqueID) { - var seg = *replica.newSegments[segID] - - seg.isNew.Store(false) - seg.isFlushed.Store(true) - replica.flushedSegments[segID] = &seg - - delete(replica.newSegments, segID) - metrics.DataNodeNumUnflushedSegments.WithLabelValues(fmt.Sprint(Params.DataNodeCfg.GetNodeID())).Dec() -} - -// normal2FlushedSegment transfers a segment from *normal* to *flushed* by changing *isFlushed* -// flag into true, and mv the segment from normalSegments map to flushedSegments map. -func (replica *SegmentReplica) normal2FlushedSegment(segID UniqueID) { - var seg = *replica.normalSegments[segID] - - seg.isFlushed.Store(true) - replica.flushedSegments[segID] = &seg - - delete(replica.normalSegments, segID) - metrics.DataNodeNumUnflushedSegments.WithLabelValues(fmt.Sprint(Params.DataNodeCfg.GetNodeID())).Dec() -} - -func (replica *SegmentReplica) getCollectionAndPartitionID(segID UniqueID) (collID, partitionID UniqueID, err error) { - replica.segMu.RLock() - defer replica.segMu.RUnlock() - - if seg, ok := replica.newSegments[segID]; ok { - return seg.collectionID, seg.partitionID, nil - } - - if seg, ok := replica.normalSegments[segID]; ok { - return seg.collectionID, seg.partitionID, nil - } - - if seg, ok := replica.flushedSegments[segID]; ok { - return seg.collectionID, seg.partitionID, nil - } - - return 0, 0, fmt.Errorf("cannot find segment, id = %v", segID) -} - -func (replica *SegmentReplica) getChannelName(segID UniqueID) (string, error) { - replica.segMu.RLock() - defer replica.segMu.RUnlock() - - if seg, ok := replica.newSegments[segID]; ok { - return seg.channelName, nil - } - - if seg, ok := replica.normalSegments[segID]; ok { - return seg.channelName, nil - } - - if seg, ok := replica.flushedSegments[segID]; ok { - return seg.channelName, nil - } - - return "", fmt.Errorf("cannot find segment, id = %v", segID) -} - -// maxRowCountPerSegment returns max row count for a segment based on estimation of row size. -func (replica *SegmentReplica) maxRowCountPerSegment(ts Timestamp) (int64, error) { - log := log.With(zap.Int64("collectionID", replica.collectionID), zap.Uint64("timpstamp", ts)) - schema, err := replica.getCollectionSchema(replica.collectionID, ts) - if err != nil { - log.Warn("failed to get collection schema", zap.Error(err)) - return 0, err - } - sizePerRecord, err := typeutil.EstimateSizePerRecord(schema) - if err != nil { - log.Warn("failed to estimate size per record", zap.Error(err)) - return 0, err - } - threshold := Params.DataCoordCfg.SegmentMaxSize * 1024 * 1024 - return int64(threshold / float64(sizePerRecord)), nil -} - -// initSegmentBloomFilter initialize segment pkFilter with a new bloom filter. -// this new BF will be initialized with estimated max rows and default false positive rate. -func (replica *SegmentReplica) initSegmentBloomFilter(s *Segment) error { - var ts Timestamp - if s.startPos != nil { - ts = s.startPos.Timestamp - } - maxRowCount, err := replica.maxRowCountPerSegment(ts) - if err != nil { - log.Warn("initSegmentBloomFilter failed, cannot estimate max row count", zap.Error(err)) - return err - } - - s.pkFilter = bloom.NewWithEstimates(uint(maxRowCount), maxBloomFalsePositive) - return nil -} - -// addSegment adds the segment to current replica. Segments can be added as *new*, *normal* or *flushed*. -// Make sure to verify `replica.hasSegment(segID)` == false before calling `replica.addSegment()`. -func (replica *SegmentReplica) addSegment(req addSegmentReq) error { - if req.collID != replica.collectionID { - log.Warn("collection mismatch", - zap.Int64("current collection ID", req.collID), - zap.Int64("expected collection ID", replica.collectionID)) - return fmt.Errorf("mismatch collection, ID=%d", req.collID) - } - log.Info("adding segment", - zap.String("segment type", req.segType.String()), - zap.Int64("segment ID", req.segID), - zap.Int64("collection ID", req.collID), - zap.Int64("partition ID", req.partitionID), - zap.String("channel name", req.channelName), - zap.Any("start position", req.startPos), - zap.Any("end position", req.endPos), - zap.Any("checkpoints", req.cp), - zap.Uint64("recover ts", req.recoverTs), - zap.Bool("importing", req.importing), - ) - seg := &Segment{ - collectionID: req.collID, - partitionID: req.partitionID, - segmentID: req.segID, - channelName: req.channelName, - numRows: req.numOfRows, // 0 if segType == NEW - } - if req.importing || req.segType == datapb.SegmentType_New { - seg.checkPoint = segmentCheckPoint{0, *req.startPos} - seg.startPos = req.startPos - seg.endPos = req.endPos - } - if req.segType == datapb.SegmentType_Normal { - if req.cp != nil { - seg.checkPoint = *req.cp - seg.endPos = &req.cp.pos - } - } - // Set up bloom filter. - err := replica.initPKBloomFilter(context.TODO(), seg, req.statsBinLogs, req.recoverTs) - if err != nil { - log.Error("failed to init bloom filter", - zap.Int64("segment ID", req.segID), - zap.Error(err)) - return err - } - // Please ignore `isNew` and `isFlushed` as they are for debugging only. - if req.segType == datapb.SegmentType_New { - seg.isNew.Store(true) - } else { - seg.isNew.Store(false) - } - if req.segType == datapb.SegmentType_Flushed { - seg.isFlushed.Store(true) - } else { - seg.isFlushed.Store(false) - } - replica.segMu.Lock() - if req.segType == datapb.SegmentType_New { - replica.newSegments[req.segID] = seg - } else if req.segType == datapb.SegmentType_Normal { - replica.normalSegments[req.segID] = seg - } else if req.segType == datapb.SegmentType_Flushed { - replica.flushedSegments[req.segID] = seg - } - replica.segMu.Unlock() - if req.segType == datapb.SegmentType_New || req.segType == datapb.SegmentType_Normal { - metrics.DataNodeNumUnflushedSegments.WithLabelValues(fmt.Sprint(Params.DataNodeCfg.GetNodeID())).Inc() - } - return nil -} - -func (replica *SegmentReplica) listCompactedSegmentIDs() map[UniqueID][]UniqueID { - replica.segMu.RLock() - defer replica.segMu.RUnlock() - - compactedTo2From := make(map[UniqueID][]UniqueID) - - for segID, seg := range replica.compactedSegments { - compactedTo2From[seg.compactedTo] = append(compactedTo2From[seg.compactedTo], segID) - } - - return compactedTo2From -} - -// filterSegments return segments with same channelName and partition ID -// get all segments -func (replica *SegmentReplica) filterSegments(channelName string, partitionID UniqueID) []*Segment { - replica.segMu.RLock() - defer replica.segMu.RUnlock() - results := make([]*Segment, 0) - - isMatched := func(segment *Segment, chanName string, partID UniqueID) bool { - return segment.channelName == chanName && (partID == common.InvalidPartitionID || segment.partitionID == partID) - } - for _, seg := range replica.newSegments { - if isMatched(seg, channelName, partitionID) { - results = append(results, seg) - } - } - for _, seg := range replica.normalSegments { - if isMatched(seg, channelName, partitionID) { - results = append(results, seg) - } - } - for _, seg := range replica.flushedSegments { - if isMatched(seg, channelName, partitionID) { - results = append(results, seg) - } - } - return results -} - -func (replica *SegmentReplica) initPKBloomFilter(ctx context.Context, s *Segment, statsBinlogs []*datapb.FieldBinlog, ts Timestamp) error { - log := log.With(zap.Int64("segmentID", s.segmentID)) - log.Info("begin to init pk bloom filter", zap.Int("stats bin logs", len(statsBinlogs))) - schema, err := replica.getCollectionSchema(s.collectionID, ts) - if err != nil { - log.Warn("failed to initPKBloomFilter, get schema return error", zap.Error(err)) - return err - } - - // get pkfield id - pkField := int64(-1) - for _, field := range schema.Fields { - if field.IsPrimaryKey { - pkField = field.FieldID - break - } - } - - // filter stats binlog files which is pk field stats log - var bloomFilterFiles []string - for _, binlog := range statsBinlogs { - if binlog.FieldID != pkField { - continue - } - for _, log := range binlog.GetBinlogs() { - bloomFilterFiles = append(bloomFilterFiles, log.GetLogPath()) - } - } - - // no stats log to parse, initialize a new BF - if len(bloomFilterFiles) == 0 { - log.Warn("no stats files to load, initializa a new one") - return replica.initSegmentBloomFilter(s) - } - - values, err := replica.chunkManager.MultiRead(ctx, bloomFilterFiles) - if err != nil { - log.Warn("failed to load bloom filter files", zap.Error(err)) - return err - } - blobs := make([]*Blob, 0) - for i := 0; i < len(values); i++ { - blobs = append(blobs, &Blob{Value: values[i]}) - } - - stats, err := storage.DeserializeStats(blobs) - if err != nil { - log.Warn("failed to deserialize bloom filter files", zap.Error(err)) - return err - } - for _, stat := range stats { - // use first BF to merge - if s.pkFilter == nil { - s.pkFilter = stat.BF - } else { - // for compatibility, statslog before 2.1.2 uses separated stats log which needs to be merged - // assuming all legacy BF has same attributes. - err = s.pkFilter.Merge(stat.BF) - if err != nil { - return err - } - } - - s.updatePk(stat.MinPk) - s.updatePk(stat.MaxPk) - } - - return nil -} - -// listNewSegmentsStartPositions gets all *New Segments* start positions and -// transfer segments states from *New* to *Normal*. -func (replica *SegmentReplica) listNewSegmentsStartPositions() []*datapb.SegmentStartPosition { - replica.segMu.Lock() - defer replica.segMu.Unlock() - - result := make([]*datapb.SegmentStartPosition, 0, len(replica.newSegments)) - for id, seg := range replica.newSegments { - result = append(result, &datapb.SegmentStartPosition{ - SegmentID: id, - StartPosition: seg.startPos, - }) - } - return result -} - -// transferNewSegments make new segment transfer to normal segments. -func (replica *SegmentReplica) transferNewSegments(segmentIDs []UniqueID) { - replica.segMu.Lock() - defer replica.segMu.Unlock() - - for _, segmentID := range segmentIDs { - replica.new2NormalSegment(segmentID) - } -} - -// listSegmentsCheckPoints gets check points from both *New* and *Normal* segments. -func (replica *SegmentReplica) listSegmentsCheckPoints() map[UniqueID]segmentCheckPoint { - replica.segMu.RLock() - defer replica.segMu.RUnlock() - - result := make(map[UniqueID]segmentCheckPoint) - - for id, seg := range replica.newSegments { - result[id] = seg.checkPoint - } - - for id, seg := range replica.normalSegments { - result[id] = seg.checkPoint - } - - return result -} - -// updateSegmentEndPosition updates *New* or *Normal* segment's end position. -func (replica *SegmentReplica) updateSegmentEndPosition(segID UniqueID, endPos *internalpb.MsgPosition) { - replica.segMu.RLock() - defer replica.segMu.RUnlock() - - seg, ok := replica.newSegments[segID] - if ok { - seg.endPos = endPos - return - } - - seg, ok = replica.normalSegments[segID] - if ok { - seg.endPos = endPos - return - } - - log.Warn("No match segment", zap.Int64("ID", segID)) -} - -func (replica *SegmentReplica) updateSegmentPKRange(segID UniqueID, ids storage.FieldData) { - replica.segMu.Lock() - defer replica.segMu.Unlock() - - seg, ok := replica.newSegments[segID] - if ok { - seg.updatePKRange(ids) - return - } - - seg, ok = replica.normalSegments[segID] - if ok { - seg.updatePKRange(ids) - return - } - - seg, ok = replica.flushedSegments[segID] - if ok { - seg.updatePKRange(ids) - return - } - - log.Warn("No match segment to update PK range", zap.Int64("ID", segID)) -} - -func (replica *SegmentReplica) removeSegments(segIDs ...UniqueID) { - replica.segMu.Lock() - defer replica.segMu.Unlock() - - log.Info("remove segments if exist", zap.Int64s("segmentIDs", segIDs)) - cnt := 0 - for _, segID := range segIDs { - if _, ok := replica.newSegments[segID]; ok { - cnt++ - } else if _, ok := replica.normalSegments[segID]; ok { - cnt++ - } - } - metrics.DataNodeNumUnflushedSegments.WithLabelValues(fmt.Sprint(Params.DataNodeCfg.GetNodeID())).Sub(float64(cnt)) - - for _, segID := range segIDs { - delete(replica.newSegments, segID) - delete(replica.normalSegments, segID) - delete(replica.flushedSegments, segID) - delete(replica.compactedSegments, segID) - } -} - -// hasSegment checks whether this replica has a segment according to segment ID. -func (replica *SegmentReplica) hasSegment(segID UniqueID, countFlushed bool) bool { - replica.segMu.RLock() - defer replica.segMu.RUnlock() - - _, inNew := replica.newSegments[segID] - _, inNormal := replica.normalSegments[segID] - - inFlush := false - if countFlushed { - _, inFlush = replica.flushedSegments[segID] - } - - return inNew || inNormal || inFlush -} -func (replica *SegmentReplica) refreshFlushedSegStatistics(segID UniqueID, numRows int64) { - replica.segMu.Lock() - defer replica.segMu.Unlock() - - if seg, ok := replica.flushedSegments[segID]; ok { - seg.memorySize = 0 - seg.numRows = numRows - return - } - - log.Warn("refresh numRow on not exists segment", zap.Int64("segID", segID)) -} - -// updateStatistics updates the number of rows of a segment in replica. -func (replica *SegmentReplica) updateStatistics(segID UniqueID, numRows int64) { - replica.segMu.Lock() - defer replica.segMu.Unlock() - - log.Info("updating segment", zap.Int64("Segment ID", segID), zap.Int64("numRows", numRows)) - if seg, ok := replica.newSegments[segID]; ok { - seg.memorySize = 0 - seg.numRows += numRows - return - } - - if seg, ok := replica.normalSegments[segID]; ok { - seg.memorySize = 0 - seg.numRows += numRows - return - } - - log.Warn("update segment num row not exist", zap.Int64("segID", segID)) -} - -// getSegmentStatisticsUpdates gives current segment's statistics updates. -func (replica *SegmentReplica) getSegmentStatisticsUpdates(segID UniqueID) (*datapb.SegmentStats, error) { - replica.segMu.RLock() - defer replica.segMu.RUnlock() - updates := &datapb.SegmentStats{SegmentID: segID} - - if seg, ok := replica.newSegments[segID]; ok { - updates.NumRows = seg.numRows - return updates, nil - } - - if seg, ok := replica.normalSegments[segID]; ok { - updates.NumRows = seg.numRows - return updates, nil - } - - if seg, ok := replica.flushedSegments[segID]; ok { - updates.NumRows = seg.numRows - return updates, nil - } - - return nil, fmt.Errorf("error, there's no segment %v", segID) -} - -// --- collection --- -func (replica *SegmentReplica) getCollectionID() UniqueID { - return replica.collectionID -} - -// getCollectionSchema gets collection schema from rootcoord for a certain timestamp. -// If you want the latest collection schema, ts should be 0. -func (replica *SegmentReplica) getCollectionSchema(collID UniqueID, ts Timestamp) (*schemapb.CollectionSchema, error) { - if !replica.validCollection(collID) { - return nil, fmt.Errorf("mismatch collection, want %d, actual %d", replica.collectionID, collID) - } - - replica.schemaMut.RLock() - if replica.collSchema == nil { - replica.schemaMut.RUnlock() - - replica.schemaMut.Lock() - defer replica.schemaMut.Unlock() - if replica.collSchema == nil { - sch, err := replica.metaService.getCollectionSchema(context.Background(), collID, ts) - if err != nil { - return nil, err - } - replica.collSchema = sch - } - } else { - defer replica.schemaMut.RUnlock() - } - - return replica.collSchema, nil -} - -func (replica *SegmentReplica) validCollection(collID UniqueID) bool { - return collID == replica.collectionID -} - -// updateSegmentCheckPoint is called when auto flush or mannul flush is done. -func (replica *SegmentReplica) updateSegmentCheckPoint(segID UniqueID) { - replica.segMu.Lock() - defer replica.segMu.Unlock() - - if seg, ok := replica.newSegments[segID]; ok { - seg.checkPoint = segmentCheckPoint{seg.numRows, *seg.endPos} - return - } - - if seg, ok := replica.normalSegments[segID]; ok { - seg.checkPoint = segmentCheckPoint{seg.numRows, *seg.endPos} - return - } - - log.Warn("There's no segment", zap.Int64("ID", segID)) -} - -func (replica *SegmentReplica) mergeFlushedSegments(seg *Segment, planID UniqueID, compactedFrom []UniqueID) error { - - log := log.With( - zap.Int64("segment ID", seg.segmentID), - zap.Int64("collection ID", seg.collectionID), - zap.Int64("partition ID", seg.partitionID), - zap.Int64s("compacted from", compactedFrom), - zap.Int64("planID", planID), - zap.String("channel name", seg.channelName)) - - if seg.collectionID != replica.collectionID { - log.Warn("Mismatch collection", - zap.Int64("expected collectionID", replica.collectionID)) - return fmt.Errorf("mismatch collection, ID=%d", seg.collectionID) - } - - var inValidSegments []UniqueID - for _, ID := range compactedFrom { - // no such segments in replica or the segments are unflushed. - if !replica.hasSegment(ID, true) || replica.hasSegment(ID, false) { - inValidSegments = append(inValidSegments, ID) - } - } - - if len(inValidSegments) > 0 { - log.Warn("no match flushed segments to merge from", zap.Int64s("invalid segmentIDs", inValidSegments)) - return fmt.Errorf("invalid compactedFrom segments: %v", inValidSegments) - } - - replica.segMu.Lock() - log.Info("merge flushed segments") - for _, ID := range compactedFrom { - // the existent of the segments are already checked - s := replica.flushedSegments[ID] - - s.compactedTo = seg.segmentID - replica.compactedSegments[ID] = s - delete(replica.flushedSegments, ID) - } - replica.segMu.Unlock() - - // only store segments with numRows > 0 - if seg.numRows > 0 { - seg.isNew.Store(false) - seg.isFlushed.Store(true) - - replica.segMu.Lock() - replica.flushedSegments[seg.segmentID] = seg - replica.segMu.Unlock() - } - - return nil -} - -// for tests only -func (replica *SegmentReplica) addFlushedSegmentWithPKs(segID, collID, partID UniqueID, channelName string, numOfRows int64, ids storage.FieldData) error { - if collID != replica.collectionID { - log.Warn("Mismatch collection", - zap.Int64("input ID", collID), - zap.Int64("expected ID", replica.collectionID)) - return fmt.Errorf("mismatch collection, ID=%d", collID) - } - - log.Info("Add Flushed segment", - zap.Int64("segment ID", segID), - zap.Int64("collection ID", collID), - zap.Int64("partition ID", partID), - zap.String("channel name", channelName), - ) - - seg := &Segment{ - collectionID: collID, - partitionID: partID, - segmentID: segID, - channelName: channelName, - numRows: numOfRows, - } - - err := replica.initSegmentBloomFilter(seg) - if err != nil { - return err - } - - seg.updatePKRange(ids) - - seg.isNew.Store(false) - seg.isFlushed.Store(true) - - replica.segMu.Lock() - replica.flushedSegments[segID] = seg - replica.segMu.Unlock() - - return nil -} - -func (replica *SegmentReplica) listAllSegmentIDs() []UniqueID { - replica.segMu.RLock() - defer replica.segMu.RUnlock() - - var segIDs []UniqueID - - for _, seg := range replica.newSegments { - segIDs = append(segIDs, seg.segmentID) - } - - for _, seg := range replica.normalSegments { - segIDs = append(segIDs, seg.segmentID) - } - - for _, seg := range replica.flushedSegments { - segIDs = append(segIDs, seg.segmentID) - } - - return segIDs -} - -func (replica *SegmentReplica) listPartitionSegments(partID UniqueID) []UniqueID { - replica.segMu.RLock() - defer replica.segMu.RUnlock() - - var segIDs []UniqueID - - for _, seg := range replica.newSegments { - if seg.partitionID == partID { - segIDs = append(segIDs, seg.segmentID) - } - } - - for _, seg := range replica.normalSegments { - if seg.partitionID == partID { - segIDs = append(segIDs, seg.segmentID) - } - } - - for _, seg := range replica.flushedSegments { - if seg.partitionID == partID { - segIDs = append(segIDs, seg.segmentID) - } - } - - return segIDs -} - -func (replica *SegmentReplica) listNotFlushedSegmentIDs() []UniqueID { - replica.segMu.RLock() - defer replica.segMu.RUnlock() - - var segIDs []UniqueID - - for _, seg := range replica.newSegments { - segIDs = append(segIDs, seg.segmentID) - } - - for _, seg := range replica.normalSegments { - segIDs = append(segIDs, seg.segmentID) - } - - return segIDs -} - -// getSegmentStatslog returns the segment statslog for the provided segment id. -func (replica *SegmentReplica) getSegmentStatslog(segID UniqueID) ([]byte, error) { - replica.segMu.RLock() - defer replica.segMu.RUnlock() - colID := replica.getCollectionID() - - schema, err := replica.getCollectionSchema(colID, 0) - if err != nil { - return nil, err - } - - var pkID UniqueID - var pkType schemapb.DataType - for _, field := range schema.GetFields() { - if field.GetIsPrimaryKey() { - pkID = field.GetFieldID() - pkType = field.GetDataType() - } - } - - if seg, ok := replica.newSegments[segID]; ok { - return seg.getSegmentStatslog(pkID, pkType) - } - - if seg, ok := replica.normalSegments[segID]; ok { - return seg.getSegmentStatslog(pkID, pkType) - } - - if seg, ok := replica.flushedSegments[segID]; ok { - return seg.getSegmentStatslog(pkID, pkType) - } - - return nil, fmt.Errorf("segment not found: %d", segID) -} diff --git a/internal/datanode/segment_replica_test.go b/internal/datanode/segment_replica_test.go deleted file mode 100644 index 9ad77565a7..0000000000 --- a/internal/datanode/segment_replica_test.go +++ /dev/null @@ -1,1320 +0,0 @@ -// 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" - "encoding/json" - "errors" - "fmt" - "math/rand" - "testing" - "time" - - "github.com/bits-and-blooms/bloom/v3" - "github.com/samber/lo" - "github.com/stretchr/testify/assert" - "github.com/stretchr/testify/require" - "github.com/stretchr/testify/suite" - - "github.com/milvus-io/milvus-proto/go-api/schemapb" - "github.com/milvus-io/milvus/internal/common" - "github.com/milvus-io/milvus/internal/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/internalpb" - "github.com/milvus-io/milvus/internal/storage" -) - -var segmentReplicaNodeTestDir = "/tmp/milvus_test/segment_replica" - -func TestNewReplica(t *testing.T) { - rc := &RootCoordFactory{} - cm := storage.NewLocalChunkManager(storage.RootPath(segmentReplicaNodeTestDir)) - defer cm.RemoveWithPrefix(context.Background(), "") - replica, err := newReplica(context.Background(), rc, cm, 0, nil) - assert.Nil(t, err) - assert.NotNil(t, replica) -} - -type mockDataCM struct { - storage.ChunkManager -} - -func (kv *mockDataCM) MultiRead(ctx context.Context, keys []string) ([][]byte, error) { - stats := &storage.PrimaryKeyStats{ - FieldID: common.RowIDField, - Min: 0, - Max: 10, - BF: bloom.NewWithEstimates(100000, maxBloomFalsePositive), - } - buffer, _ := json.Marshal(stats) - return [][]byte{buffer}, nil -} - -type mockPkfilterMergeError struct { - storage.ChunkManager -} - -func (kv *mockPkfilterMergeError) MultiRead(ctx context.Context, keys []string) ([][]byte, error) { - /* - stats := &storage.PrimaryKeyStats{ - FieldID: common.RowIDField, - Min: 0, - Max: 10, - BF: bloom.NewWithEstimates(1, 0.0001), - } - buffer, _ := json.Marshal(stats) - return [][]byte{buffer}, nil*/ - return nil, errors.New("mocked multi read error") -} - -type mockDataCMError struct { - storage.ChunkManager -} - -func (kv *mockDataCMError) MultiRead(ctx context.Context, keys []string) ([][]byte, error) { - return nil, fmt.Errorf("mock error") -} - -type mockDataCMStatsError struct { - storage.ChunkManager -} - -func (kv *mockDataCMStatsError) MultiRead(ctx context.Context, keys []string) ([][]byte, error) { - return [][]byte{[]byte("3123123,error,test")}, nil -} - -func getSimpleFieldBinlog() *datapb.FieldBinlog { - return &datapb.FieldBinlog{ - FieldID: 106, - Binlogs: []*datapb.Binlog{{LogPath: "test"}}, - } -} - -func TestSegmentReplica_getChannelName(t *testing.T) { - var ( - channelName = "TestSegmentReplica_getChannelName" - newSegments = map[UniqueID]*Segment{ - 100: {channelName: channelName}, - 101: {channelName: channelName}, - 102: {channelName: channelName}, - } - normalSegments = map[UniqueID]*Segment{ - 200: {channelName: channelName}, - 201: {channelName: channelName}, - 202: {channelName: channelName}, - } - flushedSegments = map[UniqueID]*Segment{ - 300: {channelName: channelName}, - 301: {channelName: channelName}, - 302: {channelName: channelName}, - } - ) - - sr := &SegmentReplica{ - newSegments: newSegments, - normalSegments: normalSegments, - flushedSegments: flushedSegments, - } - - tests := []struct { - description string - - seg UniqueID - ifExist bool - }{ - {"100 exists in new segments", 100, true}, - {"201 exists in normal segments", 201, true}, - {"302 exists in flushed segments", 302, true}, - {"400 not exists in all segments", 400, false}, - } - for _, test := range tests { - t.Run(test.description, func(t *testing.T) { - chanName, err := sr.getChannelName(test.seg) - if test.ifExist { - assert.NoError(t, err) - assert.Equal(t, channelName, chanName) - } else { - assert.Error(t, err) - assert.Empty(t, chanName) - } - }) - } -} - -func TestSegmentReplica_getCollectionAndPartitionID(te *testing.T) { - tests := []struct { - segInNew UniqueID - segInNormal UniqueID - segInFlushed UniqueID - - inCollID UniqueID - inParID UniqueID - description string - }{ - {100, 0, 0, 1, 10, "Segment 100 in NewSegments"}, - {0, 200, 0, 2, 20, "Segment 200 in NormalSegments"}, - {0, 0, 300, 3, 30, "Segment 300 in FlushedSegments"}, - {0, 0, 0, 4, 40, "No Segment in replica"}, - } - - for _, test := range tests { - te.Run(test.description, func(t *testing.T) { - if test.segInNew != 0 { - sr := &SegmentReplica{ - newSegments: map[UniqueID]*Segment{ - test.segInNew: { - collectionID: test.inCollID, - partitionID: test.inParID, - segmentID: test.segInNew, - }}, - } - - collID, parID, err := sr.getCollectionAndPartitionID(test.segInNew) - assert.NoError(t, err) - assert.Equal(t, test.inCollID, collID) - assert.Equal(t, test.inParID, parID) - } else if test.segInNormal != 0 { - sr := &SegmentReplica{ - normalSegments: map[UniqueID]*Segment{ - test.segInNormal: { - collectionID: test.inCollID, - partitionID: test.inParID, - segmentID: test.segInNormal, - }}, - } - - collID, parID, err := sr.getCollectionAndPartitionID(test.segInNormal) - assert.NoError(t, err) - assert.Equal(t, test.inCollID, collID) - assert.Equal(t, test.inParID, parID) - } else if test.segInFlushed != 0 { - sr := &SegmentReplica{ - flushedSegments: map[UniqueID]*Segment{ - test.segInFlushed: { - collectionID: test.inCollID, - partitionID: test.inParID, - segmentID: test.segInFlushed, - }}, - } - - collID, parID, err := sr.getCollectionAndPartitionID(test.segInFlushed) - assert.NoError(t, err) - assert.Equal(t, test.inCollID, collID) - assert.Equal(t, test.inParID, parID) - } else { - sr := &SegmentReplica{} - collID, parID, err := sr.getCollectionAndPartitionID(1000) - assert.Error(t, err) - assert.Zero(t, collID) - assert.Zero(t, parID) - } - }) - } -} - -func TestSegmentReplica(t *testing.T) { - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() - rc := &RootCoordFactory{ - pkType: schemapb.DataType_Int64, - } - collID := UniqueID(1) - cm := storage.NewLocalChunkManager(storage.RootPath(segmentReplicaNodeTestDir)) - defer cm.RemoveWithPrefix(ctx, "") - - t.Run("Test coll mot match", func(t *testing.T) { - replica, err := newReplica(context.Background(), rc, cm, collID, nil) - assert.Nil(t, err) - err = replica.addSegment( - addSegmentReq{ - segType: datapb.SegmentType_New, - segID: 1, - collID: collID + 1, - partitionID: 0, - channelName: "", - startPos: nil, - endPos: nil, - }) - assert.NotNil(t, err) - }) - - t.Run("Test segmentFlushed", func(t *testing.T) { - testReplica := &SegmentReplica{ - newSegments: make(map[UniqueID]*Segment), - normalSegments: make(map[UniqueID]*Segment), - flushedSegments: make(map[UniqueID]*Segment), - } - - type Test struct { - inisNew bool - inisFlushed bool - inSegID UniqueID - - expectedisNew bool - expectedisFlushed bool - expectedSegID UniqueID - } - - tests := []Test{ - // new segment - {true, false, 1, false, true, 1}, - {true, false, 2, false, true, 2}, - {true, false, 3, false, true, 3}, - // normal segment - {false, false, 10, false, true, 10}, - {false, false, 20, false, true, 20}, - {false, false, 30, false, true, 30}, - // flushed segment - {false, true, 100, false, true, 100}, - {false, true, 200, false, true, 200}, - {false, true, 300, false, true, 300}, - } - - newSeg := func(sr *SegmentReplica, isNew, isFlushed bool, id UniqueID) { - ns := &Segment{segmentID: id} - ns.isNew.Store(isNew) - ns.isFlushed.Store(isFlushed) - - if isNew && !isFlushed { - sr.newSegments[id] = ns - return - } - - if !isNew && !isFlushed { - sr.normalSegments[id] = ns - return - } - - if !isNew && isFlushed { - sr.flushedSegments[id] = ns - return - } - } - - for _, te := range tests { - // prepare case - newSeg(testReplica, te.inisNew, te.inisFlushed, te.inSegID) - - testReplica.segmentFlushed(te.inSegID) - - flushedSeg := testReplica.flushedSegments[te.inSegID] - assert.Equal(t, te.expectedSegID, flushedSeg.segmentID) - assert.Equal(t, te.expectedisNew, flushedSeg.isNew.Load().(bool)) - assert.Equal(t, te.expectedisFlushed, flushedSeg.isFlushed.Load().(bool)) - } - - }) -} - -func TestSegmentReplica_InterfaceMethod(t *testing.T) { - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() - rc := &RootCoordFactory{ - pkType: schemapb.DataType_Int64, - } - cm := storage.NewLocalChunkManager(storage.RootPath(segmentReplicaNodeTestDir)) - defer cm.RemoveWithPrefix(ctx, "") - - t.Run("Test addFlushedSegmentWithPKs", func(t *testing.T) { - tests := []struct { - isvalid bool - - incollID UniqueID - replicaCollID UniqueID - description string - }{ - {true, 1, 1, "valid input collection with replica collection"}, - {false, 1, 2, "invalid input collection with replica collection"}, - } - - primaryKeyData := &storage.Int64FieldData{ - Data: []int64{9}, - } - for _, test := range tests { - t.Run(test.description, func(t *testing.T) { - replica, err := newReplica(context.TODO(), rc, cm, test.replicaCollID, nil) - require.NoError(t, err) - if test.isvalid { - replica.addFlushedSegmentWithPKs(100, test.incollID, 10, "a", 1, primaryKeyData) - - assert.True(t, replica.hasSegment(100, true)) - assert.False(t, replica.hasSegment(100, false)) - } else { - replica.addFlushedSegmentWithPKs(100, test.incollID, 10, "a", 1, primaryKeyData) - assert.False(t, replica.hasSegment(100, true)) - assert.False(t, replica.hasSegment(100, false)) - } - }) - } - }) - - t.Run("Test_addNewSegment", func(t *testing.T) { - tests := []struct { - isValidCase bool - replicaCollID UniqueID - inCollID UniqueID - inSegID UniqueID - - instartPos *internalpb.MsgPosition - - expectdIsNew bool - expectedIsFlushed bool - - description string - }{ - {isValidCase: false, replicaCollID: 1, inCollID: 2, inSegID: 300, description: "input CollID 2 mismatch with Replica collID"}, - {true, 1, 1, 200, new(internalpb.MsgPosition), true, false, "nill address for startPos"}, - {true, 1, 1, 200, &internalpb.MsgPosition{}, true, false, "empty struct for startPos"}, - } - - for _, test := range tests { - t.Run(test.description, func(t *testing.T) { - sr, err := newReplica(context.Background(), rc, cm, test.replicaCollID, nil) - assert.Nil(t, err) - require.False(t, sr.hasSegment(test.inSegID, true)) - err = sr.addSegment( - addSegmentReq{ - segType: datapb.SegmentType_New, - segID: test.inSegID, - collID: test.inCollID, - partitionID: 1, - channelName: "", - startPos: test.instartPos, - endPos: &internalpb.MsgPosition{}, - }) - if test.isValidCase { - assert.NoError(t, err) - assert.True(t, sr.hasSegment(test.inSegID, true)) - assert.Equal(t, test.expectdIsNew, sr.newSegments[test.inSegID].isNew.Load().(bool)) - assert.Equal(t, test.expectedIsFlushed, sr.newSegments[test.inSegID].isFlushed.Load().(bool)) - } else { - assert.Error(t, err) - assert.False(t, sr.hasSegment(test.inSegID, true)) - } - }) - } - }) - - t.Run("Test_addNormalSegment", func(t *testing.T) { - tests := []struct { - isValidCase bool - replicaCollID UniqueID - inCollID UniqueID - inSegID UniqueID - - expectdIsNew bool - expectedIsFlushed bool - - description string - }{ - {isValidCase: false, replicaCollID: 1, inCollID: 2, inSegID: 300, description: "input CollID 2 mismatch with Replica collID"}, - {true, 1, 1, 200, false, false, "normal case"}, - } - - for _, test := range tests { - t.Run(test.description, func(t *testing.T) { - sr, err := newReplica(context.Background(), rc, &mockDataCM{}, test.replicaCollID, nil) - assert.Nil(t, err) - require.False(t, sr.hasSegment(test.inSegID, true)) - err = sr.addSegment( - addSegmentReq{ - segType: datapb.SegmentType_Normal, - segID: test.inSegID, - collID: test.inCollID, - partitionID: 1, - channelName: "", - numOfRows: 0, - statsBinLogs: []*datapb.FieldBinlog{getSimpleFieldBinlog()}, - cp: &segmentCheckPoint{}, - recoverTs: 0, - }) - if test.isValidCase { - assert.NoError(t, err) - assert.True(t, sr.hasSegment(test.inSegID, true)) - assert.Equal(t, test.expectdIsNew, sr.normalSegments[test.inSegID].isNew.Load().(bool)) - assert.Equal(t, test.expectedIsFlushed, sr.normalSegments[test.inSegID].isFlushed.Load().(bool)) - } else { - assert.Error(t, err) - assert.False(t, sr.hasSegment(test.inSegID, true)) - } - }) - } - }) - - t.Run("Test_addNormalSegmentWithNilDml", func(t *testing.T) { - sr, err := newReplica(context.Background(), rc, &mockDataCM{}, 1, nil) - require.NoError(t, err) - segID := int64(101) - require.False(t, sr.hasSegment(segID, true)) - assert.NotPanics(t, func() { - err = sr.addSegment( - addSegmentReq{ - segType: datapb.SegmentType_Normal, - segID: segID, - collID: 1, - partitionID: 10, - channelName: "empty_dml_chan", - numOfRows: 0, - statsBinLogs: []*datapb.FieldBinlog{}, - cp: nil, - recoverTs: 0, - }) - assert.NoError(t, err) - }) - }) - - t.Run("Test_listSegmentsCheckPoints", func(t *testing.T) { - tests := []struct { - newSegID UniqueID - newSegCP *segmentCheckPoint - - normalSegID UniqueID - normalSegCP *segmentCheckPoint - - flushedSegID UniqueID - flushedSegCP *segmentCheckPoint - - description string - }{ - {newSegID: 100, newSegCP: new(segmentCheckPoint), - description: "Only contain new Seg 100"}, - {normalSegID: 200, normalSegCP: new(segmentCheckPoint), - description: "Only contain normal Seg 200"}, - {flushedSegID: 300, flushedSegCP: new(segmentCheckPoint), - description: "Only contain flushed Seg 300"}, - {100, new(segmentCheckPoint), 200, new(segmentCheckPoint), 0, new(segmentCheckPoint), - "New seg 100 and normal seg 200"}, - {100, new(segmentCheckPoint), 0, new(segmentCheckPoint), 300, new(segmentCheckPoint), - "New seg 100 and flushed seg 300"}, - {0, new(segmentCheckPoint), 200, new(segmentCheckPoint), 300, new(segmentCheckPoint), - "Normal seg 200 and flushed seg 300"}, - {100, new(segmentCheckPoint), 200, new(segmentCheckPoint), 300, new(segmentCheckPoint), - "New seg 100, normal seg 200 and flushed seg 300"}, - } - - for _, test := range tests { - t.Run(test.description, func(t *testing.T) { - sr := SegmentReplica{ - newSegments: make(map[UniqueID]*Segment), - normalSegments: make(map[UniqueID]*Segment), - flushedSegments: make(map[UniqueID]*Segment), - } - - expectdCount := 0 - if test.newSegID != 0 { - sr.newSegments[test.newSegID] = &Segment{checkPoint: *test.newSegCP} - expectdCount++ - } - if test.normalSegID != 0 { - sr.normalSegments[test.normalSegID] = &Segment{checkPoint: *test.normalSegCP} - expectdCount++ - } - if test.flushedSegID != 0 { - sr.flushedSegments[test.flushedSegID] = &Segment{checkPoint: *test.flushedSegCP} - } - - scp := sr.listSegmentsCheckPoints() - assert.Equal(t, expectdCount, len(scp)) - }) - } - }) - - t.Run("Test_updateSegmentEndPosition", func(t *testing.T) { - tests := []struct { - newSegID UniqueID - normalSegID UniqueID - flushedSegID UniqueID - - inSegID UniqueID - description string - }{ - {newSegID: 100, inSegID: 100, - description: "input seg 100 in newSegments"}, - {newSegID: 100, inSegID: 101, - description: "input seg 101 not in newSegments"}, - {normalSegID: 200, inSegID: 200, - description: "input seg 200 in normalSegments"}, - {normalSegID: 200, inSegID: 201, - description: "input seg 201 not in normalSegments"}, - {flushedSegID: 300, inSegID: 300, - description: "input seg 300 in flushedSegments"}, - {flushedSegID: 300, inSegID: 301, - description: "input seg 301 not in flushedSegments"}, - } - - for _, test := range tests { - t.Run(test.description, func(t *testing.T) { - sr := SegmentReplica{ - newSegments: make(map[UniqueID]*Segment), - normalSegments: make(map[UniqueID]*Segment), - flushedSegments: make(map[UniqueID]*Segment), - } - - if test.newSegID != 0 { - sr.newSegments[test.newSegID] = &Segment{} - } - if test.normalSegID != 0 { - sr.normalSegments[test.normalSegID] = &Segment{} - } - if test.flushedSegID != 0 { - sr.flushedSegments[test.flushedSegID] = &Segment{} - } - sr.updateSegmentEndPosition(test.inSegID, new(internalpb.MsgPosition)) - sr.removeSegments(0) - - }) - } - }) - - t.Run("Test_updateStatistics", func(t *testing.T) { - tests := []struct { - isvalidCase bool - - newSegID UniqueID - normalSegID UniqueID - flushedSegID UniqueID - - inSegID UniqueID - inNumRows int64 - description string - }{ - {isvalidCase: true, newSegID: 100, inSegID: 100, inNumRows: 100, - description: "input seg 100 in newSegments with numRows 100"}, - {isvalidCase: false, newSegID: 100, inSegID: 101, inNumRows: 100, - description: "input seg 101 not in newSegments with numRows 100"}, - {isvalidCase: true, normalSegID: 200, inSegID: 200, inNumRows: 200, - description: "input seg 200 in normalSegments with numRows 200"}, - {isvalidCase: false, normalSegID: 200, inSegID: 201, inNumRows: 200, - description: "input seg 201 not in normalSegments with numRows 200"}, - {isvalidCase: true, flushedSegID: 300, inSegID: 300, inNumRows: 300, - description: "input seg 300 in flushedSegments"}, - {isvalidCase: false, flushedSegID: 300, inSegID: 301, inNumRows: 300, - description: "input seg 301 not in flushedSegments"}, - } - for _, test := range tests { - t.Run(test.description, func(t *testing.T) { - sr := SegmentReplica{ - newSegments: make(map[UniqueID]*Segment), - normalSegments: make(map[UniqueID]*Segment), - flushedSegments: make(map[UniqueID]*Segment), - } - - if test.newSegID != 0 { - sr.newSegments[test.newSegID] = &Segment{} - } - if test.normalSegID != 0 { - sr.normalSegments[test.normalSegID] = &Segment{} - } - if test.flushedSegID != 0 { // not update flushed num rows - sr.flushedSegments[test.flushedSegID] = &Segment{ - numRows: test.inNumRows, - } - } - - sr.updateStatistics(test.inSegID, test.inNumRows) - if test.isvalidCase { - - updates, err := sr.getSegmentStatisticsUpdates(test.inSegID) - assert.NoError(t, err) - assert.Equal(t, test.inNumRows, updates.GetNumRows()) - assert.Equal(t, test.inSegID, updates.GetSegmentID()) - - sr.updateSegmentCheckPoint(10000) - } else { - updates, err := sr.getSegmentStatisticsUpdates(test.inSegID) - assert.Error(t, err) - assert.Nil(t, updates) - } - }) - } - }) - - t.Run("Test_getCollectionSchema", func(t *testing.T) { - tests := []struct { - isValid bool - replicaCollID UniqueID - inputCollID UniqueID - - metaServiceErr bool - description string - }{ - {true, 1, 1, false, "Normal case"}, - {false, 1, 2, false, "Input collID 2 mismatch with replicaCollID 1"}, - {false, 1, 1, true, "RPC call fails"}, - } - - for _, test := range tests { - t.Run(test.description, func(t *testing.T) { - sr, err := newReplica(context.Background(), rc, cm, test.replicaCollID, nil) - assert.Nil(t, err) - - if test.metaServiceErr { - sr.collSchema = nil - rc.setCollectionID(-1) - } else { - rc.setCollectionID(1) - } - - s, err := sr.getCollectionSchema(test.inputCollID, Timestamp(0)) - if test.isValid { - assert.NoError(t, err) - assert.NotNil(t, s) - } else { - assert.Error(t, err) - assert.Nil(t, s) - } - }) - } - rc.setCollectionID(1) - }) - - t.Run("Test listAllSegmentIDs", func(t *testing.T) { - sr := &SegmentReplica{ - newSegments: map[UniqueID]*Segment{1: {segmentID: 1}}, - normalSegments: map[UniqueID]*Segment{2: {segmentID: 2}}, - flushedSegments: map[UniqueID]*Segment{3: {segmentID: 3}}, - } - - ids := sr.listAllSegmentIDs() - assert.ElementsMatch(t, []UniqueID{1, 2, 3}, ids) - }) - - t.Run("Test listPartitionSegments", func(t *testing.T) { - sr := &SegmentReplica{ - newSegments: map[UniqueID]*Segment{1: {segmentID: 1, partitionID: 1}, 4: {segmentID: 4, partitionID: 2}}, - normalSegments: map[UniqueID]*Segment{2: {segmentID: 2, partitionID: 1}, 5: {segmentID: 5, partitionID: 2}}, - flushedSegments: map[UniqueID]*Segment{3: {segmentID: 3, partitionID: 1}, 6: {segmentID: 6, partitionID: 2}}, - } - - ids := sr.listPartitionSegments(1) - assert.ElementsMatch(t, []UniqueID{1, 2, 3}, ids) - }) - - t.Run("Test_addSegmentMinIOLoadError", func(t *testing.T) { - sr, err := newReplica(context.Background(), rc, cm, 1, nil) - assert.Nil(t, err) - sr.chunkManager = &mockDataCMError{} - - cpPos := &internalpb.MsgPosition{ChannelName: "insert-01", Timestamp: Timestamp(10)} - cp := &segmentCheckPoint{int64(10), *cpPos} - err = sr.addSegment( - addSegmentReq{ - segType: datapb.SegmentType_Normal, - segID: 1, - collID: 1, - partitionID: 2, - channelName: "insert-01", - numOfRows: int64(10), - statsBinLogs: []*datapb.FieldBinlog{getSimpleFieldBinlog()}, - cp: cp, - recoverTs: 0, - }) - assert.NotNil(t, err) - err = sr.addSegment( - addSegmentReq{ - segType: datapb.SegmentType_Flushed, - segID: 1, - collID: 1, - partitionID: 2, - channelName: "insert-01", - numOfRows: int64(0), - statsBinLogs: []*datapb.FieldBinlog{getSimpleFieldBinlog()}, - recoverTs: 0, - }) - assert.NotNil(t, err) - }) - - t.Run("Test_addSegmentStatsError", func(t *testing.T) { - sr, err := newReplica(context.Background(), rc, cm, 1, nil) - assert.Nil(t, err) - sr.chunkManager = &mockDataCMStatsError{} - - cpPos := &internalpb.MsgPosition{ChannelName: "insert-01", Timestamp: Timestamp(10)} - cp := &segmentCheckPoint{int64(10), *cpPos} - err = sr.addSegment( - addSegmentReq{ - segType: datapb.SegmentType_Normal, - segID: 1, - collID: 1, - partitionID: 2, - channelName: "insert-01", - numOfRows: int64(10), - statsBinLogs: []*datapb.FieldBinlog{getSimpleFieldBinlog()}, - cp: cp, - recoverTs: 0, - }) - assert.NotNil(t, err) - err = sr.addSegment( - addSegmentReq{ - segType: datapb.SegmentType_Flushed, - segID: 1, - collID: 1, - partitionID: 2, - channelName: "insert-01", - numOfRows: int64(0), - statsBinLogs: []*datapb.FieldBinlog{getSimpleFieldBinlog()}, - recoverTs: 0, - }) - assert.NotNil(t, err) - }) - - t.Run("Test_addSegmentPkfilterError", func(t *testing.T) { - sr, err := newReplica(context.Background(), rc, cm, 1, nil) - assert.Nil(t, err) - sr.chunkManager = &mockPkfilterMergeError{} - - cpPos := &internalpb.MsgPosition{ChannelName: "insert-01", Timestamp: Timestamp(10)} - cp := &segmentCheckPoint{int64(10), *cpPos} - err = sr.addSegment( - addSegmentReq{ - segType: datapb.SegmentType_Normal, - segID: 1, - collID: 1, - partitionID: 2, - channelName: "insert-01", - numOfRows: int64(10), - statsBinLogs: []*datapb.FieldBinlog{getSimpleFieldBinlog()}, - cp: cp, - recoverTs: 0, - }) - assert.NotNil(t, err) - err = sr.addSegment( - addSegmentReq{ - segType: datapb.SegmentType_Flushed, - segID: 1, - collID: 1, - partitionID: 2, - channelName: "insert-01", - numOfRows: int64(0), - statsBinLogs: []*datapb.FieldBinlog{getSimpleFieldBinlog()}, - recoverTs: 0, - }) - assert.NotNil(t, err) - }) - - t.Run("Test_mergeFlushedSegments", func(t *testing.T) { - sr, err := newReplica(context.Background(), rc, cm, 1, nil) - assert.Nil(t, err) - - primaryKeyData := &storage.Int64FieldData{ - Data: []UniqueID{1}, - } - tests := []struct { - description string - isValid bool - stored bool - - inCompactedFrom []UniqueID - inSeg *Segment - }{ - {"mismatch collection", false, false, []UniqueID{1, 2}, &Segment{ - segmentID: 3, - collectionID: -1, - }}, - {"no match flushed segment", false, false, []UniqueID{1, 6}, &Segment{ - segmentID: 3, - collectionID: 1, - }}, - {"numRows==0", true, false, []UniqueID{1, 2}, &Segment{ - segmentID: 3, - collectionID: 1, - numRows: 0, - }}, - {"numRows>0", true, true, []UniqueID{1, 2}, &Segment{ - segmentID: 3, - collectionID: 1, - numRows: 15, - }}, - {"segment exists but not flushed", false, false, []UniqueID{1, 4}, &Segment{ - segmentID: 3, - collectionID: 1, - numRows: 15, - }}, - } - - for _, test := range tests { - t.Run(test.description, func(t *testing.T) { - // prepare segment replica - if !sr.hasSegment(1, true) { - sr.addFlushedSegmentWithPKs(1, 1, 0, "channel", 10, primaryKeyData) - } - - if !sr.hasSegment(2, true) { - sr.addFlushedSegmentWithPKs(2, 1, 0, "channel", 10, primaryKeyData) - } - - if !sr.hasSegment(4, false) { - sr.removeSegments(4) - sr.addSegment(addSegmentReq{ - segType: datapb.SegmentType_Normal, - segID: 4, - collID: 1, - partitionID: 0, - }) - } - - if sr.hasSegment(3, true) { - sr.removeSegments(3) - } - - require.True(t, sr.hasSegment(1, true)) - require.True(t, sr.hasSegment(2, true)) - require.True(t, sr.hasSegment(4, false)) - require.False(t, sr.hasSegment(3, true)) - - // tests start - err := sr.mergeFlushedSegments(test.inSeg, 100, test.inCompactedFrom) - if test.isValid { - assert.NoError(t, err) - } else { - assert.Error(t, err) - } - - if test.stored { - assert.True(t, sr.hasSegment(3, true)) - - to2from := sr.listCompactedSegmentIDs() - assert.NotEmpty(t, to2from) - - from, ok := to2from[3] - assert.True(t, ok) - assert.ElementsMatch(t, []UniqueID{1, 2}, from) - } else { - assert.False(t, sr.hasSegment(3, true)) - } - - }) - } - }) - -} -func TestInnerFunctionSegment(t *testing.T) { - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() - rc := &RootCoordFactory{ - pkType: schemapb.DataType_Int64, - } - collID := UniqueID(1) - cm := storage.NewLocalChunkManager(storage.RootPath(segmentReplicaNodeTestDir)) - defer cm.RemoveWithPrefix(ctx, "") - replica, err := newReplica(context.Background(), rc, cm, collID, nil) - assert.Nil(t, err) - replica.chunkManager = &mockDataCM{} - assert.False(t, replica.hasSegment(0, true)) - assert.False(t, replica.hasSegment(0, false)) - - startPos := &internalpb.MsgPosition{ChannelName: "insert-01", Timestamp: Timestamp(100)} - endPos := &internalpb.MsgPosition{ChannelName: "insert-01", Timestamp: Timestamp(200)} - err = replica.addSegment( - addSegmentReq{ - segType: datapb.SegmentType_New, - segID: 0, - collID: 1, - partitionID: 2, - channelName: "insert-01", - startPos: startPos, - endPos: endPos, - }) - assert.NoError(t, err) - assert.True(t, replica.hasSegment(0, true)) - assert.Equal(t, 1, len(replica.newSegments)) - - seg, ok := replica.newSegments[UniqueID(0)] - assert.True(t, ok) - require.NotNil(t, seg) - assert.Equal(t, UniqueID(0), seg.segmentID) - assert.Equal(t, UniqueID(1), seg.collectionID) - assert.Equal(t, UniqueID(2), seg.partitionID) - assert.Equal(t, "insert-01", seg.channelName) - assert.Equal(t, Timestamp(100), seg.startPos.Timestamp) - assert.Equal(t, Timestamp(200), seg.endPos.Timestamp) - assert.Equal(t, startPos.ChannelName, seg.checkPoint.pos.ChannelName) - assert.Equal(t, startPos.Timestamp, seg.checkPoint.pos.Timestamp) - assert.Equal(t, int64(0), seg.numRows) - assert.True(t, seg.isNew.Load().(bool)) - assert.False(t, seg.isFlushed.Load().(bool)) - - replica.updateStatistics(0, 10) - assert.Equal(t, int64(10), seg.numRows) - - cpPos := &internalpb.MsgPosition{ChannelName: "insert-01", Timestamp: Timestamp(10)} - cp := &segmentCheckPoint{int64(10), *cpPos} - err = replica.addSegment( - addSegmentReq{ - segType: datapb.SegmentType_Normal, - segID: 1, - collID: 1, - partitionID: 2, - channelName: "insert-01", - numOfRows: int64(10), - statsBinLogs: []*datapb.FieldBinlog{getSimpleFieldBinlog()}, - cp: cp, - recoverTs: 0, - }) - assert.NoError(t, err) - assert.True(t, replica.hasSegment(1, true)) - assert.Equal(t, 1, len(replica.normalSegments)) - seg, ok = replica.normalSegments[UniqueID(1)] - assert.True(t, ok) - require.NotNil(t, seg) - assert.Equal(t, UniqueID(1), seg.segmentID) - assert.Equal(t, UniqueID(1), seg.collectionID) - assert.Equal(t, UniqueID(2), seg.partitionID) - assert.Equal(t, "insert-01", seg.channelName) - assert.Equal(t, cpPos.ChannelName, seg.checkPoint.pos.ChannelName) - assert.Equal(t, cpPos.Timestamp, seg.checkPoint.pos.Timestamp) - assert.Equal(t, int64(10), seg.numRows) - assert.False(t, seg.isNew.Load().(bool)) - assert.False(t, seg.isFlushed.Load().(bool)) - - err = replica.addSegment( - addSegmentReq{ - segType: datapb.SegmentType_Normal, - segID: 1, - collID: 100000, - partitionID: 2, - channelName: "invalid", - numOfRows: int64(0), - statsBinLogs: []*datapb.FieldBinlog{getSimpleFieldBinlog()}, - cp: &segmentCheckPoint{}, - recoverTs: 0, - }) - assert.Error(t, err) - - replica.updateStatistics(1, 10) - assert.Equal(t, int64(20), seg.numRows) - - segPos := replica.listNewSegmentsStartPositions() - assert.Equal(t, 1, len(segPos)) - assert.Equal(t, UniqueID(0), segPos[0].SegmentID) - assert.Equal(t, "insert-01", segPos[0].StartPosition.ChannelName) - assert.Equal(t, Timestamp(100), segPos[0].StartPosition.Timestamp) - - // not change until transferNewSegment called - assert.Equal(t, 1, len(replica.newSegments)) - assert.Equal(t, 1, len(replica.normalSegments)) - - replica.transferNewSegments(lo.Map(segPos, func(pos *datapb.SegmentStartPosition, _ int) UniqueID { - return pos.GetSegmentID() - })) - - cps := replica.listSegmentsCheckPoints() - assert.Equal(t, 2, len(cps)) - assert.Equal(t, startPos.Timestamp, cps[UniqueID(0)].pos.Timestamp) - assert.Equal(t, int64(0), cps[UniqueID(0)].numRows) - assert.Equal(t, cp.pos.Timestamp, cps[UniqueID(1)].pos.Timestamp) - assert.Equal(t, int64(10), cps[UniqueID(1)].numRows) - - updates, err := replica.getSegmentStatisticsUpdates(0) - assert.NoError(t, err) - assert.Equal(t, int64(10), updates.NumRows) - - updates, err = replica.getSegmentStatisticsUpdates(1) - assert.NoError(t, err) - assert.Equal(t, int64(20), updates.NumRows) - - replica.updateSegmentCheckPoint(0) - assert.Equal(t, int64(10), replica.normalSegments[UniqueID(0)].checkPoint.numRows) - replica.updateSegmentCheckPoint(1) - assert.Equal(t, int64(20), replica.normalSegments[UniqueID(1)].checkPoint.numRows) - - err = replica.addSegment( - addSegmentReq{ - segType: datapb.SegmentType_Flushed, - segID: 1, - collID: 1, - partitionID: 2, - channelName: "insert-01", - numOfRows: int64(0), - statsBinLogs: []*datapb.FieldBinlog{getSimpleFieldBinlog()}, - recoverTs: 0, - }) - assert.Nil(t, err) - - totalSegments := replica.filterSegments("insert-01", common.InvalidPartitionID) - assert.Equal(t, len(totalSegments), 3) -} - -func TestSegmentReplica_UpdatePKRange(t *testing.T) { - seg := &Segment{ - pkFilter: bloom.NewWithEstimates(100000, 0.005), - } - - cases := make([]int64, 0, 100) - for i := 0; i < 100; i++ { - cases = append(cases, rand.Int63()) - } - buf := make([]byte, 8) - for _, c := range cases { - seg.updatePKRange(&storage.Int64FieldData{ - Data: []int64{c}, - }) - - pk := newInt64PrimaryKey(c) - - assert.Equal(t, true, seg.minPK.LE(pk)) - assert.Equal(t, true, seg.maxPK.GE(pk)) - - common.Endian.PutUint64(buf, uint64(c)) - assert.True(t, seg.pkFilter.Test(buf)) - } -} - -func TestSegment_getSegmentStatslog(t *testing.T) { - rand.Seed(time.Now().UnixNano()) - - cases := make([][]int64, 0, 100) - for i := 0; i < 100; i++ { - tc := make([]int64, 0, 10) - for j := 0; j < 100; j++ { - tc = append(tc, rand.Int63()) - } - cases = append(cases, tc) - } - buf := make([]byte, 8) - for _, tc := range cases { - seg := &Segment{ - pkFilter: bloom.NewWithEstimates(100000, 0.005), - } - - seg.updatePKRange(&storage.Int64FieldData{ - Data: tc, - }) - - statBytes, err := seg.getSegmentStatslog(1, schemapb.DataType_Int64) - assert.NoError(t, err) - - pks := storage.PrimaryKeyStats{} - err = json.Unmarshal(statBytes, &pks) - require.NoError(t, err) - - assert.Equal(t, int64(1), pks.FieldID) - assert.Equal(t, int64(schemapb.DataType_Int64), pks.PkType) - - for _, v := range tc { - pk := newInt64PrimaryKey(v) - assert.True(t, pks.MinPk.LE(pk)) - assert.True(t, pks.MaxPk.GE(pk)) - - common.Endian.PutUint64(buf, uint64(v)) - assert.True(t, seg.pkFilter.Test(buf)) - } - } - - pks := &storage.PrimaryKeyStats{} - _, err := json.Marshal(pks) - assert.NoError(t, err) -} - -func TestReplica_UpdatePKRange(t *testing.T) { - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() - rc := &RootCoordFactory{ - pkType: schemapb.DataType_Int64, - } - collID := UniqueID(1) - partID := UniqueID(2) - chanName := "insert-02" - startPos := &internalpb.MsgPosition{ChannelName: chanName, Timestamp: Timestamp(100)} - endPos := &internalpb.MsgPosition{ChannelName: chanName, Timestamp: Timestamp(200)} - cpPos := &internalpb.MsgPosition{ChannelName: chanName, Timestamp: Timestamp(10)} - cp := &segmentCheckPoint{int64(10), *cpPos} - - cm := storage.NewLocalChunkManager(storage.RootPath(segmentReplicaNodeTestDir)) - defer cm.RemoveWithPrefix(ctx, "") - replica, err := newReplica(context.Background(), rc, cm, collID, nil) - assert.Nil(t, err) - replica.chunkManager = &mockDataCM{} - - err = replica.addSegment( - addSegmentReq{ - segType: datapb.SegmentType_New, - segID: 1, - collID: collID, - partitionID: partID, - channelName: chanName, - startPos: startPos, - endPos: endPos, - }) - assert.Nil(t, err) - err = replica.addSegment( - addSegmentReq{ - segType: datapb.SegmentType_Normal, - segID: 2, - collID: collID, - partitionID: partID, - channelName: chanName, - numOfRows: 100, - statsBinLogs: []*datapb.FieldBinlog{getSimpleFieldBinlog()}, - cp: cp, - recoverTs: 0, - }) - assert.Nil(t, err) - - segNew := replica.newSegments[1] - segNormal := replica.normalSegments[2] - - cases := make([]int64, 0, 100) - for i := 0; i < 100; i++ { - cases = append(cases, rand.Int63()) - } - buf := make([]byte, 8) - for _, c := range cases { - replica.updateSegmentPKRange(1, &storage.Int64FieldData{Data: []int64{c}}) // new segment - replica.updateSegmentPKRange(2, &storage.Int64FieldData{Data: []int64{c}}) // normal segment - replica.updateSegmentPKRange(3, &storage.Int64FieldData{Data: []int64{c}}) // non-exist segment - - pk := newInt64PrimaryKey(c) - - assert.Equal(t, true, segNew.minPK.LE(pk)) - assert.Equal(t, true, segNew.maxPK.GE(pk)) - - assert.Equal(t, true, segNormal.minPK.LE(pk)) - assert.Equal(t, true, segNormal.maxPK.GE(pk)) - - common.Endian.PutUint64(buf, uint64(c)) - assert.True(t, segNew.pkFilter.Test(buf)) - assert.True(t, segNormal.pkFilter.Test(buf)) - - } - -} - -// SegmentReplicaSuite setup test suite for SegmentReplica -type SegmentReplicaSuite struct { - suite.Suite - sr *SegmentReplica - - collID UniqueID - partID UniqueID - vchanName string - cm *storage.LocalChunkManager -} - -func (s *SegmentReplicaSuite) SetupSuite() { - rc := &RootCoordFactory{ - pkType: schemapb.DataType_Int64, - } - s.collID = 1 - s.cm = storage.NewLocalChunkManager(storage.RootPath(segmentReplicaNodeTestDir)) - var err error - s.sr, err = newReplica(context.Background(), rc, s.cm, s.collID, nil) - s.Require().NoError(err) -} - -func (s *SegmentReplicaSuite) TearDownSuite() { - s.cm.RemoveWithPrefix(context.Background(), "") -} - -func (s *SegmentReplicaSuite) SetupTest() { - var err error - err = s.sr.addSegment(addSegmentReq{ - segType: datapb.SegmentType_New, - segID: 1, - collID: s.collID, - partitionID: s.partID, - channelName: s.vchanName, - startPos: &internalpb.MsgPosition{}, - endPos: nil, - }) - s.Require().NoError(err) - err = s.sr.addSegment(addSegmentReq{ - segType: datapb.SegmentType_Normal, - segID: 2, - collID: s.collID, - partitionID: s.partID, - channelName: s.vchanName, - numOfRows: 10, - statsBinLogs: nil, - cp: nil, - recoverTs: 0, - }) - s.Require().NoError(err) - err = s.sr.addSegment(addSegmentReq{ - segType: datapb.SegmentType_Flushed, - segID: 3, - collID: s.collID, - partitionID: s.partID, - channelName: s.vchanName, - numOfRows: 10, - statsBinLogs: nil, - recoverTs: 0, - }) - s.Require().NoError(err) -} - -func (s *SegmentReplicaSuite) TearDownTest() { - s.sr.removeSegments(1, 2, 3) -} - -func (s *SegmentReplicaSuite) TestGetSegmentStatslog() { - bs, err := s.sr.getSegmentStatslog(1) - s.NoError(err) - - segment, ok := s.getSegmentByID(1) - s.Require().True(ok) - expected, err := segment.getSegmentStatslog(106, schemapb.DataType_Int64) - s.Require().NoError(err) - s.Equal(expected, bs) - - bs, err = s.sr.getSegmentStatslog(2) - s.NoError(err) - - segment, ok = s.getSegmentByID(2) - s.Require().True(ok) - expected, err = segment.getSegmentStatslog(106, schemapb.DataType_Int64) - s.Require().NoError(err) - s.Equal(expected, bs) - - bs, err = s.sr.getSegmentStatslog(3) - s.NoError(err) - - segment, ok = s.getSegmentByID(3) - s.Require().True(ok) - expected, err = segment.getSegmentStatslog(106, schemapb.DataType_Int64) - s.Require().NoError(err) - s.Equal(expected, bs) - - _, err = s.sr.getSegmentStatslog(4) - s.Error(err) -} - -func (s *SegmentReplicaSuite) getSegmentByID(id UniqueID) (*Segment, bool) { - s.sr.segMu.RLock() - defer s.sr.segMu.RUnlock() - - seg, ok := s.sr.newSegments[id] - if ok { - return seg, true - } - - seg, ok = s.sr.normalSegments[id] - if ok { - return seg, true - } - - seg, ok = s.sr.flushedSegments[id] - if ok { - return seg, true - } - - return nil, false -} - -func TestSegmentReplicaSuite(t *testing.T) { - suite.Run(t, new(SegmentReplicaSuite)) -} diff --git a/internal/datanode/segment_test.go b/internal/datanode/segment_test.go new file mode 100644 index 0000000000..43b07f0425 --- /dev/null +++ b/internal/datanode/segment_test.go @@ -0,0 +1,103 @@ +// 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 ( + "encoding/json" + "math/rand" + "testing" + "time" + + "github.com/bits-and-blooms/bloom/v3" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + + "github.com/milvus-io/milvus-proto/go-api/schemapb" + "github.com/milvus-io/milvus/internal/common" + "github.com/milvus-io/milvus/internal/storage" +) + +func TestSegment_UpdatePKRange(t *testing.T) { + seg := &Segment{ + pkFilter: bloom.NewWithEstimates(100000, 0.005), + } + + cases := make([]int64, 0, 100) + for i := 0; i < 100; i++ { + cases = append(cases, rand.Int63()) + } + buf := make([]byte, 8) + for _, c := range cases { + seg.updatePKRange(&storage.Int64FieldData{ + Data: []int64{c}, + }) + + pk := newInt64PrimaryKey(c) + + assert.Equal(t, true, seg.minPK.LE(pk)) + assert.Equal(t, true, seg.maxPK.GE(pk)) + + common.Endian.PutUint64(buf, uint64(c)) + assert.True(t, seg.pkFilter.Test(buf)) + } +} + +func TestSegment_getSegmentStatslog(t *testing.T) { + rand.Seed(time.Now().UnixNano()) + + cases := make([][]int64, 0, 100) + for i := 0; i < 100; i++ { + tc := make([]int64, 0, 10) + for j := 0; j < 100; j++ { + tc = append(tc, rand.Int63()) + } + cases = append(cases, tc) + } + buf := make([]byte, 8) + for _, tc := range cases { + seg := &Segment{ + pkFilter: bloom.NewWithEstimates(100000, 0.005), + } + + seg.updatePKRange(&storage.Int64FieldData{ + Data: tc, + }) + + statBytes, err := seg.getSegmentStatslog(1, schemapb.DataType_Int64) + assert.NoError(t, err) + + pks := storage.PrimaryKeyStats{} + err = json.Unmarshal(statBytes, &pks) + require.NoError(t, err) + + assert.Equal(t, int64(1), pks.FieldID) + assert.Equal(t, int64(schemapb.DataType_Int64), pks.PkType) + + for _, v := range tc { + pk := newInt64PrimaryKey(v) + assert.True(t, pks.MinPk.LE(pk)) + assert.True(t, pks.MaxPk.GE(pk)) + + common.Endian.PutUint64(buf, uint64(v)) + assert.True(t, seg.pkFilter.Test(buf)) + } + } + + pks := &storage.PrimaryKeyStats{} + _, err := json.Marshal(pks) + assert.NoError(t, err) +} diff --git a/internal/proto/data_coord.proto b/internal/proto/data_coord.proto index 6f0573fcf7..381305d65b 100644 --- a/internal/proto/data_coord.proto +++ b/internal/proto/data_coord.proto @@ -16,6 +16,7 @@ enum SegmentType { New = 0; Normal = 1; Flushed = 2; + Compacted = 3; } service DataCoord { diff --git a/internal/proto/datapb/data_coord.pb.go b/internal/proto/datapb/data_coord.pb.go index cfa06b2a09..5a9cea9e7d 100644 --- a/internal/proto/datapb/data_coord.pb.go +++ b/internal/proto/datapb/data_coord.pb.go @@ -31,21 +31,24 @@ const _ = proto.ProtoPackageIsVersion3 // please upgrade the proto package type SegmentType int32 const ( - SegmentType_New SegmentType = 0 - SegmentType_Normal SegmentType = 1 - SegmentType_Flushed SegmentType = 2 + SegmentType_New SegmentType = 0 + SegmentType_Normal SegmentType = 1 + SegmentType_Flushed SegmentType = 2 + SegmentType_Compacted SegmentType = 3 ) var SegmentType_name = map[int32]string{ 0: "New", 1: "Normal", 2: "Flushed", + 3: "Compacted", } var SegmentType_value = map[string]int32{ - "New": 0, - "Normal": 1, - "Flushed": 2, + "New": 0, + "Normal": 1, + "Flushed": 2, + "Compacted": 3, } func (x SegmentType) String() string { @@ -4796,274 +4799,275 @@ func init() { func init() { proto.RegisterFile("data_coord.proto", fileDescriptor_82cd95f524594f49) } var fileDescriptor_82cd95f524594f49 = []byte{ - // 4272 bytes of a gzipped FileDescriptorProto + // 4280 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe4, 0x3c, 0x4b, 0x6f, 0x24, 0x49, 0x5a, 0x9d, 0xf5, 0x72, 0xd5, 0x57, 0x0f, 0x97, 0xa3, 0x7b, 0xec, 0xea, 0xea, 0xe7, 0x64, 0x4f, - 0xcf, 0xf4, 0xf4, 0xf4, 0x74, 0xcf, 0x78, 0x18, 0xed, 0x40, 0xef, 0xcc, 0xaa, 0xdd, 0x9e, 0x76, - 0x17, 0xd8, 0x5e, 0x6f, 0xda, 0x3d, 0x2d, 0xed, 0x22, 0x95, 0xd2, 0x95, 0xe1, 0x72, 0xae, 0xf3, - 0x51, 0x9d, 0x99, 0x65, 0xb7, 0x97, 0xc3, 0x8e, 0x58, 0x09, 0x69, 0x11, 0x62, 0x79, 0x08, 0x09, - 0x24, 0x90, 0x10, 0xa7, 0x05, 0x84, 0x84, 0xb4, 0xe2, 0x00, 0x17, 0x0e, 0x5c, 0x46, 0x70, 0x58, - 0x71, 0xe1, 0x07, 0x70, 0x00, 0xee, 0x5c, 0x39, 0xa0, 0x78, 0x64, 0xe4, 0xbb, 0x2a, 0x5d, 0xe5, + 0xcf, 0xf4, 0xf4, 0xf4, 0x74, 0xcf, 0x78, 0x18, 0xed, 0x40, 0xef, 0xcc, 0xaa, 0x6d, 0x4f, 0xbb, + 0x0b, 0x6c, 0xaf, 0x37, 0xed, 0x9e, 0x96, 0x76, 0x91, 0x4a, 0xe9, 0xca, 0x70, 0x39, 0xd7, 0xf9, + 0xa8, 0xce, 0xcc, 0xb2, 0xdb, 0xcb, 0x61, 0x47, 0xac, 0x84, 0xb4, 0x08, 0xb1, 0x3c, 0x84, 0x04, + 0x12, 0x48, 0x88, 0xd3, 0x02, 0x42, 0x42, 0x5a, 0x71, 0x80, 0x0b, 0x07, 0x2e, 0x23, 0x38, 0xac, + 0xb8, 0xf0, 0x03, 0x38, 0x00, 0x77, 0xae, 0x1c, 0x50, 0x3c, 0x32, 0xf2, 0x5d, 0x95, 0xae, 0xea, 0x9e, 0x46, 0xec, 0xcd, 0x11, 0xf5, 0x45, 0x7c, 0x11, 0xdf, 0xfb, 0xfb, 0xe2, 0x4b, 0x43, 0x5b, 0x53, 0x3d, 0xb5, 0x3f, 0xb0, 0x6d, 0x47, 0xbb, 0x3f, 0x72, 0x6c, 0xcf, 0x46, 0x4b, 0xa6, 0x6e, - 0x1c, 0x8f, 0x5d, 0x36, 0xba, 0x4f, 0x7e, 0xee, 0x36, 0x06, 0xb6, 0x69, 0xda, 0x16, 0x9b, 0xea, - 0xb6, 0x74, 0xcb, 0xc3, 0x8e, 0xa5, 0x1a, 0x7c, 0xdc, 0x08, 0x2f, 0xe8, 0x36, 0xdc, 0xc1, 0x21, - 0x36, 0x55, 0x36, 0x92, 0x17, 0xa0, 0xfc, 0xb9, 0x39, 0xf2, 0x4e, 0xe5, 0x3f, 0x96, 0xa0, 0xf1, - 0xc4, 0x18, 0xbb, 0x87, 0x0a, 0x7e, 0x31, 0xc6, 0xae, 0x87, 0x3e, 0x80, 0xd2, 0xbe, 0xea, 0xe2, - 0x8e, 0x74, 0x53, 0xba, 0x53, 0x5f, 0xbd, 0x7a, 0x3f, 0x82, 0x95, 0xe3, 0xdb, 0x72, 0x87, 0x6b, - 0xaa, 0x8b, 0x15, 0x0a, 0x89, 0x10, 0x94, 0xb4, 0xfd, 0xde, 0x7a, 0xa7, 0x70, 0x53, 0xba, 0x53, - 0x54, 0xe8, 0xdf, 0xe8, 0x3a, 0x80, 0x8b, 0x87, 0x26, 0xb6, 0xbc, 0xde, 0xba, 0xdb, 0x29, 0xde, + 0x9c, 0x8c, 0x5d, 0x36, 0xba, 0x4f, 0x7e, 0xee, 0x36, 0x06, 0xb6, 0x69, 0xda, 0x16, 0x9b, 0xea, + 0xb6, 0x74, 0xcb, 0xc3, 0x8e, 0xa5, 0x1a, 0x7c, 0xdc, 0x08, 0x2f, 0xe8, 0x36, 0xdc, 0xc1, 0x11, + 0x36, 0x55, 0x36, 0x92, 0x17, 0xa0, 0xfc, 0xb9, 0x39, 0xf2, 0xce, 0xe4, 0x3f, 0x96, 0xa0, 0xf1, + 0xd8, 0x18, 0xbb, 0x47, 0x0a, 0x7e, 0x3e, 0xc6, 0xae, 0x87, 0x3e, 0x80, 0xd2, 0x81, 0xea, 0xe2, + 0x8e, 0x74, 0x53, 0xba, 0x53, 0x5f, 0xbd, 0x7a, 0x3f, 0x82, 0x95, 0xe3, 0xdb, 0x76, 0x87, 0x6b, + 0xaa, 0x8b, 0x15, 0x0a, 0x89, 0x10, 0x94, 0xb4, 0x83, 0xde, 0x46, 0xa7, 0x70, 0x53, 0xba, 0x53, + 0x54, 0xe8, 0xdf, 0xe8, 0x3a, 0x80, 0x8b, 0x87, 0x26, 0xb6, 0xbc, 0xde, 0x86, 0xdb, 0x29, 0xde, 0x2c, 0xde, 0x29, 0x2a, 0xa1, 0x19, 0x24, 0x43, 0x63, 0x60, 0x1b, 0x06, 0x1e, 0x78, 0xba, 0x6d, - 0xf5, 0xd6, 0x3b, 0x25, 0xba, 0x36, 0x32, 0x27, 0xff, 0x87, 0x04, 0x4d, 0x7e, 0x34, 0x77, 0x64, + 0xf5, 0x36, 0x3a, 0x25, 0xba, 0x36, 0x32, 0x27, 0xff, 0x87, 0x04, 0x4d, 0x7e, 0x34, 0x77, 0x64, 0x5b, 0x2e, 0x46, 0x1f, 0x41, 0xc5, 0xf5, 0x54, 0x6f, 0xec, 0xf2, 0xd3, 0x5d, 0x49, 0x3d, 0xdd, - 0x2e, 0x05, 0x51, 0x38, 0x68, 0xea, 0xf1, 0xe2, 0xe8, 0x8b, 0x49, 0xf4, 0xb1, 0x2b, 0x94, 0x12, - 0x57, 0xb8, 0x03, 0x8b, 0x07, 0xe4, 0x74, 0xbb, 0x01, 0x50, 0x99, 0x02, 0xc5, 0xa7, 0xc9, 0x4e, - 0x9e, 0x6e, 0xe2, 0x6f, 0x1f, 0xec, 0x62, 0xd5, 0xe8, 0x54, 0x28, 0xae, 0xd0, 0x8c, 0xfc, 0xaf, + 0x1e, 0x05, 0x51, 0x38, 0x68, 0xea, 0xf1, 0xe2, 0xe8, 0x8b, 0x49, 0xf4, 0xb1, 0x2b, 0x94, 0x12, + 0x57, 0xb8, 0x03, 0x8b, 0x87, 0xe4, 0x74, 0x7b, 0x01, 0x50, 0x99, 0x02, 0xc5, 0xa7, 0xc9, 0x4e, + 0x9e, 0x6e, 0xe2, 0x6f, 0x1f, 0xee, 0x61, 0xd5, 0xe8, 0x54, 0x28, 0xae, 0xd0, 0x8c, 0xfc, 0xaf, 0x12, 0xb4, 0x05, 0xb8, 0xcf, 0x87, 0x4b, 0x50, 0x1e, 0xd8, 0x63, 0xcb, 0xa3, 0x57, 0x6d, 0x2a, - 0x6c, 0x80, 0xde, 0x84, 0xc6, 0xe0, 0x50, 0xb5, 0x2c, 0x6c, 0xf4, 0x2d, 0xd5, 0xc4, 0xf4, 0x52, - 0x35, 0xa5, 0xce, 0xe7, 0xb6, 0x55, 0x13, 0xe7, 0xba, 0xdb, 0x4d, 0xa8, 0x8f, 0x54, 0xc7, 0xd3, + 0x6c, 0x80, 0xde, 0x84, 0xc6, 0xe0, 0x48, 0xb5, 0x2c, 0x6c, 0xf4, 0x2d, 0xd5, 0xc4, 0xf4, 0x52, + 0x35, 0xa5, 0xce, 0xe7, 0x76, 0x54, 0x13, 0xe7, 0xba, 0xdb, 0x4d, 0xa8, 0x8f, 0x54, 0xc7, 0xd3, 0x23, 0xd4, 0x0f, 0x4f, 0xa1, 0x2e, 0x54, 0x75, 0xb7, 0x67, 0x8e, 0x6c, 0xc7, 0xeb, 0x94, 0x6f, - 0x4a, 0x77, 0xaa, 0x8a, 0x18, 0x13, 0x0c, 0x3a, 0xfd, 0x6b, 0x4f, 0x75, 0x8f, 0x7a, 0xeb, 0xfc, + 0x4a, 0x77, 0xaa, 0x8a, 0x18, 0x13, 0x0c, 0x3a, 0xfd, 0x6b, 0x5f, 0x75, 0x8f, 0x7b, 0x1b, 0xfc, 0x46, 0x91, 0x39, 0xf9, 0xcf, 0x25, 0x58, 0x7e, 0xe4, 0xba, 0xfa, 0xd0, 0x4a, 0xdc, 0x6c, 0x19, - 0x2a, 0x96, 0xad, 0xe1, 0xde, 0x3a, 0xbd, 0x5a, 0x51, 0xe1, 0x23, 0x74, 0x05, 0x6a, 0x23, 0x8c, + 0x2a, 0x96, 0xad, 0xe1, 0xde, 0x06, 0xbd, 0x5a, 0x51, 0xe1, 0x23, 0x74, 0x05, 0x6a, 0x23, 0x8c, 0x9d, 0xbe, 0x63, 0x1b, 0xfe, 0xc5, 0xaa, 0x64, 0x42, 0xb1, 0x0d, 0x8c, 0xbe, 0x03, 0x4b, 0x6e, 0x6c, 0x23, 0x26, 0x57, 0xf5, 0xd5, 0x5b, 0xf7, 0x13, 0x9a, 0x71, 0x3f, 0x8e, 0x54, 0x49, 0xae, 0x96, 0xbf, 0x2c, 0xc0, 0x45, 0x01, 0xc7, 0xce, 0x4a, 0xfe, 0x26, 0x94, 0x77, 0xf1, 0x50, 0x1c, 0x8f, 0x0d, 0xf2, 0x50, 0x5e, 0xb0, 0xac, 0x18, 0x66, 0x59, 0x0e, 0x51, 0x8f, 0xf3, 0xa3, 0x9c, - 0xe4, 0xc7, 0x0d, 0xa8, 0xe3, 0x97, 0x23, 0xdd, 0xc1, 0x7d, 0x22, 0x38, 0x94, 0xe4, 0x25, 0x05, - 0xd8, 0xd4, 0x9e, 0x6e, 0x86, 0x75, 0x63, 0x21, 0xb7, 0x6e, 0xc8, 0x7f, 0x21, 0xc1, 0x4a, 0x82, + 0xe4, 0xc7, 0x0d, 0xa8, 0xe3, 0x17, 0x23, 0xdd, 0xc1, 0x7d, 0x22, 0x38, 0x94, 0xe4, 0x25, 0x05, + 0xd8, 0xd4, 0xbe, 0x6e, 0x86, 0x75, 0x63, 0x21, 0xb7, 0x6e, 0xc8, 0x7f, 0x21, 0xc1, 0x4a, 0x82, 0x4b, 0x5c, 0xd9, 0x14, 0x68, 0xd3, 0x9b, 0x07, 0x94, 0x21, 0x6a, 0x47, 0x08, 0xfe, 0xf6, 0x24, - 0x82, 0x07, 0xe0, 0x4a, 0x62, 0x7d, 0xe8, 0x90, 0x85, 0xfc, 0x87, 0x3c, 0x82, 0x95, 0x0d, 0xec, + 0x82, 0x07, 0xe0, 0x4a, 0x62, 0x7d, 0xe8, 0x90, 0x85, 0xfc, 0x87, 0x3c, 0x86, 0x95, 0x4d, 0xec, 0x71, 0x04, 0xe4, 0x37, 0xec, 0xce, 0x6e, 0xac, 0xa2, 0x5a, 0x5d, 0x88, 0x6b, 0xb5, 0xfc, 0xb7, - 0x05, 0xa1, 0x8b, 0x14, 0x55, 0xcf, 0x3a, 0xb0, 0xd1, 0x55, 0xa8, 0x09, 0x10, 0x2e, 0x15, 0xc1, + 0x05, 0xa1, 0x8b, 0x14, 0x55, 0xcf, 0x3a, 0xb4, 0xd1, 0x55, 0xa8, 0x09, 0x10, 0x2e, 0x15, 0xc1, 0x04, 0xfa, 0x06, 0x94, 0xc9, 0x49, 0x99, 0x48, 0xb4, 0x56, 0xdf, 0x4c, 0xbf, 0x53, 0x68, 0x4f, 0x85, 0xc1, 0xa3, 0x1e, 0xb4, 0x5c, 0x4f, 0x75, 0xbc, 0xfe, 0xc8, 0x76, 0x29, 0x9f, 0xa9, 0xe0, - 0xd4, 0x57, 0xe5, 0xe8, 0x0e, 0xc2, 0xac, 0x6f, 0xb9, 0xc3, 0x1d, 0x0e, 0xa9, 0x34, 0xe9, 0x4a, + 0xd4, 0x57, 0xe5, 0xe8, 0x0e, 0xc2, 0xac, 0x6f, 0xbb, 0xc3, 0x5d, 0x0e, 0xa9, 0x34, 0xe9, 0x4a, 0x7f, 0x88, 0x3e, 0x87, 0x06, 0xb6, 0xb4, 0x60, 0xa3, 0x52, 0xee, 0x8d, 0xea, 0xd8, 0xd2, 0xc4, 0x36, 0x01, 0x7f, 0xca, 0xf9, 0xf9, 0xf3, 0x3b, 0x12, 0x74, 0x92, 0x0c, 0x9a, 0xc7, 0x64, 0x3f, 0x64, 0x8b, 0x30, 0x63, 0xd0, 0x44, 0x0d, 0x17, 0x4c, 0x52, 0xf8, 0x12, 0xf9, 0x8f, 0x24, 0x78, 0x23, 0x38, 0x0e, 0xfd, 0xe9, 0x55, 0x49, 0x0b, 0xba, 0x0b, 0x6d, 0xdd, 0x1a, 0x18, 0x63, 0x0d, - 0x3f, 0xb3, 0x9e, 0x62, 0xd5, 0xf0, 0x0e, 0x4f, 0x29, 0x0f, 0xab, 0x4a, 0x62, 0x5e, 0xfe, 0x91, - 0x04, 0xcb, 0xf1, 0x73, 0xcd, 0x43, 0xa4, 0x5f, 0x82, 0xb2, 0x6e, 0x1d, 0xd8, 0x3e, 0x8d, 0xae, - 0x4f, 0x50, 0x4a, 0x82, 0x8b, 0x01, 0xcb, 0x26, 0x5c, 0xd9, 0xc0, 0x5e, 0xcf, 0x72, 0xb1, 0xe3, - 0xad, 0xe9, 0x96, 0x61, 0x0f, 0x77, 0x54, 0xef, 0x70, 0x0e, 0x85, 0x8a, 0xe8, 0x46, 0x21, 0xa6, - 0x1b, 0xf2, 0x4f, 0x25, 0xb8, 0x9a, 0x8e, 0x8f, 0x5f, 0xbd, 0x0b, 0xd5, 0x03, 0x1d, 0x1b, 0x1a, + 0x3f, 0xb5, 0x9e, 0x60, 0xd5, 0xf0, 0x8e, 0xce, 0x28, 0x0f, 0xab, 0x4a, 0x62, 0x5e, 0xfe, 0x91, + 0x04, 0xcb, 0xf1, 0x73, 0xcd, 0x43, 0xa4, 0x5f, 0x82, 0xb2, 0x6e, 0x1d, 0xda, 0x3e, 0x8d, 0xae, + 0x4f, 0x50, 0x4a, 0x82, 0x8b, 0x01, 0xcb, 0x26, 0x5c, 0xd9, 0xc4, 0x5e, 0xcf, 0x72, 0xb1, 0xe3, + 0xad, 0xe9, 0x96, 0x61, 0x0f, 0x77, 0x55, 0xef, 0x68, 0x0e, 0x85, 0x8a, 0xe8, 0x46, 0x21, 0xa6, + 0x1b, 0xf2, 0x4f, 0x25, 0xb8, 0x9a, 0x8e, 0x8f, 0x5f, 0xbd, 0x0b, 0xd5, 0x43, 0x1d, 0x1b, 0x1a, 0xa1, 0xaf, 0x44, 0xe9, 0x2b, 0xc6, 0x44, 0xb1, 0x46, 0x04, 0x98, 0xdf, 0xf0, 0xcd, 0x0c, 0x69, - 0xde, 0xf5, 0x1c, 0xdd, 0x1a, 0x6e, 0xea, 0xae, 0xa7, 0x30, 0xf8, 0x10, 0x3d, 0x8b, 0xf9, 0xc5, - 0xf8, 0xb7, 0x25, 0xb8, 0xbe, 0x81, 0xbd, 0xc7, 0xc2, 0x2e, 0x93, 0xdf, 0x75, 0xd7, 0xd3, 0x07, - 0xee, 0xf9, 0xc6, 0x46, 0x39, 0x1c, 0xb4, 0xfc, 0x13, 0x09, 0x6e, 0x64, 0x1e, 0x86, 0x93, 0x8e, - 0xdb, 0x1d, 0xdf, 0x2a, 0xa7, 0xdb, 0x9d, 0x5f, 0xc3, 0xa7, 0x5f, 0xa8, 0xc6, 0x18, 0xef, 0xa8, - 0xba, 0xc3, 0xec, 0xce, 0x8c, 0x56, 0xf8, 0x6f, 0x24, 0xb8, 0xb6, 0x81, 0xbd, 0x1d, 0xdf, 0x27, + 0xde, 0xf3, 0x1c, 0xdd, 0x1a, 0x6e, 0xe9, 0xae, 0xa7, 0x30, 0xf8, 0x10, 0x3d, 0x8b, 0xf9, 0xc5, + 0xf8, 0xb7, 0x25, 0xb8, 0xbe, 0x89, 0xbd, 0x75, 0x61, 0x97, 0xc9, 0xef, 0xba, 0xeb, 0xe9, 0x03, + 0xf7, 0xe5, 0xc6, 0x46, 0x39, 0x1c, 0xb4, 0xfc, 0x13, 0x09, 0x6e, 0x64, 0x1e, 0x86, 0x93, 0x8e, + 0xdb, 0x1d, 0xdf, 0x2a, 0xa7, 0xdb, 0x9d, 0x5f, 0xc3, 0x67, 0x5f, 0xa8, 0xc6, 0x18, 0xef, 0xaa, + 0xba, 0xc3, 0xec, 0xce, 0x8c, 0x56, 0xf8, 0x6f, 0x24, 0xb8, 0xb6, 0x89, 0xbd, 0x5d, 0xdf, 0x27, 0xbd, 0x46, 0xea, 0x10, 0x98, 0x90, 0x6f, 0xf4, 0x83, 0xb3, 0xc8, 0x9c, 0xfc, 0xbb, 0x8c, 0x9d, - 0xa9, 0xe7, 0x7d, 0x2d, 0x04, 0xbc, 0x4e, 0x35, 0x21, 0xa4, 0x92, 0x8f, 0x59, 0xe8, 0xc0, 0xc9, - 0x27, 0xff, 0x99, 0x04, 0x97, 0x1f, 0x0d, 0x5e, 0x8c, 0x75, 0x07, 0x73, 0xa0, 0x4d, 0x7b, 0x70, - 0x34, 0x3b, 0x71, 0x83, 0x30, 0xab, 0x10, 0x09, 0xb3, 0xa6, 0x85, 0xe6, 0xcb, 0x50, 0xf1, 0x58, - 0x5c, 0xc7, 0x22, 0x15, 0x3e, 0xa2, 0xe7, 0x53, 0xb0, 0x81, 0x55, 0xf7, 0xff, 0xe6, 0xf9, 0x7e, - 0x52, 0x82, 0xc6, 0x17, 0x3c, 0x1c, 0xa3, 0x5e, 0x3b, 0x2e, 0x49, 0x52, 0x7a, 0xe0, 0x15, 0x8a, - 0xe0, 0xd2, 0x82, 0xba, 0x0d, 0x68, 0xba, 0x18, 0x1f, 0xcd, 0xe2, 0xa3, 0x1b, 0x64, 0xa1, 0xf0, - 0xad, 0x9b, 0xb0, 0x34, 0xb6, 0x68, 0x6a, 0x80, 0x35, 0x4e, 0x40, 0x26, 0xb9, 0xd3, 0x6d, 0x77, - 0x72, 0x21, 0x7a, 0xca, 0xb3, 0x8f, 0xd0, 0x5e, 0xe5, 0x5c, 0x7b, 0xc5, 0x97, 0xa1, 0x1e, 0xb4, - 0x35, 0xc7, 0x1e, 0x8d, 0xb0, 0xd6, 0x77, 0xfd, 0xad, 0x2a, 0xf9, 0xb6, 0xe2, 0xeb, 0xc4, 0x56, - 0x1f, 0xc0, 0xc5, 0xf8, 0x49, 0x7b, 0x1a, 0x09, 0x48, 0x09, 0x0f, 0xd3, 0x7e, 0x42, 0xf7, 0x60, - 0x29, 0x09, 0x5f, 0xa5, 0xf0, 0xc9, 0x1f, 0xd0, 0xfb, 0x80, 0x62, 0x47, 0x25, 0xe0, 0x35, 0x06, - 0x1e, 0x3d, 0x4c, 0x4f, 0x73, 0xe5, 0x1f, 0x4b, 0xb0, 0xfc, 0x5c, 0xf5, 0x06, 0x87, 0xeb, 0x26, - 0xd7, 0xb5, 0x39, 0x6c, 0xd5, 0xa7, 0x50, 0x3b, 0xe6, 0x72, 0xe1, 0x3b, 0xa4, 0x1b, 0x29, 0xf4, - 0x09, 0x4b, 0xa0, 0x12, 0xac, 0x90, 0xbf, 0x92, 0xe0, 0xd2, 0x93, 0x50, 0x5e, 0xf8, 0x1a, 0xac, - 0xe6, 0xb4, 0x84, 0xf6, 0x6d, 0x68, 0x99, 0xaa, 0x73, 0x94, 0xc8, 0x67, 0x63, 0xb3, 0xf2, 0x4b, - 0x00, 0x3e, 0xda, 0x72, 0x87, 0x33, 0x9c, 0xff, 0x13, 0x58, 0xe0, 0x58, 0xb9, 0xf9, 0x9c, 0x26, - 0x67, 0x3e, 0xb8, 0xfc, 0xfb, 0x15, 0xa8, 0x87, 0x7e, 0x40, 0x2d, 0x28, 0x08, 0xbd, 0x2e, 0xa4, - 0x50, 0xa1, 0x30, 0x3d, 0xd5, 0x2a, 0x26, 0x53, 0xad, 0xdb, 0xd0, 0xd2, 0x69, 0xbc, 0xd2, 0xe7, - 0xdc, 0xa3, 0x86, 0xa6, 0xa6, 0x34, 0xd9, 0x2c, 0x17, 0x25, 0x74, 0x1d, 0xea, 0xd6, 0xd8, 0xec, - 0xdb, 0x07, 0x7d, 0xc7, 0x3e, 0x71, 0x79, 0xce, 0x56, 0xb3, 0xc6, 0xe6, 0xb7, 0x0f, 0x14, 0xfb, - 0xc4, 0x0d, 0xd2, 0x82, 0xca, 0x19, 0xd3, 0x82, 0xeb, 0x50, 0x37, 0xd5, 0x97, 0x64, 0xd7, 0xbe, - 0x35, 0x36, 0x69, 0x3a, 0x57, 0x54, 0x6a, 0xa6, 0xfa, 0x52, 0xb1, 0x4f, 0xb6, 0xc7, 0x26, 0xba, - 0x03, 0x6d, 0x43, 0x75, 0xbd, 0x7e, 0x38, 0x1f, 0xac, 0xd2, 0x7c, 0xb0, 0x45, 0xe6, 0x3f, 0x0f, - 0x72, 0xc2, 0x64, 0x82, 0x51, 0x9b, 0x23, 0xc1, 0xd0, 0x4c, 0x23, 0xd8, 0x08, 0xf2, 0x27, 0x18, - 0x9a, 0x69, 0x88, 0x6d, 0x3e, 0x81, 0x85, 0x7d, 0x1a, 0x05, 0xba, 0x9d, 0x7a, 0xa6, 0x8d, 0x79, - 0x42, 0x02, 0x40, 0x16, 0x2c, 0x2a, 0x3e, 0x38, 0xfa, 0x26, 0xd4, 0xa8, 0xf3, 0xa5, 0x6b, 0x1b, - 0xb9, 0xd6, 0x06, 0x0b, 0xc8, 0x6a, 0x0d, 0x1b, 0x9e, 0x4a, 0x57, 0x37, 0xf3, 0xad, 0x16, 0x0b, - 0x88, 0x5d, 0x1b, 0x38, 0x58, 0xf5, 0xb0, 0xb6, 0x76, 0xfa, 0xd8, 0x36, 0x47, 0x2a, 0x15, 0xa6, - 0x4e, 0x8b, 0x46, 0xfa, 0x69, 0x3f, 0x11, 0x5d, 0x1a, 0x88, 0xd1, 0x13, 0xc7, 0x36, 0x3b, 0x8b, - 0x4c, 0x97, 0xa2, 0xb3, 0xe8, 0x1a, 0x80, 0x6f, 0xd1, 0x54, 0xaf, 0xd3, 0xa6, 0x5c, 0xac, 0xf1, - 0x99, 0x47, 0xb4, 0xdc, 0xa3, 0xbb, 0x7d, 0x56, 0x58, 0xd1, 0xad, 0x61, 0x67, 0x89, 0x62, 0xac, - 0xfb, 0x95, 0x18, 0xdd, 0x1a, 0xca, 0x3f, 0x84, 0x4b, 0x81, 0x10, 0x85, 0x18, 0x96, 0xe4, 0xbd, - 0x34, 0x2b, 0xef, 0x27, 0x87, 0xf8, 0x3f, 0x2f, 0xc1, 0xf2, 0xae, 0x7a, 0x8c, 0x5f, 0x7d, 0x36, - 0x91, 0xcb, 0xca, 0x6d, 0xc2, 0x12, 0x4d, 0x20, 0x56, 0x43, 0xe7, 0x99, 0xe0, 0x66, 0xc3, 0x1c, - 0x4f, 0x2e, 0x44, 0xdf, 0x22, 0xf1, 0x01, 0x1e, 0x1c, 0xed, 0xd8, 0x7a, 0xe0, 0x62, 0xaf, 0xa5, - 0xec, 0xf3, 0x58, 0x40, 0x29, 0xe1, 0x15, 0x68, 0x07, 0x16, 0xa3, 0x6c, 0xf0, 0x9d, 0xeb, 0x3b, - 0x13, 0x73, 0xda, 0x80, 0xfa, 0x4a, 0x2b, 0xc2, 0x0c, 0x17, 0x75, 0x60, 0x81, 0x7b, 0x46, 0x6a, - 0x1a, 0xaa, 0x8a, 0x3f, 0x44, 0x3b, 0x70, 0x91, 0xdd, 0x60, 0x97, 0xcb, 0x3d, 0xbb, 0x7c, 0x35, - 0xd7, 0xe5, 0xd3, 0x96, 0x46, 0xd5, 0xa6, 0x76, 0x56, 0xb5, 0xe9, 0xc0, 0x02, 0x17, 0x65, 0x6a, - 0x2e, 0xaa, 0x8a, 0x3f, 0x24, 0x6c, 0x0e, 0x84, 0xba, 0x4e, 0x7f, 0x0b, 0x26, 0x48, 0x26, 0x06, - 0x01, 0x3d, 0xa7, 0x54, 0x5f, 0x3e, 0x83, 0xaa, 0x90, 0xf0, 0x42, 0x6e, 0x09, 0x17, 0x6b, 0xe2, - 0x66, 0xbc, 0x18, 0x33, 0xe3, 0xf2, 0xbf, 0x48, 0xd0, 0x58, 0x27, 0x57, 0xda, 0xb4, 0x87, 0xd4, - 0xe9, 0xdc, 0x86, 0x96, 0x83, 0x07, 0xb6, 0xa3, 0xf5, 0xb1, 0xe5, 0x39, 0x3a, 0x66, 0x49, 0x7b, - 0x49, 0x69, 0xb2, 0xd9, 0xcf, 0xd9, 0x24, 0x01, 0x23, 0x96, 0xd9, 0xf5, 0x54, 0x73, 0xd4, 0x3f, - 0x20, 0x16, 0xa0, 0xc0, 0xc0, 0xc4, 0x2c, 0x35, 0x00, 0x6f, 0x42, 0x23, 0x00, 0xf3, 0x6c, 0x8a, - 0xbf, 0xa4, 0xd4, 0xc5, 0xdc, 0x9e, 0x8d, 0xde, 0x82, 0x16, 0xa5, 0x69, 0xdf, 0xb0, 0x87, 0x7d, - 0x92, 0xe0, 0x72, 0x7f, 0xd4, 0xd0, 0xf8, 0xb1, 0x08, 0xaf, 0xa2, 0x50, 0xae, 0xfe, 0x03, 0xcc, - 0x3d, 0x92, 0x80, 0xda, 0xd5, 0x7f, 0x80, 0xe5, 0x7f, 0x96, 0xa0, 0xb9, 0xae, 0x7a, 0xea, 0xb6, - 0xad, 0xe1, 0xbd, 0x19, 0xfd, 0x77, 0x8e, 0x4a, 0xe8, 0x55, 0xa8, 0x89, 0x1b, 0xf0, 0x2b, 0x05, - 0x13, 0xe8, 0x09, 0xb4, 0xfc, 0x48, 0xb3, 0xcf, 0x12, 0xb0, 0x52, 0x66, 0x3c, 0x15, 0x72, 0x90, - 0xae, 0xd2, 0xf4, 0x97, 0xd1, 0xa1, 0xfc, 0x04, 0x1a, 0xe1, 0x9f, 0x09, 0xd6, 0xdd, 0xb8, 0xa0, - 0x88, 0x09, 0x22, 0x8d, 0xdb, 0x63, 0x93, 0xf0, 0x94, 0x1b, 0x16, 0x7f, 0x28, 0xff, 0x48, 0x82, - 0x26, 0xf7, 0xea, 0xbb, 0xe2, 0xcd, 0x80, 0x5e, 0x4d, 0xa2, 0x57, 0xa3, 0x7f, 0xa3, 0x5f, 0x89, - 0x96, 0xf9, 0xde, 0x4a, 0x35, 0x02, 0x74, 0x13, 0x1a, 0x73, 0x46, 0x5c, 0x7a, 0x9e, 0x94, 0xff, - 0x4b, 0x22, 0x68, 0x9c, 0x35, 0x54, 0xd0, 0x3a, 0xb0, 0xa0, 0x6a, 0x9a, 0x83, 0x5d, 0x97, 0x9f, - 0xc3, 0x1f, 0x92, 0x5f, 0x8e, 0xb1, 0xe3, 0xfa, 0x22, 0x5f, 0x54, 0xfc, 0x21, 0xfa, 0x26, 0x54, - 0x45, 0x90, 0xca, 0xaa, 0xe3, 0x37, 0xb3, 0xcf, 0xc9, 0x13, 0x54, 0xb1, 0x42, 0xfe, 0xbb, 0x02, - 0xb4, 0x38, 0xc1, 0xd6, 0xb8, 0xdb, 0x9d, 0xac, 0x7c, 0x6b, 0xd0, 0x38, 0x08, 0x74, 0x7f, 0x52, - 0x29, 0x2a, 0x6c, 0x22, 0x22, 0x6b, 0xa6, 0x29, 0x60, 0xd4, 0xf1, 0x97, 0xe6, 0x72, 0xfc, 0xe5, - 0xb3, 0x5a, 0xb0, 0x64, 0x28, 0x58, 0x49, 0x09, 0x05, 0xe5, 0x5f, 0x87, 0x7a, 0x68, 0x03, 0x6a, - 0xa1, 0x59, 0x0d, 0x8b, 0x53, 0xcc, 0x1f, 0xa2, 0x8f, 0x82, 0xf0, 0x87, 0x91, 0xea, 0x72, 0xca, - 0x59, 0x62, 0x91, 0x8f, 0xfc, 0x8f, 0x12, 0x54, 0xf8, 0xce, 0x37, 0xa0, 0xce, 0x8d, 0x0e, 0x0d, - 0x0d, 0xd9, 0xee, 0xc0, 0xa7, 0x48, 0x6c, 0x78, 0x7e, 0x56, 0xe7, 0x32, 0x54, 0x63, 0xf6, 0x66, - 0x81, 0xbb, 0x05, 0xff, 0xa7, 0x90, 0x91, 0x21, 0x3f, 0x11, 0xfb, 0x82, 0x2e, 0x41, 0xd9, 0xb0, - 0x87, 0xe2, 0x4d, 0x88, 0x0d, 0x48, 0xf2, 0xb3, 0xb2, 0x81, 0x3d, 0x05, 0x0f, 0xec, 0x63, 0xec, - 0x9c, 0xce, 0x5f, 0xfb, 0x7c, 0x18, 0x12, 0xf3, 0x9c, 0xb9, 0x98, 0x58, 0x80, 0x1e, 0x06, 0x4c, - 0x28, 0xa6, 0x15, 0x7e, 0xc2, 0x76, 0x87, 0x0b, 0x69, 0xc0, 0x8c, 0xdf, 0x63, 0x55, 0xdc, 0xe8, - 0x55, 0x66, 0x8d, 0x76, 0xce, 0x25, 0x5f, 0x91, 0x7f, 0x2e, 0x41, 0x37, 0xa8, 0x2c, 0xb9, 0x6b, - 0xa7, 0xf3, 0xbe, 0x91, 0x9c, 0x4f, 0x1a, 0xf5, 0xcb, 0xa2, 0x88, 0x4f, 0x94, 0x36, 0x57, 0x02, + 0xa9, 0xe7, 0x7d, 0x2d, 0x04, 0xbc, 0x4e, 0x35, 0x21, 0xa4, 0x92, 0xeb, 0x2c, 0x74, 0xe0, 0xe4, + 0x93, 0xff, 0x4c, 0x82, 0xcb, 0x8f, 0x06, 0xcf, 0xc7, 0xba, 0x83, 0x39, 0xd0, 0x96, 0x3d, 0x38, + 0x9e, 0x9d, 0xb8, 0x41, 0x98, 0x55, 0x88, 0x84, 0x59, 0xd3, 0x42, 0xf3, 0x65, 0xa8, 0x78, 0x2c, + 0xae, 0x63, 0x91, 0x0a, 0x1f, 0xd1, 0xf3, 0x29, 0xd8, 0xc0, 0xaa, 0xfb, 0x7f, 0xf3, 0x7c, 0x3f, + 0x29, 0x41, 0xe3, 0x0b, 0x1e, 0x8e, 0x51, 0xaf, 0x1d, 0x97, 0x24, 0x29, 0x3d, 0xf0, 0x0a, 0x45, + 0x70, 0x69, 0x41, 0xdd, 0x26, 0x34, 0x5d, 0x8c, 0x8f, 0x67, 0xf1, 0xd1, 0x0d, 0xb2, 0x50, 0xf8, + 0xd6, 0x2d, 0x58, 0x1a, 0x5b, 0x34, 0x35, 0xc0, 0x1a, 0x27, 0x20, 0x93, 0xdc, 0xe9, 0xb6, 0x3b, + 0xb9, 0x10, 0x3d, 0xe1, 0xd9, 0x47, 0x68, 0xaf, 0x72, 0xae, 0xbd, 0xe2, 0xcb, 0x50, 0x0f, 0xda, + 0x9a, 0x63, 0x8f, 0x46, 0x58, 0xeb, 0xbb, 0xfe, 0x56, 0x95, 0x7c, 0x5b, 0xf1, 0x75, 0x62, 0xab, + 0x0f, 0xe0, 0x62, 0xfc, 0xa4, 0x3d, 0x8d, 0x04, 0xa4, 0x84, 0x87, 0x69, 0x3f, 0xa1, 0x7b, 0xb0, + 0x94, 0x84, 0xaf, 0x52, 0xf8, 0xe4, 0x0f, 0xe8, 0x7d, 0x40, 0xb1, 0xa3, 0x12, 0xf0, 0x1a, 0x03, + 0x8f, 0x1e, 0xa6, 0xa7, 0xb9, 0xf2, 0x8f, 0x25, 0x58, 0x7e, 0xa6, 0x7a, 0x83, 0xa3, 0x0d, 0x93, + 0xeb, 0xda, 0x1c, 0xb6, 0xea, 0x53, 0xa8, 0x9d, 0x70, 0xb9, 0xf0, 0x1d, 0xd2, 0x8d, 0x14, 0xfa, + 0x84, 0x25, 0x50, 0x09, 0x56, 0xc8, 0x5f, 0x49, 0x70, 0xe9, 0x71, 0x28, 0x2f, 0x7c, 0x0d, 0x56, + 0x73, 0x5a, 0x42, 0xfb, 0x36, 0xb4, 0x4c, 0xd5, 0x39, 0x4e, 0xe4, 0xb3, 0xb1, 0x59, 0xf9, 0x05, + 0x00, 0x1f, 0x6d, 0xbb, 0xc3, 0x19, 0xce, 0xff, 0x09, 0x2c, 0x70, 0xac, 0xdc, 0x7c, 0x4e, 0x93, + 0x33, 0x1f, 0x5c, 0xfe, 0xfd, 0x0a, 0xd4, 0x43, 0x3f, 0xa0, 0x16, 0x14, 0x84, 0x5e, 0x17, 0x52, + 0xa8, 0x50, 0x98, 0x9e, 0x6a, 0x15, 0x93, 0xa9, 0xd6, 0x6d, 0x68, 0xe9, 0x34, 0x5e, 0xe9, 0x73, + 0xee, 0x51, 0x43, 0x53, 0x53, 0x9a, 0x6c, 0x96, 0x8b, 0x12, 0xba, 0x0e, 0x75, 0x6b, 0x6c, 0xf6, + 0xed, 0xc3, 0xbe, 0x63, 0x9f, 0xba, 0x3c, 0x67, 0xab, 0x59, 0x63, 0xf3, 0xdb, 0x87, 0x8a, 0x7d, + 0xea, 0x06, 0x69, 0x41, 0xe5, 0x9c, 0x69, 0xc1, 0x75, 0xa8, 0x9b, 0xea, 0x0b, 0xb2, 0x6b, 0xdf, + 0x1a, 0x9b, 0x34, 0x9d, 0x2b, 0x2a, 0x35, 0x53, 0x7d, 0xa1, 0xd8, 0xa7, 0x3b, 0x63, 0x13, 0xdd, + 0x81, 0xb6, 0xa1, 0xba, 0x5e, 0x3f, 0x9c, 0x0f, 0x56, 0x69, 0x3e, 0xd8, 0x22, 0xf3, 0x9f, 0x07, + 0x39, 0x61, 0x32, 0xc1, 0xa8, 0xcd, 0x91, 0x60, 0x68, 0xa6, 0x11, 0x6c, 0x04, 0xf9, 0x13, 0x0c, + 0xcd, 0x34, 0xc4, 0x36, 0x9f, 0xc0, 0xc2, 0x01, 0x8d, 0x02, 0xdd, 0x4e, 0x3d, 0xd3, 0xc6, 0x3c, + 0x26, 0x01, 0x20, 0x0b, 0x16, 0x15, 0x1f, 0x1c, 0x7d, 0x13, 0x6a, 0xd4, 0xf9, 0xd2, 0xb5, 0x8d, + 0x5c, 0x6b, 0x83, 0x05, 0x64, 0xb5, 0x86, 0x0d, 0x4f, 0xa5, 0xab, 0x9b, 0xf9, 0x56, 0x8b, 0x05, + 0xc4, 0xae, 0x0d, 0x1c, 0xac, 0x7a, 0x58, 0x5b, 0x3b, 0x5b, 0xb7, 0xcd, 0x91, 0x4a, 0x85, 0xa9, + 0xd3, 0xa2, 0x91, 0x7e, 0xda, 0x4f, 0x44, 0x97, 0x06, 0x62, 0xf4, 0xd8, 0xb1, 0xcd, 0xce, 0x22, + 0xd3, 0xa5, 0xe8, 0x2c, 0xba, 0x06, 0xe0, 0x5b, 0x34, 0xd5, 0xeb, 0xb4, 0x29, 0x17, 0x6b, 0x7c, + 0xe6, 0x11, 0x2d, 0xf7, 0xe8, 0x6e, 0x9f, 0x15, 0x56, 0x74, 0x6b, 0xd8, 0x59, 0xa2, 0x18, 0xeb, + 0x7e, 0x25, 0x46, 0xb7, 0x86, 0xf2, 0x0f, 0xe1, 0x52, 0x20, 0x44, 0x21, 0x86, 0x25, 0x79, 0x2f, + 0xcd, 0xca, 0xfb, 0xc9, 0x21, 0xfe, 0xcf, 0x4b, 0xb0, 0xbc, 0xa7, 0x9e, 0xe0, 0x57, 0x9f, 0x4d, + 0xe4, 0xb2, 0x72, 0x5b, 0xb0, 0x44, 0x13, 0x88, 0xd5, 0xd0, 0x79, 0x26, 0xb8, 0xd9, 0x30, 0xc7, + 0x93, 0x0b, 0xd1, 0xb7, 0x48, 0x7c, 0x80, 0x07, 0xc7, 0xbb, 0xb6, 0x1e, 0xb8, 0xd8, 0x6b, 0x29, + 0xfb, 0xac, 0x0b, 0x28, 0x25, 0xbc, 0x02, 0xed, 0xc2, 0x62, 0x94, 0x0d, 0xbe, 0x73, 0x7d, 0x67, + 0x62, 0x4e, 0x1b, 0x50, 0x5f, 0x69, 0x45, 0x98, 0xe1, 0xa2, 0x0e, 0x2c, 0x70, 0xcf, 0x48, 0x4d, + 0x43, 0x55, 0xf1, 0x87, 0x68, 0x17, 0x2e, 0xb2, 0x1b, 0xec, 0x71, 0xb9, 0x67, 0x97, 0xaf, 0xe6, + 0xba, 0x7c, 0xda, 0xd2, 0xa8, 0xda, 0xd4, 0xce, 0xab, 0x36, 0x1d, 0x58, 0xe0, 0xa2, 0x4c, 0xcd, + 0x45, 0x55, 0xf1, 0x87, 0x84, 0xcd, 0x81, 0x50, 0xd7, 0xe9, 0x6f, 0xc1, 0x04, 0xc9, 0xc4, 0x20, + 0xa0, 0xe7, 0x94, 0xea, 0xcb, 0x67, 0x50, 0x15, 0x12, 0x5e, 0xc8, 0x2d, 0xe1, 0x62, 0x4d, 0xdc, + 0x8c, 0x17, 0x63, 0x66, 0x5c, 0xfe, 0x17, 0x09, 0x1a, 0x1b, 0xe4, 0x4a, 0x5b, 0xf6, 0x90, 0x3a, + 0x9d, 0xdb, 0xd0, 0x72, 0xf0, 0xc0, 0x76, 0xb4, 0x3e, 0xb6, 0x3c, 0x47, 0xc7, 0x2c, 0x69, 0x2f, + 0x29, 0x4d, 0x36, 0xfb, 0x39, 0x9b, 0x24, 0x60, 0xc4, 0x32, 0xbb, 0x9e, 0x6a, 0x8e, 0xfa, 0x87, + 0xc4, 0x02, 0x14, 0x18, 0x98, 0x98, 0xa5, 0x06, 0xe0, 0x4d, 0x68, 0x04, 0x60, 0x9e, 0x4d, 0xf1, + 0x97, 0x94, 0xba, 0x98, 0xdb, 0xb7, 0xd1, 0x5b, 0xd0, 0xa2, 0x34, 0xed, 0x1b, 0xf6, 0xb0, 0x4f, + 0x12, 0x5c, 0xee, 0x8f, 0x1a, 0x1a, 0x3f, 0x16, 0xe1, 0x55, 0x14, 0xca, 0xd5, 0x7f, 0x80, 0xb9, + 0x47, 0x12, 0x50, 0x7b, 0xfa, 0x0f, 0xb0, 0xfc, 0xcf, 0x12, 0x34, 0x37, 0x54, 0x4f, 0xdd, 0xb1, + 0x35, 0xbc, 0x3f, 0xa3, 0xff, 0xce, 0x51, 0x09, 0xbd, 0x0a, 0x35, 0x71, 0x03, 0x7e, 0xa5, 0x60, + 0x02, 0x3d, 0x86, 0x96, 0x1f, 0x69, 0xf6, 0x59, 0x02, 0x56, 0xca, 0x8c, 0xa7, 0x42, 0x0e, 0xd2, + 0x55, 0x9a, 0xfe, 0x32, 0x3a, 0x94, 0x1f, 0x43, 0x23, 0xfc, 0x33, 0xc1, 0xba, 0x17, 0x17, 0x14, + 0x31, 0x41, 0xa4, 0x71, 0x67, 0x6c, 0x12, 0x9e, 0x72, 0xc3, 0xe2, 0x0f, 0xe5, 0x1f, 0x49, 0xd0, + 0xe4, 0x5e, 0x7d, 0x4f, 0xbc, 0x19, 0xd0, 0xab, 0x49, 0xf4, 0x6a, 0xf4, 0x6f, 0xf4, 0x2b, 0xd1, + 0x32, 0xdf, 0x5b, 0xa9, 0x46, 0x80, 0x6e, 0x42, 0x63, 0xce, 0x88, 0x4b, 0xcf, 0x93, 0xf2, 0x7f, + 0x49, 0x04, 0x8d, 0xb3, 0x86, 0x0a, 0x5a, 0x07, 0x16, 0x54, 0x4d, 0x73, 0xb0, 0xeb, 0xf2, 0x73, + 0xf8, 0x43, 0xf2, 0xcb, 0x09, 0x76, 0x5c, 0x5f, 0xe4, 0x8b, 0x8a, 0x3f, 0x44, 0xdf, 0x84, 0xaa, + 0x08, 0x52, 0x59, 0x75, 0xfc, 0x66, 0xf6, 0x39, 0x79, 0x82, 0x2a, 0x56, 0xc8, 0x7f, 0x57, 0x80, + 0x16, 0x27, 0xd8, 0x1a, 0x77, 0xbb, 0x93, 0x95, 0x6f, 0x0d, 0x1a, 0x87, 0x81, 0xee, 0x4f, 0x2a, + 0x45, 0x85, 0x4d, 0x44, 0x64, 0xcd, 0x34, 0x05, 0x8c, 0x3a, 0xfe, 0xd2, 0x5c, 0x8e, 0xbf, 0x7c, + 0x5e, 0x0b, 0x96, 0x0c, 0x05, 0x2b, 0x29, 0xa1, 0xa0, 0xfc, 0xeb, 0x50, 0x0f, 0x6d, 0x40, 0x2d, + 0x34, 0xab, 0x61, 0x71, 0x8a, 0xf9, 0x43, 0xf4, 0x51, 0x10, 0xfe, 0x30, 0x52, 0x5d, 0x4e, 0x39, + 0x4b, 0x2c, 0xf2, 0x91, 0xff, 0x51, 0x82, 0x0a, 0xdf, 0xf9, 0x06, 0xd4, 0xb9, 0xd1, 0xa1, 0xa1, + 0x21, 0xdb, 0x1d, 0xf8, 0x14, 0x89, 0x0d, 0x5f, 0x9e, 0xd5, 0xb9, 0x0c, 0xd5, 0x98, 0xbd, 0x59, + 0xe0, 0x6e, 0xc1, 0xff, 0x29, 0x64, 0x64, 0xc8, 0x4f, 0xc4, 0xbe, 0xa0, 0x4b, 0x50, 0x36, 0xec, + 0xa1, 0x78, 0x13, 0x62, 0x03, 0x92, 0xfc, 0xac, 0x6c, 0x62, 0x4f, 0xc1, 0x03, 0xfb, 0x04, 0x3b, + 0x67, 0xf3, 0xd7, 0x3e, 0x1f, 0x86, 0xc4, 0x3c, 0x67, 0x2e, 0x26, 0x16, 0xa0, 0x87, 0x01, 0x13, + 0x8a, 0x69, 0x85, 0x9f, 0xb0, 0xdd, 0xe1, 0x42, 0x1a, 0x30, 0xe3, 0xf7, 0x58, 0x15, 0x37, 0x7a, + 0x95, 0x59, 0xa3, 0x9d, 0x97, 0x92, 0xaf, 0xc8, 0x3f, 0x97, 0xa0, 0x1b, 0x54, 0x96, 0xdc, 0xb5, + 0xb3, 0x79, 0xdf, 0x48, 0x5e, 0x4e, 0x1a, 0xf5, 0xcb, 0xa2, 0x88, 0x4f, 0x94, 0x36, 0x57, 0x02, 0xe4, 0x97, 0xf0, 0x2d, 0x5a, 0xa4, 0x4e, 0x5e, 0x68, 0x1e, 0x91, 0xe9, 0x42, 0x55, 0x94, 0x37, - 0x58, 0x21, 0x5f, 0x8c, 0xe5, 0x3f, 0x94, 0xe0, 0xf2, 0x06, 0xf6, 0x9e, 0x44, 0x2b, 0x23, 0xaf, + 0x58, 0x21, 0x5f, 0x8c, 0xe5, 0x3f, 0x94, 0xe0, 0xf2, 0x26, 0xf6, 0x1e, 0x47, 0x2b, 0x23, 0xaf, 0x9b, 0xaf, 0x26, 0x65, 0x6b, 0xe2, 0x50, 0xaf, 0x8a, 0x08, 0xbf, 0x25, 0x41, 0x87, 0x63, 0xa1, 0x38, 0x49, 0x36, 0x63, 0x60, 0x0f, 0x6b, 0x5f, 0x77, 0x96, 0xff, 0x3f, 0x12, 0xb4, 0xc3, 0x9e, 0x94, 0x3a, 0xc3, 0x8f, 0xa1, 0x4c, 0x8b, 0x29, 0xfc, 0x04, 0x53, 0xd5, 0x9d, 0x41, 0x13, 0x53, - 0x4c, 0xc3, 0xe7, 0x3d, 0xe1, 0xf4, 0xf9, 0x30, 0x70, 0xe7, 0xc5, 0xb3, 0xbb, 0x73, 0x1e, 0xde, + 0x4c, 0xc3, 0xe7, 0x7d, 0xe1, 0xf4, 0xf9, 0x30, 0x70, 0xe7, 0xc5, 0xf3, 0xbb, 0x73, 0x1e, 0xde, 0xd8, 0x63, 0xb2, 0x2f, 0xab, 0x42, 0x06, 0x13, 0xe8, 0x53, 0xa8, 0xb0, 0x5e, 0x0b, 0xfe, 0x88, 0x76, 0x3b, 0xba, 0x35, 0xef, 0xc3, 0x08, 0x95, 0xf6, 0xe9, 0x84, 0xc2, 0x17, 0xc9, 0xbf, 0x0a, - 0xcb, 0x41, 0x22, 0xc9, 0xd0, 0xce, 0x2a, 0x88, 0xf2, 0xbf, 0x49, 0x70, 0x71, 0xf7, 0xd4, 0x1a, + 0xcb, 0x41, 0x22, 0xc9, 0xd0, 0xce, 0x2a, 0x88, 0xf2, 0xbf, 0x49, 0x70, 0x71, 0xef, 0xcc, 0x1a, 0xc4, 0x45, 0x7a, 0x19, 0x2a, 0x23, 0x43, 0x0d, 0x8a, 0xa2, 0x7c, 0x44, 0x43, 0x3b, 0x86, 0x1b, - 0x6b, 0xc4, 0x2f, 0x30, 0x9a, 0xd5, 0xc5, 0xdc, 0x9e, 0x3d, 0xd5, 0x5d, 0xdf, 0x16, 0x99, 0x2f, + 0x6b, 0xc4, 0x2f, 0x30, 0x9a, 0xd5, 0xc5, 0xdc, 0xbe, 0x3d, 0xd5, 0x5d, 0xdf, 0x16, 0x99, 0x2f, 0xd6, 0x98, 0x07, 0x62, 0x95, 0xa6, 0xa6, 0x98, 0xa5, 0x1e, 0xe8, 0x53, 0x00, 0xea, 0xa4, 0xfb, - 0x67, 0x71, 0xcc, 0x74, 0xc5, 0x26, 0x31, 0xc3, 0x3f, 0x2b, 0x40, 0x27, 0x44, 0xa5, 0xaf, 0x3b, - 0x66, 0xc9, 0xc8, 0xb4, 0x8a, 0xe7, 0x94, 0x69, 0x95, 0xe6, 0x8f, 0x53, 0xca, 0x69, 0x71, 0xca, - 0xbf, 0x17, 0xa0, 0x15, 0x50, 0x6d, 0xc7, 0x50, 0xad, 0x4c, 0x49, 0xd8, 0x15, 0x31, 0x7a, 0x94, - 0x4e, 0xef, 0xa5, 0xe9, 0x49, 0x06, 0x23, 0x94, 0xd8, 0x16, 0xe8, 0x1a, 0x65, 0xba, 0xe3, 0xb1, - 0x9a, 0x15, 0xcf, 0x0b, 0x98, 0x42, 0xea, 0x26, 0x46, 0xf7, 0x00, 0x71, 0x2d, 0xea, 0xeb, 0x56, - 0xdf, 0xc5, 0x03, 0xdb, 0xd2, 0x98, 0x7e, 0x95, 0x95, 0x36, 0xff, 0xa5, 0x67, 0xed, 0xb2, 0x79, - 0xf4, 0x31, 0x94, 0xbc, 0xd3, 0x11, 0x8b, 0x40, 0x5a, 0xa9, 0x3e, 0x3c, 0x38, 0xd7, 0xde, 0xe9, - 0x08, 0x2b, 0x14, 0xdc, 0x6f, 0xc6, 0xf1, 0x1c, 0xf5, 0x98, 0x87, 0x73, 0x25, 0x25, 0x34, 0x43, - 0x2c, 0x86, 0x4f, 0xc3, 0x05, 0x16, 0xf6, 0xf0, 0x21, 0x93, 0x6c, 0x5f, 0x69, 0xfb, 0x9e, 0x67, - 0xd0, 0xaa, 0x1b, 0x95, 0x6c, 0x7f, 0x76, 0xcf, 0x33, 0xe4, 0xbf, 0x2f, 0x40, 0x3b, 0xc0, 0xac, - 0x60, 0x77, 0x6c, 0x64, 0x2b, 0xdc, 0xe4, 0x7a, 0xc7, 0x34, 0x5d, 0xfb, 0x16, 0xd4, 0x39, 0xdb, - 0xcf, 0x20, 0x36, 0xc0, 0x96, 0x6c, 0x4e, 0x90, 0xe3, 0xf2, 0x39, 0xc9, 0x71, 0xe5, 0x8c, 0x72, - 0x2c, 0xff, 0x54, 0x82, 0x37, 0x12, 0xc6, 0x6f, 0x22, 0x01, 0x27, 0x67, 0x65, 0xdc, 0x28, 0xc6, - 0xb7, 0xe4, 0x66, 0xfc, 0x21, 0x54, 0x1c, 0xba, 0x3b, 0x7f, 0xd3, 0xb9, 0x35, 0x51, 0x86, 0xd8, - 0x41, 0x14, 0xbe, 0x44, 0xfe, 0x03, 0x09, 0x56, 0x92, 0x47, 0x9d, 0xc3, 0x37, 0xaf, 0xc1, 0x02, - 0xdb, 0xda, 0x57, 0xb5, 0x3b, 0x93, 0x55, 0x2d, 0x20, 0x8e, 0xe2, 0x2f, 0x94, 0x77, 0x61, 0xd9, - 0x77, 0xe1, 0x01, 0x81, 0xb7, 0xb0, 0xa7, 0x4e, 0xc8, 0x49, 0x6e, 0x40, 0x9d, 0x05, 0xb7, 0x2c, - 0xd6, 0x67, 0xd9, 0x3c, 0xec, 0x8b, 0x22, 0x98, 0xfc, 0x5f, 0x12, 0x5c, 0xa2, 0x3e, 0x30, 0xfe, - 0x88, 0x92, 0xe7, 0x81, 0x4d, 0x16, 0xc5, 0x82, 0x6d, 0xd5, 0xe4, 0x0d, 0x1d, 0x35, 0x25, 0x32, - 0x87, 0x7a, 0xc9, 0x1a, 0x59, 0x6a, 0xee, 0x1a, 0xbc, 0xc8, 0x92, 0x3c, 0x99, 0x3e, 0xc8, 0xc6, - 0x8b, 0x63, 0x81, 0xef, 0x2d, 0xcd, 0xe2, 0x7b, 0x37, 0xe1, 0x8d, 0xd8, 0x4d, 0xe7, 0xe0, 0xa8, - 0xfc, 0x97, 0x12, 0x61, 0x47, 0xa4, 0x31, 0x66, 0xf6, 0x98, 0xf2, 0x9a, 0x78, 0xbd, 0xe9, 0xeb, - 0x5a, 0xdc, 0x54, 0x68, 0xe8, 0x33, 0xa8, 0x59, 0xf8, 0xa4, 0x1f, 0x0e, 0x69, 0x72, 0x04, 0xdc, - 0x55, 0x0b, 0x9f, 0xd0, 0xbf, 0xe4, 0x6d, 0x58, 0x49, 0x1c, 0x75, 0x9e, 0xbb, 0xff, 0x83, 0x04, - 0x97, 0xd7, 0x1d, 0x7b, 0xf4, 0x85, 0xee, 0x78, 0x63, 0xd5, 0x88, 0xbe, 0x75, 0xbf, 0x9a, 0xa2, - 0xd3, 0xd3, 0x50, 0x70, 0xcb, 0xe4, 0xe7, 0x5e, 0x8a, 0x06, 0x25, 0x0f, 0xc5, 0x2f, 0x1d, 0x0a, - 0x85, 0xff, 0xb3, 0x98, 0x76, 0x78, 0x0e, 0x37, 0x25, 0xbc, 0xc8, 0x13, 0xfb, 0xa7, 0xd6, 0xa8, - 0x8b, 0xb3, 0xd6, 0xa8, 0x33, 0x8c, 0x78, 0xe9, 0x9c, 0x8c, 0xf8, 0x99, 0x8b, 0x26, 0x4f, 0x21, - 0xfa, 0x7e, 0x40, 0x9d, 0xec, 0x4c, 0x0f, 0x0f, 0x6b, 0x00, 0x41, 0x2d, 0x9d, 0xf7, 0x35, 0xe6, - 0xd9, 0x26, 0xb4, 0x8a, 0x70, 0x4b, 0x38, 0x4c, 0xee, 0xb0, 0x43, 0xd5, 0xdd, 0xef, 0x40, 0x37, - 0x4d, 0x4a, 0xe7, 0x91, 0xfc, 0x9f, 0x15, 0x00, 0x7a, 0xa2, 0x15, 0x76, 0x36, 0x5f, 0x70, 0x0b, - 0x42, 0x41, 0x45, 0xa0, 0xef, 0x61, 0x29, 0xd2, 0x88, 0x4a, 0x88, 0x74, 0x91, 0xc0, 0x24, 0x52, - 0x48, 0x8d, 0xee, 0x13, 0xd2, 0x1a, 0x26, 0x14, 0x71, 0xf3, 0x7b, 0x05, 0x6a, 0x8e, 0x7d, 0xd2, - 0x27, 0x6a, 0xa6, 0xf9, 0xbd, 0xbe, 0x8e, 0x7d, 0x42, 0x94, 0x4f, 0x43, 0x2b, 0xb0, 0xe0, 0xa9, - 0xee, 0x11, 0xd9, 0xbf, 0x12, 0x6a, 0xb7, 0xd0, 0xd0, 0x25, 0x28, 0x1f, 0xe8, 0x06, 0x66, 0xaf, - 0xfb, 0x35, 0x85, 0x0d, 0xd0, 0x37, 0xfc, 0xa6, 0xb4, 0x6a, 0xee, 0x96, 0x1a, 0xd6, 0x97, 0xf6, - 0x95, 0x04, 0x8b, 0x01, 0xd5, 0xa8, 0x01, 0x22, 0x36, 0x8d, 0xda, 0xb3, 0xc7, 0xb6, 0xc6, 0x4c, - 0x45, 0x2b, 0xc3, 0x23, 0xb0, 0x85, 0xcc, 0x6a, 0x05, 0x4b, 0x26, 0x65, 0xbb, 0xe4, 0x5e, 0xe4, - 0xd2, 0xba, 0xe6, 0xb7, 0x98, 0x54, 0x1c, 0xfb, 0xa4, 0xa7, 0x09, 0x6a, 0xb0, 0x46, 0x5e, 0x96, - 0xdb, 0x11, 0x6a, 0x3c, 0xa6, 0xbd, 0xbc, 0xb7, 0xa0, 0x89, 0x1d, 0xc7, 0x76, 0xfa, 0x26, 0x76, - 0x5d, 0x75, 0x88, 0x79, 0x98, 0xdd, 0xa0, 0x93, 0x5b, 0x6c, 0x4e, 0xfe, 0xa7, 0x22, 0xb4, 0x82, - 0xab, 0xf8, 0x0f, 0xd5, 0xba, 0xe6, 0x3f, 0x54, 0xeb, 0x84, 0x75, 0xe0, 0x30, 0x53, 0x28, 0x98, - 0xbb, 0x56, 0xe8, 0x48, 0x4a, 0x8d, 0xcf, 0xf6, 0x34, 0xe2, 0x96, 0x89, 0x92, 0x59, 0xb6, 0x86, - 0x03, 0xe6, 0x82, 0x3f, 0xc5, 0x79, 0x1b, 0x91, 0x91, 0x52, 0x0e, 0x19, 0x29, 0xe7, 0x90, 0x91, - 0x4a, 0x8a, 0x8c, 0x2c, 0x43, 0x65, 0x7f, 0x3c, 0x38, 0xc2, 0x1e, 0x0f, 0x8a, 0xf9, 0x28, 0x2a, - 0x3b, 0xd5, 0x98, 0xec, 0x08, 0x11, 0xa9, 0x85, 0x45, 0xe4, 0x0a, 0xd4, 0xd8, 0x8b, 0x69, 0xdf, - 0x73, 0xe9, 0xbb, 0x50, 0x51, 0xa9, 0xb2, 0x89, 0x3d, 0x17, 0x7d, 0xe2, 0x87, 0x73, 0xf5, 0x34, - 0x65, 0xa7, 0x56, 0x27, 0x26, 0x25, 0x7e, 0x30, 0xf7, 0x0e, 0x2c, 0x86, 0xc8, 0x41, 0x7d, 0x44, - 0x83, 0x1e, 0x35, 0x14, 0xb4, 0x53, 0x37, 0x71, 0x1b, 0x5a, 0x01, 0x49, 0x28, 0x5c, 0x93, 0xe5, - 0x4a, 0x62, 0x96, 0x80, 0xc9, 0xdf, 0x07, 0x14, 0x60, 0x9a, 0x2f, 0xb2, 0x8b, 0xb1, 0xb2, 0x10, - 0x67, 0xa5, 0xfc, 0x57, 0x12, 0x2c, 0x85, 0x91, 0xcd, 0xea, 0x24, 0x3f, 0x83, 0x3a, 0x7b, 0x45, - 0xeb, 0x13, 0x25, 0xe5, 0x75, 0x97, 0x6b, 0x13, 0x69, 0xa8, 0x40, 0xd0, 0xb6, 0x4f, 0x44, 0xe1, - 0xc4, 0x76, 0x8e, 0x74, 0x6b, 0xd8, 0x27, 0x27, 0xf3, 0x55, 0xa3, 0xc1, 0x27, 0xb7, 0xc9, 0x9c, - 0xfc, 0x63, 0x09, 0xae, 0x3f, 0x1b, 0x69, 0xaa, 0x87, 0x43, 0xd1, 0xc2, 0xbc, 0x9d, 0x80, 0x1f, - 0xfb, 0xad, 0x78, 0x85, 0x7c, 0x2f, 0x41, 0x0c, 0x5a, 0xde, 0x82, 0xcb, 0x0a, 0x76, 0xb1, 0xa5, - 0x45, 0x7e, 0x9c, 0xb9, 0x5c, 0x32, 0x82, 0x6e, 0xda, 0x76, 0xf3, 0xf0, 0x9e, 0x85, 0x6d, 0x7d, - 0x87, 0x6c, 0xeb, 0x71, 0x2b, 0x44, 0xa2, 0x05, 0x8a, 0xc7, 0x93, 0xff, 0xba, 0x00, 0x2b, 0x8f, - 0x34, 0x8d, 0x1b, 0x30, 0x1e, 0x88, 0xbc, 0xaa, 0x18, 0x31, 0x1e, 0x43, 0x15, 0x93, 0x31, 0xd4, - 0x79, 0x19, 0x15, 0x6e, 0x5e, 0xad, 0xb1, 0xe9, 0xbb, 0x0d, 0x87, 0x35, 0xaf, 0x3c, 0xe4, 0xaf, - 0x39, 0x24, 0x63, 0xa5, 0xae, 0x63, 0x7a, 0x68, 0x51, 0xf5, 0xcb, 0x3e, 0xf2, 0x08, 0x3a, 0x49, - 0x62, 0xcd, 0xa9, 0x99, 0x3e, 0x45, 0x46, 0x36, 0x2b, 0x11, 0x36, 0x48, 0xf4, 0x40, 0xa7, 0x76, - 0x6c, 0x57, 0xfe, 0xef, 0x02, 0x74, 0x76, 0xd5, 0x63, 0xfc, 0x8b, 0xc3, 0xa0, 0xef, 0xc2, 0x25, - 0x57, 0x3d, 0xc6, 0xfd, 0x50, 0x4e, 0xd8, 0x77, 0xf0, 0x0b, 0x1e, 0x7d, 0xbd, 0x9b, 0xa6, 0x98, - 0xa9, 0xcd, 0x1f, 0xca, 0x92, 0x1b, 0x99, 0x57, 0xf0, 0x0b, 0xf4, 0x36, 0x2c, 0x86, 0x9b, 0x88, - 0xc8, 0xd1, 0xaa, 0x94, 0xe4, 0xcd, 0x50, 0x8f, 0x50, 0x4f, 0x93, 0x5f, 0xc0, 0xd5, 0x67, 0x96, - 0x8b, 0xbd, 0x5e, 0xd0, 0xe7, 0x32, 0x67, 0xf6, 0x74, 0x03, 0xea, 0x01, 0xe1, 0x13, 0x9d, 0xfc, - 0x9a, 0x2b, 0xdb, 0xd0, 0xdd, 0x0a, 0xda, 0xdc, 0xdc, 0x75, 0xd6, 0xa8, 0xf0, 0x0a, 0x11, 0x1e, - 0x88, 0xbe, 0x1d, 0x05, 0x1f, 0x60, 0x07, 0x5b, 0x03, 0xbc, 0x69, 0x0f, 0x8e, 0x42, 0xed, 0xad, - 0x52, 0xb8, 0xbd, 0x75, 0xd6, 0x76, 0xd9, 0xbb, 0x0f, 0x44, 0xcb, 0xdc, 0xde, 0xe9, 0x08, 0xa3, - 0x05, 0x28, 0x6e, 0xe3, 0x93, 0xf6, 0x05, 0x04, 0x50, 0xd9, 0xb6, 0x1d, 0x53, 0x35, 0xda, 0x12, - 0xaa, 0xc3, 0x02, 0x7f, 0x66, 0x68, 0x17, 0xee, 0xfe, 0x89, 0x04, 0x4b, 0x89, 0xca, 0x37, 0x6a, - 0x01, 0x3c, 0xb3, 0x06, 0xfc, 0x49, 0xa0, 0x7d, 0x01, 0x35, 0xa0, 0xea, 0x3f, 0x10, 0xb0, 0x0d, - 0xf6, 0x6c, 0x0a, 0xdd, 0x2e, 0xa0, 0x36, 0x34, 0xd8, 0xc2, 0xf1, 0x60, 0x80, 0x5d, 0xb7, 0x5d, - 0x14, 0x33, 0x4f, 0x54, 0xdd, 0x18, 0x3b, 0xb8, 0x5d, 0x42, 0x4d, 0xa8, 0xed, 0xd9, 0xbc, 0x9b, - 0xb8, 0x5d, 0x46, 0x08, 0x5a, 0x7e, 0x6b, 0x31, 0x5f, 0x54, 0x09, 0xcd, 0xf9, 0xcb, 0x16, 0xee, - 0x3e, 0x0f, 0xd7, 0x2f, 0xe9, 0x7d, 0x56, 0xe0, 0xe2, 0x33, 0x4b, 0xc3, 0x07, 0xba, 0x85, 0xb5, - 0xe0, 0xa7, 0xf6, 0x05, 0x74, 0x11, 0x16, 0xb7, 0xb0, 0x33, 0xc4, 0xa1, 0xc9, 0x02, 0x5a, 0x82, - 0xe6, 0x96, 0xfe, 0x32, 0x34, 0x55, 0x94, 0x4b, 0x55, 0xa9, 0x2d, 0xad, 0xfe, 0xe9, 0x55, 0xa8, - 0xad, 0xab, 0x9e, 0xfa, 0xd8, 0xb6, 0x1d, 0x0d, 0x19, 0x80, 0x68, 0xf3, 0xbd, 0x39, 0xb2, 0x2d, - 0xf1, 0x49, 0x0b, 0xba, 0x1f, 0x65, 0x3b, 0x1f, 0x24, 0x01, 0xb9, 0xd0, 0x74, 0xdf, 0x4a, 0x85, - 0x8f, 0x01, 0xcb, 0x17, 0x90, 0x49, 0xb1, 0xed, 0xe9, 0x26, 0xde, 0xd3, 0x07, 0x47, 0x7e, 0x7b, - 0xe1, 0x07, 0x19, 0x79, 0x4e, 0x12, 0xd4, 0xc7, 0x77, 0x2b, 0x15, 0x1f, 0xfb, 0x3a, 0xc2, 0x37, - 0x93, 0xf2, 0x05, 0xf4, 0x02, 0x2e, 0x6d, 0xe0, 0x90, 0xd3, 0xf6, 0x11, 0xae, 0x66, 0x23, 0x4c, - 0x00, 0x9f, 0x11, 0xe5, 0x26, 0x94, 0xa9, 0x7c, 0xa1, 0x34, 0xbf, 0x1e, 0xfe, 0xfa, 0xb4, 0x7b, - 0x33, 0x1b, 0x40, 0xec, 0xf6, 0x7d, 0x58, 0x8c, 0x7d, 0xb3, 0x86, 0xd2, 0xcc, 0x52, 0xfa, 0xd7, - 0x87, 0xdd, 0xbb, 0x79, 0x40, 0x05, 0xae, 0x21, 0xb4, 0xa2, 0x4d, 0xfb, 0x28, 0xad, 0x2a, 0x97, - 0xfa, 0xb9, 0x51, 0xf7, 0xdd, 0x1c, 0x90, 0x02, 0x91, 0x09, 0xed, 0xf8, 0x37, 0x54, 0xe8, 0xee, - 0xc4, 0x0d, 0xa2, 0xc2, 0xf6, 0x5e, 0x2e, 0x58, 0x81, 0xee, 0x94, 0x0a, 0x41, 0xe2, 0xb3, 0x9c, - 0xb8, 0x8c, 0xfb, 0xdb, 0x64, 0x7d, 0x2f, 0xd4, 0x7d, 0x90, 0x1b, 0x5e, 0xa0, 0xfe, 0x4d, 0xd6, - 0x0c, 0x90, 0xf6, 0x69, 0x0b, 0xfa, 0x30, 0x7d, 0xbb, 0x09, 0xdf, 0xe4, 0x74, 0x57, 0xcf, 0xb2, - 0x44, 0x1c, 0xe2, 0x87, 0xf4, 0x15, 0x3f, 0xe5, 0xe3, 0x90, 0xb8, 0xde, 0xf9, 0xfb, 0x65, 0x7f, - 0xf7, 0xd2, 0xfd, 0xf0, 0x0c, 0x2b, 0xc4, 0x01, 0xec, 0xf8, 0x47, 0x6a, 0xbe, 0x1a, 0x3e, 0x98, - 0x2a, 0x35, 0xb3, 0xe9, 0xe0, 0xf7, 0x60, 0x31, 0xe6, 0xa8, 0x51, 0x7e, 0x67, 0xde, 0x9d, 0x14, - 0x4d, 0x31, 0x95, 0x8c, 0x35, 0x45, 0xa0, 0x0c, 0xe9, 0x4f, 0x69, 0x9c, 0xe8, 0xde, 0xcd, 0x03, - 0x2a, 0x2e, 0xe2, 0x52, 0x73, 0x19, 0x7b, 0x16, 0x47, 0xf7, 0xd2, 0xf7, 0x48, 0x7f, 0xd2, 0xef, - 0xbe, 0x9f, 0x13, 0x5a, 0x20, 0x3d, 0x86, 0x8b, 0x29, 0x1d, 0x09, 0xe8, 0xfd, 0x89, 0xcc, 0x8a, - 0xb7, 0x62, 0x74, 0xef, 0xe7, 0x05, 0x17, 0x78, 0x7f, 0x03, 0xd0, 0xee, 0xa1, 0x7d, 0xf2, 0xd8, - 0xb6, 0x0e, 0xf4, 0xe1, 0xd8, 0x51, 0x59, 0x95, 0x3b, 0xcb, 0x37, 0x24, 0x41, 0x33, 0x64, 0x74, - 0xe2, 0x0a, 0x81, 0xbc, 0x0f, 0xb0, 0x81, 0xbd, 0x2d, 0xec, 0x39, 0x44, 0x31, 0xde, 0xce, 0x72, - 0x7f, 0x1c, 0xc0, 0x47, 0xf5, 0xce, 0x54, 0xb8, 0x90, 0x2b, 0x6a, 0x6f, 0xa9, 0xd6, 0x58, 0x35, - 0x42, 0x9d, 0xd3, 0xf7, 0x52, 0x97, 0xc7, 0xc1, 0x32, 0x18, 0x99, 0x09, 0x2d, 0x50, 0x9e, 0x08, - 0xd7, 0x1e, 0x7a, 0x46, 0x99, 0xec, 0xda, 0x93, 0x2f, 0xf1, 0x71, 0xb3, 0x37, 0x01, 0x5e, 0x20, - 0xfe, 0x52, 0xa2, 0x4d, 0x2d, 0x31, 0x80, 0xe7, 0xba, 0x77, 0xb8, 0x63, 0xa8, 0x96, 0x9b, 0xe7, - 0x08, 0x14, 0xf0, 0x0c, 0x47, 0xe0, 0xf0, 0xe2, 0x08, 0x1a, 0x34, 0x23, 0xaf, 0x1b, 0x28, 0xad, - 0x07, 0x39, 0xed, 0xa5, 0xa7, 0x7b, 0x67, 0x3a, 0xa0, 0xc0, 0x72, 0x08, 0x4d, 0x5f, 0x95, 0x18, - 0x71, 0xdf, 0xcd, 0x3a, 0x69, 0x00, 0x93, 0x61, 0x09, 0xd2, 0x41, 0xc3, 0x96, 0x20, 0x59, 0xbc, - 0x45, 0xf9, 0x8a, 0xfe, 0x93, 0x2c, 0x41, 0x76, 0x45, 0x98, 0x99, 0xba, 0xd8, 0x43, 0x49, 0xba, - 0x1d, 0x4d, 0x7d, 0xf7, 0x49, 0x35, 0x75, 0x19, 0xef, 0x2e, 0xf2, 0x05, 0xf4, 0x1c, 0x2a, 0xfc, - 0x5f, 0x2e, 0xbc, 0x35, 0xb9, 0x88, 0xc3, 0x77, 0xbf, 0x3d, 0x05, 0x4a, 0x6c, 0x7c, 0x04, 0x2b, - 0x19, 0x25, 0x9c, 0x54, 0x17, 0x3c, 0xb9, 0xdc, 0x33, 0xcd, 0x39, 0xa8, 0x80, 0x92, 0xdf, 0x35, - 0xa6, 0xb2, 0x29, 0xf3, 0xf3, 0xc7, 0x1c, 0x28, 0x92, 0x9f, 0x26, 0xa6, 0xa2, 0xc8, 0xfc, 0x82, - 0x71, 0x1a, 0x8a, 0x3e, 0x2c, 0x25, 0x0a, 0x01, 0xe8, 0xbd, 0x0c, 0x0f, 0x9a, 0x56, 0x2e, 0x98, - 0x86, 0x60, 0x08, 0x6f, 0xa4, 0x26, 0xbd, 0xa9, 0x11, 0xc1, 0xa4, 0xf4, 0x78, 0x1a, 0xa2, 0x01, - 0x5c, 0x4c, 0x49, 0x75, 0x53, 0x7d, 0x59, 0x76, 0x4a, 0x3c, 0x0d, 0xc9, 0x21, 0x74, 0xd7, 0x1c, - 0x5b, 0xd5, 0x06, 0xaa, 0xeb, 0x3d, 0x32, 0x3c, 0xec, 0x90, 0xf4, 0xcc, 0x0f, 0xc9, 0xe2, 0x74, - 0xe3, 0x03, 0x0a, 0x17, 0x40, 0xe5, 0xc4, 0xb4, 0x0f, 0x75, 0xfa, 0xb1, 0x00, 0xfb, 0xce, 0x1e, - 0xa5, 0xbb, 0x9f, 0x10, 0x44, 0x86, 0x4d, 0x4b, 0x03, 0xf4, 0xf5, 0x65, 0xf5, 0xab, 0x1a, 0x54, - 0xfd, 0xde, 0xec, 0xaf, 0x39, 0x3b, 0x7c, 0x0d, 0xe9, 0xda, 0xf7, 0x60, 0x31, 0xf6, 0xd9, 0x64, - 0xaa, 0x89, 0x4b, 0xff, 0xb4, 0x72, 0x1a, 0xbb, 0x9e, 0xf3, 0x7f, 0xea, 0x23, 0x22, 0xb7, 0x77, - 0xb2, 0x52, 0xbe, 0x78, 0xd0, 0x36, 0x65, 0xe3, 0xff, 0xdf, 0xa1, 0xd2, 0x36, 0x40, 0x28, 0x48, - 0x9a, 0xdc, 0xed, 0x44, 0xfc, 0xfe, 0x34, 0x6a, 0x99, 0xa9, 0x71, 0xd0, 0xbb, 0x79, 0x5a, 0x4e, - 0xb2, 0x3d, 0x59, 0x76, 0xf4, 0xf3, 0x0c, 0x1a, 0xe1, 0x3e, 0x44, 0x94, 0xfa, 0x2f, 0x64, 0x92, - 0x8d, 0x8a, 0xd3, 0x6e, 0xb1, 0x75, 0x46, 0x07, 0x39, 0x65, 0x3b, 0x97, 0xb8, 0x91, 0x78, 0xfd, - 0x3f, 0xc3, 0x8d, 0x64, 0xbc, 0x3a, 0xa4, 0x06, 0x14, 0xd9, 0x8f, 0x0a, 0x2c, 0xf3, 0x8f, 0x17, - 0xb5, 0x53, 0x33, 0xff, 0x8c, 0x67, 0x82, 0xd4, 0xcc, 0x3f, 0xab, 0x4a, 0x2e, 0x5f, 0x58, 0xfb, - 0xe8, 0xbb, 0x1f, 0x0e, 0x75, 0xef, 0x70, 0xbc, 0x4f, 0x6e, 0xff, 0x80, 0x2d, 0x7d, 0x5f, 0xb7, - 0xf9, 0x5f, 0x0f, 0x7c, 0x71, 0x7f, 0x40, 0x77, 0x7b, 0x40, 0x76, 0x1b, 0xed, 0xef, 0x57, 0xe8, - 0xe8, 0xa3, 0xff, 0x0d, 0x00, 0x00, 0xff, 0xff, 0x71, 0x05, 0x6a, 0x7e, 0x96, 0x4c, 0x00, 0x00, + 0xe7, 0x71, 0xcc, 0x74, 0xc5, 0x16, 0x31, 0xc3, 0x3f, 0x2b, 0x40, 0x27, 0x44, 0xa5, 0xaf, 0x3b, + 0x66, 0xc9, 0xc8, 0xb4, 0x8a, 0x2f, 0x29, 0xd3, 0x2a, 0xcd, 0x1f, 0xa7, 0x94, 0xd3, 0xe2, 0x94, + 0x7f, 0x2f, 0x40, 0x2b, 0xa0, 0xda, 0xae, 0xa1, 0x5a, 0x99, 0x92, 0xb0, 0x27, 0x62, 0xf4, 0x28, + 0x9d, 0xde, 0x4b, 0xd3, 0x93, 0x0c, 0x46, 0x28, 0xb1, 0x2d, 0xd0, 0x35, 0xca, 0x74, 0xc7, 0x63, + 0x35, 0x2b, 0x9e, 0x17, 0x30, 0x85, 0xd4, 0x4d, 0x8c, 0xee, 0x01, 0xe2, 0x5a, 0xd4, 0xd7, 0xad, + 0xbe, 0x8b, 0x07, 0xb6, 0xa5, 0x31, 0xfd, 0x2a, 0x2b, 0x6d, 0xfe, 0x4b, 0xcf, 0xda, 0x63, 0xf3, + 0xe8, 0x63, 0x28, 0x79, 0x67, 0x23, 0x16, 0x81, 0xb4, 0x52, 0x7d, 0x78, 0x70, 0xae, 0xfd, 0xb3, + 0x11, 0x56, 0x28, 0xb8, 0xdf, 0x8c, 0xe3, 0x39, 0xea, 0x09, 0x0f, 0xe7, 0x4a, 0x4a, 0x68, 0x86, + 0x58, 0x0c, 0x9f, 0x86, 0x0b, 0x2c, 0xec, 0xe1, 0x43, 0x26, 0xd9, 0xbe, 0xd2, 0xf6, 0x3d, 0xcf, + 0xa0, 0x55, 0x37, 0x2a, 0xd9, 0xfe, 0xec, 0xbe, 0x67, 0xc8, 0x7f, 0x5f, 0x80, 0x76, 0x80, 0x59, + 0xc1, 0xee, 0xd8, 0xc8, 0x56, 0xb8, 0xc9, 0xf5, 0x8e, 0x69, 0xba, 0xf6, 0x2d, 0xa8, 0x73, 0xb6, + 0x9f, 0x43, 0x6c, 0x80, 0x2d, 0xd9, 0x9a, 0x20, 0xc7, 0xe5, 0x97, 0x24, 0xc7, 0x95, 0x73, 0xca, + 0xb1, 0xfc, 0x53, 0x09, 0xde, 0x48, 0x18, 0xbf, 0x89, 0x04, 0x9c, 0x9c, 0x95, 0x71, 0xa3, 0x18, + 0xdf, 0x92, 0x9b, 0xf1, 0x87, 0x50, 0x71, 0xe8, 0xee, 0xfc, 0x4d, 0xe7, 0xd6, 0x44, 0x19, 0x62, + 0x07, 0x51, 0xf8, 0x12, 0xf9, 0x0f, 0x24, 0x58, 0x49, 0x1e, 0x75, 0x0e, 0xdf, 0xbc, 0x06, 0x0b, + 0x6c, 0x6b, 0x5f, 0xd5, 0xee, 0x4c, 0x56, 0xb5, 0x80, 0x38, 0x8a, 0xbf, 0x50, 0xde, 0x83, 0x65, + 0xdf, 0x85, 0x07, 0x04, 0xde, 0xc6, 0x9e, 0x3a, 0x21, 0x27, 0xb9, 0x01, 0x75, 0x16, 0xdc, 0xb2, + 0x58, 0x9f, 0x65, 0xf3, 0x70, 0x20, 0x8a, 0x60, 0xf2, 0x7f, 0x49, 0x70, 0x89, 0xfa, 0xc0, 0xf8, + 0x23, 0x4a, 0x9e, 0x07, 0x36, 0x59, 0x14, 0x0b, 0x76, 0x54, 0x93, 0x37, 0x74, 0xd4, 0x94, 0xc8, + 0x1c, 0xea, 0x25, 0x6b, 0x64, 0xa9, 0xb9, 0x6b, 0xf0, 0x22, 0x4b, 0xf2, 0x64, 0xfa, 0x20, 0x1b, + 0x2f, 0x8e, 0x05, 0xbe, 0xb7, 0x34, 0x8b, 0xef, 0xdd, 0x82, 0x37, 0x62, 0x37, 0x9d, 0x83, 0xa3, + 0xf2, 0x5f, 0x4a, 0x84, 0x1d, 0x91, 0xc6, 0x98, 0xd9, 0x63, 0xca, 0x6b, 0xe2, 0xf5, 0xa6, 0xaf, + 0x6b, 0x71, 0x53, 0xa1, 0xa1, 0xcf, 0xa0, 0x66, 0xe1, 0xd3, 0x7e, 0x38, 0xa4, 0xc9, 0x11, 0x70, + 0x57, 0x2d, 0x7c, 0x4a, 0xff, 0x92, 0x77, 0x60, 0x25, 0x71, 0xd4, 0x79, 0xee, 0xfe, 0x0f, 0x12, + 0x5c, 0xde, 0x70, 0xec, 0xd1, 0x17, 0xba, 0xe3, 0x8d, 0x55, 0x23, 0xfa, 0xd6, 0xfd, 0x6a, 0x8a, + 0x4e, 0x4f, 0x42, 0xc1, 0x2d, 0x93, 0x9f, 0x7b, 0x29, 0x1a, 0x94, 0x3c, 0x14, 0xbf, 0x74, 0x28, + 0x14, 0xfe, 0xcf, 0x62, 0xda, 0xe1, 0x39, 0xdc, 0x94, 0xf0, 0x22, 0x4f, 0xec, 0x9f, 0x5a, 0xa3, + 0x2e, 0xce, 0x5a, 0xa3, 0xce, 0x30, 0xe2, 0xa5, 0x97, 0x64, 0xc4, 0xcf, 0x5d, 0x34, 0x79, 0x02, + 0xd1, 0xf7, 0x03, 0xea, 0x64, 0x67, 0x7a, 0x78, 0x58, 0x03, 0x08, 0x6a, 0xe9, 0xbc, 0xaf, 0x31, + 0xcf, 0x36, 0xa1, 0x55, 0x84, 0x5b, 0xc2, 0x61, 0x72, 0x87, 0x1d, 0xaa, 0xee, 0x7e, 0x07, 0xba, + 0x69, 0x52, 0x3a, 0x8f, 0xe4, 0xff, 0xac, 0x00, 0xd0, 0x13, 0xad, 0xb0, 0xb3, 0xf9, 0x82, 0x5b, + 0x10, 0x0a, 0x2a, 0x02, 0x7d, 0x0f, 0x4b, 0x91, 0x46, 0x54, 0x42, 0xa4, 0x8b, 0x04, 0x26, 0x91, + 0x42, 0x6a, 0x74, 0x9f, 0x90, 0xd6, 0x30, 0xa1, 0x88, 0x9b, 0xdf, 0x2b, 0x50, 0x73, 0xec, 0xd3, + 0x3e, 0x51, 0x33, 0xcd, 0xef, 0xf5, 0x75, 0xec, 0x53, 0xa2, 0x7c, 0x1a, 0x5a, 0x81, 0x05, 0x4f, + 0x75, 0x8f, 0xc9, 0xfe, 0x95, 0x50, 0xbb, 0x85, 0x86, 0x2e, 0x41, 0xf9, 0x50, 0x37, 0x30, 0x7b, + 0xdd, 0xaf, 0x29, 0x6c, 0x80, 0xbe, 0xe1, 0x37, 0xa5, 0x55, 0x73, 0xb7, 0xd4, 0xb0, 0xbe, 0xb4, + 0xaf, 0x24, 0x58, 0x0c, 0xa8, 0x46, 0x0d, 0x10, 0xb1, 0x69, 0xd4, 0x9e, 0xad, 0xdb, 0x1a, 0x33, + 0x15, 0xad, 0x0c, 0x8f, 0xc0, 0x16, 0x32, 0xab, 0x15, 0x2c, 0x99, 0x94, 0xed, 0x92, 0x7b, 0x91, + 0x4b, 0xeb, 0x9a, 0xdf, 0x62, 0x52, 0x71, 0xec, 0xd3, 0x9e, 0x26, 0xa8, 0xc1, 0x1a, 0x79, 0x59, + 0x6e, 0x47, 0xa8, 0xb1, 0x4e, 0x7b, 0x79, 0x6f, 0x41, 0x13, 0x3b, 0x8e, 0xed, 0xf4, 0x4d, 0xec, + 0xba, 0xea, 0x10, 0xf3, 0x30, 0xbb, 0x41, 0x27, 0xb7, 0xd9, 0x9c, 0xfc, 0x4f, 0x45, 0x68, 0x05, + 0x57, 0xf1, 0x1f, 0xaa, 0x75, 0xcd, 0x7f, 0xa8, 0xd6, 0x09, 0xeb, 0xc0, 0x61, 0xa6, 0x50, 0x30, + 0x77, 0xad, 0xd0, 0x91, 0x94, 0x1a, 0x9f, 0xed, 0x69, 0xc4, 0x2d, 0x13, 0x25, 0xb3, 0x6c, 0x0d, + 0x07, 0xcc, 0x05, 0x7f, 0x8a, 0xf3, 0x36, 0x22, 0x23, 0xa5, 0x1c, 0x32, 0x52, 0xce, 0x21, 0x23, + 0x95, 0x14, 0x19, 0x59, 0x86, 0xca, 0xc1, 0x78, 0x70, 0x8c, 0x3d, 0x1e, 0x14, 0xf3, 0x51, 0x54, + 0x76, 0xaa, 0x31, 0xd9, 0x11, 0x22, 0x52, 0x0b, 0x8b, 0xc8, 0x15, 0xa8, 0xb1, 0x17, 0xd3, 0xbe, + 0xe7, 0xd2, 0x77, 0xa1, 0xa2, 0x52, 0x65, 0x13, 0xfb, 0x2e, 0xfa, 0xc4, 0x0f, 0xe7, 0xea, 0x69, + 0xca, 0x4e, 0xad, 0x4e, 0x4c, 0x4a, 0xfc, 0x60, 0xee, 0x1d, 0x58, 0x0c, 0x91, 0x83, 0xfa, 0x88, + 0x06, 0x3d, 0x6a, 0x28, 0x68, 0xa7, 0x6e, 0xe2, 0x36, 0xb4, 0x02, 0x92, 0x50, 0xb8, 0x26, 0xcb, + 0x95, 0xc4, 0x2c, 0x01, 0x93, 0xbf, 0x0f, 0x28, 0xc0, 0x34, 0x5f, 0x64, 0x17, 0x63, 0x65, 0x21, + 0xce, 0x4a, 0xf9, 0xaf, 0x24, 0x58, 0x0a, 0x23, 0x9b, 0xd5, 0x49, 0x7e, 0x06, 0x75, 0xf6, 0x8a, + 0xd6, 0x27, 0x4a, 0xca, 0xeb, 0x2e, 0xd7, 0x26, 0xd2, 0x50, 0x81, 0xa0, 0x6d, 0x9f, 0x88, 0xc2, + 0xa9, 0xed, 0x1c, 0xeb, 0xd6, 0xb0, 0x4f, 0x4e, 0xe6, 0xab, 0x46, 0x83, 0x4f, 0xee, 0x90, 0x39, + 0xf9, 0xc7, 0x12, 0x5c, 0x7f, 0x3a, 0xd2, 0x54, 0x0f, 0x87, 0xa2, 0x85, 0x79, 0x3b, 0x01, 0x3f, + 0xf6, 0x5b, 0xf1, 0x0a, 0xf9, 0x5e, 0x82, 0x18, 0xb4, 0xbc, 0x0d, 0x97, 0x15, 0xec, 0x62, 0x4b, + 0x8b, 0xfc, 0x38, 0x73, 0xb9, 0x64, 0x04, 0xdd, 0xb4, 0xed, 0xe6, 0xe1, 0x3d, 0x0b, 0xdb, 0xfa, + 0x0e, 0xd9, 0xd6, 0xe3, 0x56, 0x88, 0x44, 0x0b, 0x14, 0x8f, 0x27, 0xff, 0x75, 0x01, 0x56, 0x1e, + 0x69, 0x1a, 0x37, 0x60, 0x3c, 0x10, 0x79, 0x55, 0x31, 0x62, 0x3c, 0x86, 0x2a, 0x26, 0x63, 0xa8, + 0x97, 0x65, 0x54, 0xb8, 0x79, 0xb5, 0xc6, 0xa6, 0xef, 0x36, 0x1c, 0xd6, 0xbc, 0xf2, 0x90, 0xbf, + 0xe6, 0x90, 0x8c, 0x95, 0xba, 0x8e, 0xe9, 0xa1, 0x45, 0xd5, 0x2f, 0xfb, 0xc8, 0x23, 0xe8, 0x24, + 0x89, 0x35, 0xa7, 0x66, 0xfa, 0x14, 0x19, 0xd9, 0xac, 0x44, 0xd8, 0x20, 0xd1, 0x03, 0x9d, 0xda, + 0xb5, 0x5d, 0xf9, 0xbf, 0x0b, 0xd0, 0xd9, 0x53, 0x4f, 0xf0, 0x2f, 0x0e, 0x83, 0xbe, 0x0b, 0x97, + 0x5c, 0xf5, 0x04, 0xf7, 0x43, 0x39, 0x61, 0xdf, 0xc1, 0xcf, 0x79, 0xf4, 0xf5, 0x6e, 0x9a, 0x62, + 0xa6, 0x36, 0x7f, 0x28, 0x4b, 0x6e, 0x64, 0x5e, 0xc1, 0xcf, 0xd1, 0xdb, 0xb0, 0x18, 0x6e, 0x22, + 0x22, 0x47, 0xab, 0x52, 0x92, 0x37, 0x43, 0x3d, 0x42, 0x3d, 0x4d, 0x7e, 0x0e, 0x57, 0x9f, 0x5a, + 0x2e, 0xf6, 0x7a, 0x41, 0x9f, 0xcb, 0x9c, 0xd9, 0xd3, 0x0d, 0xa8, 0x07, 0x84, 0x4f, 0x74, 0xf2, + 0x6b, 0xae, 0x6c, 0x43, 0x77, 0x3b, 0x68, 0x73, 0x73, 0x37, 0x58, 0xa3, 0xc2, 0x2b, 0x44, 0x78, + 0x28, 0xfa, 0x76, 0x14, 0x7c, 0x88, 0x1d, 0x6c, 0x0d, 0xf0, 0x96, 0x3d, 0x38, 0x0e, 0xb5, 0xb7, + 0x4a, 0xe1, 0xf6, 0xd6, 0x59, 0xdb, 0x65, 0xef, 0x7e, 0x26, 0x5a, 0xe6, 0xf6, 0xcf, 0x46, 0x18, + 0x2d, 0x40, 0x71, 0x07, 0x9f, 0xb6, 0x2f, 0x20, 0x80, 0xca, 0x8e, 0xed, 0x98, 0xaa, 0xd1, 0x96, + 0x50, 0x1d, 0x16, 0xf8, 0x33, 0x43, 0xbb, 0x80, 0x9a, 0x50, 0x5b, 0xf7, 0x4b, 0xb5, 0xed, 0xe2, + 0xdd, 0x3f, 0x91, 0x60, 0x29, 0x51, 0x08, 0x47, 0x2d, 0x80, 0xa7, 0xd6, 0x80, 0xbf, 0x10, 0xb4, + 0x2f, 0xa0, 0x06, 0x54, 0xfd, 0xf7, 0x02, 0xb6, 0xdf, 0xbe, 0x4d, 0xa1, 0xdb, 0x05, 0xd4, 0x86, + 0x06, 0x5b, 0x38, 0x1e, 0x0c, 0xb0, 0xeb, 0xb6, 0x8b, 0x62, 0xe6, 0xb1, 0xaa, 0x1b, 0x63, 0x07, + 0xb7, 0x4b, 0x04, 0xe7, 0xbe, 0xcd, 0x9b, 0x8b, 0xdb, 0x65, 0x84, 0xa0, 0xe5, 0x77, 0x1a, 0xf3, + 0x45, 0x95, 0xd0, 0x9c, 0xbf, 0x6c, 0xe1, 0xee, 0xb3, 0x70, 0x39, 0x93, 0x5e, 0x6f, 0x05, 0x2e, + 0x3e, 0xb5, 0x34, 0x7c, 0xa8, 0x5b, 0x58, 0x0b, 0x7e, 0x6a, 0x5f, 0x40, 0x17, 0x61, 0x71, 0x1b, + 0x3b, 0x43, 0x1c, 0x9a, 0x2c, 0xa0, 0x25, 0x68, 0x6e, 0xeb, 0x2f, 0x42, 0x53, 0x45, 0xb9, 0x54, + 0x95, 0xda, 0xd2, 0xea, 0x9f, 0x5e, 0x85, 0xda, 0x86, 0xea, 0xa9, 0xeb, 0xb6, 0xed, 0x68, 0xc8, + 0x00, 0x44, 0x7b, 0xf1, 0xcd, 0x91, 0x6d, 0x89, 0x2f, 0x5c, 0xd0, 0xfd, 0xa8, 0x14, 0xf0, 0x41, + 0x12, 0x90, 0xcb, 0x50, 0xf7, 0xad, 0x54, 0xf8, 0x18, 0xb0, 0x7c, 0x01, 0x99, 0x14, 0xdb, 0xbe, + 0x6e, 0xe2, 0x7d, 0x7d, 0x70, 0xec, 0x77, 0x1b, 0x7e, 0x90, 0x91, 0xf6, 0x24, 0x41, 0x7d, 0x7c, + 0xb7, 0x52, 0xf1, 0xb1, 0x8f, 0x25, 0x7c, 0xab, 0x29, 0x5f, 0x40, 0xcf, 0xe1, 0xd2, 0x26, 0x0e, + 0xf9, 0x70, 0x1f, 0xe1, 0x6a, 0x36, 0xc2, 0x04, 0xf0, 0x39, 0x51, 0x6e, 0x41, 0x99, 0x8a, 0x1b, + 0x4a, 0x73, 0xf3, 0xe1, 0x8f, 0x51, 0xbb, 0x37, 0xb3, 0x01, 0xc4, 0x6e, 0xdf, 0x87, 0xc5, 0xd8, + 0x27, 0x6c, 0x28, 0xcd, 0x4a, 0xa5, 0x7f, 0x8c, 0xd8, 0xbd, 0x9b, 0x07, 0x54, 0xe0, 0x1a, 0x42, + 0x2b, 0xda, 0xc3, 0x8f, 0xd2, 0x8a, 0x74, 0xa9, 0x5f, 0x1f, 0x75, 0xdf, 0xcd, 0x01, 0x29, 0x10, + 0x99, 0xd0, 0x8e, 0x7f, 0x52, 0x85, 0xee, 0x4e, 0xdc, 0x20, 0x2a, 0x6c, 0xef, 0xe5, 0x82, 0x15, + 0xe8, 0xce, 0xa8, 0x10, 0x24, 0xbe, 0xd2, 0x89, 0xcb, 0xb8, 0xbf, 0x4d, 0xd6, 0xe7, 0x43, 0xdd, + 0x07, 0xb9, 0xe1, 0x05, 0xea, 0xdf, 0x64, 0xbd, 0x01, 0x69, 0x5f, 0xba, 0xa0, 0x0f, 0xd3, 0xb7, + 0x9b, 0xf0, 0x89, 0x4e, 0x77, 0xf5, 0x3c, 0x4b, 0xc4, 0x21, 0x7e, 0x48, 0x1f, 0xf5, 0x53, 0xbe, + 0x15, 0x89, 0xeb, 0x9d, 0xbf, 0x5f, 0xf6, 0x67, 0x30, 0xdd, 0x0f, 0xcf, 0xb1, 0x42, 0x1c, 0xc0, + 0x8e, 0x7f, 0xb3, 0xe6, 0xab, 0xe1, 0x83, 0xa9, 0x52, 0x33, 0x9b, 0x0e, 0x7e, 0x0f, 0x16, 0x63, + 0x7e, 0x1b, 0xe5, 0xf7, 0xed, 0xdd, 0x49, 0xc1, 0x15, 0x53, 0xc9, 0x58, 0x8f, 0x04, 0xca, 0x90, + 0xfe, 0x94, 0x3e, 0x8a, 0xee, 0xdd, 0x3c, 0xa0, 0xe2, 0x22, 0x2e, 0x35, 0x97, 0xb1, 0x57, 0x72, + 0x74, 0x2f, 0x7d, 0x8f, 0xf4, 0x17, 0xfe, 0xee, 0xfb, 0x39, 0xa1, 0x05, 0xd2, 0x13, 0xb8, 0x98, + 0xd2, 0xa0, 0x80, 0xde, 0x9f, 0xc8, 0xac, 0x78, 0x67, 0x46, 0xf7, 0x7e, 0x5e, 0x70, 0x81, 0xf7, + 0x37, 0x00, 0xed, 0x1d, 0xd9, 0xa7, 0xeb, 0xb6, 0x75, 0xa8, 0x0f, 0xc7, 0x8e, 0xca, 0x8a, 0xde, + 0x59, 0xbe, 0x21, 0x09, 0x9a, 0x21, 0xa3, 0x13, 0x57, 0x08, 0xe4, 0x7d, 0x80, 0x4d, 0xec, 0x6d, + 0x63, 0xcf, 0x21, 0x8a, 0xf1, 0x76, 0x96, 0xfb, 0xe3, 0x00, 0x3e, 0xaa, 0x77, 0xa6, 0xc2, 0x85, + 0x5c, 0x51, 0x7b, 0x5b, 0xb5, 0xc6, 0xaa, 0x11, 0x6a, 0xa4, 0xbe, 0x97, 0xba, 0x3c, 0x0e, 0x96, + 0xc1, 0xc8, 0x4c, 0x68, 0x81, 0xf2, 0x54, 0xb8, 0xf6, 0xd0, 0xab, 0xca, 0x64, 0xd7, 0x9e, 0x7c, + 0x98, 0x8f, 0x9b, 0xbd, 0x09, 0xf0, 0x02, 0xf1, 0x97, 0x12, 0xed, 0x71, 0x89, 0x01, 0x3c, 0xd3, + 0xbd, 0xa3, 0x5d, 0x43, 0xb5, 0xdc, 0x3c, 0x47, 0xa0, 0x80, 0xe7, 0x38, 0x02, 0x87, 0x17, 0x47, + 0xd0, 0xa0, 0x19, 0x79, 0xec, 0x40, 0x69, 0x2d, 0xc9, 0x69, 0x0f, 0x3f, 0xdd, 0x3b, 0xd3, 0x01, + 0x05, 0x96, 0x23, 0x68, 0xfa, 0xaa, 0xc4, 0x88, 0xfb, 0x6e, 0xd6, 0x49, 0x03, 0x98, 0x0c, 0x4b, + 0x90, 0x0e, 0x1a, 0xb6, 0x04, 0xc9, 0x5a, 0x2e, 0xca, 0xf7, 0x06, 0x30, 0xc9, 0x12, 0x64, 0x17, + 0x88, 0x99, 0xa9, 0x8b, 0xbd, 0x9b, 0xa4, 0xdb, 0xd1, 0xd4, 0x67, 0xa0, 0x54, 0x53, 0x97, 0xf1, + 0x0c, 0x23, 0x5f, 0x40, 0xcf, 0xa0, 0xc2, 0xff, 0x03, 0xc3, 0x5b, 0x93, 0x6b, 0x3a, 0x7c, 0xf7, + 0xdb, 0x53, 0xa0, 0xc4, 0xc6, 0xc7, 0xb0, 0x92, 0x51, 0xd1, 0x49, 0x75, 0xc1, 0x93, 0xab, 0x3f, + 0xd3, 0x9c, 0x83, 0x0a, 0x28, 0xf9, 0x99, 0x63, 0x2a, 0x9b, 0x32, 0xbf, 0x86, 0xcc, 0x81, 0x22, + 0xf9, 0xa5, 0x62, 0x2a, 0x8a, 0xcc, 0x0f, 0x1a, 0xa7, 0xa1, 0xe8, 0xc3, 0x52, 0xa2, 0x2e, 0x80, + 0xde, 0xcb, 0xf0, 0xa0, 0x69, 0xd5, 0x83, 0x69, 0x08, 0x86, 0xf0, 0x46, 0x6a, 0x0e, 0x9c, 0x1a, + 0x11, 0x4c, 0xca, 0x96, 0xa7, 0x21, 0x1a, 0xc0, 0xc5, 0x94, 0xcc, 0x37, 0xd5, 0x97, 0x65, 0x67, + 0xc8, 0xd3, 0x90, 0x1c, 0x41, 0x77, 0xcd, 0xb1, 0x55, 0x6d, 0xa0, 0xba, 0xde, 0x23, 0xc3, 0xc3, + 0x0e, 0x49, 0xcf, 0xfc, 0x90, 0x2c, 0x4e, 0x37, 0x3e, 0xa0, 0x70, 0x01, 0x54, 0x4e, 0x4c, 0x07, + 0x50, 0xa7, 0xdf, 0x0e, 0xb0, 0xcf, 0xee, 0x51, 0xba, 0xfb, 0x09, 0x41, 0x64, 0xd8, 0xb4, 0x34, + 0x40, 0x5f, 0x5f, 0x56, 0xbf, 0xaa, 0x41, 0xd5, 0x6f, 0xd5, 0xfe, 0x9a, 0xb3, 0xc3, 0xd7, 0x90, + 0xae, 0x7d, 0x0f, 0x16, 0x63, 0x5f, 0x51, 0xa6, 0x9a, 0xb8, 0xf4, 0x2f, 0x2d, 0xa7, 0xb1, 0xeb, + 0x19, 0xff, 0x1f, 0x3f, 0x22, 0x72, 0x7b, 0x27, 0x2b, 0xe5, 0x8b, 0x07, 0x6d, 0x53, 0x36, 0xfe, + 0xff, 0x1d, 0x2a, 0xed, 0x00, 0x84, 0x82, 0xa4, 0xc9, 0xcd, 0x4f, 0xc4, 0xef, 0x4f, 0xa3, 0x96, + 0x99, 0x1a, 0x07, 0xbd, 0x9b, 0xa7, 0x03, 0x25, 0xdb, 0x93, 0x65, 0x47, 0x3f, 0x4f, 0xa1, 0x11, + 0x6e, 0x4b, 0x44, 0xa9, 0xff, 0x51, 0x26, 0xd9, 0xb7, 0x38, 0xed, 0x16, 0xdb, 0xe7, 0x74, 0x90, + 0x53, 0xb6, 0x73, 0x89, 0x1b, 0x89, 0x3f, 0x07, 0x64, 0xb8, 0x91, 0x8c, 0x47, 0x88, 0xd4, 0x80, + 0x22, 0xfb, 0x8d, 0x81, 0x65, 0xfe, 0xf1, 0x1a, 0x77, 0x6a, 0xe6, 0x9f, 0xf1, 0x6a, 0x90, 0x9a, + 0xf9, 0x67, 0x15, 0xcd, 0xe5, 0x0b, 0x6b, 0x1f, 0x7d, 0xf7, 0xc3, 0xa1, 0xee, 0x1d, 0x8d, 0x0f, + 0xc8, 0xed, 0x1f, 0xb0, 0xa5, 0xef, 0xeb, 0x36, 0xff, 0xeb, 0x81, 0x2f, 0xee, 0x0f, 0xe8, 0x6e, + 0x0f, 0xc8, 0x6e, 0xa3, 0x83, 0x83, 0x0a, 0x1d, 0x7d, 0xf4, 0xbf, 0x01, 0x00, 0x00, 0xff, 0xff, + 0x6a, 0x0f, 0xa9, 0xe2, 0xa5, 0x4c, 0x00, 0x00, } // Reference imports to suppress errors if they are not otherwise used.