mirror of
https://gitee.com/milvus-io/milvus.git
synced 2025-12-06 17:18:35 +08:00
Add ddl binlog metaTable in writenode
Signed-off-by: XuanYang-cn <xuan.yang@zilliz.com>
This commit is contained in:
parent
4dbf4ade41
commit
c0797333ce
@ -5,4 +5,4 @@ TARGET_TAG=latest
|
|||||||
PULSAR_ADDRESS=pulsar://pulsar:6650
|
PULSAR_ADDRESS=pulsar://pulsar:6650
|
||||||
ETCD_ADDRESS=etcd:2379
|
ETCD_ADDRESS=etcd:2379
|
||||||
MASTER_ADDRESS=master:53100
|
MASTER_ADDRESS=master:53100
|
||||||
MINIO_ADDRESS=minio:9000
|
MINIO_ADDRESS=minio:9000
|
||||||
|
|||||||
@ -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:
|
||||||
|
|||||||
@ -42,3 +42,7 @@ queryNode:
|
|||||||
|
|
||||||
stats:
|
stats:
|
||||||
recvBufSize: 64
|
recvBufSize: 64
|
||||||
|
|
||||||
|
loadIndex:
|
||||||
|
recvBufSize: 512
|
||||||
|
pulsarBufSize: 512
|
||||||
|
|||||||
@ -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 {
|
||||||
@ -121,16 +122,18 @@ type Allocator struct {
|
|||||||
masterClient masterpb.MasterClient
|
masterClient masterpb.MasterClient
|
||||||
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
|
}
|
||||||
|
|
||||||
|
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
|
||||||
}
|
}
|
||||||
|
|
||||||
ta.syncFunc()
|
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) {
|
||||||
|
|||||||
@ -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 {
|
||||||
|
|||||||
@ -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()
|
return timeout || len(sa.segReqs) != 0
|
||||||
} else {
|
|
||||||
sa.checkToDoReqs()
|
|
||||||
}
|
|
||||||
|
|
||||||
return len(sa.segReqs) != 0
|
|
||||||
}
|
}
|
||||||
|
|
||||||
func (sa *SegIDAssigner) syncSegments() {
|
func (sa *SegIDAssigner) checkSegReqEqual(req1, req2 *internalpb.SegIDRequest) bool {
|
||||||
|
if req1 == nil || req2 == nil {
|
||||||
|
return false
|
||||||
|
}
|
||||||
|
|
||||||
|
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()
|
||||||
|
|||||||
@ -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 {
|
||||||
|
|||||||
23
internal/core/src/common/LoadIndex.h
Normal file
23
internal/core/src/common/LoadIndex.h
Normal 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;
|
||||||
|
};
|
||||||
@ -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}
|
||||||
)
|
)
|
||||||
|
|||||||
139
internal/core/src/segcore/load_index_c.cpp
Normal file
139
internal/core/src/segcore/load_index_c.cpp
Normal 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;
|
||||||
|
}
|
||||||
|
}
|
||||||
45
internal/core/src/segcore/load_index_c.h
Normal file
45
internal/core/src/segcore/load_index_c.h
Normal 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
|
||||||
@ -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
|
||||||
|
|||||||
@ -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
|
||||||
|
|||||||
@ -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;
|
||||||
|
}
|
||||||
|
}
|
||||||
@ -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{
|
||||||
|
|||||||
@ -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
|
||||||
|
|||||||
@ -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{
|
||||||
|
|||||||
@ -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
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|||||||
@ -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 {
|
||||||
|
|||||||
@ -2043,13 +2043,15 @@ 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"`
|
||||||
XXX_NoUnkeyedLiteral struct{} `json:"-"`
|
IndexPaths []string `protobuf:"bytes,5,rep,name=index_paths,json=indexPaths,proto3" json:"index_paths,omitempty"`
|
||||||
XXX_unrecognized []byte `json:"-"`
|
IndexParams []*commonpb.KeyValuePair `protobuf:"bytes,6,rep,name=index_params,json=indexParams,proto3" json:"index_params,omitempty"`
|
||||||
XXX_sizecache int32 `json:"-"`
|
XXX_NoUnkeyedLiteral struct{} `json:"-"`
|
||||||
|
XXX_unrecognized []byte `json:"-"`
|
||||||
|
XXX_sizecache int32 `json:"-"`
|
||||||
}
|
}
|
||||||
|
|
||||||
func (m *LoadIndex) Reset() { *m = LoadIndex{} }
|
func (m *LoadIndex) Reset() { *m = LoadIndex{} }
|
||||||
@ -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,
|
||||||
}
|
}
|
||||||
|
|||||||
@ -16,4 +16,9 @@ message SegmentFlushMeta{
|
|||||||
uint64 open_time =3;
|
uint64 open_time =3;
|
||||||
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;
|
||||||
|
}
|
||||||
|
|||||||
@ -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,
|
||||||
}
|
}
|
||||||
|
|||||||
@ -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 {
|
||||||
|
|||||||
@ -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)
|
||||||
}
|
}
|
||||||
|
|||||||
@ -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"
|
||||||
@ -18,7 +20,8 @@ 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
|
||||||
}
|
}
|
||||||
|
|||||||
@ -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()
|
||||||
|
|||||||
@ -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,
|
||||||
|
|||||||
@ -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)
|
||||||
|
|||||||
@ -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
|
|
||||||
}
|
|
||||||
98
internal/querynode/load_index_info.go
Normal file
98
internal/querynode/load_index_info.go
Normal 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
|
||||||
|
}
|
||||||
286
internal/querynode/load_index_service.go
Normal file
286
internal/querynode/load_index_service.go
Normal 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
|
||||||
|
}
|
||||||
148
internal/querynode/load_index_service_test.go
Normal file
148
internal/querynode/load_index_service_test.go
Normal 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()
|
||||||
|
}
|
||||||
@ -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")
|
||||||
|
}
|
||||||
|
|||||||
@ -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)
|
||||||
|
|||||||
@ -24,10 +24,12 @@ type QueryNode struct {
|
|||||||
|
|
||||||
replica collectionReplica
|
replica collectionReplica
|
||||||
|
|
||||||
dataSyncService *dataSyncService
|
// services
|
||||||
metaService *metaService
|
dataSyncService *dataSyncService
|
||||||
searchService *searchService
|
metaService *metaService
|
||||||
statsService *statsService
|
searchService *searchService
|
||||||
|
loadIndexService *loadIndexService
|
||||||
|
statsService *statsService
|
||||||
}
|
}
|
||||||
|
|
||||||
func Init() {
|
func Init() {
|
||||||
@ -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()
|
||||||
|
|||||||
@ -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) {
|
||||||
|
|||||||
@ -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}
|
||||||
|
|||||||
@ -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
|
||||||
|
}
|
||||||
|
|||||||
@ -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}
|
||||||
|
|||||||
@ -12,17 +12,23 @@ 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{
|
||||||
|
|||||||
@ -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()
|
||||||
}
|
}
|
||||||
|
|||||||
@ -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
|
||||||
}
|
}
|
||||||
|
|||||||
@ -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)
|
||||||
|
|
||||||
|
|||||||
@ -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
|
||||||
|
|
||||||
|
|||||||
@ -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
|
|
||||||
}
|
|
||||||
@ -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
|
||||||
ddBuffer *ddBuffer
|
ddRecords *ddRecords
|
||||||
|
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),
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
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() *ddNode {
|
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,
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|||||||
126
internal/writenode/flow_graph_dd_node_test.go
Normal file
126
internal/writenode/flow_graph_dd_node_test.go
Normal 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})
|
||||||
|
}
|
||||||
@ -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
|
||||||
|
}
|
||||||
|
|||||||
@ -5,108 +5,121 @@ 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(101, tsOpen)
|
||||||
|
assert.NoError(t, err)
|
||||||
|
exp, err := meta.getFlushOpenTime(101)
|
||||||
|
assert.NoError(t, err)
|
||||||
|
assert.Equal(t, tsOpen, exp)
|
||||||
|
|
||||||
err := meta.AddSegmentFlush(segmentID)
|
tsOpen = Timestamp(200)
|
||||||
assert.Nil(t, err)
|
err = meta.addSegmentFlush(102, tsOpen)
|
||||||
|
assert.NoError(t, err)
|
||||||
|
exp, err = meta.getFlushOpenTime(102)
|
||||||
|
assert.NoError(t, err)
|
||||||
|
assert.Equal(t, tsOpen, exp)
|
||||||
|
|
||||||
tsOpen := Timestamp(1000)
|
tsOpen = Timestamp(200)
|
||||||
err = meta.SetFlushOpenTime(segmentID, tsOpen)
|
err = meta.addSegmentFlush(103, tsOpen)
|
||||||
assert.Nil(t, err)
|
assert.NoError(t, err)
|
||||||
|
exp, err = meta.getFlushOpenTime(103)
|
||||||
|
assert.NoError(t, err)
|
||||||
|
assert.Equal(t, tsOpen, exp)
|
||||||
|
|
||||||
exp, err := meta.getFlushOpenTime(segmentID)
|
err = meta.reloadSegMetaFromKV()
|
||||||
assert.Nil(t, err)
|
assert.NoError(t, err)
|
||||||
assert.Equal(t, tsOpen, exp)
|
})
|
||||||
|
|
||||||
tsClose := Timestamp(10001)
|
t.Run("TestMetaTable_AppendSegBinlogPaths", func(t *testing.T) {
|
||||||
err = meta.SetFlushCloseTime(segmentID, tsClose)
|
segmentID := UniqueID(201)
|
||||||
assert.Nil(t, err)
|
tsOpen := Timestamp(1000)
|
||||||
|
err := meta.addSegmentFlush(segmentID, tsOpen)
|
||||||
|
assert.Nil(t, err)
|
||||||
|
|
||||||
exp, err = meta.getFlushCloseTime(segmentID)
|
exp := map[int32][]string{
|
||||||
assert.Nil(t, err)
|
1: {"a", "b", "c"},
|
||||||
assert.Equal(t, tsClose, exp)
|
2: {"b", "a", "c"},
|
||||||
}
|
}
|
||||||
|
for fieldID, dataPaths := range exp {
|
||||||
func TestMetaTable_AppendBinlogPaths(t *testing.T) {
|
for _, dp := range dataPaths {
|
||||||
meta := createMetaTable(t)
|
err = meta.AppendSegBinlogPaths(tsOpen, segmentID, fieldID, []string{dp})
|
||||||
defer meta.client.Close()
|
assert.Nil(t, err)
|
||||||
var segmentID UniqueID = 1
|
err = meta.AppendSegBinlogPaths(tsOpen, segmentID, fieldID, []string{dp})
|
||||||
err := meta.AddSegmentFlush(segmentID)
|
assert.Nil(t, err)
|
||||||
assert.Nil(t, err)
|
}
|
||||||
|
|
||||||
exp := map[int32][]string{
|
|
||||||
1: {"a", "b", "c"},
|
|
||||||
2: {"b", "a", "c"},
|
|
||||||
}
|
|
||||||
for fieldID, dataPaths := range exp {
|
|
||||||
for _, dp := range dataPaths {
|
|
||||||
err = meta.AppendBinlogPaths(segmentID, fieldID, []string{dp})
|
|
||||||
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"}},
|
||||||
func TestMetaTable_CompleteFlush(t *testing.T) {
|
ret)
|
||||||
meta := createMetaTable(t)
|
})
|
||||||
defer meta.client.Close()
|
|
||||||
|
t.Run("TestMetaTable_AppendDDLBinlogPaths", func(t *testing.T) {
|
||||||
var segmentID UniqueID = 1
|
|
||||||
|
collID2Paths := map[UniqueID][]string{
|
||||||
err := meta.AddSegmentFlush(segmentID)
|
301: {"a", "b", "c"},
|
||||||
assert.Nil(t, err)
|
302: {"c", "b", "a"},
|
||||||
|
}
|
||||||
ret, err := meta.checkFlushComplete(segmentID)
|
|
||||||
assert.Nil(t, err)
|
for collID, dataPaths := range collID2Paths {
|
||||||
assert.Equal(t, false, ret)
|
for _, dp := range dataPaths {
|
||||||
|
err = meta.AppendDDLBinlogPaths(collID, []string{dp})
|
||||||
meta.CompleteFlush(segmentID)
|
assert.Nil(t, err)
|
||||||
|
}
|
||||||
ret, err = meta.checkFlushComplete(segmentID)
|
}
|
||||||
assert.Nil(t, err)
|
|
||||||
assert.Equal(t, true, ret)
|
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)
|
||||||
|
assert.NoError(t, err)
|
||||||
|
assert.Equal(t, false, ret)
|
||||||
|
|
||||||
|
meta.CompleteFlush(segmentID, closeTime)
|
||||||
|
|
||||||
|
ret, err = meta.checkFlushComplete(segmentID)
|
||||||
|
assert.NoError(t, err)
|
||||||
|
assert.Equal(t, true, ret)
|
||||||
|
ts, err := meta.getFlushCloseTime(segmentID)
|
||||||
|
assert.NoError(t, err)
|
||||||
|
assert.Equal(t, closeTime, ts)
|
||||||
|
})
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|||||||
@ -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()
|
||||||
|
|||||||
Loading…
x
Reference in New Issue
Block a user