mirror of https://github.com/milvus-io/milvus.git
Add segment statistics updates logic for datanode
Signed-off-by: XuanYang-cn <xuan.yang@zilliz.com>pull/4973/head^2
parent
7619d968fd
commit
ec27788628
|
@ -34,7 +34,7 @@ dataNode:
|
|||
|
||||
segStatistics:
|
||||
recvBufSize: 64
|
||||
publishInterval: 1000 # milliseconds
|
||||
updateInterval: 1000 # milliseconds
|
||||
|
||||
flush:
|
||||
# max buffer size to flush
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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)
|
||||
|
||||
}
|
||||
|
|
|
@ -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()
|
||||
}
|
||||
|
|
|
@ -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)
|
||||
}
|
||||
|
|
|
@ -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()
|
||||
|
|
|
@ -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,
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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 -
|
||||
|
|
|
@ -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)
|
||||
})
|
||||
|
||||
}
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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,
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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,
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue