Add ddl binlog metaTable in writenode

Signed-off-by: XuanYang-cn <xuan.yang@zilliz.com>
This commit is contained in:
XuanYang-cn 2020-12-24 20:55:40 +08:00 committed by yefu.chen
parent 4dbf4ade41
commit c0797333ce
50 changed files with 2173 additions and 617 deletions

View File

@ -22,6 +22,8 @@ msgChannel:
writeNodeTimeTick: "writeNodeTimeTick" writeNodeTimeTick: "writeNodeTimeTick"
# old name: statsChannels: "statistic" # old name: statsChannels: "statistic"
queryNodeStats: "query-node-stats" queryNodeStats: "query-node-stats"
# cmd for loadIndex, flush, etc...
cmd: "cmd"
# sub name generation rule: ${subNamePrefix}-${NodeID} # sub name generation rule: ${subNamePrefix}-${NodeID}
subNamePrefix: subNamePrefix:

View File

@ -42,3 +42,7 @@ queryNode:
stats: stats:
recvBufSize: 64 recvBufSize: 64
loadIndex:
recvBufSize: 512
pulsarBufSize: 512

View File

@ -58,6 +58,7 @@ type segRequest struct {
partition string partition string
segInfo map[UniqueID]uint32 segInfo map[UniqueID]uint32
channelID int32 channelID int32
timestamp Timestamp
} }
type syncRequest struct { type syncRequest struct {
@ -122,15 +123,17 @@ type Allocator struct {
countPerRPC uint32 countPerRPC uint32
toDoReqs []request toDoReqs []request
canDoReqs []request
syncReqs []request syncReqs []request
tChan tickerChan tChan tickerChan
forceSyncChan chan request forceSyncChan chan request
syncFunc func() syncFunc func() bool
processFunc func(req request) error processFunc func(req request) error
checkFunc func(timeout bool) bool
checkSyncFunc func(timeout bool) bool
pickCanDoFunc func()
} }
func (ta *Allocator) Start() error { func (ta *Allocator) Start() error {
@ -145,7 +148,6 @@ func (ta *Allocator) Start() error {
} }
func (ta *Allocator) connectMaster() error { func (ta *Allocator) connectMaster() error {
log.Printf("Connected to master, master_addr=%s", ta.masterAddress)
ctx, cancel := context.WithTimeout(context.Background(), 2*time.Second) ctx, cancel := context.WithTimeout(context.Background(), 2*time.Second)
defer cancel() defer cancel()
conn, err := grpc.DialContext(ctx, ta.masterAddress, grpc.WithInsecure(), grpc.WithBlock()) conn, err := grpc.DialContext(ctx, ta.masterAddress, grpc.WithInsecure(), grpc.WithBlock())
@ -182,7 +184,13 @@ func (ta *Allocator) mainLoop() {
ta.finishSyncRequest() ta.finishSyncRequest()
case <-ta.tChan.Chan(): case <-ta.tChan.Chan():
ta.sync(true) ta.pickCanDo()
ta.finishRequest()
if ta.sync(true) {
ta.pickCanDo()
ta.finishRequest()
}
ta.failRemainRequest()
case first := <-ta.reqs: case first := <-ta.reqs:
ta.toDoReqs = append(ta.toDoReqs, first) ta.toDoReqs = append(ta.toDoReqs, first)
@ -190,9 +198,13 @@ func (ta *Allocator) mainLoop() {
for i := 0; i < pending; i++ { for i := 0; i < pending; i++ {
ta.toDoReqs = append(ta.toDoReqs, <-ta.reqs) ta.toDoReqs = append(ta.toDoReqs, <-ta.reqs)
} }
ta.sync(false) ta.pickCanDo()
ta.finishRequest() ta.finishRequest()
if ta.sync(false) {
ta.pickCanDo()
ta.finishRequest()
}
ta.failRemainRequest()
case <-loopCtx.Done(): case <-loopCtx.Done():
return return
@ -201,19 +213,32 @@ func (ta *Allocator) mainLoop() {
} }
} }
func (ta *Allocator) sync(timeout bool) { func (ta *Allocator) pickCanDo() {
if ta.syncFunc == nil { if ta.pickCanDoFunc == nil {
return return
} }
if ta.checkFunc == nil || !ta.checkFunc(timeout) { ta.pickCanDoFunc()
return
} }
ta.syncFunc() func (ta *Allocator) sync(timeout bool) bool {
if ta.syncFunc == nil || ta.checkSyncFunc == nil {
ta.canDoReqs = ta.toDoReqs
ta.toDoReqs = ta.toDoReqs[0:0]
return true
}
if !timeout && len(ta.toDoReqs) == 0 {
return false
}
if !ta.checkSyncFunc(timeout) {
return false
}
ret := ta.syncFunc()
if !timeout { if !timeout {
ta.tChan.Reset() ta.tChan.Reset()
} }
return ret
} }
func (ta *Allocator) finishSyncRequest() { func (ta *Allocator) finishSyncRequest() {
@ -225,14 +250,23 @@ func (ta *Allocator) finishSyncRequest() {
ta.syncReqs = ta.syncReqs[0:0] ta.syncReqs = ta.syncReqs[0:0]
} }
func (ta *Allocator) finishRequest() { func (ta *Allocator) failRemainRequest() {
for _, req := range ta.toDoReqs { for _, req := range ta.toDoReqs {
if req != nil {
req.Notify(errors.New("failed: unexpected error"))
}
}
ta.toDoReqs = []request{}
}
func (ta *Allocator) finishRequest() {
for _, req := range ta.canDoReqs {
if req != nil { if req != nil {
err := ta.processFunc(req) err := ta.processFunc(req)
req.Notify(err) req.Notify(err)
} }
} }
ta.toDoReqs = ta.toDoReqs[0:0] ta.canDoReqs = []request{}
} }
func (ta *Allocator) revokeRequest(err error) { func (ta *Allocator) revokeRequest(err error) {

View File

@ -21,6 +21,8 @@ type IDAllocator struct {
idStart UniqueID idStart UniqueID
idEnd UniqueID idEnd UniqueID
PeerID UniqueID
} }
func NewIDAllocator(ctx context.Context, masterAddr string) (*IDAllocator, error) { func NewIDAllocator(ctx context.Context, masterAddr string) (*IDAllocator, error) {
@ -37,16 +39,17 @@ func NewIDAllocator(ctx context.Context, masterAddr string) (*IDAllocator, error
a.tChan = &emptyTicker{} a.tChan = &emptyTicker{}
a.Allocator.syncFunc = a.syncID a.Allocator.syncFunc = a.syncID
a.Allocator.processFunc = a.processFunc a.Allocator.processFunc = a.processFunc
a.Allocator.checkFunc = a.checkFunc a.Allocator.checkSyncFunc = a.checkSyncFunc
a.Allocator.pickCanDoFunc = a.pickCanDoFunc
a.init() a.init()
return a, nil return a, nil
} }
func (ia *IDAllocator) syncID() { func (ia *IDAllocator) syncID() bool {
fmt.Println("syncID") fmt.Println("syncID")
ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second) ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second)
req := &internalpb.IDRequest{ req := &internalpb.IDRequest{
PeerID: 1, PeerID: ia.PeerID,
Role: internalpb.PeerRole_Proxy, Role: internalpb.PeerRole_Proxy,
Count: ia.countPerRPC, Count: ia.countPerRPC,
} }
@ -55,22 +58,32 @@ func (ia *IDAllocator) syncID() {
cancel() cancel()
if err != nil { if err != nil {
log.Println("syncID Failed!!!!!") log.Println("syncID Failed!!!!!")
return return false
} }
ia.idStart = resp.GetID() ia.idStart = resp.GetID()
ia.idEnd = ia.idStart + int64(resp.GetCount()) ia.idEnd = ia.idStart + int64(resp.GetCount())
return true
} }
func (ia *IDAllocator) checkFunc(timeout bool) bool { func (ia *IDAllocator) checkSyncFunc(timeout bool) bool {
if timeout { return timeout || len(ia.toDoReqs) > 0
return timeout
} }
func (ia *IDAllocator) pickCanDoFunc() {
total := uint32(ia.idEnd - ia.idStart)
need := uint32(0) need := uint32(0)
idx := 0
for _, req := range ia.toDoReqs { for _, req := range ia.toDoReqs {
iReq := req.(*idRequest) iReq := req.(*idRequest)
need += iReq.count need += iReq.count
if need <= total {
ia.canDoReqs = append(ia.canDoReqs, req)
idx++
} else {
break
} }
return ia.idStart+int64(need) >= ia.idEnd }
ia.toDoReqs = ia.toDoReqs[idx:]
} }
func (ia *IDAllocator) processFunc(req request) error { func (ia *IDAllocator) processFunc(req request) error {

View File

@ -5,10 +5,9 @@ import (
"context" "context"
"fmt" "fmt"
"log" "log"
"sort"
"time" "time"
"github.com/cznic/mathutil" "github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
"github.com/zilliztech/milvus-distributed/internal/errors" "github.com/zilliztech/milvus-distributed/internal/errors"
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb" "github.com/zilliztech/milvus-distributed/internal/proto/internalpb"
@ -16,42 +15,101 @@ import (
const ( const (
SegCountPerRPC = 20000 SegCountPerRPC = 20000
ActiveTimeDuration = 100 // Second ActiveTimeDuration = 100 //second
) )
type segInfo struct {
segID UniqueID
count uint32
expireTime Timestamp
}
type assignInfo struct { type assignInfo struct {
collName string collName string
partitionTag string partitionTag string
channelID int32 channelID int32
segInfo map[UniqueID]uint32 // segmentID->count map segID UniqueID
expireTime time.Time segInfos *list.List
segCapacity uint32
lastInsertTime time.Time lastInsertTime time.Time
} }
func (info *assignInfo) IsExpired(now time.Time) bool { func (info *segInfo) IsExpired(ts Timestamp) bool {
return now.Sub(info.expireTime) >= 0 return ts > info.expireTime || info.count <= 0
}
func (info *segInfo) Capacity(ts Timestamp) uint32 {
if info.IsExpired(ts) {
return 0
}
return info.count
}
func (info *segInfo) Assign(ts Timestamp, count uint32) uint32 {
if info.IsExpired(ts) {
return 0
}
ret := uint32(0)
if info.count >= count {
info.count -= count
ret = count
} else {
info.count = 0
ret = info.count
}
return ret
}
func (info *assignInfo) RemoveExpired(ts Timestamp) {
for e := info.segInfos.Front(); e != nil; e = e.Next() {
segInfo := e.Value.(*segInfo)
if segInfo.IsExpired(ts) {
info.segInfos.Remove(e)
}
}
}
func (info *assignInfo) Capacity(ts Timestamp) uint32 {
ret := uint32(0)
for e := info.segInfos.Front(); e != nil; e = e.Next() {
segInfo := e.Value.(*segInfo)
ret += segInfo.Capacity(ts)
}
return ret
}
func (info *assignInfo) Assign(ts Timestamp, count uint32) (map[UniqueID]uint32, error) {
capacity := info.Capacity(ts)
if capacity < count {
errMsg := fmt.Sprintf("AssignSegment Failed: capacity:%d is less than count:%d", capacity, count)
return nil, errors.New(errMsg)
}
result := make(map[UniqueID]uint32)
for e := info.segInfos.Front(); e != nil && count != 0; e = e.Next() {
segInfo := e.Value.(*segInfo)
cur := segInfo.Assign(ts, count)
count -= cur
if cur > 0 {
result[segInfo.segID] += cur
}
}
return result, nil
} }
func (info *assignInfo) IsActive(now time.Time) bool { func (info *assignInfo) IsActive(now time.Time) bool {
return now.Sub(info.lastInsertTime) <= ActiveTimeDuration*time.Second return now.Sub(info.lastInsertTime) <= ActiveTimeDuration*time.Second
} }
func (info *assignInfo) IsEnough(count uint32) bool {
total := uint32(0)
for _, count := range info.segInfo {
total += count
}
return total >= count
}
type SegIDAssigner struct { type SegIDAssigner struct {
Allocator Allocator
assignInfos map[string]*list.List // collectionName -> *list.List assignInfos map[string]*list.List // collectionName -> *list.List
segReqs []*internalpb.SegIDRequest segReqs []*internalpb.SegIDRequest
canDoReqs []request getTickFunc func() Timestamp
PeerID UniqueID
} }
func NewSegIDAssigner(ctx context.Context, masterAddr string) (*SegIDAssigner, error) { func NewSegIDAssigner(ctx context.Context, masterAddr string, getTickFunc func() Timestamp) (*SegIDAssigner, error) {
ctx1, cancel := context.WithCancel(ctx) ctx1, cancel := context.WithCancel(ctx)
sa := &SegIDAssigner{ sa := &SegIDAssigner{
Allocator: Allocator{reqs: make(chan request, maxConcurrentRequests), Allocator: Allocator{reqs: make(chan request, maxConcurrentRequests),
@ -61,72 +119,80 @@ func NewSegIDAssigner(ctx context.Context, masterAddr string) (*SegIDAssigner, e
countPerRPC: SegCountPerRPC, countPerRPC: SegCountPerRPC,
}, },
assignInfos: make(map[string]*list.List), assignInfos: make(map[string]*list.List),
getTickFunc: getTickFunc,
} }
sa.tChan = &ticker{ sa.tChan = &ticker{
updateInterval: time.Second, updateInterval: time.Second,
} }
sa.Allocator.syncFunc = sa.syncSegments sa.Allocator.syncFunc = sa.syncSegments
sa.Allocator.processFunc = sa.processFunc sa.Allocator.processFunc = sa.processFunc
sa.Allocator.checkFunc = sa.checkFunc sa.Allocator.checkSyncFunc = sa.checkSyncFunc
sa.Allocator.pickCanDoFunc = sa.pickCanDoFunc
return sa, nil return sa, nil
} }
func (sa *SegIDAssigner) collectExpired() { func (sa *SegIDAssigner) collectExpired() {
now := time.Now() ts := sa.getTickFunc()
//now := time.Now()
for _, info := range sa.assignInfos { for _, info := range sa.assignInfos {
for e := info.Front(); e != nil; e = e.Next() { for e := info.Front(); e != nil; e = e.Next() {
assign := e.Value.(*assignInfo) assign := e.Value.(*assignInfo)
if !assign.IsActive(now) || !assign.IsExpired(now) { assign.RemoveExpired(ts)
continue if assign.Capacity(ts) == 0 {
info.Remove(e)
//if assign.IsActive(now) {
// sa.segReqs = append(sa.segReqs, &internalpb.SegIDRequest{
// ChannelID: assign.channelID,
// Count: 0, // intend to set zero
// CollName: assign.collName,
// PartitionTag: assign.partitionTag,
// })
//} else {
// info.Remove(e)
//}
} }
sa.segReqs = append(sa.segReqs, &internalpb.SegIDRequest{
ChannelID: assign.channelID,
Count: sa.countPerRPC,
CollName: assign.collName,
PartitionTag: assign.partitionTag,
})
} }
} }
} }
func (sa *SegIDAssigner) checkToDoReqs() { func (sa *SegIDAssigner) pickCanDoFunc() {
if sa.toDoReqs == nil { if sa.toDoReqs == nil {
return return
} }
now := time.Now() records := make(map[string]map[string]map[int32]uint32)
newTodoReqs := sa.toDoReqs[0:0]
for _, req := range sa.toDoReqs { for _, req := range sa.toDoReqs {
segRequest := req.(*segRequest) segRequest := req.(*segRequest)
colName := segRequest.colName
partition := segRequest.partition
channelID := segRequest.channelID
if _, ok := records[colName]; !ok {
records[colName] = make(map[string]map[int32]uint32)
}
if _, ok := records[colName][partition]; !ok {
records[colName][partition] = make(map[int32]uint32)
}
if _, ok := records[colName][partition][channelID]; !ok {
records[colName][partition][channelID] = 0
}
records[colName][partition][channelID] += segRequest.count
assign := sa.getAssign(segRequest.colName, segRequest.partition, segRequest.channelID) assign := sa.getAssign(segRequest.colName, segRequest.partition, segRequest.channelID)
if assign == nil || assign.IsExpired(now) || !assign.IsEnough(segRequest.count) { if assign == nil || assign.Capacity(segRequest.timestamp) < records[colName][partition][channelID] {
sa.segReqs = append(sa.segReqs, &internalpb.SegIDRequest{ sa.segReqs = append(sa.segReqs, &internalpb.SegIDRequest{
ChannelID: segRequest.channelID, ChannelID: segRequest.channelID,
Count: segRequest.count, Count: segRequest.count,
CollName: segRequest.colName, CollName: segRequest.colName,
PartitionTag: segRequest.partition, PartitionTag: segRequest.partition,
}) })
newTodoReqs = append(newTodoReqs, req)
} else {
sa.canDoReqs = append(sa.canDoReqs, req)
} }
} }
} sa.toDoReqs = newTodoReqs
func (sa *SegIDAssigner) removeSegInfo(colName, partition string, channelID int32) {
assignInfos, ok := sa.assignInfos[colName]
if !ok {
return
}
cnt := assignInfos.Len()
if cnt == 0 {
return
}
for e := assignInfos.Front(); e != nil; e = e.Next() {
assign := e.Value.(*assignInfo)
if assign.partitionTag != partition || assign.channelID != channelID {
continue
}
assignInfos.Remove(e)
}
} }
func (sa *SegIDAssigner) getAssign(colName, partition string, channelID int32) *assignInfo { func (sa *SegIDAssigner) getAssign(colName, partition string, channelID int32) *assignInfo {
@ -145,72 +211,109 @@ func (sa *SegIDAssigner) getAssign(colName, partition string, channelID int32) *
return nil return nil
} }
func (sa *SegIDAssigner) checkFunc(timeout bool) bool { func (sa *SegIDAssigner) checkSyncFunc(timeout bool) bool {
if timeout {
sa.collectExpired() sa.collectExpired()
} else { return timeout || len(sa.segReqs) != 0
sa.checkToDoReqs()
} }
return len(sa.segReqs) != 0 func (sa *SegIDAssigner) checkSegReqEqual(req1, req2 *internalpb.SegIDRequest) bool {
if req1 == nil || req2 == nil {
return false
} }
func (sa *SegIDAssigner) syncSegments() { if req1 == req2 {
return true
}
return req1.CollName == req2.CollName && req1.PartitionTag == req2.PartitionTag && req1.ChannelID == req2.ChannelID
}
func (sa *SegIDAssigner) reduceSegReqs() {
if len(sa.segReqs) == 0 { if len(sa.segReqs) == 0 {
return return
} }
var newSegReqs []*internalpb.SegIDRequest
for _, req1 := range sa.segReqs {
var req2 *internalpb.SegIDRequest
for _, req3 := range newSegReqs {
if sa.checkSegReqEqual(req1, req3) {
req2 = req3
break
}
}
if req2 == nil { // not found
newSegReqs = append(newSegReqs, req1)
} else {
req2.Count += req1.Count
}
}
for _, req := range newSegReqs {
if req.Count == 0 {
req.Count = sa.countPerRPC
}
}
sa.segReqs = newSegReqs
}
func (sa *SegIDAssigner) syncSegments() bool {
if len(sa.segReqs) == 0 {
return true
}
sa.reduceSegReqs()
ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second) ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second)
defer cancel() defer cancel()
req := &internalpb.AssignSegIDRequest{ req := &internalpb.AssignSegIDRequest{
PeerID: 1, PeerID: sa.PeerID,
Role: internalpb.PeerRole_Proxy, Role: internalpb.PeerRole_Proxy,
PerChannelReq: sa.segReqs, PerChannelReq: sa.segReqs,
} }
sa.segReqs = sa.segReqs[0:0] sa.segReqs = []*internalpb.SegIDRequest{}
fmt.Println("OOOOO", req.PerChannelReq)
resp, err := sa.masterClient.AssignSegmentID(ctx, req) resp, err := sa.masterClient.AssignSegmentID(ctx, req)
//if resp.Status.GetErrorCode() != commonpb.ErrorCode_SUCCESS { if err != nil {
// log.Println("GRPC AssignSegmentID Failed", resp, err) log.Println("GRPC AssignSegmentID Failed", resp, err)
// return return false
//}
now := time.Now()
expiredTime := now.Add(time.Millisecond * time.Duration(1000))
for _, info := range resp.PerChannelAssignment {
sa.removeSegInfo(info.CollName, info.PartitionTag, info.ChannelID)
} }
now := time.Now()
success := false
for _, info := range resp.PerChannelAssignment { for _, info := range resp.PerChannelAssignment {
if info.Status.GetErrorCode() != commonpb.ErrorCode_SUCCESS {
log.Println("SyncSegment Error:", info.Status.Reason)
continue
}
assign := sa.getAssign(info.CollName, info.PartitionTag, info.ChannelID) assign := sa.getAssign(info.CollName, info.PartitionTag, info.ChannelID)
segInfo := &segInfo{
segID: info.SegID,
count: info.Count,
expireTime: info.ExpireTime,
}
if assign == nil { if assign == nil {
colInfos, ok := sa.assignInfos[info.CollName] colInfos, ok := sa.assignInfos[info.CollName]
if !ok { if !ok {
colInfos = list.New() colInfos = list.New()
} }
segInfo := make(map[UniqueID]uint32) segInfos := list.New()
segInfo[info.SegID] = info.Count
newAssign := &assignInfo{ segInfos.PushBack(segInfo)
assign = &assignInfo{
collName: info.CollName, collName: info.CollName,
partitionTag: info.PartitionTag, partitionTag: info.PartitionTag,
channelID: info.ChannelID, channelID: info.ChannelID,
segInfo: segInfo, segInfos: segInfos,
} }
colInfos.PushBack(newAssign) colInfos.PushBack(assign)
sa.assignInfos[info.CollName] = colInfos sa.assignInfos[info.CollName] = colInfos
} else { } else {
assign.segInfo[info.SegID] = info.Count assign.segInfos.PushBack(segInfo)
assign.expireTime = expiredTime }
assign.lastInsertTime = now assign.lastInsertTime = now
success = true
} }
} return success
if err != nil {
log.Println("syncSemgnet Failed!!!!!")
return
}
} }
func (sa *SegIDAssigner) processFunc(req request) error { func (sa *SegIDAssigner) processFunc(req request) error {
@ -219,43 +322,19 @@ func (sa *SegIDAssigner) processFunc(req request) error {
if assign == nil { if assign == nil {
return errors.New("Failed to GetSegmentID") return errors.New("Failed to GetSegmentID")
} }
result, err := assign.Assign(segRequest.timestamp, segRequest.count)
keys := make([]UniqueID, len(assign.segInfo)) segRequest.segInfo = result
i := 0 return err
for key := range assign.segInfo {
keys[i] = key
i++
}
reqCount := segRequest.count
resultSegInfo := make(map[UniqueID]uint32)
sort.Slice(keys, func(i, j int) bool { return keys[i] < keys[j] })
for _, key := range keys {
if reqCount <= 0 {
break
}
cur := assign.segInfo[key]
minCnt := mathutil.MinUint32(cur, reqCount)
resultSegInfo[key] = minCnt
cur -= minCnt
reqCount -= minCnt
if cur <= 0 {
delete(assign.segInfo, key)
} else {
assign.segInfo[key] = cur
}
}
segRequest.segInfo = resultSegInfo
return nil
} }
func (sa *SegIDAssigner) GetSegmentID(colName, partition string, channelID int32, count uint32) (map[UniqueID]uint32, error) { func (sa *SegIDAssigner) GetSegmentID(colName, partition string, channelID int32, count uint32, ts Timestamp) (map[UniqueID]uint32, error) {
req := &segRequest{ req := &segRequest{
baseRequest: baseRequest{done: make(chan error), valid: false}, baseRequest: baseRequest{done: make(chan error), valid: false},
colName: colName, colName: colName,
partition: partition, partition: partition,
channelID: channelID, channelID: channelID,
count: count, count: count,
timestamp: ts,
} }
sa.reqs <- req sa.reqs <- req
req.Wait() req.Wait()

View File

@ -19,6 +19,7 @@ type TimestampAllocator struct {
Allocator Allocator
lastTsBegin Timestamp lastTsBegin Timestamp
lastTsEnd Timestamp lastTsEnd Timestamp
PeerID UniqueID
} }
func NewTimestampAllocator(ctx context.Context, masterAddr string) (*TimestampAllocator, error) { func NewTimestampAllocator(ctx context.Context, masterAddr string) (*TimestampAllocator, error) {
@ -36,26 +37,36 @@ func NewTimestampAllocator(ctx context.Context, masterAddr string) (*TimestampAl
} }
a.Allocator.syncFunc = a.syncTs a.Allocator.syncFunc = a.syncTs
a.Allocator.processFunc = a.processFunc a.Allocator.processFunc = a.processFunc
a.Allocator.checkFunc = a.checkFunc a.Allocator.checkSyncFunc = a.checkSyncFunc
a.Allocator.pickCanDoFunc = a.pickCanDoFunc
return a, nil return a, nil
} }
func (ta *TimestampAllocator) checkFunc(timeout bool) bool { func (ta *TimestampAllocator) checkSyncFunc(timeout bool) bool {
if timeout { return timeout || len(ta.toDoReqs) > 0
return true
}
need := uint32(0)
for _, req := range ta.toDoReqs {
iReq := req.(*tsoRequest)
need += iReq.count
}
return ta.lastTsBegin+Timestamp(need) >= ta.lastTsEnd
} }
func (ta *TimestampAllocator) syncTs() { func (ta *TimestampAllocator) pickCanDoFunc() {
total := uint32(ta.lastTsEnd - ta.lastTsBegin)
need := uint32(0)
idx := 0
for _, req := range ta.toDoReqs {
tReq := req.(*tsoRequest)
need += tReq.count
if need <= total {
ta.canDoReqs = append(ta.canDoReqs, req)
idx++
} else {
break
}
}
ta.toDoReqs = ta.toDoReqs[idx:]
}
func (ta *TimestampAllocator) syncTs() bool {
ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second) ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second)
req := &internalpb.TsoRequest{ req := &internalpb.TsoRequest{
PeerID: 1, PeerID: ta.PeerID,
Role: internalpb.PeerRole_Proxy, Role: internalpb.PeerRole_Proxy,
Count: ta.countPerRPC, Count: ta.countPerRPC,
} }
@ -64,10 +75,11 @@ func (ta *TimestampAllocator) syncTs() {
cancel() cancel()
if err != nil { if err != nil {
log.Println("syncTimestamp Failed!!!!!") log.Println("syncTimestamp Failed!!!!!")
return return false
} }
ta.lastTsBegin = resp.GetTimestamp() ta.lastTsBegin = resp.GetTimestamp()
ta.lastTsEnd = ta.lastTsBegin + uint64(resp.GetCount()) ta.lastTsEnd = ta.lastTsBegin + uint64(resp.GetCount())
return true
} }
func (ta *TimestampAllocator) processFunc(req request) error { func (ta *TimestampAllocator) processFunc(req request) error {

View File

@ -0,0 +1,23 @@
// Copyright (C) 2019-2020 Zilliz. All rights reserved.
//
// Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software distributed under the License
// is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
// or implied. See the License for the specific language governing permissions and limitations under the License
#pragma once
#include <string>
#include <map>
#include "../index/knowhere/knowhere/index/vector_index/VecIndex.h"
struct LoadIndexInfo {
std::string field_name;
int64_t field_id;
std::map<std::string, std::string> index_params;
milvus::knowhere::VecIndexPtr index;
};

View File

@ -11,7 +11,8 @@ set(SEGCORE_FILES
InsertRecord.cpp InsertRecord.cpp
Reduce.cpp Reduce.cpp
plan_c.cpp plan_c.cpp
reduce_c.cpp) reduce_c.cpp
load_index_c.cpp)
add_library(milvus_segcore SHARED add_library(milvus_segcore SHARED
${SEGCORE_FILES} ${SEGCORE_FILES}
) )

View File

@ -0,0 +1,139 @@
// Copyright (C) 2019-2020 Zilliz. All rights reserved.
//
// Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software distributed under the License
// is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
// or implied. See the License for the specific language governing permissions and limitations under the License
#include "index/knowhere/knowhere/common/BinarySet.h"
#include "index/knowhere/knowhere/index/vector_index/VecIndexFactory.h"
#include "segcore/load_index_c.h"
#include "common/LoadIndex.h"
#include "utils/EasyAssert.h"
CStatus
NewLoadIndexInfo(CLoadIndexInfo* c_load_index_info) {
try {
auto load_index_info = std::make_unique<LoadIndexInfo>();
*c_load_index_info = load_index_info.release();
auto status = CStatus();
status.error_code = Success;
status.error_msg = "";
return status;
} catch (std::exception& e) {
auto status = CStatus();
status.error_code = UnexpectedException;
status.error_msg = strdup(e.what());
return status;
}
}
CStatus
AppendIndexParam(CLoadIndexInfo c_load_index_info, const char* c_index_key, const char* c_index_value) {
try {
auto load_index_info = (LoadIndexInfo*)c_load_index_info;
std::string index_key(c_index_key);
std::string index_value(c_index_value);
load_index_info->index_params[index_key] = index_value;
auto status = CStatus();
status.error_code = Success;
status.error_msg = "";
return status;
} catch (std::exception& e) {
auto status = CStatus();
status.error_code = UnexpectedException;
status.error_msg = strdup(e.what());
return status;
}
}
CStatus
AppendFieldInfo(CLoadIndexInfo c_load_index_info, const char* c_field_name, int64_t field_id) {
try {
auto load_index_info = (LoadIndexInfo*)c_load_index_info;
std::string field_name(c_field_name);
load_index_info->field_name = field_name;
load_index_info->field_id = field_id;
auto status = CStatus();
status.error_code = Success;
status.error_msg = "";
return status;
} catch (std::exception& e) {
auto status = CStatus();
status.error_code = UnexpectedException;
status.error_msg = strdup(e.what());
return status;
}
}
CStatus
AppendIndex(CLoadIndexInfo c_load_index_info, CBinarySet c_binary_set) {
try {
auto load_index_info = (LoadIndexInfo*)c_load_index_info;
auto binary_set = (milvus::knowhere::BinarySet*)c_binary_set;
auto& index_params = load_index_info->index_params;
bool find_index_type = index_params.count("index_type") > 0 ? true : false;
bool find_index_mode = index_params.count("index_mode") > 0 ? true : false;
Assert(find_index_mode == true);
Assert(find_index_type == true);
auto mode = index_params["index_mode"] == "CPU" ? milvus::knowhere::IndexMode::MODE_CPU
: milvus::knowhere::IndexMode::MODE_GPU;
load_index_info->index =
milvus::knowhere::VecIndexFactory::GetInstance().CreateVecIndex(index_params["index_type"], mode);
load_index_info->index->Load(*binary_set);
auto status = CStatus();
status.error_code = Success;
status.error_msg = "";
return status;
} catch (std::exception& e) {
auto status = CStatus();
status.error_code = UnexpectedException;
status.error_msg = strdup(e.what());
return status;
}
}
CStatus
NewBinarySet(CBinarySet* c_binary_set) {
try {
auto binary_set = std::make_unique<milvus::knowhere::BinarySet>();
*c_binary_set = binary_set.release();
auto status = CStatus();
status.error_code = Success;
status.error_msg = "";
return status;
} catch (std::exception& e) {
auto status = CStatus();
status.error_code = UnexpectedException;
status.error_msg = strdup(e.what());
return status;
}
}
CStatus
AppendBinaryIndex(CBinarySet c_binary_set, void* index_binary, int64_t index_size, const char* c_index_key) {
try {
auto binary_set = (milvus::knowhere::BinarySet*)c_binary_set;
std::string index_key(c_index_key);
uint8_t* index = (uint8_t*)index_binary;
std::shared_ptr<uint8_t[]> data(index);
binary_set->Append(index_key, data, index_size);
auto status = CStatus();
status.error_code = Success;
status.error_msg = "";
return status;
} catch (std::exception& e) {
auto status = CStatus();
status.error_code = UnexpectedException;
status.error_msg = strdup(e.what());
return status;
}
}

View File

@ -0,0 +1,45 @@
// Copyright (C) 2019-2020 Zilliz. All rights reserved.
//
// Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software distributed under the License
// is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
// or implied. See the License for the specific language governing permissions and limitations under the License
#ifdef __cplusplus
extern "C" {
#endif
#include <stdbool.h>
#include <stdlib.h>
#include <stdint.h>
#include "segcore/collection_c.h"
typedef void* CLoadIndexInfo;
typedef void* CBinarySet;
CStatus
NewLoadIndexInfo(CLoadIndexInfo* c_load_index_info);
CStatus
AppendIndexParam(CLoadIndexInfo c_load_index_info, const char* index_key, const char* index_value);
CStatus
AppendFieldInfo(CLoadIndexInfo c_load_index_info, const char* field_name, int64_t field_id);
CStatus
AppendIndex(CLoadIndexInfo c_load_index_info, CBinarySet c_binary_set);
CStatus
NewBinarySet(CBinarySet* c_binary_set);
CStatus
AppendBinaryIndex(CBinarySet c_binary_set, void* index_binary, int64_t index_size, const char* c_index_key);
#ifdef __cplusplus
}
#endif

View File

@ -19,6 +19,7 @@
#include <knowhere/index/vector_index/VecIndexFactory.h> #include <knowhere/index/vector_index/VecIndexFactory.h>
#include <cstdint> #include <cstdint>
#include <boost/concept_check.hpp> #include <boost/concept_check.hpp>
#include "common/LoadIndex.h"
CSegmentBase CSegmentBase
NewSegment(CCollection collection, uint64_t segment_id) { NewSegment(CCollection collection, uint64_t segment_id) {
@ -173,6 +174,22 @@ FillTargetEntry(CSegmentBase c_segment, CPlan c_plan, CQueryResult c_result) {
return status; return status;
} }
CStatus
UpdateSegmentIndex(CSegmentBase c_segment, CLoadIndexInfo c_load_index_info) {
auto load_index_info = (LoadIndexInfo*)c_load_index_info;
try {
auto status = CStatus();
status.error_code = Success;
status.error_msg = "";
return status;
} catch (std::exception& e) {
auto status = CStatus();
status.error_code = UnexpectedException;
status.error_msg = strdup(e.what());
return status;
}
}
////////////////////////////////////////////////////////////////// //////////////////////////////////////////////////////////////////
int int

View File

@ -18,6 +18,7 @@ extern "C" {
#include <stdint.h> #include <stdint.h>
#include "segcore/plan_c.h" #include "segcore/plan_c.h"
#include "segcore/load_index_c.h"
typedef void* CSegmentBase; typedef void* CSegmentBase;
typedef void* CQueryResult; typedef void* CQueryResult;
@ -64,6 +65,8 @@ Search(CSegmentBase c_segment,
CStatus CStatus
FillTargetEntry(CSegmentBase c_segment, CPlan c_plan, CQueryResult result); FillTargetEntry(CSegmentBase c_segment, CPlan c_plan, CQueryResult result);
CStatus
UpdateSegmentIndex(CSegmentBase c_segment, CLoadIndexInfo c_load_index_info);
////////////////////////////////////////////////////////////////// //////////////////////////////////////////////////////////////////
int int

View File

@ -13,11 +13,17 @@
#include <string> #include <string>
#include <random> #include <random>
#include <gtest/gtest.h> #include <gtest/gtest.h>
#include <chrono>
#include "pb/service_msg.pb.h" #include "pb/service_msg.pb.h"
#include "segcore/reduce_c.h" #include "segcore/reduce_c.h"
#include <chrono> #include <index/knowhere/knowhere/index/vector_index/helpers/IndexParameter.h>
#include <index/knowhere/knowhere/index/vector_index/adapter/VectorAdapter.h>
#include <index/knowhere/knowhere/index/vector_index/VecIndexFactory.h>
#include <index/knowhere/knowhere/index/vector_index/IndexIVFPQ.h>
#include <common/LoadIndex.h>
namespace chrono = std::chrono; namespace chrono = std::chrono;
TEST(CApiTest, CollectionTest) { TEST(CApiTest, CollectionTest) {
@ -338,7 +344,7 @@ TEST(CApiTest, GetMemoryUsageInBytesTest) {
namespace { namespace {
auto auto
generate_data(int N) { generate_data(int N) {
std::vector<char> raw_data; std::vector<float> raw_data;
std::vector<uint64_t> timestamps; std::vector<uint64_t> timestamps;
std::vector<int64_t> uids; std::vector<int64_t> uids;
std::default_random_engine er(42); std::default_random_engine er(42);
@ -352,7 +358,7 @@ generate_data(int N) {
for (auto& x : vec) { for (auto& x : vec) {
x = distribution(er); x = distribution(er);
} }
raw_data.insert(raw_data.end(), (const char*)std::begin(vec), (const char*)std::end(vec)); raw_data.insert(raw_data.end(), std::begin(vec), std::end(vec));
int age = ei() % 100; int age = ei() % 100;
raw_data.insert(raw_data.end(), (const char*)&age, ((const char*)&age) + sizeof(age)); raw_data.insert(raw_data.end(), (const char*)&age, ((const char*)&age) + sizeof(age));
} }
@ -678,3 +684,53 @@ TEST(CApiTest, Reduce) {
DeleteCollection(collection); DeleteCollection(collection);
DeleteSegment(segment); DeleteSegment(segment);
} }
TEST(CApiTest, LoadIndex_Search) {
// generator index
constexpr auto DIM = 16;
constexpr auto K = 10;
auto N = 1024 * 1024 * 10;
auto num_query = 100;
auto [raw_data, timestamps, uids] = generate_data(N);
auto indexing = std::make_shared<milvus::knowhere::IVFPQ>();
auto conf = milvus::knowhere::Config{{milvus::knowhere::meta::DIM, DIM},
{milvus::knowhere::meta::TOPK, K},
{milvus::knowhere::IndexParams::nlist, 100},
{milvus::knowhere::IndexParams::nprobe, 4},
{milvus::knowhere::IndexParams::m, 4},
{milvus::knowhere::IndexParams::nbits, 8},
{milvus::knowhere::Metric::TYPE, milvus::knowhere::Metric::L2},
{milvus::knowhere::meta::DEVICEID, 0}};
auto database = milvus::knowhere::GenDataset(N, DIM, raw_data.data());
indexing->Train(database, conf);
indexing->AddWithoutIds(database, conf);
EXPECT_EQ(indexing->Count(), N);
EXPECT_EQ(indexing->Dim(), DIM);
// serializ index to binarySet
auto binary_set = indexing->Serialize(conf);
// fill loadIndexInfo
LoadIndexInfo load_index_info;
auto& index_params = load_index_info.index_params;
index_params["index_type"] = "IVF_PQ";
index_params["index_mode"] = "CPU";
auto mode = milvus::knowhere::IndexMode::MODE_CPU;
load_index_info.index =
milvus::knowhere::VecIndexFactory::GetInstance().CreateVecIndex(index_params["index_type"], mode);
load_index_info.index->Load(binary_set);
// search
auto query_dataset = milvus::knowhere::GenDataset(num_query, DIM, raw_data.data() + DIM * 4200);
auto result = indexing->Query(query_dataset, conf, nullptr);
auto ids = result->Get<int64_t*>(milvus::knowhere::meta::IDS);
auto dis = result->Get<float*>(milvus::knowhere::meta::DISTANCE);
for (int i = 0; i < std::min(num_query * K, 100); ++i) {
std::cout << ids[i] << "->" << dis[i] << std::endl;
}
}

View File

@ -150,7 +150,7 @@ func CreateServer(ctx context.Context) (*Master, error) {
// stats msg stream // stats msg stream
statsMs := ms.NewPulsarMsgStream(ctx, 1024) statsMs := ms.NewPulsarMsgStream(ctx, 1024)
statsMs.SetPulsarClient(pulsarAddr) statsMs.SetPulsarClient(pulsarAddr)
statsMs.CreatePulsarConsumers([]string{Params.QueryNodeStatsChannelName}, "SegmentStats", ms.NewUnmarshalDispatcher(), 1024) statsMs.CreatePulsarConsumers([]string{Params.QueryNodeStatsChannelName}, Params.MsgChannelSubName, ms.NewUnmarshalDispatcher(), 1024)
statsMs.Start() statsMs.Start()
m := &Master{ m := &Master{

View File

@ -377,7 +377,7 @@ func (mt *metaTable) DeletePartition(collID UniqueID, tag string) error {
for _, s := range collMeta.SegmentIDs { for _, s := range collMeta.SegmentIDs {
sm, ok := mt.segID2Meta[s] sm, ok := mt.segID2Meta[s]
if !ok { if !ok {
return errors.Errorf("can't find segment id = %d", s) return errors.Errorf("DeletePartition:can't find segment id = %d", s)
} }
if sm.PartitionTag != tag { if sm.PartitionTag != tag {
seg = append(seg, s) seg = append(seg, s)
@ -444,7 +444,7 @@ func (mt *metaTable) GetSegmentByID(segID UniqueID) (*pb.SegmentMeta, error) {
sm, ok := mt.segID2Meta[segID] sm, ok := mt.segID2Meta[segID]
if !ok { if !ok {
return nil, errors.Errorf("can't find segment id = %d", segID) return nil, errors.Errorf("GetSegmentByID:can't find segment id = %d", segID)
} }
return &sm, nil return &sm, nil
} }
@ -455,7 +455,7 @@ func (mt *metaTable) DeleteSegment(segID UniqueID) error {
segMeta, ok := mt.segID2Meta[segID] segMeta, ok := mt.segID2Meta[segID]
if !ok { if !ok {
return errors.Errorf("can't find segment. id = " + strconv.FormatInt(segID, 10)) return errors.Errorf("DeleteSegment:can't find segment. id = " + strconv.FormatInt(segID, 10))
} }
collMeta, ok := mt.collID2Meta[segMeta.CollectionID] collMeta, ok := mt.collID2Meta[segMeta.CollectionID]
@ -483,7 +483,7 @@ func (mt *metaTable) CloseSegment(segID UniqueID, closeTs Timestamp) error {
segMeta, ok := mt.segID2Meta[segID] segMeta, ok := mt.segID2Meta[segID]
if !ok { if !ok {
return errors.Errorf("can't find segment id = " + strconv.FormatInt(segID, 10)) return errors.Errorf("CloseSegment:can't find segment id = " + strconv.FormatInt(segID, 10))
} }
segMeta.CloseTime = closeTs segMeta.CloseTime = closeTs

View File

@ -98,7 +98,7 @@ func (assigner *SegmentAssigner) Assign(segmentID UniqueID, numRows int) (*Assig
return res, err return res, err
} }
physicalTs, logicalTs := tsoutil.ParseTS(ts) physicalTs, logicalTs := tsoutil.ParseTS(ts)
expirePhysicalTs := physicalTs.Add(time.Duration(assigner.segmentExpireDuration)) expirePhysicalTs := physicalTs.Add(time.Duration(assigner.segmentExpireDuration) * time.Millisecond)
expireTs := tsoutil.ComposeTS(expirePhysicalTs.UnixNano()/int64(time.Millisecond), int64(logicalTs)) expireTs := tsoutil.ComposeTS(expirePhysicalTs.UnixNano()/int64(time.Millisecond), int64(logicalTs))
status.lastExpireTime = expireTs status.lastExpireTime = expireTs
status.assignments = append(status.assignments, &Assignment{ status.assignments = append(status.assignments, &Assignment{

View File

@ -34,6 +34,7 @@ func (dispatcher *UnmarshalDispatcher) addDefaultMsgTemplates() {
dropCollectionMsg := DropCollectionMsg{} dropCollectionMsg := DropCollectionMsg{}
createPartitionMsg := CreatePartitionMsg{} createPartitionMsg := CreatePartitionMsg{}
dropPartitionMsg := DropPartitionMsg{} dropPartitionMsg := DropPartitionMsg{}
loadIndexMsg := LoadIndexMsg{}
queryNodeSegStatsMsg := QueryNodeStatsMsg{} queryNodeSegStatsMsg := QueryNodeStatsMsg{}
dispatcher.tempMap = make(map[internalPb.MsgType]UnmarshalFunc) dispatcher.tempMap = make(map[internalPb.MsgType]UnmarshalFunc)
@ -47,6 +48,7 @@ func (dispatcher *UnmarshalDispatcher) addDefaultMsgTemplates() {
dispatcher.tempMap[internalPb.MsgType_kDropCollection] = dropCollectionMsg.Unmarshal dispatcher.tempMap[internalPb.MsgType_kDropCollection] = dropCollectionMsg.Unmarshal
dispatcher.tempMap[internalPb.MsgType_kCreatePartition] = createPartitionMsg.Unmarshal dispatcher.tempMap[internalPb.MsgType_kCreatePartition] = createPartitionMsg.Unmarshal
dispatcher.tempMap[internalPb.MsgType_kDropPartition] = dropPartitionMsg.Unmarshal dispatcher.tempMap[internalPb.MsgType_kDropPartition] = dropPartitionMsg.Unmarshal
dispatcher.tempMap[internalPb.MsgType_kLoadIndex] = loadIndexMsg.Unmarshal
} }

View File

@ -291,8 +291,10 @@ message Key2SegMsg {
message LoadIndex { message LoadIndex {
MsgType msg_type = 1; MsgType msg_type = 1;
int64 segmentID = 2; int64 segmentID = 2;
int64 fieldID = 3; string fieldName = 3;
repeated string index_paths = 4; int64 fieldID = 4;
repeated string index_paths = 5;
repeated common.KeyValuePair index_params = 6;
} }
message IndexStats { message IndexStats {

View File

@ -2045,8 +2045,10 @@ func (m *Key2SegMsg) GetKey2Seg() []*Key2Seg {
type LoadIndex struct { type LoadIndex struct {
MsgType MsgType `protobuf:"varint,1,opt,name=msg_type,json=msgType,proto3,enum=milvus.proto.internal.MsgType" json:"msg_type,omitempty"` MsgType MsgType `protobuf:"varint,1,opt,name=msg_type,json=msgType,proto3,enum=milvus.proto.internal.MsgType" json:"msg_type,omitempty"`
SegmentID int64 `protobuf:"varint,2,opt,name=segmentID,proto3" json:"segmentID,omitempty"` SegmentID int64 `protobuf:"varint,2,opt,name=segmentID,proto3" json:"segmentID,omitempty"`
FieldID int64 `protobuf:"varint,3,opt,name=fieldID,proto3" json:"fieldID,omitempty"` FieldName string `protobuf:"bytes,3,opt,name=fieldName,proto3" json:"fieldName,omitempty"`
IndexPaths []string `protobuf:"bytes,4,rep,name=index_paths,json=indexPaths,proto3" json:"index_paths,omitempty"` FieldID int64 `protobuf:"varint,4,opt,name=fieldID,proto3" json:"fieldID,omitempty"`
IndexPaths []string `protobuf:"bytes,5,rep,name=index_paths,json=indexPaths,proto3" json:"index_paths,omitempty"`
IndexParams []*commonpb.KeyValuePair `protobuf:"bytes,6,rep,name=index_params,json=indexParams,proto3" json:"index_params,omitempty"`
XXX_NoUnkeyedLiteral struct{} `json:"-"` XXX_NoUnkeyedLiteral struct{} `json:"-"`
XXX_unrecognized []byte `json:"-"` XXX_unrecognized []byte `json:"-"`
XXX_sizecache int32 `json:"-"` XXX_sizecache int32 `json:"-"`
@ -2091,6 +2093,13 @@ func (m *LoadIndex) GetSegmentID() int64 {
return 0 return 0
} }
func (m *LoadIndex) GetFieldName() string {
if m != nil {
return m.FieldName
}
return ""
}
func (m *LoadIndex) GetFieldID() int64 { func (m *LoadIndex) GetFieldID() int64 {
if m != nil { if m != nil {
return m.FieldID return m.FieldID
@ -2105,6 +2114,13 @@ func (m *LoadIndex) GetIndexPaths() []string {
return nil return nil
} }
func (m *LoadIndex) GetIndexParams() []*commonpb.KeyValuePair {
if m != nil {
return m.IndexParams
}
return nil
}
type IndexStats struct { type IndexStats struct {
IndexParams []*commonpb.KeyValuePair `protobuf:"bytes,1,rep,name=index_params,json=indexParams,proto3" json:"index_params,omitempty"` 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"` NumRelatedSegments int64 `protobuf:"varint,2,opt,name=num_related_segments,json=numRelatedSegments,proto3" json:"num_related_segments,omitempty"`
@ -2637,120 +2653,120 @@ func init() {
func init() { proto.RegisterFile("internal_msg.proto", fileDescriptor_7eb37f6b80b23116) } func init() { proto.RegisterFile("internal_msg.proto", fileDescriptor_7eb37f6b80b23116) }
var fileDescriptor_7eb37f6b80b23116 = []byte{ var fileDescriptor_7eb37f6b80b23116 = []byte{
// 1831 bytes of a gzipped FileDescriptorProto // 1837 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xec, 0x59, 0xcd, 0x6f, 0x23, 0x49, 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xec, 0x59, 0xcd, 0x6f, 0x23, 0x49,
0x15, 0xdf, 0xee, 0xf6, 0x47, 0xfc, 0xec, 0x38, 0x3d, 0x95, 0x64, 0xc6, 0xb3, 0xb3, 0xec, 0x64, 0x15, 0xdf, 0xee, 0xf6, 0x47, 0xfc, 0xec, 0x38, 0x3d, 0x95, 0x64, 0xc6, 0xb3, 0xb3, 0xec, 0x64,
0x7a, 0x10, 0x1b, 0x16, 0x91, 0x40, 0x86, 0x03, 0x7b, 0x83, 0xc4, 0x5a, 0xd6, 0x0c, 0x19, 0x85, 0x7a, 0x10, 0x1b, 0x16, 0x91, 0x40, 0x86, 0x03, 0x7b, 0x83, 0xc4, 0x5a, 0xd6, 0x0c, 0x19, 0x85,
0x4e, 0xb4, 0x48, 0x68, 0xa5, 0x56, 0xc7, 0x7e, 0xb1, 0x4b, 0xfd, 0xe5, 0x54, 0xb5, 0x27, 0xf1, 0x4e, 0xb4, 0x48, 0x68, 0xa5, 0x56, 0xc7, 0x7e, 0xb1, 0x4b, 0xfd, 0xe5, 0x54, 0xb5, 0x27, 0xf1,
0x1c, 0x90, 0x90, 0xe6, 0x8c, 0xf8, 0x10, 0x07, 0x4e, 0x70, 0x87, 0x1d, 0xb1, 0x20, 0xfe, 0x07, 0x1c, 0x90, 0x10, 0x73, 0x46, 0x7c, 0x88, 0x03, 0x37, 0xee, 0xb0, 0x23, 0x16, 0xc4, 0xff, 0xc0,
0x3e, 0x2f, 0xfc, 0x17, 0x70, 0x00, 0x89, 0x5d, 0x0e, 0xdc, 0x50, 0x55, 0xf5, 0x87, 0x3b, 0xb1, 0xe7, 0x85, 0xff, 0x02, 0x0e, 0x20, 0xb1, 0xcb, 0x81, 0x1b, 0xaa, 0xaa, 0xfe, 0x70, 0x27, 0xb6,
0x9d, 0x88, 0x64, 0x96, 0x59, 0xed, 0xdc, 0xaa, 0x5e, 0x97, 0xab, 0xde, 0xef, 0xf7, 0x5e, 0xbd, 0x13, 0x6d, 0x32, 0xcb, 0xa0, 0x9d, 0x5b, 0xbd, 0xe7, 0xea, 0xaa, 0xf7, 0x7e, 0xef, 0xa3, 0xde,
0x7a, 0xef, 0x19, 0x08, 0x0d, 0x63, 0x64, 0xa1, 0xeb, 0x3b, 0x01, 0xef, 0x6f, 0x0c, 0x59, 0x14, 0x7b, 0x06, 0x42, 0xc3, 0x18, 0x59, 0xe8, 0xfa, 0x4e, 0xc0, 0xfb, 0x1b, 0x43, 0x16, 0xc5, 0x11,
0x47, 0x64, 0x35, 0xa0, 0xfe, 0xe3, 0x11, 0x57, 0xb3, 0x8d, 0x74, 0xc1, 0xab, 0x8d, 0x6e, 0x14, 0x59, 0x0d, 0xa8, 0xff, 0x78, 0xc4, 0x15, 0xb5, 0x91, 0x6e, 0x78, 0xb5, 0xd1, 0x8d, 0x82, 0x20,
0x04, 0x51, 0xa8, 0xc4, 0xaf, 0xde, 0xe0, 0xc8, 0x1e, 0xd3, 0x2e, 0xe6, 0xbf, 0xb3, 0x42, 0xa8, 0x0a, 0x15, 0xfb, 0xd5, 0x1b, 0x1c, 0xd9, 0x63, 0xda, 0xc5, 0xfc, 0x3b, 0x2b, 0x84, 0x5a, 0xa7,
0x75, 0xda, 0x36, 0x1e, 0x8f, 0x90, 0xc7, 0xe4, 0x26, 0x54, 0x86, 0x88, 0xac, 0xd3, 0x6e, 0x69, 0x6d, 0xe3, 0xf1, 0x08, 0x79, 0x4c, 0x6e, 0x42, 0x65, 0x88, 0xc8, 0x3a, 0xed, 0x96, 0xb6, 0xa6,
0x6b, 0xda, 0xba, 0x61, 0x27, 0x33, 0xf2, 0x00, 0x4a, 0x2c, 0xf2, 0xb1, 0xa5, 0xaf, 0x69, 0xeb, 0xad, 0x1b, 0x76, 0x42, 0x91, 0x07, 0x50, 0x62, 0x91, 0x8f, 0x2d, 0x7d, 0x4d, 0x5b, 0x6f, 0x6e,
0xcd, 0xad, 0xbb, 0x1b, 0x53, 0xcf, 0xda, 0xd8, 0x43, 0x64, 0x76, 0xe4, 0xa3, 0x2d, 0x17, 0x93, 0xdd, 0xdd, 0x98, 0x7a, 0xd7, 0xc6, 0x1e, 0x22, 0xb3, 0x23, 0x1f, 0x6d, 0xb9, 0x99, 0xac, 0x40,
0x15, 0x28, 0x77, 0xa3, 0x51, 0x18, 0xb7, 0x8c, 0x35, 0x6d, 0x7d, 0xd1, 0x56, 0x13, 0xab, 0x0f, 0xb9, 0x1b, 0x8d, 0xc2, 0xb8, 0x65, 0xac, 0x69, 0xeb, 0x8b, 0xb6, 0x22, 0xac, 0x3e, 0x80, 0xb8,
0x20, 0xce, 0xe3, 0xc3, 0x28, 0xe4, 0x48, 0x1e, 0x40, 0x85, 0xc7, 0x6e, 0x3c, 0xe2, 0xf2, 0xc0, 0x8f, 0x0f, 0xa3, 0x90, 0x23, 0x79, 0x00, 0x15, 0x1e, 0xbb, 0xf1, 0x88, 0xcb, 0x0b, 0xeb, 0x5b,
0xfa, 0xd6, 0x9d, 0xe2, 0xd6, 0x89, 0xf2, 0xfb, 0x72, 0x89, 0x9d, 0x2c, 0x25, 0x4d, 0xd0, 0x3b, 0x77, 0x8a, 0x47, 0x27, 0xc2, 0xef, 0xcb, 0x2d, 0x76, 0xb2, 0x95, 0x34, 0x41, 0xef, 0xb4, 0xa5,
0x6d, 0xa9, 0x8b, 0x61, 0xeb, 0x9d, 0xf6, 0x8c, 0x83, 0x22, 0x80, 0x03, 0x1e, 0x7d, 0x8c, 0xc8, 0x2c, 0x86, 0xad, 0x77, 0xda, 0x33, 0x2e, 0x8a, 0x00, 0x0e, 0x78, 0xf4, 0x09, 0x6a, 0xf6, 0x18,
0x1e, 0x43, 0x5d, 0x1e, 0x78, 0x15, 0x68, 0xaf, 0x41, 0x2d, 0xa6, 0x01, 0xf2, 0xd8, 0x0d, 0x86, 0xea, 0xf2, 0xc2, 0xab, 0xa8, 0xf6, 0x1a, 0xd4, 0x62, 0x1a, 0x20, 0x8f, 0xdd, 0x60, 0x28, 0x65,
0x52, 0xa7, 0x92, 0x9d, 0x0b, 0x66, 0x9c, 0xfb, 0x54, 0x83, 0xc6, 0x3e, 0xf6, 0x73, 0x2b, 0x66, 0x2a, 0xd9, 0x39, 0x63, 0xc6, 0xbd, 0x4f, 0x35, 0x68, 0xec, 0x63, 0x3f, 0xb7, 0x62, 0xb6, 0x4d,
0xcb, 0xb4, 0x89, 0x65, 0x62, 0xeb, 0xee, 0xc0, 0x0d, 0x43, 0xf4, 0x13, 0xf2, 0xca, 0x76, 0x2e, 0x9b, 0xd8, 0x26, 0x8e, 0xee, 0x0e, 0xdc, 0x30, 0x44, 0x3f, 0x01, 0xaf, 0x6c, 0xe7, 0x0c, 0x72,
0x20, 0x77, 0xa0, 0xd6, 0x8d, 0x7c, 0xdf, 0x09, 0xdd, 0x00, 0xe5, 0xf6, 0x35, 0x7b, 0x41, 0x08, 0x07, 0x6a, 0xdd, 0xc8, 0xf7, 0x9d, 0xd0, 0x0d, 0x50, 0x1e, 0x5f, 0xb3, 0x17, 0x04, 0xe3, 0x91,
0x1e, 0xb9, 0x01, 0x92, 0xfb, 0xb0, 0x38, 0x74, 0x59, 0x4c, 0x63, 0x1a, 0x85, 0x4e, 0xec, 0xf6, 0x1b, 0x20, 0xb9, 0x0f, 0x8b, 0x43, 0x97, 0xc5, 0x34, 0xa6, 0x51, 0xe8, 0xc4, 0x6e, 0xbf, 0x55,
0x5b, 0x25, 0xb9, 0xa0, 0x91, 0x09, 0x0f, 0xdc, 0xbe, 0xf5, 0x4c, 0x03, 0xf2, 0x75, 0xce, 0x69, 0x92, 0x1b, 0x1a, 0x19, 0xf3, 0xc0, 0xed, 0x5b, 0xcf, 0x34, 0x20, 0x5f, 0xe7, 0x9c, 0xf6, 0xc3,
0x3f, 0x2c, 0x28, 0x73, 0xad, 0xc4, 0x3f, 0x84, 0xa5, 0x21, 0x32, 0x27, 0x51, 0xdb, 0x61, 0x78, 0x82, 0x30, 0xd7, 0x0a, 0xfc, 0x43, 0x58, 0x1a, 0x22, 0x73, 0x12, 0xb1, 0x1d, 0x86, 0xc7, 0x2d,
0xdc, 0x32, 0xd6, 0x8c, 0xf5, 0xfa, 0xd6, 0xfd, 0x19, 0xbf, 0x9f, 0x54, 0xc5, 0x5e, 0x1c, 0x22, 0x63, 0xcd, 0x58, 0xaf, 0x6f, 0xdd, 0x9f, 0xf1, 0xfd, 0xa4, 0x28, 0xf6, 0xe2, 0x10, 0xd9, 0x8e,
0xdb, 0x51, 0x3f, 0xb5, 0xf1, 0xd8, 0xfa, 0x50, 0x83, 0x25, 0xf9, 0x5d, 0x69, 0x1d, 0x60, 0x28, 0xfa, 0xd4, 0xc6, 0x63, 0xeb, 0x43, 0x0d, 0x96, 0xe4, 0xef, 0x4a, 0xea, 0x00, 0x43, 0x09, 0x1d,
0xa9, 0xe3, 0x42, 0x94, 0x28, 0xab, 0x26, 0x17, 0x50, 0x37, 0xd5, 0x2a, 0x45, 0x42, 0x4b, 0x17, 0x17, 0xac, 0x44, 0x58, 0x45, 0x5c, 0x00, 0xdd, 0x54, 0xab, 0x14, 0x01, 0x2d, 0x5d, 0x04, 0x68,
0x11, 0x5a, 0x3e, 0x4f, 0x28, 0xb9, 0x0b, 0x75, 0x3c, 0x1d, 0x52, 0x86, 0x8e, 0xf0, 0x80, 0x56, 0xf9, 0x3c, 0xa0, 0xe4, 0x2e, 0xd4, 0xf1, 0x74, 0x48, 0x19, 0x3a, 0xc2, 0x03, 0x5a, 0x15, 0xe9,
0x45, 0x7a, 0x03, 0x28, 0xd1, 0x01, 0x0d, 0x26, 0x3d, 0xac, 0x7a, 0x69, 0x0f, 0xb3, 0x38, 0x2c, 0x0d, 0xa0, 0x58, 0x07, 0x34, 0x98, 0xf4, 0xb0, 0xea, 0xa5, 0x3d, 0xcc, 0xe2, 0xb0, 0x5c, 0xb0,
0x17, 0xac, 0x94, 0x78, 0xeb, 0x7b, 0x70, 0x73, 0x92, 0x59, 0x37, 0xa3, 0xa4, 0xa5, 0x49, 0x82, 0x52, 0xe2, 0xad, 0xef, 0xc1, 0xcd, 0x49, 0x64, 0xdd, 0x0c, 0x92, 0x96, 0x26, 0x01, 0xfe, 0xdc,
0x3f, 0x37, 0x8f, 0xe0, 0x9c, 0x40, 0x7b, 0x25, 0xe7, 0x38, 0x97, 0x5a, 0xff, 0xd1, 0xe0, 0xd6, 0x3c, 0x80, 0x73, 0x00, 0xed, 0x95, 0x1c, 0xe3, 0x9c, 0x6b, 0xfd, 0x47, 0x83, 0x5b, 0x3b, 0x0c,
0x0e, 0x43, 0x37, 0xc6, 0x9d, 0xc8, 0xf7, 0xb1, 0x2b, 0x20, 0xa6, 0x0e, 0xf2, 0x16, 0x2c, 0x04, 0xdd, 0x18, 0x77, 0x22, 0xdf, 0xc7, 0xae, 0x50, 0x31, 0x75, 0x90, 0xb7, 0x60, 0x21, 0xe0, 0x7d,
0xbc, 0xef, 0xc4, 0xe3, 0x21, 0x4a, 0xd6, 0x9b, 0x5b, 0xaf, 0xcf, 0x38, 0x6b, 0x97, 0xf7, 0x0f, 0x27, 0x1e, 0x0f, 0x51, 0xa2, 0xde, 0xdc, 0x7a, 0x7d, 0xc6, 0x5d, 0xbb, 0xbc, 0x7f, 0x30, 0x1e,
0xc6, 0x43, 0xb4, 0xab, 0x81, 0x1a, 0x10, 0x0b, 0x1a, 0xdd, 0x6c, 0xbf, 0x2c, 0x24, 0x14, 0x64, 0xa2, 0x5d, 0x0d, 0xd4, 0x82, 0x58, 0xd0, 0xe8, 0x66, 0xe7, 0x65, 0x29, 0xa1, 0xc0, 0x13, 0xd6,
0xc2, 0x3a, 0x0c, 0x8f, 0x3b, 0x6d, 0x69, 0x1d, 0xc3, 0x56, 0x93, 0xe2, 0x3d, 0x2b, 0x9d, 0xbd, 0x61, 0x78, 0xdc, 0x69, 0x4b, 0xeb, 0x18, 0xb6, 0x22, 0x8a, 0x71, 0x56, 0x3a, 0x1b, 0x67, 0x2d,
0x67, 0x2d, 0xa8, 0x0e, 0x59, 0x74, 0x3a, 0xee, 0xb4, 0xa5, 0x61, 0x0c, 0x3b, 0x9d, 0x92, 0x2f, 0xa8, 0x0e, 0x59, 0x74, 0x3a, 0xee, 0xb4, 0xa5, 0x61, 0x0c, 0x3b, 0x25, 0xc9, 0x97, 0xa1, 0xc2,
0x43, 0x85, 0x77, 0x07, 0x18, 0xb8, 0xd2, 0x1c, 0xf5, 0xad, 0xdb, 0x53, 0x29, 0xdf, 0xf6, 0xa3, 0xbb, 0x03, 0x0c, 0x5c, 0x69, 0x8e, 0xfa, 0xd6, 0xed, 0xa9, 0x90, 0x6f, 0xfb, 0xd1, 0xa1, 0x9d,
0x43, 0x3b, 0x59, 0x68, 0xfd, 0x44, 0x87, 0xd5, 0x36, 0x8b, 0x86, 0x9f, 0x70, 0xe4, 0xbb, 0xb0, 0x6c, 0xb4, 0x7e, 0xaa, 0xc3, 0x6a, 0x9b, 0x45, 0xc3, 0xff, 0x73, 0xcd, 0x77, 0x61, 0x29, 0x3f,
0x94, 0xef, 0xae, 0xbc, 0x5a, 0x51, 0xf0, 0xd9, 0xa2, 0xce, 0xc9, 0x0b, 0xb3, 0x91, 0xc3, 0x15, 0x5d, 0x79, 0xb5, 0x82, 0xe0, 0xb3, 0x45, 0x99, 0x93, 0x17, 0x66, 0x23, 0x57, 0x57, 0x78, 0xbc,
0x1e, 0x6f, 0x37, 0xbb, 0x85, 0xb9, 0xf5, 0x0f, 0x0d, 0x56, 0xde, 0x71, 0xf9, 0xb5, 0x92, 0x92, 0xdd, 0xec, 0x16, 0x68, 0xeb, 0x1f, 0x1a, 0xac, 0xbc, 0xe3, 0xf2, 0x6b, 0x05, 0x25, 0x53, 0x58,
0x01, 0xd6, 0x67, 0x02, 0x36, 0xe6, 0x00, 0x2e, 0x5d, 0x08, 0xb8, 0x7c, 0x05, 0xc0, 0x1f, 0x6a, 0x9f, 0xa9, 0xb0, 0x31, 0x47, 0xe1, 0xd2, 0x85, 0x0a, 0x97, 0xaf, 0xa0, 0xf0, 0x87, 0x1a, 0xdc,
0x70, 0xbb, 0x8d, 0xbc, 0xcb, 0xe8, 0x21, 0x7e, 0x7a, 0x50, 0xff, 0x42, 0x83, 0xd5, 0xfd, 0x41, 0x6e, 0x23, 0xef, 0x32, 0x7a, 0x88, 0x9f, 0x1e, 0xad, 0x7f, 0xa1, 0xc1, 0xea, 0xfe, 0x20, 0x3a,
0x74, 0xf2, 0xe2, 0x22, 0xb6, 0x7e, 0xab, 0xc3, 0x4d, 0x15, 0x9b, 0xf6, 0xd2, 0xe8, 0xfb, 0x31, 0x79, 0x71, 0x35, 0xb6, 0x7e, 0xab, 0xc3, 0x4d, 0x95, 0x9b, 0xf6, 0xd2, 0xec, 0xfb, 0x09, 0x05,
0x5d, 0xd0, 0x35, 0xa8, 0x67, 0x01, 0x3f, 0xbb, 0xa6, 0x93, 0xa2, 0x1c, 0x69, 0x69, 0x26, 0xd2, 0xe8, 0x1a, 0xd4, 0xb3, 0x84, 0x9f, 0x85, 0xe9, 0x24, 0x2b, 0xd7, 0xb4, 0x34, 0x53, 0xd3, 0xf2,
0xf2, 0x1c, 0xa4, 0x95, 0xa2, 0x6d, 0xbf, 0x09, 0xcd, 0xfc, 0xd5, 0x91, 0xa6, 0x55, 0xef, 0xc6, 0x1c, 0x4d, 0x2b, 0x45, 0xdb, 0x7e, 0x13, 0x9a, 0xf9, 0xab, 0x23, 0x4d, 0xab, 0xde, 0x8d, 0xfb,
0xfd, 0xe9, 0xa6, 0xcd, 0xe8, 0x90, 0x96, 0xcd, 0x1f, 0x2c, 0x69, 0xd8, 0x0f, 0x74, 0x58, 0x11, 0xd3, 0x4d, 0x9b, 0xc1, 0x21, 0x2d, 0x9b, 0x3f, 0x58, 0xd2, 0xb0, 0x1f, 0xe8, 0xb0, 0x22, 0xb2,
0x51, 0xed, 0x25, 0x67, 0x97, 0xe7, 0xec, 0xef, 0x1a, 0x2c, 0xbf, 0xe3, 0xf2, 0xeb, 0xa4, 0xec, 0xda, 0x4b, 0xcc, 0x2e, 0x8f, 0xd9, 0xdf, 0x35, 0x58, 0x7e, 0xc7, 0xe5, 0xd7, 0x09, 0xd9, 0xf5,
0x7a, 0x2f, 0xff, 0x79, 0xb0, 0xe5, 0xff, 0x19, 0xec, 0x3f, 0x35, 0x68, 0xa5, 0xf1, 0xee, 0xd3, 0x06, 0xff, 0x79, 0x65, 0xcb, 0x1f, 0x5b, 0xd9, 0x7f, 0x6a, 0xd0, 0x4a, 0xf3, 0xdd, 0xa7, 0x43,
0x81, 0x58, 0x3c, 0x69, 0x22, 0xd6, 0xbd, 0xb8, 0x68, 0xaf, 0x39, 0xb8, 0xff, 0x4b, 0x87, 0xc5, 0x63, 0xf1, 0xa4, 0x89, 0x5c, 0xf7, 0xe2, 0x6a, 0x7b, 0xcd, 0xc9, 0xfd, 0x5f, 0x3a, 0x2c, 0x76,
0x4e, 0xc8, 0x91, 0xc5, 0xcf, 0x0d, 0xe9, 0x1b, 0xe7, 0x35, 0x56, 0xc5, 0xc9, 0x19, 0x5d, 0x2e, 0x42, 0x8e, 0x2c, 0x7e, 0x6e, 0x9a, 0xbe, 0x71, 0x5e, 0x62, 0xd5, 0x9c, 0x9c, 0x91, 0xe5, 0x52,
0x55, 0xa2, 0x08, 0xde, 0x38, 0xf6, 0x45, 0x46, 0x9a, 0xe5, 0x37, 0xb9, 0xa0, 0x98, 0xe5, 0xab, 0x2d, 0x8a, 0xc0, 0x8d, 0x63, 0x5f, 0x54, 0xa4, 0x59, 0x7d, 0x93, 0x33, 0x8a, 0x55, 0xbe, 0x4a,
0x30, 0x30, 0x91, 0xe5, 0x4f, 0xb0, 0x5a, 0x2d, 0xb2, 0xfa, 0x3a, 0x40, 0x46, 0x3e, 0x6f, 0x2d, 0x03, 0x13, 0x55, 0xfe, 0x04, 0xaa, 0xd5, 0x22, 0xaa, 0xaf, 0x03, 0x64, 0xe0, 0xf3, 0xd6, 0xc2,
0xac, 0x19, 0x22, 0x4d, 0xcf, 0x25, 0xa2, 0x02, 0x62, 0xd1, 0x49, 0xa7, 0xcd, 0x5b, 0xb5, 0x35, 0x9a, 0x21, 0xca, 0xf4, 0x9c, 0x23, 0x3a, 0x20, 0x16, 0x9d, 0x74, 0xda, 0xbc, 0x55, 0x5b, 0x33,
0x43, 0x54, 0x40, 0x6a, 0x46, 0xbe, 0x02, 0x0b, 0x2c, 0x3a, 0x71, 0x7a, 0x6e, 0xec, 0xb6, 0x40, 0x44, 0x07, 0xa4, 0x28, 0xf2, 0x15, 0x58, 0x60, 0xd1, 0x89, 0xd3, 0x73, 0x63, 0xb7, 0x05, 0xb2,
0x26, 0xd9, 0x73, 0xb2, 0xc9, 0x2a, 0x8b, 0x4e, 0xda, 0x6e, 0xec, 0x5a, 0x4f, 0x75, 0x58, 0x6c, 0xc8, 0x9e, 0x53, 0x4d, 0x56, 0x59, 0x74, 0xd2, 0x76, 0x63, 0xd7, 0x7a, 0xaa, 0xc3, 0x62, 0x1b,
0xa3, 0x8f, 0x31, 0xfe, 0xff, 0x49, 0x2f, 0x30, 0x56, 0x9a, 0xc3, 0x58, 0x79, 0x1e, 0x63, 0x95, 0x7d, 0x8c, 0xf1, 0x7f, 0x0f, 0x7a, 0x01, 0xb1, 0xd2, 0x1c, 0xc4, 0xca, 0xf3, 0x10, 0xab, 0x9c,
0x73, 0x8c, 0xdd, 0x83, 0xc6, 0x90, 0xd1, 0xc0, 0x65, 0x63, 0xc7, 0xc3, 0xb1, 0x28, 0x6f, 0x0c, 0x43, 0xec, 0x1e, 0x34, 0x86, 0x8c, 0x06, 0x2e, 0x1b, 0x3b, 0x1e, 0x8e, 0x45, 0x7b, 0x63, 0xc8,
0x19, 0xe5, 0x95, 0xec, 0x21, 0x8e, 0xb9, 0xf5, 0x91, 0x06, 0x8b, 0xfb, 0xe8, 0xb2, 0xee, 0xe0, 0x2c, 0xaf, 0x78, 0x0f, 0x71, 0xcc, 0xad, 0x8f, 0x34, 0x58, 0xdc, 0x47, 0x97, 0x75, 0x07, 0xcf,
0xb9, 0xd1, 0x30, 0xa1, 0xbf, 0x51, 0xd4, 0x7f, 0x7e, 0x0e, 0xfd, 0x79, 0x30, 0x19, 0xf2, 0x91, 0x0d, 0x86, 0x09, 0xf9, 0x8d, 0xa2, 0xfc, 0xf3, 0x6b, 0xe8, 0xcf, 0x83, 0xc9, 0x90, 0x8f, 0xfc,
0x1f, 0x3b, 0x39, 0x39, 0x8a, 0x80, 0x25, 0x25, 0xdf, 0xc9, 0x28, 0xda, 0x84, 0xf2, 0xf1, 0x08, 0xd8, 0xc9, 0xc1, 0x51, 0x00, 0x2c, 0x29, 0xfe, 0x4e, 0x06, 0xd1, 0x26, 0x94, 0x8f, 0x47, 0xc8,
0xd9, 0xf8, 0xe2, 0x6a, 0x42, 0xad, 0xb3, 0xfe, 0xaa, 0x81, 0xb9, 0x3f, 0xe6, 0x3b, 0x51, 0x78, 0xc6, 0x17, 0x77, 0x13, 0x6a, 0x9f, 0xf5, 0x57, 0x0d, 0xcc, 0xfd, 0x31, 0xdf, 0x89, 0xc2, 0x23,
0x44, 0xfb, 0x2f, 0x1c, 0x72, 0x02, 0x25, 0x69, 0xaf, 0xf2, 0x9a, 0xb1, 0x5e, 0xb3, 0xe5, 0x58, 0xda, 0x7f, 0xe1, 0x34, 0x27, 0x50, 0x92, 0xf6, 0x2a, 0xaf, 0x19, 0xeb, 0x35, 0x5b, 0xae, 0x85,
0xd8, 0xd2, 0xc3, 0xb1, 0x33, 0x64, 0x78, 0x44, 0x4f, 0x51, 0x59, 0xbb, 0x66, 0xd7, 0x3d, 0x1c, 0x2d, 0x3d, 0x1c, 0x3b, 0x43, 0x86, 0x47, 0xf4, 0x14, 0x95, 0xb5, 0x6b, 0x76, 0xdd, 0xc3, 0xf1,
0xef, 0x25, 0x22, 0xeb, 0x99, 0x0e, 0x8d, 0xd4, 0x96, 0x82, 0x9f, 0xab, 0x00, 0xca, 0x6b, 0x62, 0x5e, 0xc2, 0xb2, 0x9e, 0xe9, 0xd0, 0x48, 0x6d, 0x29, 0xf0, 0xb9, 0x8a, 0x42, 0x79, 0x4f, 0xac,
0xfd, 0xf2, 0x5d, 0x97, 0xe9, 0x95, 0xd2, 0xec, 0x38, 0x7a, 0x0f, 0x1a, 0xd2, 0x1c, 0x4e, 0x18, 0x5f, 0x7e, 0xea, 0x32, 0xbd, 0x53, 0x9a, 0x9d, 0x47, 0xef, 0x41, 0x43, 0x9a, 0xc3, 0x09, 0xa3,
0xf5, 0x30, 0xb3, 0x6e, 0x5d, 0xca, 0x1e, 0x49, 0x51, 0x91, 0xa8, 0xca, 0x65, 0x5c, 0xa4, 0x3a, 0x1e, 0x66, 0xd6, 0xad, 0x4b, 0xde, 0x23, 0xc9, 0x2a, 0x02, 0x55, 0xb9, 0x8c, 0x8b, 0x54, 0xa7,
0xdd, 0x45, 0x08, 0x94, 0x06, 0x34, 0x56, 0x71, 0xa5, 0x61, 0xcb, 0xb1, 0xf5, 0x3d, 0xa8, 0x1f, 0xbb, 0x08, 0x81, 0xd2, 0x80, 0xc6, 0x2a, 0xaf, 0x34, 0x6c, 0xb9, 0xb6, 0xbe, 0x07, 0xf5, 0x03,
0xd0, 0x00, 0x0f, 0x68, 0xd7, 0xdb, 0xe5, 0xfd, 0xab, 0xd0, 0x95, 0x77, 0x67, 0xf4, 0x42, 0x77, 0x1a, 0xe0, 0x01, 0xed, 0x7a, 0xbb, 0xbc, 0x7f, 0x15, 0xb8, 0xf2, 0xe9, 0x8c, 0x5e, 0x98, 0xce,
0x66, 0xee, 0x0b, 0x63, 0x7d, 0x5f, 0x83, 0x85, 0xb7, 0xfd, 0x11, 0x1f, 0x5c, 0xf1, 0xf4, 0x42, 0xcc, 0x7d, 0x61, 0xac, 0xef, 0x6b, 0xb0, 0xf0, 0xb6, 0x3f, 0xe2, 0x83, 0x2b, 0xde, 0x5e, 0xc8,
0x3c, 0xd6, 0xa7, 0xc4, 0xe3, 0x39, 0x3a, 0xfc, 0x4c, 0x83, 0xea, 0x43, 0x1c, 0x6f, 0xed, 0x63, 0xc7, 0xfa, 0x94, 0x7c, 0x3c, 0x47, 0x86, 0x9f, 0x6b, 0x50, 0x7d, 0x88, 0xe3, 0xad, 0x7d, 0xec,
0x5f, 0xda, 0x4f, 0xc4, 0xd4, 0xb4, 0x6b, 0x23, 0x27, 0xe4, 0x2e, 0xd4, 0x27, 0xa2, 0x48, 0xb2, 0x4b, 0xfb, 0x89, 0x9c, 0x9a, 0x4e, 0x6d, 0x24, 0x41, 0xee, 0x42, 0x7d, 0x22, 0x8b, 0x24, 0xe7,
0x3f, 0xe4, 0x41, 0xe4, 0x82, 0x67, 0xf4, 0x36, 0x2c, 0x50, 0xee, 0x3c, 0x76, 0x7d, 0xda, 0x93, 0x43, 0x9e, 0x44, 0x2e, 0x78, 0x46, 0x6f, 0xc3, 0x02, 0xe5, 0xce, 0x63, 0xd7, 0xa7, 0x3d, 0x69,
0xf6, 0x5f, 0xb0, 0xab, 0x94, 0xbf, 0x2b, 0xa6, 0x22, 0x7e, 0x65, 0x6a, 0x2a, 0x6f, 0x37, 0xec, 0xff, 0x05, 0xbb, 0x4a, 0xf9, 0xbb, 0x82, 0x14, 0xf9, 0x2b, 0x13, 0x53, 0x79, 0xbb, 0x61, 0x4f,
0x09, 0x89, 0xf5, 0x1e, 0x40, 0xa2, 0x9a, 0x20, 0x28, 0xf3, 0x2e, 0x6d, 0xd2, 0xbb, 0xbe, 0x0a, 0x70, 0xac, 0xf7, 0x00, 0x12, 0xd1, 0x04, 0x40, 0x99, 0x77, 0x69, 0x93, 0xde, 0xf5, 0x55, 0xa8,
0x55, 0x0f, 0xc7, 0x5b, 0x1c, 0xfb, 0x2d, 0x5d, 0x06, 0xff, 0x59, 0xac, 0x25, 0x3b, 0xd9, 0xe9, 0x7a, 0x38, 0xde, 0xe2, 0xd8, 0x6f, 0xe9, 0x32, 0xf9, 0xcf, 0x42, 0x2d, 0x39, 0xc9, 0x4e, 0xb7,
0x72, 0xeb, 0xe7, 0x1a, 0xd4, 0xbe, 0x15, 0xb9, 0xbd, 0x4e, 0xd8, 0xc3, 0xd3, 0xe7, 0x47, 0x7f, 0x5b, 0x3f, 0xd0, 0xa1, 0xf6, 0xad, 0xc8, 0xed, 0x75, 0xc2, 0x1e, 0x9e, 0x3e, 0x57, 0xf8, 0x8f,
0x0b, 0xaa, 0x47, 0x14, 0xfd, 0x5e, 0x1e, 0x04, 0x92, 0xa9, 0x20, 0x96, 0x8a, 0xb3, 0x9d, 0xa1, 0x28, 0xfa, 0xbd, 0x47, 0x79, 0xfe, 0xcf, 0x19, 0x22, 0x38, 0x24, 0x91, 0x07, 0x47, 0x42, 0x0a,
0x1b, 0x0f, 0x78, 0xab, 0x24, 0x6f, 0x34, 0x48, 0xd1, 0x9e, 0x90, 0x58, 0x4f, 0x35, 0x00, 0xa9, 0xd8, 0xa9, 0x90, 0xcc, 0x19, 0xba, 0xf1, 0x20, 0xcd, 0x05, 0x20, 0x59, 0x7b, 0x82, 0x43, 0xda,
0x9d, 0xb8, 0x67, 0x9c, 0xb4, 0xa1, 0x91, 0xae, 0x67, 0x6e, 0xc0, 0x93, 0x8e, 0xd2, 0xbd, 0xa9, 0xd0, 0x48, 0x37, 0x30, 0x37, 0x50, 0x19, 0xa1, 0xbe, 0x75, 0x6f, 0x6a, 0xa0, 0x3e, 0xc4, 0xf1,
0x37, 0xf3, 0x21, 0x8e, 0xdf, 0x75, 0xfd, 0x11, 0xee, 0xb9, 0x94, 0xd9, 0xf5, 0x64, 0x4f, 0xf1, 0xbb, 0xae, 0x3f, 0xc2, 0x3d, 0x97, 0x32, 0xbb, 0x9e, 0x1c, 0x22, 0xbe, 0xb2, 0x9e, 0x6a, 0x00,
0x2b, 0xf2, 0x25, 0x58, 0x09, 0x47, 0x81, 0xc3, 0xd0, 0x77, 0x63, 0xec, 0x39, 0x89, 0xa2, 0x3c, 0x12, 0x01, 0x11, 0xcb, 0xe7, 0x0f, 0xd5, 0x3e, 0xce, 0xa1, 0xe4, 0x4b, 0xb0, 0x12, 0x8e, 0x02,
0x51, 0x9c, 0x84, 0xa3, 0xc0, 0x56, 0x9f, 0xf6, 0x93, 0x2f, 0xd6, 0x0f, 0x34, 0x80, 0xb7, 0x85, 0x87, 0xa1, 0xef, 0xc6, 0xd8, 0x73, 0x12, 0x30, 0x78, 0x02, 0x0e, 0x09, 0x47, 0x81, 0xad, 0x7e,
0xce, 0x4a, 0x8d, 0xb3, 0xe5, 0x85, 0x36, 0xa5, 0xbc, 0x98, 0x00, 0xad, 0x17, 0x41, 0x6f, 0xa7, 0xda, 0x4f, 0x7e, 0xb1, 0x7e, 0xa8, 0x01, 0xbc, 0x2d, 0x34, 0x57, 0x62, 0x9c, 0x6d, 0x61, 0xb4,
0xa0, 0x45, 0xcc, 0xe0, 0x49, 0xdb, 0xf1, 0xde, 0x0c, 0xaa, 0x73, 0xf0, 0x09, 0x2f, 0x72, 0x6c, 0x29, 0x2d, 0xcc, 0x04, 0x74, 0x7a, 0x11, 0xba, 0xed, 0x14, 0x3a, 0x91, 0x97, 0x78, 0x32, 0xda,
0xfd, 0x54, 0x75, 0x6a, 0x85, 0x76, 0x4a, 0xa5, 0x82, 0x05, 0xb4, 0xb3, 0x16, 0xb8, 0x0b, 0xf5, 0xbc, 0x37, 0xc3, 0x9c, 0xb9, 0xf2, 0x09, 0xba, 0x72, 0x6d, 0xfd, 0x4c, 0x4d, 0x83, 0x85, 0x74,
0x00, 0x83, 0x88, 0x8d, 0x1d, 0x4e, 0x9f, 0x60, 0xea, 0xc0, 0x4a, 0xb4, 0x4f, 0x9f, 0xa0, 0x70, 0x4a, 0xa4, 0x82, 0x95, 0xb5, 0xb3, 0x56, 0xbe, 0x0b, 0xf5, 0x00, 0x83, 0x88, 0x8d, 0x1d, 0x4e,
0x51, 0x49, 0x49, 0x74, 0xc2, 0x53, 0x1b, 0x09, 0x1a, 0xa2, 0x13, 0x4e, 0xbe, 0x00, 0x37, 0x18, 0x9f, 0x60, 0x1a, 0x24, 0x8a, 0xb5, 0x4f, 0x9f, 0xa0, 0x08, 0x03, 0x09, 0x49, 0x74, 0xc2, 0xd3,
0x76, 0x31, 0x8c, 0xfd, 0xb1, 0x13, 0x44, 0x3d, 0x7a, 0x44, 0x31, 0x75, 0x63, 0x33, 0xfd, 0xb0, 0xc7, 0x40, 0xc0, 0x10, 0x9d, 0x70, 0xf2, 0x05, 0xb8, 0xc1, 0xb0, 0x8b, 0x61, 0xec, 0x8f, 0x9d,
0x9b, 0xc8, 0xad, 0xbf, 0x69, 0xd0, 0xfc, 0x76, 0x1a, 0xbc, 0x94, 0x66, 0xcf, 0x21, 0xa6, 0x7c, 0x20, 0xea, 0xd1, 0x23, 0x8a, 0x69, 0xa8, 0x98, 0xe9, 0x0f, 0xbb, 0x09, 0xdf, 0xfa, 0x9b, 0x06,
0x4d, 0x82, 0x2d, 0xf0, 0x37, 0xa7, 0x6d, 0x9b, 0x91, 0x64, 0x2f, 0x70, 0xec, 0x2b, 0xa5, 0xb6, 0xcd, 0x6f, 0xa7, 0x09, 0x52, 0x49, 0xf6, 0x1c, 0xf2, 0xd6, 0xd7, 0xa4, 0xb2, 0x05, 0xfc, 0xe6,
0xa1, 0x2e, 0xcd, 0x91, 0xec, 0x51, 0x9a, 0x6b, 0x83, 0xdc, 0xf2, 0x36, 0x1c, 0x65, 0x63, 0xeb, 0x8c, 0x86, 0x33, 0x90, 0xec, 0x05, 0x8e, 0x7d, 0x25, 0xd4, 0x36, 0xd4, 0xa5, 0x39, 0x92, 0x33,
0x97, 0x3a, 0x10, 0x55, 0xee, 0x4b, 0x23, 0xbd, 0x70, 0x39, 0xfa, 0x1b, 0xd3, 0x73, 0xf4, 0xf3, 0x4a, 0x73, 0x6d, 0x90, 0x5b, 0xde, 0x86, 0xa3, 0x6c, 0x6d, 0xfd, 0x52, 0x07, 0xa2, 0x46, 0x0a,
0xc9, 0xd7, 0x67, 0x40, 0xc1, 0xca, 0x7b, 0x71, 0x35, 0xbb, 0x26, 0x25, 0xf2, 0x73, 0x1b, 0x1a, 0xd2, 0x48, 0x2f, 0x5c, 0x1f, 0xf0, 0xc6, 0xf4, 0x3e, 0xe0, 0x7c, 0x81, 0xf7, 0x19, 0x50, 0x6a,
0x78, 0x1a, 0x33, 0x37, 0xbd, 0x74, 0xd5, 0x4b, 0x5f, 0x3a, 0xf9, 0x33, 0x75, 0xe9, 0xac, 0xf7, 0xe5, 0xf3, 0xbe, 0x42, 0x12, 0x68, 0x43, 0x03, 0x4f, 0x63, 0xe6, 0xa6, 0x41, 0x57, 0xbd, 0x74,
0x45, 0x99, 0x9f, 0x54, 0x71, 0x2f, 0xf9, 0xba, 0x98, 0xaf, 0xdf, 0xe9, 0xf0, 0x5a, 0x81, 0xaf, 0xd0, 0xc9, 0xcf, 0x92, 0x48, 0x7e, 0x5f, 0x87, 0x95, 0xb4, 0x53, 0x7c, 0x89, 0xd7, 0xc5, 0x78,
0x3d, 0x16, 0xf5, 0x19, 0x72, 0xfe, 0x92, 0xb7, 0x79, 0xbc, 0xbd, 0xf9, 0x17, 0x03, 0xaa, 0x09, 0xfd, 0x4e, 0x87, 0xd7, 0x0a, 0x78, 0xed, 0xb1, 0xa8, 0xcf, 0x90, 0xf3, 0x97, 0xb8, 0xcd, 0xc3,
0x60, 0x52, 0x83, 0xb2, 0xf7, 0x28, 0x0a, 0xd1, 0x7c, 0x85, 0xac, 0xc2, 0x0d, 0xef, 0xec, 0xff, 0xed, 0xcd, 0xbf, 0x18, 0x50, 0x4d, 0x14, 0x26, 0x35, 0x28, 0x7b, 0x8f, 0xa2, 0x10, 0xcd, 0x57,
0x06, 0x66, 0x8f, 0x2c, 0xc3, 0x92, 0x57, 0x6c, 0xa9, 0x9b, 0x48, 0x08, 0x34, 0xbd, 0x42, 0x47, 0xc8, 0x2a, 0xdc, 0xf0, 0xce, 0xfe, 0x37, 0x61, 0xf6, 0xc8, 0x32, 0x2c, 0x79, 0xc5, 0xb1, 0xbd,
0xd9, 0x3c, 0x22, 0xb7, 0x60, 0xd9, 0x3b, 0xdf, 0x74, 0x35, 0xc5, 0x9b, 0x6c, 0x7a, 0xc5, 0xbe, 0x89, 0x84, 0x40, 0xd3, 0x2b, 0x4c, 0xad, 0xcd, 0x23, 0x72, 0x0b, 0x96, 0xbd, 0xf3, 0x83, 0x5d,
0x24, 0x37, 0x07, 0x72, 0x8b, 0x6f, 0x60, 0x9c, 0x25, 0xd8, 0xdc, 0xa4, 0x64, 0x15, 0x4c, 0xef, 0x53, 0xbc, 0xfb, 0xa6, 0x57, 0x9c, 0x7d, 0x72, 0x73, 0x20, 0x8f, 0xf8, 0x06, 0xc6, 0x59, 0x11,
0x4c, 0x7b, 0xd0, 0xfc, 0xbd, 0x46, 0x96, 0xa1, 0xe9, 0x15, 0xfa, 0x5f, 0xe6, 0x1f, 0x34, 0x42, 0xcf, 0x4d, 0x4a, 0x56, 0xc1, 0xf4, 0xce, 0x8c, 0x20, 0xcd, 0xdf, 0x6b, 0x64, 0x19, 0x9a, 0x5e,
0x60, 0xd1, 0x9b, 0x6c, 0xf0, 0x98, 0x7f, 0xd4, 0xc8, 0x2d, 0x20, 0xde, 0xb9, 0x3e, 0x88, 0xf9, 0x61, 0xc6, 0x66, 0xfe, 0x41, 0x23, 0x04, 0x16, 0xbd, 0xc9, 0x21, 0x92, 0xf9, 0x47, 0x8d, 0xdc,
0x27, 0x8d, 0xac, 0xc0, 0x92, 0x57, 0x68, 0x17, 0x70, 0xf3, 0xcf, 0x1a, 0xb9, 0x01, 0x0d, 0x6f, 0x02, 0xe2, 0x9d, 0x9b, 0xb5, 0x98, 0x7f, 0xd2, 0xc8, 0x0a, 0x2c, 0x79, 0x85, 0x91, 0x04, 0x37,
0x22, 0x3c, 0x99, 0xbf, 0xd2, 0xd5, 0x51, 0x93, 0x3e, 0x65, 0xbe, 0xaf, 0x93, 0x3b, 0x70, 0xd3, 0xff, 0xac, 0x91, 0x1b, 0xd0, 0xf0, 0x26, 0xd2, 0x93, 0xf9, 0x2b, 0x5d, 0x5d, 0x35, 0xe9, 0x53,
0x9b, 0xea, 0x68, 0xe6, 0x33, 0x9d, 0x34, 0xa0, 0xea, 0xa9, 0xc2, 0xdc, 0xfc, 0xa1, 0x21, 0x67, 0xe6, 0xfb, 0x3a, 0xb9, 0x03, 0x37, 0xbd, 0xa9, 0x8e, 0x66, 0x3e, 0xd3, 0x49, 0x03, 0xaa, 0x9e,
0xaa, 0x62, 0x34, 0x7f, 0x64, 0x90, 0x3a, 0x54, 0x3c, 0x99, 0xbb, 0x99, 0x3f, 0x56, 0x9f, 0x54, 0x6a, 0xfe, 0xcd, 0x1f, 0x19, 0x92, 0x52, 0x5d, 0xa9, 0xf9, 0x63, 0x83, 0xd4, 0xa1, 0xe2, 0xc9,
0xe6, 0x6d, 0x7e, 0x64, 0x48, 0xf5, 0x27, 0xf3, 0x70, 0xf3, 0xdf, 0x06, 0x69, 0x42, 0xcd, 0x4b, 0xfa, 0xd0, 0xfc, 0x89, 0xfa, 0x49, 0x55, 0xf7, 0xe6, 0x47, 0x86, 0x14, 0x7f, 0xb2, 0xd6, 0x37,
0x73, 0x4d, 0xf3, 0xd7, 0x35, 0xa9, 0x75, 0xf1, 0xa9, 0x30, 0x3f, 0xa8, 0x91, 0x25, 0x00, 0x2f, 0xff, 0x6d, 0x90, 0x26, 0xd4, 0xbc, 0xb4, 0x9e, 0x35, 0x7f, 0x5d, 0x93, 0x52, 0x17, 0x9f, 0x0a,
0x4b, 0x49, 0xcc, 0xdf, 0xd4, 0xde, 0x7c, 0x0b, 0x16, 0xd2, 0xbf, 0xee, 0x08, 0x40, 0x65, 0xd7, 0xf3, 0x83, 0x1a, 0x59, 0x02, 0xf0, 0xb2, 0xb2, 0xc7, 0xfc, 0x4d, 0xed, 0xcd, 0xb7, 0x60, 0x21,
0xe5, 0x31, 0x32, 0xf3, 0x15, 0x31, 0xb6, 0xd1, 0xed, 0x21, 0x33, 0x35, 0x31, 0xfe, 0x0e, 0xa3, 0xfd, 0x7b, 0x90, 0x00, 0x54, 0x76, 0x5d, 0x1e, 0x23, 0x33, 0x5f, 0x11, 0x6b, 0x1b, 0xdd, 0x1e,
0x42, 0xae, 0x0b, 0x9b, 0xef, 0x09, 0xc7, 0x34, 0x8d, 0xed, 0xf6, 0x77, 0xb7, 0xfb, 0x34, 0x1e, 0x32, 0x53, 0x13, 0xeb, 0xef, 0x30, 0x2a, 0xf8, 0xba, 0xb0, 0xf9, 0x9e, 0x70, 0x4c, 0xd3, 0xd8,
0x8c, 0x0e, 0x85, 0xd7, 0x6c, 0x3e, 0xa1, 0xbe, 0x4f, 0x9f, 0xc4, 0xd8, 0x1d, 0x6c, 0x2a, 0x8f, 0x6e, 0x7f, 0x77, 0xbb, 0x4f, 0xe3, 0xc1, 0xe8, 0x50, 0x78, 0xcd, 0xe6, 0x13, 0xea, 0xfb, 0xf4,
0xfa, 0x62, 0x8f, 0xf2, 0x98, 0xd1, 0xc3, 0x51, 0x8c, 0xbd, 0xcd, 0xf4, 0xb2, 0x6c, 0x4a, 0x37, 0x49, 0x8c, 0xdd, 0xc1, 0xa6, 0xf2, 0xa8, 0x2f, 0xf6, 0x28, 0x8f, 0x19, 0x3d, 0x1c, 0xc5, 0xd8,
0xcb, 0xa6, 0xc3, 0xc3, 0xc3, 0x8a, 0x94, 0x3c, 0xf8, 0x6f, 0x00, 0x00, 0x00, 0xff, 0xff, 0x40, 0xdb, 0x4c, 0x83, 0x65, 0x53, 0xba, 0x59, 0x46, 0x0e, 0x0f, 0x0f, 0x2b, 0x92, 0xf3, 0xe0, 0xbf,
0xb8, 0xbd, 0x0b, 0x2b, 0x1f, 0x00, 0x00, 0x01, 0x00, 0x00, 0xff, 0xff, 0xb2, 0xdd, 0xfb, 0x16, 0x8f, 0x1f, 0x00, 0x00,
} }

View File

@ -17,3 +17,8 @@ message SegmentFlushMeta{
uint64 close_time = 4; uint64 close_time = 4;
repeated FieldFlushMeta fields = 5; repeated FieldFlushMeta fields = 5;
} }
message DDLFlushMeta {
int64 collectionID = 1;
repeated string binlog_paths = 2;
}

View File

@ -139,31 +139,81 @@ func (m *SegmentFlushMeta) GetFields() []*FieldFlushMeta {
return nil return nil
} }
type DDLFlushMeta struct {
CollectionID int64 `protobuf:"varint,1,opt,name=collectionID,proto3" json:"collectionID,omitempty"`
BinlogPaths []string `protobuf:"bytes,2,rep,name=binlog_paths,json=binlogPaths,proto3" json:"binlog_paths,omitempty"`
XXX_NoUnkeyedLiteral struct{} `json:"-"`
XXX_unrecognized []byte `json:"-"`
XXX_sizecache int32 `json:"-"`
}
func (m *DDLFlushMeta) Reset() { *m = DDLFlushMeta{} }
func (m *DDLFlushMeta) String() string { return proto.CompactTextString(m) }
func (*DDLFlushMeta) ProtoMessage() {}
func (*DDLFlushMeta) Descriptor() ([]byte, []int) {
return fileDescriptor_8ec4781d562c3e8f, []int{2}
}
func (m *DDLFlushMeta) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_DDLFlushMeta.Unmarshal(m, b)
}
func (m *DDLFlushMeta) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
return xxx_messageInfo_DDLFlushMeta.Marshal(b, m, deterministic)
}
func (m *DDLFlushMeta) XXX_Merge(src proto.Message) {
xxx_messageInfo_DDLFlushMeta.Merge(m, src)
}
func (m *DDLFlushMeta) XXX_Size() int {
return xxx_messageInfo_DDLFlushMeta.Size(m)
}
func (m *DDLFlushMeta) XXX_DiscardUnknown() {
xxx_messageInfo_DDLFlushMeta.DiscardUnknown(m)
}
var xxx_messageInfo_DDLFlushMeta proto.InternalMessageInfo
func (m *DDLFlushMeta) GetCollectionID() int64 {
if m != nil {
return m.CollectionID
}
return 0
}
func (m *DDLFlushMeta) GetBinlogPaths() []string {
if m != nil {
return m.BinlogPaths
}
return nil
}
func init() { func init() {
proto.RegisterType((*FieldFlushMeta)(nil), "milvus.proto.service.FieldFlushMeta") proto.RegisterType((*FieldFlushMeta)(nil), "milvus.proto.service.FieldFlushMeta")
proto.RegisterType((*SegmentFlushMeta)(nil), "milvus.proto.service.SegmentFlushMeta") proto.RegisterType((*SegmentFlushMeta)(nil), "milvus.proto.service.SegmentFlushMeta")
proto.RegisterType((*DDLFlushMeta)(nil), "milvus.proto.service.DDLFlushMeta")
} }
func init() { proto.RegisterFile("write_node.proto", fileDescriptor_8ec4781d562c3e8f) } func init() { proto.RegisterFile("write_node.proto", fileDescriptor_8ec4781d562c3e8f) }
var fileDescriptor_8ec4781d562c3e8f = []byte{ var fileDescriptor_8ec4781d562c3e8f = []byte{
// 287 bytes of a gzipped FileDescriptorProto // 314 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x6c, 0x90, 0xc1, 0x4b, 0xc3, 0x30, 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x8c, 0x91, 0x41, 0x4b, 0xfb, 0x40,
0x14, 0xc6, 0xe9, 0xba, 0xcd, 0x35, 0x13, 0x19, 0xc5, 0x43, 0x40, 0x85, 0x3a, 0x3c, 0xf4, 0x62, 0x10, 0xc5, 0x49, 0xd3, 0xf6, 0xdf, 0x6c, 0xcb, 0x9f, 0x12, 0x3c, 0x04, 0x54, 0x88, 0xc1, 0x43,
0x0b, 0x7a, 0xf5, 0x20, 0x53, 0x06, 0x1e, 0x06, 0x12, 0x3d, 0x79, 0x29, 0x6b, 0xfb, 0x6c, 0x1f, 0x2e, 0x26, 0xa0, 0x57, 0x0f, 0x52, 0x4b, 0x41, 0xb0, 0x20, 0x51, 0x2f, 0x5e, 0x42, 0x93, 0x8c,
0xa4, 0x49, 0x49, 0x5e, 0x27, 0xec, 0x1f, 0xf4, 0xdf, 0x92, 0xa6, 0x93, 0x21, 0x78, 0xfc, 0x7e, 0xcd, 0xc0, 0x66, 0x37, 0xec, 0x4e, 0x2a, 0xf4, 0x0b, 0xfa, 0xb5, 0x24, 0x9b, 0x6a, 0x28, 0x78,
0x1f, 0xef, 0x47, 0xf2, 0xb1, 0xc5, 0x97, 0x41, 0x82, 0x4c, 0xe9, 0x12, 0x92, 0xd6, 0x68, 0xd2, 0xf0, 0xf8, 0x7e, 0x8f, 0x19, 0xde, 0x9b, 0x61, 0xf3, 0x0f, 0x85, 0x04, 0xa9, 0x90, 0x05, 0x44,
0xe1, 0x79, 0x83, 0x72, 0xd7, 0xd9, 0x21, 0x25, 0x16, 0xcc, 0x0e, 0x0b, 0x58, 0x6e, 0xd8, 0xd9, 0xb5, 0x92, 0x24, 0xdd, 0x93, 0x0a, 0xf9, 0xae, 0xd1, 0x9d, 0x8a, 0x34, 0xa8, 0x1d, 0xe6, 0x10,
0x1a, 0x41, 0x96, 0x6b, 0xd9, 0xd9, 0x7a, 0x03, 0xb4, 0x0d, 0x39, 0x3b, 0xf9, 0xec, 0xc9, 0xcb, 0xac, 0xd9, 0xff, 0x15, 0x02, 0x2f, 0x56, 0xbc, 0xd1, 0xe5, 0x1a, 0x68, 0xe3, 0x7a, 0xec, 0xdf,
0x33, 0xf7, 0x22, 0x2f, 0x9e, 0x88, 0xdf, 0x18, 0x5e, 0xb3, 0xd3, 0x1c, 0x95, 0xd4, 0x55, 0xd6, 0x7b, 0x4b, 0x1e, 0x96, 0x9e, 0xe5, 0x5b, 0xe1, 0x28, 0xf9, 0x96, 0xee, 0x05, 0x9b, 0x65, 0x28,
0x6e, 0xa9, 0xb6, 0x7c, 0x14, 0xf9, 0x71, 0x20, 0xe6, 0x03, 0x7b, 0xed, 0xd1, 0xf2, 0xdb, 0x63, 0xb8, 0xdc, 0xa6, 0xf5, 0x86, 0x4a, 0xed, 0x0d, 0x7c, 0x3b, 0x74, 0x92, 0x69, 0xc7, 0x9e, 0x5a,
0x8b, 0x37, 0xa8, 0x1a, 0x50, 0x74, 0x34, 0x5e, 0xb2, 0xc0, 0x0e, 0xec, 0xe0, 0xf4, 0xc5, 0x11, 0x14, 0x7c, 0x5a, 0x6c, 0xfe, 0x0c, 0xdb, 0x0a, 0x04, 0xf5, 0x1b, 0xcf, 0x98, 0xa3, 0x3b, 0x76,
0x84, 0x17, 0x2c, 0x40, 0x9b, 0x15, 0x52, 0x5b, 0x28, 0xf9, 0x28, 0xf2, 0xe2, 0x99, 0x98, 0xa1, 0xd8, 0x69, 0x27, 0x3d, 0x70, 0x4f, 0x99, 0x83, 0x3a, 0xcd, 0xb9, 0xd4, 0x50, 0x78, 0x03, 0xdf,
0x7d, 0x72, 0xb9, 0x2f, 0x75, 0x0b, 0x2a, 0x23, 0x6c, 0x80, 0xfb, 0x91, 0x17, 0x8f, 0xc5, 0xac, 0x0a, 0x27, 0xc9, 0x04, 0xf5, 0xbd, 0xd1, 0xad, 0x29, 0x6b, 0x10, 0x29, 0x61, 0x05, 0x9e, 0xed,
0x07, 0xef, 0xd8, 0x40, 0x78, 0xc5, 0x98, 0x3b, 0x1b, 0xda, 0xb1, 0x6b, 0x03, 0x47, 0x5c, 0xfd, 0x5b, 0xe1, 0x30, 0x99, 0xb4, 0xe0, 0x05, 0x2b, 0x70, 0xcf, 0x19, 0x33, 0x63, 0x9d, 0x3b, 0x34,
0xc0, 0xa6, 0xee, 0xe5, 0x96, 0x4f, 0x22, 0x3f, 0x9e, 0xdf, 0xdd, 0x24, 0xff, 0x2d, 0x90, 0xfc, 0xae, 0x63, 0x88, 0xb1, 0x6f, 0xd9, 0xd8, 0x24, 0xd7, 0xde, 0xc8, 0xb7, 0xc3, 0xe9, 0xf5, 0x65,
0xfd, 0xbe, 0x38, 0xdc, 0xac, 0x56, 0x1f, 0x8f, 0x15, 0x52, 0xdd, 0xe5, 0x49, 0xa1, 0x9b, 0x74, 0xf4, 0xdb, 0x05, 0xa2, 0xe3, 0xfa, 0xc9, 0x61, 0x26, 0x78, 0x65, 0xb3, 0xe5, 0xf2, 0xb1, 0x2f,
0x8f, 0x52, 0xe2, 0x9e, 0xa0, 0xa8, 0xd3, 0x41, 0x72, 0x5b, 0xa2, 0x25, 0x83, 0x79, 0x47, 0x50, 0x11, 0xb0, 0x59, 0x2e, 0x39, 0x87, 0x9c, 0x50, 0x8a, 0x9f, 0x1e, 0x47, 0xec, 0x0f, 0x07, 0x5a,
0xa6, 0xa8, 0x08, 0x8c, 0xda, 0xca, 0xd4, 0x99, 0x53, 0x37, 0xbd, 0x69, 0xf3, 0x7c, 0xea, 0xf2, 0x2c, 0xde, 0xee, 0xb6, 0x48, 0x65, 0x93, 0x45, 0xb9, 0xac, 0xe2, 0x3d, 0x72, 0x8e, 0x7b, 0x82,
0xfd, 0x4f, 0x00, 0x00, 0x00, 0xff, 0xff, 0x9b, 0x50, 0x78, 0xd1, 0x8d, 0x01, 0x00, 0x00, 0xbc, 0x8c, 0xbb, 0x6c, 0x57, 0x05, 0x6a, 0x52, 0x98, 0x35, 0x04, 0x45, 0x8c, 0x82, 0x40, 0x89,
0x0d, 0x8f, 0x4d, 0xe0, 0xd8, 0x7c, 0x54, 0xd5, 0x59, 0x36, 0x36, 0xfa, 0xe6, 0x2b, 0x00, 0x00,
0xff, 0xff, 0x57, 0x2d, 0x9a, 0xbc, 0xe4, 0x01, 0x00, 0x00,
} }

View File

@ -69,18 +69,21 @@ func CreateProxy(ctx context.Context) (*Proxy, error) {
return nil, err return nil, err
} }
p.idAllocator = idAllocator p.idAllocator = idAllocator
p.idAllocator.PeerID = Params.ProxyID()
tsoAllocator, err := allocator.NewTimestampAllocator(p.proxyLoopCtx, masterAddr) tsoAllocator, err := allocator.NewTimestampAllocator(p.proxyLoopCtx, masterAddr)
if err != nil { if err != nil {
return nil, err return nil, err
} }
p.tsoAllocator = tsoAllocator p.tsoAllocator = tsoAllocator
p.tsoAllocator.PeerID = Params.ProxyID()
segAssigner, err := allocator.NewSegIDAssigner(p.proxyLoopCtx, masterAddr) segAssigner, err := allocator.NewSegIDAssigner(p.proxyLoopCtx, masterAddr, p.lastTick)
if err != nil { if err != nil {
panic(err) panic(err)
} }
p.segAssigner = segAssigner p.segAssigner = segAssigner
p.segAssigner.PeerID = Params.ProxyID()
p.manipulationMsgStream = msgstream.NewPulsarMsgStream(p.proxyLoopCtx, Params.MsgStreamInsertBufSize()) p.manipulationMsgStream = msgstream.NewPulsarMsgStream(p.proxyLoopCtx, Params.MsgStreamInsertBufSize())
p.manipulationMsgStream.SetPulsarClient(pulsarAddress) p.manipulationMsgStream.SetPulsarClient(pulsarAddress)
@ -105,6 +108,10 @@ func (p *Proxy) AddStartCallback(callbacks ...func()) {
p.startCallbacks = append(p.startCallbacks, callbacks...) p.startCallbacks = append(p.startCallbacks, callbacks...)
} }
func (p *Proxy) lastTick() Timestamp {
return p.tick.LastTick()
}
func (p *Proxy) startProxy() error { func (p *Proxy) startProxy() error {
err := p.connectMaster() err := p.connectMaster()
if err != nil { if err != nil {

View File

@ -10,6 +10,9 @@ import (
"strings" "strings"
"sync" "sync"
"testing" "testing"
"time"
"github.com/zilliztech/milvus-distributed/internal/util/tsoutil"
"go.uber.org/zap" "go.uber.org/zap"
"google.golang.org/grpc" "google.golang.org/grpc"
@ -399,8 +402,9 @@ func TestProxy_AssignSegID(t *testing.T) {
collectionName := "CreateCollection1" collectionName := "CreateCollection1"
createCollection(t, collectionName) createCollection(t, collectionName)
testNum := 1 testNum := 1
futureTS := tsoutil.ComposeTS(time.Now().Add(time.Second*-1000).UnixNano()/int64(time.Millisecond), 0)
for i := 0; i < testNum; i++ { for i := 0; i < testNum; i++ {
segID, err := proxyServer.segAssigner.GetSegmentID(collectionName, Params.defaultPartitionTag(), int32(i), 200000) segID, err := proxyServer.segAssigner.GetSegmentID(collectionName, Params.defaultPartitionTag(), int32(i), 200000, futureTS)
assert.Nil(t, err) assert.Nil(t, err)
fmt.Println("segID", segID) fmt.Println("segID", segID)
} }

View File

@ -4,6 +4,8 @@ import (
"log" "log"
"sort" "sort"
"github.com/zilliztech/milvus-distributed/internal/util/typeutil"
"github.com/zilliztech/milvus-distributed/internal/allocator" "github.com/zilliztech/milvus-distributed/internal/allocator"
"github.com/zilliztech/milvus-distributed/internal/errors" "github.com/zilliztech/milvus-distributed/internal/errors"
"github.com/zilliztech/milvus-distributed/internal/msgstream" "github.com/zilliztech/milvus-distributed/internal/msgstream"
@ -19,6 +21,7 @@ func insertRepackFunc(tsMsgs []msgstream.TsMsg,
result := make(map[int32]*msgstream.MsgPack) result := make(map[int32]*msgstream.MsgPack)
channelCountMap := make(map[UniqueID]map[int32]uint32) // reqID --> channelID to count channelCountMap := make(map[UniqueID]map[int32]uint32) // reqID --> channelID to count
channelMaxTSMap := make(map[UniqueID]map[int32]Timestamp) // reqID --> channelID to max Timestamp
reqSchemaMap := make(map[UniqueID][]string) reqSchemaMap := make(map[UniqueID][]string)
for i, request := range tsMsgs { for i, request := range tsMsgs {
@ -45,12 +48,23 @@ func insertRepackFunc(tsMsgs []msgstream.TsMsg,
channelCountMap[reqID] = make(map[int32]uint32) channelCountMap[reqID] = make(map[int32]uint32)
} }
if _, ok := channelMaxTSMap[reqID]; !ok {
channelMaxTSMap[reqID] = make(map[int32]Timestamp)
}
if _, ok := reqSchemaMap[reqID]; !ok { if _, ok := reqSchemaMap[reqID]; !ok {
reqSchemaMap[reqID] = []string{insertRequest.CollectionName, insertRequest.PartitionTag} reqSchemaMap[reqID] = []string{insertRequest.CollectionName, insertRequest.PartitionTag}
} }
for _, channelID := range keys { for idx, channelID := range keys {
channelCountMap[reqID][channelID]++ channelCountMap[reqID][channelID]++
if _, ok := channelMaxTSMap[reqID][channelID]; !ok {
channelMaxTSMap[reqID][channelID] = typeutil.ZeroTimestamp
}
ts := insertRequest.Timestamps[idx]
if channelMaxTSMap[reqID][channelID] < ts {
channelMaxTSMap[reqID][channelID] = ts
}
} }
} }
@ -64,7 +78,12 @@ func insertRepackFunc(tsMsgs []msgstream.TsMsg,
schema := reqSchemaMap[reqID] schema := reqSchemaMap[reqID]
collName, partitionTag := schema[0], schema[1] collName, partitionTag := schema[0], schema[1]
for channelID, count := range countInfo { for channelID, count := range countInfo {
mapInfo, err := segIDAssigner.GetSegmentID(collName, partitionTag, channelID, count) ts, ok := channelMaxTSMap[reqID][channelID]
if !ok {
ts = typeutil.ZeroTimestamp
log.Println("Warning: did not get max Timstamp!")
}
mapInfo, err := segIDAssigner.GetSegmentID(collName, partitionTag, channelID, count, ts)
if err != nil { if err != nil {
return nil, err return nil, err
} }

View File

@ -104,6 +104,10 @@ func (tt *timeTick) tickLoop() {
} }
} }
func (tt *timeTick) LastTick() Timestamp {
return tt.lastTick
}
func (tt *timeTick) Start() error { func (tt *timeTick) Start() error {
tt.lastTick = 0 tt.lastTick = 0
ts, err := tt.tsoAllocator.AllocOne() ts, err := tt.tsoAllocator.AllocOne()

View File

@ -21,7 +21,8 @@ func NewLoadIndexClient(ctx context.Context, pulsarAddress string, loadIndexChan
} }
} }
func (lic *LoadIndexClient) LoadIndex(indexPaths []string, segmentID int64, fieldID int64) error { func (lic *LoadIndexClient) LoadIndex(indexPaths []string, segmentID int64, fieldID int64, indexParam map[string]string) error {
// TODO:: add indexParam to proto
baseMsg := msgstream.BaseMsg{ baseMsg := msgstream.BaseMsg{
BeginTimestamp: 0, BeginTimestamp: 0,
EndTimestamp: 0, EndTimestamp: 0,

View File

@ -54,7 +54,7 @@ type collectionReplica interface {
// segment // segment
getSegmentNum() int getSegmentNum() int
getSegmentStatistics() *internalpb.QueryNodeStats getSegmentStatistics() []*internalpb.SegmentStats
addSegment(segmentID UniqueID, partitionTag string, collectionID UniqueID) error addSegment(segmentID UniqueID, partitionTag string, collectionID UniqueID) error
removeSegment(segmentID UniqueID) error removeSegment(segmentID UniqueID) error
getSegmentByID(segmentID UniqueID) (*Segment, error) getSegmentByID(segmentID UniqueID) (*Segment, error)
@ -317,7 +317,7 @@ func (colReplica *collectionReplicaImpl) getSegmentNum() int {
return len(colReplica.segments) return len(colReplica.segments)
} }
func (colReplica *collectionReplicaImpl) getSegmentStatistics() *internalpb.QueryNodeStats { func (colReplica *collectionReplicaImpl) getSegmentStatistics() []*internalpb.SegmentStats {
colReplica.mu.RLock() colReplica.mu.RLock()
defer colReplica.mu.RUnlock() defer colReplica.mu.RUnlock()
@ -339,10 +339,7 @@ func (colReplica *collectionReplicaImpl) getSegmentStatistics() *internalpb.Quer
segment.recentlyModified = false segment.recentlyModified = false
} }
return &internalpb.QueryNodeStats{ return statisticData
MsgType: internalpb.MsgType_kQueryNodeStats,
SegStats: statisticData,
}
} }
func (colReplica *collectionReplicaImpl) addSegment(segmentID UniqueID, partitionTag string, collectionID UniqueID) error { func (colReplica *collectionReplicaImpl) addSegment(segmentID UniqueID, partitionTag string, collectionID UniqueID) error {
@ -359,7 +356,7 @@ func (colReplica *collectionReplicaImpl) addSegment(segmentID UniqueID, partitio
colReplica.mu.Lock() colReplica.mu.Lock()
defer colReplica.mu.Unlock() defer colReplica.mu.Unlock()
var newSegment = newSegment(collection, segmentID) var newSegment = newSegment(collection, segmentID, partitionTag, collectionID)
colReplica.segments[segmentID] = newSegment colReplica.segments[segmentID] = newSegment
*partition.Segments() = append(*partition.Segments(), newSegment) *partition.Segments() = append(*partition.Segments(), newSegment)

View File

@ -1,41 +0,0 @@
package querynode
import (
"context"
"github.com/minio/minio-go/v7"
"github.com/zilliztech/milvus-distributed/internal/msgstream"
)
type LoadIndex struct {
ctx context.Context
cancel context.CancelFunc
client *minio.Client
replica collectionReplica
numCompletedSegmentsToFieldID map[int64]int64
msgBuffer chan msgstream.TsMsg
unsolvedMsg []msgstream.TsMsg
loadIndexMsgStream msgstream.MsgStream
queryNodeID UniqueID
}
func (li *LoadIndex) loadIndex(indexKey []string) [][]byte {
// TODO:: load dataStore client interface to load builtIndex according index key
return nil
}
func (li *LoadIndex) updateSegmentIndex(bytesIndex [][]byte, segmentID UniqueID) error {
// TODO:: dataStore return bytes index, load index to c++ segment
// TODO: how to deserialize bytes to segment index?
return nil
}
func (li *LoadIndex) sendQueryNodeStats() error {
// TODO:: update segment index type in replica, and publish queryNode segmentStats
return nil
}

View File

@ -0,0 +1,98 @@
package querynode
/*
#cgo CFLAGS: -I${SRCDIR}/../core/output/include
#cgo LDFLAGS: -L${SRCDIR}/../core/output/lib -lmilvus_segcore -Wl,-rpath=${SRCDIR}/../core/output/lib
#include "segcore/load_index_c.h"
*/
import "C"
import (
"errors"
"strconv"
"unsafe"
)
type LoadIndexInfo struct {
cLoadIndexInfo C.CLoadIndexInfo
}
func NewLoadIndexInfo() (*LoadIndexInfo, error) {
var cLoadIndexInfo C.CLoadIndexInfo
status := C.NewLoadIndexInfo(&cLoadIndexInfo)
errorCode := status.error_code
if errorCode != 0 {
errorMsg := C.GoString(status.error_msg)
defer C.free(unsafe.Pointer(status.error_msg))
return nil, errors.New("NewLoadIndexInfo failed, C runtime error detected, error code = " + strconv.Itoa(int(errorCode)) + ", error msg = " + errorMsg)
}
return &LoadIndexInfo{cLoadIndexInfo: cLoadIndexInfo}, nil
}
func (li *LoadIndexInfo) AppendIndexParam(indexKey string, indexValue string) error {
cIndexKey := C.CString(indexKey)
cIndexValue := C.CString(indexValue)
status := C.AppendIndexParam(li.cLoadIndexInfo, cIndexKey, cIndexValue)
errorCode := status.error_code
if errorCode != 0 {
errorMsg := C.GoString(status.error_msg)
defer C.free(unsafe.Pointer(status.error_msg))
return errors.New("AppendIndexParam failed, C runtime error detected, error code = " + strconv.Itoa(int(errorCode)) + ", error msg = " + errorMsg)
}
return nil
}
func (li *LoadIndexInfo) AppendFieldInfo(fieldName string, fieldID int64) error {
cFieldName := C.CString(fieldName)
cFieldID := C.long(fieldID)
status := C.AppendFieldInfo(li.cLoadIndexInfo, cFieldName, cFieldID)
errorCode := status.error_code
if errorCode != 0 {
errorMsg := C.GoString(status.error_msg)
defer C.free(unsafe.Pointer(status.error_msg))
return errors.New("AppendFieldInfo failed, C runtime error detected, error code = " + strconv.Itoa(int(errorCode)) + ", error msg = " + errorMsg)
}
return nil
}
func (li *LoadIndexInfo) AppendIndex(bytesIndex [][]byte, indexKeys []string) error {
var cBinarySet C.CBinarySet
status := C.NewBinarySet(&cBinarySet)
errorCode := status.error_code
if errorCode != 0 {
errorMsg := C.GoString(status.error_msg)
defer C.free(unsafe.Pointer(status.error_msg))
return errors.New("newBinarySet failed, C runtime error detected, error code = " + strconv.Itoa(int(errorCode)) + ", error msg = " + errorMsg)
}
for i, byteIndex := range bytesIndex {
indexPtr := unsafe.Pointer(&byteIndex[0])
indexLen := C.long(len(byteIndex))
indexKey := C.CString(indexKeys[i])
status = C.AppendBinaryIndex(cBinarySet, indexPtr, indexLen, indexKey)
errorCode = status.error_code
if errorCode != 0 {
break
}
}
if errorCode != 0 {
errorMsg := C.GoString(status.error_msg)
defer C.free(unsafe.Pointer(status.error_msg))
return errors.New("AppendBinaryIndex failed, C runtime error detected, error code = " + strconv.Itoa(int(errorCode)) + ", error msg = " + errorMsg)
}
status = C.AppendIndex(li.cLoadIndexInfo, cBinarySet)
errorCode = status.error_code
if errorCode != 0 {
errorMsg := C.GoString(status.error_msg)
defer C.free(unsafe.Pointer(status.error_msg))
return errors.New("AppendIndex failed, C runtime error detected, error code = " + strconv.Itoa(int(errorCode)) + ", error msg = " + errorMsg)
}
return nil
}

View File

@ -0,0 +1,286 @@
package querynode
import (
"context"
"errors"
"fmt"
"log"
"path/filepath"
"sort"
"strconv"
"strings"
"github.com/minio/minio-go/v7"
"github.com/minio/minio-go/v7/pkg/credentials"
minioKV "github.com/zilliztech/milvus-distributed/internal/kv/minio"
"github.com/zilliztech/milvus-distributed/internal/msgstream"
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
internalPb "github.com/zilliztech/milvus-distributed/internal/proto/internalpb"
)
type loadIndexService struct {
ctx context.Context
cancel context.CancelFunc
client *minioKV.MinIOKV
replica collectionReplica
fieldIndexes map[string][]*internalPb.IndexStats
fieldStatsChan chan []*internalPb.FieldStats
msgBuffer chan msgstream.TsMsg
unsolvedMsg []msgstream.TsMsg
loadIndexMsgStream msgstream.MsgStream
queryNodeID UniqueID
}
func newLoadIndexService(ctx context.Context, replica collectionReplica) *loadIndexService {
ctx1, cancel := context.WithCancel(ctx)
// init minio
minioClient, err := minio.New(Params.MinioEndPoint, &minio.Options{
Creds: credentials.NewStaticV4(Params.MinioAccessKeyID, Params.MinioSecretAccessKey, ""),
Secure: Params.MinioUseSSLStr,
})
if err != nil {
panic(err)
}
// TODO: load bucketName from config
bucketName := "query-node-load-index-service-minio"
MinioKV, err := minioKV.NewMinIOKV(ctx1, minioClient, bucketName)
if err != nil {
panic(err)
}
// init msgStream
receiveBufSize := Params.LoadIndexReceiveBufSize
pulsarBufSize := Params.LoadIndexPulsarBufSize
msgStreamURL := Params.PulsarAddress
consumeChannels := Params.LoadIndexChannelNames
consumeSubName := Params.MsgChannelSubName
loadIndexStream := msgstream.NewPulsarMsgStream(ctx, receiveBufSize)
loadIndexStream.SetPulsarClient(msgStreamURL)
unmarshalDispatcher := msgstream.NewUnmarshalDispatcher()
loadIndexStream.CreatePulsarConsumers(consumeChannels, consumeSubName, unmarshalDispatcher, pulsarBufSize)
var stream msgstream.MsgStream = loadIndexStream
return &loadIndexService{
ctx: ctx1,
cancel: cancel,
client: MinioKV,
replica: replica,
fieldIndexes: make(map[string][]*internalPb.IndexStats),
fieldStatsChan: make(chan []*internalPb.FieldStats, 1),
msgBuffer: make(chan msgstream.TsMsg, 1),
unsolvedMsg: make([]msgstream.TsMsg, 0),
loadIndexMsgStream: stream,
queryNodeID: Params.QueryNodeID,
}
}
func (lis *loadIndexService) start() {
lis.loadIndexMsgStream.Start()
for {
select {
case <-lis.ctx.Done():
return
default:
messages := lis.loadIndexMsgStream.Consume()
if messages == nil || len(messages.Msgs) <= 0 {
log.Println("null msg pack")
continue
}
for _, msg := range messages.Msgs {
indexMsg, ok := msg.(*msgstream.LoadIndexMsg)
if !ok {
log.Println("type assertion failed for LoadIndexMsg")
continue
}
/* TODO: debug
// 1. use msg's index paths to get index bytes
indexBuffer := lis.loadIndex(indexMsg.IndexPaths)
// 2. use index bytes and index path to update segment
err := lis.updateSegmentIndex(indexBuffer, indexMsg.IndexPaths, indexMsg.SegmentID)
if err != nil {
log.Println(err)
continue
}
*/
// 3. update segment index stats
err := lis.updateSegmentIndexStats(indexMsg)
if err != nil {
log.Println(err)
continue
}
}
// sendQueryNodeStats
err := lis.sendQueryNodeStats()
if err != nil {
log.Println(err)
continue
}
}
}
}
func (lis *loadIndexService) printIndexParams(index []*commonpb.KeyValuePair) {
fmt.Println("=================================================")
for i := 0; i < len(index); i++ {
fmt.Println(index[i])
}
}
func (lis *loadIndexService) indexParamsEqual(index1 []*commonpb.KeyValuePair, index2 []*commonpb.KeyValuePair) bool {
if len(index1) != len(index2) {
return false
}
for i := 0; i < len(index1); i++ {
kv1 := *index1[i]
kv2 := *index2[i]
if kv1.Key != kv2.Key || kv1.Value != kv2.Value {
return false
}
}
return true
}
func (lis *loadIndexService) fieldsStatsIDs2Key(collectionID UniqueID, fieldID UniqueID) string {
return strconv.FormatInt(collectionID, 10) + "/" + strconv.FormatInt(fieldID, 10)
}
func (lis *loadIndexService) fieldsStatsKey2IDs(key string) (UniqueID, UniqueID, error) {
ids := strings.Split(key, "/")
if len(ids) != 2 {
return 0, 0, errors.New("illegal fieldsStatsKey")
}
collectionID, err := strconv.ParseInt(ids[0], 10, 64)
if err != nil {
return 0, 0, err
}
fieldID, err := strconv.ParseInt(ids[1], 10, 64)
if err != nil {
return 0, 0, err
}
return collectionID, fieldID, nil
}
func (lis *loadIndexService) updateSegmentIndexStats(indexMsg *msgstream.LoadIndexMsg) error {
targetSegment, err := lis.replica.getSegmentByID(indexMsg.SegmentID)
if err != nil {
return err
}
fieldStatsKey := lis.fieldsStatsIDs2Key(targetSegment.collectionID, indexMsg.FieldID)
_, ok := lis.fieldIndexes[fieldStatsKey]
newIndexParams := indexMsg.IndexParams
// sort index params by key
sort.Slice(newIndexParams, func(i, j int) bool { return newIndexParams[i].Key < newIndexParams[j].Key })
if !ok {
lis.fieldIndexes[fieldStatsKey] = make([]*internalPb.IndexStats, 0)
lis.fieldIndexes[fieldStatsKey] = append(lis.fieldIndexes[fieldStatsKey],
&internalPb.IndexStats{
IndexParams: newIndexParams,
NumRelatedSegments: 1,
})
} else {
isNewIndex := true
for _, index := range lis.fieldIndexes[fieldStatsKey] {
if lis.indexParamsEqual(newIndexParams, index.IndexParams) {
index.NumRelatedSegments++
isNewIndex = false
}
}
if isNewIndex {
lis.fieldIndexes[fieldStatsKey] = append(lis.fieldIndexes[fieldStatsKey],
&internalPb.IndexStats{
IndexParams: newIndexParams,
NumRelatedSegments: 1,
})
}
}
return nil
}
func (lis *loadIndexService) loadIndex(indexPath []string) [][]byte {
index := make([][]byte, 0)
for _, path := range indexPath {
// get binarySetKey from indexPath
binarySetKey := filepath.Base(path)
indexPiece, err := (*lis.client).Load(binarySetKey)
if err != nil {
log.Println(err)
return nil
}
index = append(index, []byte(indexPiece))
}
return index
}
func (lis *loadIndexService) updateSegmentIndex(bytesIndex [][]byte, loadIndexMsg *msgstream.LoadIndexMsg) error {
segment, err := lis.replica.getSegmentByID(loadIndexMsg.SegmentID)
if err != nil {
return err
}
loadIndexInfo, err := NewLoadIndexInfo()
if err != nil {
return err
}
err = loadIndexInfo.AppendFieldInfo(loadIndexMsg.FieldName, loadIndexMsg.FieldID)
if err != nil {
return err
}
for _, indexParam := range loadIndexMsg.IndexParams {
err = loadIndexInfo.AppendIndexParam(indexParam.Key, indexParam.Value)
if err != nil {
return err
}
}
err = loadIndexInfo.AppendIndex(bytesIndex, loadIndexMsg.IndexPaths)
if err != nil {
return err
}
err = segment.updateSegmentIndex(loadIndexInfo)
if err != nil {
return err
}
return nil
}
func (lis *loadIndexService) sendQueryNodeStats() error {
resultFieldsStats := make([]*internalPb.FieldStats, 0)
for fieldStatsKey, indexStats := range lis.fieldIndexes {
colID, fieldID, err := lis.fieldsStatsKey2IDs(fieldStatsKey)
if err != nil {
return err
}
fieldStats := internalPb.FieldStats{
CollectionID: colID,
FieldID: fieldID,
IndexStats: indexStats,
}
resultFieldsStats = append(resultFieldsStats, &fieldStats)
}
lis.fieldStatsChan <- resultFieldsStats
fmt.Println("sent field stats")
return nil
}

View File

@ -0,0 +1,148 @@
package querynode
import (
"math"
"math/rand"
"sort"
"testing"
"github.com/stretchr/testify/assert"
"github.com/zilliztech/milvus-distributed/internal/msgstream"
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
internalPb "github.com/zilliztech/milvus-distributed/internal/proto/internalpb"
)
func TestLoadIndexService_PulsarAddress(t *testing.T) {
node := newQueryNode()
collectionID := rand.Int63n(1000000)
segmentID := rand.Int63n(1000000)
fieldID := rand.Int63n(1000000)
initTestMeta(t, node, "collection0", collectionID, segmentID)
// loadIndexService and statsService
node.loadIndexService = newLoadIndexService(node.queryNodeLoopCtx, node.replica)
go node.loadIndexService.start()
node.statsService = newStatsService(node.queryNodeLoopCtx, node.replica, node.loadIndexService.fieldStatsChan)
go node.statsService.start()
// gen load index message pack
const msgLength = 10
indexParams := make([]*commonpb.KeyValuePair, 0)
// init IVF_FLAT index params
const (
KeyDim = "dim"
KeyTopK = "k"
KeyNList = "nlist"
KeyNProbe = "nprobe"
KeyMetricType = "metric_type"
KeySliceSize = "SLICE_SIZE"
KeyDeviceID = "gpu_id"
)
const (
ValueDim = "128"
ValueTopK = "10"
ValueNList = "100"
ValueNProbe = "4"
ValueMetricType = "L2"
ValueSliceSize = "4"
ValueDeviceID = "0"
)
indexParams = append(indexParams, &commonpb.KeyValuePair{
Key: KeyDim,
Value: ValueDim,
})
indexParams = append(indexParams, &commonpb.KeyValuePair{
Key: KeyTopK,
Value: ValueTopK,
})
indexParams = append(indexParams, &commonpb.KeyValuePair{
Key: KeyNList,
Value: ValueNList,
})
indexParams = append(indexParams, &commonpb.KeyValuePair{
Key: KeyNProbe,
Value: ValueNProbe,
})
indexParams = append(indexParams, &commonpb.KeyValuePair{
Key: KeyMetricType,
Value: ValueMetricType,
})
indexParams = append(indexParams, &commonpb.KeyValuePair{
Key: KeySliceSize,
Value: ValueSliceSize,
})
indexParams = append(indexParams, &commonpb.KeyValuePair{
Key: KeyDeviceID,
Value: ValueDeviceID,
})
loadIndex := internalPb.LoadIndex{
MsgType: internalPb.MsgType_kLoadIndex,
SegmentID: segmentID,
FieldID: fieldID,
IndexPaths: []string{"tmp/index"}, // TODO:
IndexParams: indexParams,
}
loadIndexMsg := msgstream.LoadIndexMsg{
BaseMsg: msgstream.BaseMsg{
HashValues: []uint32{uint32(0)},
},
LoadIndex: loadIndex,
}
messages := make([]msgstream.TsMsg, 0)
for i := 0; i < msgLength; i++ {
var msg msgstream.TsMsg = &loadIndexMsg
messages = append(messages, msg)
}
msgPack := msgstream.MsgPack{
BeginTs: 0,
EndTs: math.MaxUint64,
Msgs: messages,
}
// init message stream producer
loadIndexChannelNames := Params.LoadIndexChannelNames
pulsarURL := Params.PulsarAddress
loadIndexStream := msgstream.NewPulsarMsgStream(node.queryNodeLoopCtx, Params.LoadIndexReceiveBufSize)
loadIndexStream.SetPulsarClient(pulsarURL)
loadIndexStream.CreatePulsarProducers(loadIndexChannelNames)
var loadIndexMsgStream msgstream.MsgStream = loadIndexStream
loadIndexMsgStream.Start()
err := loadIndexMsgStream.Produce(&msgPack)
assert.NoError(t, err)
// init message stream consumer and do checks
statsMs := msgstream.NewPulsarMsgStream(node.queryNodeLoopCtx, Params.StatsReceiveBufSize)
statsMs.SetPulsarClient(pulsarURL)
statsMs.CreatePulsarConsumers([]string{Params.StatsChannelName}, Params.MsgChannelSubName, msgstream.NewUnmarshalDispatcher(), Params.StatsReceiveBufSize)
statsMs.Start()
receiveMsg := msgstream.MsgStream(statsMs).Consume()
assert.NotNil(t, receiveMsg)
assert.NotEqual(t, len(receiveMsg.Msgs), 0)
statsMsg, ok := receiveMsg.Msgs[0].(*msgstream.QueryNodeStatsMsg)
assert.Equal(t, ok, true)
assert.Equal(t, len(statsMsg.FieldStats), 1)
fieldStats0 := statsMsg.FieldStats[0]
assert.Equal(t, fieldStats0.FieldID, fieldID)
assert.Equal(t, fieldStats0.CollectionID, collectionID)
assert.Equal(t, len(fieldStats0.IndexStats), 1)
indexStats0 := fieldStats0.IndexStats[0]
params := indexStats0.IndexParams
// sort index params by key
sort.Slice(indexParams, func(i, j int) bool { return indexParams[i].Key < indexParams[j].Key })
indexEqual := node.loadIndexService.indexParamsEqual(params, indexParams)
assert.Equal(t, indexEqual, true)
<-node.queryNodeLoopCtx.Done()
node.Close()
}

View File

@ -21,6 +21,12 @@ type ParamTable struct {
FlowGraphMaxQueueLength int32 FlowGraphMaxQueueLength int32
FlowGraphMaxParallelism int32 FlowGraphMaxParallelism int32
// minio
MinioEndPoint string
MinioAccessKeyID string
MinioSecretAccessKey string
MinioUseSSLStr bool
// dm // dm
InsertChannelNames []string InsertChannelNames []string
InsertChannelRange []int InsertChannelRange []int
@ -44,6 +50,11 @@ type ParamTable struct {
StatsChannelName string StatsChannelName string
StatsReceiveBufSize int64 StatsReceiveBufSize int64
// load index
LoadIndexChannelNames []string
LoadIndexReceiveBufSize int64
LoadIndexPulsarBufSize int64
GracefulTime int64 GracefulTime int64
MsgChannelSubName string MsgChannelSubName string
DefaultPartitionTag string DefaultPartitionTag string
@ -59,6 +70,11 @@ func (p *ParamTable) Init() {
panic(err) panic(err)
} }
err = p.LoadYaml("milvus.yaml")
if err != nil {
panic(err)
}
queryNodeIDStr := os.Getenv("QUERY_NODE_ID") queryNodeIDStr := os.Getenv("QUERY_NODE_ID")
if queryNodeIDStr == "" { if queryNodeIDStr == "" {
queryNodeIDList := p.QueryNodeIDList() queryNodeIDList := p.QueryNodeIDList()
@ -78,6 +94,11 @@ func (p *ParamTable) Init() {
panic(err) panic(err)
} }
p.initMinioEndPoint()
p.initMinioAccessKeyID()
p.initMinioSecretAccessKey()
p.initMinioUseSSLStr()
p.initPulsarAddress() p.initPulsarAddress()
p.initETCDAddress() p.initETCDAddress()
p.initMetaRootPath() p.initMetaRootPath()
@ -111,6 +132,46 @@ func (p *ParamTable) Init() {
p.initStatsPublishInterval() p.initStatsPublishInterval()
p.initStatsChannelName() p.initStatsChannelName()
p.initStatsReceiveBufSize() p.initStatsReceiveBufSize()
p.initLoadIndexChannelNames()
p.initLoadIndexReceiveBufSize()
p.initLoadIndexPulsarBufSize()
}
func (p *ParamTable) initMinioEndPoint() {
url, err := p.Load("_MinioAddress")
if err != nil {
panic(err)
}
p.MinioEndPoint = url
}
func (p *ParamTable) initMinioAccessKeyID() {
id, err := p.Load("minio.accessKeyID")
if err != nil {
panic(err)
}
p.MinioAccessKeyID = id
}
func (p *ParamTable) initMinioSecretAccessKey() {
key, err := p.Load("minio.secretAccessKey")
if err != nil {
panic(err)
}
p.MinioSecretAccessKey = key
}
func (p *ParamTable) initMinioUseSSLStr() {
ssl, err := p.Load("minio.useSSL")
if err != nil {
panic(err)
}
sslBoolean, err := strconv.ParseBool(ssl)
if err != nil {
panic(err)
}
p.MinioUseSSLStr = sslBoolean
} }
func (p *ParamTable) initPulsarAddress() { func (p *ParamTable) initPulsarAddress() {
@ -358,3 +419,19 @@ func (p *ParamTable) initSliceIndex() {
func (p *ParamTable) initQueryNodeNum() { func (p *ParamTable) initQueryNodeNum() {
p.QueryNodeNum = len(p.QueryNodeIDList()) p.QueryNodeNum = len(p.QueryNodeIDList())
} }
func (p *ParamTable) initLoadIndexChannelNames() {
loadIndexChannelName, err := p.Load("msgChannel.chanNamePrefix.cmd")
if err != nil {
panic(err)
}
p.LoadIndexChannelNames = []string{loadIndexChannelName}
}
func (p *ParamTable) initLoadIndexReceiveBufSize() {
p.LoadIndexReceiveBufSize = p.ParseInt64("queryNode.msgStream.loadIndex.recvBufSize")
}
func (p *ParamTable) initLoadIndexPulsarBufSize() {
p.LoadIndexPulsarBufSize = p.ParseInt64("queryNode.msgStream.loadIndex.pulsarBufSize")
}

View File

@ -15,6 +15,47 @@ func TestParamTable_PulsarAddress(t *testing.T) {
assert.Equal(t, "6650", split[len(split)-1]) assert.Equal(t, "6650", split[len(split)-1])
} }
func TestParamTable_minio(t *testing.T) {
t.Run("Test endPoint", func(t *testing.T) {
endPoint := Params.MinioEndPoint
equal := endPoint == "localhost:9000" || endPoint == "minio:9000"
assert.Equal(t, equal, true)
})
t.Run("Test accessKeyID", func(t *testing.T) {
accessKeyID := Params.MinioAccessKeyID
assert.Equal(t, accessKeyID, "minioadmin")
})
t.Run("Test secretAccessKey", func(t *testing.T) {
secretAccessKey := Params.MinioSecretAccessKey
assert.Equal(t, secretAccessKey, "minioadmin")
})
t.Run("Test useSSL", func(t *testing.T) {
useSSL := Params.MinioUseSSLStr
assert.Equal(t, useSSL, false)
})
}
func TestParamTable_LoadIndex(t *testing.T) {
t.Run("Test channel names", func(t *testing.T) {
names := Params.LoadIndexChannelNames
assert.Equal(t, len(names), 1)
assert.Contains(t, names[0], "cmd")
})
t.Run("Test recvBufSize", func(t *testing.T) {
size := Params.LoadIndexReceiveBufSize
assert.Equal(t, size, int64(512))
})
t.Run("Test pulsarBufSize", func(t *testing.T) {
size := Params.LoadIndexPulsarBufSize
assert.Equal(t, size, int64(512))
})
}
func TestParamTable_QueryNodeID(t *testing.T) { func TestParamTable_QueryNodeID(t *testing.T) {
id := Params.QueryNodeID id := Params.QueryNodeID
assert.Contains(t, Params.QueryNodeIDList(), id) assert.Contains(t, Params.QueryNodeIDList(), id)

View File

@ -24,9 +24,11 @@ type QueryNode struct {
replica collectionReplica replica collectionReplica
// services
dataSyncService *dataSyncService dataSyncService *dataSyncService
metaService *metaService metaService *metaService
searchService *searchService searchService *searchService
loadIndexService *loadIndexService
statsService *statsService statsService *statsService
} }
@ -69,11 +71,13 @@ func (node *QueryNode) Start() error {
node.dataSyncService = newDataSyncService(node.queryNodeLoopCtx, node.replica) node.dataSyncService = newDataSyncService(node.queryNodeLoopCtx, node.replica)
node.searchService = newSearchService(node.queryNodeLoopCtx, node.replica) node.searchService = newSearchService(node.queryNodeLoopCtx, node.replica)
node.metaService = newMetaService(node.queryNodeLoopCtx, node.replica) node.metaService = newMetaService(node.queryNodeLoopCtx, node.replica)
node.statsService = newStatsService(node.queryNodeLoopCtx, node.replica) node.loadIndexService = newLoadIndexService(node.queryNodeLoopCtx, node.replica)
node.statsService = newStatsService(node.queryNodeLoopCtx, node.replica, node.loadIndexService.fieldStatsChan)
go node.dataSyncService.start() go node.dataSyncService.start()
go node.searchService.start() go node.searchService.start()
go node.metaService.start() go node.metaService.start()
go node.loadIndexService.start()
go node.statsService.start() go node.statsService.start()
<-node.queryNodeLoopCtx.Done() <-node.queryNodeLoopCtx.Done()

View File

@ -26,6 +26,7 @@ func setup() {
func genTestCollectionMeta(collectionName string, collectionID UniqueID) *etcdpb.CollectionMeta { func genTestCollectionMeta(collectionName string, collectionID UniqueID) *etcdpb.CollectionMeta {
fieldVec := schemapb.FieldSchema{ fieldVec := schemapb.FieldSchema{
FieldID: UniqueID(0),
Name: "vec", Name: "vec",
IsPrimaryKey: false, IsPrimaryKey: false,
DataType: schemapb.DataType_VECTOR_FLOAT, DataType: schemapb.DataType_VECTOR_FLOAT,
@ -44,6 +45,7 @@ func genTestCollectionMeta(collectionName string, collectionID UniqueID) *etcdpb
} }
fieldInt := schemapb.FieldSchema{ fieldInt := schemapb.FieldSchema{
FieldID: UniqueID(1),
Name: "age", Name: "age",
IsPrimaryKey: false, IsPrimaryKey: false,
DataType: schemapb.DataType_INT32, DataType: schemapb.DataType_INT32,
@ -119,12 +121,13 @@ func makeNewChannelNames(names []string, suffix string) []string {
} }
func refreshChannelNames() { func refreshChannelNames() {
suffix := "-test-query-node" + strconv.FormatInt(rand.Int63n(100), 10) suffix := "-test-query-node" + strconv.FormatInt(rand.Int63n(1000000), 10)
Params.DDChannelNames = makeNewChannelNames(Params.DDChannelNames, suffix) Params.DDChannelNames = makeNewChannelNames(Params.DDChannelNames, suffix)
Params.InsertChannelNames = makeNewChannelNames(Params.InsertChannelNames, suffix) Params.InsertChannelNames = makeNewChannelNames(Params.InsertChannelNames, suffix)
Params.SearchChannelNames = makeNewChannelNames(Params.SearchChannelNames, suffix) Params.SearchChannelNames = makeNewChannelNames(Params.SearchChannelNames, suffix)
Params.SearchResultChannelNames = makeNewChannelNames(Params.SearchResultChannelNames, suffix) Params.SearchResultChannelNames = makeNewChannelNames(Params.SearchResultChannelNames, suffix)
Params.StatsChannelName = Params.StatsChannelName + suffix Params.StatsChannelName = Params.StatsChannelName + suffix
Params.LoadIndexChannelNames = makeNewChannelNames(Params.LoadIndexChannelNames, suffix)
} }
func TestMain(m *testing.M) { func TestMain(m *testing.M) {

View File

@ -21,7 +21,7 @@ func TestReduce_AllFunc(t *testing.T) {
assert.NotEqual(t, "", schemaBlob) assert.NotEqual(t, "", schemaBlob)
collection := newCollection(collectionMeta.ID, schemaBlob) collection := newCollection(collectionMeta.ID, schemaBlob)
segment := newSegment(collection, segmentID) segment := newSegment(collection, segmentID, Params.DefaultPartitionTag, collectionID)
const DIM = 16 const DIM = 16
var vec = [DIM]float32{1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16} var vec = [DIM]float32{1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16}

View File

@ -9,7 +9,6 @@ package querynode
#include "segcore/collection_c.h" #include "segcore/collection_c.h"
#include "segcore/plan_c.h" #include "segcore/plan_c.h"
#include "segcore/reduce_c.h" #include "segcore/reduce_c.h"
*/ */
import "C" import "C"
import ( import (
@ -25,6 +24,8 @@ import (
type Segment struct { type Segment struct {
segmentPtr C.CSegmentBase segmentPtr C.CSegmentBase
segmentID UniqueID segmentID UniqueID
partitionTag string // TODO: use partitionID
collectionID UniqueID
lastMemSize int64 lastMemSize int64
lastRowCount int64 lastRowCount int64
recentlyModified bool recentlyModified bool
@ -35,13 +36,18 @@ func (s *Segment) ID() UniqueID {
} }
//-------------------------------------------------------------------------------------- constructor and destructor //-------------------------------------------------------------------------------------- constructor and destructor
func newSegment(collection *Collection, segmentID int64) *Segment { func newSegment(collection *Collection, segmentID int64, partitionTag string, collectionID UniqueID) *Segment {
/* /*
CSegmentBase CSegmentBase
newSegment(CPartition partition, unsigned long segment_id); newSegment(CPartition partition, unsigned long segment_id);
*/ */
segmentPtr := C.NewSegment(collection.collectionPtr, C.ulong(segmentID)) segmentPtr := C.NewSegment(collection.collectionPtr, C.ulong(segmentID))
var newSegment = &Segment{segmentPtr: segmentPtr, segmentID: segmentID} var newSegment = &Segment{
segmentPtr: segmentPtr,
segmentID: segmentID,
partitionTag: partitionTag,
collectionID: collectionID,
}
return newSegment return newSegment
} }
@ -236,3 +242,17 @@ func (s *Segment) fillTargetEntry(plan *Plan,
return nil return nil
} }
// segment, err := loadIndexService.replica.getSegmentByID(segmentID)
func (s *Segment) updateSegmentIndex(loadIndexInfo *LoadIndexInfo) error {
status := C.UpdateSegmentIndex(s.segmentPtr, loadIndexInfo.cLoadIndexInfo)
errorCode := status.error_code
if errorCode != 0 {
errorMsg := C.GoString(status.error_msg)
defer C.free(unsafe.Pointer(status.error_msg))
return errors.New("updateSegmentIndex failed, C runtime error detected, error code = " + strconv.Itoa(int(errorCode)) + ", error msg = " + errorMsg)
}
return nil
}

View File

@ -26,7 +26,7 @@ func TestSegment_newSegment(t *testing.T) {
assert.Equal(t, collection.ID(), collectionID) assert.Equal(t, collection.ID(), collectionID)
segmentID := UniqueID(0) segmentID := UniqueID(0)
segment := newSegment(collection, segmentID) segment := newSegment(collection, segmentID, Params.DefaultPartitionTag, collectionID)
assert.Equal(t, segmentID, segment.segmentID) assert.Equal(t, segmentID, segment.segmentID)
deleteSegment(segment) deleteSegment(segment)
deleteCollection(collection) deleteCollection(collection)
@ -44,7 +44,7 @@ func TestSegment_deleteSegment(t *testing.T) {
assert.Equal(t, collection.ID(), collectionID) assert.Equal(t, collection.ID(), collectionID)
segmentID := UniqueID(0) segmentID := UniqueID(0)
segment := newSegment(collection, segmentID) segment := newSegment(collection, segmentID, Params.DefaultPartitionTag, collectionID)
assert.Equal(t, segmentID, segment.segmentID) assert.Equal(t, segmentID, segment.segmentID)
deleteSegment(segment) deleteSegment(segment)
@ -64,7 +64,7 @@ func TestSegment_getRowCount(t *testing.T) {
assert.Equal(t, collection.ID(), collectionID) assert.Equal(t, collection.ID(), collectionID)
segmentID := UniqueID(0) segmentID := UniqueID(0)
segment := newSegment(collection, segmentID) segment := newSegment(collection, segmentID, Params.DefaultPartitionTag, collectionID)
assert.Equal(t, segmentID, segment.segmentID) assert.Equal(t, segmentID, segment.segmentID)
ids := []int64{1, 2, 3} ids := []int64{1, 2, 3}
@ -115,7 +115,7 @@ func TestSegment_getDeletedCount(t *testing.T) {
assert.Equal(t, collection.ID(), collectionID) assert.Equal(t, collection.ID(), collectionID)
segmentID := UniqueID(0) segmentID := UniqueID(0)
segment := newSegment(collection, segmentID) segment := newSegment(collection, segmentID, Params.DefaultPartitionTag, collectionID)
assert.Equal(t, segmentID, segment.segmentID) assert.Equal(t, segmentID, segment.segmentID)
ids := []int64{1, 2, 3} ids := []int64{1, 2, 3}
@ -172,7 +172,7 @@ func TestSegment_getMemSize(t *testing.T) {
assert.Equal(t, collection.ID(), collectionID) assert.Equal(t, collection.ID(), collectionID)
segmentID := UniqueID(0) segmentID := UniqueID(0)
segment := newSegment(collection, segmentID) segment := newSegment(collection, segmentID, Params.DefaultPartitionTag, collectionID)
assert.Equal(t, segmentID, segment.segmentID) assert.Equal(t, segmentID, segment.segmentID)
ids := []int64{1, 2, 3} ids := []int64{1, 2, 3}
@ -223,7 +223,7 @@ func TestSegment_segmentInsert(t *testing.T) {
assert.Equal(t, collection.Name(), collectionName) assert.Equal(t, collection.Name(), collectionName)
assert.Equal(t, collection.ID(), collectionID) assert.Equal(t, collection.ID(), collectionID)
segmentID := UniqueID(0) segmentID := UniqueID(0)
segment := newSegment(collection, segmentID) segment := newSegment(collection, segmentID, Params.DefaultPartitionTag, collectionID)
assert.Equal(t, segmentID, segment.segmentID) assert.Equal(t, segmentID, segment.segmentID)
ids := []int64{1, 2, 3} ids := []int64{1, 2, 3}
@ -270,7 +270,7 @@ func TestSegment_segmentDelete(t *testing.T) {
assert.Equal(t, collection.ID(), collectionID) assert.Equal(t, collection.ID(), collectionID)
segmentID := UniqueID(0) segmentID := UniqueID(0)
segment := newSegment(collection, segmentID) segment := newSegment(collection, segmentID, Params.DefaultPartitionTag, collectionID)
assert.Equal(t, segmentID, segment.segmentID) assert.Equal(t, segmentID, segment.segmentID)
ids := []int64{1, 2, 3} ids := []int64{1, 2, 3}
@ -323,7 +323,7 @@ func TestSegment_segmentSearch(t *testing.T) {
assert.Equal(t, collection.ID(), collectionID) assert.Equal(t, collection.ID(), collectionID)
segmentID := UniqueID(0) segmentID := UniqueID(0)
segment := newSegment(collection, segmentID) segment := newSegment(collection, segmentID, Params.DefaultPartitionTag, collectionID)
assert.Equal(t, segmentID, segment.segmentID) assert.Equal(t, segmentID, segment.segmentID)
ids := []int64{1, 2, 3} ids := []int64{1, 2, 3}
@ -408,7 +408,7 @@ func TestSegment_segmentPreInsert(t *testing.T) {
assert.Equal(t, collection.ID(), collectionID) assert.Equal(t, collection.ID(), collectionID)
segmentID := UniqueID(0) segmentID := UniqueID(0)
segment := newSegment(collection, segmentID) segment := newSegment(collection, segmentID, Params.DefaultPartitionTag, collectionID)
assert.Equal(t, segmentID, segment.segmentID) assert.Equal(t, segmentID, segment.segmentID)
const DIM = 16 const DIM = 16
@ -450,7 +450,7 @@ func TestSegment_segmentPreDelete(t *testing.T) {
assert.Equal(t, collection.ID(), collectionID) assert.Equal(t, collection.ID(), collectionID)
segmentID := UniqueID(0) segmentID := UniqueID(0)
segment := newSegment(collection, segmentID) segment := newSegment(collection, segmentID, Params.DefaultPartitionTag, collectionID)
assert.Equal(t, segmentID, segment.segmentID) assert.Equal(t, segmentID, segment.segmentID)
ids := []int64{1, 2, 3} ids := []int64{1, 2, 3}

View File

@ -13,16 +13,22 @@ import (
type statsService struct { type statsService struct {
ctx context.Context ctx context.Context
statsStream msgstream.MsgStream
replica collectionReplica replica collectionReplica
fieldStatsChan chan []*internalpb.FieldStats
statsStream msgstream.MsgStream
} }
func newStatsService(ctx context.Context, replica collectionReplica) *statsService { func newStatsService(ctx context.Context, replica collectionReplica, fieldStatsChan chan []*internalpb.FieldStats) *statsService {
return &statsService{ return &statsService{
ctx: ctx, ctx: ctx,
statsStream: nil,
replica: replica, replica: replica,
fieldStatsChan: fieldStatsChan,
statsStream: nil,
} }
} }
@ -50,7 +56,9 @@ func (sService *statsService) start() {
case <-sService.ctx.Done(): case <-sService.ctx.Done():
return return
case <-time.After(time.Duration(sleepTimeInterval) * time.Millisecond): case <-time.After(time.Duration(sleepTimeInterval) * time.Millisecond):
sService.sendSegmentStatistic() sService.publicStatistic(nil)
case fieldStats := <-sService.fieldStatsChan:
sService.publicStatistic(fieldStats)
} }
} }
} }
@ -61,20 +69,21 @@ func (sService *statsService) close() {
} }
} }
func (sService *statsService) sendSegmentStatistic() { func (sService *statsService) publicStatistic(fieldStats []*internalpb.FieldStats) {
statisticData := sService.replica.getSegmentStatistics() segStats := sService.replica.getSegmentStatistics()
// fmt.Println("Publish segment statistic") queryNodeStats := internalpb.QueryNodeStats{
// fmt.Println(statisticData) MsgType: internalpb.MsgType_kQueryNodeStats,
sService.publicStatistic(statisticData) PeerID: Params.QueryNodeID,
SegStats: segStats,
FieldStats: fieldStats,
} }
func (sService *statsService) publicStatistic(statistic *internalpb.QueryNodeStats) {
var msg msgstream.TsMsg = &msgstream.QueryNodeStatsMsg{ var msg msgstream.TsMsg = &msgstream.QueryNodeStatsMsg{
BaseMsg: msgstream.BaseMsg{ BaseMsg: msgstream.BaseMsg{
HashValues: []uint32{0}, HashValues: []uint32{0},
}, },
QueryNodeStats: *statistic, QueryNodeStats: queryNodeStats,
} }
var msgPack = msgstream.MsgPack{ var msgPack = msgstream.MsgPack{

View File

@ -10,7 +10,7 @@ import (
func TestStatsService_start(t *testing.T) { func TestStatsService_start(t *testing.T) {
node := newQueryNode() node := newQueryNode()
initTestMeta(t, node, "collection0", 0, 0) initTestMeta(t, node, "collection0", 0, 0)
node.statsService = newStatsService(node.queryNodeLoopCtx, node.replica) node.statsService = newStatsService(node.queryNodeLoopCtx, node.replica, nil)
node.statsService.start() node.statsService.start()
node.Close() node.Close()
} }
@ -32,11 +32,11 @@ func TestSegmentManagement_sendSegmentStatistic(t *testing.T) {
var statsMsgStream msgstream.MsgStream = statsStream var statsMsgStream msgstream.MsgStream = statsStream
node.statsService = newStatsService(node.queryNodeLoopCtx, node.replica) node.statsService = newStatsService(node.queryNodeLoopCtx, node.replica, nil)
node.statsService.statsStream = statsMsgStream node.statsService.statsStream = statsMsgStream
node.statsService.statsStream.Start() node.statsService.statsStream.Start()
// send stats // send stats
node.statsService.sendSegmentStatistic() node.statsService.publicStatistic(nil)
node.Close() node.Close()
} }

View File

@ -13,6 +13,14 @@ type MsgStreamMsg struct {
timestampMax Timestamp timestampMax Timestamp
} }
func GenerateMsgStreamMsg(tsMessages []msgstream.TsMsg, timestampMin, timestampMax Timestamp) *MsgStreamMsg {
return &MsgStreamMsg{
tsMessages: tsMessages,
timestampMin: timestampMin,
timestampMax: timestampMax,
}
}
func (msMsg *MsgStreamMsg) TimeTick() Timestamp { func (msMsg *MsgStreamMsg) TimeTick() Timestamp {
return msMsg.timestampMax return msMsg.timestampMax
} }

View File

@ -39,7 +39,7 @@ func (dsService *dataSyncService) initNodes() {
var dmStreamNode Node = newDmInputNode(dsService.ctx) var dmStreamNode Node = newDmInputNode(dsService.ctx)
var ddStreamNode Node = newDDInputNode(dsService.ctx) var ddStreamNode Node = newDDInputNode(dsService.ctx)
var ddNode Node = newDDNode() var ddNode Node = newDDNode(dsService.ctx)
var filterDmNode Node = newFilteredDmNode() var filterDmNode Node = newFilteredDmNode()
var insertBufferNode Node = newInsertBufferNode(dsService.ctx) var insertBufferNode Node = newInsertBufferNode(dsService.ctx)

View File

@ -24,7 +24,7 @@ import (
// NOTE: start pulsar before test // NOTE: start pulsar before test
func TestDataSyncService_Start(t *testing.T) { func TestDataSyncService_Start(t *testing.T) {
newMeta() newMeta()
const ctxTimeInMillisecond = 200 const ctxTimeInMillisecond = 2000
const closeWithDeadline = true const closeWithDeadline = true
var ctx context.Context var ctx context.Context

View File

@ -1,47 +0,0 @@
package writenode
import (
"errors"
"strconv"
)
type ddBuffer struct {
collectionBuffer map[UniqueID]interface{}
partitionBuffer map[UniqueID]interface{}
}
func (d *ddBuffer) addCollection(collectionID UniqueID) error {
if _, ok := d.collectionBuffer[collectionID]; !ok {
return errors.New("collection " + strconv.FormatInt(collectionID, 10) + " is already exists")
}
d.collectionBuffer[collectionID] = nil
return nil
}
func (d *ddBuffer) removeCollection(collectionID UniqueID) error {
if _, ok := d.collectionBuffer[collectionID]; !ok {
return errors.New("cannot found collection " + strconv.FormatInt(collectionID, 10))
}
delete(d.collectionBuffer, collectionID)
return nil
}
func (d *ddBuffer) addPartition(partitionID UniqueID) error {
if _, ok := d.partitionBuffer[partitionID]; !ok {
return errors.New("partition " + strconv.FormatInt(partitionID, 10) + " is already exists")
}
d.partitionBuffer[partitionID] = nil
return nil
}
func (d *ddBuffer) removePartition(partitionID UniqueID) error {
if _, ok := d.partitionBuffer[partitionID]; !ok {
return errors.New("cannot found partition " + strconv.FormatInt(partitionID, 10))
}
delete(d.partitionBuffer, partitionID)
return nil
}

View File

@ -1,20 +1,65 @@
package writenode package writenode
import ( import (
"context"
"errors"
"log" "log"
"sort" "sort"
"strconv"
"github.com/golang/protobuf/proto" "github.com/golang/protobuf/proto"
"github.com/minio/minio-go/v7"
"github.com/minio/minio-go/v7/pkg/credentials"
"github.com/zilliztech/milvus-distributed/internal/allocator"
"github.com/zilliztech/milvus-distributed/internal/kv"
miniokv "github.com/zilliztech/milvus-distributed/internal/kv/minio"
"github.com/zilliztech/milvus-distributed/internal/msgstream" "github.com/zilliztech/milvus-distributed/internal/msgstream"
internalPb "github.com/zilliztech/milvus-distributed/internal/proto/internalpb" internalPb "github.com/zilliztech/milvus-distributed/internal/proto/internalpb"
"github.com/zilliztech/milvus-distributed/internal/proto/schemapb" "github.com/zilliztech/milvus-distributed/internal/proto/schemapb"
"github.com/zilliztech/milvus-distributed/internal/storage"
) )
type ddNode struct { type ddNode struct {
BaseNode BaseNode
ddMsg *ddMsg ddMsg *ddMsg
ddRecords *ddRecords
ddBuffer *ddBuffer ddBuffer *ddBuffer
idAllocator *allocator.IDAllocator
kv kv.Base
}
type ddData struct {
ddRequestString []string
timestamps []Timestamp
eventTypes []storage.EventTypeCode
}
type ddBuffer struct {
ddData map[UniqueID]*ddData
maxSize int
}
type ddRecords struct {
collectionRecords map[UniqueID]interface{}
partitionRecords map[UniqueID]interface{}
}
func (d *ddBuffer) size() int {
if d.ddData == nil || len(d.ddData) <= 0 {
return 0
}
size := 0
for _, data := range d.ddData {
size += len(data.ddRequestString)
}
return size
}
func (d *ddBuffer) full() bool {
return d.size() >= d.maxSize
} }
func (ddNode *ddNode) Name() string { func (ddNode *ddNode) Name() string {
@ -68,6 +113,62 @@ func (ddNode *ddNode) Operate(in []*Msg) []*Msg {
} }
} }
// generate binlog
if ddNode.ddBuffer.full() {
ddCodec := &storage.DataDefinitionCodec{}
for collectionID, data := range ddNode.ddBuffer.ddData {
// buffer data to binlog
binLogs, err := ddCodec.Serialize(data.timestamps, data.ddRequestString, data.eventTypes)
if err != nil {
log.Println(err)
continue
}
if len(binLogs) != 2 {
log.Println("illegal binLogs")
continue
}
// binLogs -> minIO/S3
if len(data.ddRequestString) != len(data.timestamps) ||
len(data.timestamps) != len(data.eventTypes) {
log.Println("illegal ddBuffer, failed to save binlog")
continue
} else {
// Blob key example:
// ${tenant}/data_definition_log/${collection_id}/ts/${log_idx}
// ${tenant}/data_definition_log/${collection_id}/ddl/${log_idx}
keyCommon := Params.DdLogRootPath + strconv.FormatInt(collectionID, 10) + "/"
// save ts binlog
timestampLogIdx, err := ddNode.idAllocator.AllocOne()
if err != nil {
log.Println(err)
}
timestampKey := keyCommon + binLogs[0].GetKey() + "/" + strconv.FormatInt(timestampLogIdx, 10)
err = ddNode.kv.Save(timestampKey, string(binLogs[0].GetValue()))
if err != nil {
log.Println(err)
}
log.Println("save ts binlog, key = ", timestampKey)
// save dd binlog
ddLogIdx, err := ddNode.idAllocator.AllocOne()
if err != nil {
log.Println(err)
}
ddKey := keyCommon + binLogs[1].GetKey() + "/" + strconv.FormatInt(ddLogIdx, 10)
err = ddNode.kv.Save(ddKey, string(binLogs[1].GetValue()))
if err != nil {
log.Println(err)
}
log.Println("save dd binlog, key = ", ddKey)
}
}
// clear buffer
ddNode.ddBuffer.ddData = make(map[UniqueID]*ddData)
log.Println("dd buffer flushed")
}
var res Msg = ddNode.ddMsg var res Msg = ddNode.ddMsg
return []*Msg{&res} return []*Msg{&res}
} }
@ -75,16 +176,18 @@ func (ddNode *ddNode) Operate(in []*Msg) []*Msg {
func (ddNode *ddNode) createCollection(msg *msgstream.CreateCollectionMsg) { func (ddNode *ddNode) createCollection(msg *msgstream.CreateCollectionMsg) {
collectionID := msg.CollectionID collectionID := msg.CollectionID
err := ddNode.ddBuffer.addCollection(collectionID) // add collection
if err != nil { if _, ok := ddNode.ddRecords.collectionRecords[collectionID]; ok {
err := errors.New("collection " + strconv.FormatInt(collectionID, 10) + " is already exists")
log.Println(err) log.Println(err)
return return
} }
ddNode.ddRecords.collectionRecords[collectionID] = nil
// TODO: add default partition? // TODO: add default partition?
var schema schemapb.CollectionSchema var schema schemapb.CollectionSchema
err = proto.Unmarshal((*msg.Schema).Value, &schema) err := proto.Unmarshal((*msg.Schema).Value, &schema)
if err != nil { if err != nil {
log.Println(err) log.Println(err)
return return
@ -96,17 +199,30 @@ func (ddNode *ddNode) createCollection(msg *msgstream.CreateCollectionMsg) {
timestamp: msg.Timestamp, timestamp: msg.Timestamp,
}) })
// TODO: write dd binlog _, ok := ddNode.ddBuffer.ddData[collectionID]
if !ok {
ddNode.ddBuffer.ddData[collectionID] = &ddData{
ddRequestString: make([]string, 0),
timestamps: make([]Timestamp, 0),
eventTypes: make([]storage.EventTypeCode, 0),
}
}
ddNode.ddBuffer.ddData[collectionID].ddRequestString = append(ddNode.ddBuffer.ddData[collectionID].ddRequestString, msg.CreateCollectionRequest.String())
ddNode.ddBuffer.ddData[collectionID].timestamps = append(ddNode.ddBuffer.ddData[collectionID].timestamps, msg.Timestamp)
ddNode.ddBuffer.ddData[collectionID].eventTypes = append(ddNode.ddBuffer.ddData[collectionID].eventTypes, storage.CreateCollectionEventType)
} }
func (ddNode *ddNode) dropCollection(msg *msgstream.DropCollectionMsg) { func (ddNode *ddNode) dropCollection(msg *msgstream.DropCollectionMsg) {
collectionID := msg.CollectionID collectionID := msg.CollectionID
err := ddNode.ddBuffer.removeCollection(collectionID) // remove collection
if err != nil { if _, ok := ddNode.ddRecords.collectionRecords[collectionID]; !ok {
err := errors.New("cannot found collection " + strconv.FormatInt(collectionID, 10))
log.Println(err) log.Println(err)
return return
} }
delete(ddNode.ddRecords.collectionRecords, collectionID)
collectionName := msg.CollectionName.CollectionName collectionName := msg.CollectionName.CollectionName
ddNode.ddMsg.collectionRecords[collectionName] = append(ddNode.ddMsg.collectionRecords[collectionName], ddNode.ddMsg.collectionRecords[collectionName] = append(ddNode.ddMsg.collectionRecords[collectionName],
@ -115,17 +231,31 @@ func (ddNode *ddNode) dropCollection(msg *msgstream.DropCollectionMsg) {
timestamp: msg.Timestamp, timestamp: msg.Timestamp,
}) })
// TODO: write dd binlog _, ok := ddNode.ddBuffer.ddData[collectionID]
if !ok {
ddNode.ddBuffer.ddData[collectionID] = &ddData{
ddRequestString: make([]string, 0),
timestamps: make([]Timestamp, 0),
eventTypes: make([]storage.EventTypeCode, 0),
}
}
ddNode.ddBuffer.ddData[collectionID].ddRequestString = append(ddNode.ddBuffer.ddData[collectionID].ddRequestString, msg.DropCollectionRequest.String())
ddNode.ddBuffer.ddData[collectionID].timestamps = append(ddNode.ddBuffer.ddData[collectionID].timestamps, msg.Timestamp)
ddNode.ddBuffer.ddData[collectionID].eventTypes = append(ddNode.ddBuffer.ddData[collectionID].eventTypes, storage.DropCollectionEventType)
} }
func (ddNode *ddNode) createPartition(msg *msgstream.CreatePartitionMsg) { func (ddNode *ddNode) createPartition(msg *msgstream.CreatePartitionMsg) {
partitionID := msg.PartitionID partitionID := msg.PartitionID
collectionID := msg.CollectionID
err := ddNode.ddBuffer.addPartition(partitionID) // add partition
if err != nil { if _, ok := ddNode.ddRecords.partitionRecords[partitionID]; ok {
err := errors.New("partition " + strconv.FormatInt(partitionID, 10) + " is already exists")
log.Println(err) log.Println(err)
return return
} }
ddNode.ddRecords.partitionRecords[partitionID] = nil
partitionTag := msg.PartitionName.Tag partitionTag := msg.PartitionName.Tag
ddNode.ddMsg.partitionRecords[partitionTag] = append(ddNode.ddMsg.partitionRecords[partitionTag], ddNode.ddMsg.partitionRecords[partitionTag] = append(ddNode.ddMsg.partitionRecords[partitionTag],
@ -134,17 +264,31 @@ func (ddNode *ddNode) createPartition(msg *msgstream.CreatePartitionMsg) {
timestamp: msg.Timestamp, timestamp: msg.Timestamp,
}) })
// TODO: write dd binlog _, ok := ddNode.ddBuffer.ddData[collectionID]
if !ok {
ddNode.ddBuffer.ddData[collectionID] = &ddData{
ddRequestString: make([]string, 0),
timestamps: make([]Timestamp, 0),
eventTypes: make([]storage.EventTypeCode, 0),
}
}
ddNode.ddBuffer.ddData[collectionID].ddRequestString = append(ddNode.ddBuffer.ddData[collectionID].ddRequestString, msg.CreatePartitionRequest.String())
ddNode.ddBuffer.ddData[collectionID].timestamps = append(ddNode.ddBuffer.ddData[collectionID].timestamps, msg.Timestamp)
ddNode.ddBuffer.ddData[collectionID].eventTypes = append(ddNode.ddBuffer.ddData[collectionID].eventTypes, storage.CreatePartitionEventType)
} }
func (ddNode *ddNode) dropPartition(msg *msgstream.DropPartitionMsg) { func (ddNode *ddNode) dropPartition(msg *msgstream.DropPartitionMsg) {
partitionID := msg.PartitionID partitionID := msg.PartitionID
collectionID := msg.CollectionID
err := ddNode.ddBuffer.removePartition(partitionID) // remove partition
if err != nil { if _, ok := ddNode.ddRecords.partitionRecords[partitionID]; !ok {
err := errors.New("cannot found partition " + strconv.FormatInt(partitionID, 10))
log.Println(err) log.Println(err)
return return
} }
delete(ddNode.ddRecords.partitionRecords, partitionID)
partitionTag := msg.PartitionName.Tag partitionTag := msg.PartitionName.Tag
ddNode.ddMsg.partitionRecords[partitionTag] = append(ddNode.ddMsg.partitionRecords[partitionTag], ddNode.ddMsg.partitionRecords[partitionTag] = append(ddNode.ddMsg.partitionRecords[partitionTag],
@ -153,10 +297,21 @@ func (ddNode *ddNode) dropPartition(msg *msgstream.DropPartitionMsg) {
timestamp: msg.Timestamp, timestamp: msg.Timestamp,
}) })
// TODO: write dd binlog _, ok := ddNode.ddBuffer.ddData[collectionID]
if !ok {
ddNode.ddBuffer.ddData[collectionID] = &ddData{
ddRequestString: make([]string, 0),
timestamps: make([]Timestamp, 0),
eventTypes: make([]storage.EventTypeCode, 0),
}
} }
func newDDNode() *ddNode { ddNode.ddBuffer.ddData[collectionID].ddRequestString = append(ddNode.ddBuffer.ddData[collectionID].ddRequestString, msg.DropPartitionRequest.String())
ddNode.ddBuffer.ddData[collectionID].timestamps = append(ddNode.ddBuffer.ddData[collectionID].timestamps, msg.Timestamp)
ddNode.ddBuffer.ddData[collectionID].eventTypes = append(ddNode.ddBuffer.ddData[collectionID].eventTypes, storage.DropPartitionEventType)
}
func newDDNode(ctx context.Context) *ddNode {
maxQueueLength := Params.FlowGraphMaxQueueLength maxQueueLength := Params.FlowGraphMaxQueueLength
maxParallelism := Params.FlowGraphMaxParallelism maxParallelism := Params.FlowGraphMaxParallelism
@ -164,13 +319,46 @@ func newDDNode() *ddNode {
baseNode.SetMaxQueueLength(maxQueueLength) baseNode.SetMaxQueueLength(maxQueueLength)
baseNode.SetMaxParallelism(maxParallelism) baseNode.SetMaxParallelism(maxParallelism)
ddBuffer := &ddBuffer{ ddRecords := &ddRecords{
collectionBuffer: make(map[UniqueID]interface{}), collectionRecords: make(map[UniqueID]interface{}),
partitionBuffer: make(map[UniqueID]interface{}), partitionRecords: make(map[UniqueID]interface{}),
}
minIOEndPoint := Params.MinioAddress
minIOAccessKeyID := Params.MinioAccessKeyID
minIOSecretAccessKey := Params.MinioSecretAccessKey
minIOUseSSL := Params.MinioUseSSL
minIOClient, err := minio.New(minIOEndPoint, &minio.Options{
Creds: credentials.NewStaticV4(minIOAccessKeyID, minIOSecretAccessKey, ""),
Secure: minIOUseSSL,
})
if err != nil {
panic(err)
}
// TODO: load bucket name from yaml?
minioKV, err := miniokv.NewMinIOKV(ctx, minIOClient, "write-node-dd-node")
if err != nil {
panic(err)
}
idAllocator, err := allocator.NewIDAllocator(ctx, Params.MasterAddress)
if err != nil {
panic(err)
}
err = idAllocator.Start()
if err != nil {
panic(err)
} }
return &ddNode{ return &ddNode{
BaseNode: baseNode, BaseNode: baseNode,
ddBuffer: ddBuffer, ddRecords: ddRecords,
ddBuffer: &ddBuffer{
ddData: make(map[UniqueID]*ddData),
maxSize: Params.FlushDdBufSize,
},
idAllocator: idAllocator,
kv: minioKV,
} }
} }

View File

@ -0,0 +1,126 @@
package writenode
import (
"context"
"testing"
"time"
"github.com/zilliztech/milvus-distributed/internal/msgstream"
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb"
"github.com/zilliztech/milvus-distributed/internal/proto/servicepb"
"github.com/zilliztech/milvus-distributed/internal/util/flowgraph"
)
func TestFlowGraphDDNode_Operate(t *testing.T) {
const ctxTimeInMillisecond = 2000
const closeWithDeadline = false
var ctx context.Context
if closeWithDeadline {
var cancel context.CancelFunc
d := time.Now().Add(ctxTimeInMillisecond * time.Millisecond)
ctx, cancel = context.WithDeadline(context.Background(), d)
defer cancel()
} else {
ctx = context.Background()
}
startMaster(ctx)
Params.FlushDdBufSize = 4
ddNode := newDDNode(ctx)
colID := UniqueID(0)
colName := "col-test-0"
// create collection
createColReq := internalpb.CreateCollectionRequest{
MsgType: internalpb.MsgType_kCreateCollection,
CollectionID: colID,
ReqID: 1,
Timestamp: 1,
ProxyID: 1,
Schema: &commonpb.Blob{},
}
createColMsg := msgstream.CreateCollectionMsg{
BaseMsg: msgstream.BaseMsg{
BeginTimestamp: Timestamp(1),
EndTimestamp: Timestamp(1),
HashValues: []uint32{uint32(0)},
},
CreateCollectionRequest: createColReq,
}
// drop collection
dropColReq := internalpb.DropCollectionRequest{
MsgType: internalpb.MsgType_kDropCollection,
CollectionID: colID,
ReqID: 2,
Timestamp: 2,
ProxyID: 2,
CollectionName: &servicepb.CollectionName{CollectionName: colName},
}
dropColMsg := msgstream.DropCollectionMsg{
BaseMsg: msgstream.BaseMsg{
BeginTimestamp: Timestamp(2),
EndTimestamp: Timestamp(2),
HashValues: []uint32{uint32(0)},
},
DropCollectionRequest: dropColReq,
}
partitionID := UniqueID(100)
partitionTag := "partition-test-0"
// create partition
createPartitionReq := internalpb.CreatePartitionRequest{
MsgType: internalpb.MsgType_kCreatePartition,
CollectionID: colID,
PartitionID: partitionID,
ReqID: 3,
Timestamp: 3,
ProxyID: 3,
PartitionName: &servicepb.PartitionName{
CollectionName: colName,
Tag: partitionTag,
},
}
createPartitionMsg := msgstream.CreatePartitionMsg{
BaseMsg: msgstream.BaseMsg{
BeginTimestamp: Timestamp(3),
EndTimestamp: Timestamp(3),
HashValues: []uint32{uint32(0)},
},
CreatePartitionRequest: createPartitionReq,
}
// drop partition
dropPartitionReq := internalpb.DropPartitionRequest{
MsgType: internalpb.MsgType_kDropPartition,
CollectionID: colID,
PartitionID: partitionID,
ReqID: 4,
Timestamp: 4,
ProxyID: 4,
PartitionName: &servicepb.PartitionName{
CollectionName: colName,
Tag: partitionTag,
},
}
dropPartitionMsg := msgstream.DropPartitionMsg{
BaseMsg: msgstream.BaseMsg{
BeginTimestamp: Timestamp(4),
EndTimestamp: Timestamp(4),
HashValues: []uint32{uint32(0)},
},
DropPartitionRequest: dropPartitionReq,
}
tsMessages := make([]msgstream.TsMsg, 0)
tsMessages = append(tsMessages, msgstream.TsMsg(&createColMsg))
tsMessages = append(tsMessages, msgstream.TsMsg(&dropColMsg))
tsMessages = append(tsMessages, msgstream.TsMsg(&createPartitionMsg))
tsMessages = append(tsMessages, msgstream.TsMsg(&dropPartitionMsg))
msgStream := flowgraph.GenerateMsgStreamMsg(tsMessages, Timestamp(0), Timestamp(3))
var inMsg Msg = msgStream
ddNode.Operate([]*Msg{&inMsg})
}

View File

@ -14,7 +14,8 @@ import (
type metaTable struct { type metaTable struct {
client kv.TxnBase // client of a reliable kv service, i.e. etcd client client kv.TxnBase // client of a reliable kv service, i.e. etcd client
segID2FlushMeta map[UniqueID]pb.SegmentFlushMeta // index id to index meta segID2FlushMeta map[UniqueID]pb.SegmentFlushMeta // segment id to flush meta
collID2DdlMeta map[UniqueID]*pb.DDLFlushMeta
lock sync.RWMutex lock sync.RWMutex
} }
@ -24,14 +25,107 @@ func NewMetaTable(kv kv.TxnBase) (*metaTable, error) {
client: kv, client: kv,
lock: sync.RWMutex{}, lock: sync.RWMutex{},
} }
err := mt.reloadFromKV() err := mt.reloadSegMetaFromKV()
if err != nil {
return nil, err
}
err = mt.reloadDdlMetaFromKV()
if err != nil { if err != nil {
return nil, err return nil, err
} }
return mt, nil return mt, nil
} }
func (mt *metaTable) reloadFromKV() error { func (mt *metaTable) AppendDDLBinlogPaths(collID UniqueID, paths []string) error {
mt.lock.Lock()
defer mt.lock.Unlock()
_, ok := mt.collID2DdlMeta[collID]
if !ok {
mt.collID2DdlMeta[collID] = &pb.DDLFlushMeta{
CollectionID: collID,
BinlogPaths: make([]string, 0),
}
}
meta := mt.collID2DdlMeta[collID]
meta.BinlogPaths = append(meta.BinlogPaths, paths...)
return mt.saveDDLFlushMeta(meta)
}
func (mt *metaTable) AppendSegBinlogPaths(timestamp Timestamp, segmentID UniqueID, fieldID int32, dataPaths []string) error {
mt.lock.Lock()
defer mt.lock.Unlock()
_, ok := mt.segID2FlushMeta[segmentID]
if !ok {
err := mt.addSegmentFlush(segmentID, timestamp)
if err != nil {
return err
}
}
meta := mt.segID2FlushMeta[segmentID]
found := false
for _, field := range meta.Fields {
if field.FieldID == fieldID {
field.BinlogPaths = append(field.BinlogPaths, dataPaths...)
found = true
break
}
}
if !found {
newField := &pb.FieldFlushMeta{
FieldID: fieldID,
BinlogPaths: dataPaths,
}
meta.Fields = append(meta.Fields, newField)
}
return mt.saveSegFlushMeta(&meta)
}
// metaTable.lock.Lock() before call this function
func (mt *metaTable) saveDDLFlushMeta(meta *pb.DDLFlushMeta) error {
value := proto.MarshalTextString(meta)
mt.collID2DdlMeta[meta.CollectionID] = meta
return mt.client.Save("/writer/ddl/"+strconv.FormatInt(meta.CollectionID, 10), value)
}
func (mt *metaTable) reloadDdlMetaFromKV() error {
mt.collID2DdlMeta = make(map[UniqueID]*pb.DDLFlushMeta)
_, values, err := mt.client.LoadWithPrefix("writer/ddl")
if err != nil {
return err
}
for _, value := range values {
ddlMeta := &pb.DDLFlushMeta{}
err = proto.UnmarshalText(value, ddlMeta)
if err != nil {
return err
}
mt.collID2DdlMeta[ddlMeta.CollectionID] = ddlMeta
}
return nil
}
// metaTable.lock.Lock() before call this function
func (mt *metaTable) saveSegFlushMeta(meta *pb.SegmentFlushMeta) error {
value := proto.MarshalTextString(meta)
mt.segID2FlushMeta[meta.SegmentID] = *meta
return mt.client.Save("/writer/segment/"+strconv.FormatInt(meta.SegmentID, 10), value)
}
func (mt *metaTable) reloadSegMetaFromKV() error {
mt.segID2FlushMeta = make(map[UniqueID]pb.SegmentFlushMeta) mt.segID2FlushMeta = make(map[UniqueID]pb.SegmentFlushMeta)
_, values, err := mt.client.LoadWithPrefix("writer/segment") _, values, err := mt.client.LoadWithPrefix("writer/segment")
@ -47,19 +141,11 @@ func (mt *metaTable) reloadFromKV() error {
} }
mt.segID2FlushMeta[flushMeta.SegmentID] = flushMeta mt.segID2FlushMeta[flushMeta.SegmentID] = flushMeta
} }
return nil return nil
} }
// metaTable.lock.Lock() before call this function func (mt *metaTable) addSegmentFlush(segmentID UniqueID, timestamp Timestamp) error {
func (mt *metaTable) saveFlushMeta(meta *pb.SegmentFlushMeta) error {
value := proto.MarshalTextString(meta)
mt.segID2FlushMeta[meta.SegmentID] = *meta
return mt.client.Save("/writer/segment/"+strconv.FormatInt(meta.SegmentID, 10), value)
}
func (mt *metaTable) AddSegmentFlush(segmentID UniqueID) error {
mt.lock.Lock() mt.lock.Lock()
defer mt.lock.Unlock() defer mt.lock.Unlock()
_, ok := mt.segID2FlushMeta[segmentID] _, ok := mt.segID2FlushMeta[segmentID]
@ -69,8 +155,9 @@ func (mt *metaTable) AddSegmentFlush(segmentID UniqueID) error {
meta := pb.SegmentFlushMeta{ meta := pb.SegmentFlushMeta{
IsClosed: false, IsClosed: false,
SegmentID: segmentID, SegmentID: segmentID,
OpenTime: timestamp,
} }
return mt.saveFlushMeta(&meta) return mt.saveSegFlushMeta(&meta)
} }
func (mt *metaTable) getFlushCloseTime(segmentID UniqueID) (Timestamp, error) { func (mt *metaTable) getFlushCloseTime(segmentID UniqueID) (Timestamp, error) {
@ -83,28 +170,6 @@ func (mt *metaTable) getFlushCloseTime(segmentID UniqueID) (Timestamp, error) {
return meta.CloseTime, nil return meta.CloseTime, nil
} }
func (mt *metaTable) SetFlushCloseTime(segmentID UniqueID, t Timestamp) error {
mt.lock.Lock()
defer mt.lock.Unlock()
meta, ok := mt.segID2FlushMeta[segmentID]
if !ok {
return errors.Errorf("segment not exists with ID = " + strconv.FormatInt(segmentID, 10))
}
meta.CloseTime = t
return mt.saveFlushMeta(&meta)
}
func (mt *metaTable) SetFlushOpenTime(segmentID UniqueID, t Timestamp) error {
mt.lock.Lock()
defer mt.lock.Unlock()
meta, ok := mt.segID2FlushMeta[segmentID]
if !ok {
return errors.Errorf("segment not exists with ID = " + strconv.FormatInt(segmentID, 10))
}
meta.OpenTime = t
return mt.saveFlushMeta(&meta)
}
func (mt *metaTable) getFlushOpenTime(segmentID UniqueID) (Timestamp, error) { func (mt *metaTable) getFlushOpenTime(segmentID UniqueID) (Timestamp, error) {
mt.lock.Lock() mt.lock.Lock()
defer mt.lock.Unlock() defer mt.lock.Unlock()
@ -115,7 +180,7 @@ func (mt *metaTable) getFlushOpenTime(segmentID UniqueID) (Timestamp, error) {
return meta.OpenTime, nil return meta.OpenTime, nil
} }
func (mt *metaTable) CompleteFlush(segmentID UniqueID) error { func (mt *metaTable) CompleteFlush(segmentID UniqueID, timestamp Timestamp) error {
mt.lock.Lock() mt.lock.Lock()
defer mt.lock.Unlock() defer mt.lock.Unlock()
meta, ok := mt.segID2FlushMeta[segmentID] meta, ok := mt.segID2FlushMeta[segmentID]
@ -123,7 +188,9 @@ func (mt *metaTable) CompleteFlush(segmentID UniqueID) error {
return errors.Errorf("segment not exists with ID = " + strconv.FormatInt(segmentID, 10)) return errors.Errorf("segment not exists with ID = " + strconv.FormatInt(segmentID, 10))
} }
meta.IsClosed = true meta.IsClosed = true
return mt.saveFlushMeta(&meta) meta.CloseTime = timestamp
return mt.saveSegFlushMeta(&meta)
} }
func (mt *metaTable) checkFlushComplete(segmentID UniqueID) (bool, error) { func (mt *metaTable) checkFlushComplete(segmentID UniqueID) (bool, error) {
@ -136,34 +203,7 @@ func (mt *metaTable) checkFlushComplete(segmentID UniqueID) (bool, error) {
return meta.IsClosed, nil return meta.IsClosed, nil
} }
func (mt *metaTable) AppendBinlogPaths(segmentID UniqueID, fieldID int32, dataPaths []string) error { func (mt *metaTable) getSegBinlogPaths(segmentID UniqueID) (map[int32][]string, error) {
mt.lock.Lock()
defer mt.lock.Unlock()
meta, ok := mt.segID2FlushMeta[segmentID]
if !ok {
return errors.Errorf("segment not exists with ID = " + strconv.FormatInt(segmentID, 10))
}
found := false
for _, field := range meta.Fields {
if field.FieldID == fieldID {
field.BinlogPaths = append(field.BinlogPaths, dataPaths...)
found = true
break
}
}
if !found {
newField := &pb.FieldFlushMeta{
FieldID: fieldID,
BinlogPaths: dataPaths,
}
meta.Fields = append(meta.Fields, newField)
}
return mt.saveFlushMeta(&meta)
}
func (mt *metaTable) getBinlogPaths(segmentID UniqueID) (map[int32][]string, error) {
mt.lock.Lock() mt.lock.Lock()
defer mt.lock.Unlock() defer mt.lock.Unlock()
@ -177,3 +217,16 @@ func (mt *metaTable) getBinlogPaths(segmentID UniqueID) (map[int32][]string, err
} }
return ret, nil return ret, nil
} }
func (mt *metaTable) getDDLBinlogPaths(collID UniqueID) (map[UniqueID][]string, error) {
mt.lock.Lock()
defer mt.lock.Unlock()
meta, ok := mt.collID2DdlMeta[collID]
if !ok {
return nil, errors.Errorf("collection not exists with ID = " + strconv.FormatInt(collID, 10))
}
ret := make(map[UniqueID][]string)
ret[meta.CollectionID] = meta.BinlogPaths
return ret, nil
}

View File

@ -5,72 +5,55 @@ import (
"testing" "testing"
"github.com/stretchr/testify/assert" "github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
etcdkv "github.com/zilliztech/milvus-distributed/internal/kv/etcd" etcdkv "github.com/zilliztech/milvus-distributed/internal/kv/etcd"
"go.etcd.io/etcd/clientv3" "go.etcd.io/etcd/clientv3"
) )
func createMetaTable(t *testing.T) *metaTable { func TestMetaTable_all(t *testing.T) {
etcdAddr := Params.EtcdAddress etcdAddr := Params.EtcdAddress
cli, err := clientv3.New(clientv3.Config{Endpoints: []string{etcdAddr}}) cli, err := clientv3.New(clientv3.Config{Endpoints: []string{etcdAddr}})
assert.Nil(t, err) require.NoError(t, err)
etcdKV := etcdkv.NewEtcdKV(cli, "/etcd/test/root/writer") etcdKV := etcdkv.NewEtcdKV(cli, "/etcd/test/root/writer")
_, err = cli.Delete(context.TODO(), "/etcd/test/root/writer", clientv3.WithPrefix()) _, err = cli.Delete(context.TODO(), "/etcd/test/root/writer", clientv3.WithPrefix())
assert.Nil(t, err) require.NoError(t, err)
meta, err := NewMetaTable(etcdKV) meta, err := NewMetaTable(etcdKV)
assert.Nil(t, err) assert.NoError(t, err)
return meta
}
func TestMetaTable_AddSegmentFlush(t *testing.T) {
meta := createMetaTable(t)
defer meta.client.Close()
err := meta.AddSegmentFlush(1)
assert.Nil(t, err)
err = meta.AddSegmentFlush(2)
assert.Nil(t, err)
err = meta.AddSegmentFlush(2)
assert.NotNil(t, err)
err = meta.reloadFromKV()
assert.Nil(t, err)
}
func TestMetaTable_SetFlushTime(t *testing.T) {
meta := createMetaTable(t)
defer meta.client.Close() defer meta.client.Close()
var segmentID UniqueID = 1 t.Run("TestMetaTable_addSegmentFlush_and_OpenTime", func(t *testing.T) {
tsOpen := Timestamp(100)
err := meta.AddSegmentFlush(segmentID) err := meta.addSegmentFlush(101, tsOpen)
assert.Nil(t, err) assert.NoError(t, err)
exp, err := meta.getFlushOpenTime(101)
tsOpen := Timestamp(1000) assert.NoError(t, err)
err = meta.SetFlushOpenTime(segmentID, tsOpen)
assert.Nil(t, err)
exp, err := meta.getFlushOpenTime(segmentID)
assert.Nil(t, err)
assert.Equal(t, tsOpen, exp) assert.Equal(t, tsOpen, exp)
tsClose := Timestamp(10001) tsOpen = Timestamp(200)
err = meta.SetFlushCloseTime(segmentID, tsClose) err = meta.addSegmentFlush(102, tsOpen)
assert.Nil(t, err) assert.NoError(t, err)
exp, err = meta.getFlushOpenTime(102)
assert.NoError(t, err)
assert.Equal(t, tsOpen, exp)
exp, err = meta.getFlushCloseTime(segmentID) tsOpen = Timestamp(200)
assert.Nil(t, err) err = meta.addSegmentFlush(103, tsOpen)
assert.Equal(t, tsClose, exp) assert.NoError(t, err)
} exp, err = meta.getFlushOpenTime(103)
assert.NoError(t, err)
assert.Equal(t, tsOpen, exp)
func TestMetaTable_AppendBinlogPaths(t *testing.T) { err = meta.reloadSegMetaFromKV()
meta := createMetaTable(t) assert.NoError(t, err)
defer meta.client.Close() })
var segmentID UniqueID = 1
err := meta.AddSegmentFlush(segmentID) t.Run("TestMetaTable_AppendSegBinlogPaths", func(t *testing.T) {
segmentID := UniqueID(201)
tsOpen := Timestamp(1000)
err := meta.addSegmentFlush(segmentID, tsOpen)
assert.Nil(t, err) assert.Nil(t, err)
exp := map[int32][]string{ exp := map[int32][]string{
@ -79,34 +62,64 @@ func TestMetaTable_AppendBinlogPaths(t *testing.T) {
} }
for fieldID, dataPaths := range exp { for fieldID, dataPaths := range exp {
for _, dp := range dataPaths { for _, dp := range dataPaths {
err = meta.AppendBinlogPaths(segmentID, fieldID, []string{dp}) err = meta.AppendSegBinlogPaths(tsOpen, segmentID, fieldID, []string{dp})
assert.Nil(t, err)
err = meta.AppendSegBinlogPaths(tsOpen, segmentID, fieldID, []string{dp})
assert.Nil(t, err) assert.Nil(t, err)
} }
} }
ret, err := meta.getBinlogPaths(segmentID) ret, err := meta.getSegBinlogPaths(segmentID)
assert.Nil(t, err) assert.Nil(t, err)
assert.Equal(t, exp, ret) assert.Equal(t,
map[int32][]string{
1: {"a", "a", "b", "b", "c", "c"},
2: {"b", "b", "a", "a", "c", "c"}},
ret)
})
t.Run("TestMetaTable_AppendDDLBinlogPaths", func(t *testing.T) {
collID2Paths := map[UniqueID][]string{
301: {"a", "b", "c"},
302: {"c", "b", "a"},
} }
func TestMetaTable_CompleteFlush(t *testing.T) { for collID, dataPaths := range collID2Paths {
meta := createMetaTable(t) for _, dp := range dataPaths {
defer meta.client.Close() err = meta.AppendDDLBinlogPaths(collID, []string{dp})
var segmentID UniqueID = 1
err := meta.AddSegmentFlush(segmentID)
assert.Nil(t, err) assert.Nil(t, err)
}
}
for k, v := range collID2Paths {
ret, err := meta.getDDLBinlogPaths(k)
assert.Nil(t, err)
assert.Equal(t, map[UniqueID][]string{k: v}, ret)
}
})
t.Run("TestMetaTable_CompleteFlush_and_CloseTime", func(t *testing.T) {
var segmentID UniqueID = 401
openTime := Timestamp(1000)
closeTime := Timestamp(10000)
err := meta.addSegmentFlush(segmentID, openTime)
assert.NoError(t, err)
ret, err := meta.checkFlushComplete(segmentID) ret, err := meta.checkFlushComplete(segmentID)
assert.Nil(t, err) assert.NoError(t, err)
assert.Equal(t, false, ret) assert.Equal(t, false, ret)
meta.CompleteFlush(segmentID) meta.CompleteFlush(segmentID, closeTime)
ret, err = meta.checkFlushComplete(segmentID) ret, err = meta.checkFlushComplete(segmentID)
assert.Nil(t, err) assert.NoError(t, err)
assert.Equal(t, true, ret) assert.Equal(t, true, ret)
ts, err := meta.getFlushCloseTime(segmentID)
assert.NoError(t, err)
assert.Equal(t, closeTime, ts)
})
} }

View File

@ -1,11 +1,18 @@
package writenode package writenode
import ( import (
"context"
"fmt" "fmt"
"log"
"math/rand" "math/rand"
"os" "os"
"strconv" "strconv"
"testing" "testing"
"go.etcd.io/etcd/clientv3"
"go.uber.org/zap"
"github.com/zilliztech/milvus-distributed/internal/master"
) )
func makeNewChannelNames(names []string, suffix string) []string { func makeNewChannelNames(names []string, suffix string) []string {
@ -22,6 +29,34 @@ func refreshChannelNames() {
Params.InsertChannelNames = makeNewChannelNames(Params.InsertChannelNames, suffix) Params.InsertChannelNames = makeNewChannelNames(Params.InsertChannelNames, suffix)
} }
func startMaster(ctx context.Context) {
master.Init()
etcdAddr := master.Params.EtcdAddress
metaRootPath := master.Params.MetaRootPath
etcdCli, err := clientv3.New(clientv3.Config{Endpoints: []string{etcdAddr}})
if err != nil {
panic(err)
}
_, err = etcdCli.Delete(context.TODO(), metaRootPath, clientv3.WithPrefix())
if err != nil {
panic(err)
}
masterPort := 53101
master.Params.Port = masterPort
svr, err := master.CreateServer(ctx)
if err != nil {
log.Print("create server failed", zap.Error(err))
}
if err := svr.Run(int64(master.Params.Port)); err != nil {
log.Fatal("run server failed", zap.Error(err))
}
fmt.Println("Waiting for server!", svr.IsServing())
Params.MasterAddress = master.Params.Address + ":" + strconv.Itoa(masterPort)
}
func TestMain(m *testing.M) { func TestMain(m *testing.M) {
Params.Init() Params.Init()
refreshChannelNames() refreshChannelNames()