mirror of
https://gitee.com/milvus-io/milvus.git
synced 2026-01-07 19:31:51 +08:00
Refactor segment replica in DN (#19815)
- Replace isNew and isFlushed with sType - Add compacted into proto - Rename replica to channel - Rename segmentReplica to channelMeta - Replace 4 maps with 1 map - Add segment.go and segment_test.go Signed-off-by: yangxuan <xuan.yang@zilliz.com> Signed-off-by: yangxuan <xuan.yang@zilliz.com>
This commit is contained in:
parent
f8cb260931
commit
2785f98de6
637
internal/datanode/channel_meta.go
Normal file
637
internal/datanode/channel_meta.go
Normal file
@ -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)
|
||||
}
|
||||
959
internal/datanode/channel_meta_test.go
Normal file
959
internal/datanode/channel_meta_test.go
Normal file
@ -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))
|
||||
}
|
||||
@ -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)),
|
||||
|
||||
@ -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)
|
||||
|
||||
@ -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{
|
||||
|
||||
@ -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{},
|
||||
})
|
||||
|
||||
@ -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(),
|
||||
|
||||
@ -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)
|
||||
|
||||
@ -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,
|
||||
|
||||
@ -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()
|
||||
|
||||
@ -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,
|
||||
|
||||
@ -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) {
|
||||
|
||||
@ -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))
|
||||
|
||||
@ -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{},
|
||||
})
|
||||
|
||||
@ -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())
|
||||
}
|
||||
|
||||
@ -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,
|
||||
},
|
||||
|
||||
@ -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
|
||||
}
|
||||
|
||||
138
internal/datanode/segment.go
Normal file
138
internal/datanode/segment.go
Normal file
@ -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)
|
||||
}
|
||||
@ -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)
|
||||
}
|
||||
File diff suppressed because it is too large
Load Diff
103
internal/datanode/segment_test.go
Normal file
103
internal/datanode/segment_test.go
Normal file
@ -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)
|
||||
}
|
||||
@ -16,6 +16,7 @@ enum SegmentType {
|
||||
New = 0;
|
||||
Normal = 1;
|
||||
Flushed = 2;
|
||||
Compacted = 3;
|
||||
}
|
||||
|
||||
service DataCoord {
|
||||
|
||||
@ -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.
|
||||
|
||||
Loading…
x
Reference in New Issue
Block a user