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:
XuanYang-cn 2022-10-18 15:33:26 +08:00 committed by GitHub
parent f8cb260931
commit 2785f98de6
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
23 changed files with 2419 additions and 2993 deletions

View 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)
}

View 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))
}

View File

@ -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)),

View File

@ -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)

View File

@ -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{

View File

@ -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{},
})

View File

@ -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(),

View File

@ -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)

View File

@ -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,

View File

@ -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()

View File

@ -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,

View File

@ -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) {

View File

@ -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))

View File

@ -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{},
})

View File

@ -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())
}

View File

@ -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,
},

View File

@ -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
}

View 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)
}

View File

@ -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

View 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)
}

View File

@ -16,6 +16,7 @@ enum SegmentType {
New = 0;
Normal = 1;
Flushed = 2;
Compacted = 3;
}
service DataCoord {

View File

@ -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.