Add segment statistics updates logic for datanode

Signed-off-by: XuanYang-cn <xuan.yang@zilliz.com>
pull/4973/head^2
XuanYang-cn 2021-01-21 09:55:25 +08:00 committed by yefu.chen
parent 7619d968fd
commit ec27788628
17 changed files with 690 additions and 382 deletions

View File

@ -34,7 +34,7 @@ dataNode:
segStatistics:
recvBufSize: 64
publishInterval: 1000 # milliseconds
updateInterval: 1000 # milliseconds
flush:
# max buffer size to flush

View File

@ -249,7 +249,7 @@ const char descriptor_table_protodef_common_2eproto[] PROTOBUF_SECTION_VARIABLE(
"PE\020\027\022\021\n\rOUT_OF_MEMORY\020\030\022\024\n\017DD_REQUEST_RA"
"CE\020\350\007*N\n\nIndexState\022\010\n\004NONE\020\000\022\014\n\010UNISSUE"
"D\020\001\022\016\n\nINPROGRESS\020\002\022\014\n\010FINISHED\020\003\022\n\n\006FAI"
"LED\020\004*\274\004\n\007MsgType\022\t\n\005kNone\020\000\022\025\n\021kCreateC"
"LED\020\004*\325\004\n\007MsgType\022\t\n\005kNone\020\000\022\025\n\021kCreateC"
"ollection\020d\022\023\n\017kDropCollection\020e\022\022\n\016kHas"
"Collection\020f\022\027\n\023kDescribeCollection\020g\022\024\n"
"\020kShowCollections\020h\022\022\n\016kGetSysConfigs\020i\022"
@ -263,9 +263,10 @@ const char descriptor_table_protodef_common_2eproto[] PROTOBUF_SECTION_VARIABLE(
"\027kGetPartitionStatistics\020\370\003\022\016\n\tkTimeTick"
"\020\260\t\022\024\n\017kQueryNodeStats\020\261\t\022\017\n\nkLoadIndex\020"
"\262\t\022\017\n\nkRequestID\020\263\t\022\020\n\013kRequestTSO\020\264\t\022\025\n"
"\020kAllocateSegment\020\265\tBBZ@github.com/zilli"
"ztech/milvus-distributed/internal/proto/"
"commonpbb\006proto3"
"\020kAllocateSegment\020\265\t\022\027\n\022kSegmentStatisti"
"cs\020\266\tBBZ@github.com/zilliztech/milvus-di"
"stributed/internal/proto/commonpbb\006proto"
"3"
;
static const ::PROTOBUF_NAMESPACE_ID::internal::DescriptorTable*const descriptor_table_common_2eproto_deps[1] = {
};
@ -281,7 +282,7 @@ static ::PROTOBUF_NAMESPACE_ID::internal::SCCInfoBase*const descriptor_table_com
static ::PROTOBUF_NAMESPACE_ID::internal::once_flag descriptor_table_common_2eproto_once;
static bool descriptor_table_common_2eproto_initialized = false;
const ::PROTOBUF_NAMESPACE_ID::internal::DescriptorTable descriptor_table_common_2eproto = {
&descriptor_table_common_2eproto_initialized, descriptor_table_protodef_common_2eproto, "common.proto", 1696,
&descriptor_table_common_2eproto_initialized, descriptor_table_protodef_common_2eproto, "common.proto", 1721,
&descriptor_table_common_2eproto_once, descriptor_table_common_2eproto_sccs, descriptor_table_common_2eproto_deps, 7, 0,
schemas, file_default_instances, TableStruct_common_2eproto::offsets,
file_level_metadata_common_2eproto, 7, file_level_enum_descriptors_common_2eproto, file_level_service_descriptors_common_2eproto,
@ -380,6 +381,7 @@ bool MsgType_IsValid(int value) {
case 1203:
case 1204:
case 1205:
case 1206:
return true;
default:
return false;

View File

@ -200,12 +200,13 @@ enum MsgType : int {
kRequestID = 1203,
kRequestTSO = 1204,
kAllocateSegment = 1205,
kSegmentStatistics = 1206,
MsgType_INT_MIN_SENTINEL_DO_NOT_USE_ = std::numeric_limits<::PROTOBUF_NAMESPACE_ID::int32>::min(),
MsgType_INT_MAX_SENTINEL_DO_NOT_USE_ = std::numeric_limits<::PROTOBUF_NAMESPACE_ID::int32>::max()
};
bool MsgType_IsValid(int value);
constexpr MsgType MsgType_MIN = kNone;
constexpr MsgType MsgType_MAX = kAllocateSegment;
constexpr MsgType MsgType_MAX = kSegmentStatistics;
constexpr int MsgType_ARRAYSIZE = MsgType_MAX + 1;
const ::PROTOBUF_NAMESPACE_ID::EnumDescriptor* MsgType_descriptor();

View File

@ -1,11 +1,11 @@
package datanode
import (
"fmt"
"strconv"
"log"
"sync"
"github.com/zilliztech/milvus-distributed/internal/errors"
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb2"
)
type collectionReplica interface {
@ -17,17 +17,100 @@ type collectionReplica interface {
getCollectionByID(collectionID UniqueID) (*Collection, error)
getCollectionByName(collectionName string) (*Collection, error)
hasCollection(collectionID UniqueID) bool
getSegmentStatistics() // GOOSE TODO
// segment
addSegment(segmentID UniqueID) error
removeSegment(segmentID UniqueID) error
hasSegment(segmentID UniqueID) bool
updateSegmentRowNums(segmentID UniqueID, numRows int64) error
getSegmentStatisticsUpdates(segmentID UniqueID) (*internalpb2.SegmentStatisticsUpdates, error)
}
type collectionReplicaImpl struct {
mu sync.RWMutex
collections []*Collection
}
type (
Segment struct {
segmentID UniqueID
numRows int64
memorySize int64
}
collectionReplicaImpl struct {
mu sync.RWMutex
collections []*Collection
segments []*Segment
}
)
//----------------------------------------------------------------------------------------------------- collection
func (colReplica *collectionReplicaImpl) getSegmentStatistics() {
// GOOSE TODO
func (colReplica *collectionReplicaImpl) addSegment(segmentID UniqueID) error {
colReplica.mu.RLock()
defer colReplica.mu.RUnlock()
log.Println("Add Segment", segmentID)
seg := &Segment{
segmentID: segmentID,
}
colReplica.segments = append(colReplica.segments, seg)
return nil
}
func (colReplica *collectionReplicaImpl) removeSegment(segmentID UniqueID) error {
colReplica.mu.RLock()
defer colReplica.mu.RUnlock()
for index, ele := range colReplica.segments {
if ele.segmentID == segmentID {
log.Println("Removing segment:", segmentID)
numOfSegs := len(colReplica.segments)
colReplica.segments[index] = colReplica.segments[numOfSegs-1]
colReplica.segments = colReplica.segments[:numOfSegs-1]
return nil
}
}
return errors.Errorf("Error, there's no segment %v", segmentID)
}
func (colReplica *collectionReplicaImpl) hasSegment(segmentID UniqueID) bool {
colReplica.mu.RLock()
defer colReplica.mu.RUnlock()
for _, ele := range colReplica.segments {
if ele.segmentID == segmentID {
return true
}
}
return false
}
func (colReplica *collectionReplicaImpl) updateSegmentRowNums(segmentID UniqueID, numRows int64) error {
colReplica.mu.RLock()
defer colReplica.mu.RUnlock()
for _, ele := range colReplica.segments {
if ele.segmentID == segmentID {
log.Printf("updating segment(%v) row nums: (%v)", segmentID, numRows)
ele.memorySize = 0
ele.numRows += numRows
return nil
}
}
return errors.Errorf("Error, there's no segment %v", segmentID)
}
func (colReplica *collectionReplicaImpl) getSegmentStatisticsUpdates(segmentID UniqueID) (*internalpb2.SegmentStatisticsUpdates, error) {
colReplica.mu.RLock()
defer colReplica.mu.RUnlock()
for _, ele := range colReplica.segments {
if ele.segmentID == segmentID {
updates := &internalpb2.SegmentStatisticsUpdates{
SegmentID: segmentID,
MemorySize: ele.memorySize,
NumRows: ele.numRows,
}
return updates, nil
}
}
return nil, errors.Errorf("Error, there's no segment %v", segmentID)
}
func (colReplica *collectionReplicaImpl) getCollectionNum() int {
@ -43,13 +126,13 @@ func (colReplica *collectionReplicaImpl) addCollection(collectionID UniqueID, sc
var newCollection = newCollection(collectionID, schemaBlob)
colReplica.collections = append(colReplica.collections, newCollection)
fmt.Println("yyy, create collection: ", newCollection.Name())
log.Println("Create collection: ", newCollection.Name())
return nil
}
func (colReplica *collectionReplicaImpl) removeCollection(collectionID UniqueID) error {
fmt.Println("drop collection:", collectionID)
// GOOSE TODO: optimize
colReplica.mu.Lock()
defer colReplica.mu.Unlock()
@ -58,7 +141,7 @@ func (colReplica *collectionReplicaImpl) removeCollection(collectionID UniqueID)
if col.ID() != collectionID {
tmpCollections = append(tmpCollections, col)
} else {
fmt.Println("yyy, drop collection name: ", col.Name())
log.Println("Drop collection : ", col.Name())
}
}
colReplica.collections = tmpCollections
@ -74,8 +157,7 @@ func (colReplica *collectionReplicaImpl) getCollectionByID(collectionID UniqueID
return collection, nil
}
}
return nil, errors.New("cannot find collection, id = " + strconv.FormatInt(collectionID, 10))
return nil, errors.Errorf("cannot find collection, id = %v", collectionID)
}
func (colReplica *collectionReplicaImpl) getCollectionByName(collectionName string) (*Collection, error) {
@ -88,7 +170,7 @@ func (colReplica *collectionReplicaImpl) getCollectionByName(collectionName stri
}
}
return nil, errors.New("Cannot found collection: " + collectionName)
return nil, errors.Errorf("Cannot found collection: %v", collectionName)
}
func (colReplica *collectionReplicaImpl) hasCollection(collectionID UniqueID) bool {

View File

@ -67,7 +67,7 @@ func genTestCollectionMeta(collectionName string, collectionID UniqueID) *etcdpb
return &collectionMeta
}
func initTestMeta(t *testing.T, replica collectionReplica, collectionName string, collectionID UniqueID, segmentID UniqueID) {
func initTestReplicaMeta(t *testing.T, replica collectionReplica, collectionName string, collectionID UniqueID, segmentID UniqueID) {
collectionMeta := genTestCollectionMeta(collectionName, collectionID)
schemaBlob := proto.MarshalTextString(collectionMeta.Schema)
@ -87,18 +87,18 @@ func initTestMeta(t *testing.T, replica collectionReplica, collectionName string
//----------------------------------------------------------------------------------------------------- collection
func TestCollectionReplica_getCollectionNum(t *testing.T) {
replica := newReplica()
initTestMeta(t, replica, "collection0", 0, 0)
initTestReplicaMeta(t, replica, "collection0", 0, 0)
assert.Equal(t, replica.getCollectionNum(), 1)
}
func TestCollectionReplica_addCollection(t *testing.T) {
replica := newReplica()
initTestMeta(t, replica, "collection0", 0, 0)
initTestReplicaMeta(t, replica, "collection0", 0, 0)
}
func TestCollectionReplica_removeCollection(t *testing.T) {
replica := newReplica()
initTestMeta(t, replica, "collection0", 0, 0)
initTestReplicaMeta(t, replica, "collection0", 0, 0)
assert.Equal(t, replica.getCollectionNum(), 1)
err := replica.removeCollection(0)
@ -110,7 +110,7 @@ func TestCollectionReplica_getCollectionByID(t *testing.T) {
replica := newReplica()
collectionName := "collection0"
collectionID := UniqueID(0)
initTestMeta(t, replica, collectionName, collectionID, 0)
initTestReplicaMeta(t, replica, collectionName, collectionID, 0)
targetCollection, err := replica.getCollectionByID(collectionID)
assert.NoError(t, err)
assert.NotNil(t, targetCollection)
@ -122,7 +122,7 @@ func TestCollectionReplica_getCollectionByName(t *testing.T) {
replica := newReplica()
collectionName := "collection0"
collectionID := UniqueID(0)
initTestMeta(t, replica, collectionName, collectionID, 0)
initTestReplicaMeta(t, replica, collectionName, collectionID, 0)
targetCollection, err := replica.getCollectionByName(collectionName)
assert.NoError(t, err)
@ -136,7 +136,7 @@ func TestCollectionReplica_hasCollection(t *testing.T) {
replica := newReplica()
collectionName := "collection0"
collectionID := UniqueID(0)
initTestMeta(t, replica, collectionName, collectionID, 0)
initTestReplicaMeta(t, replica, collectionName, collectionID, 0)
hasCollection := replica.hasCollection(collectionID)
assert.Equal(t, hasCollection, true)
@ -149,6 +149,6 @@ func TestCollectionReplica_freeAll(t *testing.T) {
replica := newReplica()
collectionName := "collection0"
collectionID := UniqueID(0)
initTestMeta(t, replica, collectionName, collectionID, 0)
initTestReplicaMeta(t, replica, collectionName, collectionID, 0)
}

View File

@ -2,8 +2,8 @@ package datanode
import (
"context"
"fmt"
"io"
"log"
"github.com/opentracing/opentracing-go"
"github.com/uber/jaeger-client-go"
@ -16,10 +16,10 @@ type DataNode struct {
dataSyncService *dataSyncService
flushSyncService *flushSyncService
metaService *metaService
// segStatsService *statsService
replica collectionReplica
tracer opentracing.Tracer
closer io.Closer
replica collectionReplica
tracer opentracing.Tracer
closer io.Closer
}
func NewDataNode(ctx context.Context, dataNodeID uint64) *DataNode {
@ -36,8 +36,7 @@ func NewDataNode(ctx context.Context, dataNodeID uint64) *DataNode {
dataSyncService: nil,
flushSyncService: nil,
metaService: nil,
// segStatsService: nil,
replica: replica,
replica: replica,
}
return node
@ -58,26 +57,26 @@ func (node *DataNode) Start() error {
LogSpans: true,
},
}
var err error
node.tracer, node.closer, err = cfg.NewTracer(config.Logger(jaeger.StdLogger))
if err != nil {
panic(fmt.Sprintf("ERROR: cannot init Jaeger: %v\n", err))
log.Printf("ERROR: cannot init Jaeger: %v\n", err)
} else {
opentracing.SetGlobalTracer(node.tracer)
}
opentracing.SetGlobalTracer(node.tracer)
// TODO GOOSE Init Size??
chanSize := 100
ddChan := make(chan *ddlFlushSyncMsg, chanSize)
insertChan := make(chan *insertFlushSyncMsg, chanSize)
node.flushSyncService = newFlushSyncService(node.ctx, ddChan, insertChan)
node.flushSyncService = newFlushSyncService(node.ctx, ddChan, insertChan)
node.dataSyncService = newDataSyncService(node.ctx, ddChan, insertChan, node.replica)
node.metaService = newMetaService(node.ctx, node.replica)
// node.segStatsService = newStatsService(node.ctx, node.replica)
go node.dataSyncService.start()
go node.flushSyncService.start()
// go node.segStatsService.start()
node.metaService.start()
return nil
@ -91,10 +90,6 @@ func (node *DataNode) Close() {
(*node.dataSyncService).close()
}
// if node.segStatsService != nil {
// (*node.segStatsService).close()
// }
if node.closer != nil {
node.closer.Close()
}

View File

@ -10,12 +10,15 @@ import (
"testing"
"time"
// "github.com/stretchr/testify/assert"
// "github.com/stretchr/testify/require"
"github.com/golang/protobuf/proto"
"github.com/stretchr/testify/require"
"go.etcd.io/etcd/clientv3"
"go.uber.org/zap"
"github.com/zilliztech/milvus-distributed/internal/master"
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
"github.com/zilliztech/milvus-distributed/internal/proto/etcdpb"
"github.com/zilliztech/milvus-distributed/internal/proto/schemapb"
)
func makeNewChannelNames(names []string, suffix string) []string {
@ -77,8 +80,6 @@ func TestMain(m *testing.M) {
}
startMaster(ctx)
// p := Params
// fmt.Println(p)
exitCode := m.Run()
os.Exit(exitCode)
}
@ -106,96 +107,90 @@ func newDataNode() *DataNode {
}
// func genTestCollectionMeta(collectionName string, collectionID UniqueID, isBinary bool) *etcdpb.CollectionMeta {
// var fieldVec schemapb.FieldSchema
// if isBinary {
// fieldVec = schemapb.FieldSchema{
// FieldID: UniqueID(100),
// Name: "vec",
// IsPrimaryKey: false,
// DataType: schemapb.DataType_VECTOR_BINARY,
// TypeParams: []*commonpb.KeyValuePair{
// {
// Key: "dim",
// Value: "128",
// },
// },
// IndexParams: []*commonpb.KeyValuePair{
// {
// Key: "metric_type",
// Value: "JACCARD",
// },
// },
// }
// } else {
// fieldVec = schemapb.FieldSchema{
// FieldID: UniqueID(100),
// Name: "vec",
// IsPrimaryKey: false,
// DataType: schemapb.DataType_VECTOR_FLOAT,
// TypeParams: []*commonpb.KeyValuePair{
// {
// Key: "dim",
// Value: "16",
// },
// },
// IndexParams: []*commonpb.KeyValuePair{
// {
// Key: "metric_type",
// Value: "L2",
// },
// },
// }
// }
//
// fieldInt := schemapb.FieldSchema{
// FieldID: UniqueID(101),
// Name: "age",
// IsPrimaryKey: false,
// DataType: schemapb.DataType_INT32,
// }
//
// schema := schemapb.CollectionSchema{
// Name: collectionName,
// AutoID: true,
// Fields: []*schemapb.FieldSchema{
// &fieldVec, &fieldInt,
// },
// }
//
// collectionMeta := etcdpb.CollectionMeta{
// ID: collectionID,
// Schema: &schema,
// CreateTime: Timestamp(0),
// SegmentIDs: []UniqueID{0},
// PartitionTags: []string{"default"},
// }
//
// return &collectionMeta
// }
func genTestDataNodeCollectionMeta(collectionName string, collectionID UniqueID, isBinary bool) *etcdpb.CollectionMeta {
var fieldVec schemapb.FieldSchema
if isBinary {
fieldVec = schemapb.FieldSchema{
FieldID: UniqueID(100),
Name: "vec",
IsPrimaryKey: false,
DataType: schemapb.DataType_VECTOR_BINARY,
TypeParams: []*commonpb.KeyValuePair{
{
Key: "dim",
Value: "128",
},
},
IndexParams: []*commonpb.KeyValuePair{
{
Key: "metric_type",
Value: "JACCARD",
},
},
}
} else {
fieldVec = schemapb.FieldSchema{
FieldID: UniqueID(100),
Name: "vec",
IsPrimaryKey: false,
DataType: schemapb.DataType_VECTOR_FLOAT,
TypeParams: []*commonpb.KeyValuePair{
{
Key: "dim",
Value: "16",
},
},
IndexParams: []*commonpb.KeyValuePair{
{
Key: "metric_type",
Value: "L2",
},
},
}
}
// func initTestMeta(t *testing.T, node *DataNode, collectionName string, collectionID UniqueID, segmentID UniqueID, optional ...bool) {
// isBinary := false
// if len(optional) > 0 {
// isBinary = optional[0]
// }
// collectionMeta := genTestCollectionMeta(collectionName, collectionID, isBinary)
//
// schemaBlob := proto.MarshalTextString(collectionMeta.Schema)
// require.NotEqual(t, "", schemaBlob)
//
// var err = node.replica.addCollection(collectionMeta.ID, schemaBlob)
// require.NoError(t, err)
//
// collection, err := node.replica.getCollectionByName(collectionName)
// require.NoError(t, err)
// require.Equal(t, collection.Name(), collectionName)
// require.Equal(t, collection.ID(), collectionID)
// require.Equal(t, node.replica.getCollectionNum(), 1)
//
// err = node.replica.addPartition(collection.ID(), collectionMeta.PartitionTags[0])
// require.NoError(t, err)
//
// err = node.replica.addSegment(segmentID, collectionMeta.PartitionTags[0], collectionID)
// require.NoError(t, err)
// }
fieldInt := schemapb.FieldSchema{
FieldID: UniqueID(101),
Name: "age",
IsPrimaryKey: false,
DataType: schemapb.DataType_INT32,
}
schema := schemapb.CollectionSchema{
Name: collectionName,
AutoID: true,
Fields: []*schemapb.FieldSchema{
&fieldVec, &fieldInt,
},
}
collectionMeta := etcdpb.CollectionMeta{
ID: collectionID,
Schema: &schema,
CreateTime: Timestamp(0),
SegmentIDs: []UniqueID{0},
PartitionTags: []string{"default"},
}
return &collectionMeta
}
func initTestMeta(t *testing.T, node *DataNode, collectionName string, collectionID UniqueID, segmentID UniqueID, optional ...bool) {
isBinary := false
if len(optional) > 0 {
isBinary = optional[0]
}
collectionMeta := genTestDataNodeCollectionMeta(collectionName, collectionID, isBinary)
schemaBlob := proto.MarshalTextString(collectionMeta.Schema)
require.NotEqual(t, "", schemaBlob)
var err = node.replica.addCollection(collectionMeta.ID, schemaBlob)
require.NoError(t, err)
collection, err := node.replica.getCollectionByName(collectionName)
require.NoError(t, err)
require.Equal(t, collection.Name(), collectionName)
require.Equal(t, collection.ID(), collectionID)
require.Equal(t, node.replica.getCollectionNum(), 1)
}

View File

@ -23,7 +23,6 @@ import (
// NOTE: start pulsar before test
func TestDataSyncService_Start(t *testing.T) {
newMeta()
const ctxTimeInMillisecond = 2000
const closeWithDeadline = true
var ctx context.Context
@ -39,7 +38,9 @@ func TestDataSyncService_Start(t *testing.T) {
// init data node
pulsarURL := Params.PulsarAddress
collMeta := newMeta()
node := NewDataNode(ctx, 0)
node.replica.addCollection(collMeta.ID, proto.MarshalTextString(collMeta.Schema))
// test data generate
// GOOSE TODO orgnize
@ -204,8 +205,8 @@ func TestDataSyncService_Start(t *testing.T) {
assert.NoError(t, err)
// dataSync
replica := newReplica()
node.dataSyncService = newDataSyncService(node.ctx, nil, nil, replica)
// replica := newReplica()
node.dataSyncService = newDataSyncService(node.ctx, nil, nil, node.replica)
go node.dataSyncService.start()
node.Close()

View File

@ -43,7 +43,8 @@ type (
minioPrifex string
idAllocator *allocator.IDAllocator
outCh chan *insertFlushSyncMsg
pulsarDataNodeTimeTickStream *pulsarms.PulsarMsgStream
pulsarDataNodeTimeTickStream msgstream.MsgStream
segmentStatisticsStream msgstream.MsgStream
replica collectionReplica
}
@ -100,6 +101,33 @@ func (ibNode *insertBufferNode) Operate(in []*Msg) []*Msg {
// TODO: add error handling
}
uniqueSeg := make(map[UniqueID]bool)
for _, msg := range iMsg.insertMessages {
currentSegID := msg.GetSegmentID()
if !ibNode.replica.hasSegment(currentSegID) {
err := ibNode.replica.addSegment(currentSegID)
if err != nil {
log.Println("Error: add segment error")
}
}
err := ibNode.replica.updateSegmentRowNums(currentSegID, int64(len(msg.RowIDs)))
if err != nil {
log.Println("Error: update Segment Row number wrong, ", err)
}
if _, ok := uniqueSeg[currentSegID]; !ok {
uniqueSeg[currentSegID] = true
}
}
segIDs := make([]UniqueID, 0, len(uniqueSeg))
for id := range uniqueSeg {
segIDs = append(segIDs, id)
}
err := ibNode.updateSegStatistics(segIDs)
if err != nil {
log.Println("Error: update segment statistics error, ", err)
}
// iMsg is insertMsg
// 1. iMsg -> buffer
for _, msg := range iMsg.insertMessages {
@ -567,22 +595,6 @@ func (ibNode *insertBufferNode) Operate(in []*Msg) []*Msg {
return []*Msg{&res}
}
func (ibNode *insertBufferNode) getCollectionSchemaByID(collectionID UniqueID) (*schemapb.CollectionSchema, error) {
ret, err := ibNode.replica.getCollectionByID(collectionID)
if err != nil {
return nil, err
}
return ret.schema, nil
}
func (ibNode *insertBufferNode) getCollectionSchemaByName(collectionName string) (*schemapb.CollectionSchema, error) {
ret, err := ibNode.replica.getCollectionByName(collectionName)
if err != nil {
return nil, err
}
return ret.schema, nil
}
func (ibNode *insertBufferNode) writeHardTimeTick(ts Timestamp) error {
msgPack := msgstream.MsgPack{}
timeTickMsg := msgstream.TimeTickMsg{
@ -604,6 +616,61 @@ func (ibNode *insertBufferNode) writeHardTimeTick(ts Timestamp) error {
return ibNode.pulsarDataNodeTimeTickStream.Produce(&msgPack)
}
func (ibNode *insertBufferNode) updateSegStatistics(segIDs []UniqueID) error {
log.Println("Updating segments statistics...")
statsUpdates := make([]*internalpb2.SegmentStatisticsUpdates, 0, len(segIDs))
for _, segID := range segIDs {
updates, err := ibNode.replica.getSegmentStatisticsUpdates(segID)
if err != nil {
log.Println("Error get segment", segID, "statistics updates", err)
continue
}
statsUpdates = append(statsUpdates, updates)
}
segStats := internalpb2.SegmentStatistics{
Base: &commonpb.MsgBase{
MsgType: commonpb.MsgType_kSegmentStatistics,
MsgID: UniqueID(0), // GOOSE TODO
Timestamp: Timestamp(0), // GOOSE TODO
SourceID: Params.DataNodeID,
},
SegStats: statsUpdates,
}
var msg msgstream.TsMsg = &msgstream.SegmentStatisticsMsg{
BaseMsg: msgstream.BaseMsg{
HashValues: []uint32{0},
},
SegmentStatistics: segStats,
}
var msgPack = msgstream.MsgPack{
Msgs: []msgstream.TsMsg{msg},
}
err := ibNode.segmentStatisticsStream.Produce(&msgPack)
if err != nil {
return err
}
return nil
}
func (ibNode *insertBufferNode) getCollectionSchemaByID(collectionID UniqueID) (*schemapb.CollectionSchema, error) {
ret, err := ibNode.replica.getCollectionByID(collectionID)
if err != nil {
return nil, err
}
return ret.schema, nil
}
func (ibNode *insertBufferNode) getCollectionSchemaByName(collectionName string) (*schemapb.CollectionSchema, error) {
ret, err := ibNode.replica.getCollectionByName(collectionName)
if err != nil {
return nil, err
}
return ret.schema, nil
}
func newInsertBufferNode(ctx context.Context, outCh chan *insertFlushSyncMsg, replica collectionReplica) *insertBufferNode {
maxQueueLength := Params.FlowGraphMaxQueueLength
maxParallelism := Params.FlowGraphMaxParallelism
@ -619,7 +686,6 @@ func newInsertBufferNode(ctx context.Context, outCh chan *insertFlushSyncMsg, re
}
// MinIO
option := &miniokv.Option{
Address: Params.MinioAddress,
AccessKeyID: Params.MinioAccessKeyID,
@ -644,9 +710,24 @@ func newInsertBufferNode(ctx context.Context, outCh chan *insertFlushSyncMsg, re
panic(err)
}
wTt := pulsarms.NewPulsarMsgStream(ctx, 1024) //input stream, data node time tick
// GOOSE TODO: Pulsar stream Start() ???
//input stream, data node time tick
wTt := pulsarms.NewPulsarMsgStream(ctx, 1024)
wTt.SetPulsarClient(Params.PulsarAddress)
wTt.CreatePulsarProducers([]string{Params.TimeTickChannelName})
var wTtMsgStream msgstream.MsgStream = wTt
// var wTtMsgStream pulsarms.PulsarMsgStream = *wTt
wTtMsgStream.Start()
// wTt.Start()
// update statistics channel
segS := pulsarms.NewPulsarMsgStream(ctx, Params.SegmentStatisticsBufSize)
segS.SetPulsarClient(Params.PulsarAddress)
segS.CreatePulsarProducers([]string{Params.SegmentStatisticsChannelName})
var segStatisticsMsgStream msgstream.MsgStream = segS
// var segStatisticsMsgStream pulsarms.PulsarMsgStream = segS
segStatisticsMsgStream.Start()
// segS.Start()
return &insertBufferNode{
BaseNode: baseNode,
@ -655,7 +736,10 @@ func newInsertBufferNode(ctx context.Context, outCh chan *insertFlushSyncMsg, re
minioPrifex: minioPrefix,
idAllocator: idAllocator,
outCh: outCh,
pulsarDataNodeTimeTickStream: wTt,
replica: replica,
pulsarDataNodeTimeTickStream: wTtMsgStream,
segmentStatisticsStream: segStatisticsMsgStream,
// pulsarDataNodeTimeTickStream: wTt,
// segmentStatisticsStream: segS,
replica: replica,
}
}

View File

@ -20,7 +20,7 @@ import (
"github.com/zilliztech/milvus-distributed/internal/util/flowgraph"
)
func TestFlowGraphInputBufferNode_Operate(t *testing.T) {
func TestFlowGraphInsertBufferNode_Operate(t *testing.T) {
const ctxTimeInMillisecond = 2000
const closeWithDeadline = false
var ctx context.Context

View File

@ -43,9 +43,9 @@ type ParamTable struct {
DDPulsarBufSize int64
// - seg statistics channel -
SegmentStatisticsChannelName string
SegmentStatisticsBufSize int64
SegmentStatisticsPublishInterval int
SegmentStatisticsChannelName string
SegmentStatisticsBufSize int64
SegmentStatisticsUpdateInterval int // GOOSE TODO remove
// - timetick channel -
TimeTickChannelName string
@ -108,7 +108,7 @@ func (p *ParamTable) Init() {
// - seg statistics channel -
p.initSegmentStatisticsChannelName()
p.initSegmentStatisticsBufSize()
p.initSegmentStatisticsPublishInterval()
p.initSegmentStatisticsUpdateInterval()
// - timetick channel -
p.initTimeTickChannelName()
@ -309,8 +309,8 @@ func (p *ParamTable) initSegmentStatisticsBufSize() {
p.SegmentStatisticsBufSize = p.ParseInt64("dataNode.msgStream.segStatistics.recvBufSize")
}
func (p *ParamTable) initSegmentStatisticsPublishInterval() {
p.SegmentStatisticsPublishInterval = p.ParseInt("dataNode.msgStream.segStatistics.publishInterval")
func (p *ParamTable) initSegmentStatisticsUpdateInterval() {
p.SegmentStatisticsUpdateInterval = p.ParseInt("dataNode.msgStream.segStatistics.updateInterval")
}
// - Timetick channel -

View File

@ -2,171 +2,156 @@ package datanode
import (
"log"
"strings"
"testing"
"github.com/stretchr/testify/assert"
)
func TestParamTable_DataNode(t *testing.T) {
Params.Init()
log.Println("Params in ParamTable test: ", Params)
t.Run("Test DataNodeID", func(t *testing.T) {
id := Params.DataNodeID
assert.Equal(t, id, UniqueID(3))
log.Println("DataNodeID:", id)
})
t.Run("Test flowGraphMaxQueueLength", func(t *testing.T) {
length := Params.FlowGraphMaxQueueLength
assert.Equal(t, length, int32(1024))
log.Println("flowGraphMaxQueueLength:", length)
})
t.Run("Test flowGraphMaxParallelism", func(t *testing.T) {
maxParallelism := Params.FlowGraphMaxParallelism
assert.Equal(t, maxParallelism, int32(1024))
log.Println("flowGraphMaxParallelism:", maxParallelism)
})
t.Run("Test FlushInsertBufSize", func(t *testing.T) {
size := Params.FlushInsertBufferSize
assert.Equal(t, int32(500), size)
log.Println("FlushInsertBufferSize:", size)
})
t.Run("Test FlushDdBufSize", func(t *testing.T) {
size := Params.FlushDdBufferSize
assert.Equal(t, int32(20), size)
log.Println("FlushDdBufferSize:", size)
})
t.Run("Test InsertBinlogRootPath", func(t *testing.T) {
path := Params.InsertBinlogRootPath
assert.Equal(t, "by-dev/insert_log", path)
log.Println("InsertBinlogRootPath:", path)
})
t.Run("Test DdBinlogRootPath", func(t *testing.T) {
path := Params.DdBinlogRootPath
assert.Equal(t, "by-dev/data_definition_log", path)
log.Println("DdBinlogRootPath:", path)
})
t.Run("Test MasterAddress", func(t *testing.T) {
address := Params.MasterAddress
split := strings.Split(address, ":")
assert.Equal(t, "localhost", split[0])
assert.Equal(t, "53100", split[1])
log.Println("MasterAddress:", address)
})
t.Run("Test PulsarAddress", func(t *testing.T) {
address := Params.PulsarAddress
split := strings.Split(address, ":")
assert.Equal(t, split[0], "pulsar")
assert.Equal(t, split[len(split)-1], "6650")
log.Println("PulsarAddress:", address)
})
t.Run("Test insertChannelNames", func(t *testing.T) {
names := Params.InsertChannelNames
assert.Equal(t, len(names), 2)
assert.Equal(t, names[0], "insert-0")
assert.Equal(t, names[1], "insert-1")
log.Println("InsertChannelNames:", names)
})
t.Run("Test insertChannelRange", func(t *testing.T) {
channelRange := Params.InsertChannelRange
assert.Equal(t, len(channelRange), 2)
assert.Equal(t, channelRange[0], 0)
assert.Equal(t, channelRange[1], 2)
log.Println("InsertChannelRange:", channelRange)
})
t.Run("Test insertMsgStreamReceiveBufSize", func(t *testing.T) {
bufSize := Params.InsertReceiveBufSize
assert.Equal(t, bufSize, int64(1024))
log.Println("InsertReceiveBufSize:", bufSize)
})
t.Run("Test insertPulsarBufSize", func(t *testing.T) {
bufSize := Params.InsertPulsarBufSize
assert.Equal(t, bufSize, int64(1024))
log.Println("InsertPulsarBufSize:", bufSize)
})
t.Run("Test ddChannelNames", func(t *testing.T) {
names := Params.DDChannelNames
assert.Equal(t, len(names), 1)
assert.Equal(t, names[0], "data-definition-0")
log.Println("DDChannelNames:", names)
})
t.Run("Test DdMsgStreamReceiveBufSize", func(t *testing.T) {
bufSize := Params.DDReceiveBufSize
assert.Equal(t, int64(64), bufSize)
log.Println("DDReceiveBufSize:", bufSize)
})
t.Run("Test DdPulsarBufSize", func(t *testing.T) {
bufSize := Params.DDPulsarBufSize
assert.Equal(t, int64(64), bufSize)
log.Println("DDPulsarBufSize:", bufSize)
})
t.Run("Test SegmentStatisticsChannelName", func(t *testing.T) {
name := Params.SegmentStatisticsChannelName
assert.Equal(t, "dataNodeSegStatistics", name)
log.Println("SegmentStatisticsChannelName:", name)
})
t.Run("Test SegmentStatisticsBufSize", func(t *testing.T) {
size := Params.SegmentStatisticsBufSize
assert.Equal(t, int64(64), size)
log.Println("SegmentStatisticsBufSize:", size)
})
t.Run("Test SegmentStatisticsPublishInterval", func(t *testing.T) {
interval := Params.SegmentStatisticsPublishInterval
assert.Equal(t, 1000, interval)
t.Run("Test SegmentStatisticsUpdateInterval", func(t *testing.T) {
interval := Params.SegmentStatisticsUpdateInterval
log.Println("SegmentStatisticsUpdateInterval:", interval)
})
t.Run("Test timeTickChannelName", func(t *testing.T) {
name := Params.TimeTickChannelName
assert.Equal(t, "dataNodeTimeTick-3", name)
log.Println("TimeTickChannelName:", name)
})
t.Run("Test msgChannelSubName", func(t *testing.T) {
name := Params.MsgChannelSubName
assert.Equal(t, "dataNode-3", name)
log.Println("MsgChannelSubName:", name)
})
t.Run("Test EtcdAddress", func(t *testing.T) {
addr := Params.EtcdAddress
split := strings.Split(addr, ":")
assert.Equal(t, "localhost", split[0])
assert.Equal(t, "2379", split[1])
log.Println("EtcdAddress:", addr)
})
t.Run("Test MetaRootPath", func(t *testing.T) {
path := Params.MetaRootPath
assert.Equal(t, "by-dev/meta", path)
log.Println("MetaRootPath:", path)
})
t.Run("Test SegFlushMetaSubPath", func(t *testing.T) {
path := Params.SegFlushMetaSubPath
assert.Equal(t, "writer/segment", path)
log.Println("SegFlushMetaSubPath:", path)
})
t.Run("Test DDLFlushMetaSubPath", func(t *testing.T) {
path := Params.DDLFlushMetaSubPath
assert.Equal(t, "writer/ddl", path)
log.Println("DDLFlushMetaSubPath:", path)
})
t.Run("Test minioAccessKeyID", func(t *testing.T) {
id := Params.MinioAccessKeyID
assert.Equal(t, "minioadmin", id)
log.Println("MinioAccessKeyID:", id)
})
t.Run("Test minioSecretAccessKey", func(t *testing.T) {
key := Params.MinioSecretAccessKey
assert.Equal(t, "minioadmin", key)
log.Println("MinioSecretAccessKey:", key)
})
t.Run("Test MinioUseSSL", func(t *testing.T) {
useSSL := Params.MinioUseSSL
assert.Equal(t, false, useSSL)
log.Println("MinioUseSSL:", useSSL)
})
t.Run("Test MinioBucketName", func(t *testing.T) {
name := Params.MinioBucketName
assert.Equal(t, "a-bucket", name)
log.Println("MinioBucketName:", name)
})
}

View File

@ -321,6 +321,7 @@ func (tst *TimeTickMsg) Unmarshal(input []byte) (TsMsg, error) {
}
/////////////////////////////////////////QueryNodeStats//////////////////////////////////////////
// GOOSE TODO: remove QueryNodeStats
type QueryNodeStatsMsg struct {
BaseMsg
internalpb2.QueryNodeStats
@ -359,6 +360,45 @@ func (qs *QueryNodeStatsMsg) Unmarshal(input []byte) (TsMsg, error) {
return queryNodeSegStatsMsg, nil
}
/////////////////////////////////////////SegmentStatisticsMsg//////////////////////////////////////////
type SegmentStatisticsMsg struct {
BaseMsg
internalpb2.SegmentStatistics
}
func (ss *SegmentStatisticsMsg) Type() MsgType {
return ss.Base.MsgType
}
func (ss *SegmentStatisticsMsg) GetMsgContext() context.Context {
return ss.MsgCtx
}
func (ss *SegmentStatisticsMsg) SetMsgContext(ctx context.Context) {
ss.MsgCtx = ctx
}
func (ss *SegmentStatisticsMsg) Marshal(input TsMsg) ([]byte, error) {
segStatsTask := input.(*SegmentStatisticsMsg)
segStats := &segStatsTask.SegmentStatistics
mb, err := proto.Marshal(segStats)
if err != nil {
return nil, err
}
return mb, nil
}
func (ss *SegmentStatisticsMsg) Unmarshal(input []byte) (TsMsg, error) {
segStats := internalpb2.SegmentStatistics{}
err := proto.Unmarshal(input, &segStats)
if err != nil {
return nil, err
}
segStatsMsg := &SegmentStatisticsMsg{SegmentStatistics: segStats}
return segStatsMsg, nil
}
///////////////////////////////////////////Key2Seg//////////////////////////////////////////
//type Key2SegMsg struct {
// BaseMsg

View File

@ -101,11 +101,12 @@ enum MsgType {
/* System Control */
kTimeTick = 1200;
kQueryNodeStats = 1201;
kQueryNodeStats = 1201; // GOOSE TODO: Remove kQueryNodeStats
kLoadIndex = 1202;
kRequestID = 1203;
kRequestTSO = 1204;
kAllocateSegment = 1205;
kSegmentStatistics = 1206;
}
message MsgBase {
@ -119,4 +120,4 @@ message MsgBase {
// Don't Modify This. @czs
message MsgHeader {
common.MsgBase base = 1;
}
}

View File

@ -180,12 +180,13 @@ const (
MsgType_kGetCollectionStatistics MsgType = 503
MsgType_kGetPartitionStatistics MsgType = 504
// System Control
MsgType_kTimeTick MsgType = 1200
MsgType_kQueryNodeStats MsgType = 1201
MsgType_kLoadIndex MsgType = 1202
MsgType_kRequestID MsgType = 1203
MsgType_kRequestTSO MsgType = 1204
MsgType_kAllocateSegment MsgType = 1205
MsgType_kTimeTick MsgType = 1200
MsgType_kQueryNodeStats MsgType = 1201
MsgType_kLoadIndex MsgType = 1202
MsgType_kRequestID MsgType = 1203
MsgType_kRequestTSO MsgType = 1204
MsgType_kAllocateSegment MsgType = 1205
MsgType_kSegmentStatistics MsgType = 1206
)
var MsgType_name = map[int32]string{
@ -217,6 +218,7 @@ var MsgType_name = map[int32]string{
1203: "kRequestID",
1204: "kRequestTSO",
1205: "kAllocateSegment",
1206: "kSegmentStatistics",
}
var MsgType_value = map[string]int32{
@ -248,6 +250,7 @@ var MsgType_value = map[string]int32{
"kRequestID": 1203,
"kRequestTSO": 1204,
"kAllocateSegment": 1205,
"kSegmentStatistics": 1206,
}
func (x MsgType) String() string {
@ -588,73 +591,73 @@ func init() {
func init() { proto.RegisterFile("common.proto", fileDescriptor_555bd8c177793206) }
var fileDescriptor_555bd8c177793206 = []byte{
// 1077 bytes of a gzipped FileDescriptorProto
// 1086 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x74, 0x54, 0xcb, 0x6e, 0xe3, 0x36,
0x14, 0x1d, 0x3f, 0x12, 0x47, 0xd7, 0x1e, 0x87, 0x61, 0x5e, 0x6e, 0x9b, 0x16, 0x81, 0x57, 0x41,
0x80, 0x49, 0x8a, 0x16, 0x68, 0x57, 0x03, 0x54, 0x91, 0xe8, 0x84, 0x18, 0x59, 0xf2, 0x50, 0xf2,
0x34, 0xd3, 0x8d, 0x20, 0xdb, 0x1c, 0x5b, 0x90, 0x6c, 0xb9, 0x22, 0x3d, 0xad, 0xe7, 0x2b, 0xda,
0xf9, 0x8e, 0x16, 0xe8, 0x13, 0xe8, 0xa2, 0x1f, 0xd0, 0xd7, 0x87, 0xf4, 0x03, 0xfa, 0x5a, 0x16,
0x94, 0xac, 0xd8, 0x28, 0xa6, 0x3b, 0xde, 0x73, 0x78, 0x0f, 0xef, 0xb9, 0xe4, 0x25, 0x34, 0x86,
0xc9, 0x74, 0x9a, 0xcc, 0x2e, 0xe6, 0x69, 0x22, 0x13, 0xbc, 0x3f, 0x0d, 0xe3, 0xe7, 0x0b, 0x91,
0x47, 0x17, 0x39, 0xd5, 0xae, 0xc1, 0x16, 0x99, 0xce, 0xe5, 0xb2, 0xed, 0xc3, 0xb6, 0x2b, 0x03,
0xb9, 0x10, 0xf8, 0x21, 0x00, 0x4f, 0xd3, 0x24, 0xf5, 0x87, 0xc9, 0x88, 0xb7, 0x4a, 0xa7, 0xa5,
0xb3, 0xe6, 0x3b, 0x6f, 0x5d, 0xbc, 0x22, 0xf9, 0x82, 0xa8, 0x6d, 0x46, 0x32, 0xe2, 0x4c, 0xe3,
0xc5, 0x12, 0x1f, 0xc1, 0x76, 0xca, 0x03, 0x91, 0xcc, 0x5a, 0xe5, 0xd3, 0xd2, 0x99, 0xc6, 0x56,
0x51, 0xfb, 0x3d, 0x68, 0x3c, 0xe2, 0xcb, 0x27, 0x41, 0xbc, 0xe0, 0xbd, 0x20, 0x4c, 0x31, 0x82,
0x4a, 0xc4, 0x97, 0x99, 0xbe, 0xc6, 0xd4, 0x12, 0x1f, 0xc0, 0xd6, 0x73, 0x45, 0xaf, 0x12, 0xf3,
0xa0, 0x7d, 0x02, 0xd5, 0xab, 0x38, 0x19, 0xac, 0x59, 0x95, 0xd1, 0x28, 0xd8, 0x07, 0x50, 0xd3,
0x47, 0xa3, 0x94, 0x0b, 0x81, 0x9b, 0x50, 0x0e, 0xe7, 0x2b, 0xbd, 0x72, 0x38, 0xc7, 0x18, 0xaa,
0xf3, 0x24, 0x95, 0x99, 0x5a, 0x85, 0x65, 0xeb, 0xf6, 0xcb, 0x12, 0xd4, 0xba, 0x62, 0x7c, 0x15,
0x08, 0x8e, 0xdf, 0x87, 0x9d, 0xa9, 0x18, 0xfb, 0x72, 0x39, 0x2f, 0x5c, 0x9e, 0xbc, 0xd2, 0x65,
0x57, 0x8c, 0xbd, 0xe5, 0x9c, 0xb3, 0xda, 0x34, 0x5f, 0xa8, 0x4a, 0xa6, 0x62, 0x4c, 0xcd, 0x95,
0x72, 0x1e, 0xe0, 0x13, 0xd0, 0x64, 0x38, 0xe5, 0x42, 0x06, 0xd3, 0x79, 0xab, 0x72, 0x5a, 0x3a,
0xab, 0xb2, 0x35, 0x80, 0x5f, 0x87, 0x1d, 0x91, 0x2c, 0xd2, 0x21, 0xa7, 0x66, 0xab, 0x9a, 0xa5,
0xdd, 0xc5, 0xed, 0x87, 0xa0, 0x75, 0xc5, 0xf8, 0x86, 0x07, 0x23, 0x9e, 0xe2, 0xb7, 0xa1, 0x3a,
0x08, 0x44, 0x5e, 0x51, 0xfd, 0xff, 0x2b, 0x52, 0x0e, 0x58, 0xb6, 0xf3, 0xfc, 0x87, 0x2a, 0x68,
0x77, 0x37, 0x81, 0xeb, 0x50, 0x73, 0xfb, 0x86, 0x41, 0x5c, 0x17, 0xdd, 0xc3, 0x07, 0x80, 0xfa,
0x36, 0xb9, 0xed, 0x11, 0xc3, 0x23, 0xa6, 0x4f, 0x18, 0x73, 0x18, 0x2a, 0x61, 0x0c, 0x4d, 0xc3,
0xb1, 0x6d, 0x62, 0x78, 0x7e, 0x47, 0xa7, 0x16, 0x31, 0x51, 0x19, 0x1f, 0xc2, 0x5e, 0x8f, 0xb0,
0x2e, 0x75, 0x5d, 0xea, 0xd8, 0xbe, 0x49, 0x6c, 0x4a, 0x4c, 0x54, 0xc1, 0xaf, 0xc1, 0xa1, 0xe1,
0x58, 0x16, 0x31, 0x3c, 0x05, 0xdb, 0x8e, 0xe7, 0x93, 0x5b, 0xea, 0x7a, 0x2e, 0xaa, 0x2a, 0x6d,
0x6a, 0x59, 0xe4, 0x5a, 0xb7, 0x7c, 0x9d, 0x5d, 0xf7, 0xbb, 0xc4, 0xf6, 0xd0, 0x96, 0xd2, 0x29,
0x50, 0x93, 0x76, 0x89, 0xad, 0xe4, 0x50, 0x0d, 0x1f, 0x01, 0x2e, 0x60, 0x6a, 0x9b, 0xe4, 0xd6,
0xf7, 0x9e, 0xf6, 0x08, 0xda, 0xc1, 0x6f, 0xc0, 0x71, 0x81, 0x6f, 0x9e, 0xa3, 0x77, 0x09, 0xd2,
0x30, 0x82, 0x46, 0x41, 0x7a, 0x4e, 0xef, 0x11, 0x82, 0x4d, 0x75, 0xe6, 0x7c, 0xc8, 0x88, 0xe1,
0x30, 0x13, 0xd5, 0x37, 0xe1, 0x27, 0xc4, 0xf0, 0x1c, 0xe6, 0x53, 0x13, 0x35, 0x54, 0xf1, 0x05,
0xec, 0x12, 0x9d, 0x19, 0x37, 0x3e, 0x23, 0x6e, 0xdf, 0xf2, 0xd0, 0x7d, 0xd5, 0x82, 0x0e, 0xb5,
0x48, 0xe6, 0xa8, 0xe3, 0xf4, 0x6d, 0x13, 0x35, 0xf1, 0x2e, 0xd4, 0xbb, 0xc4, 0xd3, 0x8b, 0x9e,
0xec, 0xaa, 0xf3, 0x0d, 0xdd, 0xb8, 0x21, 0x05, 0x82, 0x70, 0x0b, 0x0e, 0x0c, 0xdd, 0x56, 0x49,
0x06, 0x23, 0xba, 0x47, 0xfc, 0x8e, 0x63, 0x99, 0x84, 0xa1, 0x3d, 0x65, 0xf0, 0x3f, 0x0c, 0xb5,
0x08, 0xc2, 0x1b, 0x19, 0x26, 0xb1, 0xc8, 0x3a, 0x63, 0x7f, 0x23, 0xa3, 0x60, 0x54, 0xc6, 0x81,
0x32, 0x73, 0xd5, 0xa7, 0x96, 0xb9, 0x6a, 0x54, 0x7e, 0x69, 0x87, 0x78, 0x0f, 0xee, 0x17, 0x66,
0x6c, 0x8b, 0xba, 0x1e, 0x3a, 0xc2, 0xc7, 0xb0, 0x5f, 0x40, 0x5d, 0xe2, 0x31, 0x6a, 0xe4, 0x5d,
0x3d, 0x56, 0x7b, 0x9d, 0xbe, 0xe7, 0x3b, 0x1d, 0xbf, 0x4b, 0xba, 0x0e, 0x7b, 0x8a, 0x5a, 0xf8,
0x00, 0x76, 0x4d, 0xd3, 0x67, 0xe4, 0x71, 0x9f, 0xb8, 0x9e, 0xcf, 0x74, 0x83, 0xa0, 0xdf, 0x6b,
0xe7, 0x36, 0x00, 0x9d, 0x8d, 0xf8, 0xa7, 0x6a, 0xf2, 0x39, 0xde, 0x81, 0xaa, 0xed, 0xd8, 0x04,
0xdd, 0xc3, 0x0d, 0xd8, 0xe9, 0xdb, 0xd4, 0x75, 0xfb, 0xc4, 0x44, 0x25, 0xdc, 0x04, 0xa0, 0x76,
0x8f, 0x39, 0xd7, 0x4c, 0xbd, 0xaa, 0xb2, 0x62, 0x3b, 0xd4, 0xa6, 0xee, 0x4d, 0xf6, 0x44, 0x00,
0xb6, 0x57, 0xfd, 0xa9, 0x9e, 0xff, 0x58, 0xcd, 0xc6, 0x2b, 0x9b, 0x12, 0x0d, 0xb6, 0x22, 0x3b,
0x99, 0x71, 0x74, 0x4f, 0x59, 0x8a, 0x8c, 0x94, 0x07, 0x92, 0x1b, 0x49, 0x1c, 0xf3, 0xa1, 0x0c,
0x93, 0x19, 0x1a, 0xe1, 0x7d, 0xd8, 0x8d, 0xcc, 0x34, 0x99, 0x6f, 0x80, 0x5c, 0xdd, 0x4c, 0x74,
0x13, 0x88, 0x0d, 0xec, 0x99, 0x32, 0x1a, 0x99, 0x5c, 0x0c, 0xd3, 0x70, 0xb0, 0xa9, 0x30, 0x56,
0x6f, 0x30, 0x72, 0x27, 0xc9, 0x27, 0x6b, 0x50, 0xa0, 0x49, 0x26, 0x71, 0xcd, 0xa5, 0xbb, 0x14,
0x46, 0x32, 0x7b, 0x16, 0x8e, 0x05, 0x0a, 0xf1, 0x21, 0xa0, 0x55, 0x09, 0xbd, 0x20, 0x95, 0x61,
0x96, 0xff, 0x53, 0x09, 0xef, 0x43, 0x33, 0x2b, 0x61, 0x0d, 0xfe, 0xac, 0xe6, 0xe3, 0xbe, 0x2a,
0x61, 0x8d, 0xfd, 0x52, 0xc2, 0xc7, 0x80, 0xef, 0x4a, 0x58, 0x13, 0xbf, 0x96, 0x54, 0x63, 0xb3,
0x12, 0xee, 0x40, 0x81, 0x7e, 0x2b, 0xe1, 0x3d, 0x68, 0xac, 0x8e, 0xcb, 0xfa, 0x8b, 0xbe, 0x28,
0xe7, 0x47, 0xad, 0x14, 0x72, 0xf0, 0x4b, 0xd5, 0xca, 0x5a, 0x44, 0x67, 0x82, 0xa7, 0x12, 0x7d,
0x56, 0xc9, 0x22, 0x93, 0xc7, 0x5c, 0x72, 0xf4, 0x79, 0x05, 0xd7, 0x61, 0x3b, 0xea, 0xc4, 0x0b,
0x31, 0x41, 0x2f, 0x73, 0xca, 0xe5, 0x41, 0x3a, 0x9c, 0xa0, 0x3f, 0x2a, 0x59, 0x85, 0x79, 0xc4,
0xb8, 0x58, 0xc4, 0x12, 0xfd, 0x59, 0xc9, 0xf4, 0xaf, 0xb9, 0x5c, 0xdf, 0x27, 0xfa, 0xab, 0x82,
0xdf, 0x84, 0x96, 0x02, 0xd7, 0xfd, 0x51, 0x4c, 0x28, 0x64, 0x38, 0x14, 0xe8, 0xef, 0x0a, 0x3e,
0x81, 0x63, 0x45, 0xdf, 0xd5, 0xbe, 0xc1, 0xfe, 0x53, 0xc1, 0x4d, 0xd0, 0x22, 0x2f, 0x9c, 0x72,
0x2f, 0x1c, 0x46, 0xe8, 0x2b, 0x2d, 0xb3, 0xfa, 0x78, 0xc1, 0xd3, 0xa5, 0x9d, 0x8c, 0xb8, 0xda,
0x2a, 0xd0, 0xd7, 0x1a, 0xde, 0x05, 0x88, 0xac, 0x24, 0x18, 0xe5, 0x9e, 0xbe, 0xc9, 0x01, 0xc6,
0x3f, 0x5e, 0x70, 0x21, 0xa9, 0x89, 0xbe, 0x55, 0x73, 0x5c, 0x2f, 0x00, 0xcf, 0x75, 0xd0, 0x77,
0x5a, 0x76, 0x1b, 0x7a, 0x1c, 0x27, 0xc3, 0x40, 0x72, 0x97, 0x8f, 0xa7, 0x7c, 0x26, 0xd1, 0xf7,
0xda, 0xd5, 0xd5, 0x47, 0x1f, 0x8c, 0x43, 0x39, 0x59, 0x0c, 0xd4, 0x47, 0x77, 0xf9, 0x22, 0x8c,
0xe3, 0xf0, 0x85, 0xe4, 0xc3, 0xc9, 0x65, 0xfe, 0x09, 0x3e, 0x18, 0x85, 0x42, 0xa6, 0xe1, 0x60,
0x21, 0xf9, 0xe8, 0x32, 0x9c, 0x49, 0x9e, 0xce, 0x82, 0xf8, 0x32, 0xfb, 0x19, 0x2f, 0xf3, 0x9f,
0x71, 0x3e, 0x18, 0x6c, 0x67, 0xf1, 0xbb, 0xff, 0x06, 0x00, 0x00, 0xff, 0xff, 0x8e, 0xe1, 0x76,
0x9b, 0xfc, 0x06, 0x00, 0x00,
0x17, 0x1e, 0x5f, 0x12, 0x47, 0xc7, 0x1e, 0x87, 0x61, 0x6e, 0xfe, 0xff, 0xa6, 0x45, 0xe0, 0x55,
0x10, 0x60, 0x92, 0xa2, 0x05, 0xda, 0xd5, 0x00, 0x55, 0x24, 0x3a, 0x21, 0x46, 0x96, 0x3c, 0x94,
0x3c, 0xcd, 0x74, 0x23, 0xc8, 0x36, 0xc7, 0x16, 0x24, 0x5b, 0xae, 0x48, 0x4f, 0xeb, 0x79, 0x8a,
0x76, 0x9e, 0xa3, 0x05, 0x7a, 0x47, 0x1f, 0xa1, 0xb7, 0x6d, 0xdf, 0xa1, 0x0f, 0xd0, 0xdb, 0xb2,
0xa0, 0x64, 0xc5, 0x46, 0x31, 0xdd, 0xf1, 0x7c, 0x1f, 0xcf, 0xc7, 0xef, 0x1c, 0xf2, 0x10, 0x1a,
0xc3, 0x64, 0x3a, 0x4d, 0x66, 0x17, 0xf3, 0x34, 0x91, 0x09, 0xde, 0x9f, 0x86, 0xf1, 0xf3, 0x85,
0xc8, 0xa3, 0x8b, 0x9c, 0x6a, 0xd7, 0x60, 0x8b, 0x4c, 0xe7, 0x72, 0xd9, 0xf6, 0x61, 0xdb, 0x95,
0x81, 0x5c, 0x08, 0xfc, 0x10, 0x80, 0xa7, 0x69, 0x92, 0xfa, 0xc3, 0x64, 0xc4, 0x5b, 0xa5, 0xd3,
0xd2, 0x59, 0xf3, 0xad, 0x37, 0x2e, 0x5e, 0x91, 0x7c, 0x41, 0xd4, 0x36, 0x23, 0x19, 0x71, 0xa6,
0xf1, 0x62, 0x89, 0x8f, 0x60, 0x3b, 0xe5, 0x81, 0x48, 0x66, 0xad, 0xf2, 0x69, 0xe9, 0x4c, 0x63,
0xab, 0xa8, 0xfd, 0x0e, 0x34, 0x1e, 0xf1, 0xe5, 0x93, 0x20, 0x5e, 0xf0, 0x5e, 0x10, 0xa6, 0x18,
0x41, 0x25, 0xe2, 0xcb, 0x4c, 0x5f, 0x63, 0x6a, 0x89, 0x0f, 0x60, 0xeb, 0xb9, 0xa2, 0x57, 0x89,
0x79, 0xd0, 0x3e, 0x81, 0xea, 0x55, 0x9c, 0x0c, 0xd6, 0xac, 0xca, 0x68, 0x14, 0xec, 0x03, 0xa8,
0xe9, 0xa3, 0x51, 0xca, 0x85, 0xc0, 0x4d, 0x28, 0x87, 0xf3, 0x95, 0x5e, 0x39, 0x9c, 0x63, 0x0c,
0xd5, 0x79, 0x92, 0xca, 0x4c, 0xad, 0xc2, 0xb2, 0x75, 0xfb, 0x65, 0x09, 0x6a, 0x5d, 0x31, 0xbe,
0x0a, 0x04, 0xc7, 0xef, 0xc2, 0xce, 0x54, 0x8c, 0x7d, 0xb9, 0x9c, 0x17, 0x55, 0x9e, 0xbc, 0xb2,
0xca, 0xae, 0x18, 0x7b, 0xcb, 0x39, 0x67, 0xb5, 0x69, 0xbe, 0x50, 0x4e, 0xa6, 0x62, 0x4c, 0xcd,
0x95, 0x72, 0x1e, 0xe0, 0x13, 0xd0, 0x64, 0x38, 0xe5, 0x42, 0x06, 0xd3, 0x79, 0xab, 0x72, 0x5a,
0x3a, 0xab, 0xb2, 0x35, 0x80, 0xff, 0x0f, 0x3b, 0x22, 0x59, 0xa4, 0x43, 0x4e, 0xcd, 0x56, 0x35,
0x4b, 0xbb, 0x8b, 0xdb, 0x0f, 0x41, 0xeb, 0x8a, 0xf1, 0x0d, 0x0f, 0x46, 0x3c, 0xc5, 0x6f, 0x42,
0x75, 0x10, 0x88, 0xdc, 0x51, 0xfd, 0xbf, 0x1d, 0xa9, 0x0a, 0x58, 0xb6, 0xf3, 0xfc, 0xfb, 0x2a,
0x68, 0x77, 0x37, 0x81, 0xeb, 0x50, 0x73, 0xfb, 0x86, 0x41, 0x5c, 0x17, 0xdd, 0xc3, 0x07, 0x80,
0xfa, 0x36, 0xb9, 0xed, 0x11, 0xc3, 0x23, 0xa6, 0x4f, 0x18, 0x73, 0x18, 0x2a, 0x61, 0x0c, 0x4d,
0xc3, 0xb1, 0x6d, 0x62, 0x78, 0x7e, 0x47, 0xa7, 0x16, 0x31, 0x51, 0x19, 0x1f, 0xc2, 0x5e, 0x8f,
0xb0, 0x2e, 0x75, 0x5d, 0xea, 0xd8, 0xbe, 0x49, 0x6c, 0x4a, 0x4c, 0x54, 0xc1, 0xff, 0x83, 0x43,
0xc3, 0xb1, 0x2c, 0x62, 0x78, 0x0a, 0xb6, 0x1d, 0xcf, 0x27, 0xb7, 0xd4, 0xf5, 0x5c, 0x54, 0x55,
0xda, 0xd4, 0xb2, 0xc8, 0xb5, 0x6e, 0xf9, 0x3a, 0xbb, 0xee, 0x77, 0x89, 0xed, 0xa1, 0x2d, 0xa5,
0x53, 0xa0, 0x26, 0xed, 0x12, 0x5b, 0xc9, 0xa1, 0x1a, 0x3e, 0x02, 0x5c, 0xc0, 0xd4, 0x36, 0xc9,
0xad, 0xef, 0x3d, 0xed, 0x11, 0xb4, 0x83, 0x5f, 0x83, 0xe3, 0x02, 0xdf, 0x3c, 0x47, 0xef, 0x12,
0xa4, 0x61, 0x04, 0x8d, 0x82, 0xf4, 0x9c, 0xde, 0x23, 0x04, 0x9b, 0xea, 0xcc, 0x79, 0x9f, 0x11,
0xc3, 0x61, 0x26, 0xaa, 0x6f, 0xc2, 0x4f, 0x88, 0xe1, 0x39, 0xcc, 0xa7, 0x26, 0x6a, 0x28, 0xf3,
0x05, 0xec, 0x12, 0x9d, 0x19, 0x37, 0x3e, 0x23, 0x6e, 0xdf, 0xf2, 0xd0, 0x7d, 0xd5, 0x82, 0x0e,
0xb5, 0x48, 0x56, 0x51, 0xc7, 0xe9, 0xdb, 0x26, 0x6a, 0xe2, 0x5d, 0xa8, 0x77, 0x89, 0xa7, 0x17,
0x3d, 0xd9, 0x55, 0xe7, 0x1b, 0xba, 0x71, 0x43, 0x0a, 0x04, 0xe1, 0x16, 0x1c, 0x18, 0xba, 0xad,
0x92, 0x0c, 0x46, 0x74, 0x8f, 0xf8, 0x1d, 0xc7, 0x32, 0x09, 0x43, 0x7b, 0xaa, 0xc0, 0x7f, 0x31,
0xd4, 0x22, 0x08, 0x6f, 0x64, 0x98, 0xc4, 0x22, 0xeb, 0x8c, 0xfd, 0x8d, 0x8c, 0x82, 0x51, 0x19,
0x07, 0xaa, 0x98, 0xab, 0x3e, 0xb5, 0xcc, 0x55, 0xa3, 0xf2, 0x4b, 0x3b, 0xc4, 0x7b, 0x70, 0xbf,
0x28, 0xc6, 0xb6, 0xa8, 0xeb, 0xa1, 0x23, 0x7c, 0x0c, 0xfb, 0x05, 0xd4, 0x25, 0x1e, 0xa3, 0x46,
0xde, 0xd5, 0x63, 0xb5, 0xd7, 0xe9, 0x7b, 0xbe, 0xd3, 0xf1, 0xbb, 0xa4, 0xeb, 0xb0, 0xa7, 0xa8,
0x85, 0x0f, 0x60, 0xd7, 0x34, 0x7d, 0x46, 0x1e, 0xf7, 0x89, 0xeb, 0xf9, 0x4c, 0x37, 0x08, 0xfa,
0xad, 0x76, 0x6e, 0x03, 0xd0, 0xd9, 0x88, 0x7f, 0xac, 0x26, 0x9f, 0xe3, 0x1d, 0xa8, 0xda, 0x8e,
0x4d, 0xd0, 0x3d, 0xdc, 0x80, 0x9d, 0xbe, 0x4d, 0x5d, 0xb7, 0x4f, 0x4c, 0x54, 0xc2, 0x4d, 0x00,
0x6a, 0xf7, 0x98, 0x73, 0xcd, 0xd4, 0xab, 0x2a, 0x2b, 0xb6, 0x43, 0x6d, 0xea, 0xde, 0x64, 0x4f,
0x04, 0x60, 0x7b, 0xd5, 0x9f, 0xea, 0xf9, 0xaf, 0xd5, 0x6c, 0xbc, 0xb2, 0x29, 0xd1, 0x60, 0x2b,
0xb2, 0x93, 0x19, 0x47, 0xf7, 0x54, 0x49, 0x91, 0x91, 0xf2, 0x40, 0x72, 0x23, 0x89, 0x63, 0x3e,
0x94, 0x61, 0x32, 0x43, 0x23, 0xbc, 0x0f, 0xbb, 0x91, 0x99, 0x26, 0xf3, 0x0d, 0x90, 0xab, 0x9b,
0x89, 0x6e, 0x02, 0xb1, 0x81, 0x3d, 0x53, 0x85, 0x46, 0x26, 0x17, 0xc3, 0x34, 0x1c, 0x6c, 0x2a,
0x8c, 0xd5, 0x1b, 0x8c, 0xdc, 0x49, 0xf2, 0xd1, 0x1a, 0x14, 0x68, 0x92, 0x49, 0x5c, 0x73, 0xe9,
0x2e, 0x85, 0x91, 0xcc, 0x9e, 0x85, 0x63, 0x81, 0x42, 0x7c, 0x08, 0x68, 0x65, 0xa1, 0x17, 0xa4,
0x32, 0xcc, 0xf2, 0x7f, 0x28, 0xe1, 0x7d, 0x68, 0x66, 0x16, 0xd6, 0xe0, 0x8f, 0x6a, 0x3e, 0xee,
0x2b, 0x0b, 0x6b, 0xec, 0xa7, 0x12, 0x3e, 0x06, 0x7c, 0x67, 0x61, 0x4d, 0xfc, 0x5c, 0x52, 0x8d,
0xcd, 0x2c, 0xdc, 0x81, 0x02, 0xfd, 0x52, 0xc2, 0x7b, 0xd0, 0x58, 0x1d, 0x97, 0xf5, 0x17, 0x7d,
0x56, 0xce, 0x8f, 0x5a, 0x29, 0xe4, 0xe0, 0xe7, 0xaa, 0x95, 0xb5, 0x88, 0xce, 0x04, 0x4f, 0x25,
0xfa, 0xa4, 0x92, 0x45, 0x26, 0x8f, 0xb9, 0xe4, 0xe8, 0xd3, 0x0a, 0xae, 0xc3, 0x76, 0xd4, 0x89,
0x17, 0x62, 0x82, 0x5e, 0xe6, 0x94, 0xcb, 0x83, 0x74, 0x38, 0x41, 0xbf, 0x57, 0x32, 0x87, 0x79,
0xc4, 0xb8, 0x58, 0xc4, 0x12, 0xfd, 0x51, 0xc9, 0xf4, 0xaf, 0xb9, 0x5c, 0xdf, 0x27, 0xfa, 0xb3,
0x82, 0x5f, 0x87, 0x96, 0x02, 0xd7, 0xfd, 0x51, 0x4c, 0x28, 0x64, 0x38, 0x14, 0xe8, 0xaf, 0x0a,
0x3e, 0x81, 0x63, 0x45, 0xdf, 0x79, 0xdf, 0x60, 0xff, 0xae, 0xe0, 0x26, 0x68, 0x91, 0x17, 0x4e,
0xb9, 0x17, 0x0e, 0x23, 0xf4, 0x85, 0x96, 0x95, 0xfa, 0x78, 0xc1, 0xd3, 0xa5, 0x9d, 0x8c, 0xb8,
0xda, 0x2a, 0xd0, 0x97, 0x1a, 0xde, 0x05, 0x88, 0xac, 0x24, 0x18, 0xe5, 0x35, 0x7d, 0x95, 0x03,
0x8c, 0x7f, 0xb8, 0xe0, 0x42, 0x52, 0x13, 0x7d, 0xad, 0xe6, 0xb8, 0x5e, 0x00, 0x9e, 0xeb, 0xa0,
0x6f, 0xb4, 0xec, 0x36, 0xf4, 0x38, 0x4e, 0x86, 0x81, 0xe4, 0x2e, 0x1f, 0x4f, 0xf9, 0x4c, 0xa2,
0x6f, 0xb5, 0xac, 0xc9, 0xab, 0x70, 0xc3, 0xc9, 0x77, 0xda, 0xd5, 0xd5, 0x07, 0xef, 0x8d, 0x43,
0x39, 0x59, 0x0c, 0xd4, 0x0f, 0x78, 0xf9, 0x22, 0x8c, 0xe3, 0xf0, 0x85, 0xe4, 0xc3, 0xc9, 0x65,
0xfe, 0x3b, 0x3e, 0x18, 0x85, 0x42, 0xa6, 0xe1, 0x60, 0x21, 0xf9, 0xe8, 0x32, 0x9c, 0x49, 0x9e,
0xce, 0x82, 0xf8, 0x32, 0xfb, 0x32, 0x2f, 0xf3, 0x2f, 0x73, 0x3e, 0x18, 0x6c, 0x67, 0xf1, 0xdb,
0xff, 0x04, 0x00, 0x00, 0xff, 0xff, 0x6f, 0x3f, 0x68, 0x3b, 0x15, 0x07, 0x00, 0x00,
}

View File

@ -162,6 +162,17 @@ message LoadIndex {
repeated common.KeyValuePair index_params = 6;
}
message SegmentStatisticsUpdates {
int64 SegmentID = 1;
int64 MemorySize = 2;
int64 NumRows = 3;
}
message SegmentStatistics {
common.MsgBase base = 1;
repeated SegmentStatisticsUpdates SegStats = 2;
}
message IndexStats {
repeated common.KeyValuePair index_params = 1;
@ -191,4 +202,4 @@ message MsgPosition {
string channel_name = 1;
string msgID = 2;
uint64 timestamp = 3;
}
}

View File

@ -1193,6 +1193,108 @@ func (m *LoadIndex) GetIndexParams() []*commonpb.KeyValuePair {
return nil
}
type SegmentStatisticsUpdates struct {
SegmentID int64 `protobuf:"varint,1,opt,name=SegmentID,proto3" json:"SegmentID,omitempty"`
MemorySize int64 `protobuf:"varint,2,opt,name=MemorySize,proto3" json:"MemorySize,omitempty"`
NumRows int64 `protobuf:"varint,3,opt,name=NumRows,proto3" json:"NumRows,omitempty"`
XXX_NoUnkeyedLiteral struct{} `json:"-"`
XXX_unrecognized []byte `json:"-"`
XXX_sizecache int32 `json:"-"`
}
func (m *SegmentStatisticsUpdates) Reset() { *m = SegmentStatisticsUpdates{} }
func (m *SegmentStatisticsUpdates) String() string { return proto.CompactTextString(m) }
func (*SegmentStatisticsUpdates) ProtoMessage() {}
func (*SegmentStatisticsUpdates) Descriptor() ([]byte, []int) {
return fileDescriptor_41f4a519b878ee3b, []int{17}
}
func (m *SegmentStatisticsUpdates) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_SegmentStatisticsUpdates.Unmarshal(m, b)
}
func (m *SegmentStatisticsUpdates) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
return xxx_messageInfo_SegmentStatisticsUpdates.Marshal(b, m, deterministic)
}
func (m *SegmentStatisticsUpdates) XXX_Merge(src proto.Message) {
xxx_messageInfo_SegmentStatisticsUpdates.Merge(m, src)
}
func (m *SegmentStatisticsUpdates) XXX_Size() int {
return xxx_messageInfo_SegmentStatisticsUpdates.Size(m)
}
func (m *SegmentStatisticsUpdates) XXX_DiscardUnknown() {
xxx_messageInfo_SegmentStatisticsUpdates.DiscardUnknown(m)
}
var xxx_messageInfo_SegmentStatisticsUpdates proto.InternalMessageInfo
func (m *SegmentStatisticsUpdates) GetSegmentID() int64 {
if m != nil {
return m.SegmentID
}
return 0
}
func (m *SegmentStatisticsUpdates) GetMemorySize() int64 {
if m != nil {
return m.MemorySize
}
return 0
}
func (m *SegmentStatisticsUpdates) GetNumRows() int64 {
if m != nil {
return m.NumRows
}
return 0
}
type SegmentStatistics struct {
Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"`
SegStats []*SegmentStatisticsUpdates `protobuf:"bytes,2,rep,name=SegStats,proto3" json:"SegStats,omitempty"`
XXX_NoUnkeyedLiteral struct{} `json:"-"`
XXX_unrecognized []byte `json:"-"`
XXX_sizecache int32 `json:"-"`
}
func (m *SegmentStatistics) Reset() { *m = SegmentStatistics{} }
func (m *SegmentStatistics) String() string { return proto.CompactTextString(m) }
func (*SegmentStatistics) ProtoMessage() {}
func (*SegmentStatistics) Descriptor() ([]byte, []int) {
return fileDescriptor_41f4a519b878ee3b, []int{18}
}
func (m *SegmentStatistics) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_SegmentStatistics.Unmarshal(m, b)
}
func (m *SegmentStatistics) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
return xxx_messageInfo_SegmentStatistics.Marshal(b, m, deterministic)
}
func (m *SegmentStatistics) XXX_Merge(src proto.Message) {
xxx_messageInfo_SegmentStatistics.Merge(m, src)
}
func (m *SegmentStatistics) XXX_Size() int {
return xxx_messageInfo_SegmentStatistics.Size(m)
}
func (m *SegmentStatistics) XXX_DiscardUnknown() {
xxx_messageInfo_SegmentStatistics.DiscardUnknown(m)
}
var xxx_messageInfo_SegmentStatistics proto.InternalMessageInfo
func (m *SegmentStatistics) GetBase() *commonpb.MsgBase {
if m != nil {
return m.Base
}
return nil
}
func (m *SegmentStatistics) GetSegStats() []*SegmentStatisticsUpdates {
if m != nil {
return m.SegStats
}
return nil
}
type IndexStats struct {
IndexParams []*commonpb.KeyValuePair `protobuf:"bytes,1,rep,name=index_params,json=indexParams,proto3" json:"index_params,omitempty"`
NumRelatedSegments int64 `protobuf:"varint,2,opt,name=num_related_segments,json=numRelatedSegments,proto3" json:"num_related_segments,omitempty"`
@ -1205,7 +1307,7 @@ func (m *IndexStats) Reset() { *m = IndexStats{} }
func (m *IndexStats) String() string { return proto.CompactTextString(m) }
func (*IndexStats) ProtoMessage() {}
func (*IndexStats) Descriptor() ([]byte, []int) {
return fileDescriptor_41f4a519b878ee3b, []int{17}
return fileDescriptor_41f4a519b878ee3b, []int{19}
}
func (m *IndexStats) XXX_Unmarshal(b []byte) error {
@ -1253,7 +1355,7 @@ func (m *FieldStats) Reset() { *m = FieldStats{} }
func (m *FieldStats) String() string { return proto.CompactTextString(m) }
func (*FieldStats) ProtoMessage() {}
func (*FieldStats) Descriptor() ([]byte, []int) {
return fileDescriptor_41f4a519b878ee3b, []int{18}
return fileDescriptor_41f4a519b878ee3b, []int{20}
}
func (m *FieldStats) XXX_Unmarshal(b []byte) error {
@ -1309,7 +1411,7 @@ func (m *SegmentStats) Reset() { *m = SegmentStats{} }
func (m *SegmentStats) String() string { return proto.CompactTextString(m) }
func (*SegmentStats) ProtoMessage() {}
func (*SegmentStats) Descriptor() ([]byte, []int) {
return fileDescriptor_41f4a519b878ee3b, []int{19}
return fileDescriptor_41f4a519b878ee3b, []int{21}
}
func (m *SegmentStats) XXX_Unmarshal(b []byte) error {
@ -1371,7 +1473,7 @@ func (m *QueryNodeStats) Reset() { *m = QueryNodeStats{} }
func (m *QueryNodeStats) String() string { return proto.CompactTextString(m) }
func (*QueryNodeStats) ProtoMessage() {}
func (*QueryNodeStats) Descriptor() ([]byte, []int) {
return fileDescriptor_41f4a519b878ee3b, []int{20}
return fileDescriptor_41f4a519b878ee3b, []int{22}
}
func (m *QueryNodeStats) XXX_Unmarshal(b []byte) error {
@ -1426,7 +1528,7 @@ func (m *MsgPosition) Reset() { *m = MsgPosition{} }
func (m *MsgPosition) String() string { return proto.CompactTextString(m) }
func (*MsgPosition) ProtoMessage() {}
func (*MsgPosition) Descriptor() ([]byte, []int) {
return fileDescriptor_41f4a519b878ee3b, []int{21}
return fileDescriptor_41f4a519b878ee3b, []int{23}
}
func (m *MsgPosition) XXX_Unmarshal(b []byte) error {
@ -1487,6 +1589,8 @@ func init() {
proto.RegisterType((*DeleteRequest)(nil), "milvus.proto.internal.DeleteRequest")
proto.RegisterType((*FlushMsg)(nil), "milvus.proto.internal.FlushMsg")
proto.RegisterType((*LoadIndex)(nil), "milvus.proto.internal.LoadIndex")
proto.RegisterType((*SegmentStatisticsUpdates)(nil), "milvus.proto.internal.SegmentStatisticsUpdates")
proto.RegisterType((*SegmentStatistics)(nil), "milvus.proto.internal.SegmentStatistics")
proto.RegisterType((*IndexStats)(nil), "milvus.proto.internal.IndexStats")
proto.RegisterType((*FieldStats)(nil), "milvus.proto.internal.FieldStats")
proto.RegisterType((*SegmentStats)(nil), "milvus.proto.internal.SegmentStats")
@ -1497,85 +1601,89 @@ func init() {
func init() { proto.RegisterFile("internal.proto", fileDescriptor_41f4a519b878ee3b) }
var fileDescriptor_41f4a519b878ee3b = []byte{
// 1275 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xec, 0x57, 0xcd, 0x6f, 0x1b, 0x45,
0x14, 0x67, 0x6d, 0xc7, 0x1f, 0xcf, 0x4e, 0x1a, 0x96, 0x7e, 0x6c, 0xa1, 0x50, 0x77, 0x29, 0x60,
0x40, 0x24, 0x55, 0x8a, 0x10, 0xe2, 0xd2, 0x26, 0x71, 0x4b, 0x57, 0x4d, 0x42, 0x98, 0x98, 0x4a,
0xf4, 0xb2, 0x1a, 0x7b, 0x27, 0xf6, 0xd0, 0xdd, 0x1d, 0x77, 0x66, 0xdc, 0xd4, 0x39, 0x73, 0x43,
0x70, 0xe3, 0xc8, 0x05, 0xf1, 0x77, 0x80, 0xc4, 0x09, 0x89, 0x2b, 0xe2, 0xc4, 0x5f, 0xc2, 0x09,
0xcd, 0xc7, 0xfa, 0x23, 0xdd, 0x46, 0x69, 0x00, 0xa1, 0x4a, 0xdc, 0x76, 0x7e, 0xf3, 0xf6, 0xed,
0xfb, 0xfd, 0xde, 0x9b, 0xdf, 0xee, 0xc2, 0x12, 0x4d, 0x25, 0xe1, 0x29, 0x8e, 0x57, 0x86, 0x9c,
0x49, 0xe6, 0x9e, 0x4b, 0x68, 0xfc, 0x68, 0x24, 0xcc, 0x6a, 0x25, 0xdb, 0x7c, 0xb9, 0xd1, 0x63,
0x49, 0xc2, 0x52, 0x03, 0xfb, 0x3f, 0x3a, 0xb0, 0xb8, 0xc9, 0x92, 0x21, 0x4b, 0x49, 0x2a, 0x83,
0x74, 0x9f, 0xb9, 0xe7, 0xa1, 0x9c, 0xb2, 0x88, 0x04, 0x6d, 0xcf, 0x69, 0x3a, 0xad, 0x22, 0xb2,
0x2b, 0xd7, 0x85, 0x12, 0x67, 0x31, 0xf1, 0x0a, 0x4d, 0xa7, 0x55, 0x43, 0xfa, 0xda, 0xbd, 0x01,
0x20, 0x24, 0x96, 0x24, 0xec, 0xb1, 0x88, 0x78, 0xc5, 0xa6, 0xd3, 0x5a, 0x5a, 0x6b, 0xae, 0xe4,
0x3e, 0x77, 0x65, 0x4f, 0x05, 0x6e, 0xb2, 0x88, 0xa0, 0x9a, 0xc8, 0x2e, 0xdd, 0x9b, 0x00, 0xe4,
0xb1, 0xe4, 0x38, 0xa4, 0xe9, 0x3e, 0xf3, 0x4a, 0xcd, 0x62, 0xab, 0xbe, 0x76, 0x65, 0x3e, 0x81,
0x2d, 0xf7, 0x2e, 0x19, 0xdf, 0xc3, 0xf1, 0x88, 0xec, 0x62, 0xca, 0x51, 0x4d, 0xdf, 0xa4, 0xca,
0xf5, 0x7f, 0x70, 0xe0, 0xcc, 0x84, 0x80, 0x7e, 0x86, 0x70, 0x3f, 0x82, 0x05, 0xfd, 0x08, 0xcd,
0xa0, 0xbe, 0x76, 0xf5, 0x29, 0x15, 0xcd, 0xf1, 0x46, 0xe6, 0x16, 0xf7, 0x33, 0x78, 0x49, 0x8c,
0xba, 0xbd, 0x6c, 0x2b, 0xd4, 0xa8, 0xf0, 0x0a, 0xba, 0xb4, 0x93, 0x65, 0x72, 0x67, 0x13, 0x98,
0x92, 0xfc, 0x7b, 0x50, 0xdd, 0x51, 0x3a, 0x2a, 0x85, 0x3f, 0x80, 0x0a, 0x8e, 0x22, 0x4e, 0x84,
0xb0, 0x05, 0x5e, 0xca, 0x65, 0xbc, 0x6e, 0x62, 0x50, 0x16, 0x9c, 0xd7, 0x01, 0xff, 0x0b, 0x80,
0x20, 0xa5, 0x72, 0x17, 0x73, 0x9c, 0x88, 0xa7, 0xf6, 0xae, 0x0d, 0x0d, 0x21, 0x31, 0x97, 0xe1,
0x50, 0xc7, 0x59, 0x36, 0x27, 0x10, 0xba, 0xae, 0x6f, 0x33, 0xd9, 0xfd, 0xab, 0x00, 0x7b, 0x92,
0xd3, 0xb4, 0xbf, 0x45, 0x85, 0x54, 0xcf, 0x7a, 0xa4, 0xe2, 0x4c, 0xb6, 0x1a, 0xb2, 0x2b, 0xff,
0x06, 0xd4, 0x3b, 0x34, 0x21, 0x1d, 0xda, 0x7b, 0xb0, 0x2d, 0xfa, 0xee, 0x35, 0x28, 0x75, 0xb1,
0x20, 0xc7, 0x32, 0xdd, 0x16, 0xfd, 0x0d, 0x2c, 0x08, 0xd2, 0x91, 0xfe, 0x1f, 0x0e, 0x5c, 0xd8,
0xe4, 0x44, 0x8f, 0x48, 0x1c, 0x93, 0x9e, 0xa4, 0x2c, 0x45, 0xe4, 0xe1, 0x88, 0x08, 0xf9, 0xec,
0xd9, 0xdc, 0x0b, 0x50, 0x89, 0xba, 0x61, 0x8a, 0x93, 0x4c, 0xb7, 0x72, 0xd4, 0xdd, 0xc1, 0x09,
0x71, 0xdf, 0x84, 0xa5, 0xde, 0x24, 0xbf, 0x42, 0xf4, 0xfc, 0xd6, 0xd0, 0x11, 0x54, 0xa9, 0x1e,
0x75, 0x83, 0xb6, 0x57, 0xd2, 0x8a, 0xea, 0x6b, 0xd7, 0x87, 0xc6, 0x34, 0x2a, 0x68, 0x7b, 0x0b,
0x7a, 0x6f, 0x0e, 0x53, 0xfa, 0x88, 0xde, 0x80, 0x24, 0xd8, 0x2b, 0x37, 0x9d, 0x56, 0x03, 0xd9,
0x95, 0xff, 0xb3, 0x03, 0xe7, 0xda, 0x9c, 0x0d, 0x9f, 0x67, 0x72, 0xfe, 0xd7, 0x05, 0x38, 0x6f,
0x7a, 0xb4, 0x8b, 0xb9, 0xa4, 0xff, 0x12, 0x8b, 0xb7, 0xe0, 0xcc, 0xf4, 0xa9, 0x26, 0x20, 0x9f,
0xc6, 0x1b, 0xb0, 0x34, 0xcc, 0xea, 0x30, 0x71, 0x25, 0x1d, 0xb7, 0x38, 0x41, 0xe7, 0xd8, 0x2e,
0x1c, 0xc3, 0xb6, 0x9c, 0xd3, 0xca, 0x26, 0xd4, 0x27, 0x89, 0x82, 0xb6, 0x57, 0xd1, 0x21, 0xb3,
0x90, 0xff, 0x55, 0x01, 0xce, 0xaa, 0xa6, 0xfe, 0xaf, 0x86, 0x52, 0xe3, 0xa7, 0x02, 0xb8, 0x66,
0x3a, 0x82, 0x34, 0x22, 0x8f, 0xff, 0x4b, 0x2d, 0x5e, 0x05, 0xd8, 0xa7, 0x24, 0x8e, 0x66, 0x75,
0xa8, 0x69, 0xe4, 0x6f, 0x69, 0xe0, 0x41, 0x45, 0x27, 0x99, 0xf0, 0xcf, 0x96, 0xca, 0x6a, 0xcd,
0x1b, 0xcd, 0x5a, 0x6d, 0xf5, 0xc4, 0x56, 0xab, 0x6f, 0xb3, 0x56, 0xfb, 0x5b, 0x01, 0x16, 0x83,
0x54, 0x10, 0x2e, 0x9f, 0x83, 0x41, 0xba, 0x04, 0x35, 0x41, 0xfa, 0x89, 0x7a, 0xfd, 0x65, 0x4a,
0x4e, 0x01, 0xb5, 0xdb, 0x1b, 0xe0, 0x34, 0x25, 0xb1, 0xd5, 0xb2, 0x86, 0xa6, 0x80, 0xfb, 0x1a,
0x80, 0xa4, 0x09, 0x11, 0x12, 0x27, 0x43, 0xe1, 0x55, 0x9a, 0xc5, 0x56, 0x09, 0xcd, 0x20, 0xca,
0x45, 0x39, 0x3b, 0x08, 0xda, 0x46, 0xc8, 0x22, 0xb2, 0x2b, 0xf7, 0x7d, 0xa8, 0x72, 0x76, 0x10,
0x46, 0x58, 0x62, 0xaf, 0xa6, 0x25, 0xbe, 0x98, 0x2b, 0xc9, 0x46, 0xcc, 0xba, 0xa8, 0xc2, 0xd9,
0x41, 0x1b, 0x4b, 0xec, 0x7f, 0xe7, 0xc0, 0xe2, 0x1e, 0xc1, 0xbc, 0x37, 0x38, 0xbd, 0xac, 0x6f,
0xc3, 0x32, 0x27, 0x62, 0x14, 0xcb, 0x70, 0x4a, 0xcb, 0xe8, 0x7b, 0xc6, 0xe0, 0x9b, 0x13, 0x72,
0xab, 0xb0, 0xf0, 0x70, 0x44, 0xf8, 0x58, 0xcb, 0x7b, 0x6c, 0x85, 0x26, 0xce, 0xff, 0x7d, 0xa6,
0x3e, 0x95, 0x4a, 0x9c, 0xa2, 0xbe, 0xeb, 0x50, 0x56, 0xdf, 0x2c, 0x23, 0xa1, 0xab, 0xaa, 0xaf,
0xbd, 0x92, 0x7b, 0xcf, 0x9e, 0x0e, 0x41, 0x36, 0x34, 0x97, 0x54, 0x31, 0x9f, 0xd4, 0x65, 0xa8,
0x27, 0x44, 0x72, 0xda, 0x0b, 0xe5, 0x78, 0x98, 0x4d, 0x04, 0x18, 0xa8, 0x33, 0x1e, 0xea, 0x33,
0x35, 0xa0, 0x52, 0x78, 0x0b, 0xcd, 0x62, 0xab, 0x81, 0xf4, 0xb5, 0xff, 0xab, 0x03, 0x8b, 0x6d,
0x12, 0x13, 0x49, 0x4e, 0x2f, 0x7c, 0xce, 0xd8, 0x16, 0x72, 0xc7, 0x76, 0x6e, 0xe2, 0x8a, 0xc7,
0x4f, 0x5c, 0xe9, 0x89, 0x89, 0xbb, 0x02, 0x8d, 0x21, 0xa7, 0x09, 0xe6, 0xe3, 0xf0, 0x01, 0x19,
0x1b, 0x1a, 0xca, 0xdf, 0x0c, 0x76, 0x97, 0x8c, 0x85, 0xff, 0xbd, 0x03, 0xd5, 0xdb, 0xf1, 0x48,
0x0c, 0x4e, 0xf5, 0x81, 0x33, 0x7f, 0x5e, 0x0a, 0x47, 0xcf, 0xcb, 0x51, 0xfb, 0x29, 0xe6, 0xd8,
0x8f, 0x0f, 0x8d, 0xc9, 0x11, 0xec, 0xe0, 0xbe, 0x6d, 0xc2, 0x1c, 0xe6, 0xff, 0xe9, 0x40, 0x6d,
0x8b, 0xe1, 0x48, 0x5b, 0xf0, 0x3f, 0x5e, 0xe5, 0x25, 0x98, 0xba, 0x68, 0xa6, 0xf1, 0xd4, 0x56,
0x67, 0xec, 0xb1, 0x34, 0x6f, 0x8f, 0x97, 0xa1, 0x4e, 0x55, 0x41, 0xe1, 0x10, 0xcb, 0x81, 0x11,
0xb7, 0x86, 0x40, 0x43, 0xbb, 0x0a, 0x51, 0xfe, 0x99, 0x05, 0x68, 0xff, 0x2c, 0x9f, 0xd8, 0x3f,
0x6d, 0x12, 0xed, 0x9f, 0x5f, 0x3a, 0xea, 0xbb, 0x38, 0x22, 0x8f, 0xd5, 0x9c, 0x3f, 0x99, 0xd4,
0x39, 0x4d, 0x52, 0xf7, 0x1a, 0x9c, 0x4d, 0x47, 0x49, 0xc8, 0x49, 0x8c, 0x25, 0x89, 0x42, 0x2b,
0x86, 0xb0, 0xe2, 0xb8, 0xe9, 0x28, 0x41, 0x66, 0x6b, 0xcf, 0xee, 0xf8, 0xdf, 0x38, 0x00, 0xb7,
0x15, 0x73, 0x53, 0xc6, 0xd1, 0xd6, 0x3a, 0xc7, 0xbf, 0x59, 0x0a, 0xf3, 0xd2, 0x6d, 0x64, 0xd2,
0xa9, 0x33, 0x2b, 0xbc, 0x62, 0x1e, 0x87, 0xc9, 0x1f, 0xc9, 0x94, 0xbc, 0x55, 0x57, 0x5f, 0xfb,
0xdf, 0x3a, 0xd0, 0xb0, 0xd5, 0x99, 0x92, 0xe6, 0xba, 0xec, 0x1c, 0xed, 0xb2, 0x3e, 0xeb, 0x09,
0xe3, 0xe3, 0x50, 0xd0, 0x43, 0x62, 0x0b, 0x02, 0x03, 0xed, 0xd1, 0x43, 0xe2, 0x5e, 0x84, 0xaa,
0x96, 0x84, 0x1d, 0x08, 0x3b, 0xa8, 0x15, 0x25, 0x03, 0x3b, 0x10, 0xee, 0xbb, 0xf0, 0x22, 0x27,
0x3d, 0x92, 0xca, 0x78, 0x1c, 0x26, 0x2c, 0xa2, 0xfb, 0x94, 0x44, 0x7a, 0x1a, 0xaa, 0x68, 0x39,
0xdb, 0xd8, 0xb6, 0xb8, 0xff, 0x8b, 0x03, 0x4b, 0x9f, 0x2a, 0x0b, 0x54, 0x3f, 0x49, 0xa6, 0xb2,
0x67, 0x9f, 0xd8, 0x9b, 0x9a, 0x8b, 0x95, 0xc7, 0xfc, 0xe2, 0xbc, 0xfe, 0xb4, 0x9f, 0xd1, 0x19,
0x0d, 0x50, 0x55, 0x90, 0xbe, 0x79, 0xe6, 0x06, 0xd4, 0xcd, 0xd7, 0xc2, 0x49, 0x24, 0x9e, 0x36,
0x16, 0x99, 0x6f, 0x0c, 0x23, 0x71, 0x04, 0xf5, 0x6d, 0xd1, 0xdf, 0x65, 0x42, 0x9f, 0x44, 0x65,
0x27, 0xd6, 0x7b, 0x8c, 0x65, 0x39, 0xfa, 0xac, 0xd4, 0x2d, 0xa6, 0x4f, 0xcb, 0x59, 0x58, 0x48,
0x44, 0x7f, 0xf2, 0x1a, 0x31, 0x0b, 0xd5, 0x99, 0x89, 0x2b, 0x69, 0x6d, 0x4b, 0x68, 0x0a, 0xbc,
0xf3, 0x21, 0xd4, 0x26, 0x3f, 0xd4, 0xee, 0x32, 0x34, 0x82, 0x9d, 0xa0, 0x13, 0xac, 0x6f, 0x05,
0xf7, 0x83, 0x9d, 0x8f, 0x97, 0x5f, 0x70, 0xeb, 0x50, 0xb9, 0x73, 0x6b, 0x7d, 0xab, 0x73, 0xe7,
0xf3, 0x65, 0xc7, 0x6d, 0x40, 0x75, 0x7d, 0x63, 0xe7, 0x13, 0xb4, 0xbd, 0xbe, 0xb5, 0x5c, 0xd8,
0xb8, 0x75, 0x7f, 0xb3, 0x4f, 0xe5, 0x60, 0xd4, 0x55, 0x22, 0xae, 0x1e, 0xd2, 0x38, 0xa6, 0x87,
0x92, 0xf4, 0x06, 0xab, 0x86, 0xe5, 0x7b, 0x11, 0x15, 0x92, 0xd3, 0xee, 0x48, 0x92, 0x68, 0x35,
0xe3, 0xba, 0xaa, 0xa9, 0x4f, 0x96, 0xc3, 0xee, 0x5a, 0xb7, 0xac, 0xa1, 0xeb, 0x7f, 0x05, 0x00,
0x00, 0xff, 0xff, 0xa4, 0x3b, 0x4f, 0xdd, 0x76, 0x10, 0x00, 0x00,
// 1340 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xec, 0x58, 0xcf, 0x6f, 0xdc, 0x44,
0x18, 0xc5, 0xbb, 0x9b, 0xfd, 0xf1, 0xed, 0x26, 0x4d, 0x4d, 0x7f, 0xb8, 0x10, 0xe8, 0xd6, 0x14,
0x58, 0x40, 0x24, 0x55, 0x8a, 0x10, 0xe2, 0xd2, 0x26, 0xd9, 0x96, 0x5a, 0x4d, 0x42, 0x98, 0x4d,
0x2b, 0xd1, 0x8b, 0x35, 0x6b, 0x4f, 0x76, 0x87, 0xfa, 0xc7, 0x76, 0x66, 0xdc, 0x74, 0x7b, 0xe6,
0x86, 0xe0, 0x80, 0xc4, 0x91, 0x0b, 0xe2, 0xef, 0x00, 0x89, 0x13, 0x12, 0x57, 0xc4, 0x89, 0xbf,
0x84, 0x13, 0x9a, 0x19, 0xdb, 0xbb, 0x9b, 0x38, 0x21, 0x0d, 0x20, 0x54, 0x89, 0xdb, 0xce, 0xf3,
0xe7, 0xcf, 0xdf, 0x7b, 0xf3, 0xfc, 0x6c, 0x2f, 0x2c, 0xd0, 0x48, 0x10, 0x16, 0xe1, 0x60, 0x79,
0xc4, 0x62, 0x11, 0x9b, 0xe7, 0x43, 0x1a, 0x3c, 0x4e, 0xb8, 0x5e, 0x2d, 0x67, 0x07, 0x5f, 0x6a,
0x79, 0x71, 0x18, 0xc6, 0x91, 0x86, 0xed, 0x1f, 0x0c, 0x98, 0xdf, 0x88, 0xc3, 0x51, 0x1c, 0x91,
0x48, 0x38, 0xd1, 0x5e, 0x6c, 0x5e, 0x80, 0x6a, 0x14, 0xfb, 0xc4, 0xe9, 0x5a, 0x46, 0xdb, 0xe8,
0x94, 0x51, 0xba, 0x32, 0x4d, 0xa8, 0xb0, 0x38, 0x20, 0x56, 0xa9, 0x6d, 0x74, 0x1a, 0x48, 0xfd,
0x36, 0x6f, 0x00, 0x70, 0x81, 0x05, 0x71, 0xbd, 0xd8, 0x27, 0x56, 0xb9, 0x6d, 0x74, 0x16, 0x56,
0xdb, 0xcb, 0x85, 0xd7, 0x5d, 0xee, 0xc9, 0xc2, 0x8d, 0xd8, 0x27, 0xa8, 0xc1, 0xb3, 0x9f, 0xe6,
0x4d, 0x00, 0xf2, 0x44, 0x30, 0xec, 0xd2, 0x68, 0x2f, 0xb6, 0x2a, 0xed, 0x72, 0xa7, 0xb9, 0x7a,
0x65, 0xb6, 0x41, 0x3a, 0xee, 0x5d, 0x32, 0xbe, 0x8f, 0x83, 0x84, 0xec, 0x60, 0xca, 0x50, 0x43,
0x9d, 0x24, 0xc7, 0xb5, 0xbf, 0x37, 0xe0, 0x4c, 0x4e, 0x40, 0x5d, 0x83, 0x9b, 0x1f, 0xc2, 0x9c,
0xba, 0x84, 0x62, 0xd0, 0x5c, 0xbd, 0x7a, 0xc4, 0x44, 0x33, 0xbc, 0x91, 0x3e, 0xc5, 0xbc, 0x07,
0x2f, 0xf2, 0xa4, 0xef, 0x65, 0x87, 0x5c, 0x85, 0x72, 0xab, 0xa4, 0x46, 0x3b, 0x59, 0x27, 0x73,
0xba, 0x81, 0x1e, 0xc9, 0xbe, 0x0f, 0xf5, 0x6d, 0xa9, 0xa3, 0x54, 0xf8, 0x7d, 0xa8, 0x61, 0xdf,
0x67, 0x84, 0xf3, 0x74, 0xc0, 0xa5, 0x42, 0xc6, 0x6b, 0xba, 0x06, 0x65, 0xc5, 0x45, 0x3b, 0x60,
0x7f, 0x06, 0xe0, 0x44, 0x54, 0xec, 0x60, 0x86, 0x43, 0x7e, 0xe4, 0xde, 0x75, 0xa1, 0xc5, 0x05,
0x66, 0xc2, 0x1d, 0xa9, 0xba, 0x94, 0xcd, 0x09, 0x84, 0x6e, 0xaa, 0xd3, 0x74, 0x77, 0xfb, 0x2a,
0x40, 0x4f, 0x30, 0x1a, 0x0d, 0x36, 0x29, 0x17, 0xf2, 0x5a, 0x8f, 0x65, 0x9d, 0xee, 0xd6, 0x40,
0xe9, 0xca, 0xbe, 0x01, 0xcd, 0x5d, 0x1a, 0x92, 0x5d, 0xea, 0x3d, 0xdc, 0xe2, 0x03, 0xf3, 0x1a,
0x54, 0xfa, 0x98, 0x93, 0x63, 0x99, 0x6e, 0xf1, 0xc1, 0x3a, 0xe6, 0x04, 0xa9, 0x4a, 0xfb, 0x77,
0x03, 0x2e, 0x6e, 0x30, 0xa2, 0x2c, 0x12, 0x04, 0xc4, 0x13, 0x34, 0x8e, 0x10, 0x79, 0x94, 0x10,
0x2e, 0x9e, 0xbd, 0x9b, 0x79, 0x11, 0x6a, 0x7e, 0xdf, 0x8d, 0x70, 0x98, 0xe9, 0x56, 0xf5, 0xfb,
0xdb, 0x38, 0x24, 0xe6, 0x1b, 0xb0, 0xe0, 0xe5, 0xfd, 0x25, 0xa2, 0xfc, 0xdb, 0x40, 0x07, 0x50,
0xa9, 0xba, 0xdf, 0x77, 0xba, 0x56, 0x45, 0x29, 0xaa, 0x7e, 0x9b, 0x36, 0xb4, 0x26, 0x55, 0x4e,
0xd7, 0x9a, 0x53, 0xc7, 0x66, 0x30, 0xa9, 0x0f, 0xf7, 0x86, 0x24, 0xc4, 0x56, 0xb5, 0x6d, 0x74,
0x5a, 0x28, 0x5d, 0xd9, 0x3f, 0x19, 0x70, 0xbe, 0xcb, 0xe2, 0xd1, 0xf3, 0x4c, 0xce, 0xfe, 0xb2,
0x04, 0x17, 0xf4, 0x1e, 0xed, 0x60, 0x26, 0xe8, 0xbf, 0xc4, 0xe2, 0x4d, 0x38, 0x33, 0xb9, 0xaa,
0x2e, 0x28, 0xa6, 0xf1, 0x3a, 0x2c, 0x8c, 0xb2, 0x39, 0x74, 0x5d, 0x45, 0xd5, 0xcd, 0xe7, 0xe8,
0x0c, 0xdb, 0xb9, 0x63, 0xd8, 0x56, 0x0b, 0xb6, 0xb2, 0x0d, 0xcd, 0xbc, 0x91, 0xd3, 0xb5, 0x6a,
0xaa, 0x64, 0x1a, 0xb2, 0xbf, 0x28, 0xc1, 0x39, 0xb9, 0xa9, 0xff, 0xab, 0x21, 0xd5, 0xf8, 0xb1,
0x04, 0xa6, 0x76, 0x87, 0x13, 0xf9, 0xe4, 0xc9, 0x7f, 0xa9, 0xc5, 0x2b, 0x00, 0x7b, 0x94, 0x04,
0xfe, 0xb4, 0x0e, 0x0d, 0x85, 0xfc, 0x2d, 0x0d, 0x2c, 0xa8, 0xa9, 0x26, 0x39, 0xff, 0x6c, 0x29,
0xa3, 0x56, 0x3f, 0xd1, 0xd2, 0xa8, 0xad, 0x9f, 0x38, 0x6a, 0xd5, 0x69, 0x69, 0xd4, 0xfe, 0x5a,
0x82, 0x79, 0x27, 0xe2, 0x84, 0x89, 0xe7, 0xc0, 0x48, 0x4b, 0xd0, 0xe0, 0x64, 0x10, 0xca, 0xc7,
0x5f, 0xa6, 0xe4, 0x04, 0x90, 0x47, 0xbd, 0x21, 0x8e, 0x22, 0x12, 0xa4, 0x5a, 0x36, 0xd0, 0x04,
0x30, 0x5f, 0x05, 0x10, 0x34, 0x24, 0x5c, 0xe0, 0x70, 0xc4, 0xad, 0x5a, 0xbb, 0xdc, 0xa9, 0xa0,
0x29, 0x44, 0xa6, 0x28, 0x8b, 0xf7, 0x9d, 0xae, 0x16, 0xb2, 0x8c, 0xd2, 0x95, 0xf9, 0x1e, 0xd4,
0x59, 0xbc, 0xef, 0xfa, 0x58, 0x60, 0xab, 0xa1, 0x24, 0xbe, 0x54, 0x28, 0xc9, 0x7a, 0x10, 0xf7,
0x51, 0x8d, 0xc5, 0xfb, 0x5d, 0x2c, 0xb0, 0xfd, 0xad, 0x01, 0xf3, 0x3d, 0x82, 0x99, 0x37, 0x3c,
0xbd, 0xac, 0x6f, 0xc1, 0x22, 0x23, 0x3c, 0x09, 0x84, 0x3b, 0xa1, 0xa5, 0xf5, 0x3d, 0xa3, 0xf1,
0x8d, 0x9c, 0xdc, 0x0a, 0xcc, 0x3d, 0x4a, 0x08, 0x1b, 0x2b, 0x79, 0x8f, 0x9d, 0x50, 0xd7, 0xd9,
0xbf, 0x4d, 0xcd, 0x27, 0x5b, 0xf1, 0x53, 0xcc, 0x77, 0x1d, 0xaa, 0xf2, 0x9d, 0x25, 0xe1, 0x6a,
0xaa, 0xe6, 0xea, 0xcb, 0x85, 0xe7, 0xf4, 0x54, 0x09, 0x4a, 0x4b, 0x0b, 0x49, 0x95, 0x8b, 0x49,
0x5d, 0x86, 0x66, 0x48, 0x04, 0xa3, 0x9e, 0x2b, 0xc6, 0xa3, 0xcc, 0x11, 0xa0, 0xa1, 0xdd, 0xf1,
0x48, 0xdd, 0x53, 0x43, 0x2a, 0xb8, 0x35, 0xd7, 0x2e, 0x77, 0x5a, 0x48, 0xfd, 0xb6, 0x7f, 0x31,
0x60, 0xbe, 0x4b, 0x02, 0x22, 0xc8, 0xe9, 0x85, 0x2f, 0xb0, 0x6d, 0xa9, 0xd0, 0xb6, 0x33, 0x8e,
0x2b, 0x1f, 0xef, 0xb8, 0xca, 0x21, 0xc7, 0x5d, 0x81, 0xd6, 0x88, 0xd1, 0x10, 0xb3, 0xb1, 0xfb,
0x90, 0x8c, 0x35, 0x0d, 0x99, 0x6f, 0x1a, 0xbb, 0x4b, 0xc6, 0xdc, 0xfe, 0xce, 0x80, 0xfa, 0xed,
0x20, 0xe1, 0xc3, 0x53, 0xbd, 0xe0, 0xcc, 0xde, 0x2f, 0xa5, 0x83, 0xf7, 0xcb, 0xc1, 0xf8, 0x29,
0x17, 0xc4, 0x8f, 0x0d, 0xad, 0xfc, 0x16, 0xdc, 0xc5, 0x83, 0x74, 0x13, 0x66, 0x30, 0xfb, 0x0f,
0x03, 0x1a, 0x9b, 0x31, 0xf6, 0x55, 0x04, 0xff, 0xe3, 0x53, 0x2e, 0xc1, 0x24, 0x45, 0x33, 0x8d,
0x27, 0xb1, 0x3a, 0x15, 0x8f, 0x95, 0xd9, 0x78, 0xbc, 0x0c, 0x4d, 0x2a, 0x07, 0x72, 0x47, 0x58,
0x0c, 0xb5, 0xb8, 0x0d, 0x04, 0x0a, 0xda, 0x91, 0x88, 0xcc, 0xcf, 0xac, 0x40, 0xe5, 0x67, 0xf5,
0xc4, 0xf9, 0x99, 0x36, 0x51, 0xf9, 0xc9, 0xc0, 0xea, 0xe9, 0x59, 0xa5, 0xd1, 0x29, 0x17, 0xd4,
0xe3, 0xf7, 0x46, 0xbe, 0xfa, 0x3a, 0x58, 0x82, 0x46, 0x2f, 0x27, 0xa6, 0xdf, 0x93, 0x27, 0x80,
0xb4, 0xc7, 0x16, 0x09, 0x63, 0x36, 0xee, 0xd1, 0xa7, 0x24, 0xe5, 0x3d, 0x85, 0x48, 0x6a, 0xdb,
0x49, 0x88, 0xe2, 0x7d, 0x9e, 0xee, 0x4c, 0xb6, 0xb4, 0xbf, 0x36, 0xe0, 0xec, 0xa1, 0x8b, 0x9e,
0x42, 0xf8, 0xbb, 0x50, 0xef, 0x91, 0x81, 0x6c, 0x91, 0xbd, 0xa8, 0xaf, 0x1c, 0xf5, 0x49, 0x75,
0x04, 0x45, 0x94, 0x37, 0xb0, 0x3f, 0x37, 0xe4, 0x07, 0x82, 0x4f, 0x9e, 0xa8, 0xe5, 0x21, 0x75,
0x8d, 0xd3, 0xa8, 0x6b, 0x5e, 0x83, 0x73, 0x51, 0x12, 0xba, 0x8c, 0x04, 0x58, 0x10, 0xdf, 0x4d,
0x5d, 0xc1, 0x53, 0xb5, 0xcc, 0x28, 0x09, 0x91, 0x3e, 0x94, 0x0e, 0xc8, 0xed, 0xaf, 0x0c, 0x80,
0xdb, 0xd2, 0x02, 0x7a, 0x8c, 0x83, 0x1e, 0x37, 0x8e, 0x7f, 0xc4, 0x96, 0x66, 0x3d, 0xb4, 0x9e,
0x79, 0x88, 0x2b, 0x8d, 0xca, 0x45, 0x1c, 0x72, 0x8d, 0x26, 0xe4, 0x53, 0x9b, 0x69, 0x5d, 0xbe,
0x31, 0xa0, 0x35, 0x25, 0x1f, 0x9f, 0xb5, 0xbb, 0x71, 0xd0, 0xee, 0x2a, 0xf4, 0xa4, 0x07, 0x5c,
0x3e, 0x65, 0x8b, 0x70, 0x62, 0x8b, 0x4b, 0x50, 0x57, 0x92, 0x4c, 0xf9, 0x22, 0xd2, 0xbe, 0x30,
0xdf, 0x81, 0xb3, 0x8c, 0x78, 0x24, 0x12, 0xc1, 0xd8, 0x0d, 0x63, 0x9f, 0xee, 0x51, 0xe2, 0xab,
0xdb, 0xa2, 0x8e, 0x16, 0xb3, 0x03, 0x5b, 0x29, 0x6e, 0xff, 0x6c, 0xc0, 0xc2, 0x27, 0xf2, 0x59,
0x20, 0xbf, 0x16, 0xf5, 0x64, 0xcf, 0xee, 0xa0, 0x9b, 0x8a, 0x4b, 0x2a, 0x8f, 0xb6, 0xd0, 0x6b,
0x7f, 0x6d, 0x21, 0x8e, 0xea, 0x3c, 0xb5, 0x8d, 0x94, 0x58, 0xbf, 0x36, 0x9d, 0x44, 0xe2, 0xc9,
0xc6, 0x22, 0xfd, 0xb2, 0xa5, 0x25, 0xf6, 0xa1, 0xb9, 0xc5, 0x07, 0x3b, 0x31, 0x57, 0x91, 0x24,
0x73, 0x35, 0x0d, 0x61, 0x9d, 0xdd, 0x86, 0x0a, 0x8d, 0x66, 0x8a, 0xa9, 0xd8, 0x38, 0x07, 0x73,
0x21, 0x1f, 0xe4, 0xcf, 0x53, 0xbd, 0x90, 0x3b, 0x93, 0xc7, 0xb3, 0xd2, 0xb6, 0x82, 0x26, 0xc0,
0xdb, 0x1f, 0x40, 0x23, 0xff, 0x67, 0xc1, 0x5c, 0x84, 0x96, 0xb3, 0xed, 0xec, 0x3a, 0x6b, 0x9b,
0xce, 0x03, 0x67, 0xfb, 0xa3, 0xc5, 0x17, 0xcc, 0x26, 0xd4, 0xee, 0xdc, 0x5a, 0xdb, 0xdc, 0xbd,
0xf3, 0xe9, 0xa2, 0x61, 0xb6, 0xa0, 0xbe, 0xb6, 0xbe, 0xfd, 0x31, 0xda, 0x5a, 0xdb, 0x5c, 0x2c,
0xad, 0xdf, 0x7a, 0xb0, 0x31, 0xa0, 0x62, 0x98, 0xf4, 0xa5, 0x88, 0x2b, 0x4f, 0x69, 0x10, 0xd0,
0xa7, 0x82, 0x78, 0xc3, 0x15, 0xcd, 0xf2, 0x5d, 0x9f, 0x72, 0xc1, 0x68, 0x3f, 0x11, 0xc4, 0x5f,
0xc9, 0xb8, 0xae, 0x28, 0xea, 0xf9, 0x72, 0xd4, 0x5f, 0xed, 0x57, 0x15, 0x74, 0xfd, 0xcf, 0x00,
0x00, 0x00, 0xff, 0xff, 0xff, 0xf9, 0xa9, 0x8f, 0x7f, 0x11, 0x00, 0x00,
}