diff --git a/build/docker/deploy/.env b/build/docker/deploy/.env index 4a3735af88..01c909d01f 100644 --- a/build/docker/deploy/.env +++ b/build/docker/deploy/.env @@ -5,4 +5,4 @@ TARGET_TAG=latest PULSAR_ADDRESS=pulsar://pulsar:6650 ETCD_ADDRESS=etcd:2379 MASTER_ADDRESS=master:53100 -MINIO_ADDRESS=minio:9000 +MINIO_ADDRESS=minio:9000 \ No newline at end of file diff --git a/configs/advanced/channel.yaml b/configs/advanced/channel.yaml index 135c2eb3e8..8116601e97 100644 --- a/configs/advanced/channel.yaml +++ b/configs/advanced/channel.yaml @@ -22,8 +22,6 @@ msgChannel: writeNodeTimeTick: "writeNodeTimeTick" # old name: statsChannels: "statistic" queryNodeStats: "query-node-stats" - # cmd for loadIndex, flush, etc... - cmd: "cmd" # sub name generation rule: ${subNamePrefix}-${NodeID} subNamePrefix: diff --git a/configs/advanced/query_node.yaml b/configs/advanced/query_node.yaml index ec5e6603b7..862da2cbdd 100644 --- a/configs/advanced/query_node.yaml +++ b/configs/advanced/query_node.yaml @@ -42,7 +42,3 @@ queryNode: stats: recvBufSize: 64 - - loadIndex: - recvBufSize: 512 - pulsarBufSize: 512 diff --git a/internal/allocator/allocator.go b/internal/allocator/allocator.go index 9cd7e935d7..fe4870fc3a 100644 --- a/internal/allocator/allocator.go +++ b/internal/allocator/allocator.go @@ -58,7 +58,6 @@ type segRequest struct { partition string segInfo map[UniqueID]uint32 channelID int32 - timestamp Timestamp } type syncRequest struct { @@ -122,18 +121,16 @@ type Allocator struct { masterClient masterpb.MasterClient countPerRPC uint32 - toDoReqs []request - canDoReqs []request - syncReqs []request + toDoReqs []request + + syncReqs []request tChan tickerChan forceSyncChan chan request - syncFunc func() bool + syncFunc func() processFunc func(req request) error - - checkSyncFunc func(timeout bool) bool - pickCanDoFunc func() + checkFunc func(timeout bool) bool } func (ta *Allocator) Start() error { @@ -148,6 +145,7 @@ func (ta *Allocator) Start() 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) defer cancel() conn, err := grpc.DialContext(ctx, ta.masterAddress, grpc.WithInsecure(), grpc.WithBlock()) @@ -184,13 +182,7 @@ func (ta *Allocator) mainLoop() { ta.finishSyncRequest() case <-ta.tChan.Chan(): - ta.pickCanDo() - ta.finishRequest() - if ta.sync(true) { - ta.pickCanDo() - ta.finishRequest() - } - ta.failRemainRequest() + ta.sync(true) case first := <-ta.reqs: ta.toDoReqs = append(ta.toDoReqs, first) @@ -198,13 +190,9 @@ func (ta *Allocator) mainLoop() { for i := 0; i < pending; i++ { ta.toDoReqs = append(ta.toDoReqs, <-ta.reqs) } - ta.pickCanDo() + ta.sync(false) + ta.finishRequest() - if ta.sync(false) { - ta.pickCanDo() - ta.finishRequest() - } - ta.failRemainRequest() case <-loopCtx.Done(): return @@ -213,32 +201,19 @@ func (ta *Allocator) mainLoop() { } } -func (ta *Allocator) pickCanDo() { - if ta.pickCanDoFunc == nil { +func (ta *Allocator) sync(timeout bool) { + if ta.syncFunc == nil { return } - ta.pickCanDoFunc() -} - -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 + if ta.checkFunc == nil || !ta.checkFunc(timeout) { + return } - ret := ta.syncFunc() + ta.syncFunc() if !timeout { ta.tChan.Reset() } - return ret } func (ta *Allocator) finishSyncRequest() { @@ -250,23 +225,14 @@ func (ta *Allocator) finishSyncRequest() { ta.syncReqs = ta.syncReqs[0:0] } -func (ta *Allocator) failRemainRequest() { - 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 { + for _, req := range ta.toDoReqs { if req != nil { err := ta.processFunc(req) req.Notify(err) } } - ta.canDoReqs = []request{} + ta.toDoReqs = ta.toDoReqs[0:0] } func (ta *Allocator) revokeRequest(err error) { diff --git a/internal/allocator/id.go b/internal/allocator/id.go index 65890d92c8..fc1d7c06b5 100644 --- a/internal/allocator/id.go +++ b/internal/allocator/id.go @@ -21,8 +21,6 @@ type IDAllocator struct { idStart UniqueID idEnd UniqueID - - PeerID UniqueID } func NewIDAllocator(ctx context.Context, masterAddr string) (*IDAllocator, error) { @@ -39,17 +37,16 @@ func NewIDAllocator(ctx context.Context, masterAddr string) (*IDAllocator, error a.tChan = &emptyTicker{} a.Allocator.syncFunc = a.syncID a.Allocator.processFunc = a.processFunc - a.Allocator.checkSyncFunc = a.checkSyncFunc - a.Allocator.pickCanDoFunc = a.pickCanDoFunc + a.Allocator.checkFunc = a.checkFunc a.init() return a, nil } -func (ia *IDAllocator) syncID() bool { +func (ia *IDAllocator) syncID() { fmt.Println("syncID") ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second) req := &internalpb.IDRequest{ - PeerID: ia.PeerID, + PeerID: 1, Role: internalpb.PeerRole_Proxy, Count: ia.countPerRPC, } @@ -58,32 +55,22 @@ func (ia *IDAllocator) syncID() bool { cancel() if err != nil { log.Println("syncID Failed!!!!!") - return false + return } ia.idStart = resp.GetID() ia.idEnd = ia.idStart + int64(resp.GetCount()) - return true } -func (ia *IDAllocator) checkSyncFunc(timeout bool) bool { - return timeout || len(ia.toDoReqs) > 0 -} - -func (ia *IDAllocator) pickCanDoFunc() { - total := uint32(ia.idEnd - ia.idStart) +func (ia *IDAllocator) checkFunc(timeout bool) bool { + if timeout { + return timeout + } need := uint32(0) - idx := 0 for _, req := range ia.toDoReqs { iReq := req.(*idRequest) need += iReq.count - if need <= total { - ia.canDoReqs = append(ia.canDoReqs, req) - idx++ - } else { - break - } } - ia.toDoReqs = ia.toDoReqs[idx:] + return ia.idStart+int64(need) >= ia.idEnd } func (ia *IDAllocator) processFunc(req request) error { diff --git a/internal/allocator/segment.go b/internal/allocator/segment.go index 7d1b535947..2209747b45 100644 --- a/internal/allocator/segment.go +++ b/internal/allocator/segment.go @@ -5,9 +5,10 @@ import ( "context" "fmt" "log" + "sort" "time" - "github.com/zilliztech/milvus-distributed/internal/proto/commonpb" + "github.com/cznic/mathutil" "github.com/zilliztech/milvus-distributed/internal/errors" "github.com/zilliztech/milvus-distributed/internal/proto/internalpb" @@ -15,101 +16,42 @@ import ( const ( SegCountPerRPC = 20000 - ActiveTimeDuration = 100 //second + ActiveTimeDuration = 100 // Second ) -type segInfo struct { - segID UniqueID - count uint32 - expireTime Timestamp -} - type assignInfo struct { collName string partitionTag string channelID int32 - segID UniqueID - segInfos *list.List - segCapacity uint32 + segInfo map[UniqueID]uint32 // segmentID->count map + expireTime time.Time lastInsertTime time.Time } -func (info *segInfo) IsExpired(ts Timestamp) bool { - 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) IsExpired(now time.Time) bool { + return now.Sub(info.expireTime) >= 0 } func (info *assignInfo) IsActive(now time.Time) bool { 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 { Allocator assignInfos map[string]*list.List // collectionName -> *list.List segReqs []*internalpb.SegIDRequest - getTickFunc func() Timestamp - PeerID UniqueID + canDoReqs []request } -func NewSegIDAssigner(ctx context.Context, masterAddr string, getTickFunc func() Timestamp) (*SegIDAssigner, error) { +func NewSegIDAssigner(ctx context.Context, masterAddr string) (*SegIDAssigner, error) { ctx1, cancel := context.WithCancel(ctx) sa := &SegIDAssigner{ Allocator: Allocator{reqs: make(chan request, maxConcurrentRequests), @@ -119,80 +61,72 @@ func NewSegIDAssigner(ctx context.Context, masterAddr string, getTickFunc func() countPerRPC: SegCountPerRPC, }, assignInfos: make(map[string]*list.List), - getTickFunc: getTickFunc, } sa.tChan = &ticker{ updateInterval: time.Second, } sa.Allocator.syncFunc = sa.syncSegments sa.Allocator.processFunc = sa.processFunc - sa.Allocator.checkSyncFunc = sa.checkSyncFunc - sa.Allocator.pickCanDoFunc = sa.pickCanDoFunc + sa.Allocator.checkFunc = sa.checkFunc return sa, nil } func (sa *SegIDAssigner) collectExpired() { - ts := sa.getTickFunc() - //now := time.Now() + now := time.Now() for _, info := range sa.assignInfos { for e := info.Front(); e != nil; e = e.Next() { assign := e.Value.(*assignInfo) - assign.RemoveExpired(ts) - 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) - //} + if !assign.IsActive(now) || !assign.IsExpired(now) { + continue } + sa.segReqs = append(sa.segReqs, &internalpb.SegIDRequest{ + ChannelID: assign.channelID, + Count: sa.countPerRPC, + CollName: assign.collName, + PartitionTag: assign.partitionTag, + }) } } } -func (sa *SegIDAssigner) pickCanDoFunc() { +func (sa *SegIDAssigner) checkToDoReqs() { if sa.toDoReqs == nil { return } - records := make(map[string]map[string]map[int32]uint32) - newTodoReqs := sa.toDoReqs[0:0] + now := time.Now() for _, req := range sa.toDoReqs { 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) - if assign == nil || assign.Capacity(segRequest.timestamp) < records[colName][partition][channelID] { + if assign == nil || assign.IsExpired(now) || !assign.IsEnough(segRequest.count) { sa.segReqs = append(sa.segReqs, &internalpb.SegIDRequest{ ChannelID: segRequest.channelID, Count: segRequest.count, CollName: segRequest.colName, 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 { @@ -211,109 +145,72 @@ func (sa *SegIDAssigner) getAssign(colName, partition string, channelID int32) * return nil } -func (sa *SegIDAssigner) checkSyncFunc(timeout bool) bool { - sa.collectExpired() - return timeout || len(sa.segReqs) != 0 -} - -func (sa *SegIDAssigner) checkSegReqEqual(req1, req2 *internalpb.SegIDRequest) bool { - if req1 == nil || req2 == nil { - return false +func (sa *SegIDAssigner) checkFunc(timeout bool) bool { + if timeout { + sa.collectExpired() + } else { + sa.checkToDoReqs() } - if req1 == req2 { - return true - } - return req1.CollName == req2.CollName && req1.PartitionTag == req2.PartitionTag && req1.ChannelID == req2.ChannelID + return len(sa.segReqs) != 0 } -func (sa *SegIDAssigner) reduceSegReqs() { - +func (sa *SegIDAssigner) syncSegments() { if len(sa.segReqs) == 0 { 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) defer cancel() req := &internalpb.AssignSegIDRequest{ - PeerID: sa.PeerID, + PeerID: 1, Role: internalpb.PeerRole_Proxy, PerChannelReq: sa.segReqs, } - sa.segReqs = []*internalpb.SegIDRequest{} + sa.segReqs = sa.segReqs[0:0] + fmt.Println("OOOOO", req.PerChannelReq) resp, err := sa.masterClient.AssignSegmentID(ctx, req) - if err != nil { - log.Println("GRPC AssignSegmentID Failed", resp, err) - return false - } + //if resp.Status.GetErrorCode() != commonpb.ErrorCode_SUCCESS { + // log.Println("GRPC AssignSegmentID Failed", resp, err) + // return + //} now := time.Now() - success := false + expiredTime := now.Add(time.Millisecond * time.Duration(1000)) + for _, info := range resp.PerChannelAssignment { + sa.removeSegInfo(info.CollName, info.PartitionTag, info.ChannelID) + } + 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) - segInfo := &segInfo{ - segID: info.SegID, - count: info.Count, - expireTime: info.ExpireTime, - } if assign == nil { colInfos, ok := sa.assignInfos[info.CollName] if !ok { colInfos = list.New() } - segInfos := list.New() - - segInfos.PushBack(segInfo) - assign = &assignInfo{ + segInfo := make(map[UniqueID]uint32) + segInfo[info.SegID] = info.Count + newAssign := &assignInfo{ collName: info.CollName, partitionTag: info.PartitionTag, channelID: info.ChannelID, - segInfos: segInfos, + segInfo: segInfo, } - colInfos.PushBack(assign) + colInfos.PushBack(newAssign) sa.assignInfos[info.CollName] = colInfos } else { - assign.segInfos.PushBack(segInfo) + assign.segInfo[info.SegID] = info.Count + 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 { @@ -322,19 +219,43 @@ func (sa *SegIDAssigner) processFunc(req request) error { if assign == nil { return errors.New("Failed to GetSegmentID") } - result, err := assign.Assign(segRequest.timestamp, segRequest.count) - segRequest.segInfo = result - return err + + keys := make([]UniqueID, len(assign.segInfo)) + i := 0 + 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, ts Timestamp) (map[UniqueID]uint32, error) { +func (sa *SegIDAssigner) GetSegmentID(colName, partition string, channelID int32, count uint32) (map[UniqueID]uint32, error) { req := &segRequest{ baseRequest: baseRequest{done: make(chan error), valid: false}, colName: colName, partition: partition, channelID: channelID, count: count, - timestamp: ts, } sa.reqs <- req req.Wait() diff --git a/internal/allocator/timestamp.go b/internal/allocator/timestamp.go index 8f69a0dfd4..035fc27ebc 100644 --- a/internal/allocator/timestamp.go +++ b/internal/allocator/timestamp.go @@ -19,7 +19,6 @@ type TimestampAllocator struct { Allocator lastTsBegin Timestamp lastTsEnd Timestamp - PeerID UniqueID } func NewTimestampAllocator(ctx context.Context, masterAddr string) (*TimestampAllocator, error) { @@ -37,36 +36,26 @@ func NewTimestampAllocator(ctx context.Context, masterAddr string) (*TimestampAl } a.Allocator.syncFunc = a.syncTs a.Allocator.processFunc = a.processFunc - a.Allocator.checkSyncFunc = a.checkSyncFunc - a.Allocator.pickCanDoFunc = a.pickCanDoFunc + a.Allocator.checkFunc = a.checkFunc return a, nil } -func (ta *TimestampAllocator) checkSyncFunc(timeout bool) bool { - return timeout || len(ta.toDoReqs) > 0 -} - -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 - } +func (ta *TimestampAllocator) checkFunc(timeout bool) bool { + if timeout { + return true } - ta.toDoReqs = ta.toDoReqs[idx:] + 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() bool { +func (ta *TimestampAllocator) syncTs() { ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second) req := &internalpb.TsoRequest{ - PeerID: ta.PeerID, + PeerID: 1, Role: internalpb.PeerRole_Proxy, Count: ta.countPerRPC, } @@ -75,11 +64,10 @@ func (ta *TimestampAllocator) syncTs() bool { cancel() if err != nil { log.Println("syncTimestamp Failed!!!!!") - return false + return } ta.lastTsBegin = resp.GetTimestamp() ta.lastTsEnd = ta.lastTsBegin + uint64(resp.GetCount()) - return true } func (ta *TimestampAllocator) processFunc(req request) error { diff --git a/internal/core/src/common/LoadIndex.h b/internal/core/src/common/LoadIndex.h deleted file mode 100644 index 377b533082..0000000000 --- a/internal/core/src/common/LoadIndex.h +++ /dev/null @@ -1,23 +0,0 @@ -// 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 -#include - -#include "../index/knowhere/knowhere/index/vector_index/VecIndex.h" - -struct LoadIndexInfo { - std::string field_name; - int64_t field_id; - std::map index_params; - milvus::knowhere::VecIndexPtr index; -}; diff --git a/internal/core/src/segcore/CMakeLists.txt b/internal/core/src/segcore/CMakeLists.txt index 02eb732ad7..2dfcf78e79 100644 --- a/internal/core/src/segcore/CMakeLists.txt +++ b/internal/core/src/segcore/CMakeLists.txt @@ -11,8 +11,7 @@ set(SEGCORE_FILES InsertRecord.cpp Reduce.cpp plan_c.cpp - reduce_c.cpp - load_index_c.cpp) + reduce_c.cpp) add_library(milvus_segcore SHARED ${SEGCORE_FILES} ) diff --git a/internal/core/src/segcore/load_index_c.cpp b/internal/core/src/segcore/load_index_c.cpp deleted file mode 100644 index 01c9789e5d..0000000000 --- a/internal/core/src/segcore/load_index_c.cpp +++ /dev/null @@ -1,139 +0,0 @@ -// 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(); - *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(); - *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 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; - } -} diff --git a/internal/core/src/segcore/load_index_c.h b/internal/core/src/segcore/load_index_c.h deleted file mode 100644 index 88985a1e81..0000000000 --- a/internal/core/src/segcore/load_index_c.h +++ /dev/null @@ -1,45 +0,0 @@ -// 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 -#include -#include - -#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 diff --git a/internal/core/src/segcore/segment_c.cpp b/internal/core/src/segcore/segment_c.cpp index a9e2d5d521..e151f3b68b 100644 --- a/internal/core/src/segcore/segment_c.cpp +++ b/internal/core/src/segcore/segment_c.cpp @@ -19,7 +19,6 @@ #include #include #include -#include "common/LoadIndex.h" CSegmentBase NewSegment(CCollection collection, uint64_t segment_id) { @@ -174,22 +173,6 @@ FillTargetEntry(CSegmentBase c_segment, CPlan c_plan, CQueryResult c_result) { 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 diff --git a/internal/core/src/segcore/segment_c.h b/internal/core/src/segcore/segment_c.h index 6bec566dd3..0dc3f7cdcd 100644 --- a/internal/core/src/segcore/segment_c.h +++ b/internal/core/src/segcore/segment_c.h @@ -18,7 +18,6 @@ extern "C" { #include #include "segcore/plan_c.h" -#include "segcore/load_index_c.h" typedef void* CSegmentBase; typedef void* CQueryResult; @@ -65,8 +64,6 @@ Search(CSegmentBase c_segment, CStatus FillTargetEntry(CSegmentBase c_segment, CPlan c_plan, CQueryResult result); -CStatus -UpdateSegmentIndex(CSegmentBase c_segment, CLoadIndexInfo c_load_index_info); ////////////////////////////////////////////////////////////////// int diff --git a/internal/core/unittest/test_c_api.cpp b/internal/core/unittest/test_c_api.cpp index 2aaff331d8..63f510aa61 100644 --- a/internal/core/unittest/test_c_api.cpp +++ b/internal/core/unittest/test_c_api.cpp @@ -13,17 +13,11 @@ #include #include #include -#include #include "pb/service_msg.pb.h" #include "segcore/reduce_c.h" -#include -#include -#include -#include -#include - +#include namespace chrono = std::chrono; TEST(CApiTest, CollectionTest) { @@ -344,7 +338,7 @@ TEST(CApiTest, GetMemoryUsageInBytesTest) { namespace { auto generate_data(int N) { - std::vector raw_data; + std::vector raw_data; std::vector timestamps; std::vector uids; std::default_random_engine er(42); @@ -358,7 +352,7 @@ generate_data(int N) { for (auto& x : vec) { x = distribution(er); } - raw_data.insert(raw_data.end(), std::begin(vec), std::end(vec)); + raw_data.insert(raw_data.end(), (const char*)std::begin(vec), (const char*)std::end(vec)); int age = ei() % 100; raw_data.insert(raw_data.end(), (const char*)&age, ((const char*)&age) + sizeof(age)); } @@ -684,53 +678,3 @@ TEST(CApiTest, Reduce) { DeleteCollection(collection); 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(); - 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(milvus::knowhere::meta::IDS); - auto dis = result->Get(milvus::knowhere::meta::DISTANCE); - for (int i = 0; i < std::min(num_query * K, 100); ++i) { - std::cout << ids[i] << "->" << dis[i] << std::endl; - } -} \ No newline at end of file diff --git a/internal/master/global_allocator_test.go b/internal/master/global_allocator_test.go index 35abae57ec..31580a425d 100644 --- a/internal/master/global_allocator_test.go +++ b/internal/master/global_allocator_test.go @@ -11,62 +11,69 @@ import ( var gTestTsoAllocator Allocator var gTestIDAllocator *GlobalIDAllocator -func TestGlobalTSOAllocator_Initialize(t *testing.T) { - err := gTestTsoAllocator.Initialize() - assert.Nil(t, err) -} +func TestGlobalTSOAllocator_All(t *testing.T) { + Init() + gTestTsoAllocator = NewGlobalTSOAllocator("timestamp", tsoutil.NewTSOKVBase([]string{Params.EtcdAddress}, "/test/root/kv", "tso")) + gTestIDAllocator = NewGlobalIDAllocator("idTimestamp", tsoutil.NewTSOKVBase([]string{Params.EtcdAddress}, "/test/root/kv", "gid")) -func TestGlobalTSOAllocator_GenerateTSO(t *testing.T) { - count := 1000 - perCount := uint32(100) - startTs, err := gTestTsoAllocator.GenerateTSO(perCount) - assert.Nil(t, err) - lastPhysical, lastLogical := tsoutil.ParseTS(startTs) - for i := 0; i < count; i++ { - ts, _ := gTestTsoAllocator.GenerateTSO(perCount) - physical, logical := tsoutil.ParseTS(ts) - if lastPhysical.Equal(physical) { - diff := logical - lastLogical - assert.Equal(t, uint64(perCount), diff) + t.Run("Initialize", func(t *testing.T) { + err := gTestTsoAllocator.Initialize() + assert.Nil(t, err) + }) + + t.Run("GenerateTSO", func(t *testing.T) { + count := 1000 + perCount := uint32(100) + startTs, err := gTestTsoAllocator.GenerateTSO(perCount) + assert.Nil(t, err) + lastPhysical, lastLogical := tsoutil.ParseTS(startTs) + for i := 0; i < count; i++ { + ts, _ := gTestTsoAllocator.GenerateTSO(perCount) + physical, logical := tsoutil.ParseTS(ts) + if lastPhysical.Equal(physical) { + diff := logical - lastLogical + assert.Equal(t, uint64(perCount), diff) + } + lastPhysical, lastLogical = physical, logical } - lastPhysical, lastLogical = physical, logical - } -} + }) -func TestGlobalTSOAllocator_SetTSO(t *testing.T) { - curTime := time.Now() - nextTime := curTime.Add(2 * time.Second) - physical := nextTime.UnixNano() / int64(time.Millisecond) - logical := int64(0) - err := gTestTsoAllocator.SetTSO(tsoutil.ComposeTS(physical, logical)) - assert.Nil(t, err) -} + t.Run("SetTSO", func(t *testing.T) { + curTime := time.Now() + nextTime := curTime.Add(2 * time.Second) + physical := nextTime.UnixNano() / int64(time.Millisecond) + logical := int64(0) + err := gTestTsoAllocator.SetTSO(tsoutil.ComposeTS(physical, logical)) + assert.Nil(t, err) + }) -func TestGlobalTSOAllocator_UpdateTSO(t *testing.T) { - err := gTestTsoAllocator.UpdateTSO() - assert.Nil(t, err) -} + t.Run("UpdateTSO", func(t *testing.T) { + err := gTestTsoAllocator.UpdateTSO() + assert.Nil(t, err) + }) -func TestGlobalTSOAllocator_Reset(t *testing.T) { - gTestTsoAllocator.Reset() -} + t.Run("Reset", func(t *testing.T) { + gTestTsoAllocator.Reset() + }) -func TestGlobalIdAllocator_Initialize(t *testing.T) { - err := gTestIDAllocator.Initialize() - assert.Nil(t, err) -} + t.Run("Initialize", func(t *testing.T) { + err := gTestIDAllocator.Initialize() + assert.Nil(t, err) + }) -func TestGlobalIdAllocator_AllocOne(t *testing.T) { - one, err := gTestIDAllocator.AllocOne() - assert.Nil(t, err) - ano, err := gTestIDAllocator.AllocOne() - assert.Nil(t, err) - assert.NotEqual(t, one, ano) -} + t.Run("AllocOne", func(t *testing.T) { + one, err := gTestIDAllocator.AllocOne() + assert.Nil(t, err) + ano, err := gTestIDAllocator.AllocOne() + assert.Nil(t, err) + assert.NotEqual(t, one, ano) + }) + + t.Run("Alloc", func(t *testing.T) { + count := uint32(2 << 10) + idStart, idEnd, err := gTestIDAllocator.Alloc(count) + assert.Nil(t, err) + assert.Equal(t, count, uint32(idEnd-idStart)) + }) -func TestGlobalIdAllocator_Alloc(t *testing.T) { - count := uint32(2 << 10) - idStart, idEnd, err := gTestIDAllocator.Alloc(count) - assert.Nil(t, err) - assert.Equal(t, count, uint32(idEnd-idStart)) } diff --git a/internal/master/master.go b/internal/master/master.go index 3313eaf5e5..f5eb1da71e 100644 --- a/internal/master/master.go +++ b/internal/master/master.go @@ -150,7 +150,7 @@ func CreateServer(ctx context.Context) (*Master, error) { // stats msg stream statsMs := ms.NewPulsarMsgStream(ctx, 1024) statsMs.SetPulsarClient(pulsarAddr) - statsMs.CreatePulsarConsumers([]string{Params.QueryNodeStatsChannelName}, Params.MsgChannelSubName, ms.NewUnmarshalDispatcher(), 1024) + statsMs.CreatePulsarConsumers([]string{Params.QueryNodeStatsChannelName}, "SegmentStats", ms.NewUnmarshalDispatcher(), 1024) statsMs.Start() m := &Master{ diff --git a/internal/master/master_test.go b/internal/master/master_test.go index c163a6849f..6dac702a77 100644 --- a/internal/master/master_test.go +++ b/internal/master/master_test.go @@ -2,10 +2,8 @@ package master import ( "context" - "fmt" "log" "math/rand" - "os" "strconv" "strings" "testing" @@ -23,6 +21,7 @@ import ( "github.com/zilliztech/milvus-distributed/internal/proto/masterpb" "github.com/zilliztech/milvus-distributed/internal/proto/schemapb" "github.com/zilliztech/milvus-distributed/internal/proto/servicepb" + "go.etcd.io/etcd/clientv3" "go.uber.org/zap" "google.golang.org/grpc" ) @@ -61,6 +60,7 @@ func refreshChannelNames() { Params.InsertChannelNames = makeNewChannalNames(Params.InsertChannelNames, suffix) Params.K2SChannelNames = makeNewChannalNames(Params.K2SChannelNames, suffix) Params.ProxyTimeTickChannelNames = makeNewChannalNames(Params.ProxyTimeTickChannelNames, suffix) + Params.MetaRootPath = "/test" + strconv.FormatInt(rand.Int63n(100), 10) + "/root/kv" } func receiveTimeTickMsg(stream *ms.MsgStream) bool { @@ -80,20 +80,18 @@ func getTimeTickMsgPack(ttmsgs [][2]uint64) *ms.MsgPack { return &msgPack } -func TestMain(m *testing.M) { +func TestMaster(t *testing.T) { Init() refreshMasterAddress() refreshChannelNames() etcdAddr := Params.EtcdAddress - gTestTsoAllocator = NewGlobalTSOAllocator("timestamp", tsoutil.NewTSOKVBase([]string{etcdAddr}, "/test/root/kv", "tso")) - gTestIDAllocator = NewGlobalIDAllocator("idTimestamp", tsoutil.NewTSOKVBase([]string{etcdAddr}, "/test/root/kv", "gid")) - exitCode := m.Run() - os.Exit(exitCode) -} + etcdCli, err := clientv3.New(clientv3.Config{Endpoints: []string{etcdAddr}}) + assert.Nil(t, err) + _, err = etcdCli.Delete(context.Background(), Params.MetaRootPath, clientv3.WithPrefix()) + assert.Nil(t, err) -func TestMaster(t *testing.T) { - Init() - refreshMasterAddress() + gTestTsoAllocator = NewGlobalTSOAllocator("timestamp", tsoutil.NewTSOKVBase([]string{etcdAddr}, Params.MetaRootPath, "tso")) + gTestIDAllocator = NewGlobalIDAllocator("idTimestamp", tsoutil.NewTSOKVBase([]string{etcdAddr}, Params.MetaRootPath, "gid")) pulsarAddr := Params.PulsarAddress Params.ProxyIDList = []UniqueID{0} //Param @@ -198,7 +196,6 @@ func TestMaster(t *testing.T) { }) t.Run("TestCollectionTask", func(t *testing.T) { - fmt.Println("point 3") sch := schemapb.CollectionSchema{ Name: "col1", Description: "test collection", diff --git a/internal/master/meta_table.go b/internal/master/meta_table.go index 89e8267968..216583a005 100644 --- a/internal/master/meta_table.go +++ b/internal/master/meta_table.go @@ -377,7 +377,7 @@ func (mt *metaTable) DeletePartition(collID UniqueID, tag string) error { for _, s := range collMeta.SegmentIDs { sm, ok := mt.segID2Meta[s] if !ok { - return errors.Errorf("DeletePartition:can't find segment id = %d", s) + return errors.Errorf("can't find segment id = %d", s) } if sm.PartitionTag != tag { seg = append(seg, s) @@ -444,7 +444,7 @@ func (mt *metaTable) GetSegmentByID(segID UniqueID) (*pb.SegmentMeta, error) { sm, ok := mt.segID2Meta[segID] if !ok { - return nil, errors.Errorf("GetSegmentByID:can't find segment id = %d", segID) + return nil, errors.Errorf("can't find segment id = %d", segID) } return &sm, nil } @@ -455,7 +455,7 @@ func (mt *metaTable) DeleteSegment(segID UniqueID) error { segMeta, ok := mt.segID2Meta[segID] if !ok { - return errors.Errorf("DeleteSegment:can't find segment. id = " + strconv.FormatInt(segID, 10)) + return errors.Errorf("can't find segment. id = " + strconv.FormatInt(segID, 10)) } collMeta, ok := mt.collID2Meta[segMeta.CollectionID] @@ -483,7 +483,7 @@ func (mt *metaTable) CloseSegment(segID UniqueID, closeTs Timestamp) error { segMeta, ok := mt.segID2Meta[segID] if !ok { - return errors.Errorf("CloseSegment:can't find segment id = " + strconv.FormatInt(segID, 10)) + return errors.Errorf("can't find segment id = " + strconv.FormatInt(segID, 10)) } segMeta.CloseTime = closeTs diff --git a/internal/master/segment_assigner.go b/internal/master/segment_assigner.go index 3585392152..6c9fd4c907 100644 --- a/internal/master/segment_assigner.go +++ b/internal/master/segment_assigner.go @@ -98,7 +98,7 @@ func (assigner *SegmentAssigner) Assign(segmentID UniqueID, numRows int) (*Assig return res, err } physicalTs, logicalTs := tsoutil.ParseTS(ts) - expirePhysicalTs := physicalTs.Add(time.Duration(assigner.segmentExpireDuration) * time.Millisecond) + expirePhysicalTs := physicalTs.Add(time.Duration(assigner.segmentExpireDuration)) expireTs := tsoutil.ComposeTS(expirePhysicalTs.UnixNano()/int64(time.Millisecond), int64(logicalTs)) status.lastExpireTime = expireTs status.assignments = append(status.assignments, &Assignment{ diff --git a/internal/msgstream/unmarshal.go b/internal/msgstream/unmarshal.go index 3c516e84ec..967b5d652a 100644 --- a/internal/msgstream/unmarshal.go +++ b/internal/msgstream/unmarshal.go @@ -34,7 +34,6 @@ func (dispatcher *UnmarshalDispatcher) addDefaultMsgTemplates() { dropCollectionMsg := DropCollectionMsg{} createPartitionMsg := CreatePartitionMsg{} dropPartitionMsg := DropPartitionMsg{} - loadIndexMsg := LoadIndexMsg{} queryNodeSegStatsMsg := QueryNodeStatsMsg{} dispatcher.tempMap = make(map[internalPb.MsgType]UnmarshalFunc) @@ -48,7 +47,6 @@ func (dispatcher *UnmarshalDispatcher) addDefaultMsgTemplates() { dispatcher.tempMap[internalPb.MsgType_kDropCollection] = dropCollectionMsg.Unmarshal dispatcher.tempMap[internalPb.MsgType_kCreatePartition] = createPartitionMsg.Unmarshal dispatcher.tempMap[internalPb.MsgType_kDropPartition] = dropPartitionMsg.Unmarshal - dispatcher.tempMap[internalPb.MsgType_kLoadIndex] = loadIndexMsg.Unmarshal } diff --git a/internal/proto/internal_msg.proto b/internal/proto/internal_msg.proto index b9961e309a..b5da1f0459 100644 --- a/internal/proto/internal_msg.proto +++ b/internal/proto/internal_msg.proto @@ -291,10 +291,8 @@ message Key2SegMsg { message LoadIndex { MsgType msg_type = 1; int64 segmentID = 2; - string fieldName = 3; - int64 fieldID = 4; - repeated string index_paths = 5; - repeated common.KeyValuePair index_params = 6; + int64 fieldID = 3; + repeated string index_paths = 4; } message IndexStats { diff --git a/internal/proto/internalpb/internal_msg.pb.go b/internal/proto/internalpb/internal_msg.pb.go index 4432e64492..50c09702ff 100644 --- a/internal/proto/internalpb/internal_msg.pb.go +++ b/internal/proto/internalpb/internal_msg.pb.go @@ -2043,15 +2043,13 @@ func (m *Key2SegMsg) GetKey2Seg() []*Key2Seg { } type LoadIndex struct { - 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"` - FieldName string `protobuf:"bytes,3,opt,name=fieldName,proto3" json:"fieldName,omitempty"` - FieldID int64 `protobuf:"varint,4,opt,name=fieldID,proto3" json:"fieldID,omitempty"` - IndexPaths []string `protobuf:"bytes,5,rep,name=index_paths,json=indexPaths,proto3" json:"index_paths,omitempty"` - IndexParams []*commonpb.KeyValuePair `protobuf:"bytes,6,rep,name=index_params,json=indexParams,proto3" json:"index_params,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` + 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"` + FieldID int64 `protobuf:"varint,3,opt,name=fieldID,proto3" json:"fieldID,omitempty"` + IndexPaths []string `protobuf:"bytes,4,rep,name=index_paths,json=indexPaths,proto3" json:"index_paths,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` } func (m *LoadIndex) Reset() { *m = LoadIndex{} } @@ -2093,13 +2091,6 @@ func (m *LoadIndex) GetSegmentID() int64 { return 0 } -func (m *LoadIndex) GetFieldName() string { - if m != nil { - return m.FieldName - } - return "" -} - func (m *LoadIndex) GetFieldID() int64 { if m != nil { return m.FieldID @@ -2114,13 +2105,6 @@ func (m *LoadIndex) GetIndexPaths() []string { return nil } -func (m *LoadIndex) GetIndexParams() []*commonpb.KeyValuePair { - if m != nil { - return m.IndexParams - } - return nil -} - type IndexStats struct { IndexParams []*commonpb.KeyValuePair `protobuf:"bytes,1,rep,name=index_params,json=indexParams,proto3" json:"index_params,omitempty"` NumRelatedSegments int64 `protobuf:"varint,2,opt,name=num_related_segments,json=numRelatedSegments,proto3" json:"num_related_segments,omitempty"` @@ -2653,120 +2637,120 @@ func init() { func init() { proto.RegisterFile("internal_msg.proto", fileDescriptor_7eb37f6b80b23116) } var fileDescriptor_7eb37f6b80b23116 = []byte{ - // 1837 bytes of a gzipped FileDescriptorProto + // 1831 bytes of a gzipped FileDescriptorProto 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, 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, - 0x1c, 0x90, 0x10, 0x73, 0x46, 0x7c, 0x88, 0x03, 0x37, 0xee, 0xb0, 0x23, 0x16, 0xc4, 0xff, 0xc0, - 0xe7, 0x85, 0xff, 0x02, 0x0e, 0x20, 0xb1, 0xcb, 0x81, 0x1b, 0xaa, 0xaa, 0xfe, 0x70, 0x27, 0xb6, - 0x13, 0x6d, 0x32, 0xcb, 0xa0, 0x9d, 0x5b, 0xbd, 0xe7, 0xea, 0xaa, 0xf7, 0x7e, 0xef, 0xa3, 0xde, - 0x7b, 0x06, 0x42, 0xc3, 0x18, 0x59, 0xe8, 0xfa, 0x4e, 0xc0, 0xfb, 0x1b, 0x43, 0x16, 0xc5, 0x11, - 0x59, 0x0d, 0xa8, 0xff, 0x78, 0xc4, 0x15, 0xb5, 0x91, 0x6e, 0x78, 0xb5, 0xd1, 0x8d, 0x82, 0x20, - 0x0a, 0x15, 0xfb, 0xd5, 0x1b, 0x1c, 0xd9, 0x63, 0xda, 0xc5, 0xfc, 0x3b, 0x2b, 0x84, 0x5a, 0xa7, - 0x6d, 0xe3, 0xf1, 0x08, 0x79, 0x4c, 0x6e, 0x42, 0x65, 0x88, 0xc8, 0x3a, 0xed, 0x96, 0xb6, 0xa6, - 0xad, 0x1b, 0x76, 0x42, 0x91, 0x07, 0x50, 0x62, 0x91, 0x8f, 0x2d, 0x7d, 0x4d, 0x5b, 0x6f, 0x6e, - 0xdd, 0xdd, 0x98, 0x7a, 0xd7, 0xc6, 0x1e, 0x22, 0xb3, 0x23, 0x1f, 0x6d, 0xb9, 0x99, 0xac, 0x40, - 0xb9, 0x1b, 0x8d, 0xc2, 0xb8, 0x65, 0xac, 0x69, 0xeb, 0x8b, 0xb6, 0x22, 0xac, 0x3e, 0x80, 0xb8, - 0x8f, 0x0f, 0xa3, 0x90, 0x23, 0x79, 0x00, 0x15, 0x1e, 0xbb, 0xf1, 0x88, 0xcb, 0x0b, 0xeb, 0x5b, - 0x77, 0x8a, 0x47, 0x27, 0xc2, 0xef, 0xcb, 0x2d, 0x76, 0xb2, 0x95, 0x34, 0x41, 0xef, 0xb4, 0xa5, - 0x2c, 0x86, 0xad, 0x77, 0xda, 0x33, 0x2e, 0x8a, 0x00, 0x0e, 0x78, 0xf4, 0x09, 0x6a, 0xf6, 0x18, - 0xea, 0xf2, 0xc2, 0xab, 0xa8, 0xf6, 0x1a, 0xd4, 0x62, 0x1a, 0x20, 0x8f, 0xdd, 0x60, 0x28, 0x65, - 0x2a, 0xd9, 0x39, 0x63, 0xc6, 0xbd, 0x4f, 0x35, 0x68, 0xec, 0x63, 0x3f, 0xb7, 0x62, 0xb6, 0x4d, - 0x9b, 0xd8, 0x26, 0x8e, 0xee, 0x0e, 0xdc, 0x30, 0x44, 0x3f, 0x01, 0xaf, 0x6c, 0xe7, 0x0c, 0x72, - 0x07, 0x6a, 0xdd, 0xc8, 0xf7, 0x9d, 0xd0, 0x0d, 0x50, 0x1e, 0x5f, 0xb3, 0x17, 0x04, 0xe3, 0x91, - 0x1b, 0x20, 0xb9, 0x0f, 0x8b, 0x43, 0x97, 0xc5, 0x34, 0xa6, 0x51, 0xe8, 0xc4, 0x6e, 0xbf, 0x55, - 0x92, 0x1b, 0x1a, 0x19, 0xf3, 0xc0, 0xed, 0x5b, 0xcf, 0x34, 0x20, 0x5f, 0xe7, 0x9c, 0xf6, 0xc3, - 0x82, 0x30, 0xd7, 0x0a, 0xfc, 0x43, 0x58, 0x1a, 0x22, 0x73, 0x12, 0xb1, 0x1d, 0x86, 0xc7, 0x2d, - 0x63, 0xcd, 0x58, 0xaf, 0x6f, 0xdd, 0x9f, 0xf1, 0xfd, 0xa4, 0x28, 0xf6, 0xe2, 0x10, 0xd9, 0x8e, - 0xfa, 0xd4, 0xc6, 0x63, 0xeb, 0x43, 0x0d, 0x96, 0xe4, 0xef, 0x4a, 0xea, 0x00, 0x43, 0x09, 0x1d, - 0x17, 0xac, 0x44, 0x58, 0x45, 0x5c, 0x00, 0xdd, 0x54, 0xab, 0x14, 0x01, 0x2d, 0x5d, 0x04, 0x68, - 0xf9, 0x3c, 0xa0, 0xe4, 0x2e, 0xd4, 0xf1, 0x74, 0x48, 0x19, 0x3a, 0xc2, 0x03, 0x5a, 0x15, 0xe9, - 0x0d, 0xa0, 0x58, 0x07, 0x34, 0x98, 0xf4, 0xb0, 0xea, 0xa5, 0x3d, 0xcc, 0xe2, 0xb0, 0x5c, 0xb0, - 0x52, 0xe2, 0xad, 0xef, 0xc1, 0xcd, 0x49, 0x64, 0xdd, 0x0c, 0x92, 0x96, 0x26, 0x01, 0xfe, 0xdc, - 0x3c, 0x80, 0x73, 0x00, 0xed, 0x95, 0x1c, 0xe3, 0x9c, 0x6b, 0xfd, 0x47, 0x83, 0x5b, 0x3b, 0x0c, - 0xdd, 0x18, 0x77, 0x22, 0xdf, 0xc7, 0xae, 0x50, 0x31, 0x75, 0x90, 0xb7, 0x60, 0x21, 0xe0, 0x7d, - 0x27, 0x1e, 0x0f, 0x51, 0xa2, 0xde, 0xdc, 0x7a, 0x7d, 0xc6, 0x5d, 0xbb, 0xbc, 0x7f, 0x30, 0x1e, - 0xa2, 0x5d, 0x0d, 0xd4, 0x82, 0x58, 0xd0, 0xe8, 0x66, 0xe7, 0x65, 0x29, 0xa1, 0xc0, 0x13, 0xd6, - 0x61, 0x78, 0xdc, 0x69, 0x4b, 0xeb, 0x18, 0xb6, 0x22, 0x8a, 0x71, 0x56, 0x3a, 0x1b, 0x67, 0x2d, - 0xa8, 0x0e, 0x59, 0x74, 0x3a, 0xee, 0xb4, 0xa5, 0x61, 0x0c, 0x3b, 0x25, 0xc9, 0x97, 0xa1, 0xc2, - 0xbb, 0x03, 0x0c, 0x5c, 0x69, 0x8e, 0xfa, 0xd6, 0xed, 0xa9, 0x90, 0x6f, 0xfb, 0xd1, 0xa1, 0x9d, - 0x6c, 0xb4, 0x7e, 0xaa, 0xc3, 0x6a, 0x9b, 0x45, 0xc3, 0xff, 0x73, 0xcd, 0x77, 0x61, 0x29, 0x3f, - 0x5d, 0x79, 0xb5, 0x82, 0xe0, 0xb3, 0x45, 0x99, 0x93, 0x17, 0x66, 0x23, 0x57, 0x57, 0x78, 0xbc, - 0xdd, 0xec, 0x16, 0x68, 0xeb, 0x1f, 0x1a, 0xac, 0xbc, 0xe3, 0xf2, 0x6b, 0x05, 0x25, 0x53, 0x58, - 0x9f, 0xa9, 0xb0, 0x31, 0x47, 0xe1, 0xd2, 0x85, 0x0a, 0x97, 0xaf, 0xa0, 0xf0, 0x87, 0x1a, 0xdc, - 0x6e, 0x23, 0xef, 0x32, 0x7a, 0x88, 0x9f, 0x1e, 0xad, 0x7f, 0xa1, 0xc1, 0xea, 0xfe, 0x20, 0x3a, - 0x79, 0x71, 0x35, 0xb6, 0x7e, 0xab, 0xc3, 0x4d, 0x95, 0x9b, 0xf6, 0xd2, 0xec, 0xfb, 0x09, 0x05, - 0xe8, 0x1a, 0xd4, 0xb3, 0x84, 0x9f, 0x85, 0xe9, 0x24, 0x2b, 0xd7, 0xb4, 0x34, 0x53, 0xd3, 0xf2, - 0x1c, 0x4d, 0x2b, 0x45, 0xdb, 0x7e, 0x13, 0x9a, 0xf9, 0xab, 0x23, 0x4d, 0xab, 0xde, 0x8d, 0xfb, - 0xd3, 0x4d, 0x9b, 0xc1, 0x21, 0x2d, 0x9b, 0x3f, 0x58, 0xd2, 0xb0, 0x1f, 0xe8, 0xb0, 0x22, 0xb2, - 0xda, 0x4b, 0xcc, 0x2e, 0x8f, 0xd9, 0xdf, 0x35, 0x58, 0x7e, 0xc7, 0xe5, 0xd7, 0x09, 0xd9, 0xf5, - 0x06, 0xff, 0x79, 0x65, 0xcb, 0x1f, 0x5b, 0xd9, 0x7f, 0x6a, 0xd0, 0x4a, 0xf3, 0xdd, 0xa7, 0x43, - 0x63, 0xf1, 0xa4, 0x89, 0x5c, 0xf7, 0xe2, 0x6a, 0x7b, 0xcd, 0xc9, 0xfd, 0x5f, 0x3a, 0x2c, 0x76, - 0x42, 0x8e, 0x2c, 0x7e, 0x6e, 0x9a, 0xbe, 0x71, 0x5e, 0x62, 0xd5, 0x9c, 0x9c, 0x91, 0xe5, 0x52, - 0x2d, 0x8a, 0xc0, 0x8d, 0x63, 0x5f, 0x54, 0xa4, 0x59, 0x7d, 0x93, 0x33, 0x8a, 0x55, 0xbe, 0x4a, - 0x03, 0x13, 0x55, 0xfe, 0x04, 0xaa, 0xd5, 0x22, 0xaa, 0xaf, 0x03, 0x64, 0xe0, 0xf3, 0xd6, 0xc2, - 0x9a, 0x21, 0xca, 0xf4, 0x9c, 0x23, 0x3a, 0x20, 0x16, 0x9d, 0x74, 0xda, 0xbc, 0x55, 0x5b, 0x33, - 0x44, 0x07, 0xa4, 0x28, 0xf2, 0x15, 0x58, 0x60, 0xd1, 0x89, 0xd3, 0x73, 0x63, 0xb7, 0x05, 0xb2, - 0xc8, 0x9e, 0x53, 0x4d, 0x56, 0x59, 0x74, 0xd2, 0x76, 0x63, 0xd7, 0x7a, 0xaa, 0xc3, 0x62, 0x1b, - 0x7d, 0x8c, 0xf1, 0x7f, 0x0f, 0x7a, 0x01, 0xb1, 0xd2, 0x1c, 0xc4, 0xca, 0xf3, 0x10, 0xab, 0x9c, - 0x43, 0xec, 0x1e, 0x34, 0x86, 0x8c, 0x06, 0x2e, 0x1b, 0x3b, 0x1e, 0x8e, 0x45, 0x7b, 0x63, 0xc8, - 0x2c, 0xaf, 0x78, 0x0f, 0x71, 0xcc, 0xad, 0x8f, 0x34, 0x58, 0xdc, 0x47, 0x97, 0x75, 0x07, 0xcf, - 0x0d, 0x86, 0x09, 0xf9, 0x8d, 0xa2, 0xfc, 0xf3, 0x6b, 0xe8, 0xcf, 0x83, 0xc9, 0x90, 0x8f, 0xfc, - 0xd8, 0xc9, 0xc1, 0x51, 0x00, 0x2c, 0x29, 0xfe, 0x4e, 0x06, 0xd1, 0x26, 0x94, 0x8f, 0x47, 0xc8, - 0xc6, 0x17, 0x77, 0x13, 0x6a, 0x9f, 0xf5, 0x57, 0x0d, 0xcc, 0xfd, 0x31, 0xdf, 0x89, 0xc2, 0x23, - 0xda, 0x7f, 0xe1, 0x34, 0x27, 0x50, 0x92, 0xf6, 0x2a, 0xaf, 0x19, 0xeb, 0x35, 0x5b, 0xae, 0x85, - 0x2d, 0x3d, 0x1c, 0x3b, 0x43, 0x86, 0x47, 0xf4, 0x14, 0x95, 0xb5, 0x6b, 0x76, 0xdd, 0xc3, 0xf1, - 0x5e, 0xc2, 0xb2, 0x9e, 0xe9, 0xd0, 0x48, 0x6d, 0x29, 0xf0, 0xb9, 0x8a, 0x42, 0x79, 0x4f, 0xac, - 0x5f, 0x7e, 0xea, 0x32, 0xbd, 0x53, 0x9a, 0x9d, 0x47, 0xef, 0x41, 0x43, 0x9a, 0xc3, 0x09, 0xa3, - 0x1e, 0x66, 0xd6, 0xad, 0x4b, 0xde, 0x23, 0xc9, 0x2a, 0x02, 0x55, 0xb9, 0x8c, 0x8b, 0x54, 0xa7, - 0xbb, 0x08, 0x81, 0xd2, 0x80, 0xc6, 0x2a, 0xaf, 0x34, 0x6c, 0xb9, 0xb6, 0xbe, 0x07, 0xf5, 0x03, - 0x1a, 0xe0, 0x01, 0xed, 0x7a, 0xbb, 0xbc, 0x7f, 0x15, 0xb8, 0xf2, 0xe9, 0x8c, 0x5e, 0x98, 0xce, - 0xcc, 0x7d, 0x61, 0xac, 0xef, 0x6b, 0xb0, 0xf0, 0xb6, 0x3f, 0xe2, 0x83, 0x2b, 0xde, 0x5e, 0xc8, - 0xc7, 0xfa, 0x94, 0x7c, 0x3c, 0x47, 0x86, 0x9f, 0x6b, 0x50, 0x7d, 0x88, 0xe3, 0xad, 0x7d, 0xec, - 0x4b, 0xfb, 0x89, 0x9c, 0x9a, 0x4e, 0x6d, 0x24, 0x41, 0xee, 0x42, 0x7d, 0x22, 0x8b, 0x24, 0xe7, - 0x43, 0x9e, 0x44, 0x2e, 0x78, 0x46, 0x6f, 0xc3, 0x02, 0xe5, 0xce, 0x63, 0xd7, 0xa7, 0x3d, 0x69, - 0xff, 0x05, 0xbb, 0x4a, 0xf9, 0xbb, 0x82, 0x14, 0xf9, 0x2b, 0x13, 0x53, 0x79, 0xbb, 0x61, 0x4f, - 0x70, 0xac, 0xf7, 0x00, 0x12, 0xd1, 0x04, 0x40, 0x99, 0x77, 0x69, 0x93, 0xde, 0xf5, 0x55, 0xa8, - 0x7a, 0x38, 0xde, 0xe2, 0xd8, 0x6f, 0xe9, 0x32, 0xf9, 0xcf, 0x42, 0x2d, 0x39, 0xc9, 0x4e, 0xb7, - 0x5b, 0x3f, 0xd0, 0xa1, 0xf6, 0xad, 0xc8, 0xed, 0x75, 0xc2, 0x1e, 0x9e, 0x3e, 0x57, 0xf8, 0x8f, - 0x28, 0xfa, 0xbd, 0x47, 0x79, 0xfe, 0xcf, 0x19, 0x22, 0x38, 0x24, 0x91, 0x07, 0x47, 0x42, 0x0a, - 0xd8, 0xa9, 0x90, 0xcc, 0x19, 0xba, 0xf1, 0x20, 0xcd, 0x05, 0x20, 0x59, 0x7b, 0x82, 0x43, 0xda, - 0xd0, 0x48, 0x37, 0x30, 0x37, 0x50, 0x19, 0xa1, 0xbe, 0x75, 0x6f, 0x6a, 0xa0, 0x3e, 0xc4, 0xf1, - 0xbb, 0xae, 0x3f, 0xc2, 0x3d, 0x97, 0x32, 0xbb, 0x9e, 0x1c, 0x22, 0xbe, 0xb2, 0x9e, 0x6a, 0x00, - 0x12, 0x01, 0x11, 0xcb, 0xe7, 0x0f, 0xd5, 0x3e, 0xce, 0xa1, 0xe4, 0x4b, 0xb0, 0x12, 0x8e, 0x02, - 0x87, 0xa1, 0xef, 0xc6, 0xd8, 0x73, 0x12, 0x30, 0x78, 0x02, 0x0e, 0x09, 0x47, 0x81, 0xad, 0x7e, - 0xda, 0x4f, 0x7e, 0xb1, 0x7e, 0xa8, 0x01, 0xbc, 0x2d, 0x34, 0x57, 0x62, 0x9c, 0x6d, 0x61, 0xb4, - 0x29, 0x2d, 0xcc, 0x04, 0x74, 0x7a, 0x11, 0xba, 0xed, 0x14, 0x3a, 0x91, 0x97, 0x78, 0x32, 0xda, - 0xbc, 0x37, 0xc3, 0x9c, 0xb9, 0xf2, 0x09, 0xba, 0x72, 0x6d, 0xfd, 0x4c, 0x4d, 0x83, 0x85, 0x74, - 0x4a, 0xa4, 0x82, 0x95, 0xb5, 0xb3, 0x56, 0xbe, 0x0b, 0xf5, 0x00, 0x83, 0x88, 0x8d, 0x1d, 0x4e, - 0x9f, 0x60, 0x1a, 0x24, 0x8a, 0xb5, 0x4f, 0x9f, 0xa0, 0x08, 0x03, 0x09, 0x49, 0x74, 0xc2, 0xd3, - 0xc7, 0x40, 0xc0, 0x10, 0x9d, 0x70, 0xf2, 0x05, 0xb8, 0xc1, 0xb0, 0x8b, 0x61, 0xec, 0x8f, 0x9d, - 0x20, 0xea, 0xd1, 0x23, 0x8a, 0x69, 0xa8, 0x98, 0xe9, 0x0f, 0xbb, 0x09, 0xdf, 0xfa, 0x9b, 0x06, - 0xcd, 0x6f, 0xa7, 0x09, 0x52, 0x49, 0xf6, 0x1c, 0xf2, 0xd6, 0xd7, 0xa4, 0xb2, 0x05, 0xfc, 0xe6, - 0x8c, 0x86, 0x33, 0x90, 0xec, 0x05, 0x8e, 0x7d, 0x25, 0xd4, 0x36, 0xd4, 0xa5, 0x39, 0x92, 0x33, - 0x4a, 0x73, 0x6d, 0x90, 0x5b, 0xde, 0x86, 0xa3, 0x6c, 0x6d, 0xfd, 0x52, 0x07, 0xa2, 0x46, 0x0a, - 0xd2, 0x48, 0x2f, 0x5c, 0x1f, 0xf0, 0xc6, 0xf4, 0x3e, 0xe0, 0x7c, 0x81, 0xf7, 0x19, 0x50, 0x6a, - 0xe5, 0xf3, 0xbe, 0x42, 0x12, 0x68, 0x43, 0x03, 0x4f, 0x63, 0xe6, 0xa6, 0x41, 0x57, 0xbd, 0x74, - 0xd0, 0xc9, 0xcf, 0x92, 0x48, 0x7e, 0x5f, 0x87, 0x95, 0xb4, 0x53, 0x7c, 0x89, 0xd7, 0xc5, 0x78, - 0xfd, 0x4e, 0x87, 0xd7, 0x0a, 0x78, 0xed, 0xb1, 0xa8, 0xcf, 0x90, 0xf3, 0x97, 0xb8, 0xcd, 0xc3, - 0xed, 0xcd, 0xbf, 0x18, 0x50, 0x4d, 0x14, 0x26, 0x35, 0x28, 0x7b, 0x8f, 0xa2, 0x10, 0xcd, 0x57, - 0xc8, 0x2a, 0xdc, 0xf0, 0xce, 0xfe, 0x37, 0x61, 0xf6, 0xc8, 0x32, 0x2c, 0x79, 0xc5, 0xb1, 0xbd, - 0x89, 0x84, 0x40, 0xd3, 0x2b, 0x4c, 0xad, 0xcd, 0x23, 0x72, 0x0b, 0x96, 0xbd, 0xf3, 0x83, 0x5d, - 0x53, 0xbc, 0xfb, 0xa6, 0x57, 0x9c, 0x7d, 0x72, 0x73, 0x20, 0x8f, 0xf8, 0x06, 0xc6, 0x59, 0x11, - 0xcf, 0x4d, 0x4a, 0x56, 0xc1, 0xf4, 0xce, 0x8c, 0x20, 0xcd, 0xdf, 0x6b, 0x64, 0x19, 0x9a, 0x5e, - 0x61, 0xc6, 0x66, 0xfe, 0x41, 0x23, 0x04, 0x16, 0xbd, 0xc9, 0x21, 0x92, 0xf9, 0x47, 0x8d, 0xdc, - 0x02, 0xe2, 0x9d, 0x9b, 0xb5, 0x98, 0x7f, 0xd2, 0xc8, 0x0a, 0x2c, 0x79, 0x85, 0x91, 0x04, 0x37, - 0xff, 0xac, 0x91, 0x1b, 0xd0, 0xf0, 0x26, 0xd2, 0x93, 0xf9, 0x2b, 0x5d, 0x5d, 0x35, 0xe9, 0x53, - 0xe6, 0xfb, 0x3a, 0xb9, 0x03, 0x37, 0xbd, 0xa9, 0x8e, 0x66, 0x3e, 0xd3, 0x49, 0x03, 0xaa, 0x9e, - 0x6a, 0xfe, 0xcd, 0x1f, 0x19, 0x92, 0x52, 0x5d, 0xa9, 0xf9, 0x63, 0x83, 0xd4, 0xa1, 0xe2, 0xc9, - 0xfa, 0xd0, 0xfc, 0x89, 0xfa, 0x49, 0x55, 0xf7, 0xe6, 0x47, 0x86, 0x14, 0x7f, 0xb2, 0xd6, 0x37, - 0xff, 0x6d, 0x90, 0x26, 0xd4, 0xbc, 0xb4, 0x9e, 0x35, 0x7f, 0x5d, 0x93, 0x52, 0x17, 0x9f, 0x0a, - 0xf3, 0x83, 0x1a, 0x59, 0x02, 0xf0, 0xb2, 0xb2, 0xc7, 0xfc, 0x4d, 0xed, 0xcd, 0xb7, 0x60, 0x21, - 0xfd, 0x7b, 0x90, 0x00, 0x54, 0x76, 0x5d, 0x1e, 0x23, 0x33, 0x5f, 0x11, 0x6b, 0x1b, 0xdd, 0x1e, - 0x32, 0x53, 0x13, 0xeb, 0xef, 0x30, 0x2a, 0xf8, 0xba, 0xb0, 0xf9, 0x9e, 0x70, 0x4c, 0xd3, 0xd8, - 0x6e, 0x7f, 0x77, 0xbb, 0x4f, 0xe3, 0xc1, 0xe8, 0x50, 0x78, 0xcd, 0xe6, 0x13, 0xea, 0xfb, 0xf4, - 0x49, 0x8c, 0xdd, 0xc1, 0xa6, 0xf2, 0xa8, 0x2f, 0xf6, 0x28, 0x8f, 0x19, 0x3d, 0x1c, 0xc5, 0xd8, - 0xdb, 0x4c, 0x83, 0x65, 0x53, 0xba, 0x59, 0x46, 0x0e, 0x0f, 0x0f, 0x2b, 0x92, 0xf3, 0xe0, 0xbf, - 0x01, 0x00, 0x00, 0xff, 0xff, 0xb2, 0xdd, 0xfb, 0x16, 0x8f, 0x1f, 0x00, 0x00, + 0x1c, 0x90, 0x90, 0xe6, 0x8c, 0xf8, 0x10, 0x07, 0x4e, 0x70, 0x87, 0x1d, 0xb1, 0x20, 0xfe, 0x07, + 0x3e, 0x2f, 0xfc, 0x17, 0x70, 0x00, 0x89, 0x5d, 0x0e, 0xdc, 0x50, 0x55, 0xf5, 0x87, 0x3b, 0xb1, + 0x9d, 0x88, 0x64, 0x96, 0x59, 0xed, 0xdc, 0xaa, 0x5e, 0x97, 0xab, 0xde, 0xef, 0xf7, 0x5e, 0xbd, + 0x7a, 0xef, 0x19, 0x08, 0x0d, 0x63, 0x64, 0xa1, 0xeb, 0x3b, 0x01, 0xef, 0x6f, 0x0c, 0x59, 0x14, + 0x47, 0x64, 0x35, 0xa0, 0xfe, 0xe3, 0x11, 0x57, 0xb3, 0x8d, 0x74, 0xc1, 0xab, 0x8d, 0x6e, 0x14, + 0x04, 0x51, 0xa8, 0xc4, 0xaf, 0xde, 0xe0, 0xc8, 0x1e, 0xd3, 0x2e, 0xe6, 0xbf, 0xb3, 0x42, 0xa8, + 0x75, 0xda, 0x36, 0x1e, 0x8f, 0x90, 0xc7, 0xe4, 0x26, 0x54, 0x86, 0x88, 0xac, 0xd3, 0x6e, 0x69, + 0x6b, 0xda, 0xba, 0x61, 0x27, 0x33, 0xf2, 0x00, 0x4a, 0x2c, 0xf2, 0xb1, 0xa5, 0xaf, 0x69, 0xeb, + 0xcd, 0xad, 0xbb, 0x1b, 0x53, 0xcf, 0xda, 0xd8, 0x43, 0x64, 0x76, 0xe4, 0xa3, 0x2d, 0x17, 0x93, + 0x15, 0x28, 0x77, 0xa3, 0x51, 0x18, 0xb7, 0x8c, 0x35, 0x6d, 0x7d, 0xd1, 0x56, 0x13, 0xab, 0x0f, + 0x20, 0xce, 0xe3, 0xc3, 0x28, 0xe4, 0x48, 0x1e, 0x40, 0x85, 0xc7, 0x6e, 0x3c, 0xe2, 0xf2, 0xc0, + 0xfa, 0xd6, 0x9d, 0xe2, 0xd6, 0x89, 0xf2, 0xfb, 0x72, 0x89, 0x9d, 0x2c, 0x25, 0x4d, 0xd0, 0x3b, + 0x6d, 0xa9, 0x8b, 0x61, 0xeb, 0x9d, 0xf6, 0x8c, 0x83, 0x22, 0x80, 0x03, 0x1e, 0x7d, 0x8c, 0xc8, + 0x1e, 0x43, 0x5d, 0x1e, 0x78, 0x15, 0x68, 0xaf, 0x41, 0x2d, 0xa6, 0x01, 0xf2, 0xd8, 0x0d, 0x86, + 0x52, 0xa7, 0x92, 0x9d, 0x0b, 0x66, 0x9c, 0xfb, 0x54, 0x83, 0xc6, 0x3e, 0xf6, 0x73, 0x2b, 0x66, + 0xcb, 0xb4, 0x89, 0x65, 0x62, 0xeb, 0xee, 0xc0, 0x0d, 0x43, 0xf4, 0x13, 0xf2, 0xca, 0x76, 0x2e, + 0x20, 0x77, 0xa0, 0xd6, 0x8d, 0x7c, 0xdf, 0x09, 0xdd, 0x00, 0xe5, 0xf6, 0x35, 0x7b, 0x41, 0x08, + 0x1e, 0xb9, 0x01, 0x92, 0xfb, 0xb0, 0x38, 0x74, 0x59, 0x4c, 0x63, 0x1a, 0x85, 0x4e, 0xec, 0xf6, + 0x5b, 0x25, 0xb9, 0xa0, 0x91, 0x09, 0x0f, 0xdc, 0xbe, 0xf5, 0x4c, 0x03, 0xf2, 0x75, 0xce, 0x69, + 0x3f, 0x2c, 0x28, 0x73, 0xad, 0xc4, 0x3f, 0x84, 0xa5, 0x21, 0x32, 0x27, 0x51, 0xdb, 0x61, 0x78, + 0xdc, 0x32, 0xd6, 0x8c, 0xf5, 0xfa, 0xd6, 0xfd, 0x19, 0xbf, 0x9f, 0x54, 0xc5, 0x5e, 0x1c, 0x22, + 0xdb, 0x51, 0x3f, 0xb5, 0xf1, 0xd8, 0xfa, 0x50, 0x83, 0x25, 0xf9, 0x5d, 0x69, 0x1d, 0x60, 0x28, + 0xa9, 0xe3, 0x42, 0x94, 0x28, 0xab, 0x26, 0x17, 0x50, 0x37, 0xd5, 0x2a, 0x45, 0x42, 0x4b, 0x17, + 0x11, 0x5a, 0x3e, 0x4f, 0x28, 0xb9, 0x0b, 0x75, 0x3c, 0x1d, 0x52, 0x86, 0x8e, 0xf0, 0x80, 0x56, + 0x45, 0x7a, 0x03, 0x28, 0xd1, 0x01, 0x0d, 0x26, 0x3d, 0xac, 0x7a, 0x69, 0x0f, 0xb3, 0x38, 0x2c, + 0x17, 0xac, 0x94, 0x78, 0xeb, 0x7b, 0x70, 0x73, 0x92, 0x59, 0x37, 0xa3, 0xa4, 0xa5, 0x49, 0x82, + 0x3f, 0x37, 0x8f, 0xe0, 0x9c, 0x40, 0x7b, 0x25, 0xe7, 0x38, 0x97, 0x5a, 0xff, 0xd1, 0xe0, 0xd6, + 0x0e, 0x43, 0x37, 0xc6, 0x9d, 0xc8, 0xf7, 0xb1, 0x2b, 0x20, 0xa6, 0x0e, 0xf2, 0x16, 0x2c, 0x04, + 0xbc, 0xef, 0xc4, 0xe3, 0x21, 0x4a, 0xd6, 0x9b, 0x5b, 0xaf, 0xcf, 0x38, 0x6b, 0x97, 0xf7, 0x0f, + 0xc6, 0x43, 0xb4, 0xab, 0x81, 0x1a, 0x10, 0x0b, 0x1a, 0xdd, 0x6c, 0xbf, 0x2c, 0x24, 0x14, 0x64, + 0xc2, 0x3a, 0x0c, 0x8f, 0x3b, 0x6d, 0x69, 0x1d, 0xc3, 0x56, 0x93, 0xe2, 0x3d, 0x2b, 0x9d, 0xbd, + 0x67, 0x2d, 0xa8, 0x0e, 0x59, 0x74, 0x3a, 0xee, 0xb4, 0xa5, 0x61, 0x0c, 0x3b, 0x9d, 0x92, 0x2f, + 0x43, 0x85, 0x77, 0x07, 0x18, 0xb8, 0xd2, 0x1c, 0xf5, 0xad, 0xdb, 0x53, 0x29, 0xdf, 0xf6, 0xa3, + 0x43, 0x3b, 0x59, 0x68, 0xfd, 0x44, 0x87, 0xd5, 0x36, 0x8b, 0x86, 0x9f, 0x70, 0xe4, 0xbb, 0xb0, + 0x94, 0xef, 0xae, 0xbc, 0x5a, 0x51, 0xf0, 0xd9, 0xa2, 0xce, 0xc9, 0x0b, 0xb3, 0x91, 0xc3, 0x15, + 0x1e, 0x6f, 0x37, 0xbb, 0x85, 0xb9, 0xf5, 0x0f, 0x0d, 0x56, 0xde, 0x71, 0xf9, 0xb5, 0x92, 0x92, + 0x01, 0xd6, 0x67, 0x02, 0x36, 0xe6, 0x00, 0x2e, 0x5d, 0x08, 0xb8, 0x7c, 0x05, 0xc0, 0x1f, 0x6a, + 0x70, 0xbb, 0x8d, 0xbc, 0xcb, 0xe8, 0x21, 0x7e, 0x7a, 0x50, 0xff, 0x42, 0x83, 0xd5, 0xfd, 0x41, + 0x74, 0xf2, 0xe2, 0x22, 0xb6, 0x7e, 0xab, 0xc3, 0x4d, 0x15, 0x9b, 0xf6, 0xd2, 0xe8, 0xfb, 0x31, + 0x5d, 0xd0, 0x35, 0xa8, 0x67, 0x01, 0x3f, 0xbb, 0xa6, 0x93, 0xa2, 0x1c, 0x69, 0x69, 0x26, 0xd2, + 0xf2, 0x1c, 0xa4, 0x95, 0xa2, 0x6d, 0xbf, 0x09, 0xcd, 0xfc, 0xd5, 0x91, 0xa6, 0x55, 0xef, 0xc6, + 0xfd, 0xe9, 0xa6, 0xcd, 0xe8, 0x90, 0x96, 0xcd, 0x1f, 0x2c, 0x69, 0xd8, 0x0f, 0x74, 0x58, 0x11, + 0x51, 0xed, 0x25, 0x67, 0x97, 0xe7, 0xec, 0xef, 0x1a, 0x2c, 0xbf, 0xe3, 0xf2, 0xeb, 0xa4, 0xec, + 0x7a, 0x2f, 0xff, 0x79, 0xb0, 0xe5, 0xff, 0x19, 0xec, 0x3f, 0x35, 0x68, 0xa5, 0xf1, 0xee, 0xd3, + 0x81, 0x58, 0x3c, 0x69, 0x22, 0xd6, 0xbd, 0xb8, 0x68, 0xaf, 0x39, 0xb8, 0xff, 0x4b, 0x87, 0xc5, + 0x4e, 0xc8, 0x91, 0xc5, 0xcf, 0x0d, 0xe9, 0x1b, 0xe7, 0x35, 0x56, 0xc5, 0xc9, 0x19, 0x5d, 0x2e, + 0x55, 0xa2, 0x08, 0xde, 0x38, 0xf6, 0x45, 0x46, 0x9a, 0xe5, 0x37, 0xb9, 0xa0, 0x98, 0xe5, 0xab, + 0x30, 0x30, 0x91, 0xe5, 0x4f, 0xb0, 0x5a, 0x2d, 0xb2, 0xfa, 0x3a, 0x40, 0x46, 0x3e, 0x6f, 0x2d, + 0xac, 0x19, 0x22, 0x4d, 0xcf, 0x25, 0xa2, 0x02, 0x62, 0xd1, 0x49, 0xa7, 0xcd, 0x5b, 0xb5, 0x35, + 0x43, 0x54, 0x40, 0x6a, 0x46, 0xbe, 0x02, 0x0b, 0x2c, 0x3a, 0x71, 0x7a, 0x6e, 0xec, 0xb6, 0x40, + 0x26, 0xd9, 0x73, 0xb2, 0xc9, 0x2a, 0x8b, 0x4e, 0xda, 0x6e, 0xec, 0x5a, 0x4f, 0x75, 0x58, 0x6c, + 0xa3, 0x8f, 0x31, 0xfe, 0xff, 0x49, 0x2f, 0x30, 0x56, 0x9a, 0xc3, 0x58, 0x79, 0x1e, 0x63, 0x95, + 0x73, 0x8c, 0xdd, 0x83, 0xc6, 0x90, 0xd1, 0xc0, 0x65, 0x63, 0xc7, 0xc3, 0xb1, 0x28, 0x6f, 0x0c, + 0x19, 0xe5, 0x95, 0xec, 0x21, 0x8e, 0xb9, 0xf5, 0x91, 0x06, 0x8b, 0xfb, 0xe8, 0xb2, 0xee, 0xe0, + 0xb9, 0xd1, 0x30, 0xa1, 0xbf, 0x51, 0xd4, 0x7f, 0x7e, 0x0e, 0xfd, 0x79, 0x30, 0x19, 0xf2, 0x91, + 0x1f, 0x3b, 0x39, 0x39, 0x8a, 0x80, 0x25, 0x25, 0xdf, 0xc9, 0x28, 0xda, 0x84, 0xf2, 0xf1, 0x08, + 0xd9, 0xf8, 0xe2, 0x6a, 0x42, 0xad, 0xb3, 0xfe, 0xaa, 0x81, 0xb9, 0x3f, 0xe6, 0x3b, 0x51, 0x78, + 0x44, 0xfb, 0x2f, 0x1c, 0x72, 0x02, 0x25, 0x69, 0xaf, 0xf2, 0x9a, 0xb1, 0x5e, 0xb3, 0xe5, 0x58, + 0xd8, 0xd2, 0xc3, 0xb1, 0x33, 0x64, 0x78, 0x44, 0x4f, 0x51, 0x59, 0xbb, 0x66, 0xd7, 0x3d, 0x1c, + 0xef, 0x25, 0x22, 0xeb, 0x99, 0x0e, 0x8d, 0xd4, 0x96, 0x82, 0x9f, 0xab, 0x00, 0xca, 0x6b, 0x62, + 0xfd, 0xf2, 0x5d, 0x97, 0xe9, 0x95, 0xd2, 0xec, 0x38, 0x7a, 0x0f, 0x1a, 0xd2, 0x1c, 0x4e, 0x18, + 0xf5, 0x30, 0xb3, 0x6e, 0x5d, 0xca, 0x1e, 0x49, 0x51, 0x91, 0xa8, 0xca, 0x65, 0x5c, 0xa4, 0x3a, + 0xdd, 0x45, 0x08, 0x94, 0x06, 0x34, 0x56, 0x71, 0xa5, 0x61, 0xcb, 0xb1, 0xf5, 0x3d, 0xa8, 0x1f, + 0xd0, 0x00, 0x0f, 0x68, 0xd7, 0xdb, 0xe5, 0xfd, 0xab, 0xd0, 0x95, 0x77, 0x67, 0xf4, 0x42, 0x77, + 0x66, 0xee, 0x0b, 0x63, 0x7d, 0x5f, 0x83, 0x85, 0xb7, 0xfd, 0x11, 0x1f, 0x5c, 0xf1, 0xf4, 0x42, + 0x3c, 0xd6, 0xa7, 0xc4, 0xe3, 0x39, 0x3a, 0xfc, 0x4c, 0x83, 0xea, 0x43, 0x1c, 0x6f, 0xed, 0x63, + 0x5f, 0xda, 0x4f, 0xc4, 0xd4, 0xb4, 0x6b, 0x23, 0x27, 0xe4, 0x2e, 0xd4, 0x27, 0xa2, 0x48, 0xb2, + 0x3f, 0xe4, 0x41, 0xe4, 0x82, 0x67, 0xf4, 0x36, 0x2c, 0x50, 0xee, 0x3c, 0x76, 0x7d, 0xda, 0x93, + 0xf6, 0x5f, 0xb0, 0xab, 0x94, 0xbf, 0x2b, 0xa6, 0x22, 0x7e, 0x65, 0x6a, 0x2a, 0x6f, 0x37, 0xec, + 0x09, 0x89, 0xf5, 0x1e, 0x40, 0xa2, 0x9a, 0x20, 0x28, 0xf3, 0x2e, 0x6d, 0xd2, 0xbb, 0xbe, 0x0a, + 0x55, 0x0f, 0xc7, 0x5b, 0x1c, 0xfb, 0x2d, 0x5d, 0x06, 0xff, 0x59, 0xac, 0x25, 0x3b, 0xd9, 0xe9, + 0x72, 0xeb, 0xe7, 0x1a, 0xd4, 0xbe, 0x15, 0xb9, 0xbd, 0x4e, 0xd8, 0xc3, 0xd3, 0xe7, 0x47, 0x7f, + 0x0b, 0xaa, 0x47, 0x14, 0xfd, 0x5e, 0x1e, 0x04, 0x92, 0xa9, 0x20, 0x96, 0x8a, 0xb3, 0x9d, 0xa1, + 0x1b, 0x0f, 0x78, 0xab, 0x24, 0x6f, 0x34, 0x48, 0xd1, 0x9e, 0x90, 0x58, 0x4f, 0x35, 0x00, 0xa9, + 0x9d, 0xb8, 0x67, 0x9c, 0xb4, 0xa1, 0x91, 0xae, 0x67, 0x6e, 0xc0, 0x93, 0x8e, 0xd2, 0xbd, 0xa9, + 0x37, 0xf3, 0x21, 0x8e, 0xdf, 0x75, 0xfd, 0x11, 0xee, 0xb9, 0x94, 0xd9, 0xf5, 0x64, 0x4f, 0xf1, + 0x2b, 0xf2, 0x25, 0x58, 0x09, 0x47, 0x81, 0xc3, 0xd0, 0x77, 0x63, 0xec, 0x39, 0x89, 0xa2, 0x3c, + 0x51, 0x9c, 0x84, 0xa3, 0xc0, 0x56, 0x9f, 0xf6, 0x93, 0x2f, 0xd6, 0x0f, 0x34, 0x80, 0xb7, 0x85, + 0xce, 0x4a, 0x8d, 0xb3, 0xe5, 0x85, 0x36, 0xa5, 0xbc, 0x98, 0x00, 0xad, 0x17, 0x41, 0x6f, 0xa7, + 0xa0, 0x45, 0xcc, 0xe0, 0x49, 0xdb, 0xf1, 0xde, 0x0c, 0xaa, 0x73, 0xf0, 0x09, 0x2f, 0x72, 0x6c, + 0xfd, 0x54, 0x75, 0x6a, 0x85, 0x76, 0x4a, 0xa5, 0x82, 0x05, 0xb4, 0xb3, 0x16, 0xb8, 0x0b, 0xf5, + 0x00, 0x83, 0x88, 0x8d, 0x1d, 0x4e, 0x9f, 0x60, 0xea, 0xc0, 0x4a, 0xb4, 0x4f, 0x9f, 0xa0, 0x70, + 0x51, 0x49, 0x49, 0x74, 0xc2, 0x53, 0x1b, 0x09, 0x1a, 0xa2, 0x13, 0x4e, 0xbe, 0x00, 0x37, 0x18, + 0x76, 0x31, 0x8c, 0xfd, 0xb1, 0x13, 0x44, 0x3d, 0x7a, 0x44, 0x31, 0x75, 0x63, 0x33, 0xfd, 0xb0, + 0x9b, 0xc8, 0xad, 0xbf, 0x69, 0xd0, 0xfc, 0x76, 0x1a, 0xbc, 0x94, 0x66, 0xcf, 0x21, 0xa6, 0x7c, + 0x4d, 0x82, 0x2d, 0xf0, 0x37, 0xa7, 0x6d, 0x9b, 0x91, 0x64, 0x2f, 0x70, 0xec, 0x2b, 0xa5, 0xb6, + 0xa1, 0x2e, 0xcd, 0x91, 0xec, 0x51, 0x9a, 0x6b, 0x83, 0xdc, 0xf2, 0x36, 0x1c, 0x65, 0x63, 0xeb, + 0x97, 0x3a, 0x10, 0x55, 0xee, 0x4b, 0x23, 0xbd, 0x70, 0x39, 0xfa, 0x1b, 0xd3, 0x73, 0xf4, 0xf3, + 0xc9, 0xd7, 0x67, 0x40, 0xc1, 0xca, 0x7b, 0x71, 0x35, 0xbb, 0x26, 0x25, 0xf2, 0x73, 0x1b, 0x1a, + 0x78, 0x1a, 0x33, 0x37, 0xbd, 0x74, 0xd5, 0x4b, 0x5f, 0x3a, 0xf9, 0x33, 0x75, 0xe9, 0xac, 0xf7, + 0x45, 0x99, 0x9f, 0x54, 0x71, 0x2f, 0xf9, 0xba, 0x98, 0xaf, 0xdf, 0xe9, 0xf0, 0x5a, 0x81, 0xaf, + 0x3d, 0x16, 0xf5, 0x19, 0x72, 0xfe, 0x92, 0xb7, 0x79, 0xbc, 0xbd, 0xf9, 0x17, 0x03, 0xaa, 0x09, + 0x60, 0x52, 0x83, 0xb2, 0xf7, 0x28, 0x0a, 0xd1, 0x7c, 0x85, 0xac, 0xc2, 0x0d, 0xef, 0xec, 0xff, + 0x06, 0x66, 0x8f, 0x2c, 0xc3, 0x92, 0x57, 0x6c, 0xa9, 0x9b, 0x48, 0x08, 0x34, 0xbd, 0x42, 0x47, + 0xd9, 0x3c, 0x22, 0xb7, 0x60, 0xd9, 0x3b, 0xdf, 0x74, 0x35, 0xc5, 0x9b, 0x6c, 0x7a, 0xc5, 0xbe, + 0x24, 0x37, 0x07, 0x72, 0x8b, 0x6f, 0x60, 0x9c, 0x25, 0xd8, 0xdc, 0xa4, 0x64, 0x15, 0x4c, 0xef, + 0x4c, 0x7b, 0xd0, 0xfc, 0xbd, 0x46, 0x96, 0xa1, 0xe9, 0x15, 0xfa, 0x5f, 0xe6, 0x1f, 0x34, 0x42, + 0x60, 0xd1, 0x9b, 0x6c, 0xf0, 0x98, 0x7f, 0xd4, 0xc8, 0x2d, 0x20, 0xde, 0xb9, 0x3e, 0x88, 0xf9, + 0x27, 0x8d, 0xac, 0xc0, 0x92, 0x57, 0x68, 0x17, 0x70, 0xf3, 0xcf, 0x1a, 0xb9, 0x01, 0x0d, 0x6f, + 0x22, 0x3c, 0x99, 0xbf, 0xd2, 0xd5, 0x51, 0x93, 0x3e, 0x65, 0xbe, 0xaf, 0x93, 0x3b, 0x70, 0xd3, + 0x9b, 0xea, 0x68, 0xe6, 0x33, 0x9d, 0x34, 0xa0, 0xea, 0xa9, 0xc2, 0xdc, 0xfc, 0xa1, 0x21, 0x67, + 0xaa, 0x62, 0x34, 0x7f, 0x64, 0x90, 0x3a, 0x54, 0x3c, 0x99, 0xbb, 0x99, 0x3f, 0x56, 0x9f, 0x54, + 0xe6, 0x6d, 0x7e, 0x64, 0x48, 0xf5, 0x27, 0xf3, 0x70, 0xf3, 0xdf, 0x06, 0x69, 0x42, 0xcd, 0x4b, + 0x73, 0x4d, 0xf3, 0xd7, 0x35, 0xa9, 0x75, 0xf1, 0xa9, 0x30, 0x3f, 0xa8, 0x91, 0x25, 0x00, 0x2f, + 0x4b, 0x49, 0xcc, 0xdf, 0xd4, 0xde, 0x7c, 0x0b, 0x16, 0xd2, 0xbf, 0xee, 0x08, 0x40, 0x65, 0xd7, + 0xe5, 0x31, 0x32, 0xf3, 0x15, 0x31, 0xb6, 0xd1, 0xed, 0x21, 0x33, 0x35, 0x31, 0xfe, 0x0e, 0xa3, + 0x42, 0xae, 0x0b, 0x9b, 0xef, 0x09, 0xc7, 0x34, 0x8d, 0xed, 0xf6, 0x77, 0xb7, 0xfb, 0x34, 0x1e, + 0x8c, 0x0e, 0x85, 0xd7, 0x6c, 0x3e, 0xa1, 0xbe, 0x4f, 0x9f, 0xc4, 0xd8, 0x1d, 0x6c, 0x2a, 0x8f, + 0xfa, 0x62, 0x8f, 0xf2, 0x98, 0xd1, 0xc3, 0x51, 0x8c, 0xbd, 0xcd, 0xf4, 0xb2, 0x6c, 0x4a, 0x37, + 0xcb, 0xa6, 0xc3, 0xc3, 0xc3, 0x8a, 0x94, 0x3c, 0xf8, 0x6f, 0x00, 0x00, 0x00, 0xff, 0xff, 0x40, + 0xb8, 0xbd, 0x0b, 0x2b, 0x1f, 0x00, 0x00, } diff --git a/internal/proxy/proxy.go b/internal/proxy/proxy.go index f4232bc822..02995dbffc 100644 --- a/internal/proxy/proxy.go +++ b/internal/proxy/proxy.go @@ -69,21 +69,18 @@ func CreateProxy(ctx context.Context) (*Proxy, error) { return nil, err } p.idAllocator = idAllocator - p.idAllocator.PeerID = Params.ProxyID() tsoAllocator, err := allocator.NewTimestampAllocator(p.proxyLoopCtx, masterAddr) if err != nil { return nil, err } p.tsoAllocator = tsoAllocator - p.tsoAllocator.PeerID = Params.ProxyID() - segAssigner, err := allocator.NewSegIDAssigner(p.proxyLoopCtx, masterAddr, p.lastTick) + segAssigner, err := allocator.NewSegIDAssigner(p.proxyLoopCtx, masterAddr) if err != nil { panic(err) } p.segAssigner = segAssigner - p.segAssigner.PeerID = Params.ProxyID() p.manipulationMsgStream = msgstream.NewPulsarMsgStream(p.proxyLoopCtx, Params.MsgStreamInsertBufSize()) p.manipulationMsgStream.SetPulsarClient(pulsarAddress) @@ -108,10 +105,6 @@ func (p *Proxy) AddStartCallback(callbacks ...func()) { p.startCallbacks = append(p.startCallbacks, callbacks...) } -func (p *Proxy) lastTick() Timestamp { - return p.tick.LastTick() -} - func (p *Proxy) startProxy() error { err := p.connectMaster() if err != nil { diff --git a/internal/proxy/proxy_test.go b/internal/proxy/proxy_test.go index 29c32f94e9..01e8e74fd1 100644 --- a/internal/proxy/proxy_test.go +++ b/internal/proxy/proxy_test.go @@ -10,9 +10,6 @@ import ( "strings" "sync" "testing" - "time" - - "github.com/zilliztech/milvus-distributed/internal/util/tsoutil" "go.uber.org/zap" "google.golang.org/grpc" @@ -402,9 +399,8 @@ func TestProxy_AssignSegID(t *testing.T) { collectionName := "CreateCollection1" createCollection(t, collectionName) testNum := 1 - futureTS := tsoutil.ComposeTS(time.Now().Add(time.Second*-1000).UnixNano()/int64(time.Millisecond), 0) for i := 0; i < testNum; i++ { - segID, err := proxyServer.segAssigner.GetSegmentID(collectionName, Params.defaultPartitionTag(), int32(i), 200000, futureTS) + segID, err := proxyServer.segAssigner.GetSegmentID(collectionName, Params.defaultPartitionTag(), int32(i), 200000) assert.Nil(t, err) fmt.Println("segID", segID) } diff --git a/internal/proxy/repack_func.go b/internal/proxy/repack_func.go index 44139999e0..83a44e5b27 100644 --- a/internal/proxy/repack_func.go +++ b/internal/proxy/repack_func.go @@ -4,8 +4,6 @@ import ( "log" "sort" - "github.com/zilliztech/milvus-distributed/internal/util/typeutil" - "github.com/zilliztech/milvus-distributed/internal/allocator" "github.com/zilliztech/milvus-distributed/internal/errors" "github.com/zilliztech/milvus-distributed/internal/msgstream" @@ -20,8 +18,7 @@ func insertRepackFunc(tsMsgs []msgstream.TsMsg, result := make(map[int32]*msgstream.MsgPack) - channelCountMap := make(map[UniqueID]map[int32]uint32) // reqID --> channelID to count - channelMaxTSMap := make(map[UniqueID]map[int32]Timestamp) // reqID --> channelID to max Timestamp + channelCountMap := make(map[UniqueID]map[int32]uint32) // reqID --> channelID to count reqSchemaMap := make(map[UniqueID][]string) for i, request := range tsMsgs { @@ -48,23 +45,12 @@ func insertRepackFunc(tsMsgs []msgstream.TsMsg, channelCountMap[reqID] = make(map[int32]uint32) } - if _, ok := channelMaxTSMap[reqID]; !ok { - channelMaxTSMap[reqID] = make(map[int32]Timestamp) - } - if _, ok := reqSchemaMap[reqID]; !ok { reqSchemaMap[reqID] = []string{insertRequest.CollectionName, insertRequest.PartitionTag} } - for idx, channelID := range keys { + for _, channelID := range keys { 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 - } } } @@ -78,12 +64,7 @@ func insertRepackFunc(tsMsgs []msgstream.TsMsg, schema := reqSchemaMap[reqID] collName, partitionTag := schema[0], schema[1] for channelID, count := range countInfo { - 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) + mapInfo, err := segIDAssigner.GetSegmentID(collName, partitionTag, channelID, count) if err != nil { return nil, err } diff --git a/internal/proxy/timetick.go b/internal/proxy/timetick.go index 34b79ec26c..ffbfb50771 100644 --- a/internal/proxy/timetick.go +++ b/internal/proxy/timetick.go @@ -104,10 +104,6 @@ func (tt *timeTick) tickLoop() { } } -func (tt *timeTick) LastTick() Timestamp { - return tt.lastTick -} - func (tt *timeTick) Start() error { tt.lastTick = 0 ts, err := tt.tsoAllocator.AllocOne() diff --git a/internal/querynode/client/client.go b/internal/querynode/client/client.go index 5cbfc2a08e..19ffdae3c0 100644 --- a/internal/querynode/client/client.go +++ b/internal/querynode/client/client.go @@ -21,8 +21,7 @@ func NewLoadIndexClient(ctx context.Context, pulsarAddress string, loadIndexChan } } -func (lic *LoadIndexClient) LoadIndex(indexPaths []string, segmentID int64, fieldID int64, indexParam map[string]string) error { - // TODO:: add indexParam to proto +func (lic *LoadIndexClient) LoadIndex(indexPaths []string, segmentID int64, fieldID int64) error { baseMsg := msgstream.BaseMsg{ BeginTimestamp: 0, EndTimestamp: 0, diff --git a/internal/querynode/collection_replica.go b/internal/querynode/collection_replica.go index 430fd55dd7..688841c31b 100644 --- a/internal/querynode/collection_replica.go +++ b/internal/querynode/collection_replica.go @@ -54,7 +54,7 @@ type collectionReplica interface { // segment getSegmentNum() int - getSegmentStatistics() []*internalpb.SegmentStats + getSegmentStatistics() *internalpb.QueryNodeStats addSegment(segmentID UniqueID, partitionTag string, collectionID UniqueID) error removeSegment(segmentID UniqueID) error getSegmentByID(segmentID UniqueID) (*Segment, error) @@ -317,7 +317,7 @@ func (colReplica *collectionReplicaImpl) getSegmentNum() int { return len(colReplica.segments) } -func (colReplica *collectionReplicaImpl) getSegmentStatistics() []*internalpb.SegmentStats { +func (colReplica *collectionReplicaImpl) getSegmentStatistics() *internalpb.QueryNodeStats { colReplica.mu.RLock() defer colReplica.mu.RUnlock() @@ -339,7 +339,10 @@ func (colReplica *collectionReplicaImpl) getSegmentStatistics() []*internalpb.Se segment.recentlyModified = false } - return statisticData + return &internalpb.QueryNodeStats{ + MsgType: internalpb.MsgType_kQueryNodeStats, + SegStats: statisticData, + } } func (colReplica *collectionReplicaImpl) addSegment(segmentID UniqueID, partitionTag string, collectionID UniqueID) error { @@ -356,7 +359,7 @@ func (colReplica *collectionReplicaImpl) addSegment(segmentID UniqueID, partitio colReplica.mu.Lock() defer colReplica.mu.Unlock() - var newSegment = newSegment(collection, segmentID, partitionTag, collectionID) + var newSegment = newSegment(collection, segmentID) colReplica.segments[segmentID] = newSegment *partition.Segments() = append(*partition.Segments(), newSegment) diff --git a/internal/querynode/load_index.go b/internal/querynode/load_index.go new file mode 100644 index 0000000000..d49dad234e --- /dev/null +++ b/internal/querynode/load_index.go @@ -0,0 +1,41 @@ +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 +} diff --git a/internal/querynode/load_index_info.go b/internal/querynode/load_index_info.go deleted file mode 100644 index 362b687b76..0000000000 --- a/internal/querynode/load_index_info.go +++ /dev/null @@ -1,98 +0,0 @@ -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 -} diff --git a/internal/querynode/load_index_service.go b/internal/querynode/load_index_service.go deleted file mode 100644 index a2eaac7bfe..0000000000 --- a/internal/querynode/load_index_service.go +++ /dev/null @@ -1,286 +0,0 @@ -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 -} diff --git a/internal/querynode/load_index_service_test.go b/internal/querynode/load_index_service_test.go deleted file mode 100644 index 49d2156703..0000000000 --- a/internal/querynode/load_index_service_test.go +++ /dev/null @@ -1,148 +0,0 @@ -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() -} diff --git a/internal/querynode/param_table.go b/internal/querynode/param_table.go index 421cfed56a..d8c825e761 100644 --- a/internal/querynode/param_table.go +++ b/internal/querynode/param_table.go @@ -21,12 +21,6 @@ type ParamTable struct { FlowGraphMaxQueueLength int32 FlowGraphMaxParallelism int32 - // minio - MinioEndPoint string - MinioAccessKeyID string - MinioSecretAccessKey string - MinioUseSSLStr bool - // dm InsertChannelNames []string InsertChannelRange []int @@ -50,11 +44,6 @@ type ParamTable struct { StatsChannelName string StatsReceiveBufSize int64 - // load index - LoadIndexChannelNames []string - LoadIndexReceiveBufSize int64 - LoadIndexPulsarBufSize int64 - GracefulTime int64 MsgChannelSubName string DefaultPartitionTag string @@ -70,11 +59,6 @@ func (p *ParamTable) Init() { panic(err) } - err = p.LoadYaml("milvus.yaml") - if err != nil { - panic(err) - } - queryNodeIDStr := os.Getenv("QUERY_NODE_ID") if queryNodeIDStr == "" { queryNodeIDList := p.QueryNodeIDList() @@ -94,11 +78,6 @@ func (p *ParamTable) Init() { panic(err) } - p.initMinioEndPoint() - p.initMinioAccessKeyID() - p.initMinioSecretAccessKey() - p.initMinioUseSSLStr() - p.initPulsarAddress() p.initETCDAddress() p.initMetaRootPath() @@ -132,46 +111,6 @@ func (p *ParamTable) Init() { p.initStatsPublishInterval() p.initStatsChannelName() 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() { @@ -419,19 +358,3 @@ func (p *ParamTable) initSliceIndex() { func (p *ParamTable) initQueryNodeNum() { 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") -} diff --git a/internal/querynode/param_table_test.go b/internal/querynode/param_table_test.go index 461073146f..8d77bcd046 100644 --- a/internal/querynode/param_table_test.go +++ b/internal/querynode/param_table_test.go @@ -15,47 +15,6 @@ func TestParamTable_PulsarAddress(t *testing.T) { 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) { id := Params.QueryNodeID assert.Contains(t, Params.QueryNodeIDList(), id) diff --git a/internal/querynode/query_node.go b/internal/querynode/query_node.go index 819d2b8554..ed7101824a 100644 --- a/internal/querynode/query_node.go +++ b/internal/querynode/query_node.go @@ -24,12 +24,10 @@ type QueryNode struct { replica collectionReplica - // services - dataSyncService *dataSyncService - metaService *metaService - searchService *searchService - loadIndexService *loadIndexService - statsService *statsService + dataSyncService *dataSyncService + metaService *metaService + searchService *searchService + statsService *statsService } func Init() { @@ -71,13 +69,11 @@ func (node *QueryNode) Start() error { node.dataSyncService = newDataSyncService(node.queryNodeLoopCtx, node.replica) node.searchService = newSearchService(node.queryNodeLoopCtx, node.replica) node.metaService = newMetaService(node.queryNodeLoopCtx, node.replica) - node.loadIndexService = newLoadIndexService(node.queryNodeLoopCtx, node.replica) - node.statsService = newStatsService(node.queryNodeLoopCtx, node.replica, node.loadIndexService.fieldStatsChan) + node.statsService = newStatsService(node.queryNodeLoopCtx, node.replica) go node.dataSyncService.start() go node.searchService.start() go node.metaService.start() - go node.loadIndexService.start() go node.statsService.start() <-node.queryNodeLoopCtx.Done() diff --git a/internal/querynode/query_node_test.go b/internal/querynode/query_node_test.go index 098f2cae2d..3bdce39e27 100644 --- a/internal/querynode/query_node_test.go +++ b/internal/querynode/query_node_test.go @@ -26,7 +26,6 @@ func setup() { func genTestCollectionMeta(collectionName string, collectionID UniqueID) *etcdpb.CollectionMeta { fieldVec := schemapb.FieldSchema{ - FieldID: UniqueID(0), Name: "vec", IsPrimaryKey: false, DataType: schemapb.DataType_VECTOR_FLOAT, @@ -45,7 +44,6 @@ func genTestCollectionMeta(collectionName string, collectionID UniqueID) *etcdpb } fieldInt := schemapb.FieldSchema{ - FieldID: UniqueID(1), Name: "age", IsPrimaryKey: false, DataType: schemapb.DataType_INT32, @@ -121,13 +119,12 @@ func makeNewChannelNames(names []string, suffix string) []string { } func refreshChannelNames() { - suffix := "-test-query-node" + strconv.FormatInt(rand.Int63n(1000000), 10) + suffix := "-test-query-node" + strconv.FormatInt(rand.Int63n(100), 10) Params.DDChannelNames = makeNewChannelNames(Params.DDChannelNames, suffix) Params.InsertChannelNames = makeNewChannelNames(Params.InsertChannelNames, suffix) Params.SearchChannelNames = makeNewChannelNames(Params.SearchChannelNames, suffix) Params.SearchResultChannelNames = makeNewChannelNames(Params.SearchResultChannelNames, suffix) Params.StatsChannelName = Params.StatsChannelName + suffix - Params.LoadIndexChannelNames = makeNewChannelNames(Params.LoadIndexChannelNames, suffix) } func TestMain(m *testing.M) { diff --git a/internal/querynode/reduce_test.go b/internal/querynode/reduce_test.go index afbc87bcfc..7cd03b2b7f 100644 --- a/internal/querynode/reduce_test.go +++ b/internal/querynode/reduce_test.go @@ -21,7 +21,7 @@ func TestReduce_AllFunc(t *testing.T) { assert.NotEqual(t, "", schemaBlob) collection := newCollection(collectionMeta.ID, schemaBlob) - segment := newSegment(collection, segmentID, Params.DefaultPartitionTag, collectionID) + segment := newSegment(collection, segmentID) const DIM = 16 var vec = [DIM]float32{1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16} diff --git a/internal/querynode/segment.go b/internal/querynode/segment.go index 0379092865..e3f7d81252 100644 --- a/internal/querynode/segment.go +++ b/internal/querynode/segment.go @@ -9,6 +9,7 @@ package querynode #include "segcore/collection_c.h" #include "segcore/plan_c.h" #include "segcore/reduce_c.h" + */ import "C" import ( @@ -24,8 +25,6 @@ import ( type Segment struct { segmentPtr C.CSegmentBase segmentID UniqueID - partitionTag string // TODO: use partitionID - collectionID UniqueID lastMemSize int64 lastRowCount int64 recentlyModified bool @@ -36,18 +35,13 @@ func (s *Segment) ID() UniqueID { } //-------------------------------------------------------------------------------------- constructor and destructor -func newSegment(collection *Collection, segmentID int64, partitionTag string, collectionID UniqueID) *Segment { +func newSegment(collection *Collection, segmentID int64) *Segment { /* CSegmentBase newSegment(CPartition partition, unsigned long segment_id); */ segmentPtr := C.NewSegment(collection.collectionPtr, C.ulong(segmentID)) - var newSegment = &Segment{ - segmentPtr: segmentPtr, - segmentID: segmentID, - partitionTag: partitionTag, - collectionID: collectionID, - } + var newSegment = &Segment{segmentPtr: segmentPtr, segmentID: segmentID} return newSegment } @@ -242,17 +236,3 @@ func (s *Segment) fillTargetEntry(plan *Plan, 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 -} diff --git a/internal/querynode/segment_test.go b/internal/querynode/segment_test.go index 23cff02718..f24d060d66 100644 --- a/internal/querynode/segment_test.go +++ b/internal/querynode/segment_test.go @@ -26,7 +26,7 @@ func TestSegment_newSegment(t *testing.T) { assert.Equal(t, collection.ID(), collectionID) segmentID := UniqueID(0) - segment := newSegment(collection, segmentID, Params.DefaultPartitionTag, collectionID) + segment := newSegment(collection, segmentID) assert.Equal(t, segmentID, segment.segmentID) deleteSegment(segment) deleteCollection(collection) @@ -44,7 +44,7 @@ func TestSegment_deleteSegment(t *testing.T) { assert.Equal(t, collection.ID(), collectionID) segmentID := UniqueID(0) - segment := newSegment(collection, segmentID, Params.DefaultPartitionTag, collectionID) + segment := newSegment(collection, segmentID) assert.Equal(t, segmentID, segment.segmentID) deleteSegment(segment) @@ -64,7 +64,7 @@ func TestSegment_getRowCount(t *testing.T) { assert.Equal(t, collection.ID(), collectionID) segmentID := UniqueID(0) - segment := newSegment(collection, segmentID, Params.DefaultPartitionTag, collectionID) + segment := newSegment(collection, segmentID) assert.Equal(t, segmentID, segment.segmentID) ids := []int64{1, 2, 3} @@ -115,7 +115,7 @@ func TestSegment_getDeletedCount(t *testing.T) { assert.Equal(t, collection.ID(), collectionID) segmentID := UniqueID(0) - segment := newSegment(collection, segmentID, Params.DefaultPartitionTag, collectionID) + segment := newSegment(collection, segmentID) assert.Equal(t, segmentID, segment.segmentID) ids := []int64{1, 2, 3} @@ -172,7 +172,7 @@ func TestSegment_getMemSize(t *testing.T) { assert.Equal(t, collection.ID(), collectionID) segmentID := UniqueID(0) - segment := newSegment(collection, segmentID, Params.DefaultPartitionTag, collectionID) + segment := newSegment(collection, segmentID) assert.Equal(t, segmentID, segment.segmentID) 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.ID(), collectionID) segmentID := UniqueID(0) - segment := newSegment(collection, segmentID, Params.DefaultPartitionTag, collectionID) + segment := newSegment(collection, segmentID) assert.Equal(t, segmentID, segment.segmentID) ids := []int64{1, 2, 3} @@ -270,7 +270,7 @@ func TestSegment_segmentDelete(t *testing.T) { assert.Equal(t, collection.ID(), collectionID) segmentID := UniqueID(0) - segment := newSegment(collection, segmentID, Params.DefaultPartitionTag, collectionID) + segment := newSegment(collection, segmentID) assert.Equal(t, segmentID, segment.segmentID) ids := []int64{1, 2, 3} @@ -323,7 +323,7 @@ func TestSegment_segmentSearch(t *testing.T) { assert.Equal(t, collection.ID(), collectionID) segmentID := UniqueID(0) - segment := newSegment(collection, segmentID, Params.DefaultPartitionTag, collectionID) + segment := newSegment(collection, segmentID) assert.Equal(t, segmentID, segment.segmentID) ids := []int64{1, 2, 3} @@ -408,7 +408,7 @@ func TestSegment_segmentPreInsert(t *testing.T) { assert.Equal(t, collection.ID(), collectionID) segmentID := UniqueID(0) - segment := newSegment(collection, segmentID, Params.DefaultPartitionTag, collectionID) + segment := newSegment(collection, segmentID) assert.Equal(t, segmentID, segment.segmentID) const DIM = 16 @@ -450,7 +450,7 @@ func TestSegment_segmentPreDelete(t *testing.T) { assert.Equal(t, collection.ID(), collectionID) segmentID := UniqueID(0) - segment := newSegment(collection, segmentID, Params.DefaultPartitionTag, collectionID) + segment := newSegment(collection, segmentID) assert.Equal(t, segmentID, segment.segmentID) ids := []int64{1, 2, 3} diff --git a/internal/querynode/stats_service.go b/internal/querynode/stats_service.go index 17c8bd9473..b46ccdbc2b 100644 --- a/internal/querynode/stats_service.go +++ b/internal/querynode/stats_service.go @@ -12,23 +12,17 @@ import ( ) type statsService struct { - ctx context.Context - - replica collectionReplica - - fieldStatsChan chan []*internalpb.FieldStats - statsStream msgstream.MsgStream + ctx context.Context + statsStream msgstream.MsgStream + replica collectionReplica } -func newStatsService(ctx context.Context, replica collectionReplica, fieldStatsChan chan []*internalpb.FieldStats) *statsService { +func newStatsService(ctx context.Context, replica collectionReplica) *statsService { return &statsService{ - ctx: ctx, - - replica: replica, - - fieldStatsChan: fieldStatsChan, - statsStream: nil, + ctx: ctx, + statsStream: nil, + replica: replica, } } @@ -56,9 +50,7 @@ func (sService *statsService) start() { case <-sService.ctx.Done(): return case <-time.After(time.Duration(sleepTimeInterval) * time.Millisecond): - sService.publicStatistic(nil) - case fieldStats := <-sService.fieldStatsChan: - sService.publicStatistic(fieldStats) + sService.sendSegmentStatistic() } } } @@ -69,21 +61,20 @@ func (sService *statsService) close() { } } -func (sService *statsService) publicStatistic(fieldStats []*internalpb.FieldStats) { - segStats := sService.replica.getSegmentStatistics() +func (sService *statsService) sendSegmentStatistic() { + statisticData := sService.replica.getSegmentStatistics() - queryNodeStats := internalpb.QueryNodeStats{ - MsgType: internalpb.MsgType_kQueryNodeStats, - PeerID: Params.QueryNodeID, - SegStats: segStats, - FieldStats: fieldStats, - } + // fmt.Println("Publish segment statistic") + // fmt.Println(statisticData) + sService.publicStatistic(statisticData) +} +func (sService *statsService) publicStatistic(statistic *internalpb.QueryNodeStats) { var msg msgstream.TsMsg = &msgstream.QueryNodeStatsMsg{ BaseMsg: msgstream.BaseMsg{ HashValues: []uint32{0}, }, - QueryNodeStats: queryNodeStats, + QueryNodeStats: *statistic, } var msgPack = msgstream.MsgPack{ diff --git a/internal/querynode/stats_service_test.go b/internal/querynode/stats_service_test.go index c7ff4b3f94..8097931bdc 100644 --- a/internal/querynode/stats_service_test.go +++ b/internal/querynode/stats_service_test.go @@ -10,7 +10,7 @@ import ( func TestStatsService_start(t *testing.T) { node := newQueryNode() initTestMeta(t, node, "collection0", 0, 0) - node.statsService = newStatsService(node.queryNodeLoopCtx, node.replica, nil) + node.statsService = newStatsService(node.queryNodeLoopCtx, node.replica) node.statsService.start() node.Close() } @@ -32,11 +32,11 @@ func TestSegmentManagement_sendSegmentStatistic(t *testing.T) { var statsMsgStream msgstream.MsgStream = statsStream - node.statsService = newStatsService(node.queryNodeLoopCtx, node.replica, nil) + node.statsService = newStatsService(node.queryNodeLoopCtx, node.replica) node.statsService.statsStream = statsMsgStream node.statsService.statsStream.Start() // send stats - node.statsService.publicStatistic(nil) + node.statsService.sendSegmentStatistic() node.Close() } diff --git a/internal/util/flowgraph/message.go b/internal/util/flowgraph/message.go index e5c01d7d4e..579f51398b 100644 --- a/internal/util/flowgraph/message.go +++ b/internal/util/flowgraph/message.go @@ -13,14 +13,6 @@ type MsgStreamMsg struct { timestampMax Timestamp } -func GenerateMsgStreamMsg(tsMessages []msgstream.TsMsg, timestampMin, timestampMax Timestamp) *MsgStreamMsg { - return &MsgStreamMsg{ - tsMessages: tsMessages, - timestampMin: timestampMin, - timestampMax: timestampMax, - } -} - func (msMsg *MsgStreamMsg) TimeTick() Timestamp { return msMsg.timestampMax } diff --git a/internal/writenode/data_sync_service.go b/internal/writenode/data_sync_service.go index 3c18b4eaa3..6e3a07265c 100644 --- a/internal/writenode/data_sync_service.go +++ b/internal/writenode/data_sync_service.go @@ -39,7 +39,7 @@ func (dsService *dataSyncService) initNodes() { var dmStreamNode Node = newDmInputNode(dsService.ctx) var ddStreamNode Node = newDDInputNode(dsService.ctx) - var ddNode Node = newDDNode(dsService.ctx) + var ddNode Node = newDDNode() var filterDmNode Node = newFilteredDmNode() var insertBufferNode Node = newInsertBufferNode(dsService.ctx) diff --git a/internal/writenode/data_sync_service_test.go b/internal/writenode/data_sync_service_test.go index 92d9e29c09..13bb6513c5 100644 --- a/internal/writenode/data_sync_service_test.go +++ b/internal/writenode/data_sync_service_test.go @@ -24,7 +24,7 @@ import ( // NOTE: start pulsar before test func TestDataSyncService_Start(t *testing.T) { newMeta() - const ctxTimeInMillisecond = 2000 + const ctxTimeInMillisecond = 200 const closeWithDeadline = true var ctx context.Context diff --git a/internal/writenode/dd_buffer.go b/internal/writenode/dd_buffer.go new file mode 100644 index 0000000000..b312f856ef --- /dev/null +++ b/internal/writenode/dd_buffer.go @@ -0,0 +1,47 @@ +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 +} diff --git a/internal/writenode/flow_graph_dd_node.go b/internal/writenode/flow_graph_dd_node.go index f9b8a39463..d113ed120b 100644 --- a/internal/writenode/flow_graph_dd_node.go +++ b/internal/writenode/flow_graph_dd_node.go @@ -1,65 +1,20 @@ package writenode import ( - "context" - "errors" "log" "sort" - "strconv" "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" internalPb "github.com/zilliztech/milvus-distributed/internal/proto/internalpb" "github.com/zilliztech/milvus-distributed/internal/proto/schemapb" - "github.com/zilliztech/milvus-distributed/internal/storage" ) type ddNode struct { BaseNode - ddMsg *ddMsg - 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 + ddMsg *ddMsg + ddBuffer *ddBuffer } func (ddNode *ddNode) Name() string { @@ -113,62 +68,6 @@ 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 return []*Msg{&res} } @@ -176,18 +75,16 @@ func (ddNode *ddNode) Operate(in []*Msg) []*Msg { func (ddNode *ddNode) createCollection(msg *msgstream.CreateCollectionMsg) { collectionID := msg.CollectionID - // add collection - if _, ok := ddNode.ddRecords.collectionRecords[collectionID]; ok { - err := errors.New("collection " + strconv.FormatInt(collectionID, 10) + " is already exists") + err := ddNode.ddBuffer.addCollection(collectionID) + if err != nil { log.Println(err) return } - ddNode.ddRecords.collectionRecords[collectionID] = nil // TODO: add default partition? var schema schemapb.CollectionSchema - err := proto.Unmarshal((*msg.Schema).Value, &schema) + err = proto.Unmarshal((*msg.Schema).Value, &schema) if err != nil { log.Println(err) return @@ -199,30 +96,17 @@ func (ddNode *ddNode) createCollection(msg *msgstream.CreateCollectionMsg) { timestamp: msg.Timestamp, }) - _, 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) + // TODO: write dd binlog } func (ddNode *ddNode) dropCollection(msg *msgstream.DropCollectionMsg) { collectionID := msg.CollectionID - // remove collection - if _, ok := ddNode.ddRecords.collectionRecords[collectionID]; !ok { - err := errors.New("cannot found collection " + strconv.FormatInt(collectionID, 10)) + err := ddNode.ddBuffer.removeCollection(collectionID) + if err != nil { log.Println(err) return } - delete(ddNode.ddRecords.collectionRecords, collectionID) collectionName := msg.CollectionName.CollectionName ddNode.ddMsg.collectionRecords[collectionName] = append(ddNode.ddMsg.collectionRecords[collectionName], @@ -231,31 +115,17 @@ func (ddNode *ddNode) dropCollection(msg *msgstream.DropCollectionMsg) { timestamp: msg.Timestamp, }) - _, 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) + // TODO: write dd binlog } func (ddNode *ddNode) createPartition(msg *msgstream.CreatePartitionMsg) { partitionID := msg.PartitionID - collectionID := msg.CollectionID - // add partition - if _, ok := ddNode.ddRecords.partitionRecords[partitionID]; ok { - err := errors.New("partition " + strconv.FormatInt(partitionID, 10) + " is already exists") + err := ddNode.ddBuffer.addPartition(partitionID) + if err != nil { log.Println(err) return } - ddNode.ddRecords.partitionRecords[partitionID] = nil partitionTag := msg.PartitionName.Tag ddNode.ddMsg.partitionRecords[partitionTag] = append(ddNode.ddMsg.partitionRecords[partitionTag], @@ -264,31 +134,17 @@ func (ddNode *ddNode) createPartition(msg *msgstream.CreatePartitionMsg) { timestamp: msg.Timestamp, }) - _, 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) + // TODO: write dd binlog } func (ddNode *ddNode) dropPartition(msg *msgstream.DropPartitionMsg) { partitionID := msg.PartitionID - collectionID := msg.CollectionID - // remove partition - if _, ok := ddNode.ddRecords.partitionRecords[partitionID]; !ok { - err := errors.New("cannot found partition " + strconv.FormatInt(partitionID, 10)) + err := ddNode.ddBuffer.removePartition(partitionID) + if err != nil { log.Println(err) return } - delete(ddNode.ddRecords.partitionRecords, partitionID) partitionTag := msg.PartitionName.Tag ddNode.ddMsg.partitionRecords[partitionTag] = append(ddNode.ddMsg.partitionRecords[partitionTag], @@ -297,21 +153,10 @@ func (ddNode *ddNode) dropPartition(msg *msgstream.DropPartitionMsg) { timestamp: msg.Timestamp, }) - _, 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) + // TODO: write dd binlog } -func newDDNode(ctx context.Context) *ddNode { +func newDDNode() *ddNode { maxQueueLength := Params.FlowGraphMaxQueueLength maxParallelism := Params.FlowGraphMaxParallelism @@ -319,46 +164,13 @@ func newDDNode(ctx context.Context) *ddNode { baseNode.SetMaxQueueLength(maxQueueLength) baseNode.SetMaxParallelism(maxParallelism) - ddRecords := &ddRecords{ - collectionRecords: 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) + ddBuffer := &ddBuffer{ + collectionBuffer: make(map[UniqueID]interface{}), + partitionBuffer: make(map[UniqueID]interface{}), } return &ddNode{ - BaseNode: baseNode, - ddRecords: ddRecords, - ddBuffer: &ddBuffer{ - ddData: make(map[UniqueID]*ddData), - maxSize: Params.FlushDdBufSize, - }, - - idAllocator: idAllocator, - kv: minioKV, + BaseNode: baseNode, + ddBuffer: ddBuffer, } } diff --git a/internal/writenode/flow_graph_dd_node_test.go b/internal/writenode/flow_graph_dd_node_test.go deleted file mode 100644 index 5903a47b9f..0000000000 --- a/internal/writenode/flow_graph_dd_node_test.go +++ /dev/null @@ -1,126 +0,0 @@ -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}) -} diff --git a/internal/writenode/write_node_test.go b/internal/writenode/write_node_test.go index b99288b3cd..3905b3ab30 100644 --- a/internal/writenode/write_node_test.go +++ b/internal/writenode/write_node_test.go @@ -1,18 +1,11 @@ package writenode import ( - "context" "fmt" - "log" "math/rand" "os" "strconv" "testing" - - "go.etcd.io/etcd/clientv3" - "go.uber.org/zap" - - "github.com/zilliztech/milvus-distributed/internal/master" ) func makeNewChannelNames(names []string, suffix string) []string { @@ -29,34 +22,6 @@ func refreshChannelNames() { 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) { Params.Init() refreshChannelNames()