mirror of https://github.com/milvus-io/milvus.git
Add logic of getting milvus distributed config from master by grpc
Signed-off-by: XuanYang-cn <xuan.yang@zilliz.com>pull/4973/head^2
parent
8ce0deca4e
commit
0a16a9a651
12
Makefile
12
Makefile
|
@ -56,8 +56,8 @@ verifiers: cppcheck fmt lint ruleguard
|
|||
# Builds various components locally.
|
||||
build-go:
|
||||
@echo "Building each component's binary to './'"
|
||||
@echo "Building query node ..."
|
||||
@mkdir -p $(INSTALL_PATH) && GO111MODULE=on $(GO) build -o $(INSTALL_PATH)/querynode $(PWD)/cmd/querynode/query_node.go 1>/dev/null
|
||||
@echo "Building reader ..."
|
||||
@mkdir -p $(INSTALL_PATH) && GO111MODULE=on $(GO) build -o $(INSTALL_PATH)/reader $(PWD)/cmd/reader/reader.go 1>/dev/null
|
||||
@echo "Building master ..."
|
||||
@mkdir -p $(INSTALL_PATH) && GO111MODULE=on $(GO) build -o $(INSTALL_PATH)/master $(PWD)/cmd/master/main.go 1>/dev/null
|
||||
@echo "Building proxy ..."
|
||||
|
@ -72,7 +72,7 @@ build-cpp-with-unittest:
|
|||
# Runs the tests.
|
||||
unittest: test-cpp test-go
|
||||
|
||||
#TODO: proxy master query node writer's unittest
|
||||
#TODO: proxy master reader writer's unittest
|
||||
test-go:
|
||||
@echo "Running go unittests..."
|
||||
@(env bash $(PWD)/scripts/run_go_unittest.sh)
|
||||
|
@ -83,14 +83,14 @@ test-cpp: build-cpp-with-unittest
|
|||
|
||||
#TODO: build each component to docker
|
||||
docker: verifiers
|
||||
@echo "Building query node docker image '$(TAG)'"
|
||||
@echo "Building reader docker image '$(TAG)'"
|
||||
@echo "Building proxy docker image '$(TAG)'"
|
||||
@echo "Building master docker image '$(TAG)'"
|
||||
|
||||
# Builds each component and installs it to $GOPATH/bin.
|
||||
install: all
|
||||
@echo "Installing binary to './bin'"
|
||||
@mkdir -p $(GOPATH)/bin && cp -f $(PWD)/bin/querynode $(GOPATH)/bin/querynode
|
||||
@mkdir -p $(GOPATH)/bin && cp -f $(PWD)/bin/reader $(GOPATH)/bin/reader
|
||||
@mkdir -p $(GOPATH)/bin && cp -f $(PWD)/bin/master $(GOPATH)/bin/master
|
||||
@mkdir -p $(GOPATH)/bin && cp -f $(PWD)/bin/proxy $(GOPATH)/bin/proxy
|
||||
@mkdir -p $(LIBRARY_PATH) && cp -f $(PWD)/internal/core/output/lib/* $(LIBRARY_PATH)
|
||||
|
@ -100,6 +100,6 @@ clean:
|
|||
@echo "Cleaning up all the generated files"
|
||||
@find . -name '*.test' | xargs rm -fv
|
||||
@find . -name '*~' | xargs rm -fv
|
||||
@rm -rvf querynode
|
||||
@rm -rvf reader
|
||||
@rm -rvf master
|
||||
@rm -rvf proxy
|
||||
|
|
|
@ -6,14 +6,14 @@ import (
|
|||
"os/signal"
|
||||
"syscall"
|
||||
|
||||
"github.com/zilliztech/milvus-distributed/internal/querynode"
|
||||
"github.com/zilliztech/milvus-distributed/internal/reader"
|
||||
)
|
||||
|
||||
func main() {
|
||||
ctx, cancel := context.WithCancel(context.Background())
|
||||
defer cancel()
|
||||
|
||||
querynode.Init()
|
||||
reader.Init()
|
||||
|
||||
sc := make(chan os.Signal, 1)
|
||||
signal.Notify(sc,
|
||||
|
@ -28,7 +28,7 @@ func main() {
|
|||
cancel()
|
||||
}()
|
||||
|
||||
querynode.StartQueryNode(ctx)
|
||||
reader.StartQueryNode(ctx)
|
||||
|
||||
switch sig {
|
||||
case syscall.SIGTERM:
|
|
@ -30,10 +30,10 @@ msgChannel:
|
|||
queryNodeSubNamePrefix: "queryNode"
|
||||
writeNodeSubNamePrefix: "writeNode"
|
||||
|
||||
# default channel range [0, 1)
|
||||
# default channel range [0, 0]
|
||||
channelRange:
|
||||
insert: [0, 1]
|
||||
delete: [0, 1]
|
||||
k2s: [0, 1]
|
||||
search: [0, 1]
|
||||
insert: [0, 15]
|
||||
delete: [0, 15]
|
||||
k2s: [0, 15]
|
||||
search: [0, 0]
|
||||
searchResult: [0, 1]
|
|
@ -1,13 +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.
|
||||
|
||||
common:
|
||||
defaultPartitionTag: _default
|
|
@ -20,6 +20,4 @@ master:
|
|||
minIDAssignCnt: 1024
|
||||
maxIDAssignCnt: 16384
|
||||
# old name: segmentExpireDuration: 2000
|
||||
IDAssignExpiration: 2000 # ms
|
||||
|
||||
maxPartitionNum: 4096
|
||||
IDAssignExpiration: 2000 # ms
|
|
@ -25,7 +25,4 @@ proxy:
|
|||
pulsarBufSize: 1024 # pulsar chan buffer size
|
||||
|
||||
timeTick:
|
||||
bufSize: 512
|
||||
|
||||
maxNameLength: 255
|
||||
maxFieldNum: 64
|
||||
bufSize: 512
|
|
@ -9,7 +9,7 @@
|
|||
# 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.
|
||||
|
||||
queryNode:
|
||||
reader:
|
||||
stats:
|
||||
publishInterval: 1000 # milliseconds
|
||||
|
||||
|
@ -19,6 +19,10 @@ queryNode:
|
|||
maxParallelism: 1024
|
||||
|
||||
msgStream:
|
||||
dm: # TODO: rm dm
|
||||
#streamBufSize: 1024 # msgPack chan buffer size
|
||||
recvBufSize: 1024 # msgPack chan buffer size
|
||||
pulsarBufSize: 1024 # pulsar chan buffer size
|
||||
insert:
|
||||
#streamBufSize: 1024 # msgPack chan buffer size
|
||||
recvBufSize: 1024 # msgPack chan buffer size
|
||||
|
|
|
@ -11,9 +11,9 @@
|
|||
|
||||
|
||||
nodeID: # will be deprecated after v0.2
|
||||
proxyIDList: [1]
|
||||
queryNodeIDList: [2]
|
||||
writeNodeIDList: [3]
|
||||
proxyIDList: [1, 2]
|
||||
queryNodeIDList: [3, 4]
|
||||
writeNodeIDList: [5, 6]
|
||||
|
||||
etcd:
|
||||
address: localhost
|
||||
|
|
|
@ -380,13 +380,18 @@ func (segMgr *SegmentManager) AssignSegmentID(segIDReq []*internalpb.SegIDReques
|
|||
// "/msg_stream/insert"
|
||||
|
||||
message SysConfigRequest {
|
||||
repeated string keys = 1;
|
||||
repeated string key_prefixes = 2;
|
||||
MsgType msg_type = 1;
|
||||
int64 reqID = 2;
|
||||
int64 proxyID = 3;
|
||||
uint64 timestamp = 4;
|
||||
repeated string keys = 5;
|
||||
repeated string key_prefixes = 6;
|
||||
}
|
||||
|
||||
message SysConfigResponse {
|
||||
repeated string keys = 1;
|
||||
repeated string values = 2;
|
||||
common.Status status = 1;
|
||||
repeated string keys = 2;
|
||||
repeated string values = 3;
|
||||
}
|
||||
```
|
||||
|
||||
|
@ -394,12 +399,11 @@ message SysConfigResponse {
|
|||
|
||||
```go
|
||||
type SysConfig struct {
|
||||
etcdKV *etcd
|
||||
etcdPathPrefix string
|
||||
kv *kv.EtcdKV
|
||||
}
|
||||
|
||||
func (conf *SysConfig) InitFromFile(filePath string) (error)
|
||||
func (conf *SysConfig) GetByPrefix(keyPrefix string) ([]string, error)
|
||||
func (conf *SysConfig) GetByPrefix(keyPrefix string) (keys []string, values []string, err error)
|
||||
func (conf *SysConfig) Get(keys []string) ([]string, error)
|
||||
```
|
||||
|
||||
|
|
1
go.mod
1
go.mod
|
@ -7,7 +7,6 @@ require (
|
|||
github.com/apache/pulsar-client-go v0.1.1
|
||||
github.com/aws/aws-sdk-go v1.30.8
|
||||
github.com/coreos/etcd v3.3.25+incompatible // indirect
|
||||
github.com/cznic/mathutil v0.0.0-20181122101859-297441e03548
|
||||
github.com/frankban/quicktest v1.10.2 // indirect
|
||||
github.com/fsnotify/fsnotify v1.4.9 // indirect
|
||||
github.com/golang/groupcache v0.0.0-20200121045136-8c9f03a8e57e // indirect
|
||||
|
|
2
go.sum
2
go.sum
|
@ -65,7 +65,6 @@ github.com/coreos/pkg v0.0.0-20180928190104-399ea9e2e55f h1:lBNOc5arjvs8E5mO2tbp
|
|||
github.com/coreos/pkg v0.0.0-20180928190104-399ea9e2e55f/go.mod h1:E3G3o1h8I7cfcXa63jLwjI0eiQQMgzzUDFVpN/nH/eA=
|
||||
github.com/creack/pty v1.1.7/go.mod h1:lj5s0c3V2DBrqTV7llrYr5NG6My20zk30Fl46Y7DoTY=
|
||||
github.com/creack/pty v1.1.9/go.mod h1:oKZEueFk5CKHvIhNR5MUki03XCEU+Q6VDXinZuGJ33E=
|
||||
github.com/cznic/mathutil v0.0.0-20181122101859-297441e03548 h1:iwZdTE0PVqJCos1vaoKsclOGD3ADKpshg3SRtYBbwso=
|
||||
github.com/cznic/mathutil v0.0.0-20181122101859-297441e03548/go.mod h1:e6NPNENfs9mPDVNRekM7lKScauxd5kXTr1Mfyig6TDM=
|
||||
github.com/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38=
|
||||
github.com/davecgh/go-spew v1.1.1 h1:vj9j/u1bqnvCEfJOwUhtlOARqs3+rkHYY13jYWTU97c=
|
||||
|
@ -330,7 +329,6 @@ github.com/prometheus/procfs v0.0.8/go.mod h1:7Qr8sr6344vo1JqZ6HhLceV9o3AJ1Ff+Gx
|
|||
github.com/prometheus/procfs v0.1.3 h1:F0+tqvhOksq22sc6iCHF5WGlWjdwj92p0udFh1VFBS8=
|
||||
github.com/prometheus/procfs v0.1.3/go.mod h1:lV6e/gmhEcM9IjHGsFOCxxuZ+z1YqCvr4OA4YeYWdaU=
|
||||
github.com/prometheus/tsdb v0.7.1/go.mod h1:qhTCs0VvXwvX/y3TZrWD7rabWM+ijKTux40TwIPHuXU=
|
||||
github.com/remyoudompheng/bigfft v0.0.0-20170806203942-52369c62f446 h1:/NRJ5vAYoqz+7sG51ubIDHXeWO8DlTSrToPu6q11ziA=
|
||||
github.com/remyoudompheng/bigfft v0.0.0-20170806203942-52369c62f446/go.mod h1:uYEyJGbgTkfkS4+E/PavXkNJcbFIpEtjt2B0KDQ5+9M=
|
||||
github.com/rogpeppe/fastuuid v0.0.0-20150106093220-6724a57986af/go.mod h1:XWv6SoW27p1b0cqNHllgS5HIMJraePCO15w5zCzIWYg=
|
||||
github.com/rogpeppe/go-internal v1.3.0/go.mod h1:M8bDsm7K2OlrFYOpmOWEs/qY81heoFRclV5y23lUDJ4=
|
||||
|
|
|
@ -57,7 +57,7 @@ type segRequest struct {
|
|||
count uint32
|
||||
colName string
|
||||
partition string
|
||||
segInfo map[UniqueID]uint32
|
||||
segID UniqueID
|
||||
channelID int32
|
||||
}
|
||||
|
||||
|
|
|
@ -1,15 +1,11 @@
|
|||
package allocator
|
||||
|
||||
import (
|
||||
"container/list"
|
||||
"context"
|
||||
"fmt"
|
||||
"log"
|
||||
"sort"
|
||||
"time"
|
||||
|
||||
"github.com/cznic/mathutil"
|
||||
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
|
||||
|
||||
"github.com/zilliztech/milvus-distributed/internal/errors"
|
||||
|
@ -22,10 +18,7 @@ const (
|
|||
)
|
||||
|
||||
type assignInfo struct {
|
||||
collName string
|
||||
partitionTag string
|
||||
channelID int32
|
||||
segInfo map[UniqueID]uint32 // segmentID->count map
|
||||
internalpb.SegIDAssignment
|
||||
expireTime time.Time
|
||||
lastInsertTime time.Time
|
||||
}
|
||||
|
@ -39,16 +32,12 @@ func (info *assignInfo) IsActive(now time.Time) bool {
|
|||
}
|
||||
|
||||
func (info *assignInfo) IsEnough(count uint32) bool {
|
||||
total := uint32(0)
|
||||
for _, count := range info.segInfo {
|
||||
total += count
|
||||
}
|
||||
return total >= count
|
||||
return info.Count >= count
|
||||
}
|
||||
|
||||
type SegIDAssigner struct {
|
||||
Allocator
|
||||
assignInfos map[string]*list.List // collectionName -> *list.List
|
||||
assignInfos map[string][]*assignInfo // collectionName -> [] *assignInfo
|
||||
segReqs []*internalpb.SegIDRequest
|
||||
canDoReqs []request
|
||||
}
|
||||
|
@ -61,8 +50,11 @@ func NewSegIDAssigner(ctx context.Context, masterAddr string) (*SegIDAssigner, e
|
|||
cancel: cancel,
|
||||
masterAddress: masterAddr,
|
||||
countPerRPC: SegCountPerRPC,
|
||||
//toDoReqs: []request,
|
||||
},
|
||||
assignInfos: make(map[string]*list.List),
|
||||
assignInfos: make(map[string][]*assignInfo),
|
||||
//segReqs: make([]*internalpb.SegIDRequest, maxConcurrentRequests),
|
||||
//canDoReqs: make([]request, maxConcurrentRequests),
|
||||
}
|
||||
sa.tChan = &ticker{
|
||||
updateInterval: time.Second,
|
||||
|
@ -75,17 +67,16 @@ func NewSegIDAssigner(ctx context.Context, masterAddr string) (*SegIDAssigner, e
|
|||
|
||||
func (sa *SegIDAssigner) collectExpired() {
|
||||
now := time.Now()
|
||||
for _, info := range sa.assignInfos {
|
||||
for e := info.Front(); e != nil; e = e.Next() {
|
||||
assign := e.Value.(*assignInfo)
|
||||
for _, colInfos := range sa.assignInfos {
|
||||
for _, assign := range colInfos {
|
||||
if !assign.IsActive(now) || !assign.IsExpired(now) {
|
||||
continue
|
||||
}
|
||||
sa.segReqs = append(sa.segReqs, &internalpb.SegIDRequest{
|
||||
ChannelID: assign.channelID,
|
||||
ChannelID: assign.ChannelID,
|
||||
Count: sa.countPerRPC,
|
||||
CollName: assign.collName,
|
||||
PartitionTag: assign.partitionTag,
|
||||
CollName: assign.CollName,
|
||||
PartitionTag: assign.PartitionTag,
|
||||
})
|
||||
}
|
||||
}
|
||||
|
@ -97,6 +88,7 @@ func (sa *SegIDAssigner) checkToDoReqs() {
|
|||
}
|
||||
now := time.Now()
|
||||
for _, req := range sa.toDoReqs {
|
||||
fmt.Println("DDDDD????", req)
|
||||
segRequest := req.(*segRequest)
|
||||
assign := sa.getAssign(segRequest.colName, segRequest.partition, segRequest.channelID)
|
||||
if assign == nil || assign.IsExpired(now) || !assign.IsEnough(segRequest.count) {
|
||||
|
@ -110,36 +102,13 @@ func (sa *SegIDAssigner) checkToDoReqs() {
|
|||
}
|
||||
}
|
||||
|
||||
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 {
|
||||
assignInfos, ok := sa.assignInfos[colName]
|
||||
colInfos, ok := sa.assignInfos[colName]
|
||||
if !ok {
|
||||
return nil
|
||||
}
|
||||
|
||||
for e := assignInfos.Front(); e != nil; e = e.Next() {
|
||||
info := e.Value.(*assignInfo)
|
||||
if info.partitionTag != partition || info.channelID != channelID {
|
||||
for _, info := range colInfos {
|
||||
if info.PartitionTag != partition || info.ChannelID != channelID {
|
||||
continue
|
||||
}
|
||||
return info
|
||||
|
@ -182,26 +151,19 @@ func (sa *SegIDAssigner) syncSegments() {
|
|||
|
||||
now := time.Now()
|
||||
expiredTime := now.Add(time.Millisecond * time.Duration(resp.ExpireDuration))
|
||||
for _, info := range resp.PerChannelAssignment {
|
||||
sa.removeSegInfo(info.CollName, info.PartitionTag, info.ChannelID)
|
||||
}
|
||||
|
||||
for _, info := range resp.PerChannelAssignment {
|
||||
assign := sa.getAssign(info.CollName, info.PartitionTag, info.ChannelID)
|
||||
if assign == nil {
|
||||
colInfos := sa.assignInfos[info.CollName]
|
||||
segInfo := make(map[UniqueID]uint32)
|
||||
segInfo[info.SegID] = info.Count
|
||||
newAssign := &assignInfo{
|
||||
collName: info.CollName,
|
||||
partitionTag: info.PartitionTag,
|
||||
channelID: info.ChannelID,
|
||||
segInfo: segInfo,
|
||||
SegIDAssignment: *info,
|
||||
expireTime: expiredTime,
|
||||
lastInsertTime: now,
|
||||
}
|
||||
colInfos.PushBack(newAssign)
|
||||
colInfos = append(colInfos, newAssign)
|
||||
sa.assignInfos[info.CollName] = colInfos
|
||||
} else {
|
||||
assign.segInfo[info.SegID] = info.Count
|
||||
assign.SegIDAssignment = *info
|
||||
assign.expireTime = expiredTime
|
||||
assign.lastInsertTime = now
|
||||
}
|
||||
|
@ -219,38 +181,13 @@ func (sa *SegIDAssigner) processFunc(req request) error {
|
|||
if assign == nil {
|
||||
return errors.New("Failed to GetSegmentID")
|
||||
}
|
||||
|
||||
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
|
||||
segRequest.segID = assign.SegID
|
||||
assign.Count -= segRequest.count
|
||||
fmt.Println("process segmentID")
|
||||
return nil
|
||||
}
|
||||
|
||||
func (sa *SegIDAssigner) GetSegmentID(colName, partition string, channelID int32, count uint32) (map[UniqueID]uint32, error) {
|
||||
func (sa *SegIDAssigner) GetSegmentID(colName, partition string, channelID int32, count uint32) (UniqueID, error) {
|
||||
req := &segRequest{
|
||||
baseRequest: baseRequest{done: make(chan error), valid: false},
|
||||
colName: colName,
|
||||
|
@ -262,7 +199,7 @@ func (sa *SegIDAssigner) GetSegmentID(colName, partition string, channelID int32
|
|||
req.Wait()
|
||||
|
||||
if !req.IsValid() {
|
||||
return nil, errors.New("GetSegmentID Failed")
|
||||
return 0, errors.New("GetSegmentID Failed")
|
||||
}
|
||||
return req.segInfo, nil
|
||||
return req.segID, nil
|
||||
}
|
||||
|
|
|
@ -13,7 +13,7 @@ import (
|
|||
type Timestamp = typeutil.Timestamp
|
||||
|
||||
const (
|
||||
tsCountPerRPC = 2 << 15
|
||||
tsCountPerRPC = 2 << 18 * 10
|
||||
)
|
||||
|
||||
type TimestampAllocator struct {
|
||||
|
@ -37,7 +37,6 @@ func NewTimestampAllocator(ctx context.Context, masterAddr string) (*TimestampAl
|
|||
}
|
||||
a.Allocator.syncFunc = a.syncTs
|
||||
a.Allocator.processFunc = a.processFunc
|
||||
a.Allocator.checkFunc = a.checkFunc
|
||||
return a, nil
|
||||
}
|
||||
|
||||
|
|
|
@ -65,8 +65,7 @@ StructuredIndexFlat<T>::NotIn(const size_t n, const T* values) {
|
|||
if (!is_built_) {
|
||||
build();
|
||||
}
|
||||
TargetBitmapPtr bitset = std::make_unique<TargetBitmap>(data_.size());
|
||||
bitset->set();
|
||||
TargetBitmapPtr bitset = std::make_unique<TargetBitmap>(data_.size(), true);
|
||||
for (size_t i = 0; i < n; ++i) {
|
||||
for (const auto& index : data_) {
|
||||
if (index->a_ == *(values + i)) {
|
||||
|
|
|
@ -120,8 +120,7 @@ StructuredIndexSort<T>::NotIn(const size_t n, const T* values) {
|
|||
if (!is_built_) {
|
||||
build();
|
||||
}
|
||||
TargetBitmapPtr bitset = std::make_unique<TargetBitmap>(data_.size());
|
||||
bitset->set();
|
||||
TargetBitmapPtr bitset = std::make_unique<TargetBitmap>(data_.size(), true);
|
||||
for (size_t i = 0; i < n; ++i) {
|
||||
auto lb = std::lower_bound(data_.begin(), data_.end(), IndexStructure<T>(*(values + i)));
|
||||
auto ub = std::upper_bound(data_.begin(), data_.end(), IndexStructure<T>(*(values + i)));
|
||||
|
|
|
@ -78,6 +78,10 @@ class PartitionDescriptionDefaultTypeInternal {
|
|||
public:
|
||||
::PROTOBUF_NAMESPACE_ID::internal::ExplicitlyConstructed<PartitionDescription> _instance;
|
||||
} _PartitionDescription_default_instance_;
|
||||
class SysConfigResponseDefaultTypeInternal {
|
||||
public:
|
||||
::PROTOBUF_NAMESPACE_ID::internal::ExplicitlyConstructed<SysConfigResponse> _instance;
|
||||
} _SysConfigResponse_default_instance_;
|
||||
class ScoreDefaultTypeInternal {
|
||||
public:
|
||||
::PROTOBUF_NAMESPACE_ID::internal::ExplicitlyConstructed<Score> _instance;
|
||||
|
@ -334,7 +338,22 @@ static void InitDefaultsscc_info_StringResponse_service_5fmsg_2eproto() {
|
|||
{{ATOMIC_VAR_INIT(::PROTOBUF_NAMESPACE_ID::internal::SCCInfoBase::kUninitialized), 1, InitDefaultsscc_info_StringResponse_service_5fmsg_2eproto}, {
|
||||
&scc_info_Status_common_2eproto.base,}};
|
||||
|
||||
static ::PROTOBUF_NAMESPACE_ID::Metadata file_level_metadata_service_5fmsg_2eproto[16];
|
||||
static void InitDefaultsscc_info_SysConfigResponse_service_5fmsg_2eproto() {
|
||||
GOOGLE_PROTOBUF_VERIFY_VERSION;
|
||||
|
||||
{
|
||||
void* ptr = &::milvus::proto::service::_SysConfigResponse_default_instance_;
|
||||
new (ptr) ::milvus::proto::service::SysConfigResponse();
|
||||
::PROTOBUF_NAMESPACE_ID::internal::OnShutdownDestroyMessage(ptr);
|
||||
}
|
||||
::milvus::proto::service::SysConfigResponse::InitAsDefaultInstance();
|
||||
}
|
||||
|
||||
::PROTOBUF_NAMESPACE_ID::internal::SCCInfo<1> scc_info_SysConfigResponse_service_5fmsg_2eproto =
|
||||
{{ATOMIC_VAR_INIT(::PROTOBUF_NAMESPACE_ID::internal::SCCInfoBase::kUninitialized), 1, InitDefaultsscc_info_SysConfigResponse_service_5fmsg_2eproto}, {
|
||||
&scc_info_Status_common_2eproto.base,}};
|
||||
|
||||
static ::PROTOBUF_NAMESPACE_ID::Metadata file_level_metadata_service_5fmsg_2eproto[17];
|
||||
static const ::PROTOBUF_NAMESPACE_ID::EnumDescriptor* file_level_enum_descriptors_service_5fmsg_2eproto[1];
|
||||
static constexpr ::PROTOBUF_NAMESPACE_ID::ServiceDescriptor const** file_level_service_descriptors_service_5fmsg_2eproto = nullptr;
|
||||
|
||||
|
@ -437,6 +456,14 @@ const ::PROTOBUF_NAMESPACE_ID::uint32 TableStruct_service_5fmsg_2eproto::offsets
|
|||
PROTOBUF_FIELD_OFFSET(::milvus::proto::service::PartitionDescription, name_),
|
||||
PROTOBUF_FIELD_OFFSET(::milvus::proto::service::PartitionDescription, statistics_),
|
||||
~0u, // no _has_bits_
|
||||
PROTOBUF_FIELD_OFFSET(::milvus::proto::service::SysConfigResponse, _internal_metadata_),
|
||||
~0u, // no _extensions_
|
||||
~0u, // no _oneof_case_
|
||||
~0u, // no _weak_field_map_
|
||||
PROTOBUF_FIELD_OFFSET(::milvus::proto::service::SysConfigResponse, status_),
|
||||
PROTOBUF_FIELD_OFFSET(::milvus::proto::service::SysConfigResponse, keys_),
|
||||
PROTOBUF_FIELD_OFFSET(::milvus::proto::service::SysConfigResponse, values_),
|
||||
~0u, // no _has_bits_
|
||||
PROTOBUF_FIELD_OFFSET(::milvus::proto::service::Score, _internal_metadata_),
|
||||
~0u, // no _extensions_
|
||||
~0u, // no _oneof_case_
|
||||
|
@ -473,9 +500,10 @@ static const ::PROTOBUF_NAMESPACE_ID::internal::MigrationSchema schemas[] PROTOB
|
|||
{ 73, -1, sizeof(::milvus::proto::service::IntegerRangeResponse)},
|
||||
{ 81, -1, sizeof(::milvus::proto::service::CollectionDescription)},
|
||||
{ 89, -1, sizeof(::milvus::proto::service::PartitionDescription)},
|
||||
{ 97, -1, sizeof(::milvus::proto::service::Score)},
|
||||
{ 104, -1, sizeof(::milvus::proto::service::Hits)},
|
||||
{ 112, -1, sizeof(::milvus::proto::service::QueryResult)},
|
||||
{ 97, -1, sizeof(::milvus::proto::service::SysConfigResponse)},
|
||||
{ 105, -1, sizeof(::milvus::proto::service::Score)},
|
||||
{ 112, -1, sizeof(::milvus::proto::service::Hits)},
|
||||
{ 120, -1, sizeof(::milvus::proto::service::QueryResult)},
|
||||
};
|
||||
|
||||
static ::PROTOBUF_NAMESPACE_ID::Message const * const file_default_instances[] = {
|
||||
|
@ -492,6 +520,7 @@ static ::PROTOBUF_NAMESPACE_ID::Message const * const file_default_instances[] =
|
|||
reinterpret_cast<const ::PROTOBUF_NAMESPACE_ID::Message*>(&::milvus::proto::service::_IntegerRangeResponse_default_instance_),
|
||||
reinterpret_cast<const ::PROTOBUF_NAMESPACE_ID::Message*>(&::milvus::proto::service::_CollectionDescription_default_instance_),
|
||||
reinterpret_cast<const ::PROTOBUF_NAMESPACE_ID::Message*>(&::milvus::proto::service::_PartitionDescription_default_instance_),
|
||||
reinterpret_cast<const ::PROTOBUF_NAMESPACE_ID::Message*>(&::milvus::proto::service::_SysConfigResponse_default_instance_),
|
||||
reinterpret_cast<const ::PROTOBUF_NAMESPACE_ID::Message*>(&::milvus::proto::service::_Score_default_instance_),
|
||||
reinterpret_cast<const ::PROTOBUF_NAMESPACE_ID::Message*>(&::milvus::proto::service::_Hits_default_instance_),
|
||||
reinterpret_cast<const ::PROTOBUF_NAMESPACE_ID::Message*>(&::milvus::proto::service::_QueryResult_default_instance_),
|
||||
|
@ -531,23 +560,25 @@ const char descriptor_table_protodef_service_5fmsg_2eproto[] PROTOBUF_SECTION_VA
|
|||
"\006status\030\001 \001(\0132\033.milvus.proto.common.Stat"
|
||||
"us\0221\n\004name\030\002 \001(\0132#.milvus.proto.service."
|
||||
"PartitionName\0225\n\nstatistics\030\003 \003(\0132!.milv"
|
||||
"us.proto.common.KeyValuePair\"$\n\005Score\022\013\n"
|
||||
"\003tag\030\001 \001(\t\022\016\n\006values\030\002 \003(\002\"m\n\004Hits\022\013\n\003ID"
|
||||
"s\030\001 \003(\003\022+\n\010row_data\030\002 \003(\0132\031.milvus.proto"
|
||||
".common.Blob\022+\n\006scores\030\003 \003(\0132\033.milvus.pr"
|
||||
"oto.service.Score\"d\n\013QueryResult\022+\n\006stat"
|
||||
"us\030\001 \001(\0132\033.milvus.proto.common.Status\022(\n"
|
||||
"\004hits\030\002 \003(\0132\032.milvus.proto.service.Hits*"
|
||||
"@\n\017PlaceholderType\022\010\n\004NONE\020\000\022\021\n\rVECTOR_B"
|
||||
"INARY\020d\022\020\n\014VECTOR_FLOAT\020eBCZAgithub.com/"
|
||||
"zilliztech/milvus-distributed/internal/p"
|
||||
"roto/servicepbb\006proto3"
|
||||
"us.proto.common.KeyValuePair\"^\n\021SysConfi"
|
||||
"gResponse\022+\n\006status\030\001 \001(\0132\033.milvus.proto"
|
||||
".common.Status\022\014\n\004keys\030\002 \003(\t\022\016\n\006values\030\003"
|
||||
" \003(\t\"$\n\005Score\022\013\n\003tag\030\001 \001(\t\022\016\n\006values\030\002 \003"
|
||||
"(\002\"m\n\004Hits\022\013\n\003IDs\030\001 \003(\003\022+\n\010row_data\030\002 \003("
|
||||
"\0132\031.milvus.proto.common.Blob\022+\n\006scores\030\003"
|
||||
" \003(\0132\033.milvus.proto.service.Score\"d\n\013Que"
|
||||
"ryResult\022+\n\006status\030\001 \001(\0132\033.milvus.proto."
|
||||
"common.Status\022(\n\004hits\030\002 \003(\0132\032.milvus.pro"
|
||||
"to.service.Hits*@\n\017PlaceholderType\022\010\n\004NO"
|
||||
"NE\020\000\022\021\n\rVECTOR_BINARY\020d\022\020\n\014VECTOR_FLOAT\020"
|
||||
"eBCZAgithub.com/zilliztech/milvus-distri"
|
||||
"buted/internal/proto/servicepbb\006proto3"
|
||||
;
|
||||
static const ::PROTOBUF_NAMESPACE_ID::internal::DescriptorTable*const descriptor_table_service_5fmsg_2eproto_deps[2] = {
|
||||
&::descriptor_table_common_2eproto,
|
||||
&::descriptor_table_schema_2eproto,
|
||||
};
|
||||
static ::PROTOBUF_NAMESPACE_ID::internal::SCCInfoBase*const descriptor_table_service_5fmsg_2eproto_sccs[16] = {
|
||||
static ::PROTOBUF_NAMESPACE_ID::internal::SCCInfoBase*const descriptor_table_service_5fmsg_2eproto_sccs[17] = {
|
||||
&scc_info_BoolResponse_service_5fmsg_2eproto.base,
|
||||
&scc_info_CollectionDescription_service_5fmsg_2eproto.base,
|
||||
&scc_info_CollectionName_service_5fmsg_2eproto.base,
|
||||
|
@ -564,14 +595,15 @@ static ::PROTOBUF_NAMESPACE_ID::internal::SCCInfoBase*const descriptor_table_ser
|
|||
&scc_info_Score_service_5fmsg_2eproto.base,
|
||||
&scc_info_StringListResponse_service_5fmsg_2eproto.base,
|
||||
&scc_info_StringResponse_service_5fmsg_2eproto.base,
|
||||
&scc_info_SysConfigResponse_service_5fmsg_2eproto.base,
|
||||
};
|
||||
static ::PROTOBUF_NAMESPACE_ID::internal::once_flag descriptor_table_service_5fmsg_2eproto_once;
|
||||
static bool descriptor_table_service_5fmsg_2eproto_initialized = false;
|
||||
const ::PROTOBUF_NAMESPACE_ID::internal::DescriptorTable descriptor_table_service_5fmsg_2eproto = {
|
||||
&descriptor_table_service_5fmsg_2eproto_initialized, descriptor_table_protodef_service_5fmsg_2eproto, "service_msg.proto", 1742,
|
||||
&descriptor_table_service_5fmsg_2eproto_once, descriptor_table_service_5fmsg_2eproto_sccs, descriptor_table_service_5fmsg_2eproto_deps, 16, 2,
|
||||
&descriptor_table_service_5fmsg_2eproto_initialized, descriptor_table_protodef_service_5fmsg_2eproto, "service_msg.proto", 1838,
|
||||
&descriptor_table_service_5fmsg_2eproto_once, descriptor_table_service_5fmsg_2eproto_sccs, descriptor_table_service_5fmsg_2eproto_deps, 17, 2,
|
||||
schemas, file_default_instances, TableStruct_service_5fmsg_2eproto::offsets,
|
||||
file_level_metadata_service_5fmsg_2eproto, 16, file_level_enum_descriptors_service_5fmsg_2eproto, file_level_service_descriptors_service_5fmsg_2eproto,
|
||||
file_level_metadata_service_5fmsg_2eproto, 17, file_level_enum_descriptors_service_5fmsg_2eproto, file_level_service_descriptors_service_5fmsg_2eproto,
|
||||
};
|
||||
|
||||
// Force running AddDescriptors() at dynamic initialization time.
|
||||
|
@ -5195,6 +5227,398 @@ void PartitionDescription::InternalSwap(PartitionDescription* other) {
|
|||
}
|
||||
|
||||
|
||||
// ===================================================================
|
||||
|
||||
void SysConfigResponse::InitAsDefaultInstance() {
|
||||
::milvus::proto::service::_SysConfigResponse_default_instance_._instance.get_mutable()->status_ = const_cast< ::milvus::proto::common::Status*>(
|
||||
::milvus::proto::common::Status::internal_default_instance());
|
||||
}
|
||||
class SysConfigResponse::_Internal {
|
||||
public:
|
||||
static const ::milvus::proto::common::Status& status(const SysConfigResponse* msg);
|
||||
};
|
||||
|
||||
const ::milvus::proto::common::Status&
|
||||
SysConfigResponse::_Internal::status(const SysConfigResponse* msg) {
|
||||
return *msg->status_;
|
||||
}
|
||||
void SysConfigResponse::clear_status() {
|
||||
if (GetArenaNoVirtual() == nullptr && status_ != nullptr) {
|
||||
delete status_;
|
||||
}
|
||||
status_ = nullptr;
|
||||
}
|
||||
SysConfigResponse::SysConfigResponse()
|
||||
: ::PROTOBUF_NAMESPACE_ID::Message(), _internal_metadata_(nullptr) {
|
||||
SharedCtor();
|
||||
// @@protoc_insertion_point(constructor:milvus.proto.service.SysConfigResponse)
|
||||
}
|
||||
SysConfigResponse::SysConfigResponse(const SysConfigResponse& from)
|
||||
: ::PROTOBUF_NAMESPACE_ID::Message(),
|
||||
_internal_metadata_(nullptr),
|
||||
keys_(from.keys_),
|
||||
values_(from.values_) {
|
||||
_internal_metadata_.MergeFrom(from._internal_metadata_);
|
||||
if (from.has_status()) {
|
||||
status_ = new ::milvus::proto::common::Status(*from.status_);
|
||||
} else {
|
||||
status_ = nullptr;
|
||||
}
|
||||
// @@protoc_insertion_point(copy_constructor:milvus.proto.service.SysConfigResponse)
|
||||
}
|
||||
|
||||
void SysConfigResponse::SharedCtor() {
|
||||
::PROTOBUF_NAMESPACE_ID::internal::InitSCC(&scc_info_SysConfigResponse_service_5fmsg_2eproto.base);
|
||||
status_ = nullptr;
|
||||
}
|
||||
|
||||
SysConfigResponse::~SysConfigResponse() {
|
||||
// @@protoc_insertion_point(destructor:milvus.proto.service.SysConfigResponse)
|
||||
SharedDtor();
|
||||
}
|
||||
|
||||
void SysConfigResponse::SharedDtor() {
|
||||
if (this != internal_default_instance()) delete status_;
|
||||
}
|
||||
|
||||
void SysConfigResponse::SetCachedSize(int size) const {
|
||||
_cached_size_.Set(size);
|
||||
}
|
||||
const SysConfigResponse& SysConfigResponse::default_instance() {
|
||||
::PROTOBUF_NAMESPACE_ID::internal::InitSCC(&::scc_info_SysConfigResponse_service_5fmsg_2eproto.base);
|
||||
return *internal_default_instance();
|
||||
}
|
||||
|
||||
|
||||
void SysConfigResponse::Clear() {
|
||||
// @@protoc_insertion_point(message_clear_start:milvus.proto.service.SysConfigResponse)
|
||||
::PROTOBUF_NAMESPACE_ID::uint32 cached_has_bits = 0;
|
||||
// Prevent compiler warnings about cached_has_bits being unused
|
||||
(void) cached_has_bits;
|
||||
|
||||
keys_.Clear();
|
||||
values_.Clear();
|
||||
if (GetArenaNoVirtual() == nullptr && status_ != nullptr) {
|
||||
delete status_;
|
||||
}
|
||||
status_ = nullptr;
|
||||
_internal_metadata_.Clear();
|
||||
}
|
||||
|
||||
#if GOOGLE_PROTOBUF_ENABLE_EXPERIMENTAL_PARSER
|
||||
const char* SysConfigResponse::_InternalParse(const char* ptr, ::PROTOBUF_NAMESPACE_ID::internal::ParseContext* ctx) {
|
||||
#define CHK_(x) if (PROTOBUF_PREDICT_FALSE(!(x))) goto failure
|
||||
while (!ctx->Done(&ptr)) {
|
||||
::PROTOBUF_NAMESPACE_ID::uint32 tag;
|
||||
ptr = ::PROTOBUF_NAMESPACE_ID::internal::ReadTag(ptr, &tag);
|
||||
CHK_(ptr);
|
||||
switch (tag >> 3) {
|
||||
// .milvus.proto.common.Status status = 1;
|
||||
case 1:
|
||||
if (PROTOBUF_PREDICT_TRUE(static_cast<::PROTOBUF_NAMESPACE_ID::uint8>(tag) == 10)) {
|
||||
ptr = ctx->ParseMessage(mutable_status(), ptr);
|
||||
CHK_(ptr);
|
||||
} else goto handle_unusual;
|
||||
continue;
|
||||
// repeated string keys = 2;
|
||||
case 2:
|
||||
if (PROTOBUF_PREDICT_TRUE(static_cast<::PROTOBUF_NAMESPACE_ID::uint8>(tag) == 18)) {
|
||||
ptr -= 1;
|
||||
do {
|
||||
ptr += 1;
|
||||
ptr = ::PROTOBUF_NAMESPACE_ID::internal::InlineGreedyStringParserUTF8(add_keys(), ptr, ctx, "milvus.proto.service.SysConfigResponse.keys");
|
||||
CHK_(ptr);
|
||||
if (!ctx->DataAvailable(ptr)) break;
|
||||
} while (::PROTOBUF_NAMESPACE_ID::internal::UnalignedLoad<::PROTOBUF_NAMESPACE_ID::uint8>(ptr) == 18);
|
||||
} else goto handle_unusual;
|
||||
continue;
|
||||
// repeated string values = 3;
|
||||
case 3:
|
||||
if (PROTOBUF_PREDICT_TRUE(static_cast<::PROTOBUF_NAMESPACE_ID::uint8>(tag) == 26)) {
|
||||
ptr -= 1;
|
||||
do {
|
||||
ptr += 1;
|
||||
ptr = ::PROTOBUF_NAMESPACE_ID::internal::InlineGreedyStringParserUTF8(add_values(), ptr, ctx, "milvus.proto.service.SysConfigResponse.values");
|
||||
CHK_(ptr);
|
||||
if (!ctx->DataAvailable(ptr)) break;
|
||||
} while (::PROTOBUF_NAMESPACE_ID::internal::UnalignedLoad<::PROTOBUF_NAMESPACE_ID::uint8>(ptr) == 26);
|
||||
} else goto handle_unusual;
|
||||
continue;
|
||||
default: {
|
||||
handle_unusual:
|
||||
if ((tag & 7) == 4 || tag == 0) {
|
||||
ctx->SetLastTag(tag);
|
||||
goto success;
|
||||
}
|
||||
ptr = UnknownFieldParse(tag, &_internal_metadata_, ptr, ctx);
|
||||
CHK_(ptr != nullptr);
|
||||
continue;
|
||||
}
|
||||
} // switch
|
||||
} // while
|
||||
success:
|
||||
return ptr;
|
||||
failure:
|
||||
ptr = nullptr;
|
||||
goto success;
|
||||
#undef CHK_
|
||||
}
|
||||
#else // GOOGLE_PROTOBUF_ENABLE_EXPERIMENTAL_PARSER
|
||||
bool SysConfigResponse::MergePartialFromCodedStream(
|
||||
::PROTOBUF_NAMESPACE_ID::io::CodedInputStream* input) {
|
||||
#define DO_(EXPRESSION) if (!PROTOBUF_PREDICT_TRUE(EXPRESSION)) goto failure
|
||||
::PROTOBUF_NAMESPACE_ID::uint32 tag;
|
||||
// @@protoc_insertion_point(parse_start:milvus.proto.service.SysConfigResponse)
|
||||
for (;;) {
|
||||
::std::pair<::PROTOBUF_NAMESPACE_ID::uint32, bool> p = input->ReadTagWithCutoffNoLastTag(127u);
|
||||
tag = p.first;
|
||||
if (!p.second) goto handle_unusual;
|
||||
switch (::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::GetTagFieldNumber(tag)) {
|
||||
// .milvus.proto.common.Status status = 1;
|
||||
case 1: {
|
||||
if (static_cast< ::PROTOBUF_NAMESPACE_ID::uint8>(tag) == (10 & 0xFF)) {
|
||||
DO_(::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::ReadMessage(
|
||||
input, mutable_status()));
|
||||
} else {
|
||||
goto handle_unusual;
|
||||
}
|
||||
break;
|
||||
}
|
||||
|
||||
// repeated string keys = 2;
|
||||
case 2: {
|
||||
if (static_cast< ::PROTOBUF_NAMESPACE_ID::uint8>(tag) == (18 & 0xFF)) {
|
||||
DO_(::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::ReadString(
|
||||
input, this->add_keys()));
|
||||
DO_(::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::VerifyUtf8String(
|
||||
this->keys(this->keys_size() - 1).data(),
|
||||
static_cast<int>(this->keys(this->keys_size() - 1).length()),
|
||||
::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::PARSE,
|
||||
"milvus.proto.service.SysConfigResponse.keys"));
|
||||
} else {
|
||||
goto handle_unusual;
|
||||
}
|
||||
break;
|
||||
}
|
||||
|
||||
// repeated string values = 3;
|
||||
case 3: {
|
||||
if (static_cast< ::PROTOBUF_NAMESPACE_ID::uint8>(tag) == (26 & 0xFF)) {
|
||||
DO_(::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::ReadString(
|
||||
input, this->add_values()));
|
||||
DO_(::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::VerifyUtf8String(
|
||||
this->values(this->values_size() - 1).data(),
|
||||
static_cast<int>(this->values(this->values_size() - 1).length()),
|
||||
::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::PARSE,
|
||||
"milvus.proto.service.SysConfigResponse.values"));
|
||||
} else {
|
||||
goto handle_unusual;
|
||||
}
|
||||
break;
|
||||
}
|
||||
|
||||
default: {
|
||||
handle_unusual:
|
||||
if (tag == 0) {
|
||||
goto success;
|
||||
}
|
||||
DO_(::PROTOBUF_NAMESPACE_ID::internal::WireFormat::SkipField(
|
||||
input, tag, _internal_metadata_.mutable_unknown_fields()));
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
success:
|
||||
// @@protoc_insertion_point(parse_success:milvus.proto.service.SysConfigResponse)
|
||||
return true;
|
||||
failure:
|
||||
// @@protoc_insertion_point(parse_failure:milvus.proto.service.SysConfigResponse)
|
||||
return false;
|
||||
#undef DO_
|
||||
}
|
||||
#endif // GOOGLE_PROTOBUF_ENABLE_EXPERIMENTAL_PARSER
|
||||
|
||||
void SysConfigResponse::SerializeWithCachedSizes(
|
||||
::PROTOBUF_NAMESPACE_ID::io::CodedOutputStream* output) const {
|
||||
// @@protoc_insertion_point(serialize_start:milvus.proto.service.SysConfigResponse)
|
||||
::PROTOBUF_NAMESPACE_ID::uint32 cached_has_bits = 0;
|
||||
(void) cached_has_bits;
|
||||
|
||||
// .milvus.proto.common.Status status = 1;
|
||||
if (this->has_status()) {
|
||||
::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::WriteMessageMaybeToArray(
|
||||
1, _Internal::status(this), output);
|
||||
}
|
||||
|
||||
// repeated string keys = 2;
|
||||
for (int i = 0, n = this->keys_size(); i < n; i++) {
|
||||
::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::VerifyUtf8String(
|
||||
this->keys(i).data(), static_cast<int>(this->keys(i).length()),
|
||||
::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::SERIALIZE,
|
||||
"milvus.proto.service.SysConfigResponse.keys");
|
||||
::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::WriteString(
|
||||
2, this->keys(i), output);
|
||||
}
|
||||
|
||||
// repeated string values = 3;
|
||||
for (int i = 0, n = this->values_size(); i < n; i++) {
|
||||
::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::VerifyUtf8String(
|
||||
this->values(i).data(), static_cast<int>(this->values(i).length()),
|
||||
::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::SERIALIZE,
|
||||
"milvus.proto.service.SysConfigResponse.values");
|
||||
::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::WriteString(
|
||||
3, this->values(i), output);
|
||||
}
|
||||
|
||||
if (_internal_metadata_.have_unknown_fields()) {
|
||||
::PROTOBUF_NAMESPACE_ID::internal::WireFormat::SerializeUnknownFields(
|
||||
_internal_metadata_.unknown_fields(), output);
|
||||
}
|
||||
// @@protoc_insertion_point(serialize_end:milvus.proto.service.SysConfigResponse)
|
||||
}
|
||||
|
||||
::PROTOBUF_NAMESPACE_ID::uint8* SysConfigResponse::InternalSerializeWithCachedSizesToArray(
|
||||
::PROTOBUF_NAMESPACE_ID::uint8* target) const {
|
||||
// @@protoc_insertion_point(serialize_to_array_start:milvus.proto.service.SysConfigResponse)
|
||||
::PROTOBUF_NAMESPACE_ID::uint32 cached_has_bits = 0;
|
||||
(void) cached_has_bits;
|
||||
|
||||
// .milvus.proto.common.Status status = 1;
|
||||
if (this->has_status()) {
|
||||
target = ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::
|
||||
InternalWriteMessageToArray(
|
||||
1, _Internal::status(this), target);
|
||||
}
|
||||
|
||||
// repeated string keys = 2;
|
||||
for (int i = 0, n = this->keys_size(); i < n; i++) {
|
||||
::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::VerifyUtf8String(
|
||||
this->keys(i).data(), static_cast<int>(this->keys(i).length()),
|
||||
::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::SERIALIZE,
|
||||
"milvus.proto.service.SysConfigResponse.keys");
|
||||
target = ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::
|
||||
WriteStringToArray(2, this->keys(i), target);
|
||||
}
|
||||
|
||||
// repeated string values = 3;
|
||||
for (int i = 0, n = this->values_size(); i < n; i++) {
|
||||
::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::VerifyUtf8String(
|
||||
this->values(i).data(), static_cast<int>(this->values(i).length()),
|
||||
::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::SERIALIZE,
|
||||
"milvus.proto.service.SysConfigResponse.values");
|
||||
target = ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::
|
||||
WriteStringToArray(3, this->values(i), target);
|
||||
}
|
||||
|
||||
if (_internal_metadata_.have_unknown_fields()) {
|
||||
target = ::PROTOBUF_NAMESPACE_ID::internal::WireFormat::SerializeUnknownFieldsToArray(
|
||||
_internal_metadata_.unknown_fields(), target);
|
||||
}
|
||||
// @@protoc_insertion_point(serialize_to_array_end:milvus.proto.service.SysConfigResponse)
|
||||
return target;
|
||||
}
|
||||
|
||||
size_t SysConfigResponse::ByteSizeLong() const {
|
||||
// @@protoc_insertion_point(message_byte_size_start:milvus.proto.service.SysConfigResponse)
|
||||
size_t total_size = 0;
|
||||
|
||||
if (_internal_metadata_.have_unknown_fields()) {
|
||||
total_size +=
|
||||
::PROTOBUF_NAMESPACE_ID::internal::WireFormat::ComputeUnknownFieldsSize(
|
||||
_internal_metadata_.unknown_fields());
|
||||
}
|
||||
::PROTOBUF_NAMESPACE_ID::uint32 cached_has_bits = 0;
|
||||
// Prevent compiler warnings about cached_has_bits being unused
|
||||
(void) cached_has_bits;
|
||||
|
||||
// repeated string keys = 2;
|
||||
total_size += 1 *
|
||||
::PROTOBUF_NAMESPACE_ID::internal::FromIntSize(this->keys_size());
|
||||
for (int i = 0, n = this->keys_size(); i < n; i++) {
|
||||
total_size += ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::StringSize(
|
||||
this->keys(i));
|
||||
}
|
||||
|
||||
// repeated string values = 3;
|
||||
total_size += 1 *
|
||||
::PROTOBUF_NAMESPACE_ID::internal::FromIntSize(this->values_size());
|
||||
for (int i = 0, n = this->values_size(); i < n; i++) {
|
||||
total_size += ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::StringSize(
|
||||
this->values(i));
|
||||
}
|
||||
|
||||
// .milvus.proto.common.Status status = 1;
|
||||
if (this->has_status()) {
|
||||
total_size += 1 +
|
||||
::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::MessageSize(
|
||||
*status_);
|
||||
}
|
||||
|
||||
int cached_size = ::PROTOBUF_NAMESPACE_ID::internal::ToCachedSize(total_size);
|
||||
SetCachedSize(cached_size);
|
||||
return total_size;
|
||||
}
|
||||
|
||||
void SysConfigResponse::MergeFrom(const ::PROTOBUF_NAMESPACE_ID::Message& from) {
|
||||
// @@protoc_insertion_point(generalized_merge_from_start:milvus.proto.service.SysConfigResponse)
|
||||
GOOGLE_DCHECK_NE(&from, this);
|
||||
const SysConfigResponse* source =
|
||||
::PROTOBUF_NAMESPACE_ID::DynamicCastToGenerated<SysConfigResponse>(
|
||||
&from);
|
||||
if (source == nullptr) {
|
||||
// @@protoc_insertion_point(generalized_merge_from_cast_fail:milvus.proto.service.SysConfigResponse)
|
||||
::PROTOBUF_NAMESPACE_ID::internal::ReflectionOps::Merge(from, this);
|
||||
} else {
|
||||
// @@protoc_insertion_point(generalized_merge_from_cast_success:milvus.proto.service.SysConfigResponse)
|
||||
MergeFrom(*source);
|
||||
}
|
||||
}
|
||||
|
||||
void SysConfigResponse::MergeFrom(const SysConfigResponse& from) {
|
||||
// @@protoc_insertion_point(class_specific_merge_from_start:milvus.proto.service.SysConfigResponse)
|
||||
GOOGLE_DCHECK_NE(&from, this);
|
||||
_internal_metadata_.MergeFrom(from._internal_metadata_);
|
||||
::PROTOBUF_NAMESPACE_ID::uint32 cached_has_bits = 0;
|
||||
(void) cached_has_bits;
|
||||
|
||||
keys_.MergeFrom(from.keys_);
|
||||
values_.MergeFrom(from.values_);
|
||||
if (from.has_status()) {
|
||||
mutable_status()->::milvus::proto::common::Status::MergeFrom(from.status());
|
||||
}
|
||||
}
|
||||
|
||||
void SysConfigResponse::CopyFrom(const ::PROTOBUF_NAMESPACE_ID::Message& from) {
|
||||
// @@protoc_insertion_point(generalized_copy_from_start:milvus.proto.service.SysConfigResponse)
|
||||
if (&from == this) return;
|
||||
Clear();
|
||||
MergeFrom(from);
|
||||
}
|
||||
|
||||
void SysConfigResponse::CopyFrom(const SysConfigResponse& from) {
|
||||
// @@protoc_insertion_point(class_specific_copy_from_start:milvus.proto.service.SysConfigResponse)
|
||||
if (&from == this) return;
|
||||
Clear();
|
||||
MergeFrom(from);
|
||||
}
|
||||
|
||||
bool SysConfigResponse::IsInitialized() const {
|
||||
return true;
|
||||
}
|
||||
|
||||
void SysConfigResponse::InternalSwap(SysConfigResponse* other) {
|
||||
using std::swap;
|
||||
_internal_metadata_.Swap(&other->_internal_metadata_);
|
||||
keys_.InternalSwap(CastToBase(&other->keys_));
|
||||
values_.InternalSwap(CastToBase(&other->values_));
|
||||
swap(status_, other->status_);
|
||||
}
|
||||
|
||||
::PROTOBUF_NAMESPACE_ID::Metadata SysConfigResponse::GetMetadata() const {
|
||||
return GetMetadataStatic();
|
||||
}
|
||||
|
||||
|
||||
// ===================================================================
|
||||
|
||||
void Score::InitAsDefaultInstance() {
|
||||
|
@ -6290,6 +6714,9 @@ template<> PROTOBUF_NOINLINE ::milvus::proto::service::CollectionDescription* Ar
|
|||
template<> PROTOBUF_NOINLINE ::milvus::proto::service::PartitionDescription* Arena::CreateMaybeMessage< ::milvus::proto::service::PartitionDescription >(Arena* arena) {
|
||||
return Arena::CreateInternal< ::milvus::proto::service::PartitionDescription >(arena);
|
||||
}
|
||||
template<> PROTOBUF_NOINLINE ::milvus::proto::service::SysConfigResponse* Arena::CreateMaybeMessage< ::milvus::proto::service::SysConfigResponse >(Arena* arena) {
|
||||
return Arena::CreateInternal< ::milvus::proto::service::SysConfigResponse >(arena);
|
||||
}
|
||||
template<> PROTOBUF_NOINLINE ::milvus::proto::service::Score* Arena::CreateMaybeMessage< ::milvus::proto::service::Score >(Arena* arena) {
|
||||
return Arena::CreateInternal< ::milvus::proto::service::Score >(arena);
|
||||
}
|
||||
|
|
|
@ -50,7 +50,7 @@ struct TableStruct_service_5fmsg_2eproto {
|
|||
PROTOBUF_SECTION_VARIABLE(protodesc_cold);
|
||||
static const ::PROTOBUF_NAMESPACE_ID::internal::AuxillaryParseTableField aux[]
|
||||
PROTOBUF_SECTION_VARIABLE(protodesc_cold);
|
||||
static const ::PROTOBUF_NAMESPACE_ID::internal::ParseTable schema[16]
|
||||
static const ::PROTOBUF_NAMESPACE_ID::internal::ParseTable schema[17]
|
||||
PROTOBUF_SECTION_VARIABLE(protodesc_cold);
|
||||
static const ::PROTOBUF_NAMESPACE_ID::internal::FieldMetadata field_metadata[];
|
||||
static const ::PROTOBUF_NAMESPACE_ID::internal::SerializationTable serialization_table[];
|
||||
|
@ -108,6 +108,9 @@ extern StringListResponseDefaultTypeInternal _StringListResponse_default_instanc
|
|||
class StringResponse;
|
||||
class StringResponseDefaultTypeInternal;
|
||||
extern StringResponseDefaultTypeInternal _StringResponse_default_instance_;
|
||||
class SysConfigResponse;
|
||||
class SysConfigResponseDefaultTypeInternal;
|
||||
extern SysConfigResponseDefaultTypeInternal _SysConfigResponse_default_instance_;
|
||||
} // namespace service
|
||||
} // namespace proto
|
||||
} // namespace milvus
|
||||
|
@ -128,6 +131,7 @@ template<> ::milvus::proto::service::RowBatch* Arena::CreateMaybeMessage<::milvu
|
|||
template<> ::milvus::proto::service::Score* Arena::CreateMaybeMessage<::milvus::proto::service::Score>(Arena*);
|
||||
template<> ::milvus::proto::service::StringListResponse* Arena::CreateMaybeMessage<::milvus::proto::service::StringListResponse>(Arena*);
|
||||
template<> ::milvus::proto::service::StringResponse* Arena::CreateMaybeMessage<::milvus::proto::service::StringResponse>(Arena*);
|
||||
template<> ::milvus::proto::service::SysConfigResponse* Arena::CreateMaybeMessage<::milvus::proto::service::SysConfigResponse>(Arena*);
|
||||
PROTOBUF_NAMESPACE_CLOSE
|
||||
namespace milvus {
|
||||
namespace proto {
|
||||
|
@ -2158,6 +2162,178 @@ class PartitionDescription :
|
|||
};
|
||||
// -------------------------------------------------------------------
|
||||
|
||||
class SysConfigResponse :
|
||||
public ::PROTOBUF_NAMESPACE_ID::Message /* @@protoc_insertion_point(class_definition:milvus.proto.service.SysConfigResponse) */ {
|
||||
public:
|
||||
SysConfigResponse();
|
||||
virtual ~SysConfigResponse();
|
||||
|
||||
SysConfigResponse(const SysConfigResponse& from);
|
||||
SysConfigResponse(SysConfigResponse&& from) noexcept
|
||||
: SysConfigResponse() {
|
||||
*this = ::std::move(from);
|
||||
}
|
||||
|
||||
inline SysConfigResponse& operator=(const SysConfigResponse& from) {
|
||||
CopyFrom(from);
|
||||
return *this;
|
||||
}
|
||||
inline SysConfigResponse& operator=(SysConfigResponse&& from) noexcept {
|
||||
if (GetArenaNoVirtual() == from.GetArenaNoVirtual()) {
|
||||
if (this != &from) InternalSwap(&from);
|
||||
} else {
|
||||
CopyFrom(from);
|
||||
}
|
||||
return *this;
|
||||
}
|
||||
|
||||
static const ::PROTOBUF_NAMESPACE_ID::Descriptor* descriptor() {
|
||||
return GetDescriptor();
|
||||
}
|
||||
static const ::PROTOBUF_NAMESPACE_ID::Descriptor* GetDescriptor() {
|
||||
return GetMetadataStatic().descriptor;
|
||||
}
|
||||
static const ::PROTOBUF_NAMESPACE_ID::Reflection* GetReflection() {
|
||||
return GetMetadataStatic().reflection;
|
||||
}
|
||||
static const SysConfigResponse& default_instance();
|
||||
|
||||
static void InitAsDefaultInstance(); // FOR INTERNAL USE ONLY
|
||||
static inline const SysConfigResponse* internal_default_instance() {
|
||||
return reinterpret_cast<const SysConfigResponse*>(
|
||||
&_SysConfigResponse_default_instance_);
|
||||
}
|
||||
static constexpr int kIndexInFileMessages =
|
||||
13;
|
||||
|
||||
friend void swap(SysConfigResponse& a, SysConfigResponse& b) {
|
||||
a.Swap(&b);
|
||||
}
|
||||
inline void Swap(SysConfigResponse* other) {
|
||||
if (other == this) return;
|
||||
InternalSwap(other);
|
||||
}
|
||||
|
||||
// implements Message ----------------------------------------------
|
||||
|
||||
inline SysConfigResponse* New() const final {
|
||||
return CreateMaybeMessage<SysConfigResponse>(nullptr);
|
||||
}
|
||||
|
||||
SysConfigResponse* New(::PROTOBUF_NAMESPACE_ID::Arena* arena) const final {
|
||||
return CreateMaybeMessage<SysConfigResponse>(arena);
|
||||
}
|
||||
void CopyFrom(const ::PROTOBUF_NAMESPACE_ID::Message& from) final;
|
||||
void MergeFrom(const ::PROTOBUF_NAMESPACE_ID::Message& from) final;
|
||||
void CopyFrom(const SysConfigResponse& from);
|
||||
void MergeFrom(const SysConfigResponse& from);
|
||||
PROTOBUF_ATTRIBUTE_REINITIALIZES void Clear() final;
|
||||
bool IsInitialized() const final;
|
||||
|
||||
size_t ByteSizeLong() const final;
|
||||
#if GOOGLE_PROTOBUF_ENABLE_EXPERIMENTAL_PARSER
|
||||
const char* _InternalParse(const char* ptr, ::PROTOBUF_NAMESPACE_ID::internal::ParseContext* ctx) final;
|
||||
#else
|
||||
bool MergePartialFromCodedStream(
|
||||
::PROTOBUF_NAMESPACE_ID::io::CodedInputStream* input) final;
|
||||
#endif // GOOGLE_PROTOBUF_ENABLE_EXPERIMENTAL_PARSER
|
||||
void SerializeWithCachedSizes(
|
||||
::PROTOBUF_NAMESPACE_ID::io::CodedOutputStream* output) const final;
|
||||
::PROTOBUF_NAMESPACE_ID::uint8* InternalSerializeWithCachedSizesToArray(
|
||||
::PROTOBUF_NAMESPACE_ID::uint8* target) const final;
|
||||
int GetCachedSize() const final { return _cached_size_.Get(); }
|
||||
|
||||
private:
|
||||
inline void SharedCtor();
|
||||
inline void SharedDtor();
|
||||
void SetCachedSize(int size) const final;
|
||||
void InternalSwap(SysConfigResponse* other);
|
||||
friend class ::PROTOBUF_NAMESPACE_ID::internal::AnyMetadata;
|
||||
static ::PROTOBUF_NAMESPACE_ID::StringPiece FullMessageName() {
|
||||
return "milvus.proto.service.SysConfigResponse";
|
||||
}
|
||||
private:
|
||||
inline ::PROTOBUF_NAMESPACE_ID::Arena* GetArenaNoVirtual() const {
|
||||
return nullptr;
|
||||
}
|
||||
inline void* MaybeArenaPtr() const {
|
||||
return nullptr;
|
||||
}
|
||||
public:
|
||||
|
||||
::PROTOBUF_NAMESPACE_ID::Metadata GetMetadata() const final;
|
||||
private:
|
||||
static ::PROTOBUF_NAMESPACE_ID::Metadata GetMetadataStatic() {
|
||||
::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors(&::descriptor_table_service_5fmsg_2eproto);
|
||||
return ::descriptor_table_service_5fmsg_2eproto.file_level_metadata[kIndexInFileMessages];
|
||||
}
|
||||
|
||||
public:
|
||||
|
||||
// nested types ----------------------------------------------------
|
||||
|
||||
// accessors -------------------------------------------------------
|
||||
|
||||
enum : int {
|
||||
kKeysFieldNumber = 2,
|
||||
kValuesFieldNumber = 3,
|
||||
kStatusFieldNumber = 1,
|
||||
};
|
||||
// repeated string keys = 2;
|
||||
int keys_size() const;
|
||||
void clear_keys();
|
||||
const std::string& keys(int index) const;
|
||||
std::string* mutable_keys(int index);
|
||||
void set_keys(int index, const std::string& value);
|
||||
void set_keys(int index, std::string&& value);
|
||||
void set_keys(int index, const char* value);
|
||||
void set_keys(int index, const char* value, size_t size);
|
||||
std::string* add_keys();
|
||||
void add_keys(const std::string& value);
|
||||
void add_keys(std::string&& value);
|
||||
void add_keys(const char* value);
|
||||
void add_keys(const char* value, size_t size);
|
||||
const ::PROTOBUF_NAMESPACE_ID::RepeatedPtrField<std::string>& keys() const;
|
||||
::PROTOBUF_NAMESPACE_ID::RepeatedPtrField<std::string>* mutable_keys();
|
||||
|
||||
// repeated string values = 3;
|
||||
int values_size() const;
|
||||
void clear_values();
|
||||
const std::string& values(int index) const;
|
||||
std::string* mutable_values(int index);
|
||||
void set_values(int index, const std::string& value);
|
||||
void set_values(int index, std::string&& value);
|
||||
void set_values(int index, const char* value);
|
||||
void set_values(int index, const char* value, size_t size);
|
||||
std::string* add_values();
|
||||
void add_values(const std::string& value);
|
||||
void add_values(std::string&& value);
|
||||
void add_values(const char* value);
|
||||
void add_values(const char* value, size_t size);
|
||||
const ::PROTOBUF_NAMESPACE_ID::RepeatedPtrField<std::string>& values() const;
|
||||
::PROTOBUF_NAMESPACE_ID::RepeatedPtrField<std::string>* mutable_values();
|
||||
|
||||
// .milvus.proto.common.Status status = 1;
|
||||
bool has_status() const;
|
||||
void clear_status();
|
||||
const ::milvus::proto::common::Status& status() const;
|
||||
::milvus::proto::common::Status* release_status();
|
||||
::milvus::proto::common::Status* mutable_status();
|
||||
void set_allocated_status(::milvus::proto::common::Status* status);
|
||||
|
||||
// @@protoc_insertion_point(class_scope:milvus.proto.service.SysConfigResponse)
|
||||
private:
|
||||
class _Internal;
|
||||
|
||||
::PROTOBUF_NAMESPACE_ID::internal::InternalMetadataWithArena _internal_metadata_;
|
||||
::PROTOBUF_NAMESPACE_ID::RepeatedPtrField<std::string> keys_;
|
||||
::PROTOBUF_NAMESPACE_ID::RepeatedPtrField<std::string> values_;
|
||||
::milvus::proto::common::Status* status_;
|
||||
mutable ::PROTOBUF_NAMESPACE_ID::internal::CachedSize _cached_size_;
|
||||
friend struct ::TableStruct_service_5fmsg_2eproto;
|
||||
};
|
||||
// -------------------------------------------------------------------
|
||||
|
||||
class Score :
|
||||
public ::PROTOBUF_NAMESPACE_ID::Message /* @@protoc_insertion_point(class_definition:milvus.proto.service.Score) */ {
|
||||
public:
|
||||
|
@ -2200,7 +2376,7 @@ class Score :
|
|||
&_Score_default_instance_);
|
||||
}
|
||||
static constexpr int kIndexInFileMessages =
|
||||
13;
|
||||
14;
|
||||
|
||||
friend void swap(Score& a, Score& b) {
|
||||
a.Swap(&b);
|
||||
|
@ -2351,7 +2527,7 @@ class Hits :
|
|||
&_Hits_default_instance_);
|
||||
}
|
||||
static constexpr int kIndexInFileMessages =
|
||||
14;
|
||||
15;
|
||||
|
||||
friend void swap(Hits& a, Hits& b) {
|
||||
a.Swap(&b);
|
||||
|
@ -2515,7 +2691,7 @@ class QueryResult :
|
|||
&_QueryResult_default_instance_);
|
||||
}
|
||||
static constexpr int kIndexInFileMessages =
|
||||
15;
|
||||
16;
|
||||
|
||||
friend void swap(QueryResult& a, QueryResult& b) {
|
||||
a.Swap(&b);
|
||||
|
@ -4022,6 +4198,185 @@ PartitionDescription::statistics() const {
|
|||
|
||||
// -------------------------------------------------------------------
|
||||
|
||||
// SysConfigResponse
|
||||
|
||||
// .milvus.proto.common.Status status = 1;
|
||||
inline bool SysConfigResponse::has_status() const {
|
||||
return this != internal_default_instance() && status_ != nullptr;
|
||||
}
|
||||
inline const ::milvus::proto::common::Status& SysConfigResponse::status() const {
|
||||
const ::milvus::proto::common::Status* p = status_;
|
||||
// @@protoc_insertion_point(field_get:milvus.proto.service.SysConfigResponse.status)
|
||||
return p != nullptr ? *p : *reinterpret_cast<const ::milvus::proto::common::Status*>(
|
||||
&::milvus::proto::common::_Status_default_instance_);
|
||||
}
|
||||
inline ::milvus::proto::common::Status* SysConfigResponse::release_status() {
|
||||
// @@protoc_insertion_point(field_release:milvus.proto.service.SysConfigResponse.status)
|
||||
|
||||
::milvus::proto::common::Status* temp = status_;
|
||||
status_ = nullptr;
|
||||
return temp;
|
||||
}
|
||||
inline ::milvus::proto::common::Status* SysConfigResponse::mutable_status() {
|
||||
|
||||
if (status_ == nullptr) {
|
||||
auto* p = CreateMaybeMessage<::milvus::proto::common::Status>(GetArenaNoVirtual());
|
||||
status_ = p;
|
||||
}
|
||||
// @@protoc_insertion_point(field_mutable:milvus.proto.service.SysConfigResponse.status)
|
||||
return status_;
|
||||
}
|
||||
inline void SysConfigResponse::set_allocated_status(::milvus::proto::common::Status* status) {
|
||||
::PROTOBUF_NAMESPACE_ID::Arena* message_arena = GetArenaNoVirtual();
|
||||
if (message_arena == nullptr) {
|
||||
delete reinterpret_cast< ::PROTOBUF_NAMESPACE_ID::MessageLite*>(status_);
|
||||
}
|
||||
if (status) {
|
||||
::PROTOBUF_NAMESPACE_ID::Arena* submessage_arena = nullptr;
|
||||
if (message_arena != submessage_arena) {
|
||||
status = ::PROTOBUF_NAMESPACE_ID::internal::GetOwnedMessage(
|
||||
message_arena, status, submessage_arena);
|
||||
}
|
||||
|
||||
} else {
|
||||
|
||||
}
|
||||
status_ = status;
|
||||
// @@protoc_insertion_point(field_set_allocated:milvus.proto.service.SysConfigResponse.status)
|
||||
}
|
||||
|
||||
// repeated string keys = 2;
|
||||
inline int SysConfigResponse::keys_size() const {
|
||||
return keys_.size();
|
||||
}
|
||||
inline void SysConfigResponse::clear_keys() {
|
||||
keys_.Clear();
|
||||
}
|
||||
inline const std::string& SysConfigResponse::keys(int index) const {
|
||||
// @@protoc_insertion_point(field_get:milvus.proto.service.SysConfigResponse.keys)
|
||||
return keys_.Get(index);
|
||||
}
|
||||
inline std::string* SysConfigResponse::mutable_keys(int index) {
|
||||
// @@protoc_insertion_point(field_mutable:milvus.proto.service.SysConfigResponse.keys)
|
||||
return keys_.Mutable(index);
|
||||
}
|
||||
inline void SysConfigResponse::set_keys(int index, const std::string& value) {
|
||||
// @@protoc_insertion_point(field_set:milvus.proto.service.SysConfigResponse.keys)
|
||||
keys_.Mutable(index)->assign(value);
|
||||
}
|
||||
inline void SysConfigResponse::set_keys(int index, std::string&& value) {
|
||||
// @@protoc_insertion_point(field_set:milvus.proto.service.SysConfigResponse.keys)
|
||||
keys_.Mutable(index)->assign(std::move(value));
|
||||
}
|
||||
inline void SysConfigResponse::set_keys(int index, const char* value) {
|
||||
GOOGLE_DCHECK(value != nullptr);
|
||||
keys_.Mutable(index)->assign(value);
|
||||
// @@protoc_insertion_point(field_set_char:milvus.proto.service.SysConfigResponse.keys)
|
||||
}
|
||||
inline void SysConfigResponse::set_keys(int index, const char* value, size_t size) {
|
||||
keys_.Mutable(index)->assign(
|
||||
reinterpret_cast<const char*>(value), size);
|
||||
// @@protoc_insertion_point(field_set_pointer:milvus.proto.service.SysConfigResponse.keys)
|
||||
}
|
||||
inline std::string* SysConfigResponse::add_keys() {
|
||||
// @@protoc_insertion_point(field_add_mutable:milvus.proto.service.SysConfigResponse.keys)
|
||||
return keys_.Add();
|
||||
}
|
||||
inline void SysConfigResponse::add_keys(const std::string& value) {
|
||||
keys_.Add()->assign(value);
|
||||
// @@protoc_insertion_point(field_add:milvus.proto.service.SysConfigResponse.keys)
|
||||
}
|
||||
inline void SysConfigResponse::add_keys(std::string&& value) {
|
||||
keys_.Add(std::move(value));
|
||||
// @@protoc_insertion_point(field_add:milvus.proto.service.SysConfigResponse.keys)
|
||||
}
|
||||
inline void SysConfigResponse::add_keys(const char* value) {
|
||||
GOOGLE_DCHECK(value != nullptr);
|
||||
keys_.Add()->assign(value);
|
||||
// @@protoc_insertion_point(field_add_char:milvus.proto.service.SysConfigResponse.keys)
|
||||
}
|
||||
inline void SysConfigResponse::add_keys(const char* value, size_t size) {
|
||||
keys_.Add()->assign(reinterpret_cast<const char*>(value), size);
|
||||
// @@protoc_insertion_point(field_add_pointer:milvus.proto.service.SysConfigResponse.keys)
|
||||
}
|
||||
inline const ::PROTOBUF_NAMESPACE_ID::RepeatedPtrField<std::string>&
|
||||
SysConfigResponse::keys() const {
|
||||
// @@protoc_insertion_point(field_list:milvus.proto.service.SysConfigResponse.keys)
|
||||
return keys_;
|
||||
}
|
||||
inline ::PROTOBUF_NAMESPACE_ID::RepeatedPtrField<std::string>*
|
||||
SysConfigResponse::mutable_keys() {
|
||||
// @@protoc_insertion_point(field_mutable_list:milvus.proto.service.SysConfigResponse.keys)
|
||||
return &keys_;
|
||||
}
|
||||
|
||||
// repeated string values = 3;
|
||||
inline int SysConfigResponse::values_size() const {
|
||||
return values_.size();
|
||||
}
|
||||
inline void SysConfigResponse::clear_values() {
|
||||
values_.Clear();
|
||||
}
|
||||
inline const std::string& SysConfigResponse::values(int index) const {
|
||||
// @@protoc_insertion_point(field_get:milvus.proto.service.SysConfigResponse.values)
|
||||
return values_.Get(index);
|
||||
}
|
||||
inline std::string* SysConfigResponse::mutable_values(int index) {
|
||||
// @@protoc_insertion_point(field_mutable:milvus.proto.service.SysConfigResponse.values)
|
||||
return values_.Mutable(index);
|
||||
}
|
||||
inline void SysConfigResponse::set_values(int index, const std::string& value) {
|
||||
// @@protoc_insertion_point(field_set:milvus.proto.service.SysConfigResponse.values)
|
||||
values_.Mutable(index)->assign(value);
|
||||
}
|
||||
inline void SysConfigResponse::set_values(int index, std::string&& value) {
|
||||
// @@protoc_insertion_point(field_set:milvus.proto.service.SysConfigResponse.values)
|
||||
values_.Mutable(index)->assign(std::move(value));
|
||||
}
|
||||
inline void SysConfigResponse::set_values(int index, const char* value) {
|
||||
GOOGLE_DCHECK(value != nullptr);
|
||||
values_.Mutable(index)->assign(value);
|
||||
// @@protoc_insertion_point(field_set_char:milvus.proto.service.SysConfigResponse.values)
|
||||
}
|
||||
inline void SysConfigResponse::set_values(int index, const char* value, size_t size) {
|
||||
values_.Mutable(index)->assign(
|
||||
reinterpret_cast<const char*>(value), size);
|
||||
// @@protoc_insertion_point(field_set_pointer:milvus.proto.service.SysConfigResponse.values)
|
||||
}
|
||||
inline std::string* SysConfigResponse::add_values() {
|
||||
// @@protoc_insertion_point(field_add_mutable:milvus.proto.service.SysConfigResponse.values)
|
||||
return values_.Add();
|
||||
}
|
||||
inline void SysConfigResponse::add_values(const std::string& value) {
|
||||
values_.Add()->assign(value);
|
||||
// @@protoc_insertion_point(field_add:milvus.proto.service.SysConfigResponse.values)
|
||||
}
|
||||
inline void SysConfigResponse::add_values(std::string&& value) {
|
||||
values_.Add(std::move(value));
|
||||
// @@protoc_insertion_point(field_add:milvus.proto.service.SysConfigResponse.values)
|
||||
}
|
||||
inline void SysConfigResponse::add_values(const char* value) {
|
||||
GOOGLE_DCHECK(value != nullptr);
|
||||
values_.Add()->assign(value);
|
||||
// @@protoc_insertion_point(field_add_char:milvus.proto.service.SysConfigResponse.values)
|
||||
}
|
||||
inline void SysConfigResponse::add_values(const char* value, size_t size) {
|
||||
values_.Add()->assign(reinterpret_cast<const char*>(value), size);
|
||||
// @@protoc_insertion_point(field_add_pointer:milvus.proto.service.SysConfigResponse.values)
|
||||
}
|
||||
inline const ::PROTOBUF_NAMESPACE_ID::RepeatedPtrField<std::string>&
|
||||
SysConfigResponse::values() const {
|
||||
// @@protoc_insertion_point(field_list:milvus.proto.service.SysConfigResponse.values)
|
||||
return values_;
|
||||
}
|
||||
inline ::PROTOBUF_NAMESPACE_ID::RepeatedPtrField<std::string>*
|
||||
SysConfigResponse::mutable_values() {
|
||||
// @@protoc_insertion_point(field_mutable_list:milvus.proto.service.SysConfigResponse.values)
|
||||
return &values_;
|
||||
}
|
||||
|
||||
// -------------------------------------------------------------------
|
||||
|
||||
// Score
|
||||
|
||||
// string tag = 1;
|
||||
|
@ -4308,6 +4663,8 @@ QueryResult::hits() const {
|
|||
|
||||
// -------------------------------------------------------------------
|
||||
|
||||
// -------------------------------------------------------------------
|
||||
|
||||
|
||||
// @@protoc_insertion_point(namespace_scope)
|
||||
|
||||
|
|
|
@ -130,7 +130,13 @@ ExecExprVisitor::ExecRangeVisitorDispatcher(RangeExpr& expr_raw) -> RetType {
|
|||
}
|
||||
|
||||
case OpType::NotEqual: {
|
||||
auto index_func = [val](Index* index) { return index->NotIn(1, &val); };
|
||||
auto index_func = [val](Index* index) {
|
||||
// Note: index->NotIn() is buggy, investigating
|
||||
// this is a workaround
|
||||
auto res = index->In(1, &val);
|
||||
*res = ~std::move(*res);
|
||||
return res;
|
||||
};
|
||||
return ExecRangeVisitorImpl(expr, index_func, [val](T x) { return !(x != val); });
|
||||
}
|
||||
|
||||
|
|
|
@ -18,7 +18,6 @@
|
|||
#include <knowhere/index/vector_index/adapter/VectorAdapter.h>
|
||||
#include <knowhere/index/vector_index/VecIndexFactory.h>
|
||||
#include <cstdint>
|
||||
#include <boost/concept_check.hpp>
|
||||
|
||||
CSegmentBase
|
||||
NewSegment(CCollection collection, uint64_t segment_id) {
|
||||
|
@ -42,7 +41,7 @@ DeleteSegment(CSegmentBase segment) {
|
|||
|
||||
//////////////////////////////////////////////////////////////////
|
||||
|
||||
CStatus
|
||||
int
|
||||
Insert(CSegmentBase c_segment,
|
||||
int64_t reserved_offset,
|
||||
int64_t size,
|
||||
|
@ -58,22 +57,11 @@ Insert(CSegmentBase c_segment,
|
|||
dataChunk.sizeof_per_row = sizeof_per_row;
|
||||
dataChunk.count = count;
|
||||
|
||||
try {
|
||||
auto res = segment->Insert(reserved_offset, size, row_ids, timestamps, dataChunk);
|
||||
|
||||
auto status = CStatus();
|
||||
status.error_code = Success;
|
||||
status.error_msg = "";
|
||||
return status;
|
||||
} catch (std::runtime_error& e) {
|
||||
auto status = CStatus();
|
||||
status.error_code = UnexpectedException;
|
||||
status.error_msg = strdup(e.what());
|
||||
return status;
|
||||
}
|
||||
auto res = segment->Insert(reserved_offset, size, row_ids, timestamps, dataChunk);
|
||||
|
||||
// TODO: delete print
|
||||
// std::cout << "do segment insert, sizeof_per_row = " << sizeof_per_row << std::endl;
|
||||
return res.code();
|
||||
}
|
||||
|
||||
int64_t
|
||||
|
@ -85,24 +73,13 @@ PreInsert(CSegmentBase c_segment, int64_t size) {
|
|||
return segment->PreInsert(size);
|
||||
}
|
||||
|
||||
CStatus
|
||||
int
|
||||
Delete(
|
||||
CSegmentBase c_segment, int64_t reserved_offset, int64_t size, const int64_t* row_ids, const uint64_t* timestamps) {
|
||||
auto segment = (milvus::segcore::SegmentBase*)c_segment;
|
||||
|
||||
try {
|
||||
auto res = segment->Delete(reserved_offset, size, row_ids, timestamps);
|
||||
|
||||
auto status = CStatus();
|
||||
status.error_code = Success;
|
||||
status.error_msg = "";
|
||||
return status;
|
||||
} catch (std::runtime_error& e) {
|
||||
auto status = CStatus();
|
||||
status.error_code = UnexpectedException;
|
||||
status.error_msg = strdup(e.what());
|
||||
return status;
|
||||
}
|
||||
auto res = segment->Delete(reserved_offset, size, row_ids, timestamps);
|
||||
return res.code();
|
||||
}
|
||||
|
||||
int64_t
|
||||
|
@ -114,7 +91,7 @@ PreDelete(CSegmentBase c_segment, int64_t size) {
|
|||
return segment->PreDelete(size);
|
||||
}
|
||||
|
||||
CStatus
|
||||
int
|
||||
Search(CSegmentBase c_segment,
|
||||
CPlan c_plan,
|
||||
CPlaceholderGroup* c_placeholder_groups,
|
||||
|
@ -130,22 +107,14 @@ Search(CSegmentBase c_segment,
|
|||
}
|
||||
milvus::segcore::QueryResult query_result;
|
||||
|
||||
auto status = CStatus();
|
||||
try {
|
||||
auto res = segment->Search(plan, placeholder_groups.data(), timestamps, num_groups, query_result);
|
||||
status.error_code = Success;
|
||||
status.error_msg = "";
|
||||
} catch (std::runtime_error& e) {
|
||||
status.error_code = UnexpectedException;
|
||||
status.error_msg = strdup(e.what());
|
||||
}
|
||||
auto res = segment->Search(plan, placeholder_groups.data(), timestamps, num_groups, query_result);
|
||||
|
||||
// result_ids and result_distances have been allocated memory in goLang,
|
||||
// so we don't need to malloc here.
|
||||
memcpy(result_ids, query_result.result_ids_.data(), query_result.get_row_count() * sizeof(int64_t));
|
||||
memcpy(result_distances, query_result.result_distances_.data(), query_result.get_row_count() * sizeof(float));
|
||||
|
||||
return status;
|
||||
return res.code();
|
||||
}
|
||||
|
||||
//////////////////////////////////////////////////////////////////
|
||||
|
|
|
@ -14,24 +14,12 @@ extern "C" {
|
|||
#endif
|
||||
|
||||
#include <stdbool.h>
|
||||
#include <stdlib.h>
|
||||
#include <stdint.h>
|
||||
|
||||
#include "segcore/collection_c.h"
|
||||
#include "segcore/plan_c.h"
|
||||
#include <stdint.h>
|
||||
|
||||
typedef void* CSegmentBase;
|
||||
|
||||
enum ErrorCode {
|
||||
Success = 0,
|
||||
UnexpectedException = 1,
|
||||
};
|
||||
|
||||
typedef struct CStatus {
|
||||
int error_code;
|
||||
const char* error_msg;
|
||||
} CStatus;
|
||||
|
||||
CSegmentBase
|
||||
NewSegment(CCollection collection, uint64_t segment_id);
|
||||
|
||||
|
@ -40,7 +28,7 @@ DeleteSegment(CSegmentBase segment);
|
|||
|
||||
//////////////////////////////////////////////////////////////////
|
||||
|
||||
CStatus
|
||||
int
|
||||
Insert(CSegmentBase c_segment,
|
||||
int64_t reserved_offset,
|
||||
int64_t size,
|
||||
|
@ -53,14 +41,14 @@ Insert(CSegmentBase c_segment,
|
|||
int64_t
|
||||
PreInsert(CSegmentBase c_segment, int64_t size);
|
||||
|
||||
CStatus
|
||||
int
|
||||
Delete(
|
||||
CSegmentBase c_segment, int64_t reserved_offset, int64_t size, const int64_t* row_ids, const uint64_t* timestamps);
|
||||
|
||||
int64_t
|
||||
PreDelete(CSegmentBase c_segment, int64_t size);
|
||||
|
||||
CStatus
|
||||
int
|
||||
Search(CSegmentBase c_segment,
|
||||
CPlan plan,
|
||||
CPlaceholderGroup* placeholder_groups,
|
||||
|
|
|
@ -65,7 +65,7 @@ TEST(CApiTest, InsertTest) {
|
|||
|
||||
auto res = Insert(segment, offset, N, uids.data(), timestamps.data(), raw_data.data(), (int)line_sizeof, N);
|
||||
|
||||
assert(res.error_code == Success);
|
||||
assert(res == 0);
|
||||
|
||||
DeleteCollection(collection);
|
||||
DeleteSegment(segment);
|
||||
|
@ -82,7 +82,7 @@ TEST(CApiTest, DeleteTest) {
|
|||
auto offset = PreDelete(segment, 3);
|
||||
|
||||
auto del_res = Delete(segment, offset, 3, delete_row_ids, delete_timestamps);
|
||||
assert(del_res.error_code == Success);
|
||||
assert(del_res == 0);
|
||||
|
||||
DeleteCollection(collection);
|
||||
DeleteSegment(segment);
|
||||
|
@ -116,7 +116,7 @@ TEST(CApiTest, SearchTest) {
|
|||
auto offset = PreInsert(segment, N);
|
||||
|
||||
auto ins_res = Insert(segment, offset, N, uids.data(), timestamps.data(), raw_data.data(), (int)line_sizeof, N);
|
||||
assert(ins_res.error_code == Success);
|
||||
assert(ins_res == 0);
|
||||
|
||||
const char* dsl_string = R"(
|
||||
{
|
||||
|
@ -163,7 +163,7 @@ TEST(CApiTest, SearchTest) {
|
|||
float result_distances[100];
|
||||
|
||||
auto sea_res = Search(segment, plan, placeholderGroups.data(), timestamps.data(), 1, result_ids, result_distances);
|
||||
assert(sea_res.error_code == Success);
|
||||
assert(sea_res == 0);
|
||||
|
||||
DeletePlan(plan);
|
||||
DeletePlaceholderGroup(placeholderGroup);
|
||||
|
@ -199,7 +199,7 @@ TEST(CApiTest, BuildIndexTest) {
|
|||
auto offset = PreInsert(segment, N);
|
||||
|
||||
auto ins_res = Insert(segment, offset, N, uids.data(), timestamps.data(), raw_data.data(), (int)line_sizeof, N);
|
||||
assert(ins_res.error_code == Success);
|
||||
assert(ins_res == 0);
|
||||
|
||||
// TODO: add index ptr
|
||||
Close(segment);
|
||||
|
@ -250,7 +250,7 @@ TEST(CApiTest, BuildIndexTest) {
|
|||
float result_distances[100];
|
||||
|
||||
auto sea_res = Search(segment, plan, placeholderGroups.data(), timestamps.data(), 1, result_ids, result_distances);
|
||||
assert(sea_res.error_code == Success);
|
||||
assert(sea_res == 0);
|
||||
|
||||
DeletePlan(plan);
|
||||
DeletePlaceholderGroup(placeholderGroup);
|
||||
|
@ -315,7 +315,7 @@ TEST(CApiTest, GetMemoryUsageInBytesTest) {
|
|||
|
||||
auto res = Insert(segment, offset, N, uids.data(), timestamps.data(), raw_data.data(), (int)line_sizeof, N);
|
||||
|
||||
assert(res.error_code == Success);
|
||||
assert(res == 0);
|
||||
|
||||
auto memory_usage_size = GetMemoryUsageInBytes(segment);
|
||||
|
||||
|
@ -482,7 +482,7 @@ TEST(CApiTest, GetDeletedCountTest) {
|
|||
auto offset = PreDelete(segment, 3);
|
||||
|
||||
auto del_res = Delete(segment, offset, 3, delete_row_ids, delete_timestamps);
|
||||
assert(del_res.error_code == Success);
|
||||
assert(del_res == 0);
|
||||
|
||||
// TODO: assert(deleted_count == len(delete_row_ids))
|
||||
auto deleted_count = GetDeletedCount(segment);
|
||||
|
@ -502,7 +502,7 @@ TEST(CApiTest, GetRowCountTest) {
|
|||
auto line_sizeof = (sizeof(int) + sizeof(float) * 16);
|
||||
auto offset = PreInsert(segment, N);
|
||||
auto res = Insert(segment, offset, N, uids.data(), timestamps.data(), raw_data.data(), (int)line_sizeof, N);
|
||||
assert(res.error_code == Success);
|
||||
assert(res == 0);
|
||||
|
||||
auto row_count = GetRowCount(segment);
|
||||
assert(row_count == N);
|
||||
|
|
|
@ -0,0 +1,79 @@
|
|||
package master
|
||||
|
||||
import (
|
||||
"log"
|
||||
|
||||
"github.com/zilliztech/milvus-distributed/internal/errors"
|
||||
"github.com/zilliztech/milvus-distributed/internal/kv"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb"
|
||||
)
|
||||
|
||||
type getSysConfigsTask struct {
|
||||
baseTask
|
||||
configkv *kv.EtcdKV
|
||||
req *internalpb.SysConfigRequest
|
||||
keys []string
|
||||
values []string
|
||||
}
|
||||
|
||||
func (t *getSysConfigsTask) Type() internalpb.MsgType {
|
||||
if t.req == nil {
|
||||
log.Printf("null request")
|
||||
return 0
|
||||
}
|
||||
return t.req.MsgType
|
||||
}
|
||||
|
||||
func (t *getSysConfigsTask) Ts() (Timestamp, error) {
|
||||
if t.req == nil {
|
||||
return 0, errors.New("null request")
|
||||
}
|
||||
return t.req.Timestamp, nil
|
||||
}
|
||||
|
||||
func (t *getSysConfigsTask) Execute() error {
|
||||
if t.req == nil {
|
||||
return errors.New("null request")
|
||||
}
|
||||
|
||||
sc := &SysConfig{kv: t.configkv}
|
||||
keyMap := make(map[string]bool)
|
||||
|
||||
// Load configs with prefix
|
||||
for _, prefix := range t.req.KeyPrefixes {
|
||||
prefixKeys, prefixVals, err := sc.GetByPrefix(prefix)
|
||||
if err != nil {
|
||||
return errors.Errorf("Load configs by prefix wrong: %s", err.Error())
|
||||
}
|
||||
t.keys = append(t.keys, prefixKeys...)
|
||||
t.values = append(t.values, prefixVals...)
|
||||
}
|
||||
|
||||
for _, key := range t.keys {
|
||||
keyMap[key] = true
|
||||
}
|
||||
|
||||
// Load specific configs
|
||||
if len(t.req.Keys) > 0 {
|
||||
// To clean up duplicated keys
|
||||
cleanKeys := []string{}
|
||||
for _, key := range t.req.Keys {
|
||||
if v, ok := keyMap[key]; (!ok) || (ok && !v) {
|
||||
cleanKeys = append(cleanKeys, key)
|
||||
keyMap[key] = true
|
||||
continue
|
||||
}
|
||||
log.Println("[GetSysConfigs] Warning: duplicate key:", key)
|
||||
}
|
||||
|
||||
v, err := sc.Get(cleanKeys)
|
||||
if err != nil {
|
||||
return errors.Errorf("Load configs wrong: %s", err.Error())
|
||||
}
|
||||
|
||||
t.keys = append(t.keys, cleanKeys...)
|
||||
t.values = append(t.values, v...)
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
|
@ -0,0 +1,150 @@
|
|||
package master
|
||||
|
||||
import (
|
||||
"context"
|
||||
"strings"
|
||||
"testing"
|
||||
|
||||
"github.com/stretchr/testify/assert"
|
||||
"github.com/stretchr/testify/require"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/masterpb"
|
||||
"github.com/zilliztech/milvus-distributed/internal/util/typeutil"
|
||||
"go.etcd.io/etcd/clientv3"
|
||||
"google.golang.org/grpc"
|
||||
)
|
||||
|
||||
func TestMaster_ConfigTask(t *testing.T) {
|
||||
Init()
|
||||
|
||||
ctx, cancel := context.WithCancel(context.TODO())
|
||||
defer cancel()
|
||||
|
||||
etcdCli, err := clientv3.New(clientv3.Config{Endpoints: []string{Params.EtcdAddress}})
|
||||
require.Nil(t, err)
|
||||
_, err = etcdCli.Delete(ctx, "/test/root", clientv3.WithPrefix())
|
||||
require.Nil(t, err)
|
||||
|
||||
Params = ParamTable{
|
||||
Address: Params.Address,
|
||||
Port: Params.Port,
|
||||
|
||||
EtcdAddress: Params.EtcdAddress,
|
||||
EtcdRootPath: "/test/root",
|
||||
PulsarAddress: Params.PulsarAddress,
|
||||
|
||||
ProxyIDList: []typeutil.UniqueID{1, 2},
|
||||
WriteNodeIDList: []typeutil.UniqueID{3, 4},
|
||||
|
||||
TopicNum: 5,
|
||||
QueryNodeNum: 3,
|
||||
SoftTimeTickBarrierInterval: 300,
|
||||
|
||||
// segment
|
||||
SegmentSize: 536870912 / 1024 / 1024,
|
||||
SegmentSizeFactor: 0.75,
|
||||
DefaultRecordSize: 1024,
|
||||
MinSegIDAssignCnt: 1048576 / 1024,
|
||||
MaxSegIDAssignCnt: Params.MaxSegIDAssignCnt,
|
||||
SegIDAssignExpiration: 2000,
|
||||
|
||||
// msgChannel
|
||||
ProxyTimeTickChannelNames: []string{"proxy1", "proxy2"},
|
||||
WriteNodeTimeTickChannelNames: []string{"write3", "write4"},
|
||||
InsertChannelNames: []string{"dm0", "dm1"},
|
||||
K2SChannelNames: []string{"k2s0", "k2s1"},
|
||||
QueryNodeStatsChannelName: "statistic",
|
||||
MsgChannelSubName: Params.MsgChannelSubName,
|
||||
}
|
||||
|
||||
svr, err := CreateServer(ctx)
|
||||
require.Nil(t, err)
|
||||
err = svr.Run(10002)
|
||||
defer svr.Close()
|
||||
require.Nil(t, err)
|
||||
|
||||
conn, err := grpc.DialContext(ctx, "127.0.0.1:10002", grpc.WithInsecure(), grpc.WithBlock())
|
||||
require.Nil(t, err)
|
||||
defer conn.Close()
|
||||
|
||||
cli := masterpb.NewMasterClient(conn)
|
||||
testKeys := []string{
|
||||
"/etcd/address",
|
||||
"/master/port",
|
||||
"/master/proxyidlist",
|
||||
"/master/segmentthresholdfactor",
|
||||
"/pulsar/token",
|
||||
"/reader/stopflag",
|
||||
"/proxy/timezone",
|
||||
"/proxy/network/address",
|
||||
"/proxy/storage/path",
|
||||
"/storage/accesskey",
|
||||
}
|
||||
|
||||
testVals := []string{
|
||||
"localhost",
|
||||
"53100",
|
||||
"[1 2]",
|
||||
"0.75",
|
||||
"eyJhbGciOiJIUzI1NiJ9.eyJzdWIiOiJKb2UifQ.ipevRNuRP6HflG8cFKnmUPtypruRC4fb1DWtoLL62SY",
|
||||
"-1",
|
||||
"UTC+8",
|
||||
"0.0.0.0",
|
||||
"/var/lib/milvus",
|
||||
"",
|
||||
}
|
||||
|
||||
sc := SysConfig{kv: svr.kvBase}
|
||||
sc.InitFromFile(".")
|
||||
|
||||
configRequest := &internalpb.SysConfigRequest{
|
||||
MsgType: internalpb.MsgType_kGetSysConfigs,
|
||||
ReqID: 1,
|
||||
Timestamp: 11,
|
||||
ProxyID: 1,
|
||||
Keys: testKeys,
|
||||
KeyPrefixes: []string{},
|
||||
}
|
||||
|
||||
response, err := cli.GetSysConfigs(ctx, configRequest)
|
||||
assert.Nil(t, err)
|
||||
assert.ElementsMatch(t, testKeys, response.Keys)
|
||||
assert.ElementsMatch(t, testVals, response.Values)
|
||||
assert.Equal(t, len(response.GetKeys()), len(response.GetValues()))
|
||||
|
||||
configRequest = &internalpb.SysConfigRequest{
|
||||
MsgType: internalpb.MsgType_kGetSysConfigs,
|
||||
ReqID: 1,
|
||||
Timestamp: 11,
|
||||
ProxyID: 1,
|
||||
Keys: []string{},
|
||||
KeyPrefixes: []string{"/master"},
|
||||
}
|
||||
|
||||
response, err = cli.GetSysConfigs(ctx, configRequest)
|
||||
assert.Nil(t, err)
|
||||
for i := range response.GetKeys() {
|
||||
assert.True(t, strings.HasPrefix(response.GetKeys()[i], "/master"))
|
||||
}
|
||||
assert.Equal(t, len(response.GetKeys()), len(response.GetValues()))
|
||||
|
||||
t.Run("Test duplicate keys and key prefix", func(t *testing.T) {
|
||||
configRequest.Keys = []string{}
|
||||
configRequest.KeyPrefixes = []string{"/master"}
|
||||
|
||||
resp, err := cli.GetSysConfigs(ctx, configRequest)
|
||||
require.Nil(t, err)
|
||||
assert.Equal(t, len(resp.GetKeys()), len(resp.GetValues()))
|
||||
assert.NotEqual(t, 0, len(resp.GetKeys()))
|
||||
|
||||
configRequest.Keys = []string{"/master/port"}
|
||||
configRequest.KeyPrefixes = []string{"/master"}
|
||||
|
||||
respDup, err := cli.GetSysConfigs(ctx, configRequest)
|
||||
require.Nil(t, err)
|
||||
assert.Equal(t, len(respDup.GetKeys()), len(respDup.GetValues()))
|
||||
assert.NotEqual(t, 0, len(respDup.GetKeys()))
|
||||
assert.Equal(t, len(respDup.GetKeys()), len(resp.GetKeys()))
|
||||
})
|
||||
|
||||
}
|
|
@ -0,0 +1,111 @@
|
|||
# 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.
|
||||
|
||||
master: # 21
|
||||
address: localhost
|
||||
port: 53100
|
||||
pulsarmoniterinterval: 1
|
||||
pulsartopic: "monitor-topic"
|
||||
|
||||
proxyidlist: [1, 2]
|
||||
proxyTimeSyncChannels: ["proxy1", "proxy2"]
|
||||
proxyTimeSyncSubName: "proxy-topic"
|
||||
softTimeTickBarrierInterval: 500
|
||||
|
||||
writeidlist: [3, 4]
|
||||
writeTimeSyncChannels: ["write3", "write4"]
|
||||
writeTimeSyncSubName: "write-topic"
|
||||
|
||||
dmTimeSyncChannels: ["dm5", "dm6"]
|
||||
k2sTimeSyncChannels: ["k2s7", "k2s8"]
|
||||
|
||||
defaultSizePerRecord: 1024
|
||||
minimumAssignSize: 1048576
|
||||
segmentThreshold: 536870912
|
||||
segmentExpireDuration: 2000
|
||||
segmentThresholdFactor: 0.75
|
||||
querynodenum: 1
|
||||
writenodenum: 1
|
||||
statsChannels: "statistic"
|
||||
|
||||
etcd: # 4
|
||||
address: localhost
|
||||
port: 2379
|
||||
rootpath: by-dev
|
||||
segthreshold: 10000
|
||||
|
||||
timesync: # 1
|
||||
interval: 400
|
||||
|
||||
storage: # 5
|
||||
driver: TIKV
|
||||
address: localhost
|
||||
port: 2379
|
||||
accesskey:
|
||||
secretkey:
|
||||
|
||||
pulsar: # 6
|
||||
authentication: false
|
||||
user: user-default
|
||||
token: eyJhbGciOiJIUzI1NiJ9.eyJzdWIiOiJKb2UifQ.ipevRNuRP6HflG8cFKnmUPtypruRC4fb1DWtoLL62SY
|
||||
address: localhost
|
||||
port: 6650
|
||||
topicnum: 128
|
||||
|
||||
reader: # 7
|
||||
clientid: 0
|
||||
stopflag: -1
|
||||
readerqueuesize: 10000
|
||||
searchchansize: 10000
|
||||
key2segchansize: 10000
|
||||
topicstart: 0
|
||||
topicend: 128
|
||||
|
||||
writer: # 8
|
||||
clientid: 0
|
||||
stopflag: -2
|
||||
readerqueuesize: 10000
|
||||
searchbyidchansize: 10000
|
||||
parallelism: 100
|
||||
topicstart: 0
|
||||
topicend: 128
|
||||
bucket: "zilliz-hz"
|
||||
|
||||
proxy: # 21
|
||||
timezone: UTC+8
|
||||
proxy_id: 1
|
||||
numReaderNodes: 2
|
||||
tsoSaveInterval: 200
|
||||
timeTickInterval: 200
|
||||
|
||||
pulsarTopics:
|
||||
readerTopicPrefix: "milvusReader"
|
||||
numReaderTopics: 2
|
||||
deleteTopic: "milvusDeleter"
|
||||
queryTopic: "milvusQuery"
|
||||
resultTopic: "milvusResult"
|
||||
resultGroup: "milvusResultGroup"
|
||||
timeTickTopic: "milvusTimeTick"
|
||||
|
||||
network:
|
||||
address: 0.0.0.0
|
||||
port: 19530
|
||||
|
||||
logs:
|
||||
level: debug
|
||||
trace.enable: true
|
||||
path: /tmp/logs
|
||||
max_log_file_size: 1024MB
|
||||
log_rotate_num: 0
|
||||
|
||||
storage:
|
||||
path: /var/lib/milvus
|
||||
auto_flush_interval: 1
|
|
@ -359,6 +359,43 @@ func (s *Master) ShowPartitions(ctx context.Context, in *internalpb.ShowPartitio
|
|||
return t.(*showPartitionTask).stringListResponse, nil
|
||||
}
|
||||
|
||||
func (s *Master) GetSysConfigs(ctx context.Context, in *internalpb.SysConfigRequest) (*servicepb.SysConfigResponse, error) {
|
||||
var t task = &getSysConfigsTask{
|
||||
req: in,
|
||||
configkv: s.kvBase,
|
||||
baseTask: baseTask{
|
||||
sch: s.scheduler,
|
||||
mt: s.metaTable,
|
||||
cv: make(chan error),
|
||||
},
|
||||
keys: []string{},
|
||||
values: []string{},
|
||||
}
|
||||
|
||||
response := &servicepb.SysConfigResponse{
|
||||
Status: &commonpb.Status{
|
||||
ErrorCode: commonpb.ErrorCode_UNEXPECTED_ERROR,
|
||||
},
|
||||
}
|
||||
|
||||
var err = s.scheduler.Enqueue(t)
|
||||
if err != nil {
|
||||
response.Status.Reason = "Enqueue failed: " + err.Error()
|
||||
return response, nil
|
||||
}
|
||||
|
||||
err = t.WaitToFinish(ctx)
|
||||
if err != nil {
|
||||
response.Status.Reason = "Get System Config failed: " + err.Error()
|
||||
return response, nil
|
||||
}
|
||||
|
||||
response.Keys = t.(*getSysConfigsTask).keys
|
||||
response.Values = t.(*getSysConfigsTask).values
|
||||
response.Status.ErrorCode = commonpb.ErrorCode_SUCCESS
|
||||
return response, nil
|
||||
}
|
||||
|
||||
//----------------------------------------Internal GRPC Service--------------------------------
|
||||
|
||||
func (s *Master) AllocTimestamp(ctx context.Context, request *internalpb.TsoRequest) (*internalpb.TsoResponse, error) {
|
||||
|
|
|
@ -52,7 +52,7 @@ func (mt *metaTable) reloadFromKV() error {
|
|||
|
||||
for _, value := range values {
|
||||
tenantMeta := pb.TenantMeta{}
|
||||
err := proto.UnmarshalText(value, &tenantMeta)
|
||||
err := proto.Unmarshal([]byte(value), &tenantMeta)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
@ -66,7 +66,7 @@ func (mt *metaTable) reloadFromKV() error {
|
|||
|
||||
for _, value := range values {
|
||||
proxyMeta := pb.ProxyMeta{}
|
||||
err = proto.UnmarshalText(value, &proxyMeta)
|
||||
err = proto.Unmarshal([]byte(value), &proxyMeta)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
@ -80,7 +80,7 @@ func (mt *metaTable) reloadFromKV() error {
|
|||
|
||||
for _, value := range values {
|
||||
collectionMeta := pb.CollectionMeta{}
|
||||
err = proto.UnmarshalText(value, &collectionMeta)
|
||||
err = proto.Unmarshal([]byte(value), &collectionMeta)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
@ -95,7 +95,7 @@ func (mt *metaTable) reloadFromKV() error {
|
|||
|
||||
for _, value := range values {
|
||||
segmentMeta := pb.SegmentMeta{}
|
||||
err = proto.UnmarshalText(value, &segmentMeta)
|
||||
err = proto.Unmarshal([]byte(value), &segmentMeta)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
@ -107,7 +107,10 @@ func (mt *metaTable) reloadFromKV() error {
|
|||
|
||||
// metaTable.ddLock.Lock() before call this function
|
||||
func (mt *metaTable) saveCollectionMeta(coll *pb.CollectionMeta) error {
|
||||
collBytes := proto.MarshalTextString(coll)
|
||||
collBytes, err := proto.Marshal(coll)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
mt.collID2Meta[coll.ID] = *coll
|
||||
mt.collName2ID[coll.Schema.Name] = coll.ID
|
||||
return mt.client.Save("/collection/"+strconv.FormatInt(coll.ID, 10), string(collBytes))
|
||||
|
@ -115,7 +118,10 @@ func (mt *metaTable) saveCollectionMeta(coll *pb.CollectionMeta) error {
|
|||
|
||||
// metaTable.ddLock.Lock() before call this function
|
||||
func (mt *metaTable) saveSegmentMeta(seg *pb.SegmentMeta) error {
|
||||
segBytes := proto.MarshalTextString(seg)
|
||||
segBytes, err := proto.Marshal(seg)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
mt.segID2Meta[seg.SegmentID] = *seg
|
||||
|
||||
|
@ -130,7 +136,10 @@ func (mt *metaTable) saveCollectionAndDeleteSegmentsMeta(coll *pb.CollectionMeta
|
|||
}
|
||||
|
||||
kvs := make(map[string]string)
|
||||
collStrs := proto.MarshalTextString(coll)
|
||||
collStrs, err := proto.Marshal(coll)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
kvs["/collection/"+strconv.FormatInt(coll.ID, 10)] = string(collStrs)
|
||||
|
||||
|
@ -150,15 +159,19 @@ func (mt *metaTable) saveCollectionAndDeleteSegmentsMeta(coll *pb.CollectionMeta
|
|||
// metaTable.ddLock.Lock() before call this function
|
||||
func (mt *metaTable) saveCollectionsAndSegmentsMeta(coll *pb.CollectionMeta, seg *pb.SegmentMeta) error {
|
||||
kvs := make(map[string]string)
|
||||
collBytes := proto.MarshalTextString(coll)
|
||||
|
||||
collBytes, err := proto.Marshal(coll)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
kvs["/collection/"+strconv.FormatInt(coll.ID, 10)] = string(collBytes)
|
||||
|
||||
mt.collID2Meta[coll.ID] = *coll
|
||||
mt.collName2ID[coll.Schema.Name] = coll.ID
|
||||
|
||||
segBytes := proto.MarshalTextString(seg)
|
||||
|
||||
segBytes, err := proto.Marshal(seg)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
kvs["/segment/"+strconv.FormatInt(seg.SegmentID, 10)] = string(segBytes)
|
||||
|
||||
mt.segID2Meta[seg.SegmentID] = *seg
|
||||
|
@ -207,7 +220,7 @@ func (mt *metaTable) AddCollection(coll *pb.CollectionMeta) error {
|
|||
}
|
||||
|
||||
if len(coll.PartitionTags) == 0 {
|
||||
coll.PartitionTags = append(coll.PartitionTags, Params.DefaultPartitionTag)
|
||||
coll.PartitionTags = append(coll.PartitionTags, "default")
|
||||
}
|
||||
_, ok := mt.collName2ID[coll.Schema.Name]
|
||||
if ok {
|
||||
|
@ -279,10 +292,6 @@ func (mt *metaTable) AddPartition(collID UniqueID, tag string) error {
|
|||
return errors.Errorf("can't find collection. id = " + strconv.FormatInt(collID, 10))
|
||||
}
|
||||
|
||||
// number of partition tags (except _default) should be limited to 4096 by default
|
||||
if int64(len(coll.PartitionTags)) > Params.MaxPartitionNum {
|
||||
return errors.New("maximum partition's number should be limit to " + strconv.FormatInt(Params.MaxPartitionNum, 10))
|
||||
}
|
||||
for _, t := range coll.PartitionTags {
|
||||
if t == tag {
|
||||
return errors.Errorf("partition already exists.")
|
||||
|
@ -317,29 +326,17 @@ func (mt *metaTable) DeletePartition(collID UniqueID, tag string) error {
|
|||
mt.ddLock.Lock()
|
||||
defer mt.ddLock.Unlock()
|
||||
|
||||
if tag == Params.DefaultPartitionTag {
|
||||
return errors.New("default partition cannot be deleted")
|
||||
}
|
||||
|
||||
collMeta, ok := mt.collID2Meta[collID]
|
||||
if !ok {
|
||||
return errors.Errorf("can't find collection. id = " + strconv.FormatInt(collID, 10))
|
||||
}
|
||||
|
||||
// check tag exists
|
||||
exist := false
|
||||
|
||||
pt := make([]string, 0, len(collMeta.PartitionTags))
|
||||
for _, t := range collMeta.PartitionTags {
|
||||
if t != tag {
|
||||
pt = append(pt, t)
|
||||
} else {
|
||||
exist = true
|
||||
}
|
||||
}
|
||||
if !exist {
|
||||
return errors.New("partition " + tag + " does not exist")
|
||||
}
|
||||
if len(pt) == len(collMeta.PartitionTags) {
|
||||
return nil
|
||||
}
|
||||
|
|
|
@ -3,7 +3,6 @@ package master
|
|||
import (
|
||||
"context"
|
||||
"reflect"
|
||||
"strconv"
|
||||
"testing"
|
||||
|
||||
"github.com/stretchr/testify/assert"
|
||||
|
@ -239,10 +238,6 @@ func TestMetaTable_DeletePartition(t *testing.T) {
|
|||
assert.Equal(t, 1, len(meta.collName2ID))
|
||||
assert.Equal(t, 1, len(meta.collID2Meta))
|
||||
assert.Equal(t, 1, len(meta.segID2Meta))
|
||||
|
||||
// delete not exist
|
||||
err = meta.DeletePartition(100, "not_exist")
|
||||
assert.NotNil(t, err)
|
||||
}
|
||||
|
||||
func TestMetaTable_Segment(t *testing.T) {
|
||||
|
@ -371,39 +366,3 @@ func TestMetaTable_UpdateSegment(t *testing.T) {
|
|||
assert.Nil(t, err)
|
||||
assert.Equal(t, seg.NumRows, int64(210))
|
||||
}
|
||||
|
||||
func TestMetaTable_AddPartition_Limit(t *testing.T) {
|
||||
Init()
|
||||
Params.MaxPartitionNum = 256 // adding 4096 partitions is too slow
|
||||
etcdAddr := Params.EtcdAddress
|
||||
|
||||
cli, err := clientv3.New(clientv3.Config{Endpoints: []string{etcdAddr}})
|
||||
assert.Nil(t, err)
|
||||
etcdKV := kv.NewEtcdKV(cli, "/etcd/test/root")
|
||||
|
||||
_, err = cli.Delete(context.TODO(), "/etcd/test/root", clientv3.WithPrefix())
|
||||
assert.Nil(t, err)
|
||||
|
||||
meta, err := NewMetaTable(etcdKV)
|
||||
assert.Nil(t, err)
|
||||
defer meta.client.Close()
|
||||
|
||||
colMeta := pb.CollectionMeta{
|
||||
ID: 100,
|
||||
Schema: &schemapb.CollectionSchema{
|
||||
Name: "coll1",
|
||||
},
|
||||
CreateTime: 0,
|
||||
SegmentIDs: []UniqueID{},
|
||||
PartitionTags: []string{},
|
||||
}
|
||||
err = meta.AddCollection(&colMeta)
|
||||
assert.Nil(t, err)
|
||||
|
||||
for i := 0; i < int(Params.MaxPartitionNum); i++ {
|
||||
err := meta.AddPartition(100, "partition_"+strconv.Itoa(i))
|
||||
assert.Nil(t, err)
|
||||
}
|
||||
err = meta.AddPartition(100, "partition_limit")
|
||||
assert.NotNil(t, err)
|
||||
}
|
||||
|
|
|
@ -43,9 +43,6 @@ type ParamTable struct {
|
|||
K2SChannelNames []string
|
||||
QueryNodeStatsChannelName string
|
||||
MsgChannelSubName string
|
||||
|
||||
MaxPartitionNum int64
|
||||
DefaultPartitionTag string
|
||||
}
|
||||
|
||||
var Params ParamTable
|
||||
|
@ -65,10 +62,6 @@ func (p *ParamTable) Init() {
|
|||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
err = p.LoadYaml("advanced/common.yaml")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
|
||||
// set members
|
||||
p.initAddress()
|
||||
|
@ -98,8 +91,6 @@ func (p *ParamTable) Init() {
|
|||
p.initK2SChannelNames()
|
||||
p.initQueryNodeStatsChannelName()
|
||||
p.initMsgChannelSubName()
|
||||
p.initMaxPartitionNum()
|
||||
p.initDefaultPartitionTag()
|
||||
}
|
||||
|
||||
func (p *ParamTable) initAddress() {
|
||||
|
@ -369,33 +360,18 @@ func (p *ParamTable) initInsertChannelNames() {
|
|||
if err != nil {
|
||||
log.Fatal(err)
|
||||
}
|
||||
channelRange, err := p.Load("msgChannel.channelRange.insert")
|
||||
id, err := p.Load("nodeID.queryNodeIDList")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
log.Panicf("load query node id list error, %s", err.Error())
|
||||
}
|
||||
|
||||
chanRange := strings.Split(channelRange, ",")
|
||||
if len(chanRange) != 2 {
|
||||
panic("Illegal channel range num")
|
||||
}
|
||||
channelBegin, err := strconv.Atoi(chanRange[0])
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
channelEnd, err := strconv.Atoi(chanRange[1])
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
if channelBegin < 0 || channelEnd < 0 {
|
||||
panic("Illegal channel range value")
|
||||
}
|
||||
if channelBegin > channelEnd {
|
||||
panic("Illegal channel range value")
|
||||
}
|
||||
|
||||
channels := make([]string, channelEnd-channelBegin)
|
||||
for i := 0; i < channelEnd-channelBegin; i++ {
|
||||
channels[i] = ch + "-" + strconv.Itoa(channelBegin+i)
|
||||
ids := strings.Split(id, ",")
|
||||
channels := make([]string, 0, len(ids))
|
||||
for _, i := range ids {
|
||||
_, err := strconv.ParseInt(i, 10, 64)
|
||||
if err != nil {
|
||||
log.Panicf("load query node id list error, %s", err.Error())
|
||||
}
|
||||
channels = append(channels, ch+"-"+i)
|
||||
}
|
||||
p.InsertChannelNames = channels
|
||||
}
|
||||
|
@ -420,24 +396,3 @@ func (p *ParamTable) initK2SChannelNames() {
|
|||
}
|
||||
p.K2SChannelNames = channels
|
||||
}
|
||||
|
||||
func (p *ParamTable) initMaxPartitionNum() {
|
||||
str, err := p.Load("master.maxPartitionNum")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
maxPartitionNum, err := strconv.ParseInt(str, 10, 64)
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
p.MaxPartitionNum = maxPartitionNum
|
||||
}
|
||||
|
||||
func (p *ParamTable) initDefaultPartitionTag() {
|
||||
defaultTag, err := p.Load("common.defaultPartitionTag")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
|
||||
p.DefaultPartitionTag = defaultTag
|
||||
}
|
||||
|
|
|
@ -31,7 +31,7 @@ func TestParamTable_EtcdRootPath(t *testing.T) {
|
|||
func TestParamTable_TopicNum(t *testing.T) {
|
||||
Params.Init()
|
||||
num := Params.TopicNum
|
||||
assert.Equal(t, num, 1)
|
||||
assert.Equal(t, num, 15)
|
||||
}
|
||||
|
||||
func TestParamTable_SegmentSize(t *testing.T) {
|
||||
|
@ -73,7 +73,7 @@ func TestParamTable_SegIDAssignExpiration(t *testing.T) {
|
|||
func TestParamTable_QueryNodeNum(t *testing.T) {
|
||||
Params.Init()
|
||||
num := Params.QueryNodeNum
|
||||
assert.Equal(t, num, 1)
|
||||
assert.Equal(t, num, 2)
|
||||
}
|
||||
|
||||
func TestParamTable_QueryNodeStatsChannelName(t *testing.T) {
|
||||
|
@ -85,15 +85,17 @@ func TestParamTable_QueryNodeStatsChannelName(t *testing.T) {
|
|||
func TestParamTable_ProxyIDList(t *testing.T) {
|
||||
Params.Init()
|
||||
ids := Params.ProxyIDList
|
||||
assert.Equal(t, len(ids), 1)
|
||||
assert.Equal(t, len(ids), 2)
|
||||
assert.Equal(t, ids[0], int64(1))
|
||||
assert.Equal(t, ids[1], int64(2))
|
||||
}
|
||||
|
||||
func TestParamTable_ProxyTimeTickChannelNames(t *testing.T) {
|
||||
Params.Init()
|
||||
names := Params.ProxyTimeTickChannelNames
|
||||
assert.Equal(t, len(names), 1)
|
||||
assert.Equal(t, len(names), 2)
|
||||
assert.Equal(t, names[0], "proxyTimeTick-1")
|
||||
assert.Equal(t, names[1], "proxyTimeTick-2")
|
||||
}
|
||||
|
||||
func TestParamTable_MsgChannelSubName(t *testing.T) {
|
||||
|
@ -111,27 +113,31 @@ func TestParamTable_SoftTimeTickBarrierInterval(t *testing.T) {
|
|||
func TestParamTable_WriteNodeIDList(t *testing.T) {
|
||||
Params.Init()
|
||||
ids := Params.WriteNodeIDList
|
||||
assert.Equal(t, len(ids), 1)
|
||||
assert.Equal(t, ids[0], int64(3))
|
||||
assert.Equal(t, len(ids), 2)
|
||||
assert.Equal(t, ids[0], int64(5))
|
||||
assert.Equal(t, ids[1], int64(6))
|
||||
}
|
||||
|
||||
func TestParamTable_WriteNodeTimeTickChannelNames(t *testing.T) {
|
||||
Params.Init()
|
||||
names := Params.WriteNodeTimeTickChannelNames
|
||||
assert.Equal(t, len(names), 1)
|
||||
assert.Equal(t, names[0], "writeNodeTimeTick-3")
|
||||
assert.Equal(t, len(names), 2)
|
||||
assert.Equal(t, names[0], "writeNodeTimeTick-5")
|
||||
assert.Equal(t, names[1], "writeNodeTimeTick-6")
|
||||
}
|
||||
|
||||
func TestParamTable_InsertChannelNames(t *testing.T) {
|
||||
Params.Init()
|
||||
names := Params.InsertChannelNames
|
||||
assert.Equal(t, len(names), 1)
|
||||
assert.Equal(t, names[0], "insert-0")
|
||||
assert.Equal(t, len(names), 2)
|
||||
assert.Equal(t, names[0], "insert-3")
|
||||
assert.Equal(t, names[1], "insert-4")
|
||||
}
|
||||
|
||||
func TestParamTable_K2SChannelNames(t *testing.T) {
|
||||
Params.Init()
|
||||
names := Params.K2SChannelNames
|
||||
assert.Equal(t, len(names), 1)
|
||||
assert.Equal(t, names[0], "k2s-3")
|
||||
assert.Equal(t, len(names), 2)
|
||||
assert.Equal(t, names[0], "k2s-5")
|
||||
assert.Equal(t, names[1], "k2s-6")
|
||||
}
|
||||
|
|
|
@ -191,12 +191,10 @@ func (t *showPartitionTask) Execute() error {
|
|||
return errors.New("null request")
|
||||
}
|
||||
|
||||
collMeta, err := t.mt.GetCollectionByName(t.req.CollectionName.CollectionName)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
partitions := make([]string, 0)
|
||||
partitions = append(partitions, collMeta.PartitionTags...)
|
||||
for _, collection := range t.mt.collID2Meta {
|
||||
partitions = append(partitions, collection.PartitionTags...)
|
||||
}
|
||||
|
||||
stringListResponse := servicepb.StringListResponse{
|
||||
Status: &commonpb.Status{
|
||||
|
|
|
@ -60,9 +60,6 @@ func TestMaster_Partition(t *testing.T) {
|
|||
K2SChannelNames: []string{"k2s0", "k2s1"},
|
||||
QueryNodeStatsChannelName: "statistic",
|
||||
MsgChannelSubName: Params.MsgChannelSubName,
|
||||
|
||||
MaxPartitionNum: int64(4096),
|
||||
DefaultPartitionTag: "_default",
|
||||
}
|
||||
|
||||
port := 10000 + rand.Intn(1000)
|
||||
|
@ -215,7 +212,7 @@ func TestMaster_Partition(t *testing.T) {
|
|||
|
||||
//assert.Equal(t, collMeta.PartitionTags[0], "partition1")
|
||||
//assert.Equal(t, collMeta.PartitionTags[1], "partition2")
|
||||
assert.ElementsMatch(t, []string{"_default", "partition1", "partition2"}, collMeta.PartitionTags)
|
||||
assert.ElementsMatch(t, []string{"default", "partition1", "partition2"}, collMeta.PartitionTags)
|
||||
|
||||
showPartitionReq := internalpb.ShowPartitionRequest{
|
||||
MsgType: internalpb.MsgType_kShowPartitions,
|
||||
|
@ -227,7 +224,7 @@ func TestMaster_Partition(t *testing.T) {
|
|||
|
||||
stringList, err := cli.ShowPartitions(ctx, &showPartitionReq)
|
||||
assert.Nil(t, err)
|
||||
assert.ElementsMatch(t, []string{"_default", "partition1", "partition2"}, stringList.Values)
|
||||
assert.ElementsMatch(t, []string{"default", "partition1", "partition2"}, stringList.Values)
|
||||
|
||||
showPartitionReq = internalpb.ShowPartitionRequest{
|
||||
MsgType: internalpb.MsgType_kShowPartitions,
|
||||
|
|
|
@ -261,9 +261,6 @@ func startupMaster() {
|
|||
K2SChannelNames: []string{"k2s0", "k2s1"},
|
||||
QueryNodeStatsChannelName: "statistic",
|
||||
MsgChannelSubName: Params.MsgChannelSubName,
|
||||
|
||||
MaxPartitionNum: int64(4096),
|
||||
DefaultPartitionTag: "_default",
|
||||
}
|
||||
|
||||
master, err = CreateServer(ctx)
|
||||
|
|
|
@ -0,0 +1,117 @@
|
|||
package master
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"log"
|
||||
"os"
|
||||
"path"
|
||||
"path/filepath"
|
||||
"strings"
|
||||
|
||||
"github.com/spf13/viper"
|
||||
"github.com/zilliztech/milvus-distributed/internal/errors"
|
||||
"github.com/zilliztech/milvus-distributed/internal/kv"
|
||||
)
|
||||
|
||||
type SysConfig struct {
|
||||
kv *kv.EtcdKV
|
||||
}
|
||||
|
||||
// Initialize Configs from config files, and store them in Etcd.
|
||||
func (conf *SysConfig) InitFromFile(filePath string) error {
|
||||
memConfigs, err := conf.getConfigFiles(filePath)
|
||||
if err != nil {
|
||||
return errors.Errorf("[Init SysConfig] %s\n", err.Error())
|
||||
}
|
||||
|
||||
for _, memConfig := range memConfigs {
|
||||
if err := conf.saveToEtcd(memConfig, "config"); err != nil {
|
||||
return errors.Errorf("[Init SysConfig] %s\n", err.Error())
|
||||
}
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (conf *SysConfig) GetByPrefix(keyPrefix string) (keys []string, values []string, err error) {
|
||||
realPrefix := path.Join("config", strings.ToLower(keyPrefix))
|
||||
keys, values, err = conf.kv.LoadWithPrefix(realPrefix)
|
||||
for index := range keys {
|
||||
keys[index] = strings.Replace(keys[index], conf.kv.GetPath("config"), "", 1)
|
||||
}
|
||||
if err != nil {
|
||||
return nil, nil, err
|
||||
}
|
||||
log.Println("Loaded", len(keys), "pairs of configs with prefix", keyPrefix)
|
||||
return keys, values, err
|
||||
}
|
||||
|
||||
// Get specific configs for keys.
|
||||
func (conf *SysConfig) Get(keys []string) ([]string, error) {
|
||||
var keysToLoad []string
|
||||
for i := range keys {
|
||||
keysToLoad = append(keysToLoad, path.Join("config", strings.ToLower(keys[i])))
|
||||
}
|
||||
|
||||
values, err := conf.kv.MultiLoad(keysToLoad)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
return values, nil
|
||||
}
|
||||
|
||||
func (conf *SysConfig) getConfigFiles(filePath string) ([]*viper.Viper, error) {
|
||||
|
||||
var vipers []*viper.Viper
|
||||
err := filepath.Walk(filePath,
|
||||
func(path string, info os.FileInfo, err error) error {
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
// all names
|
||||
if !info.IsDir() && filepath.Ext(path) == ".yaml" {
|
||||
log.Println("Config files ", info.Name())
|
||||
|
||||
currentConf := viper.New()
|
||||
currentConf.SetConfigFile(path)
|
||||
if err := currentConf.ReadInConfig(); err != nil {
|
||||
log.Panic("Config file error: ", err)
|
||||
}
|
||||
vipers = append(vipers, currentConf)
|
||||
}
|
||||
return nil
|
||||
})
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
if len(vipers) == 0 {
|
||||
return nil, errors.Errorf("There are no config files in the path `%s`.\n", filePath)
|
||||
}
|
||||
return vipers, nil
|
||||
}
|
||||
|
||||
func (conf *SysConfig) saveToEtcd(memConfig *viper.Viper, secondRootPath string) error {
|
||||
configMaps := map[string]string{}
|
||||
|
||||
allKeys := memConfig.AllKeys()
|
||||
for _, key := range allKeys {
|
||||
etcdKey := strings.ReplaceAll(key, ".", "/")
|
||||
|
||||
etcdKey = path.Join(secondRootPath, etcdKey)
|
||||
|
||||
val := memConfig.Get(key)
|
||||
if val == nil {
|
||||
configMaps[etcdKey] = ""
|
||||
continue
|
||||
}
|
||||
configMaps[etcdKey] = fmt.Sprintf("%v", val)
|
||||
}
|
||||
|
||||
if err := conf.kv.MultiSave(configMaps); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
|
@ -0,0 +1,209 @@
|
|||
package master
|
||||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"log"
|
||||
"path"
|
||||
"strings"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/zilliztech/milvus-distributed/internal/kv"
|
||||
|
||||
"github.com/spf13/viper"
|
||||
"github.com/stretchr/testify/assert"
|
||||
"github.com/stretchr/testify/require"
|
||||
"go.etcd.io/etcd/clientv3"
|
||||
)
|
||||
|
||||
func Test_SysConfig(t *testing.T) {
|
||||
Init()
|
||||
ctx, cancel := context.WithCancel(context.TODO())
|
||||
defer cancel()
|
||||
|
||||
cli, err := clientv3.New(clientv3.Config{
|
||||
Endpoints: []string{Params.EtcdAddress},
|
||||
DialTimeout: 5 * time.Second,
|
||||
})
|
||||
require.Nil(t, err)
|
||||
_, err = cli.Delete(ctx, "/test/root", clientv3.WithPrefix())
|
||||
require.Nil(t, err)
|
||||
|
||||
rootPath := "/test/root"
|
||||
configKV := kv.NewEtcdKV(cli, rootPath)
|
||||
defer configKV.Close()
|
||||
|
||||
sc := SysConfig{kv: configKV}
|
||||
require.Equal(t, rootPath, sc.kv.GetPath("."))
|
||||
|
||||
t.Run("tests on contig_test.yaml", func(t *testing.T) {
|
||||
err = sc.InitFromFile(".")
|
||||
require.Nil(t, err)
|
||||
|
||||
testKeys := []string{
|
||||
"/etcd/address",
|
||||
"/master/port",
|
||||
"/master/proxyidlist",
|
||||
"/master/segmentthresholdfactor",
|
||||
"/pulsar/token",
|
||||
"/reader/stopflag",
|
||||
"/proxy/timezone",
|
||||
"/proxy/network/address",
|
||||
"/proxy/storage/path",
|
||||
"/storage/accesskey",
|
||||
}
|
||||
|
||||
testVals := []string{
|
||||
"localhost",
|
||||
"53100",
|
||||
"[1 2]",
|
||||
"0.75",
|
||||
"eyJhbGciOiJIUzI1NiJ9.eyJzdWIiOiJKb2UifQ.ipevRNuRP6HflG8cFKnmUPtypruRC4fb1DWtoLL62SY",
|
||||
"-1",
|
||||
"UTC+8",
|
||||
"0.0.0.0",
|
||||
"/var/lib/milvus",
|
||||
"",
|
||||
}
|
||||
|
||||
vals, err := sc.Get(testKeys)
|
||||
assert.Nil(t, err)
|
||||
for i := range testVals {
|
||||
assert.Equal(t, testVals[i], vals[i])
|
||||
}
|
||||
|
||||
keys, vals, err := sc.GetByPrefix("/master")
|
||||
assert.Nil(t, err)
|
||||
for i := range keys {
|
||||
assert.True(t, strings.HasPrefix(keys[i], "/master/"))
|
||||
}
|
||||
assert.Equal(t, len(keys), len(vals))
|
||||
assert.Equal(t, 21, len(keys))
|
||||
|
||||
// Test get all configs
|
||||
keys, vals, err = sc.GetByPrefix("/")
|
||||
assert.Nil(t, err)
|
||||
assert.Equal(t, len(keys), len(vals))
|
||||
assert.Equal(t, 73, len(vals))
|
||||
|
||||
// Test get configs with prefix not exist
|
||||
keys, vals, err = sc.GetByPrefix("/config")
|
||||
assert.Nil(t, err)
|
||||
assert.Equal(t, len(keys), len(vals))
|
||||
assert.Equal(t, 0, len(keys))
|
||||
assert.Equal(t, 0, len(vals))
|
||||
|
||||
_, _, err = sc.GetByPrefix("//././../../../../../..//../")
|
||||
assert.Nil(t, err)
|
||||
_, _, err = sc.GetByPrefix("/master/./address")
|
||||
assert.Nil(t, err)
|
||||
_, _, err = sc.GetByPrefix(".")
|
||||
assert.Nil(t, err)
|
||||
_, _, err = sc.GetByPrefix("\\")
|
||||
assert.Nil(t, err)
|
||||
|
||||
})
|
||||
|
||||
t.Run("getConfigFiles", func(t *testing.T) {
|
||||
filePath := "../../configs"
|
||||
vipers, err := sc.getConfigFiles(filePath)
|
||||
assert.Nil(t, err)
|
||||
assert.NotNil(t, vipers[0])
|
||||
|
||||
filePath = "/path/not/exists"
|
||||
_, err = sc.getConfigFiles(filePath)
|
||||
assert.NotNil(t, err)
|
||||
log.Println(err)
|
||||
})
|
||||
|
||||
t.Run("Test saveToEtcd Normal", func(t *testing.T) {
|
||||
_, err = cli.Delete(ctx, "/test/root/config", clientv3.WithPrefix())
|
||||
require.Nil(t, err)
|
||||
|
||||
v := viper.New()
|
||||
|
||||
v.Set("a.suba1", "v1")
|
||||
v.Set("a.suba2", "v2")
|
||||
v.Set("a.suba3.subsuba1", "v3")
|
||||
v.Set("a.suba3.subsuba2", "v4")
|
||||
|
||||
secondRootPath := "config"
|
||||
err := sc.saveToEtcd(v, secondRootPath)
|
||||
assert.Nil(t, err)
|
||||
|
||||
value, err := sc.kv.Load(path.Join(secondRootPath, "a/suba1"))
|
||||
assert.Nil(t, err)
|
||||
assert.Equal(t, "v1", value)
|
||||
value, err = sc.kv.Load(path.Join(secondRootPath, "a/suba2"))
|
||||
assert.Nil(t, err)
|
||||
assert.Equal(t, "v2", value)
|
||||
value, err = sc.kv.Load(path.Join(secondRootPath, "a/suba3/subsuba1"))
|
||||
assert.Nil(t, err)
|
||||
assert.Equal(t, "v3", value)
|
||||
value, err = sc.kv.Load(path.Join(secondRootPath, "a/suba3/subsuba2"))
|
||||
assert.Nil(t, err)
|
||||
assert.Equal(t, "v4", value)
|
||||
|
||||
keys, values, err := sc.kv.LoadWithPrefix(path.Join(secondRootPath, "a"))
|
||||
assert.Nil(t, err)
|
||||
assert.Equal(t, 4, len(keys))
|
||||
assert.Equal(t, 4, len(values))
|
||||
assert.ElementsMatch(t, []string{
|
||||
path.Join(sc.kv.GetPath(secondRootPath), "/a/suba1"),
|
||||
path.Join(sc.kv.GetPath(secondRootPath), "/a/suba2"),
|
||||
path.Join(sc.kv.GetPath(secondRootPath), "/a/suba3/subsuba1"),
|
||||
path.Join(sc.kv.GetPath(secondRootPath), "/a/suba3/subsuba2"),
|
||||
}, keys)
|
||||
assert.ElementsMatch(t, []string{"v1", "v2", "v3", "v4"}, values)
|
||||
|
||||
keys = []string{
|
||||
"/a/suba1",
|
||||
"/a/suba2",
|
||||
"/a/suba3/subsuba1",
|
||||
"/a/suba3/subsuba2",
|
||||
}
|
||||
values, err = sc.Get(keys)
|
||||
assert.Nil(t, err)
|
||||
assert.ElementsMatch(t, []string{"v1", "v2", "v3", "v4"}, values)
|
||||
|
||||
keysAfter, values, err := sc.GetByPrefix("/a")
|
||||
fmt.Println(keysAfter)
|
||||
assert.Nil(t, err)
|
||||
assert.ElementsMatch(t, []string{"v1", "v2", "v3", "v4"}, values)
|
||||
assert.ElementsMatch(t, keys, keysAfter)
|
||||
|
||||
})
|
||||
|
||||
t.Run("Test saveToEtcd Different value types", func(t *testing.T) {
|
||||
v := viper.New()
|
||||
|
||||
v.Set("string", "string")
|
||||
v.Set("number", 1)
|
||||
v.Set("nil", nil)
|
||||
v.Set("float", 1.2)
|
||||
v.Set("intslice", []int{100, 200})
|
||||
v.Set("stringslice", []string{"a", "b"})
|
||||
v.Set("stringmapstring", map[string]string{"k1": "1", "k2": "2"})
|
||||
|
||||
secondRootPath := "test_save_to_etcd_different_value_types"
|
||||
err := sc.saveToEtcd(v, secondRootPath)
|
||||
require.Nil(t, err)
|
||||
|
||||
keys, values, err := sc.kv.LoadWithPrefix(path.Join("/", secondRootPath))
|
||||
assert.Nil(t, err)
|
||||
assert.Equal(t, 7, len(keys))
|
||||
assert.Equal(t, 7, len(values))
|
||||
|
||||
assert.ElementsMatch(t, []string{
|
||||
path.Join(sc.kv.GetPath(secondRootPath), "nil"),
|
||||
path.Join(sc.kv.GetPath(secondRootPath), "string"),
|
||||
path.Join(sc.kv.GetPath(secondRootPath), "number"),
|
||||
path.Join(sc.kv.GetPath(secondRootPath), "float"),
|
||||
path.Join(sc.kv.GetPath(secondRootPath), "intslice"),
|
||||
path.Join(sc.kv.GetPath(secondRootPath), "stringslice"),
|
||||
path.Join(sc.kv.GetPath(secondRootPath), "stringmapstring"),
|
||||
}, keys)
|
||||
assert.ElementsMatch(t, []string{"", "string", "1", "1.2", "[100 200]", "[a b]", "map[k1:1 k2:2]"}, values)
|
||||
})
|
||||
}
|
|
@ -70,7 +70,7 @@ func (ms *PulsarMsgStream) CreatePulsarProducers(channels []string) {
|
|||
for i := 0; i < len(channels); i++ {
|
||||
pp, err := (*ms.client).CreateProducer(pulsar.ProducerOptions{Topic: channels[i]})
|
||||
if err != nil {
|
||||
log.Printf("Failed to create querynode producer %s, error = %v", channels[i], err)
|
||||
log.Printf("Failed to create reader producer %s, error = %v", channels[i], err)
|
||||
}
|
||||
ms.producers = append(ms.producers, &pp)
|
||||
}
|
||||
|
|
|
@ -14,6 +14,7 @@ enum MsgType {
|
|||
kHasCollection = 102;
|
||||
kDescribeCollection = 103;
|
||||
kShowCollections = 104;
|
||||
kGetSysConfigs = 105;
|
||||
|
||||
/* Definition Requests: partition */
|
||||
kCreatePartition = 200;
|
||||
|
@ -33,6 +34,7 @@ enum MsgType {
|
|||
/* System Control */
|
||||
kTimeTick = 1200;
|
||||
kQueryNodeSegStats = 1201;
|
||||
|
||||
}
|
||||
|
||||
enum PeerRole {
|
||||
|
@ -223,6 +225,19 @@ message SearchRequest {
|
|||
|
||||
}
|
||||
|
||||
/**
|
||||
* @brief Request of DescribePartition
|
||||
*/
|
||||
message SysConfigRequest {
|
||||
MsgType msg_type = 1;
|
||||
int64 reqID = 2;
|
||||
int64 proxyID = 3;
|
||||
uint64 timestamp = 4;
|
||||
repeated string keys = 5;
|
||||
repeated string key_prefixes = 6;
|
||||
}
|
||||
|
||||
|
||||
message SearchResult {
|
||||
MsgType msg_type = 1;
|
||||
common.Status status = 2;
|
||||
|
@ -266,4 +281,4 @@ message QueryNodeSegStats {
|
|||
MsgType msg_type = 1;
|
||||
int64 peerID = 2;
|
||||
repeated SegmentStats seg_stats = 3;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -32,6 +32,7 @@ const (
|
|||
MsgType_kHasCollection MsgType = 102
|
||||
MsgType_kDescribeCollection MsgType = 103
|
||||
MsgType_kShowCollections MsgType = 104
|
||||
MsgType_kGetSysConfigs MsgType = 105
|
||||
// Definition Requests: partition
|
||||
MsgType_kCreatePartition MsgType = 200
|
||||
MsgType_kDropPartition MsgType = 201
|
||||
|
@ -56,6 +57,7 @@ var MsgType_name = map[int32]string{
|
|||
102: "kHasCollection",
|
||||
103: "kDescribeCollection",
|
||||
104: "kShowCollections",
|
||||
105: "kGetSysConfigs",
|
||||
200: "kCreatePartition",
|
||||
201: "kDropPartition",
|
||||
202: "kHasPartition",
|
||||
|
@ -76,6 +78,7 @@ var MsgType_value = map[string]int32{
|
|||
"kHasCollection": 102,
|
||||
"kDescribeCollection": 103,
|
||||
"kShowCollections": 104,
|
||||
"kGetSysConfigs": 105,
|
||||
"kCreatePartition": 200,
|
||||
"kDropPartition": 201,
|
||||
"kHasPartition": 202,
|
||||
|
@ -1579,6 +1582,87 @@ func (m *SearchRequest) GetQuery() *commonpb.Blob {
|
|||
return nil
|
||||
}
|
||||
|
||||
//*
|
||||
// @brief Request of DescribePartition
|
||||
type SysConfigRequest struct {
|
||||
MsgType MsgType `protobuf:"varint,1,opt,name=msg_type,json=msgType,proto3,enum=milvus.proto.internal.MsgType" json:"msg_type,omitempty"`
|
||||
ReqID int64 `protobuf:"varint,2,opt,name=reqID,proto3" json:"reqID,omitempty"`
|
||||
ProxyID int64 `protobuf:"varint,3,opt,name=proxyID,proto3" json:"proxyID,omitempty"`
|
||||
Timestamp uint64 `protobuf:"varint,4,opt,name=timestamp,proto3" json:"timestamp,omitempty"`
|
||||
Keys []string `protobuf:"bytes,5,rep,name=keys,proto3" json:"keys,omitempty"`
|
||||
KeyPrefixes []string `protobuf:"bytes,6,rep,name=key_prefixes,json=keyPrefixes,proto3" json:"key_prefixes,omitempty"`
|
||||
XXX_NoUnkeyedLiteral struct{} `json:"-"`
|
||||
XXX_unrecognized []byte `json:"-"`
|
||||
XXX_sizecache int32 `json:"-"`
|
||||
}
|
||||
|
||||
func (m *SysConfigRequest) Reset() { *m = SysConfigRequest{} }
|
||||
func (m *SysConfigRequest) String() string { return proto.CompactTextString(m) }
|
||||
func (*SysConfigRequest) ProtoMessage() {}
|
||||
func (*SysConfigRequest) Descriptor() ([]byte, []int) {
|
||||
return fileDescriptor_7eb37f6b80b23116, []int{21}
|
||||
}
|
||||
|
||||
func (m *SysConfigRequest) XXX_Unmarshal(b []byte) error {
|
||||
return xxx_messageInfo_SysConfigRequest.Unmarshal(m, b)
|
||||
}
|
||||
func (m *SysConfigRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
|
||||
return xxx_messageInfo_SysConfigRequest.Marshal(b, m, deterministic)
|
||||
}
|
||||
func (m *SysConfigRequest) XXX_Merge(src proto.Message) {
|
||||
xxx_messageInfo_SysConfigRequest.Merge(m, src)
|
||||
}
|
||||
func (m *SysConfigRequest) XXX_Size() int {
|
||||
return xxx_messageInfo_SysConfigRequest.Size(m)
|
||||
}
|
||||
func (m *SysConfigRequest) XXX_DiscardUnknown() {
|
||||
xxx_messageInfo_SysConfigRequest.DiscardUnknown(m)
|
||||
}
|
||||
|
||||
var xxx_messageInfo_SysConfigRequest proto.InternalMessageInfo
|
||||
|
||||
func (m *SysConfigRequest) GetMsgType() MsgType {
|
||||
if m != nil {
|
||||
return m.MsgType
|
||||
}
|
||||
return MsgType_kNone
|
||||
}
|
||||
|
||||
func (m *SysConfigRequest) GetReqID() int64 {
|
||||
if m != nil {
|
||||
return m.ReqID
|
||||
}
|
||||
return 0
|
||||
}
|
||||
|
||||
func (m *SysConfigRequest) GetProxyID() int64 {
|
||||
if m != nil {
|
||||
return m.ProxyID
|
||||
}
|
||||
return 0
|
||||
}
|
||||
|
||||
func (m *SysConfigRequest) GetTimestamp() uint64 {
|
||||
if m != nil {
|
||||
return m.Timestamp
|
||||
}
|
||||
return 0
|
||||
}
|
||||
|
||||
func (m *SysConfigRequest) GetKeys() []string {
|
||||
if m != nil {
|
||||
return m.Keys
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (m *SysConfigRequest) GetKeyPrefixes() []string {
|
||||
if m != nil {
|
||||
return m.KeyPrefixes
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
type SearchResult struct {
|
||||
MsgType MsgType `protobuf:"varint,1,opt,name=msg_type,json=msgType,proto3,enum=milvus.proto.internal.MsgType" json:"msg_type,omitempty"`
|
||||
Status *commonpb.Status `protobuf:"bytes,2,opt,name=status,proto3" json:"status,omitempty"`
|
||||
|
@ -1597,7 +1681,7 @@ func (m *SearchResult) Reset() { *m = SearchResult{} }
|
|||
func (m *SearchResult) String() string { return proto.CompactTextString(m) }
|
||||
func (*SearchResult) ProtoMessage() {}
|
||||
func (*SearchResult) Descriptor() ([]byte, []int) {
|
||||
return fileDescriptor_7eb37f6b80b23116, []int{21}
|
||||
return fileDescriptor_7eb37f6b80b23116, []int{22}
|
||||
}
|
||||
|
||||
func (m *SearchResult) XXX_Unmarshal(b []byte) error {
|
||||
|
@ -1687,7 +1771,7 @@ func (m *TimeTickMsg) Reset() { *m = TimeTickMsg{} }
|
|||
func (m *TimeTickMsg) String() string { return proto.CompactTextString(m) }
|
||||
func (*TimeTickMsg) ProtoMessage() {}
|
||||
func (*TimeTickMsg) Descriptor() ([]byte, []int) {
|
||||
return fileDescriptor_7eb37f6b80b23116, []int{22}
|
||||
return fileDescriptor_7eb37f6b80b23116, []int{23}
|
||||
}
|
||||
|
||||
func (m *TimeTickMsg) XXX_Unmarshal(b []byte) error {
|
||||
|
@ -1744,7 +1828,7 @@ func (m *Key2Seg) Reset() { *m = Key2Seg{} }
|
|||
func (m *Key2Seg) String() string { return proto.CompactTextString(m) }
|
||||
func (*Key2Seg) ProtoMessage() {}
|
||||
func (*Key2Seg) Descriptor() ([]byte, []int) {
|
||||
return fileDescriptor_7eb37f6b80b23116, []int{23}
|
||||
return fileDescriptor_7eb37f6b80b23116, []int{24}
|
||||
}
|
||||
|
||||
func (m *Key2Seg) XXX_Unmarshal(b []byte) error {
|
||||
|
@ -1812,7 +1896,7 @@ func (m *Key2SegMsg) Reset() { *m = Key2SegMsg{} }
|
|||
func (m *Key2SegMsg) String() string { return proto.CompactTextString(m) }
|
||||
func (*Key2SegMsg) ProtoMessage() {}
|
||||
func (*Key2SegMsg) Descriptor() ([]byte, []int) {
|
||||
return fileDescriptor_7eb37f6b80b23116, []int{24}
|
||||
return fileDescriptor_7eb37f6b80b23116, []int{25}
|
||||
}
|
||||
|
||||
func (m *Key2SegMsg) XXX_Unmarshal(b []byte) error {
|
||||
|
@ -1861,7 +1945,7 @@ func (m *SegmentStats) Reset() { *m = SegmentStats{} }
|
|||
func (m *SegmentStats) String() string { return proto.CompactTextString(m) }
|
||||
func (*SegmentStats) ProtoMessage() {}
|
||||
func (*SegmentStats) Descriptor() ([]byte, []int) {
|
||||
return fileDescriptor_7eb37f6b80b23116, []int{25}
|
||||
return fileDescriptor_7eb37f6b80b23116, []int{26}
|
||||
}
|
||||
|
||||
func (m *SegmentStats) XXX_Unmarshal(b []byte) error {
|
||||
|
@ -1923,7 +2007,7 @@ func (m *QueryNodeSegStats) Reset() { *m = QueryNodeSegStats{} }
|
|||
func (m *QueryNodeSegStats) String() string { return proto.CompactTextString(m) }
|
||||
func (*QueryNodeSegStats) ProtoMessage() {}
|
||||
func (*QueryNodeSegStats) Descriptor() ([]byte, []int) {
|
||||
return fileDescriptor_7eb37f6b80b23116, []int{26}
|
||||
return fileDescriptor_7eb37f6b80b23116, []int{27}
|
||||
}
|
||||
|
||||
func (m *QueryNodeSegStats) XXX_Unmarshal(b []byte) error {
|
||||
|
@ -1989,6 +2073,7 @@ func init() {
|
|||
proto.RegisterType((*InsertRequest)(nil), "milvus.proto.internal.InsertRequest")
|
||||
proto.RegisterType((*DeleteRequest)(nil), "milvus.proto.internal.DeleteRequest")
|
||||
proto.RegisterType((*SearchRequest)(nil), "milvus.proto.internal.SearchRequest")
|
||||
proto.RegisterType((*SysConfigRequest)(nil), "milvus.proto.internal.SysConfigRequest")
|
||||
proto.RegisterType((*SearchResult)(nil), "milvus.proto.internal.SearchResult")
|
||||
proto.RegisterType((*TimeTickMsg)(nil), "milvus.proto.internal.TimeTickMsg")
|
||||
proto.RegisterType((*Key2Seg)(nil), "milvus.proto.internal.Key2Seg")
|
||||
|
@ -2000,95 +2085,98 @@ func init() {
|
|||
func init() { proto.RegisterFile("internal_msg.proto", fileDescriptor_7eb37f6b80b23116) }
|
||||
|
||||
var fileDescriptor_7eb37f6b80b23116 = []byte{
|
||||
// 1427 bytes of a gzipped FileDescriptorProto
|
||||
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe4, 0x58, 0x4d, 0x6f, 0x1c, 0x45,
|
||||
0x13, 0x4e, 0xef, 0xac, 0xf7, 0xa3, 0xd6, 0x5e, 0x8f, 0xdb, 0x76, 0xb2, 0x49, 0x5e, 0x25, 0xce,
|
||||
0xe4, 0x15, 0x31, 0x41, 0xd8, 0x62, 0xc3, 0x81, 0xdc, 0x20, 0xde, 0x43, 0x96, 0xc8, 0x51, 0x18,
|
||||
0x5b, 0x20, 0xa1, 0x48, 0xa3, 0xf1, 0x6e, 0x65, 0x77, 0x34, 0x5f, 0xeb, 0xee, 0x59, 0x3b, 0xeb,
|
||||
0x03, 0xa7, 0xfc, 0x00, 0x38, 0x70, 0xe0, 0x80, 0xc4, 0x91, 0x13, 0x82, 0x7f, 0x01, 0x81, 0x2b,
|
||||
0x07, 0xfe, 0x02, 0x08, 0x22, 0x41, 0xb8, 0xa3, 0xee, 0x9e, 0x8f, 0x1d, 0x7b, 0x6d, 0x47, 0x4a,
|
||||
0x0c, 0x96, 0x7c, 0xdb, 0xaa, 0xad, 0xe9, 0xaa, 0x7a, 0x9e, 0xee, 0xea, 0xaa, 0x06, 0xea, 0x04,
|
||||
0x11, 0xb2, 0xc0, 0xf6, 0x2c, 0x9f, 0xf7, 0x56, 0x06, 0x2c, 0x8c, 0x42, 0xba, 0xe8, 0x3b, 0xde,
|
||||
0xce, 0x90, 0x2b, 0x69, 0x25, 0x31, 0xb8, 0x34, 0xdd, 0x09, 0x7d, 0x3f, 0x0c, 0x94, 0xfa, 0xd2,
|
||||
0x1c, 0x47, 0xb6, 0xe3, 0x74, 0x30, 0xfb, 0xce, 0x08, 0xa0, 0xda, 0x6e, 0x99, 0xb8, 0x3d, 0x44,
|
||||
0x1e, 0xd1, 0xf3, 0x50, 0x1a, 0x20, 0xb2, 0x76, 0xab, 0x41, 0x96, 0xc8, 0xb2, 0x66, 0xc6, 0x12,
|
||||
0xbd, 0x05, 0x45, 0x16, 0x7a, 0xd8, 0x28, 0x2c, 0x91, 0xe5, 0x7a, 0xf3, 0xea, 0xca, 0x44, 0x5f,
|
||||
0x2b, 0x0f, 0x10, 0x99, 0x19, 0x7a, 0x68, 0x4a, 0x63, 0xba, 0x00, 0x53, 0x9d, 0x70, 0x18, 0x44,
|
||||
0x0d, 0x6d, 0x89, 0x2c, 0xcf, 0x98, 0x4a, 0x30, 0x7a, 0x00, 0xc2, 0x1f, 0x1f, 0x84, 0x01, 0x47,
|
||||
0x7a, 0x0b, 0x4a, 0x3c, 0xb2, 0xa3, 0x21, 0x97, 0x0e, 0x6b, 0xcd, 0xcb, 0xf9, 0xa5, 0xe3, 0xe0,
|
||||
0x37, 0xa4, 0x89, 0x19, 0x9b, 0xd2, 0x3a, 0x14, 0xda, 0x2d, 0x19, 0x8b, 0x66, 0x16, 0xda, 0xad,
|
||||
0x43, 0x1c, 0x85, 0x00, 0x9b, 0x3c, 0xfc, 0x17, 0x33, 0xdb, 0x81, 0x9a, 0x74, 0xf8, 0x32, 0xa9,
|
||||
0xfd, 0x0f, 0xaa, 0x91, 0xe3, 0x23, 0x8f, 0x6c, 0x7f, 0x20, 0x63, 0x2a, 0x9a, 0x99, 0xe2, 0x10,
|
||||
0xbf, 0x4f, 0x08, 0x4c, 0x6f, 0x60, 0x2f, 0x63, 0x31, 0x35, 0x23, 0x63, 0x66, 0x62, 0xe9, 0x4e,
|
||||
0xdf, 0x0e, 0x02, 0xf4, 0x62, 0xf0, 0xa6, 0xcc, 0x4c, 0x41, 0x2f, 0x43, 0xb5, 0x13, 0x7a, 0x9e,
|
||||
0x15, 0xd8, 0x3e, 0xca, 0xe5, 0xab, 0x66, 0x45, 0x28, 0xee, 0xdb, 0x3e, 0xd2, 0xeb, 0x30, 0x33,
|
||||
0xb0, 0x59, 0xe4, 0x44, 0x4e, 0x18, 0x58, 0x91, 0xdd, 0x6b, 0x14, 0xa5, 0xc1, 0x74, 0xaa, 0xdc,
|
||||
0xb4, 0x7b, 0xc6, 0x37, 0x04, 0xe8, 0x7b, 0x9c, 0x3b, 0xbd, 0x20, 0x17, 0xcc, 0x2b, 0x05, 0xfe,
|
||||
0x1e, 0xcc, 0x0e, 0x90, 0x59, 0x71, 0xd8, 0x16, 0xc3, 0xed, 0x86, 0xb6, 0xa4, 0x2d, 0xd7, 0x9a,
|
||||
0xd7, 0x0f, 0xf9, 0x7e, 0x3c, 0x14, 0x73, 0x66, 0x80, 0x6c, 0x4d, 0x7d, 0x6a, 0xe2, 0xb6, 0xf1,
|
||||
0x25, 0x81, 0x59, 0xf9, 0xbf, 0x8a, 0xda, 0xc7, 0x40, 0x42, 0xc7, 0x85, 0x2a, 0x0e, 0x56, 0x09,
|
||||
0xc7, 0x40, 0x37, 0x91, 0x95, 0x3c, 0xa0, 0xc5, 0xe3, 0x00, 0x9d, 0x9a, 0x00, 0xe8, 0x33, 0x02,
|
||||
0xf3, 0x39, 0x40, 0x4f, 0x6e, 0x63, 0xdd, 0x80, 0x59, 0x7c, 0x3c, 0x70, 0x18, 0x5a, 0xdd, 0x21,
|
||||
0xb3, 0x45, 0x00, 0x32, 0x99, 0xa2, 0x59, 0x57, 0xea, 0x56, 0xac, 0xa5, 0x0f, 0xe1, 0xfc, 0x38,
|
||||
0x01, 0x76, 0x8a, 0x5c, 0xa3, 0x28, 0x79, 0x78, 0xed, 0x28, 0x1e, 0x32, 0x9c, 0xcd, 0x85, 0x8c,
|
||||
0x8a, 0x4c, 0x6b, 0xfc, 0x4c, 0xe0, 0xc2, 0x1a, 0x43, 0x3b, 0xc2, 0xb5, 0xd0, 0xf3, 0xb0, 0x23,
|
||||
0x5c, 0x26, 0xfb, 0xe8, 0x36, 0x54, 0x7c, 0xde, 0xb3, 0xa2, 0xd1, 0x00, 0x65, 0xde, 0xf5, 0xe6,
|
||||
0x95, 0x43, 0x7c, 0xad, 0xf3, 0xde, 0xe6, 0x68, 0x80, 0x66, 0xd9, 0x57, 0x3f, 0x04, 0x41, 0x0c,
|
||||
0xb7, 0xd3, 0x92, 0xa1, 0x84, 0x3c, 0x22, 0xda, 0x7e, 0x44, 0x1a, 0x50, 0x1e, 0xb0, 0xf0, 0xf1,
|
||||
0xa8, 0xdd, 0x92, 0xe4, 0x69, 0x66, 0x22, 0xd2, 0xb7, 0xa0, 0xc4, 0x3b, 0x7d, 0xf4, 0x6d, 0x49,
|
||||
0x5a, 0xad, 0x79, 0x71, 0x22, 0xfc, 0x77, 0xbc, 0x70, 0xcb, 0x8c, 0x0d, 0x05, 0x93, 0x8b, 0x2d,
|
||||
0x16, 0x0e, 0x4e, 0x71, 0x56, 0xeb, 0x30, 0xdb, 0x49, 0xa3, 0x53, 0x9b, 0x56, 0xa5, 0xf7, 0xff,
|
||||
0x7c, 0x3c, 0xf1, 0x05, 0xb2, 0x92, 0xa5, 0x22, 0x36, 0xb4, 0x59, 0xef, 0xe4, 0x64, 0xe3, 0x0f,
|
||||
0x02, 0x0b, 0x77, 0x6d, 0x7e, 0x76, 0x12, 0xfe, 0x8b, 0xc0, 0xc5, 0x16, 0xf2, 0x0e, 0x73, 0xb6,
|
||||
0xf0, 0xec, 0x64, 0xfd, 0x15, 0x81, 0xc5, 0x8d, 0x7e, 0xb8, 0x7b, 0x7a, 0x33, 0x36, 0x7e, 0x27,
|
||||
0x70, 0x5e, 0xd5, 0x94, 0x07, 0x49, 0x71, 0x3d, 0x75, 0xac, 0xbc, 0x0f, 0xf5, 0xec, 0x3a, 0x18,
|
||||
0x23, 0xe5, 0xfa, 0x64, 0x52, 0xd2, 0x44, 0x24, 0x27, 0xd9, 0x4d, 0x22, 0x29, 0xf9, 0x8d, 0xc0,
|
||||
0x82, 0xa8, 0x35, 0x67, 0x23, 0xdb, 0x5f, 0x09, 0xcc, 0xdf, 0xb5, 0xf9, 0xd9, 0x48, 0xf6, 0x19,
|
||||
0x81, 0x46, 0x52, 0x63, 0xce, 0x46, 0xc6, 0xe2, 0x1a, 0x11, 0xf5, 0xe5, 0xf4, 0x66, 0xfb, 0x8a,
|
||||
0x0b, 0xea, 0x9f, 0x05, 0x98, 0x69, 0x07, 0x1c, 0x59, 0x74, 0x62, 0x99, 0xde, 0x38, 0x18, 0xb1,
|
||||
0xea, 0xf7, 0xf7, 0xc5, 0xf2, 0x42, 0x5d, 0xbf, 0xc0, 0x8d, 0x63, 0x4f, 0x74, 0x6f, 0xed, 0x96,
|
||||
0xcc, 0x5c, 0x33, 0x33, 0x45, 0xbe, 0x71, 0x2e, 0xa9, 0x7f, 0xb3, 0xc6, 0x79, 0x0c, 0xd5, 0x72,
|
||||
0x1e, 0xd5, 0x2b, 0x00, 0x29, 0xf8, 0xbc, 0x51, 0x59, 0xd2, 0x96, 0x8b, 0xe6, 0x98, 0x46, 0x0c,
|
||||
0x15, 0x2c, 0xdc, 0x6d, 0xb7, 0x78, 0xa3, 0xba, 0xa4, 0x89, 0xa1, 0x42, 0x49, 0xf4, 0x6d, 0xa8,
|
||||
0xb0, 0x70, 0xd7, 0xea, 0xda, 0x91, 0xdd, 0x00, 0xd9, 0x90, 0x1e, 0xd1, 0x9d, 0x95, 0x59, 0xb8,
|
||||
0xdb, 0xb2, 0x23, 0xdb, 0x78, 0x52, 0x80, 0x99, 0x16, 0x7a, 0x18, 0xe1, 0x7f, 0x0f, 0x7a, 0x0e,
|
||||
0xb1, 0xe2, 0x11, 0x88, 0x4d, 0x1d, 0x85, 0x58, 0xe9, 0x00, 0x62, 0xd7, 0x60, 0x7a, 0xc0, 0x1c,
|
||||
0xdf, 0x66, 0x23, 0xcb, 0xc5, 0x11, 0x6f, 0x94, 0x25, 0x6e, 0xb5, 0x58, 0x77, 0x0f, 0x47, 0xdc,
|
||||
0x78, 0x4e, 0x60, 0x66, 0x03, 0x6d, 0xd6, 0xe9, 0x9f, 0x18, 0x0c, 0x63, 0xf1, 0x6b, 0xf9, 0xf8,
|
||||
0x73, 0xe7, 0xaf, 0xb8, 0xff, 0xfc, 0xbd, 0x0e, 0x3a, 0x43, 0x3e, 0xf4, 0x22, 0x2b, 0x03, 0x47,
|
||||
0x01, 0x30, 0xab, 0xf4, 0x6b, 0x29, 0x44, 0xab, 0x30, 0xb5, 0x3d, 0x44, 0x36, 0x92, 0xdb, 0xed,
|
||||
0x48, 0xfe, 0x95, 0x9d, 0xf1, 0xb4, 0x20, 0xc6, 0x67, 0x95, 0xb6, 0x58, 0xea, 0x65, 0xb2, 0xce,
|
||||
0x46, 0xb3, 0xc2, 0x8b, 0x8f, 0x66, 0x29, 0x54, 0xda, 0x21, 0x50, 0xed, 0x2b, 0x39, 0xd7, 0x60,
|
||||
0x5a, 0x46, 0x6e, 0x05, 0x61, 0x17, 0x53, 0x20, 0x6a, 0x52, 0x77, 0x5f, 0xaa, 0xf2, 0x68, 0x96,
|
||||
0x5e, 0x04, 0xcd, 0xf2, 0x64, 0x34, 0x57, 0xa0, 0xd8, 0x77, 0x22, 0x75, 0x04, 0x6b, 0xcd, 0x4b,
|
||||
0x93, 0x6b, 0xda, 0x5d, 0x27, 0xe2, 0xa6, 0xb4, 0x33, 0x3e, 0x81, 0xda, 0xa6, 0xe3, 0xe3, 0xa6,
|
||||
0xd3, 0x71, 0xd7, 0x79, 0xef, 0x65, 0xa0, 0xcc, 0xde, 0x0d, 0x0a, 0xb9, 0x77, 0x83, 0x23, 0x0b,
|
||||
0xb5, 0xf1, 0x05, 0x81, 0xf2, 0x3d, 0x1c, 0x35, 0x37, 0xb0, 0x27, 0x71, 0x15, 0x65, 0x21, 0x99,
|
||||
0xe5, 0xa5, 0x40, 0xaf, 0x42, 0x6d, 0xec, 0x20, 0xc4, 0x8b, 0x43, 0x76, 0x0e, 0x8e, 0xb9, 0x09,
|
||||
0x2e, 0x42, 0xc5, 0xe1, 0xd6, 0x8e, 0xed, 0x39, 0x5d, 0xc9, 0x4b, 0xc5, 0x2c, 0x3b, 0xfc, 0x43,
|
||||
0x21, 0x8a, 0x23, 0x98, 0x56, 0x3e, 0xde, 0x98, 0x92, 0x07, 0x6c, 0x4c, 0x63, 0x3c, 0x04, 0x88,
|
||||
0x43, 0x13, 0xd0, 0xa4, 0xac, 0x93, 0x71, 0xd6, 0xdf, 0x81, 0xb2, 0x8b, 0xa3, 0x26, 0xc7, 0x5e,
|
||||
0xa3, 0x20, 0x21, 0x3f, 0x0c, 0xaf, 0x78, 0x25, 0x33, 0x31, 0x37, 0x3e, 0x57, 0xaf, 0x40, 0xc2,
|
||||
0x99, 0xd8, 0x5f, 0x3c, 0x5f, 0x99, 0xc9, 0xfe, 0xca, 0x7c, 0x15, 0x6a, 0x3e, 0xfa, 0x21, 0x1b,
|
||||
0x59, 0xdc, 0xd9, 0xc3, 0x04, 0x06, 0xa5, 0xda, 0x70, 0xf6, 0x50, 0x24, 0x1a, 0x0c, 0x7d, 0x8b,
|
||||
0x85, 0xbb, 0x3c, 0x39, 0xab, 0xc1, 0xd0, 0x37, 0xc3, 0x5d, 0x4e, 0xdf, 0x80, 0x39, 0x86, 0x1d,
|
||||
0x0c, 0x22, 0x6f, 0x64, 0xf9, 0x61, 0xd7, 0x79, 0xe4, 0x60, 0x02, 0x86, 0x9e, 0xfc, 0xb1, 0x1e,
|
||||
0xeb, 0x8d, 0xaf, 0x09, 0xcc, 0x7d, 0x90, 0x6c, 0xcd, 0x0d, 0xec, 0xa9, 0xe0, 0x4e, 0x60, 0x63,
|
||||
0xbc, 0x2b, 0xf3, 0xb5, 0xc4, 0xa1, 0xe2, 0xc7, 0xbf, 0x0a, 0xa5, 0x38, 0x99, 0x15, 0x1e, 0x07,
|
||||
0x75, 0xf3, 0x97, 0x02, 0x94, 0x63, 0x77, 0xb4, 0x0a, 0x53, 0xee, 0xfd, 0x30, 0x40, 0xfd, 0x1c,
|
||||
0x5d, 0x84, 0x39, 0x77, 0xff, 0xab, 0x84, 0xde, 0xa5, 0xf3, 0x30, 0xeb, 0xe6, 0x87, 0x7a, 0x1d,
|
||||
0x29, 0x85, 0xba, 0x9b, 0x9b, 0x7b, 0xf5, 0x47, 0xf4, 0x02, 0xcc, 0xbb, 0x07, 0x47, 0x43, 0x5d,
|
||||
0x6c, 0x01, 0xdd, 0xcd, 0x4f, 0x4f, 0x5c, 0xef, 0xd3, 0x45, 0xd0, 0xdd, 0x7d, 0x03, 0x8b, 0xfe,
|
||||
0x3d, 0xa1, 0xf3, 0x50, 0x77, 0x73, 0x7d, 0xbd, 0xfe, 0x03, 0xa1, 0x14, 0x66, 0xdc, 0xf1, 0xf6,
|
||||
0x57, 0x7f, 0x4a, 0xe8, 0x05, 0xa0, 0xee, 0x81, 0x2e, 0x51, 0xff, 0x91, 0xd0, 0x05, 0x98, 0x75,
|
||||
0x73, 0xcd, 0x14, 0xd7, 0x7f, 0x22, 0x74, 0x1a, 0xca, 0xae, 0xea, 0x38, 0xf4, 0x4f, 0x35, 0x29,
|
||||
0xa9, 0xab, 0x50, 0xff, 0x4c, 0x49, 0xaa, 0x34, 0xea, 0xcf, 0x35, 0xe9, 0x6c, 0xbc, 0x50, 0xea,
|
||||
0x7f, 0x6b, 0xb4, 0x0e, 0x55, 0x37, 0x39, 0xf0, 0xfa, 0xb7, 0x55, 0xe9, 0xfc, 0x00, 0xdb, 0xfa,
|
||||
0x77, 0xd5, 0x9b, 0xb7, 0xa1, 0x92, 0xbc, 0xe5, 0x51, 0x80, 0xd2, 0xba, 0xcd, 0x23, 0x64, 0xfa,
|
||||
0x39, 0xf1, 0xdb, 0x44, 0xbb, 0x8b, 0x4c, 0x27, 0xe2, 0xf7, 0x47, 0xcc, 0x11, 0xfa, 0x82, 0xc0,
|
||||
0xff, 0x81, 0xa8, 0x77, 0xba, 0x76, 0xa7, 0xf5, 0xf1, 0x9d, 0x9e, 0x13, 0xf5, 0x87, 0x5b, 0xa2,
|
||||
0x7e, 0xae, 0xee, 0x39, 0x9e, 0xe7, 0xec, 0x45, 0xd8, 0xe9, 0xaf, 0x2a, 0x72, 0xdf, 0xec, 0x3a,
|
||||
0x3c, 0x62, 0xce, 0xd6, 0x30, 0xc2, 0xee, 0x6a, 0x42, 0xf1, 0xaa, 0x64, 0x3c, 0x15, 0x07, 0x5b,
|
||||
0x5b, 0x25, 0xa9, 0xb9, 0xf5, 0x4f, 0x00, 0x00, 0x00, 0xff, 0xff, 0xdf, 0x3a, 0xa9, 0xbc, 0x3c,
|
||||
0x17, 0x00, 0x00,
|
||||
// 1482 bytes of a gzipped FileDescriptorProto
|
||||
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe4, 0x58, 0x4b, 0x6f, 0x1b, 0x47,
|
||||
0x12, 0xf6, 0x70, 0x28, 0x3e, 0x8a, 0x12, 0x35, 0x6a, 0x49, 0x36, 0x6d, 0x2f, 0x6c, 0x79, 0xbc,
|
||||
0x58, 0x6b, 0xbd, 0x58, 0x09, 0x4b, 0xef, 0x61, 0x7d, 0xdb, 0xb5, 0x08, 0xac, 0xb9, 0x86, 0x0c,
|
||||
0xed, 0x50, 0x48, 0x80, 0xc0, 0xc0, 0x60, 0x44, 0x96, 0x86, 0x8d, 0x79, 0xaa, 0x7b, 0x28, 0x99,
|
||||
0x3a, 0xe4, 0xe4, 0x1f, 0x90, 0x1c, 0x72, 0xc8, 0x21, 0x40, 0x8e, 0x39, 0x05, 0xc9, 0xbf, 0x48,
|
||||
0x9c, 0x9c, 0x02, 0xe4, 0x4f, 0x24, 0x70, 0x0c, 0x24, 0xce, 0x3d, 0xe8, 0x9e, 0x17, 0x47, 0x4f,
|
||||
0x03, 0xb6, 0x12, 0x01, 0xba, 0x4d, 0x15, 0x8b, 0x5d, 0x55, 0xdf, 0x57, 0x5d, 0x5d, 0xdd, 0x40,
|
||||
0xa8, 0x1f, 0x21, 0xf3, 0x2d, 0xd7, 0xf4, 0xb8, 0xbd, 0x12, 0xb2, 0x20, 0x0a, 0xc8, 0xa2, 0x47,
|
||||
0xdd, 0xdd, 0x11, 0x8f, 0xa5, 0x95, 0xd4, 0xe0, 0xda, 0x74, 0x3f, 0xf0, 0xbc, 0xc0, 0x8f, 0xd5,
|
||||
0xd7, 0xe6, 0x38, 0xb2, 0x5d, 0xda, 0xc7, 0xfc, 0x7f, 0xba, 0x0f, 0xf5, 0x6e, 0xc7, 0xc0, 0x9d,
|
||||
0x11, 0xf2, 0x88, 0x5c, 0x86, 0x4a, 0x88, 0xc8, 0xba, 0x9d, 0x96, 0xb2, 0xa4, 0x2c, 0xab, 0x46,
|
||||
0x22, 0x91, 0x7b, 0x50, 0x66, 0x81, 0x8b, 0xad, 0xd2, 0x92, 0xb2, 0xdc, 0x6c, 0xdf, 0x5c, 0x39,
|
||||
0xd2, 0xd7, 0xca, 0x06, 0x22, 0x33, 0x02, 0x17, 0x0d, 0x69, 0x4c, 0x16, 0x60, 0xaa, 0x1f, 0x8c,
|
||||
0xfc, 0xa8, 0xa5, 0x2e, 0x29, 0xcb, 0x33, 0x46, 0x2c, 0xe8, 0x36, 0x80, 0xf0, 0xc7, 0xc3, 0xc0,
|
||||
0xe7, 0x48, 0xee, 0x41, 0x85, 0x47, 0x56, 0x34, 0xe2, 0xd2, 0x61, 0xa3, 0x7d, 0xbd, 0xb8, 0x74,
|
||||
0x12, 0x7c, 0x4f, 0x9a, 0x18, 0x89, 0x29, 0x69, 0x42, 0xa9, 0xdb, 0x91, 0xb1, 0xa8, 0x46, 0xa9,
|
||||
0xdb, 0x39, 0xc6, 0x51, 0x00, 0xb0, 0xc9, 0x83, 0xdf, 0x31, 0xb3, 0x5d, 0x68, 0x48, 0x87, 0x6f,
|
||||
0x92, 0xda, 0x9f, 0xa0, 0x1e, 0x51, 0x0f, 0x79, 0x64, 0x79, 0xa1, 0x8c, 0xa9, 0x6c, 0xe4, 0x8a,
|
||||
0x63, 0xfc, 0x3e, 0x53, 0x60, 0xba, 0x87, 0x76, 0xce, 0x62, 0x66, 0xa6, 0x4c, 0x98, 0x89, 0xa5,
|
||||
0xfb, 0x43, 0xcb, 0xf7, 0xd1, 0x4d, 0xc0, 0x9b, 0x32, 0x72, 0x05, 0xb9, 0x0e, 0xf5, 0x7e, 0xe0,
|
||||
0xba, 0xa6, 0x6f, 0x79, 0x28, 0x97, 0xaf, 0x1b, 0x35, 0xa1, 0x78, 0x6c, 0x79, 0x48, 0x6e, 0xc3,
|
||||
0x4c, 0x68, 0xb1, 0x88, 0x46, 0x34, 0xf0, 0xcd, 0xc8, 0xb2, 0x5b, 0x65, 0x69, 0x30, 0x9d, 0x29,
|
||||
0x37, 0x2d, 0x5b, 0xff, 0x5c, 0x01, 0xf2, 0x1f, 0xce, 0xa9, 0xed, 0x17, 0x82, 0x79, 0xab, 0xc0,
|
||||
0x3f, 0x82, 0xd9, 0x10, 0x99, 0x99, 0x84, 0x6d, 0x32, 0xdc, 0x69, 0xa9, 0x4b, 0xea, 0x72, 0xa3,
|
||||
0x7d, 0xfb, 0x98, 0xff, 0x4f, 0x86, 0x62, 0xcc, 0x84, 0xc8, 0xd6, 0xe2, 0xbf, 0x1a, 0xb8, 0xa3,
|
||||
0x7f, 0xa2, 0xc0, 0xac, 0xfc, 0x3d, 0x8e, 0xda, 0x43, 0x5f, 0x42, 0xc7, 0x85, 0x2a, 0x09, 0x36,
|
||||
0x16, 0x4e, 0x81, 0xee, 0x48, 0x56, 0x8a, 0x80, 0x96, 0x4f, 0x03, 0x74, 0xea, 0x08, 0x40, 0x5f,
|
||||
0x2a, 0x30, 0x5f, 0x00, 0xf4, 0xec, 0x0a, 0xeb, 0x0e, 0xcc, 0xe2, 0xd3, 0x90, 0x32, 0x34, 0x07,
|
||||
0x23, 0x66, 0x89, 0x00, 0x64, 0x32, 0x65, 0xa3, 0x19, 0xab, 0x3b, 0x89, 0x96, 0x3c, 0x81, 0xcb,
|
||||
0x93, 0x04, 0x58, 0x19, 0x72, 0xad, 0xb2, 0xe4, 0xe1, 0x2f, 0x27, 0xf1, 0x90, 0xe3, 0x6c, 0x2c,
|
||||
0xe4, 0x54, 0xe4, 0x5a, 0xfd, 0x7b, 0x05, 0xae, 0xac, 0x31, 0xb4, 0x22, 0x5c, 0x0b, 0x5c, 0x17,
|
||||
0xfb, 0xc2, 0x65, 0x5a, 0x47, 0xf7, 0xa1, 0xe6, 0x71, 0xdb, 0x8c, 0xc6, 0x21, 0xca, 0xbc, 0x9b,
|
||||
0xed, 0x1b, 0xc7, 0xf8, 0x5a, 0xe7, 0xf6, 0xe6, 0x38, 0x44, 0xa3, 0xea, 0xc5, 0x1f, 0x82, 0x20,
|
||||
0x86, 0x3b, 0x59, 0xcb, 0x88, 0x85, 0x22, 0x22, 0xea, 0x41, 0x44, 0x5a, 0x50, 0x0d, 0x59, 0xf0,
|
||||
0x74, 0xdc, 0xed, 0x48, 0xf2, 0x54, 0x23, 0x15, 0xc9, 0x3f, 0xa0, 0xc2, 0xfb, 0x43, 0xf4, 0x2c,
|
||||
0x49, 0x5a, 0xa3, 0x7d, 0xf5, 0x48, 0xf8, 0x1f, 0xb8, 0xc1, 0x96, 0x91, 0x18, 0x0a, 0x26, 0x17,
|
||||
0x3b, 0x2c, 0x08, 0xcf, 0x71, 0x56, 0xeb, 0x30, 0xdb, 0xcf, 0xa2, 0x8b, 0x8b, 0x36, 0x4e, 0xef,
|
||||
0xcf, 0xc5, 0x78, 0x92, 0x03, 0x64, 0x25, 0x4f, 0x45, 0x14, 0xb4, 0xd1, 0xec, 0x17, 0x64, 0xfd,
|
||||
0x27, 0x05, 0x16, 0x1e, 0x5a, 0xfc, 0xe2, 0x24, 0xfc, 0x8b, 0x02, 0x57, 0x3b, 0xc8, 0xfb, 0x8c,
|
||||
0x6e, 0xe1, 0xc5, 0xc9, 0xfa, 0x53, 0x05, 0x16, 0x7b, 0xc3, 0x60, 0xef, 0xfc, 0x66, 0xac, 0xbf,
|
||||
0x50, 0xe0, 0x72, 0xdc, 0x53, 0x36, 0xd2, 0xe6, 0x7a, 0xee, 0x58, 0xf9, 0x1f, 0x34, 0xf3, 0xe3,
|
||||
0x60, 0x82, 0x94, 0xdb, 0x47, 0x93, 0x92, 0x25, 0x22, 0x39, 0xc9, 0x4f, 0x12, 0x49, 0xc9, 0x8f,
|
||||
0x0a, 0x2c, 0x88, 0x5e, 0x73, 0x31, 0xb2, 0xfd, 0x41, 0x81, 0xf9, 0x87, 0x16, 0xbf, 0x18, 0xc9,
|
||||
0xbe, 0x54, 0xa0, 0x95, 0xf6, 0x98, 0x8b, 0x91, 0xb1, 0x38, 0x46, 0x44, 0x7f, 0x39, 0xbf, 0xd9,
|
||||
0xbe, 0xe5, 0x86, 0xfa, 0x73, 0x09, 0x66, 0xba, 0x3e, 0x47, 0x16, 0x9d, 0x59, 0xa6, 0x77, 0x0e,
|
||||
0x47, 0x1c, 0xcf, 0xfb, 0x07, 0x62, 0x79, 0xad, 0xa9, 0x5f, 0xe0, 0xc6, 0xd1, 0x16, 0xd3, 0x5b,
|
||||
0xb7, 0x23, 0x33, 0x57, 0x8d, 0x5c, 0x51, 0x1c, 0x9c, 0x2b, 0xf1, 0xaf, 0xf9, 0xe0, 0x3c, 0x81,
|
||||
0x6a, 0xb5, 0x88, 0xea, 0x0d, 0x80, 0x0c, 0x7c, 0xde, 0xaa, 0x2d, 0xa9, 0xcb, 0x65, 0x63, 0x42,
|
||||
0x23, 0x2e, 0x15, 0x2c, 0xd8, 0xeb, 0x76, 0x78, 0xab, 0xbe, 0xa4, 0x8a, 0x4b, 0x45, 0x2c, 0x91,
|
||||
0x7f, 0x42, 0x8d, 0x05, 0x7b, 0xe6, 0xc0, 0x8a, 0xac, 0x16, 0xc8, 0x81, 0xf4, 0x84, 0xe9, 0xac,
|
||||
0xca, 0x82, 0xbd, 0x8e, 0x15, 0x59, 0xfa, 0xb3, 0x12, 0xcc, 0x74, 0xd0, 0xc5, 0x08, 0xff, 0x78,
|
||||
0xd0, 0x0b, 0x88, 0x95, 0x4f, 0x40, 0x6c, 0xea, 0x24, 0xc4, 0x2a, 0x87, 0x10, 0xbb, 0x05, 0xd3,
|
||||
0x21, 0xa3, 0x9e, 0xc5, 0xc6, 0xa6, 0x83, 0x63, 0xde, 0xaa, 0x4a, 0xdc, 0x1a, 0x89, 0xee, 0x11,
|
||||
0x8e, 0xb9, 0xfe, 0x4a, 0x81, 0x99, 0x1e, 0x5a, 0xac, 0x3f, 0x3c, 0x33, 0x18, 0x26, 0xe2, 0x57,
|
||||
0x8b, 0xf1, 0x17, 0xf6, 0x5f, 0xf9, 0xe0, 0xfe, 0xfb, 0x2b, 0x68, 0x0c, 0xf9, 0xc8, 0x8d, 0xcc,
|
||||
0x1c, 0x9c, 0x18, 0x80, 0xd9, 0x58, 0xbf, 0x96, 0x41, 0xb4, 0x0a, 0x53, 0x3b, 0x23, 0x64, 0x63,
|
||||
0x59, 0x6e, 0x27, 0xf2, 0x1f, 0xdb, 0xe9, 0xdf, 0x29, 0xa0, 0xf5, 0xc6, 0x7c, 0x2d, 0xf0, 0xb7,
|
||||
0xa9, 0x7d, 0xee, 0x32, 0x27, 0x50, 0x96, 0x7c, 0x4d, 0x2d, 0xa9, 0xcb, 0x75, 0x43, 0x7e, 0x0b,
|
||||
0x2e, 0x1d, 0x1c, 0x9b, 0x21, 0xc3, 0x6d, 0xfa, 0x14, 0x63, 0xb6, 0xeb, 0x46, 0xc3, 0xc1, 0xf1,
|
||||
0x46, 0xa2, 0xd2, 0x9f, 0x97, 0x60, 0x3a, 0xe5, 0x52, 0xe0, 0xf3, 0x26, 0x09, 0xe5, 0xf7, 0xcd,
|
||||
0xd2, 0xeb, 0xdf, 0x37, 0x33, 0x14, 0xd4, 0x63, 0x50, 0x38, 0xd0, 0x47, 0x6f, 0xc1, 0xb4, 0xa4,
|
||||
0xc3, 0xf4, 0x83, 0x01, 0x66, 0xec, 0x36, 0xa4, 0xee, 0xb1, 0x54, 0x15, 0x81, 0xaa, 0xbc, 0x4e,
|
||||
0x89, 0x54, 0x8f, 0x2e, 0x91, 0x15, 0x28, 0x0f, 0x69, 0x14, 0xf7, 0x95, 0x46, 0xfb, 0xda, 0xd1,
|
||||
0x8d, 0xfa, 0x21, 0x8d, 0xb8, 0x21, 0xed, 0xf4, 0xf7, 0xa1, 0xb1, 0x49, 0x3d, 0xdc, 0xa4, 0x7d,
|
||||
0x67, 0x9d, 0xdb, 0x6f, 0x02, 0x65, 0xfe, 0x18, 0x52, 0x2a, 0x3c, 0x86, 0x9c, 0x78, 0xfa, 0xe8,
|
||||
0x1f, 0x2b, 0x50, 0x7d, 0x84, 0xe3, 0x76, 0x0f, 0x6d, 0x89, 0xab, 0xe8, 0x75, 0xe9, 0x03, 0x85,
|
||||
0x14, 0xc8, 0x4d, 0x68, 0x4c, 0xec, 0xee, 0x64, 0x71, 0xc8, 0x37, 0xf7, 0x29, 0xc7, 0xdb, 0x55,
|
||||
0xa8, 0x51, 0x6e, 0xee, 0x5a, 0x2e, 0x1d, 0x48, 0x5e, 0x6a, 0x46, 0x95, 0xf2, 0x77, 0x84, 0x28,
|
||||
0xfa, 0x4a, 0xd6, 0xce, 0xe3, 0x2a, 0x54, 0x8d, 0x09, 0x8d, 0xfe, 0x04, 0x20, 0x09, 0x4d, 0x40,
|
||||
0x93, 0xb1, 0xae, 0x4c, 0xb2, 0xfe, 0x2f, 0xa8, 0x3a, 0x38, 0x6e, 0x73, 0xb4, 0x5b, 0x25, 0x09,
|
||||
0xf9, 0x71, 0x78, 0x25, 0x2b, 0x19, 0xa9, 0xb9, 0xfe, 0x51, 0xfc, 0xb4, 0x25, 0x9c, 0x89, 0xfa,
|
||||
0xe2, 0xc5, 0xe3, 0x46, 0x39, 0x78, 0xdc, 0xdc, 0x84, 0x86, 0x87, 0x5e, 0xc0, 0xc6, 0x26, 0xa7,
|
||||
0xfb, 0x98, 0xc2, 0x10, 0xab, 0x7a, 0x74, 0x1f, 0x45, 0xa2, 0xfe, 0xc8, 0x33, 0x59, 0xb0, 0xc7,
|
||||
0xd3, 0x6d, 0xe8, 0x8f, 0x3c, 0x23, 0xd8, 0xe3, 0xe4, 0x6f, 0x30, 0xc7, 0xb0, 0x8f, 0x7e, 0xe4,
|
||||
0x8e, 0x4d, 0x2f, 0x18, 0xd0, 0x6d, 0x8a, 0x29, 0x18, 0x5a, 0xfa, 0xc3, 0x7a, 0xa2, 0xd7, 0x3f,
|
||||
0x53, 0x60, 0xee, 0xff, 0x69, 0x69, 0xf6, 0xd0, 0x8e, 0x83, 0x3b, 0x83, 0xc2, 0xf8, 0xb7, 0xcc,
|
||||
0xd7, 0x14, 0x9b, 0x8a, 0x9f, 0xfe, 0xd4, 0x95, 0xe1, 0x64, 0xd4, 0x78, 0x12, 0xd4, 0xdd, 0x17,
|
||||
0x25, 0xa8, 0x26, 0xee, 0x48, 0x1d, 0xa6, 0x9c, 0xc7, 0x81, 0x8f, 0xda, 0x25, 0xb2, 0x08, 0x73,
|
||||
0xce, 0xc1, 0xa7, 0x16, 0x6d, 0x40, 0xe6, 0x61, 0xd6, 0x29, 0xbe, 0x54, 0x68, 0x48, 0x08, 0x34,
|
||||
0x9d, 0xc2, 0x65, 0x5e, 0xdb, 0x26, 0x57, 0x60, 0xde, 0x39, 0x7c, 0xdf, 0xd5, 0x44, 0x09, 0x68,
|
||||
0x4e, 0xf1, 0x4a, 0xc8, 0xb5, 0xa1, 0x5c, 0xe2, 0xbf, 0x18, 0x65, 0x7d, 0x96, 0x6b, 0x94, 0x2c,
|
||||
0x82, 0xe6, 0x1c, 0xb8, 0x99, 0x69, 0x5f, 0x29, 0x64, 0x1e, 0x9a, 0x4e, 0xe1, 0x02, 0xa3, 0x7d,
|
||||
0xad, 0x10, 0x02, 0x33, 0xce, 0xe4, 0x9c, 0xaf, 0x3d, 0x57, 0xc8, 0x15, 0x20, 0xce, 0xa1, 0x71,
|
||||
0x58, 0xfb, 0x46, 0x21, 0x0b, 0x30, 0xeb, 0x14, 0xa6, 0x46, 0xae, 0x7d, 0xab, 0x90, 0x69, 0xa8,
|
||||
0x3a, 0xf1, 0x68, 0xa5, 0x7d, 0xa0, 0x4a, 0x29, 0x3e, 0xf3, 0xb5, 0x0f, 0x63, 0x29, 0x6e, 0x97,
|
||||
0xda, 0x2b, 0x55, 0x3a, 0x9b, 0x6c, 0x9e, 0xda, 0xaf, 0x2a, 0x69, 0x42, 0xdd, 0x49, 0x9b, 0x80,
|
||||
0xf6, 0x45, 0x5d, 0x3a, 0x3f, 0x54, 0x01, 0xda, 0x97, 0xf5, 0xbb, 0xf7, 0xa1, 0x96, 0x3e, 0x5a,
|
||||
0x12, 0x80, 0xca, 0xba, 0xc5, 0x23, 0x64, 0xda, 0x25, 0xf1, 0x6d, 0xa0, 0x35, 0x40, 0xa6, 0x29,
|
||||
0xe2, 0xfb, 0x5d, 0x46, 0x85, 0xbe, 0x24, 0x38, 0xd9, 0x10, 0x3d, 0x50, 0x53, 0x1f, 0x74, 0xde,
|
||||
0x7b, 0x60, 0xd3, 0x68, 0x38, 0xda, 0x12, 0x3d, 0x75, 0x75, 0x9f, 0xba, 0x2e, 0xdd, 0x8f, 0xb0,
|
||||
0x3f, 0x5c, 0x8d, 0x09, 0xff, 0xfb, 0x80, 0xf2, 0x88, 0xd1, 0xad, 0x51, 0x84, 0x83, 0xd5, 0x94,
|
||||
0xf6, 0x55, 0x59, 0x05, 0x99, 0x18, 0x6e, 0x6d, 0x55, 0xa4, 0xe6, 0xde, 0x6f, 0x01, 0x00, 0x00,
|
||||
0xff, 0xff, 0xa7, 0xd7, 0xb1, 0x87, 0x25, 0x18, 0x00, 0x00,
|
||||
}
|
||||
|
|
|
@ -89,6 +89,15 @@ service Master {
|
|||
rpc ShowPartitions(internal.ShowPartitionRequest) returns (service.StringListResponse) {}
|
||||
|
||||
|
||||
/**
|
||||
* @brief This method is used to get system configs
|
||||
*
|
||||
* @param SysConfigRequest, keys or key_prefixes of the configs.
|
||||
*
|
||||
* @return SysConfigResponse
|
||||
*/
|
||||
rpc GetSysConfigs(internal.SysConfigRequest) returns (service.SysConfigResponse) {}
|
||||
|
||||
rpc AllocTimestamp(internal.TsoRequest) returns (internal.TsoResponse) {}
|
||||
rpc AllocID(internal.IDRequest) returns (internal.IDResponse) {}
|
||||
|
||||
|
|
|
@ -30,36 +30,38 @@ const _ = proto.ProtoPackageIsVersion3 // please upgrade the proto package
|
|||
func init() { proto.RegisterFile("master.proto", fileDescriptor_f9c348dec43a6705) }
|
||||
|
||||
var fileDescriptor_f9c348dec43a6705 = []byte{
|
||||
// 458 bytes of a gzipped FileDescriptorProto
|
||||
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x9c, 0x94, 0x41, 0x6f, 0xd3, 0x30,
|
||||
0x14, 0xc7, 0x7b, 0x1a, 0x92, 0xd5, 0xb5, 0xcc, 0xdc, 0xca, 0x85, 0xf5, 0x04, 0x2d, 0x4b, 0x10,
|
||||
0x7c, 0x01, 0xd6, 0xe5, 0xb0, 0x4a, 0x20, 0x4d, 0xeb, 0x2e, 0x80, 0xd0, 0x70, 0xb2, 0xa7, 0xf4,
|
||||
0x81, 0x13, 0x07, 0xbf, 0x97, 0x21, 0xed, 0x23, 0xf1, 0x29, 0x51, 0x93, 0x26, 0xa9, 0x69, 0x5d,
|
||||
0xca, 0x6e, 0xb5, 0xfd, 0xf3, 0xef, 0x5f, 0xbf, 0xf7, 0x14, 0xd1, 0xcf, 0x14, 0x31, 0xd8, 0xa0,
|
||||
0xb0, 0x86, 0x8d, 0x7c, 0x96, 0xa1, 0xbe, 0x2f, 0xa9, 0x5e, 0x05, 0xf5, 0xd1, 0xa8, 0x9f, 0x98,
|
||||
0x2c, 0x33, 0x79, 0xbd, 0x39, 0x92, 0x98, 0x33, 0xd8, 0x5c, 0xe9, 0xdb, 0x8c, 0xd2, 0xf5, 0xde,
|
||||
0x09, 0x81, 0xbd, 0xc7, 0x04, 0xba, 0xad, 0xb7, 0xbf, 0x85, 0x38, 0xfa, 0x58, 0xdd, 0x97, 0x4a,
|
||||
0x3c, 0xbd, 0xb0, 0xa0, 0x18, 0x2e, 0x8c, 0xd6, 0x90, 0x30, 0x9a, 0x5c, 0x06, 0x81, 0x93, 0xd4,
|
||||
0x38, 0x83, 0xbf, 0xc1, 0x6b, 0xf8, 0x59, 0x02, 0xf1, 0xe8, 0xb9, 0xcb, 0xaf, 0xff, 0xd1, 0x82,
|
||||
0x15, 0x97, 0x34, 0xee, 0xc9, 0xaf, 0x62, 0x10, 0x59, 0x53, 0x6c, 0x04, 0xbc, 0xf6, 0x04, 0xb8,
|
||||
0xd8, 0x81, 0xfa, 0x58, 0x1c, 0x5f, 0x2a, 0xda, 0xb0, 0x4f, 0x3d, 0x76, 0x87, 0x6a, 0xe4, 0x63,
|
||||
0x17, 0x5e, 0xd7, 0x2a, 0x98, 0x19, 0xa3, 0xaf, 0x81, 0x0a, 0x93, 0x13, 0x8c, 0x7b, 0xb2, 0x14,
|
||||
0x32, 0x02, 0x4a, 0x2c, 0xc6, 0x9b, 0x75, 0x7a, 0xe3, 0x7b, 0xc6, 0x16, 0xda, 0xa4, 0x4d, 0x77,
|
||||
0xa7, 0x75, 0x60, 0x7d, 0xb5, 0x58, 0xfd, 0x1c, 0xf7, 0xe4, 0x0f, 0x31, 0x5c, 0x2c, 0xcd, 0xaf,
|
||||
0xee, 0x98, 0xbc, 0xa5, 0x73, 0xb9, 0x26, 0xef, 0xe5, 0xee, 0xbc, 0x05, 0x5b, 0xcc, 0xd3, 0x0f,
|
||||
0x48, 0xbc, 0xf1, 0xc6, 0x5b, 0x31, 0xac, 0x1b, 0x7c, 0xa5, 0x2c, 0x63, 0xf5, 0xc0, 0xb3, 0xbd,
|
||||
0x83, 0xd0, 0x72, 0x07, 0x36, 0xea, 0x8b, 0x38, 0x5e, 0x35, 0xb8, 0xd3, 0x4f, 0xf7, 0x8c, 0xc1,
|
||||
0xff, 0xca, 0xbf, 0x89, 0xfe, 0xa5, 0xa2, 0xce, 0x3d, 0xf1, 0x0f, 0xc1, 0x96, 0xfa, 0xb0, 0x19,
|
||||
0xb0, 0xe2, 0xa4, 0x69, 0x6c, 0x17, 0x13, 0xfe, 0x63, 0x04, 0xb6, 0xb2, 0x26, 0xbb, 0xb3, 0x5a,
|
||||
0xce, 0x1d, 0x00, 0x14, 0x83, 0x55, 0x63, 0xdb, 0x53, 0xf2, 0xd6, 0xcc, 0xc1, 0x1e, 0xd3, 0xfe,
|
||||
0x4f, 0x62, 0x70, 0xae, 0xb5, 0x49, 0x6e, 0x30, 0x03, 0x62, 0x95, 0x15, 0xf2, 0xd4, 0x13, 0x75,
|
||||
0x43, 0xc6, 0x53, 0x39, 0x17, 0x69, 0xd5, 0x57, 0xe2, 0x49, 0xa5, 0x9e, 0x47, 0xf2, 0x85, 0xe7,
|
||||
0xc2, 0x3c, 0x6a, 0x94, 0xa7, 0x7b, 0x88, 0xd6, 0xf8, 0x5d, 0x0c, 0xcf, 0x89, 0x30, 0xcd, 0x17,
|
||||
0x90, 0x66, 0x90, 0xf3, 0x3c, 0x92, 0xaf, 0x3c, 0xf7, 0x5a, 0xae, 0x8b, 0x98, 0x1c, 0x82, 0x36,
|
||||
0x59, 0xb3, 0xd9, 0xe7, 0xf7, 0x29, 0xf2, 0xb2, 0x8c, 0x57, 0x33, 0x17, 0x3e, 0xa0, 0xd6, 0xf8,
|
||||
0xc0, 0x90, 0x2c, 0xc3, 0x5a, 0x72, 0x76, 0x87, 0xc4, 0x16, 0xe3, 0x92, 0xe1, 0x2e, 0x6c, 0x54,
|
||||
0x61, 0x65, 0x0e, 0xeb, 0x6f, 0x74, 0x11, 0xc7, 0x47, 0xd5, 0xfa, 0xdd, 0x9f, 0x00, 0x00, 0x00,
|
||||
0xff, 0xff, 0xa0, 0xb5, 0xeb, 0xf6, 0xd1, 0x05, 0x00, 0x00,
|
||||
// 484 bytes of a gzipped FileDescriptorProto
|
||||
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x9c, 0x95, 0xdf, 0x6e, 0xd3, 0x30,
|
||||
0x14, 0x87, 0x7b, 0x35, 0x24, 0xd3, 0x3f, 0xcc, 0xdc, 0x95, 0x1b, 0xd6, 0x9b, 0x41, 0xcb, 0x12,
|
||||
0x04, 0x2f, 0xc0, 0xda, 0x48, 0xac, 0x12, 0x48, 0xd3, 0xb2, 0x1b, 0x40, 0x68, 0x24, 0xd9, 0x21,
|
||||
0x35, 0x24, 0x76, 0xf0, 0x39, 0x19, 0xa2, 0x2f, 0xc1, 0x2b, 0xa3, 0x26, 0x75, 0x52, 0xd3, 0xba,
|
||||
0x94, 0xdd, 0xd5, 0xf6, 0xe7, 0xdf, 0x57, 0x9f, 0x73, 0xa4, 0xb0, 0x6e, 0x1e, 0x21, 0x81, 0xf6,
|
||||
0x0a, 0xad, 0x48, 0xf1, 0xc7, 0xb9, 0xc8, 0xee, 0x4a, 0xac, 0x57, 0x5e, 0x7d, 0x34, 0xec, 0x26,
|
||||
0x2a, 0xcf, 0x95, 0xac, 0x37, 0x87, 0x5c, 0x48, 0x02, 0x2d, 0xa3, 0xec, 0x26, 0xc7, 0x74, 0xbd,
|
||||
0x77, 0x8c, 0xa0, 0xef, 0x44, 0x02, 0xed, 0xd6, 0xab, 0xdf, 0x0f, 0xd9, 0xd1, 0xfb, 0xea, 0x3e,
|
||||
0x8f, 0xd8, 0xa3, 0x99, 0x86, 0x88, 0x60, 0xa6, 0xb2, 0x0c, 0x12, 0x12, 0x4a, 0x72, 0xcf, 0xb3,
|
||||
0x4c, 0x26, 0xd3, 0xfb, 0x1b, 0xbc, 0x82, 0x1f, 0x25, 0x20, 0x0d, 0x9f, 0xd8, 0xfc, 0xfa, 0x1f,
|
||||
0x85, 0x14, 0x51, 0x89, 0xa3, 0x0e, 0xff, 0xcc, 0xfa, 0x81, 0x56, 0xc5, 0x86, 0xe0, 0x85, 0x43,
|
||||
0x60, 0x63, 0x07, 0xc6, 0xc7, 0xac, 0x77, 0x11, 0xe1, 0x46, 0xfa, 0xc4, 0x91, 0x6e, 0x51, 0x26,
|
||||
0x7c, 0x64, 0xc3, 0xeb, 0x5a, 0x79, 0x53, 0xa5, 0xb2, 0x2b, 0xc0, 0x42, 0x49, 0x84, 0x51, 0x87,
|
||||
0x97, 0x8c, 0x07, 0x80, 0x89, 0x16, 0xf1, 0x66, 0x9d, 0x5e, 0xba, 0x9e, 0xb1, 0x85, 0x1a, 0xdb,
|
||||
0x64, 0xb7, 0xad, 0x05, 0xeb, 0xab, 0xc5, 0xea, 0xe7, 0xa8, 0xc3, 0xbf, 0xb3, 0x41, 0xb8, 0x50,
|
||||
0x3f, 0xdb, 0x63, 0x74, 0x96, 0xce, 0xe6, 0x8c, 0xef, 0xd9, 0x6e, 0x5f, 0x48, 0x5a, 0xc8, 0xf4,
|
||||
0x9d, 0x40, 0xda, 0x78, 0xe3, 0x0d, 0x1b, 0xd4, 0x0d, 0xbe, 0x8c, 0x34, 0x89, 0xea, 0x81, 0x67,
|
||||
0x7b, 0x07, 0xa1, 0xe1, 0x0e, 0x6c, 0xd4, 0x27, 0xd6, 0x5b, 0x35, 0xb8, 0x8d, 0x9f, 0xec, 0x19,
|
||||
0x83, 0xff, 0x0d, 0xff, 0xc2, 0xba, 0x17, 0x11, 0xb6, 0xd9, 0x63, 0xf7, 0x10, 0x6c, 0x45, 0x1f,
|
||||
0x36, 0x03, 0x9a, 0x1d, 0x9b, 0xc6, 0xb6, 0x1a, 0xff, 0x1f, 0x23, 0xb0, 0xe5, 0x1a, 0xef, 0x76,
|
||||
0x35, 0x9c, 0x3d, 0x00, 0x82, 0xf5, 0x57, 0x8d, 0x6d, 0x4e, 0xd1, 0x59, 0x33, 0x0b, 0xbb, 0x4f,
|
||||
0xfb, 0x13, 0xd6, 0x7b, 0x0b, 0x14, 0xfe, 0xc2, 0x99, 0x92, 0x5f, 0x45, 0x8a, 0xfc, 0xd4, 0x65,
|
||||
0x32, 0x88, 0xb1, 0x9c, 0x3a, 0x2c, 0x2d, 0xd7, 0x48, 0x3e, 0xb0, 0xfe, 0x79, 0x96, 0xa9, 0xe4,
|
||||
0x5a, 0xe4, 0x80, 0x14, 0xe5, 0x05, 0x3f, 0x71, 0x58, 0xae, 0x51, 0x39, 0xda, 0x63, 0x23, 0x4d,
|
||||
0xf4, 0x25, 0x7b, 0x50, 0x45, 0xcf, 0x03, 0xfe, 0xd4, 0x71, 0x61, 0x1e, 0x98, 0xc8, 0x93, 0x3d,
|
||||
0x44, 0x93, 0xf8, 0x8d, 0x0d, 0xce, 0x11, 0x45, 0x2a, 0x43, 0x48, 0x73, 0x90, 0x34, 0x0f, 0xf8,
|
||||
0x73, 0xc7, 0xbd, 0x86, 0x6b, 0x15, 0xe3, 0x43, 0x50, 0xe3, 0x9a, 0x4e, 0x3f, 0xbe, 0x49, 0x05,
|
||||
0x2d, 0xca, 0x78, 0x35, 0xd8, 0xfe, 0x52, 0x64, 0x99, 0x58, 0x12, 0x24, 0x0b, 0xbf, 0x0e, 0x39,
|
||||
0xbb, 0x15, 0x48, 0x5a, 0xc4, 0x25, 0xc1, 0xad, 0x6f, 0xa2, 0xfc, 0x2a, 0xd9, 0xaf, 0x3f, 0x04,
|
||||
0x45, 0x1c, 0x1f, 0x55, 0xeb, 0xd7, 0x7f, 0x02, 0x00, 0x00, 0xff, 0xff, 0x8d, 0x75, 0x7d, 0xec,
|
||||
0x36, 0x06, 0x00, 0x00,
|
||||
}
|
||||
|
||||
// Reference imports to suppress errors if they are not otherwise used.
|
||||
|
@ -134,6 +136,13 @@ type MasterClient interface {
|
|||
//
|
||||
// @return StringListResponse
|
||||
ShowPartitions(ctx context.Context, in *internalpb.ShowPartitionRequest, opts ...grpc.CallOption) (*servicepb.StringListResponse, error)
|
||||
//*
|
||||
// @brief This method is used to get system configs
|
||||
//
|
||||
// @param SysConfigRequest, keys or key_prefixes of the configs.
|
||||
//
|
||||
// @return SysConfigResponse
|
||||
GetSysConfigs(ctx context.Context, in *internalpb.SysConfigRequest, opts ...grpc.CallOption) (*servicepb.SysConfigResponse, error)
|
||||
AllocTimestamp(ctx context.Context, in *internalpb.TsoRequest, opts ...grpc.CallOption) (*internalpb.TsoResponse, error)
|
||||
AllocID(ctx context.Context, in *internalpb.IDRequest, opts ...grpc.CallOption) (*internalpb.IDResponse, error)
|
||||
AssignSegmentID(ctx context.Context, in *internalpb.AssignSegIDRequest, opts ...grpc.CallOption) (*internalpb.AssignSegIDResponse, error)
|
||||
|
@ -237,6 +246,15 @@ func (c *masterClient) ShowPartitions(ctx context.Context, in *internalpb.ShowPa
|
|||
return out, nil
|
||||
}
|
||||
|
||||
func (c *masterClient) GetSysConfigs(ctx context.Context, in *internalpb.SysConfigRequest, opts ...grpc.CallOption) (*servicepb.SysConfigResponse, error) {
|
||||
out := new(servicepb.SysConfigResponse)
|
||||
err := c.cc.Invoke(ctx, "/milvus.proto.master.Master/GetSysConfigs", in, out, opts...)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return out, nil
|
||||
}
|
||||
|
||||
func (c *masterClient) AllocTimestamp(ctx context.Context, in *internalpb.TsoRequest, opts ...grpc.CallOption) (*internalpb.TsoResponse, error) {
|
||||
out := new(internalpb.TsoResponse)
|
||||
err := c.cc.Invoke(ctx, "/milvus.proto.master.Master/AllocTimestamp", in, out, opts...)
|
||||
|
@ -326,6 +344,13 @@ type MasterServer interface {
|
|||
//
|
||||
// @return StringListResponse
|
||||
ShowPartitions(context.Context, *internalpb.ShowPartitionRequest) (*servicepb.StringListResponse, error)
|
||||
//*
|
||||
// @brief This method is used to get system configs
|
||||
//
|
||||
// @param SysConfigRequest, keys or key_prefixes of the configs.
|
||||
//
|
||||
// @return SysConfigResponse
|
||||
GetSysConfigs(context.Context, *internalpb.SysConfigRequest) (*servicepb.SysConfigResponse, error)
|
||||
AllocTimestamp(context.Context, *internalpb.TsoRequest) (*internalpb.TsoResponse, error)
|
||||
AllocID(context.Context, *internalpb.IDRequest) (*internalpb.IDResponse, error)
|
||||
AssignSegmentID(context.Context, *internalpb.AssignSegIDRequest) (*internalpb.AssignSegIDResponse, error)
|
||||
|
@ -365,6 +390,9 @@ func (*UnimplementedMasterServer) DescribePartition(ctx context.Context, req *in
|
|||
func (*UnimplementedMasterServer) ShowPartitions(ctx context.Context, req *internalpb.ShowPartitionRequest) (*servicepb.StringListResponse, error) {
|
||||
return nil, status.Errorf(codes.Unimplemented, "method ShowPartitions not implemented")
|
||||
}
|
||||
func (*UnimplementedMasterServer) GetSysConfigs(ctx context.Context, req *internalpb.SysConfigRequest) (*servicepb.SysConfigResponse, error) {
|
||||
return nil, status.Errorf(codes.Unimplemented, "method GetSysConfigs not implemented")
|
||||
}
|
||||
func (*UnimplementedMasterServer) AllocTimestamp(ctx context.Context, req *internalpb.TsoRequest) (*internalpb.TsoResponse, error) {
|
||||
return nil, status.Errorf(codes.Unimplemented, "method AllocTimestamp not implemented")
|
||||
}
|
||||
|
@ -559,6 +587,24 @@ func _Master_ShowPartitions_Handler(srv interface{}, ctx context.Context, dec fu
|
|||
return interceptor(ctx, in, info, handler)
|
||||
}
|
||||
|
||||
func _Master_GetSysConfigs_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
|
||||
in := new(internalpb.SysConfigRequest)
|
||||
if err := dec(in); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if interceptor == nil {
|
||||
return srv.(MasterServer).GetSysConfigs(ctx, in)
|
||||
}
|
||||
info := &grpc.UnaryServerInfo{
|
||||
Server: srv,
|
||||
FullMethod: "/milvus.proto.master.Master/GetSysConfigs",
|
||||
}
|
||||
handler := func(ctx context.Context, req interface{}) (interface{}, error) {
|
||||
return srv.(MasterServer).GetSysConfigs(ctx, req.(*internalpb.SysConfigRequest))
|
||||
}
|
||||
return interceptor(ctx, in, info, handler)
|
||||
}
|
||||
|
||||
func _Master_AllocTimestamp_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
|
||||
in := new(internalpb.TsoRequest)
|
||||
if err := dec(in); err != nil {
|
||||
|
@ -657,6 +703,10 @@ var _Master_serviceDesc = grpc.ServiceDesc{
|
|||
MethodName: "ShowPartitions",
|
||||
Handler: _Master_ShowPartitions_Handler,
|
||||
},
|
||||
{
|
||||
MethodName: "GetSysConfigs",
|
||||
Handler: _Master_GetSysConfigs_Handler,
|
||||
},
|
||||
{
|
||||
MethodName: "AllocTimestamp",
|
||||
Handler: _Master_AllocTimestamp_Handler,
|
||||
|
|
|
@ -135,6 +135,14 @@ message PartitionDescription {
|
|||
repeated common.KeyValuePair statistics = 3;
|
||||
}
|
||||
|
||||
/**
|
||||
* @brief Response of GetSysConfig
|
||||
*/
|
||||
message SysConfigResponse {
|
||||
common.Status status = 1;
|
||||
repeated string keys = 2;
|
||||
repeated string values = 3;
|
||||
}
|
||||
|
||||
/**
|
||||
* @brief Scores of a query.
|
||||
|
|
|
@ -737,6 +737,63 @@ func (m *PartitionDescription) GetStatistics() []*commonpb.KeyValuePair {
|
|||
return nil
|
||||
}
|
||||
|
||||
//*
|
||||
// @brief Response of GetSysConfig
|
||||
type SysConfigResponse struct {
|
||||
Status *commonpb.Status `protobuf:"bytes,1,opt,name=status,proto3" json:"status,omitempty"`
|
||||
Keys []string `protobuf:"bytes,2,rep,name=keys,proto3" json:"keys,omitempty"`
|
||||
Values []string `protobuf:"bytes,3,rep,name=values,proto3" json:"values,omitempty"`
|
||||
XXX_NoUnkeyedLiteral struct{} `json:"-"`
|
||||
XXX_unrecognized []byte `json:"-"`
|
||||
XXX_sizecache int32 `json:"-"`
|
||||
}
|
||||
|
||||
func (m *SysConfigResponse) Reset() { *m = SysConfigResponse{} }
|
||||
func (m *SysConfigResponse) String() string { return proto.CompactTextString(m) }
|
||||
func (*SysConfigResponse) ProtoMessage() {}
|
||||
func (*SysConfigResponse) Descriptor() ([]byte, []int) {
|
||||
return fileDescriptor_b4b40b84dd2f74cb, []int{13}
|
||||
}
|
||||
|
||||
func (m *SysConfigResponse) XXX_Unmarshal(b []byte) error {
|
||||
return xxx_messageInfo_SysConfigResponse.Unmarshal(m, b)
|
||||
}
|
||||
func (m *SysConfigResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
|
||||
return xxx_messageInfo_SysConfigResponse.Marshal(b, m, deterministic)
|
||||
}
|
||||
func (m *SysConfigResponse) XXX_Merge(src proto.Message) {
|
||||
xxx_messageInfo_SysConfigResponse.Merge(m, src)
|
||||
}
|
||||
func (m *SysConfigResponse) XXX_Size() int {
|
||||
return xxx_messageInfo_SysConfigResponse.Size(m)
|
||||
}
|
||||
func (m *SysConfigResponse) XXX_DiscardUnknown() {
|
||||
xxx_messageInfo_SysConfigResponse.DiscardUnknown(m)
|
||||
}
|
||||
|
||||
var xxx_messageInfo_SysConfigResponse proto.InternalMessageInfo
|
||||
|
||||
func (m *SysConfigResponse) GetStatus() *commonpb.Status {
|
||||
if m != nil {
|
||||
return m.Status
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (m *SysConfigResponse) GetKeys() []string {
|
||||
if m != nil {
|
||||
return m.Keys
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (m *SysConfigResponse) GetValues() []string {
|
||||
if m != nil {
|
||||
return m.Values
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
//*
|
||||
// @brief Scores of a query.
|
||||
// The default value of tag is "root".
|
||||
|
@ -753,7 +810,7 @@ func (m *Score) Reset() { *m = Score{} }
|
|||
func (m *Score) String() string { return proto.CompactTextString(m) }
|
||||
func (*Score) ProtoMessage() {}
|
||||
func (*Score) Descriptor() ([]byte, []int) {
|
||||
return fileDescriptor_b4b40b84dd2f74cb, []int{13}
|
||||
return fileDescriptor_b4b40b84dd2f74cb, []int{14}
|
||||
}
|
||||
|
||||
func (m *Score) XXX_Unmarshal(b []byte) error {
|
||||
|
@ -803,7 +860,7 @@ func (m *Hits) Reset() { *m = Hits{} }
|
|||
func (m *Hits) String() string { return proto.CompactTextString(m) }
|
||||
func (*Hits) ProtoMessage() {}
|
||||
func (*Hits) Descriptor() ([]byte, []int) {
|
||||
return fileDescriptor_b4b40b84dd2f74cb, []int{14}
|
||||
return fileDescriptor_b4b40b84dd2f74cb, []int{15}
|
||||
}
|
||||
|
||||
func (m *Hits) XXX_Unmarshal(b []byte) error {
|
||||
|
@ -859,7 +916,7 @@ func (m *QueryResult) Reset() { *m = QueryResult{} }
|
|||
func (m *QueryResult) String() string { return proto.CompactTextString(m) }
|
||||
func (*QueryResult) ProtoMessage() {}
|
||||
func (*QueryResult) Descriptor() ([]byte, []int) {
|
||||
return fileDescriptor_b4b40b84dd2f74cb, []int{15}
|
||||
return fileDescriptor_b4b40b84dd2f74cb, []int{16}
|
||||
}
|
||||
|
||||
func (m *QueryResult) XXX_Unmarshal(b []byte) error {
|
||||
|
@ -909,6 +966,7 @@ func init() {
|
|||
proto.RegisterType((*IntegerRangeResponse)(nil), "milvus.proto.service.IntegerRangeResponse")
|
||||
proto.RegisterType((*CollectionDescription)(nil), "milvus.proto.service.CollectionDescription")
|
||||
proto.RegisterType((*PartitionDescription)(nil), "milvus.proto.service.PartitionDescription")
|
||||
proto.RegisterType((*SysConfigResponse)(nil), "milvus.proto.service.SysConfigResponse")
|
||||
proto.RegisterType((*Score)(nil), "milvus.proto.service.Score")
|
||||
proto.RegisterType((*Hits)(nil), "milvus.proto.service.Hits")
|
||||
proto.RegisterType((*QueryResult)(nil), "milvus.proto.service.QueryResult")
|
||||
|
@ -917,53 +975,55 @@ func init() {
|
|||
func init() { proto.RegisterFile("service_msg.proto", fileDescriptor_b4b40b84dd2f74cb) }
|
||||
|
||||
var fileDescriptor_b4b40b84dd2f74cb = []byte{
|
||||
// 763 bytes of a gzipped FileDescriptorProto
|
||||
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xac, 0x54, 0x5d, 0x6f, 0xf3, 0x34,
|
||||
0x14, 0x26, 0x4d, 0x5b, 0xba, 0xd3, 0xb4, 0x6f, 0x67, 0x0a, 0x0a, 0xdb, 0x4d, 0xc9, 0xab, 0x41,
|
||||
0x05, 0xa2, 0x15, 0x1b, 0x12, 0xda, 0x05, 0x12, 0xed, 0x36, 0x60, 0x1f, 0xea, 0x86, 0x5b, 0x4d,
|
||||
0x1a, 0x48, 0x54, 0x6e, 0x62, 0x25, 0x16, 0x69, 0x1c, 0xd9, 0xce, 0xaa, 0xee, 0x96, 0xff, 0xc0,
|
||||
0x9f, 0xe0, 0x87, 0x70, 0xc7, 0x6f, 0x42, 0x71, 0xd2, 0xaf, 0x51, 0xc4, 0xde, 0x6d, 0x77, 0xf6,
|
||||
0xb1, 0xcf, 0x79, 0x9e, 0xf3, 0xf8, 0xf8, 0x81, 0x5d, 0x49, 0xc5, 0x3d, 0x73, 0xe9, 0x78, 0x2a,
|
||||
0xfd, 0x4e, 0x2c, 0xb8, 0xe2, 0xa8, 0x39, 0x65, 0xe1, 0x7d, 0x22, 0xb3, 0x5d, 0x27, 0x3f, 0xdf,
|
||||
0xb3, 0x5c, 0x3e, 0x9d, 0xf2, 0x28, 0x8b, 0xee, 0x59, 0xd2, 0x0d, 0xe8, 0x94, 0x64, 0x3b, 0xe7,
|
||||
0x18, 0xea, 0x27, 0x3c, 0x0c, 0xa9, 0xab, 0x18, 0x8f, 0x06, 0x64, 0x4a, 0xd1, 0x67, 0xf0, 0xc6,
|
||||
0x5d, 0x46, 0xc6, 0x11, 0x99, 0x52, 0xdb, 0x68, 0x19, 0xed, 0x1d, 0x5c, 0x77, 0x37, 0x2e, 0x3a,
|
||||
0x17, 0x50, 0xbb, 0x21, 0x42, 0xb1, 0x77, 0xce, 0x44, 0x0d, 0x30, 0x15, 0xf1, 0xed, 0x82, 0x3e,
|
||||
0x4c, 0x97, 0xce, 0x9f, 0x06, 0x54, 0x30, 0x9f, 0xf5, 0x89, 0x72, 0x83, 0xa7, 0xd7, 0x79, 0x0b,
|
||||
0xb5, 0x78, 0xc1, 0x60, 0xbc, 0xaa, 0x68, 0x2d, 0x83, 0x23, 0xe2, 0xa3, 0xaf, 0xa1, 0x22, 0xf8,
|
||||
0x6c, 0xec, 0x11, 0x45, 0x6c, 0xb3, 0x65, 0xb6, 0xab, 0x87, 0x1f, 0x77, 0x36, 0x64, 0xca, 0xd5,
|
||||
0xe9, 0x87, 0x7c, 0x82, 0xdf, 0x17, 0x7c, 0x76, 0x4a, 0x14, 0x41, 0xfb, 0xb0, 0x13, 0x10, 0x19,
|
||||
0x8c, 0x7f, 0xa3, 0x73, 0x69, 0x17, 0x5b, 0x66, 0xbb, 0x84, 0x2b, 0x69, 0xe0, 0x92, 0xce, 0xa5,
|
||||
0x33, 0x83, 0xc6, 0x4d, 0x48, 0x5c, 0x1a, 0xf0, 0xd0, 0xa3, 0xe2, 0x96, 0x84, 0xc9, 0xb2, 0x27,
|
||||
0x63, 0xd9, 0x13, 0x3a, 0x86, 0xa2, 0x9a, 0xc7, 0x54, 0x93, 0xaa, 0x1f, 0x1e, 0x74, 0xb6, 0xbd,
|
||||
0x4d, 0x67, 0xad, 0xce, 0x68, 0x1e, 0x53, 0xac, 0x53, 0xd0, 0x47, 0x50, 0xbe, 0x4f, 0xab, 0x4a,
|
||||
0xcd, 0xd8, 0xc2, 0xf9, 0xce, 0xf9, 0x75, 0x03, 0xf8, 0x07, 0xc1, 0x93, 0x18, 0x5d, 0x80, 0x15,
|
||||
0xaf, 0x62, 0xd2, 0x36, 0x74, 0x8f, 0x9f, 0xfe, 0x2f, 0x9c, 0xa6, 0x8d, 0x37, 0x72, 0x9d, 0x3f,
|
||||
0x0c, 0x28, 0xfd, 0x94, 0x50, 0x31, 0x7f, 0xfa, 0x1b, 0x1c, 0x40, 0x7d, 0xe3, 0x0d, 0xa4, 0x5d,
|
||||
0x68, 0x99, 0xed, 0x1d, 0x5c, 0x5b, 0x7f, 0x04, 0x99, 0xca, 0xe3, 0xc9, 0xd0, 0x36, 0x33, 0x79,
|
||||
0x3c, 0x19, 0xa2, 0x2f, 0x60, 0x77, 0x0d, 0x7b, 0xec, 0xa7, 0xcd, 0xd8, 0xc5, 0x96, 0xd1, 0xb6,
|
||||
0x70, 0x23, 0x7e, 0xd4, 0xa4, 0xf3, 0x0b, 0xd4, 0x87, 0x4a, 0xb0, 0xc8, 0xc7, 0x54, 0xc6, 0x3c,
|
||||
0x92, 0x14, 0x1d, 0x41, 0x59, 0x2a, 0xa2, 0x12, 0xa9, 0x79, 0x55, 0x0f, 0xf7, 0xb7, 0x3e, 0xea,
|
||||
0x50, 0x5f, 0xc1, 0xf9, 0x55, 0xd4, 0x84, 0x92, 0x56, 0x32, 0x1f, 0x94, 0x6c, 0xe3, 0xdc, 0x81,
|
||||
0xd5, 0xe7, 0x3c, 0x7c, 0xc5, 0xd2, 0x95, 0x45, 0x69, 0x02, 0x28, 0xe3, 0x7d, 0xc5, 0xa4, 0x7a,
|
||||
0x19, 0xc0, 0x6a, 0x26, 0x32, 0x81, 0x17, 0x33, 0x31, 0x81, 0x0f, 0xce, 0x23, 0x45, 0x7d, 0x2a,
|
||||
0x5e, 0x1b, 0xc3, 0x5c, 0x62, 0x48, 0x68, 0xe6, 0x18, 0x98, 0x44, 0x3e, 0x7d, 0xb1, 0x52, 0x13,
|
||||
0xea, 0xb3, 0x48, 0x2b, 0x65, 0xe2, 0x6c, 0x93, 0x0e, 0x08, 0x8d, 0x3c, 0x3d, 0x20, 0x26, 0x4e,
|
||||
0x97, 0xce, 0xdf, 0x06, 0x7c, 0xb8, 0xf2, 0xa6, 0x53, 0x2a, 0x5d, 0xc1, 0xe2, 0x74, 0xf9, 0x3c,
|
||||
0xd8, 0x6f, 0xa1, 0x9c, 0x39, 0x9f, 0xc6, 0xad, 0xfe, 0xeb, 0x43, 0x66, 0xae, 0xb8, 0x02, 0x1c,
|
||||
0xea, 0x00, 0xce, 0x93, 0x50, 0x0f, 0x20, 0x2d, 0xc4, 0xa4, 0x62, 0xae, 0xcc, 0x8d, 0xe4, 0x93,
|
||||
0xad, 0xb8, 0x97, 0x74, 0xae, 0xff, 0xd6, 0x0d, 0x61, 0x02, 0xaf, 0x25, 0x39, 0x7f, 0x19, 0xd0,
|
||||
0x5c, 0x3a, 0xe6, 0x8b, 0xfb, 0xf9, 0x06, 0x8a, 0xfa, 0x5b, 0x66, 0xdd, 0xbc, 0xfd, 0x8f, 0xff,
|
||||
0xbe, 0x6e, 0xd0, 0x58, 0x27, 0xbc, 0x46, 0x27, 0x5f, 0x41, 0x69, 0xe8, 0x72, 0xb1, 0xcd, 0xf5,
|
||||
0x36, 0x47, 0xa8, 0xb0, 0x1c, 0xa1, 0xdf, 0x0d, 0x28, 0xfe, 0xc8, 0x94, 0x76, 0x82, 0xf3, 0xd3,
|
||||
0xcc, 0xa6, 0x4c, 0x9c, 0x2e, 0x37, 0x1c, 0xba, 0xf0, 0x64, 0x87, 0x4e, 0x45, 0x4b, 0x39, 0x2c,
|
||||
0x5a, 0xd8, 0xdf, 0xae, 0x80, 0xe6, 0x89, 0xf3, 0xab, 0x8e, 0x80, 0xaa, 0xf6, 0x37, 0x4c, 0x65,
|
||||
0x12, 0xaa, 0xe7, 0x09, 0xdf, 0x81, 0x62, 0xc0, 0x94, 0xcc, 0xa9, 0xee, 0x6d, 0x87, 0x4d, 0x5b,
|
||||
0xc5, 0xfa, 0xde, 0xe7, 0xdf, 0xc1, 0x9b, 0x47, 0x2e, 0x8f, 0x2a, 0x50, 0x1c, 0x5c, 0x0f, 0xce,
|
||||
0x1a, 0xef, 0xa1, 0x5d, 0xa8, 0xdd, 0x9e, 0x9d, 0x8c, 0xae, 0xf1, 0xb8, 0x7f, 0x3e, 0xe8, 0xe1,
|
||||
0xbb, 0x86, 0x87, 0x1a, 0x60, 0xe5, 0xa1, 0xef, 0xaf, 0xae, 0x7b, 0xa3, 0x06, 0xed, 0x9f, 0xfc,
|
||||
0xdc, 0xf3, 0x99, 0x0a, 0x92, 0x49, 0xca, 0xa8, 0xfb, 0xc0, 0xc2, 0x90, 0x3d, 0x28, 0xea, 0x06,
|
||||
0xdd, 0x0c, 0xfa, 0x4b, 0x8f, 0x49, 0x25, 0xd8, 0x24, 0x51, 0xd4, 0xeb, 0xb2, 0x48, 0x51, 0x11,
|
||||
0x91, 0xb0, 0xab, 0xf9, 0x74, 0x73, 0x3e, 0xf1, 0x64, 0x52, 0xd6, 0x81, 0xa3, 0x7f, 0x02, 0x00,
|
||||
0x00, 0xff, 0xff, 0x12, 0x37, 0x33, 0x02, 0x37, 0x08, 0x00, 0x00,
|
||||
// 790 bytes of a gzipped FileDescriptorProto
|
||||
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xac, 0x55, 0xdd, 0x8e, 0xe3, 0x34,
|
||||
0x14, 0x26, 0x4d, 0x5a, 0x3a, 0xa7, 0x69, 0xb7, 0x35, 0x05, 0x85, 0x99, 0x9b, 0x92, 0xd5, 0x42,
|
||||
0x05, 0xa2, 0x15, 0xb3, 0x48, 0x68, 0x2f, 0x90, 0x68, 0x3b, 0x0b, 0xcc, 0xee, 0xaa, 0x33, 0xb8,
|
||||
0xd5, 0x4a, 0x0b, 0x12, 0x95, 0x9b, 0x98, 0xc4, 0x22, 0x8d, 0x23, 0xdb, 0x99, 0xaa, 0x7b, 0xcb,
|
||||
0x3b, 0xf0, 0x12, 0x3c, 0x08, 0x77, 0x3c, 0x13, 0x8a, 0x93, 0xfe, 0x0d, 0x45, 0x0c, 0xd3, 0xb9,
|
||||
0xb3, 0x4f, 0x7c, 0xce, 0xf7, 0x9d, 0xcf, 0xc7, 0x5f, 0xa0, 0x25, 0xa9, 0xb8, 0x61, 0x1e, 0x9d,
|
||||
0x2d, 0x64, 0xd0, 0x4b, 0x04, 0x57, 0x1c, 0xb5, 0x17, 0x2c, 0xba, 0x49, 0x65, 0xbe, 0xeb, 0x15,
|
||||
0xdf, 0x4f, 0x6d, 0x8f, 0x2f, 0x16, 0x3c, 0xce, 0xa3, 0xa7, 0xb6, 0xf4, 0x42, 0xba, 0x20, 0xf9,
|
||||
0xce, 0x7d, 0x06, 0x8d, 0x11, 0x8f, 0x22, 0xea, 0x29, 0xc6, 0xe3, 0x31, 0x59, 0x50, 0xf4, 0x09,
|
||||
0x3c, 0xf2, 0x36, 0x91, 0x59, 0x4c, 0x16, 0xd4, 0x31, 0x3a, 0x46, 0xf7, 0x04, 0x37, 0xbc, 0xbd,
|
||||
0x83, 0xee, 0x0b, 0xa8, 0x5f, 0x13, 0xa1, 0xd8, 0xff, 0xce, 0x44, 0x4d, 0x30, 0x15, 0x09, 0x9c,
|
||||
0x92, 0xfe, 0x98, 0x2d, 0xdd, 0x3f, 0x0c, 0xa8, 0x62, 0xbe, 0x1c, 0x12, 0xe5, 0x85, 0x77, 0xaf,
|
||||
0xf3, 0x18, 0xea, 0xc9, 0x9a, 0xc1, 0x6c, 0x5b, 0xd1, 0xde, 0x04, 0xa7, 0x24, 0x40, 0x5f, 0x42,
|
||||
0x55, 0xf0, 0xe5, 0xcc, 0x27, 0x8a, 0x38, 0x66, 0xc7, 0xec, 0xd6, 0xce, 0x3f, 0xec, 0xed, 0xc9,
|
||||
0x54, 0xa8, 0x33, 0x8c, 0xf8, 0x1c, 0xbf, 0x2b, 0xf8, 0xf2, 0x82, 0x28, 0x82, 0xce, 0xe0, 0x24,
|
||||
0x24, 0x32, 0x9c, 0xfd, 0x4a, 0x57, 0xd2, 0xb1, 0x3a, 0x66, 0xb7, 0x8c, 0xab, 0x59, 0xe0, 0x25,
|
||||
0x5d, 0x49, 0x77, 0x09, 0xcd, 0xeb, 0x88, 0x78, 0x34, 0xe4, 0x91, 0x4f, 0xc5, 0x6b, 0x12, 0xa5,
|
||||
0x9b, 0x9e, 0x8c, 0x4d, 0x4f, 0xe8, 0x19, 0x58, 0x6a, 0x95, 0x50, 0x4d, 0xaa, 0x71, 0xfe, 0xa4,
|
||||
0x77, 0xe8, 0x6e, 0x7a, 0x3b, 0x75, 0xa6, 0xab, 0x84, 0x62, 0x9d, 0x82, 0x3e, 0x80, 0xca, 0x4d,
|
||||
0x56, 0x55, 0x6a, 0xc6, 0x36, 0x2e, 0x76, 0xee, 0xcf, 0x7b, 0xc0, 0xdf, 0x09, 0x9e, 0x26, 0xe8,
|
||||
0x05, 0xd8, 0xc9, 0x36, 0x26, 0x1d, 0x43, 0xf7, 0xf8, 0xf1, 0x7f, 0xc2, 0x69, 0xda, 0x78, 0x2f,
|
||||
0xd7, 0xfd, 0xdd, 0x80, 0xf2, 0x0f, 0x29, 0x15, 0xab, 0xbb, 0xdf, 0xc1, 0x13, 0x68, 0xec, 0xdd,
|
||||
0x81, 0x74, 0x4a, 0x1d, 0xb3, 0x7b, 0x82, 0xeb, 0xbb, 0x97, 0x20, 0x33, 0x79, 0x7c, 0x19, 0x39,
|
||||
0x66, 0x2e, 0x8f, 0x2f, 0x23, 0xf4, 0x19, 0xb4, 0x76, 0xb0, 0x67, 0x41, 0xd6, 0x8c, 0x63, 0x75,
|
||||
0x8c, 0xae, 0x8d, 0x9b, 0xc9, 0xad, 0x26, 0xdd, 0x9f, 0xa0, 0x31, 0x51, 0x82, 0xc5, 0x01, 0xa6,
|
||||
0x32, 0xe1, 0xb1, 0xa4, 0xe8, 0x29, 0x54, 0xa4, 0x22, 0x2a, 0x95, 0x9a, 0x57, 0xed, 0xfc, 0xec,
|
||||
0xe0, 0xa5, 0x4e, 0xf4, 0x11, 0x5c, 0x1c, 0x45, 0x6d, 0x28, 0x6b, 0x25, 0x8b, 0x41, 0xc9, 0x37,
|
||||
0xee, 0x1b, 0xb0, 0x87, 0x9c, 0x47, 0x0f, 0x58, 0xba, 0xba, 0x2e, 0x4d, 0x00, 0xe5, 0xbc, 0x5f,
|
||||
0x31, 0xa9, 0x8e, 0x03, 0xd8, 0xce, 0x44, 0x2e, 0xf0, 0x7a, 0x26, 0xe6, 0xf0, 0xde, 0x65, 0xac,
|
||||
0x68, 0x40, 0xc5, 0x43, 0x63, 0x98, 0x1b, 0x0c, 0x09, 0xed, 0x02, 0x03, 0x93, 0x38, 0xa0, 0x47,
|
||||
0x2b, 0x35, 0xa7, 0x01, 0x8b, 0xb5, 0x52, 0x26, 0xce, 0x37, 0xd9, 0x80, 0xd0, 0xd8, 0xd7, 0x03,
|
||||
0x62, 0xe2, 0x6c, 0xe9, 0xfe, 0x65, 0xc0, 0xfb, 0x5b, 0x6f, 0xba, 0xa0, 0xd2, 0x13, 0x2c, 0xc9,
|
||||
0x96, 0xf7, 0x83, 0xfd, 0x1a, 0x2a, 0xb9, 0xf3, 0x69, 0xdc, 0xda, 0x3f, 0x1e, 0x64, 0xee, 0x8a,
|
||||
0x5b, 0xc0, 0x89, 0x0e, 0xe0, 0x22, 0x09, 0x0d, 0x00, 0xb2, 0x42, 0x4c, 0x2a, 0xe6, 0xc9, 0xc2,
|
||||
0x48, 0x3e, 0x3a, 0x88, 0xfb, 0x92, 0xae, 0xf4, 0xdb, 0xba, 0x26, 0x4c, 0xe0, 0x9d, 0x24, 0xf7,
|
||||
0x4f, 0x03, 0xda, 0x1b, 0xc7, 0x3c, 0xba, 0x9f, 0xaf, 0xc0, 0xd2, 0xcf, 0x32, 0xef, 0xe6, 0xf1,
|
||||
0xbf, 0xbc, 0xf7, 0x5d, 0x83, 0xc6, 0x3a, 0xe1, 0x21, 0x3a, 0x51, 0xd0, 0x9a, 0xac, 0xe4, 0x88,
|
||||
0xc7, 0xbf, 0xb0, 0x23, 0x5f, 0x24, 0x02, 0x4b, 0x5b, 0x6c, 0x3e, 0xd3, 0x7a, 0x7d, 0xcb, 0xfd,
|
||||
0xb6, 0x93, 0xfe, 0x05, 0x94, 0x27, 0x1e, 0x17, 0x87, 0xbc, 0x76, 0x7f, 0x70, 0x4b, 0x9b, 0x94,
|
||||
0xdf, 0x0c, 0xb0, 0xbe, 0x67, 0x4a, 0xfb, 0xcf, 0xe5, 0x45, 0x6e, 0x8e, 0x26, 0xce, 0x96, 0x7b,
|
||||
0xff, 0x85, 0xd2, 0x9d, 0xff, 0x0b, 0x59, 0x93, 0x19, 0x87, 0xb5, 0x70, 0x67, 0x87, 0x75, 0xd7,
|
||||
0x3c, 0x71, 0x71, 0xd4, 0x15, 0x50, 0xd3, 0xae, 0x8a, 0xa9, 0x4c, 0x23, 0x75, 0x3f, 0xa1, 0x7a,
|
||||
0x60, 0x85, 0x4c, 0xc9, 0x82, 0xea, 0xe9, 0x61, 0xd8, 0xac, 0x55, 0xac, 0xcf, 0x7d, 0xfa, 0x0d,
|
||||
0x3c, 0xba, 0xf5, 0x6f, 0x41, 0x55, 0xb0, 0xc6, 0x57, 0xe3, 0xe7, 0xcd, 0x77, 0x50, 0x0b, 0xea,
|
||||
0xaf, 0x9f, 0x8f, 0xa6, 0x57, 0x78, 0x36, 0xbc, 0x1c, 0x0f, 0xf0, 0x9b, 0xa6, 0x8f, 0x9a, 0x60,
|
||||
0x17, 0xa1, 0x6f, 0x5f, 0x5d, 0x0d, 0xa6, 0x4d, 0x3a, 0x1c, 0xfd, 0x38, 0x08, 0x98, 0x0a, 0xd3,
|
||||
0x79, 0xc6, 0xa8, 0xff, 0x96, 0x45, 0x11, 0x7b, 0xab, 0xa8, 0x17, 0xf6, 0x73, 0xe8, 0xcf, 0x7d,
|
||||
0x26, 0x95, 0x60, 0xf3, 0x54, 0x51, 0xbf, 0xcf, 0x62, 0x45, 0x45, 0x4c, 0xa2, 0xbe, 0xe6, 0xd3,
|
||||
0x2f, 0xf8, 0x24, 0xf3, 0x79, 0x45, 0x07, 0x9e, 0xfe, 0x1d, 0x00, 0x00, 0xff, 0xff, 0xa7, 0x8a,
|
||||
0x8b, 0xe7, 0xad, 0x08, 0x00, 0x00,
|
||||
}
|
||||
|
|
|
@ -36,9 +36,6 @@ func (p *Proxy) Insert(ctx context.Context, in *servicepb.RowBatch) (*servicepb.
|
|||
manipulationMsgStream: p.manipulationMsgStream,
|
||||
rowIDAllocator: p.idAllocator,
|
||||
}
|
||||
if len(it.PartitionTag) <= 0 {
|
||||
it.PartitionTag = Params.defaultPartitionTag()
|
||||
}
|
||||
|
||||
var cancel func()
|
||||
it.ctx, cancel = context.WithTimeout(ctx, reqTimeoutInterval)
|
||||
|
@ -85,8 +82,9 @@ func (p *Proxy) CreateCollection(ctx context.Context, req *schemapb.CollectionSc
|
|||
Schema: &commonpb.Blob{},
|
||||
},
|
||||
masterClient: p.masterClient,
|
||||
schema: req,
|
||||
}
|
||||
schemaBytes, _ := proto.Marshal(req)
|
||||
cct.CreateCollectionRequest.Schema.Value = schemaBytes
|
||||
var cancel func()
|
||||
cct.ctx, cancel = context.WithTimeout(ctx, reqTimeoutInterval)
|
||||
defer cancel()
|
||||
|
@ -127,7 +125,6 @@ func (p *Proxy) Search(ctx context.Context, req *servicepb.Query) (*servicepb.Qu
|
|||
},
|
||||
queryMsgStream: p.queryMsgStream,
|
||||
resultBuf: make(chan []*internalpb.SearchResult),
|
||||
query: req,
|
||||
}
|
||||
var cancel func()
|
||||
qt.ctx, cancel = context.WithTimeout(ctx, reqTimeoutInterval)
|
||||
|
|
|
@ -31,10 +31,6 @@ func (pt *ParamTable) Init() {
|
|||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
err = pt.LoadYaml("advanced/common.yaml")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
|
||||
proxyIDStr := os.Getenv("PROXY_ID")
|
||||
if proxyIDStr == "" {
|
||||
|
@ -100,27 +96,6 @@ func (pt *ParamTable) ProxyIDList() []UniqueID {
|
|||
return ret
|
||||
}
|
||||
|
||||
func (pt *ParamTable) queryNodeNum() int {
|
||||
return len(pt.queryNodeIDList())
|
||||
}
|
||||
|
||||
func (pt *ParamTable) queryNodeIDList() []UniqueID {
|
||||
queryNodeIDStr, err := pt.Load("nodeID.queryNodeIDList")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
var ret []UniqueID
|
||||
queryNodeIDs := strings.Split(queryNodeIDStr, ",")
|
||||
for _, i := range queryNodeIDs {
|
||||
v, err := strconv.Atoi(i)
|
||||
if err != nil {
|
||||
log.Panicf("load proxy id list error, %s", err.Error())
|
||||
}
|
||||
ret = append(ret, UniqueID(v))
|
||||
}
|
||||
return ret
|
||||
}
|
||||
|
||||
func (pt *ParamTable) ProxyID() UniqueID {
|
||||
proxyID, err := pt.Load("_proxyID")
|
||||
if err != nil {
|
||||
|
@ -347,143 +322,3 @@ func (pt *ParamTable) MsgStreamSearchResultPulsarBufSize() int64 {
|
|||
func (pt *ParamTable) MsgStreamTimeTickBufSize() int64 {
|
||||
return pt.parseInt64("proxy.msgStream.timeTick.bufSize")
|
||||
}
|
||||
|
||||
func (pt *ParamTable) insertChannelNames() []string {
|
||||
ch, err := pt.Load("msgChannel.chanNamePrefix.insert")
|
||||
if err != nil {
|
||||
log.Fatal(err)
|
||||
}
|
||||
channelRange, err := pt.Load("msgChannel.channelRange.insert")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
|
||||
chanRange := strings.Split(channelRange, ",")
|
||||
if len(chanRange) != 2 {
|
||||
panic("Illegal channel range num")
|
||||
}
|
||||
channelBegin, err := strconv.Atoi(chanRange[0])
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
channelEnd, err := strconv.Atoi(chanRange[1])
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
if channelBegin < 0 || channelEnd < 0 {
|
||||
panic("Illegal channel range value")
|
||||
}
|
||||
if channelBegin > channelEnd {
|
||||
panic("Illegal channel range value")
|
||||
}
|
||||
|
||||
channels := make([]string, channelEnd-channelBegin)
|
||||
for i := 0; i < channelEnd-channelBegin; i++ {
|
||||
channels[i] = ch + "-" + strconv.Itoa(channelBegin+i)
|
||||
}
|
||||
return channels
|
||||
}
|
||||
|
||||
func (pt *ParamTable) searchChannelNames() []string {
|
||||
ch, err := pt.Load("msgChannel.chanNamePrefix.search")
|
||||
if err != nil {
|
||||
log.Fatal(err)
|
||||
}
|
||||
channelRange, err := pt.Load("msgChannel.channelRange.search")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
|
||||
chanRange := strings.Split(channelRange, ",")
|
||||
if len(chanRange) != 2 {
|
||||
panic("Illegal channel range num")
|
||||
}
|
||||
channelBegin, err := strconv.Atoi(chanRange[0])
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
channelEnd, err := strconv.Atoi(chanRange[1])
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
if channelBegin < 0 || channelEnd < 0 {
|
||||
panic("Illegal channel range value")
|
||||
}
|
||||
if channelBegin > channelEnd {
|
||||
panic("Illegal channel range value")
|
||||
}
|
||||
|
||||
channels := make([]string, channelEnd-channelBegin)
|
||||
for i := 0; i < channelEnd-channelBegin; i++ {
|
||||
channels[i] = ch + "-" + strconv.Itoa(channelBegin+i)
|
||||
}
|
||||
return channels
|
||||
}
|
||||
|
||||
func (pt *ParamTable) searchResultChannelNames() []string {
|
||||
ch, err := pt.Load("msgChannel.chanNamePrefix.searchResult")
|
||||
if err != nil {
|
||||
log.Fatal(err)
|
||||
}
|
||||
channelRange, err := pt.Load("msgChannel.channelRange.searchResult")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
|
||||
chanRange := strings.Split(channelRange, ",")
|
||||
if len(chanRange) != 2 {
|
||||
panic("Illegal channel range num")
|
||||
}
|
||||
channelBegin, err := strconv.Atoi(chanRange[0])
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
channelEnd, err := strconv.Atoi(chanRange[1])
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
if channelBegin < 0 || channelEnd < 0 {
|
||||
panic("Illegal channel range value")
|
||||
}
|
||||
if channelBegin > channelEnd {
|
||||
panic("Illegal channel range value")
|
||||
}
|
||||
|
||||
channels := make([]string, channelEnd-channelBegin)
|
||||
for i := 0; i < channelEnd-channelBegin; i++ {
|
||||
channels[i] = ch + "-" + strconv.Itoa(channelBegin+i)
|
||||
}
|
||||
return channels
|
||||
}
|
||||
|
||||
func (pt *ParamTable) MaxNameLength() int64 {
|
||||
str, err := pt.Load("proxy.maxNameLength")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
maxNameLength, err := strconv.ParseInt(str, 10, 64)
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
return maxNameLength
|
||||
}
|
||||
|
||||
func (pt *ParamTable) MaxFieldNum() int64 {
|
||||
str, err := pt.Load("proxy.maxFieldNum")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
maxFieldNum, err := strconv.ParseInt(str, 10, 64)
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
return maxFieldNum
|
||||
}
|
||||
|
||||
func (pt *ParamTable) defaultPartitionTag() string {
|
||||
tag, err := pt.Load("common.defaultPartitionTag")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
return tag
|
||||
}
|
||||
|
|
|
@ -69,8 +69,3 @@ func TestParamTable_MsgStreamTimeTickBufSize(t *testing.T) {
|
|||
ret := Params.MsgStreamTimeTickBufSize()
|
||||
fmt.Println(ret)
|
||||
}
|
||||
|
||||
func TestParamTable_defaultPartitionTag(t *testing.T) {
|
||||
ret := Params.defaultPartitionTag()
|
||||
fmt.Println("default partition tag: ", ret)
|
||||
}
|
||||
|
|
|
@ -55,11 +55,12 @@ func CreateProxy(ctx context.Context) (*Proxy, error) {
|
|||
proxyLoopCancel: cancel,
|
||||
}
|
||||
|
||||
// TODO: use config instead
|
||||
pulsarAddress := Params.PulsarAddress()
|
||||
|
||||
p.queryMsgStream = msgstream.NewPulsarMsgStream(p.proxyLoopCtx, Params.MsgStreamSearchBufSize())
|
||||
p.queryMsgStream.SetPulsarClient(pulsarAddress)
|
||||
p.queryMsgStream.CreatePulsarProducers(Params.searchChannelNames())
|
||||
p.queryMsgStream.CreatePulsarProducers(Params.SearchChannelNames())
|
||||
|
||||
masterAddr := Params.MasterAddress()
|
||||
idAllocator, err := allocator.NewIDAllocator(p.proxyLoopCtx, masterAddr)
|
||||
|
@ -83,7 +84,7 @@ func CreateProxy(ctx context.Context) (*Proxy, error) {
|
|||
|
||||
p.manipulationMsgStream = msgstream.NewPulsarMsgStream(p.proxyLoopCtx, Params.MsgStreamInsertBufSize())
|
||||
p.manipulationMsgStream.SetPulsarClient(pulsarAddress)
|
||||
p.manipulationMsgStream.CreatePulsarProducers(Params.insertChannelNames())
|
||||
p.manipulationMsgStream.CreatePulsarProducers(Params.InsertChannelNames())
|
||||
repackFuncImpl := func(tsMsgs []msgstream.TsMsg, hashKeys [][]int32) (map[int32]*msgstream.MsgPack, error) {
|
||||
return insertRepackFunc(tsMsgs, hashKeys, p.segAssigner, false)
|
||||
}
|
||||
|
|
|
@ -229,7 +229,7 @@ func TestProxy_Insert(t *testing.T) {
|
|||
collectionName := "CreateCollection" + strconv.FormatInt(int64(i), 10)
|
||||
req := &servicepb.RowBatch{
|
||||
CollectionName: collectionName,
|
||||
PartitionTag: "haha",
|
||||
PartitionTag: "",
|
||||
RowData: make([]*commonpb.Blob, 0),
|
||||
HashKeys: make([]int32, 0),
|
||||
}
|
||||
|
@ -237,7 +237,6 @@ func TestProxy_Insert(t *testing.T) {
|
|||
wg.Add(1)
|
||||
go func(group *sync.WaitGroup) {
|
||||
defer group.Done()
|
||||
createCollection(t, collectionName)
|
||||
has := hasCollection(t, collectionName)
|
||||
if has {
|
||||
resp, err := proxyClient.Insert(ctx, req)
|
||||
|
|
|
@ -1,9 +1,6 @@
|
|||
package proxy
|
||||
|
||||
import (
|
||||
"log"
|
||||
"sort"
|
||||
|
||||
"github.com/zilliztech/milvus-distributed/internal/allocator"
|
||||
"github.com/zilliztech/milvus-distributed/internal/errors"
|
||||
"github.com/zilliztech/milvus-distributed/internal/msgstream"
|
||||
|
@ -18,9 +15,6 @@ func insertRepackFunc(tsMsgs []msgstream.TsMsg,
|
|||
|
||||
result := make(map[int32]*msgstream.MsgPack)
|
||||
|
||||
channelCountMap := make(map[UniqueID]map[int32]uint32) // reqID --> channelID to count
|
||||
reqSchemaMap := make(map[UniqueID][]string)
|
||||
|
||||
for i, request := range tsMsgs {
|
||||
if request.Type() != internalpb.MsgType_kInsert {
|
||||
return nil, errors.New(string("msg's must be Insert"))
|
||||
|
@ -29,8 +23,8 @@ func insertRepackFunc(tsMsgs []msgstream.TsMsg,
|
|||
if !ok {
|
||||
return nil, errors.New(string("msg's must be Insert"))
|
||||
}
|
||||
|
||||
keys := hashKeys[i]
|
||||
|
||||
timestampLen := len(insertRequest.Timestamps)
|
||||
rowIDLen := len(insertRequest.RowIDs)
|
||||
rowDataLen := len(insertRequest.RowData)
|
||||
|
@ -40,84 +34,10 @@ func insertRepackFunc(tsMsgs []msgstream.TsMsg,
|
|||
return nil, errors.New(string("the length of hashValue, timestamps, rowIDs, RowData are not equal"))
|
||||
}
|
||||
|
||||
reqID := insertRequest.ReqID
|
||||
if _, ok := channelCountMap[reqID]; !ok {
|
||||
channelCountMap[reqID] = make(map[int32]uint32)
|
||||
}
|
||||
|
||||
if _, ok := reqSchemaMap[reqID]; !ok {
|
||||
reqSchemaMap[reqID] = []string{insertRequest.CollectionName, insertRequest.PartitionTag}
|
||||
}
|
||||
|
||||
for _, channelID := range keys {
|
||||
channelCountMap[reqID][channelID]++
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
reqSegCountMap := make(map[UniqueID]map[int32]map[UniqueID]uint32)
|
||||
|
||||
for reqID, countInfo := range channelCountMap {
|
||||
schema := reqSchemaMap[reqID]
|
||||
collName, partitionTag := schema[0], schema[1]
|
||||
for channelID, count := range countInfo {
|
||||
mapInfo, err := segIDAssigner.GetSegmentID(collName, partitionTag, channelID, count)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
reqSegCountMap[reqID][channelID] = mapInfo
|
||||
}
|
||||
}
|
||||
|
||||
reqSegAccumulateCountMap := make(map[UniqueID]map[int32][]uint32)
|
||||
reqSegIDMap := make(map[UniqueID]map[int32][]UniqueID)
|
||||
reqSegAllocateCounter := make(map[UniqueID]map[int32]uint32)
|
||||
|
||||
for reqID, channelInfo := range reqSegCountMap {
|
||||
for channelID, segInfo := range channelInfo {
|
||||
reqSegAllocateCounter[reqID][channelID] = 0
|
||||
keys := make([]UniqueID, len(segInfo))
|
||||
i := 0
|
||||
for key := range segInfo {
|
||||
keys[i] = key
|
||||
i++
|
||||
}
|
||||
sort.Slice(keys, func(i, j int) bool { return keys[i] < keys[j] })
|
||||
accumulate := uint32(0)
|
||||
for _, key := range keys {
|
||||
accumulate += segInfo[key]
|
||||
reqSegAccumulateCountMap[reqID][channelID] = append(
|
||||
reqSegAccumulateCountMap[reqID][channelID],
|
||||
accumulate,
|
||||
)
|
||||
reqSegIDMap[reqID][channelID] = append(
|
||||
reqSegIDMap[reqID][channelID],
|
||||
key,
|
||||
)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
var getSegmentID = func(reqID UniqueID, channelID int32) UniqueID {
|
||||
reqSegAllocateCounter[reqID][channelID]++
|
||||
cur := reqSegAllocateCounter[reqID][channelID]
|
||||
accumulateSlice := reqSegAccumulateCountMap[reqID][channelID]
|
||||
segIDSlice := reqSegIDMap[reqID][channelID]
|
||||
for index, count := range accumulateSlice {
|
||||
if cur <= count {
|
||||
return segIDSlice[index]
|
||||
}
|
||||
}
|
||||
log.Panic("Can't Found SegmentID")
|
||||
return 0
|
||||
}
|
||||
|
||||
for i, request := range tsMsgs {
|
||||
insertRequest := request.(*msgstream.InsertMsg)
|
||||
keys := hashKeys[i]
|
||||
reqID := insertRequest.ReqID
|
||||
collectionName := insertRequest.CollectionName
|
||||
partitionTag := insertRequest.PartitionTag
|
||||
channelID := insertRequest.ChannelID
|
||||
proxyID := insertRequest.ProxyID
|
||||
for index, key := range keys {
|
||||
ts := insertRequest.Timestamps[index]
|
||||
|
@ -128,14 +48,13 @@ func insertRepackFunc(tsMsgs []msgstream.TsMsg,
|
|||
msgPack := msgstream.MsgPack{}
|
||||
result[key] = &msgPack
|
||||
}
|
||||
segmentID := getSegmentID(reqID, key)
|
||||
sliceRequest := internalpb.InsertRequest{
|
||||
MsgType: internalpb.MsgType_kInsert,
|
||||
ReqID: reqID,
|
||||
CollectionName: collectionName,
|
||||
PartitionTag: partitionTag,
|
||||
SegmentID: segmentID,
|
||||
ChannelID: int64(key),
|
||||
SegmentID: 0, // will be assigned later if together
|
||||
ChannelID: channelID,
|
||||
ProxyID: proxyID,
|
||||
Timestamps: []uint64{ts},
|
||||
RowIDs: []int64{rowID},
|
||||
|
@ -154,10 +73,25 @@ func insertRepackFunc(tsMsgs []msgstream.TsMsg,
|
|||
accMsgs.RowData = append(accMsgs.RowData, row)
|
||||
}
|
||||
} else { // every row is a message
|
||||
segID, _ := segIDAssigner.GetSegmentID(collectionName, partitionTag, int32(channelID), 1)
|
||||
insertMsg.SegmentID = segID
|
||||
result[key].Msgs = append(result[key].Msgs, insertMsg)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if together {
|
||||
for key := range result {
|
||||
insertMsg, _ := result[key].Msgs[0].(*msgstream.InsertMsg)
|
||||
rowNums := len(insertMsg.RowIDs)
|
||||
collectionName := insertMsg.CollectionName
|
||||
partitionTag := insertMsg.PartitionTag
|
||||
channelID := insertMsg.ChannelID
|
||||
segID, _ := segIDAssigner.GetSegmentID(collectionName, partitionTag, int32(channelID), uint32(rowNums))
|
||||
insertMsg.SegmentID = segID
|
||||
result[key].Msgs[0] = insertMsg
|
||||
}
|
||||
}
|
||||
|
||||
return result, nil
|
||||
}
|
||||
|
|
|
@ -4,15 +4,12 @@ import (
|
|||
"context"
|
||||
"errors"
|
||||
"log"
|
||||
"strconv"
|
||||
|
||||
"github.com/golang/protobuf/proto"
|
||||
"github.com/zilliztech/milvus-distributed/internal/allocator"
|
||||
"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/masterpb"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/schemapb"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/servicepb"
|
||||
)
|
||||
|
||||
|
@ -35,6 +32,7 @@ type BaseInsertTask = msgstream.InsertMsg
|
|||
type InsertTask struct {
|
||||
BaseInsertTask
|
||||
Condition
|
||||
ts Timestamp
|
||||
result *servicepb.IntegerRangeResponse
|
||||
manipulationMsgStream *msgstream.PulsarMsgStream
|
||||
ctx context.Context
|
||||
|
@ -46,21 +44,15 @@ func (it *InsertTask) SetID(uid UniqueID) {
|
|||
}
|
||||
|
||||
func (it *InsertTask) SetTs(ts Timestamp) {
|
||||
rowNum := len(it.RowData)
|
||||
it.Timestamps = make([]uint64, rowNum)
|
||||
for index := range it.Timestamps {
|
||||
it.Timestamps[index] = ts
|
||||
}
|
||||
it.BeginTimestamp = ts
|
||||
it.EndTimestamp = ts
|
||||
it.ts = ts
|
||||
}
|
||||
|
||||
func (it *InsertTask) BeginTs() Timestamp {
|
||||
return it.BeginTimestamp
|
||||
return it.ts
|
||||
}
|
||||
|
||||
func (it *InsertTask) EndTs() Timestamp {
|
||||
return it.EndTimestamp
|
||||
return it.ts
|
||||
}
|
||||
|
||||
func (it *InsertTask) ID() UniqueID {
|
||||
|
@ -72,15 +64,6 @@ func (it *InsertTask) Type() internalpb.MsgType {
|
|||
}
|
||||
|
||||
func (it *InsertTask) PreExecute() error {
|
||||
collectionName := it.BaseInsertTask.CollectionName
|
||||
if err := ValidateCollectionName(collectionName); err != nil {
|
||||
return err
|
||||
}
|
||||
partitionTag := it.BaseInsertTask.PartitionTag
|
||||
if err := ValidatePartitionTag(partitionTag, true); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
|
@ -137,7 +120,6 @@ type CreateCollectionTask struct {
|
|||
masterClient masterpb.MasterClient
|
||||
result *commonpb.Status
|
||||
ctx context.Context
|
||||
schema *schemapb.CollectionSchema
|
||||
}
|
||||
|
||||
func (cct *CreateCollectionTask) ID() UniqueID {
|
||||
|
@ -165,28 +147,10 @@ func (cct *CreateCollectionTask) SetTs(ts Timestamp) {
|
|||
}
|
||||
|
||||
func (cct *CreateCollectionTask) PreExecute() error {
|
||||
if int64(len(cct.schema.Fields)) > Params.MaxFieldNum() {
|
||||
return errors.New("maximum field's number should be limited to " + strconv.FormatInt(Params.MaxFieldNum(), 10))
|
||||
}
|
||||
|
||||
// validate collection name
|
||||
if err := ValidateCollectionName(cct.schema.Name); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
// validate field name
|
||||
for _, field := range cct.schema.Fields {
|
||||
if err := ValidateFieldName(field.Name); err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
func (cct *CreateCollectionTask) Execute() error {
|
||||
schemaBytes, _ := proto.Marshal(cct.schema)
|
||||
cct.CreateCollectionRequest.Schema.Value = schemaBytes
|
||||
resp, err := cct.masterClient.CreateCollection(cct.ctx, &cct.CreateCollectionRequest)
|
||||
if err != nil {
|
||||
log.Printf("create collection failed, error= %v", err)
|
||||
|
@ -237,9 +201,6 @@ func (dct *DropCollectionTask) SetTs(ts Timestamp) {
|
|||
}
|
||||
|
||||
func (dct *DropCollectionTask) PreExecute() error {
|
||||
if err := ValidateCollectionName(dct.CollectionName.CollectionName); err != nil {
|
||||
return err
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
|
@ -268,7 +229,6 @@ type QueryTask struct {
|
|||
resultBuf chan []*internalpb.SearchResult
|
||||
result *servicepb.QueryResult
|
||||
ctx context.Context
|
||||
query *servicepb.Query
|
||||
}
|
||||
|
||||
func (qt *QueryTask) ID() UniqueID {
|
||||
|
@ -296,15 +256,6 @@ func (qt *QueryTask) SetTs(ts Timestamp) {
|
|||
}
|
||||
|
||||
func (qt *QueryTask) PreExecute() error {
|
||||
if err := ValidateCollectionName(qt.query.CollectionName); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
for _, tag := range qt.query.PartitionTags {
|
||||
if err := ValidatePartitionTag(tag, false); err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
|
@ -416,9 +367,6 @@ func (hct *HasCollectionTask) SetTs(ts Timestamp) {
|
|||
}
|
||||
|
||||
func (hct *HasCollectionTask) PreExecute() error {
|
||||
if err := ValidateCollectionName(hct.CollectionName.CollectionName); err != nil {
|
||||
return err
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
|
@ -476,9 +424,6 @@ func (dct *DescribeCollectionTask) SetTs(ts Timestamp) {
|
|||
}
|
||||
|
||||
func (dct *DescribeCollectionTask) PreExecute() error {
|
||||
if err := ValidateCollectionName(dct.CollectionName.CollectionName); err != nil {
|
||||
return err
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
|
@ -587,16 +532,6 @@ func (cpt *CreatePartitionTask) SetTs(ts Timestamp) {
|
|||
}
|
||||
|
||||
func (cpt *CreatePartitionTask) PreExecute() error {
|
||||
collName, partitionTag := cpt.PartitionName.CollectionName, cpt.PartitionName.Tag
|
||||
|
||||
if err := ValidateCollectionName(collName); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
if err := ValidatePartitionTag(partitionTag, true); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
|
@ -642,16 +577,6 @@ func (dpt *DropPartitionTask) SetTs(ts Timestamp) {
|
|||
}
|
||||
|
||||
func (dpt *DropPartitionTask) PreExecute() error {
|
||||
collName, partitionTag := dpt.PartitionName.CollectionName, dpt.PartitionName.Tag
|
||||
|
||||
if err := ValidateCollectionName(collName); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
if err := ValidatePartitionTag(partitionTag, true); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
|
@ -697,15 +622,6 @@ func (hpt *HasPartitionTask) SetTs(ts Timestamp) {
|
|||
}
|
||||
|
||||
func (hpt *HasPartitionTask) PreExecute() error {
|
||||
collName, partitionTag := hpt.PartitionName.CollectionName, hpt.PartitionName.Tag
|
||||
|
||||
if err := ValidateCollectionName(collName); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
if err := ValidatePartitionTag(partitionTag, true); err != nil {
|
||||
return err
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
|
@ -751,15 +667,6 @@ func (dpt *DescribePartitionTask) SetTs(ts Timestamp) {
|
|||
}
|
||||
|
||||
func (dpt *DescribePartitionTask) PreExecute() error {
|
||||
collName, partitionTag := dpt.PartitionName.CollectionName, dpt.PartitionName.Tag
|
||||
|
||||
if err := ValidateCollectionName(collName); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
if err := ValidatePartitionTag(partitionTag, true); err != nil {
|
||||
return err
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
|
@ -805,9 +712,6 @@ func (spt *ShowPartitionsTask) SetTs(ts Timestamp) {
|
|||
}
|
||||
|
||||
func (spt *ShowPartitionsTask) PreExecute() error {
|
||||
if err := ValidateCollectionName(spt.CollectionName.CollectionName); err != nil {
|
||||
return err
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
|
|
|
@ -186,7 +186,16 @@ type DqTaskQueue struct {
|
|||
func (queue *DdTaskQueue) Enqueue(t task) error {
|
||||
queue.lock.Lock()
|
||||
defer queue.lock.Unlock()
|
||||
return queue.BaseTaskQueue.Enqueue(t)
|
||||
|
||||
ts, _ := queue.sched.tsoAllocator.AllocOne()
|
||||
log.Printf("[Proxy] allocate timestamp: %v", ts)
|
||||
t.SetTs(ts)
|
||||
|
||||
reqID, _ := queue.sched.idAllocator.AllocOne()
|
||||
log.Printf("[Proxy] allocate reqID: %v", reqID)
|
||||
t.SetID(reqID)
|
||||
|
||||
return queue.addUnissuedTask(t)
|
||||
}
|
||||
|
||||
func NewDdTaskQueue(sched *TaskScheduler) *DdTaskQueue {
|
||||
|
@ -360,14 +369,14 @@ func (sched *TaskScheduler) queryLoop() {
|
|||
func (sched *TaskScheduler) queryResultLoop() {
|
||||
defer sched.wg.Done()
|
||||
|
||||
// TODO: use config instead
|
||||
unmarshal := msgstream.NewUnmarshalDispatcher()
|
||||
queryResultMsgStream := msgstream.NewPulsarMsgStream(sched.ctx, Params.MsgStreamSearchResultBufSize())
|
||||
queryResultMsgStream.SetPulsarClient(Params.PulsarAddress())
|
||||
queryResultMsgStream.CreatePulsarConsumers(Params.searchResultChannelNames(),
|
||||
queryResultMsgStream.CreatePulsarConsumers(Params.SearchResultChannelNames(),
|
||||
Params.ProxySubName(),
|
||||
unmarshal,
|
||||
Params.MsgStreamSearchResultPulsarBufSize())
|
||||
queryNodeNum := Params.queryNodeNum()
|
||||
|
||||
queryResultMsgStream.Start()
|
||||
defer queryResultMsgStream.Close()
|
||||
|
@ -392,7 +401,8 @@ func (sched *TaskScheduler) queryResultLoop() {
|
|||
queryResultBuf[reqID] = make([]*internalpb.SearchResult, 0)
|
||||
}
|
||||
queryResultBuf[reqID] = append(queryResultBuf[reqID], &searchResultMsg.SearchResult)
|
||||
if len(queryResultBuf[reqID]) == queryNodeNum {
|
||||
if len(queryResultBuf[reqID]) == 4 {
|
||||
// TODO: use the number of query node instead
|
||||
t := sched.getTaskByReqID(reqID)
|
||||
if t != nil {
|
||||
qt, ok := t.(*QueryTask)
|
||||
|
|
|
@ -1,118 +0,0 @@
|
|||
package proxy
|
||||
|
||||
import (
|
||||
"strconv"
|
||||
"strings"
|
||||
|
||||
"github.com/zilliztech/milvus-distributed/internal/errors"
|
||||
)
|
||||
|
||||
func isAlpha(c uint8) bool {
|
||||
if (c < 'A' || c > 'Z') && (c < 'a' || c > 'z') {
|
||||
return false
|
||||
}
|
||||
return true
|
||||
}
|
||||
|
||||
func isNumber(c uint8) bool {
|
||||
if c < '0' || c > '9' {
|
||||
return false
|
||||
}
|
||||
return true
|
||||
}
|
||||
|
||||
func ValidateCollectionName(collName string) error {
|
||||
collName = strings.TrimSpace(collName)
|
||||
|
||||
if collName == "" {
|
||||
return errors.New("Collection name should not be empty")
|
||||
}
|
||||
|
||||
invalidMsg := "Invalid collection name: " + collName + ". "
|
||||
if int64(len(collName)) > Params.MaxNameLength() {
|
||||
msg := invalidMsg + "The length of a collection name must be less than " +
|
||||
strconv.FormatInt(Params.MaxNameLength(), 10) + " characters."
|
||||
return errors.New(msg)
|
||||
}
|
||||
|
||||
firstChar := collName[0]
|
||||
if firstChar != '_' && !isAlpha(firstChar) {
|
||||
msg := invalidMsg + "The first character of a collection name must be an underscore or letter."
|
||||
return errors.New(msg)
|
||||
}
|
||||
|
||||
for i := 1; i < len(collName); i++ {
|
||||
c := collName[i]
|
||||
if c != '_' && c != '$' && !isAlpha(c) && !isNumber(c) {
|
||||
msg := invalidMsg + "Collection name can only contain numbers, letters, dollars and underscores."
|
||||
return errors.New(msg)
|
||||
}
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func ValidatePartitionTag(partitionTag string, strictCheck bool) error {
|
||||
partitionTag = strings.TrimSpace(partitionTag)
|
||||
|
||||
invalidMsg := "Invalid partition tag: " + partitionTag + ". "
|
||||
if partitionTag == "" {
|
||||
msg := invalidMsg + "Partition tag should not be empty."
|
||||
return errors.New(msg)
|
||||
}
|
||||
|
||||
if int64(len(partitionTag)) > Params.MaxNameLength() {
|
||||
msg := invalidMsg + "The length of a partition tag must be less than " +
|
||||
strconv.FormatInt(Params.MaxNameLength(), 10) + " characters."
|
||||
return errors.New(msg)
|
||||
}
|
||||
|
||||
if strictCheck {
|
||||
firstChar := partitionTag[0]
|
||||
if firstChar != '_' && !isAlpha(firstChar) && !isNumber(firstChar) {
|
||||
msg := invalidMsg + "The first character of a partition tag must be an underscore or letter."
|
||||
return errors.New(msg)
|
||||
}
|
||||
|
||||
tagSize := len(partitionTag)
|
||||
for i := 1; i < tagSize; i++ {
|
||||
c := partitionTag[i]
|
||||
if c != '_' && c != '$' && !isAlpha(c) && !isNumber(c) {
|
||||
msg := invalidMsg + "Partition tag can only contain numbers, letters, dollars and underscores."
|
||||
return errors.New(msg)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
func ValidateFieldName(fieldName string) error {
|
||||
fieldName = strings.TrimSpace(fieldName)
|
||||
|
||||
if fieldName == "" {
|
||||
return errors.New("Field name should not be empty")
|
||||
}
|
||||
|
||||
invalidMsg := "Invalid field name: " + fieldName + ". "
|
||||
if int64(len(fieldName)) > Params.MaxNameLength() {
|
||||
msg := invalidMsg + "The length of a field name must be less than " +
|
||||
strconv.FormatInt(Params.MaxNameLength(), 10) + " characters."
|
||||
return errors.New(msg)
|
||||
}
|
||||
|
||||
firstChar := fieldName[0]
|
||||
if firstChar != '_' && !isAlpha(firstChar) {
|
||||
msg := invalidMsg + "The first character of a field name must be an underscore or letter."
|
||||
return errors.New(msg)
|
||||
}
|
||||
|
||||
fieldNameSize := len(fieldName)
|
||||
for i := 1; i < fieldNameSize; i++ {
|
||||
c := fieldName[i]
|
||||
if c != '_' && !isAlpha(c) && !isNumber(c) {
|
||||
msg := invalidMsg + "Field name cannot only contain numbers, letters, and underscores."
|
||||
return errors.New(msg)
|
||||
}
|
||||
}
|
||||
return nil
|
||||
}
|
|
@ -1,84 +0,0 @@
|
|||
package proxy
|
||||
|
||||
import (
|
||||
"testing"
|
||||
|
||||
"github.com/stretchr/testify/assert"
|
||||
)
|
||||
|
||||
func TestValidateCollectionName(t *testing.T) {
|
||||
Params.Init()
|
||||
assert.Nil(t, ValidateCollectionName("abc"))
|
||||
assert.Nil(t, ValidateCollectionName("_123abc"))
|
||||
assert.Nil(t, ValidateCollectionName("abc123_$"))
|
||||
|
||||
longName := make([]byte, 256)
|
||||
for i := 0; i < len(longName); i++ {
|
||||
longName[i] = 'a'
|
||||
}
|
||||
invalidNames := []string{
|
||||
"123abc",
|
||||
"$abc",
|
||||
"_12 ac",
|
||||
" ",
|
||||
"",
|
||||
string(longName),
|
||||
"中文",
|
||||
}
|
||||
|
||||
for _, name := range invalidNames {
|
||||
assert.NotNil(t, ValidateCollectionName(name))
|
||||
}
|
||||
}
|
||||
|
||||
func TestValidatePartitionTag(t *testing.T) {
|
||||
Params.Init()
|
||||
assert.Nil(t, ValidatePartitionTag("abc", true))
|
||||
assert.Nil(t, ValidatePartitionTag("_123abc", true))
|
||||
assert.Nil(t, ValidatePartitionTag("abc123_$", true))
|
||||
|
||||
longName := make([]byte, 256)
|
||||
for i := 0; i < len(longName); i++ {
|
||||
longName[i] = 'a'
|
||||
}
|
||||
invalidNames := []string{
|
||||
"123abc",
|
||||
"$abc",
|
||||
"_12 ac",
|
||||
" ",
|
||||
"",
|
||||
string(longName),
|
||||
"中文",
|
||||
}
|
||||
|
||||
for _, name := range invalidNames {
|
||||
assert.NotNil(t, ValidatePartitionTag(name, true))
|
||||
}
|
||||
|
||||
assert.Nil(t, ValidatePartitionTag("ab cd", false))
|
||||
assert.Nil(t, ValidatePartitionTag("ab*", false))
|
||||
}
|
||||
|
||||
func TestValidateFieldName(t *testing.T) {
|
||||
Params.Init()
|
||||
assert.Nil(t, ValidateFieldName("abc"))
|
||||
assert.Nil(t, ValidateFieldName("_123abc"))
|
||||
|
||||
longName := make([]byte, 256)
|
||||
for i := 0; i < len(longName); i++ {
|
||||
longName[i] = 'a'
|
||||
}
|
||||
invalidNames := []string{
|
||||
"123abc",
|
||||
"$abc",
|
||||
"_12 ac",
|
||||
" ",
|
||||
"",
|
||||
string(longName),
|
||||
"中文",
|
||||
}
|
||||
|
||||
for _, name := range invalidNames {
|
||||
assert.NotNil(t, ValidateFieldName(name))
|
||||
}
|
||||
}
|
|
@ -1,327 +0,0 @@
|
|||
package querynode
|
||||
|
||||
import (
|
||||
"log"
|
||||
"strconv"
|
||||
"strings"
|
||||
|
||||
"github.com/zilliztech/milvus-distributed/internal/util/paramtable"
|
||||
)
|
||||
|
||||
type ParamTable struct {
|
||||
paramtable.BaseTable
|
||||
}
|
||||
|
||||
var Params ParamTable
|
||||
|
||||
func (p *ParamTable) Init() {
|
||||
p.BaseTable.Init()
|
||||
err := p.LoadYaml("advanced/query_node.yaml")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
err = p.LoadYaml("advanced/channel.yaml")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
}
|
||||
|
||||
func (p *ParamTable) pulsarAddress() (string, error) {
|
||||
url, err := p.Load("_PulsarAddress")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
return url, nil
|
||||
}
|
||||
|
||||
func (p *ParamTable) queryNodeID() int {
|
||||
queryNodeID, err := p.Load("reader.clientid")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
id, err := strconv.Atoi(queryNodeID)
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
return id
|
||||
}
|
||||
|
||||
func (p *ParamTable) insertChannelRange() []int {
|
||||
insertChannelRange, err := p.Load("msgChannel.channelRange.insert")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
|
||||
channelRange := strings.Split(insertChannelRange, ",")
|
||||
if len(channelRange) != 2 {
|
||||
panic("Illegal channel range num")
|
||||
}
|
||||
channelBegin, err := strconv.Atoi(channelRange[0])
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
channelEnd, err := strconv.Atoi(channelRange[1])
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
if channelBegin < 0 || channelEnd < 0 {
|
||||
panic("Illegal channel range value")
|
||||
}
|
||||
if channelBegin > channelEnd {
|
||||
panic("Illegal channel range value")
|
||||
}
|
||||
return []int{channelBegin, channelEnd}
|
||||
}
|
||||
|
||||
// advanced params
|
||||
// stats
|
||||
func (p *ParamTable) statsPublishInterval() int {
|
||||
timeInterval, err := p.Load("queryNode.stats.publishInterval")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
interval, err := strconv.Atoi(timeInterval)
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
return interval
|
||||
}
|
||||
|
||||
// dataSync:
|
||||
func (p *ParamTable) flowGraphMaxQueueLength() int32 {
|
||||
queueLength, err := p.Load("queryNode.dataSync.flowGraph.maxQueueLength")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
length, err := strconv.Atoi(queueLength)
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
return int32(length)
|
||||
}
|
||||
|
||||
func (p *ParamTable) flowGraphMaxParallelism() int32 {
|
||||
maxParallelism, err := p.Load("queryNode.dataSync.flowGraph.maxParallelism")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
maxPara, err := strconv.Atoi(maxParallelism)
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
return int32(maxPara)
|
||||
}
|
||||
|
||||
// msgStream
|
||||
func (p *ParamTable) insertReceiveBufSize() int64 {
|
||||
revBufSize, err := p.Load("queryNode.msgStream.insert.recvBufSize")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
bufSize, err := strconv.Atoi(revBufSize)
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
return int64(bufSize)
|
||||
}
|
||||
|
||||
func (p *ParamTable) insertPulsarBufSize() int64 {
|
||||
pulsarBufSize, err := p.Load("queryNode.msgStream.insert.pulsarBufSize")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
bufSize, err := strconv.Atoi(pulsarBufSize)
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
return int64(bufSize)
|
||||
}
|
||||
|
||||
func (p *ParamTable) searchReceiveBufSize() int64 {
|
||||
revBufSize, err := p.Load("queryNode.msgStream.search.recvBufSize")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
bufSize, err := strconv.Atoi(revBufSize)
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
return int64(bufSize)
|
||||
}
|
||||
|
||||
func (p *ParamTable) searchPulsarBufSize() int64 {
|
||||
pulsarBufSize, err := p.Load("queryNode.msgStream.search.pulsarBufSize")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
bufSize, err := strconv.Atoi(pulsarBufSize)
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
return int64(bufSize)
|
||||
}
|
||||
|
||||
func (p *ParamTable) searchResultReceiveBufSize() int64 {
|
||||
revBufSize, err := p.Load("queryNode.msgStream.searchResult.recvBufSize")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
bufSize, err := strconv.Atoi(revBufSize)
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
return int64(bufSize)
|
||||
}
|
||||
|
||||
func (p *ParamTable) statsReceiveBufSize() int64 {
|
||||
revBufSize, err := p.Load("queryNode.msgStream.stats.recvBufSize")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
bufSize, err := strconv.Atoi(revBufSize)
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
return int64(bufSize)
|
||||
}
|
||||
|
||||
func (p *ParamTable) etcdAddress() string {
|
||||
etcdAddress, err := p.Load("_EtcdAddress")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
return etcdAddress
|
||||
}
|
||||
|
||||
func (p *ParamTable) etcdRootPath() string {
|
||||
etcdRootPath, err := p.Load("etcd.rootpath")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
return etcdRootPath
|
||||
}
|
||||
|
||||
func (p *ParamTable) insertChannelNames() []string {
|
||||
ch, err := p.Load("msgChannel.chanNamePrefix.insert")
|
||||
if err != nil {
|
||||
log.Fatal(err)
|
||||
}
|
||||
channelRange, err := p.Load("msgChannel.channelRange.insert")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
|
||||
chanRange := strings.Split(channelRange, ",")
|
||||
if len(chanRange) != 2 {
|
||||
panic("Illegal channel range num")
|
||||
}
|
||||
channelBegin, err := strconv.Atoi(chanRange[0])
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
channelEnd, err := strconv.Atoi(chanRange[1])
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
if channelBegin < 0 || channelEnd < 0 {
|
||||
panic("Illegal channel range value")
|
||||
}
|
||||
if channelBegin > channelEnd {
|
||||
panic("Illegal channel range value")
|
||||
}
|
||||
|
||||
channels := make([]string, channelEnd-channelBegin)
|
||||
for i := 0; i < channelEnd-channelBegin; i++ {
|
||||
channels[i] = ch + "-" + strconv.Itoa(channelBegin+i)
|
||||
}
|
||||
return channels
|
||||
}
|
||||
|
||||
func (p *ParamTable) searchChannelNames() []string {
|
||||
ch, err := p.Load("msgChannel.chanNamePrefix.search")
|
||||
if err != nil {
|
||||
log.Fatal(err)
|
||||
}
|
||||
channelRange, err := p.Load("msgChannel.channelRange.search")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
|
||||
chanRange := strings.Split(channelRange, ",")
|
||||
if len(chanRange) != 2 {
|
||||
panic("Illegal channel range num")
|
||||
}
|
||||
channelBegin, err := strconv.Atoi(chanRange[0])
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
channelEnd, err := strconv.Atoi(chanRange[1])
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
if channelBegin < 0 || channelEnd < 0 {
|
||||
panic("Illegal channel range value")
|
||||
}
|
||||
if channelBegin > channelEnd {
|
||||
panic("Illegal channel range value")
|
||||
}
|
||||
|
||||
channels := make([]string, channelEnd-channelBegin)
|
||||
for i := 0; i < channelEnd-channelBegin; i++ {
|
||||
channels[i] = ch + "-" + strconv.Itoa(channelBegin+i)
|
||||
}
|
||||
return channels
|
||||
}
|
||||
|
||||
func (p *ParamTable) searchResultChannelNames() []string {
|
||||
ch, err := p.Load("msgChannel.chanNamePrefix.searchResult")
|
||||
if err != nil {
|
||||
log.Fatal(err)
|
||||
}
|
||||
channelRange, err := p.Load("msgChannel.channelRange.searchResult")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
|
||||
chanRange := strings.Split(channelRange, ",")
|
||||
if len(chanRange) != 2 {
|
||||
panic("Illegal channel range num")
|
||||
}
|
||||
channelBegin, err := strconv.Atoi(chanRange[0])
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
channelEnd, err := strconv.Atoi(chanRange[1])
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
if channelBegin < 0 || channelEnd < 0 {
|
||||
panic("Illegal channel range value")
|
||||
}
|
||||
if channelBegin > channelEnd {
|
||||
panic("Illegal channel range value")
|
||||
}
|
||||
|
||||
channels := make([]string, channelEnd-channelBegin)
|
||||
for i := 0; i < channelEnd-channelBegin; i++ {
|
||||
channels[i] = ch + "-" + strconv.Itoa(channelBegin+i)
|
||||
}
|
||||
return channels
|
||||
}
|
||||
|
||||
func (p *ParamTable) msgChannelSubName() string {
|
||||
// TODO: subName = namePrefix + "-" + queryNodeID, queryNodeID is assigned by master
|
||||
name, err := p.Load("msgChannel.subNamePrefix.queryNodeSubNamePrefix")
|
||||
if err != nil {
|
||||
log.Panic(err)
|
||||
}
|
||||
return name
|
||||
}
|
||||
|
||||
func (p *ParamTable) statsChannelName() string {
|
||||
channels, err := p.Load("msgChannel.chanNamePrefix.queryNodeStats")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
return channels
|
||||
}
|
|
@ -1,4 +1,4 @@
|
|||
package querynode
|
||||
package reader
|
||||
|
||||
/*
|
||||
|
|
@ -1,4 +1,4 @@
|
|||
package querynode
|
||||
package reader
|
||||
|
||||
/*
|
||||
|
|
@ -1,4 +1,4 @@
|
|||
package querynode
|
||||
package reader
|
||||
|
||||
import (
|
||||
"context"
|
|
@ -1,4 +1,4 @@
|
|||
package querynode
|
||||
package reader
|
||||
|
||||
import (
|
||||
"context"
|
|
@ -1,4 +1,4 @@
|
|||
package querynode
|
||||
package reader
|
||||
|
||||
import (
|
||||
"context"
|
|
@ -1,4 +1,4 @@
|
|||
package querynode
|
||||
package reader
|
||||
|
||||
import (
|
||||
"context"
|
||||
|
@ -175,7 +175,7 @@ func TestDataSyncService_Start(t *testing.T) {
|
|||
|
||||
// pulsar produce
|
||||
const receiveBufSize = 1024
|
||||
producerChannels := Params.insertChannelNames()
|
||||
producerChannels := []string{"insert"}
|
||||
|
||||
insertStream := msgstream.NewPulsarMsgStream(ctx, receiveBufSize)
|
||||
insertStream.SetPulsarClient(pulsarURL)
|
|
@ -1,4 +1,4 @@
|
|||
package querynode
|
||||
package reader
|
||||
|
||||
type deleteNode struct {
|
||||
BaseNode
|
|
@ -1,4 +1,4 @@
|
|||
package querynode
|
||||
package reader
|
||||
|
||||
import (
|
||||
"log"
|
||||
|
@ -29,6 +29,8 @@ func (fdmNode *filterDmNode) Operate(in []*Msg) []*Msg {
|
|||
// TODO: add error handling
|
||||
}
|
||||
|
||||
// TODO: add time range check
|
||||
|
||||
var iMsg = insertMsg{
|
||||
insertMessages: make([]*msgstream.InsertMsg, 0),
|
||||
timeRange: TimeRange{
|
|
@ -1,4 +1,4 @@
|
|||
package querynode
|
||||
package reader
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
|
@ -106,7 +106,6 @@ func (iNode *insertNode) insert(insertData *InsertData, segmentID int64, wg *syn
|
|||
if err != nil {
|
||||
log.Println("cannot find segment:", segmentID)
|
||||
// TODO: add error handling
|
||||
wg.Done()
|
||||
return
|
||||
}
|
||||
|
||||
|
@ -117,9 +116,8 @@ func (iNode *insertNode) insert(insertData *InsertData, segmentID int64, wg *syn
|
|||
|
||||
err = targetSegment.segmentInsert(offsets, &ids, ×tamps, &records)
|
||||
if err != nil {
|
||||
log.Println(err)
|
||||
log.Println("insert failed")
|
||||
// TODO: add error handling
|
||||
wg.Done()
|
||||
return
|
||||
}
|
||||
|
|
@ -1,4 +1,4 @@
|
|||
package querynode
|
||||
package reader
|
||||
|
||||
type key2SegNode struct {
|
||||
BaseNode
|
|
@ -1,4 +1,4 @@
|
|||
package querynode
|
||||
package reader
|
||||
|
||||
import (
|
||||
"github.com/zilliztech/milvus-distributed/internal/msgstream"
|
|
@ -1,4 +1,4 @@
|
|||
package querynode
|
||||
package reader
|
||||
|
||||
import (
|
||||
"context"
|
||||
|
@ -9,16 +9,16 @@ import (
|
|||
)
|
||||
|
||||
func newDmInputNode(ctx context.Context) *flowgraph.InputNode {
|
||||
receiveBufSize := Params.insertReceiveBufSize()
|
||||
pulsarBufSize := Params.insertPulsarBufSize()
|
||||
receiveBufSize := Params.dmReceiveBufSize()
|
||||
pulsarBufSize := Params.dmPulsarBufSize()
|
||||
|
||||
msgStreamURL, err := Params.pulsarAddress()
|
||||
if err != nil {
|
||||
log.Fatal(err)
|
||||
}
|
||||
|
||||
consumeChannels := Params.insertChannelNames()
|
||||
consumeSubName := Params.msgChannelSubName()
|
||||
consumeChannels := []string{"insert"}
|
||||
consumeSubName := "insertSub"
|
||||
|
||||
insertStream := msgstream.NewPulsarTtMsgStream(ctx, receiveBufSize)
|
||||
insertStream.SetPulsarClient(msgStreamURL)
|
|
@ -1,4 +1,4 @@
|
|||
package querynode
|
||||
package reader
|
||||
|
||||
import "github.com/zilliztech/milvus-distributed/internal/util/flowgraph"
|
||||
|
|
@ -1,4 +1,4 @@
|
|||
package querynode
|
||||
package reader
|
||||
|
||||
type schemaUpdateNode struct {
|
||||
BaseNode
|
|
@ -1,4 +1,4 @@
|
|||
package querynode
|
||||
package reader
|
||||
|
||||
import (
|
||||
"log"
|
|
@ -1,4 +1,4 @@
|
|||
package querynode
|
||||
package reader
|
||||
|
||||
/*
|
||||
|
|
@ -1,4 +1,4 @@
|
|||
package querynode
|
||||
package reader
|
||||
|
||||
import (
|
||||
"context"
|
||||
|
@ -111,8 +111,8 @@ func isSegmentChannelRangeInQueryNodeChannelRange(segment *etcdpb.SegmentMeta) b
|
|||
}
|
||||
|
||||
Params.Init()
|
||||
var queryNodeChannelStart = Params.insertChannelRange()[0]
|
||||
var queryNodeChannelEnd = Params.insertChannelRange()[1]
|
||||
var queryNodeChannelStart = Params.topicStart()
|
||||
var queryNodeChannelEnd = Params.topicEnd()
|
||||
|
||||
if segment.ChannelStart >= int32(queryNodeChannelStart) && segment.ChannelEnd <= int32(queryNodeChannelEnd) {
|
||||
return true
|
||||
|
@ -167,7 +167,6 @@ func (mService *metaService) processSegmentCreate(id string, value string) {
|
|||
|
||||
seg := mService.segmentUnmarshal(value)
|
||||
if !isSegmentChannelRangeInQueryNodeChannelRange(seg) {
|
||||
log.Println("Illegal segment channel range")
|
||||
return
|
||||
}
|
||||
|
|
@ -1,4 +1,4 @@
|
|||
package querynode
|
||||
package reader
|
||||
|
||||
import (
|
||||
"context"
|
||||
|
@ -93,7 +93,7 @@ func TestMetaService_isSegmentChannelRangeInQueryNodeChannelRange(t *testing.T)
|
|||
CollectionID: UniqueID(0),
|
||||
PartitionTag: "partition0",
|
||||
ChannelStart: 0,
|
||||
ChannelEnd: 1,
|
||||
ChannelEnd: 128,
|
||||
OpenTime: Timestamp(0),
|
||||
CloseTime: Timestamp(math.MaxUint64),
|
||||
NumRows: UniqueID(0),
|
||||
|
@ -264,9 +264,10 @@ func TestMetaService_processSegmentCreate(t *testing.T) {
|
|||
PartitionTags: []string{"default"},
|
||||
}
|
||||
|
||||
colMetaBlob := proto.MarshalTextString(&collectionMeta)
|
||||
colMetaBlob, err := proto.Marshal(&collectionMeta)
|
||||
assert.NoError(t, err)
|
||||
|
||||
err := (*node.replica).addCollection(&collectionMeta, string(colMetaBlob))
|
||||
err = (*node.replica).addCollection(&collectionMeta, string(colMetaBlob))
|
||||
assert.NoError(t, err)
|
||||
|
||||
err = (*node.replica).addPartition(UniqueID(0), "default")
|
||||
|
@ -275,7 +276,7 @@ func TestMetaService_processSegmentCreate(t *testing.T) {
|
|||
id := "0"
|
||||
value := `partition_tag: "default"
|
||||
channel_start: 0
|
||||
channel_end: 1
|
||||
channel_end: 128
|
||||
close_time: 18446744073709551615
|
||||
`
|
||||
|
||||
|
@ -330,7 +331,7 @@ func TestMetaService_processCreate(t *testing.T) {
|
|||
key2 := "by-dev/segment/0"
|
||||
msg2 := `partition_tag: "default"
|
||||
channel_start: 0
|
||||
channel_end: 1
|
||||
channel_end: 128
|
||||
close_time: 18446744073709551615
|
||||
`
|
||||
|
||||
|
@ -387,9 +388,10 @@ func TestMetaService_processSegmentModify(t *testing.T) {
|
|||
PartitionTags: []string{"default"},
|
||||
}
|
||||
|
||||
colMetaBlob := proto.MarshalTextString(&collectionMeta)
|
||||
colMetaBlob, err := proto.Marshal(&collectionMeta)
|
||||
assert.NoError(t, err)
|
||||
|
||||
err := (*node.replica).addCollection(&collectionMeta, string(colMetaBlob))
|
||||
err = (*node.replica).addCollection(&collectionMeta, string(colMetaBlob))
|
||||
assert.NoError(t, err)
|
||||
|
||||
err = (*node.replica).addPartition(UniqueID(0), "default")
|
||||
|
@ -398,7 +400,7 @@ func TestMetaService_processSegmentModify(t *testing.T) {
|
|||
id := "0"
|
||||
value := `partition_tag: "default"
|
||||
channel_start: 0
|
||||
channel_end: 1
|
||||
channel_end: 128
|
||||
close_time: 18446744073709551615
|
||||
`
|
||||
|
||||
|
@ -409,7 +411,7 @@ func TestMetaService_processSegmentModify(t *testing.T) {
|
|||
|
||||
newValue := `partition_tag: "default"
|
||||
channel_start: 0
|
||||
channel_end: 1
|
||||
channel_end: 128
|
||||
close_time: 18446744073709551615
|
||||
`
|
||||
|
||||
|
@ -579,7 +581,7 @@ func TestMetaService_processModify(t *testing.T) {
|
|||
key2 := "by-dev/segment/0"
|
||||
msg2 := `partition_tag: "p1"
|
||||
channel_start: 0
|
||||
channel_end: 1
|
||||
channel_end: 128
|
||||
close_time: 18446744073709551615
|
||||
`
|
||||
|
||||
|
@ -635,7 +637,7 @@ func TestMetaService_processModify(t *testing.T) {
|
|||
|
||||
msg4 := `partition_tag: "p1"
|
||||
channel_start: 0
|
||||
channel_end: 1
|
||||
channel_end: 128
|
||||
close_time: 18446744073709551615
|
||||
`
|
||||
|
||||
|
@ -692,9 +694,10 @@ func TestMetaService_processSegmentDelete(t *testing.T) {
|
|||
PartitionTags: []string{"default"},
|
||||
}
|
||||
|
||||
colMetaBlob := proto.MarshalTextString(&collectionMeta)
|
||||
colMetaBlob, err := proto.Marshal(&collectionMeta)
|
||||
assert.NoError(t, err)
|
||||
|
||||
err := (*node.replica).addCollection(&collectionMeta, string(colMetaBlob))
|
||||
err = (*node.replica).addCollection(&collectionMeta, string(colMetaBlob))
|
||||
assert.NoError(t, err)
|
||||
|
||||
err = (*node.replica).addPartition(UniqueID(0), "default")
|
||||
|
@ -703,7 +706,7 @@ func TestMetaService_processSegmentDelete(t *testing.T) {
|
|||
id := "0"
|
||||
value := `partition_tag: "default"
|
||||
channel_start: 0
|
||||
channel_end: 1
|
||||
channel_end: 128
|
||||
close_time: 18446744073709551615
|
||||
`
|
||||
|
||||
|
@ -807,7 +810,7 @@ func TestMetaService_processDelete(t *testing.T) {
|
|||
key2 := "by-dev/segment/0"
|
||||
msg2 := `partition_tag: "default"
|
||||
channel_start: 0
|
||||
channel_end: 1
|
||||
channel_end: 128
|
||||
close_time: 18446744073709551615
|
||||
`
|
||||
|
|
@ -0,0 +1,195 @@
|
|||
package reader
|
||||
|
||||
import (
|
||||
"strconv"
|
||||
|
||||
"github.com/zilliztech/milvus-distributed/internal/util/paramtable"
|
||||
)
|
||||
|
||||
type ParamTable struct {
|
||||
paramtable.BaseTable
|
||||
}
|
||||
|
||||
var Params ParamTable
|
||||
|
||||
func (p *ParamTable) Init() {
|
||||
p.BaseTable.Init()
|
||||
err := p.LoadYaml("advanced/query_node.yaml")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
}
|
||||
|
||||
func (p *ParamTable) pulsarAddress() (string, error) {
|
||||
url, err := p.Load("_PulsarAddress")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
return url, nil
|
||||
}
|
||||
|
||||
func (p *ParamTable) queryNodeID() int {
|
||||
queryNodeID, err := p.Load("reader.clientid")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
id, err := strconv.Atoi(queryNodeID)
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
return id
|
||||
}
|
||||
|
||||
// TODO: func (p *ParamTable) DmChannelRange() []int {
|
||||
func (p *ParamTable) topicStart() int {
|
||||
topicStart, err := p.Load("reader.topicstart")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
topicStartNum, err := strconv.Atoi(topicStart)
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
return topicStartNum
|
||||
}
|
||||
|
||||
func (p *ParamTable) topicEnd() int {
|
||||
topicEnd, err := p.Load("reader.topicend")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
topicEndNum, err := strconv.Atoi(topicEnd)
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
return topicEndNum
|
||||
}
|
||||
|
||||
// advanced params
|
||||
// stats
|
||||
func (p *ParamTable) statsPublishInterval() int {
|
||||
timeInterval, err := p.Load("reader.stats.publishInterval")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
interval, err := strconv.Atoi(timeInterval)
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
return interval
|
||||
}
|
||||
|
||||
// dataSync:
|
||||
func (p *ParamTable) flowGraphMaxQueueLength() int32 {
|
||||
queueLength, err := p.Load("reader.dataSync.flowGraph.maxQueueLength")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
length, err := strconv.Atoi(queueLength)
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
return int32(length)
|
||||
}
|
||||
|
||||
func (p *ParamTable) flowGraphMaxParallelism() int32 {
|
||||
maxParallelism, err := p.Load("reader.dataSync.flowGraph.maxParallelism")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
maxPara, err := strconv.Atoi(maxParallelism)
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
return int32(maxPara)
|
||||
}
|
||||
|
||||
// msgStream
|
||||
// TODO: func (p *ParamTable) insertStreamBufSize() int64
|
||||
func (p *ParamTable) dmReceiveBufSize() int64 {
|
||||
revBufSize, err := p.Load("reader.msgStream.dm.recvBufSize")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
bufSize, err := strconv.Atoi(revBufSize)
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
return int64(bufSize)
|
||||
}
|
||||
|
||||
func (p *ParamTable) dmPulsarBufSize() int64 {
|
||||
pulsarBufSize, err := p.Load("reader.msgStream.dm.pulsarBufSize")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
bufSize, err := strconv.Atoi(pulsarBufSize)
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
return int64(bufSize)
|
||||
}
|
||||
|
||||
func (p *ParamTable) searchReceiveBufSize() int64 {
|
||||
revBufSize, err := p.Load("reader.msgStream.search.recvBufSize")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
bufSize, err := strconv.Atoi(revBufSize)
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
return int64(bufSize)
|
||||
}
|
||||
|
||||
func (p *ParamTable) searchPulsarBufSize() int64 {
|
||||
pulsarBufSize, err := p.Load("reader.msgStream.search.pulsarBufSize")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
bufSize, err := strconv.Atoi(pulsarBufSize)
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
return int64(bufSize)
|
||||
}
|
||||
|
||||
func (p *ParamTable) searchResultReceiveBufSize() int64 {
|
||||
revBufSize, err := p.Load("reader.msgStream.searchResult.recvBufSize")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
bufSize, err := strconv.Atoi(revBufSize)
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
return int64(bufSize)
|
||||
}
|
||||
|
||||
func (p *ParamTable) statsReceiveBufSize() int64 {
|
||||
revBufSize, err := p.Load("reader.msgStream.stats.recvBufSize")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
bufSize, err := strconv.Atoi(revBufSize)
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
return int64(bufSize)
|
||||
}
|
||||
|
||||
func (p *ParamTable) etcdAddress() string {
|
||||
etcdAddress, err := p.Load("_EtcdAddress")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
return etcdAddress
|
||||
}
|
||||
|
||||
func (p *ParamTable) etcdRootPath() string {
|
||||
etcdRootPath, err := p.Load("etcd.rootpath")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
return etcdRootPath
|
||||
}
|
|
@ -1,4 +1,4 @@
|
|||
package querynode
|
||||
package reader
|
||||
|
||||
import (
|
||||
"strings"
|
||||
|
@ -26,12 +26,16 @@ func TestParamTable_QueryNodeID(t *testing.T) {
|
|||
assert.Equal(t, id, 0)
|
||||
}
|
||||
|
||||
func TestParamTable_insertChannelRange(t *testing.T) {
|
||||
func TestParamTable_TopicStart(t *testing.T) {
|
||||
Params.Init()
|
||||
channelRange := Params.insertChannelRange()
|
||||
assert.Equal(t, len(channelRange), 2)
|
||||
assert.Equal(t, channelRange[0], 0)
|
||||
assert.Equal(t, channelRange[1], 1)
|
||||
topicStart := Params.topicStart()
|
||||
assert.Equal(t, topicStart, 0)
|
||||
}
|
||||
|
||||
func TestParamTable_TopicEnd(t *testing.T) {
|
||||
Params.Init()
|
||||
topicEnd := Params.topicEnd()
|
||||
assert.Equal(t, topicEnd, 128)
|
||||
}
|
||||
|
||||
func TestParamTable_statsServiceTimeInterval(t *testing.T) {
|
||||
|
@ -46,9 +50,9 @@ func TestParamTable_statsMsgStreamReceiveBufSize(t *testing.T) {
|
|||
assert.Equal(t, bufSize, int64(64))
|
||||
}
|
||||
|
||||
func TestParamTable_insertMsgStreamReceiveBufSize(t *testing.T) {
|
||||
func TestParamTable_dmMsgStreamReceiveBufSize(t *testing.T) {
|
||||
Params.Init()
|
||||
bufSize := Params.insertReceiveBufSize()
|
||||
bufSize := Params.dmReceiveBufSize()
|
||||
assert.Equal(t, bufSize, int64(1024))
|
||||
}
|
||||
|
||||
|
@ -70,9 +74,9 @@ func TestParamTable_searchPulsarBufSize(t *testing.T) {
|
|||
assert.Equal(t, bufSize, int64(512))
|
||||
}
|
||||
|
||||
func TestParamTable_insertPulsarBufSize(t *testing.T) {
|
||||
func TestParamTable_dmPulsarBufSize(t *testing.T) {
|
||||
Params.Init()
|
||||
bufSize := Params.insertPulsarBufSize()
|
||||
bufSize := Params.dmPulsarBufSize()
|
||||
assert.Equal(t, bufSize, int64(1024))
|
||||
}
|
||||
|
||||
|
@ -87,36 +91,3 @@ func TestParamTable_flowGraphMaxParallelism(t *testing.T) {
|
|||
maxParallelism := Params.flowGraphMaxParallelism()
|
||||
assert.Equal(t, maxParallelism, int32(1024))
|
||||
}
|
||||
|
||||
func TestParamTable_insertChannelNames(t *testing.T) {
|
||||
Params.Init()
|
||||
names := Params.insertChannelNames()
|
||||
assert.Equal(t, len(names), 1)
|
||||
assert.Equal(t, names[0], "insert-0")
|
||||
}
|
||||
|
||||
func TestParamTable_searchChannelNames(t *testing.T) {
|
||||
Params.Init()
|
||||
names := Params.searchChannelNames()
|
||||
assert.Equal(t, len(names), 1)
|
||||
assert.Equal(t, names[0], "search-0")
|
||||
}
|
||||
|
||||
func TestParamTable_searchResultChannelNames(t *testing.T) {
|
||||
Params.Init()
|
||||
names := Params.searchResultChannelNames()
|
||||
assert.Equal(t, len(names), 1)
|
||||
assert.Equal(t, names[0], "searchResult-0")
|
||||
}
|
||||
|
||||
func TestParamTable_msgChannelSubName(t *testing.T) {
|
||||
Params.Init()
|
||||
name := Params.msgChannelSubName()
|
||||
assert.Equal(t, name, "queryNode")
|
||||
}
|
||||
|
||||
func TestParamTable_statsChannelName(t *testing.T) {
|
||||
Params.Init()
|
||||
name := Params.statsChannelName()
|
||||
assert.Equal(t, name, "query-node-stats")
|
||||
}
|
|
@ -1,4 +1,4 @@
|
|||
package querynode
|
||||
package reader
|
||||
|
||||
/*
|
||||
|
|
@ -1,4 +1,4 @@
|
|||
package querynode
|
||||
package reader
|
||||
|
||||
import (
|
||||
"context"
|
|
@ -1,4 +1,4 @@
|
|||
package querynode
|
||||
package reader
|
||||
|
||||
/*
|
||||
#cgo CFLAGS: -I${SRCDIR}/../core/output/include
|
|
@ -1,4 +1,4 @@
|
|||
package querynode
|
||||
package reader
|
||||
|
||||
/*
|
||||
|
|
@ -1,4 +1,4 @@
|
|||
package querynode
|
||||
package reader
|
||||
|
||||
import (
|
||||
"context"
|
||||
|
@ -6,7 +6,7 @@ import (
|
|||
"time"
|
||||
)
|
||||
|
||||
const ctxTimeInMillisecond = 200
|
||||
const ctxTimeInMillisecond = 2000
|
||||
const closeWithDeadline = true
|
||||
|
||||
// NOTE: start pulsar and etcd before test
|
|
@ -1,4 +1,4 @@
|
|||
package querynode
|
||||
package reader
|
||||
|
||||
import (
|
||||
"context"
|
|
@ -1,4 +1,4 @@
|
|||
package querynode
|
||||
package reader
|
||||
|
||||
import "C"
|
||||
import (
|
||||
|
@ -42,15 +42,15 @@ func newSearchService(ctx context.Context, replica *collectionReplica) *searchSe
|
|||
log.Fatal(err)
|
||||
}
|
||||
|
||||
consumeChannels := Params.searchChannelNames()
|
||||
consumeSubName := Params.msgChannelSubName()
|
||||
consumeChannels := []string{"search"}
|
||||
consumeSubName := "subSearch"
|
||||
searchStream := msgstream.NewPulsarMsgStream(ctx, receiveBufSize)
|
||||
searchStream.SetPulsarClient(msgStreamURL)
|
||||
unmarshalDispatcher := msgstream.NewUnmarshalDispatcher()
|
||||
searchStream.CreatePulsarConsumers(consumeChannels, consumeSubName, unmarshalDispatcher, pulsarBufSize)
|
||||
var inputStream msgstream.MsgStream = searchStream
|
||||
|
||||
producerChannels := Params.searchResultChannelNames()
|
||||
producerChannels := []string{"searchResult"}
|
||||
searchResultStream := msgstream.NewPulsarMsgStream(ctx, receiveBufSize)
|
||||
searchResultStream.SetPulsarClient(msgStreamURL)
|
||||
searchResultStream.CreatePulsarProducers(producerChannels)
|
|
@ -1,4 +1,4 @@
|
|||
package querynode
|
||||
package reader
|
||||
|
||||
import (
|
||||
"context"
|
|
@ -1,4 +1,4 @@
|
|||
package querynode
|
||||
package reader
|
||||
|
||||
/*
|
||||
|
||||
|
@ -109,7 +109,7 @@ func (s *Segment) segmentPreDelete(numOfRecords int) int64 {
|
|||
//-------------------------------------------------------------------------------------- dm & search functions
|
||||
func (s *Segment) segmentInsert(offset int64, entityIDs *[]UniqueID, timestamps *[]Timestamp, records *[]*commonpb.Blob) error {
|
||||
/*
|
||||
CStatus
|
||||
int
|
||||
Insert(CSegmentBase c_segment,
|
||||
long int reserved_offset,
|
||||
signed long int size,
|
||||
|
@ -148,12 +148,8 @@ func (s *Segment) segmentInsert(offset int64, entityIDs *[]UniqueID, timestamps
|
|||
cSizeofPerRow,
|
||||
cNumOfRows)
|
||||
|
||||
errorCode := status.error_code
|
||||
|
||||
if errorCode != 0 {
|
||||
errorMsg := C.GoString(status.error_msg)
|
||||
defer C.free(unsafe.Pointer(status.error_msg))
|
||||
return errors.New("Insert failed, C runtime error detected, error code = " + strconv.Itoa(int(errorCode)) + ", error msg = " + errorMsg)
|
||||
if status != 0 {
|
||||
return errors.New("Insert failed, error code = " + strconv.Itoa(int(status)))
|
||||
}
|
||||
|
||||
s.recentlyModified = true
|
||||
|
@ -162,7 +158,7 @@ func (s *Segment) segmentInsert(offset int64, entityIDs *[]UniqueID, timestamps
|
|||
|
||||
func (s *Segment) segmentDelete(offset int64, entityIDs *[]UniqueID, timestamps *[]Timestamp) error {
|
||||
/*
|
||||
CStatus
|
||||
int
|
||||
Delete(CSegmentBase c_segment,
|
||||
long int reserved_offset,
|
||||
long size,
|
||||
|
@ -176,12 +172,8 @@ func (s *Segment) segmentDelete(offset int64, entityIDs *[]UniqueID, timestamps
|
|||
|
||||
var status = C.Delete(s.segmentPtr, cOffset, cSize, cEntityIdsPtr, cTimestampsPtr)
|
||||
|
||||
errorCode := status.error_code
|
||||
|
||||
if errorCode != 0 {
|
||||
errorMsg := C.GoString(status.error_msg)
|
||||
defer C.free(unsafe.Pointer(status.error_msg))
|
||||
return errors.New("Delete failed, C runtime error detected, error code = " + strconv.Itoa(int(errorCode)) + ", error msg = " + errorMsg)
|
||||
if status != 0 {
|
||||
return errors.New("Delete failed, error code = " + strconv.Itoa(int(status)))
|
||||
}
|
||||
|
||||
return nil
|
||||
|
@ -195,8 +187,7 @@ func (s *Segment) segmentSearch(plan *Plan,
|
|||
numQueries int64,
|
||||
topK int64) error {
|
||||
/*
|
||||
CStatus
|
||||
Search(void* plan,
|
||||
void* Search(void* plan,
|
||||
void* placeholder_groups,
|
||||
uint64_t* timestamps,
|
||||
int num_groups,
|
||||
|
@ -220,20 +211,16 @@ func (s *Segment) segmentSearch(plan *Plan,
|
|||
var cNumGroups = C.int(len(placeHolderGroups))
|
||||
|
||||
var status = C.Search(s.segmentPtr, plan.cPlan, cPlaceHolder, cTimestamp, cNumGroups, cNewResultIds, cNewResultDistances)
|
||||
errorCode := status.error_code
|
||||
|
||||
if errorCode != 0 {
|
||||
errorMsg := C.GoString(status.error_msg)
|
||||
defer C.free(unsafe.Pointer(status.error_msg))
|
||||
return errors.New("Search failed, C runtime error detected, error code = " + strconv.Itoa(int(errorCode)) + ", error msg = " + errorMsg)
|
||||
if status != 0 {
|
||||
return errors.New("search failed, error code = " + strconv.Itoa(int(status)))
|
||||
}
|
||||
|
||||
cNumQueries := C.long(numQueries)
|
||||
cTopK := C.long(topK)
|
||||
// reduce search result
|
||||
mergeStatus := C.MergeInto(cNumQueries, cTopK, cResultDistances, cResultIds, cNewResultDistances, cNewResultIds)
|
||||
if mergeStatus != 0 {
|
||||
return errors.New("merge search result failed, error code = " + strconv.Itoa(int(mergeStatus)))
|
||||
status = C.MergeInto(cNumQueries, cTopK, cResultDistances, cResultIds, cNewResultDistances, cNewResultIds)
|
||||
if status != 0 {
|
||||
return errors.New("merge search result failed, error code = " + strconv.Itoa(int(status)))
|
||||
}
|
||||
return nil
|
||||
}
|
|
@ -1,4 +1,4 @@
|
|||
package querynode
|
||||
package reader
|
||||
|
||||
import (
|
||||
"context"
|
||||
|
@ -463,6 +463,7 @@ func TestSegment_segmentInsert(t *testing.T) {
|
|||
|
||||
err := segment.segmentInsert(offset, &ids, ×tamps, &records)
|
||||
assert.NoError(t, err)
|
||||
|
||||
deleteSegment(segment)
|
||||
deleteCollection(collection)
|
||||
}
|
||||
|
@ -639,7 +640,7 @@ func TestSegment_segmentSearch(t *testing.T) {
|
|||
|
||||
pulsarURL, _ := Params.pulsarAddress()
|
||||
const receiveBufSize = 1024
|
||||
searchProducerChannels := Params.searchChannelNames()
|
||||
searchProducerChannels := []string{"search"}
|
||||
searchStream := msgstream.NewPulsarMsgStream(ctx, receiveBufSize)
|
||||
searchStream.SetPulsarClient(pulsarURL)
|
||||
searchStream.CreatePulsarProducers(searchProducerChannels)
|
|
@ -1,4 +1,4 @@
|
|||
package querynode
|
||||
package reader
|
||||
|
||||
import (
|
||||
"context"
|
||||
|
@ -36,7 +36,7 @@ func (sService *statsService) start() {
|
|||
if err != nil {
|
||||
log.Fatal(err)
|
||||
}
|
||||
producerChannels := []string{Params.statsChannelName()}
|
||||
producerChannels := []string{"statistic"}
|
||||
|
||||
statsStream := msgstream.NewPulsarMsgStream(sService.ctx, receiveBufSize)
|
||||
statsStream.SetPulsarClient(msgStreamURL)
|
|
@ -1,4 +1,4 @@
|
|||
package querynode
|
||||
package reader
|
||||
|
||||
import (
|
||||
"context"
|
||||
|
@ -171,7 +171,7 @@ func TestSegmentManagement_SegmentStatisticService(t *testing.T) {
|
|||
|
||||
const receiveBufSize = 1024
|
||||
// start pulsar
|
||||
producerChannels := []string{Params.statsChannelName()}
|
||||
producerChannels := []string{"statistic"}
|
||||
|
||||
statsStream := msgstream.NewPulsarMsgStream(ctx, receiveBufSize)
|
||||
statsStream.SetPulsarClient(pulsarURL)
|
|
@ -1,4 +1,4 @@
|
|||
package querynode
|
||||
package reader
|
||||
|
||||
import (
|
||||
"sync"
|
|
@ -1,4 +1,4 @@
|
|||
package querynode
|
||||
package reader
|
||||
|
||||
import (
|
||||
"testing"
|
|
@ -1,4 +1,4 @@
|
|||
package querynode
|
||||
package reader
|
||||
|
||||
import "github.com/zilliztech/milvus-distributed/internal/util/typeutil"
|
||||
|
|
@ -13,5 +13,5 @@ SCRIPTS_DIR="$( cd -P "$( dirname "$SOURCE" )" && pwd )"
|
|||
# ignore Minio,S3 unittes
|
||||
MILVUS_DIR="${SCRIPTS_DIR}/../internal/"
|
||||
echo $MILVUS_DIR
|
||||
#go test -cover "${MILVUS_DIR}/kv/..." "${MILVUS_DIR}/msgstream/..." "${MILVUS_DIR}/master/..." "${MILVUS_DIR}/querynode/..." "${MILVUS_DIR}/proxy/..." -failfast
|
||||
go test -cover "${MILVUS_DIR}/kv/..." "${MILVUS_DIR}/msgstream/..." "${MILVUS_DIR}/master/..." "${MILVUS_DIR}/querynode/..." -failfast
|
||||
#go test -cover "${MILVUS_DIR}/kv/..." "${MILVUS_DIR}/msgstream/..." "${MILVUS_DIR}/master/..." "${MILVUS_DIR}/reader/..." "${MILVUS_DIR}/proxy/..." -failfast
|
||||
go test -cover "${MILVUS_DIR}/kv/..." "${MILVUS_DIR}/msgstream/..." "${MILVUS_DIR}/master/..." "${MILVUS_DIR}/reader/..." -failfast
|
||||
|
|
Loading…
Reference in New Issue