Fix segment fault

Signed-off-by: zhenshan.cao <zhenshan.cao@zilliz.com>
pull/4973/head^2
zhenshan.cao 2020-11-26 14:29:01 +08:00 committed by yefu.chen
parent 0a16a9a651
commit 045fc3d808
97 changed files with 3147 additions and 3797 deletions

View File

@ -56,8 +56,8 @@ verifiers: cppcheck fmt lint ruleguard
# Builds various components locally.
build-go:
@echo "Building each component's binary to './'"
@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 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 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 reader writer's unittest
#TODO: proxy master query node 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 reader docker image '$(TAG)'"
@echo "Building query node 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/reader $(GOPATH)/bin/reader
@mkdir -p $(GOPATH)/bin && cp -f $(PWD)/bin/querynode $(GOPATH)/bin/querynode
@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 reader
@rm -rvf querynode
@rm -rvf master
@rm -rvf proxy

View File

@ -6,14 +6,14 @@ import (
"os/signal"
"syscall"
"github.com/zilliztech/milvus-distributed/internal/reader"
"github.com/zilliztech/milvus-distributed/internal/querynode"
)
func main() {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
reader.Init()
querynode.Init()
sc := make(chan os.Signal, 1)
signal.Notify(sc,
@ -28,7 +28,7 @@ func main() {
cancel()
}()
reader.StartQueryNode(ctx)
querynode.StartQueryNode(ctx)
switch sig {
case syscall.SIGTERM:

View File

@ -30,10 +30,10 @@ msgChannel:
queryNodeSubNamePrefix: "queryNode"
writeNodeSubNamePrefix: "writeNode"
# default channel range [0, 0]
# default channel range [0, 1)
channelRange:
insert: [0, 15]
delete: [0, 15]
k2s: [0, 15]
search: [0, 0]
insert: [0, 1]
delete: [0, 1]
k2s: [0, 1]
search: [0, 1]
searchResult: [0, 1]

View File

@ -0,0 +1,13 @@
# 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

View File

@ -20,4 +20,6 @@ master:
minIDAssignCnt: 1024
maxIDAssignCnt: 16384
# old name: segmentExpireDuration: 2000
IDAssignExpiration: 2000 # ms
IDAssignExpiration: 2000 # ms
maxPartitionNum: 4096

View File

@ -25,4 +25,7 @@ proxy:
pulsarBufSize: 1024 # pulsar chan buffer size
timeTick:
bufSize: 512
bufSize: 512
maxNameLength: 255
maxFieldNum: 64

View File

@ -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.
reader:
queryNode:
stats:
publishInterval: 1000 # milliseconds
@ -19,10 +19,6 @@ reader:
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

View File

@ -11,9 +11,9 @@
nodeID: # will be deprecated after v0.2
proxyIDList: [1, 2]
queryNodeIDList: [3, 4]
writeNodeIDList: [5, 6]
proxyIDList: [1]
queryNodeIDList: [2]
writeNodeIDList: [3]
etcd:
address: localhost

View File

@ -380,18 +380,13 @@ func (segMgr *SegmentManager) AssignSegmentID(segIDReq []*internalpb.SegIDReques
// "/msg_stream/insert"
message SysConfigRequest {
MsgType msg_type = 1;
int64 reqID = 2;
int64 proxyID = 3;
uint64 timestamp = 4;
repeated string keys = 5;
repeated string key_prefixes = 6;
repeated string keys = 1;
repeated string key_prefixes = 2;
}
message SysConfigResponse {
common.Status status = 1;
repeated string keys = 2;
repeated string values = 3;
repeated string keys = 1;
repeated string values = 2;
}
```
@ -399,11 +394,12 @@ message SysConfigResponse {
```go
type SysConfig struct {
kv *kv.EtcdKV
etcdKV *etcd
etcdPathPrefix string
}
func (conf *SysConfig) InitFromFile(filePath string) (error)
func (conf *SysConfig) GetByPrefix(keyPrefix string) (keys []string, values []string, err error)
func (conf *SysConfig) GetByPrefix(keyPrefix string) ([]string, error)
func (conf *SysConfig) Get(keys []string) ([]string, error)
```

1
go.mod
View File

@ -7,6 +7,7 @@ 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
View File

@ -65,6 +65,7 @@ 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=
@ -329,6 +330,7 @@ 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=

View File

@ -57,7 +57,7 @@ type segRequest struct {
count uint32
colName string
partition string
segID UniqueID
segInfo map[UniqueID]uint32
channelID int32
}

View File

@ -1,11 +1,15 @@
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"
@ -18,7 +22,10 @@ const (
)
type assignInfo struct {
internalpb.SegIDAssignment
collName string
partitionTag string
channelID int32
segInfo map[UniqueID]uint32 // segmentID->count map
expireTime time.Time
lastInsertTime time.Time
}
@ -32,12 +39,16 @@ func (info *assignInfo) IsActive(now time.Time) bool {
}
func (info *assignInfo) IsEnough(count uint32) bool {
return info.Count >= count
total := uint32(0)
for _, count := range info.segInfo {
total += count
}
return total >= count
}
type SegIDAssigner struct {
Allocator
assignInfos map[string][]*assignInfo // collectionName -> [] *assignInfo
assignInfos map[string]*list.List // collectionName -> *list.List
segReqs []*internalpb.SegIDRequest
canDoReqs []request
}
@ -50,11 +61,8 @@ func NewSegIDAssigner(ctx context.Context, masterAddr string) (*SegIDAssigner, e
cancel: cancel,
masterAddress: masterAddr,
countPerRPC: SegCountPerRPC,
//toDoReqs: []request,
},
assignInfos: make(map[string][]*assignInfo),
//segReqs: make([]*internalpb.SegIDRequest, maxConcurrentRequests),
//canDoReqs: make([]request, maxConcurrentRequests),
assignInfos: make(map[string]*list.List),
}
sa.tChan = &ticker{
updateInterval: time.Second,
@ -67,16 +75,17 @@ func NewSegIDAssigner(ctx context.Context, masterAddr string) (*SegIDAssigner, e
func (sa *SegIDAssigner) collectExpired() {
now := time.Now()
for _, colInfos := range sa.assignInfos {
for _, assign := range colInfos {
for _, info := range sa.assignInfos {
for e := info.Front(); e != nil; e = e.Next() {
assign := e.Value.(*assignInfo)
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,
})
}
}
@ -88,7 +97,6 @@ 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) {
@ -102,13 +110,36 @@ 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 {
colInfos, ok := sa.assignInfos[colName]
assignInfos, ok := sa.assignInfos[colName]
if !ok {
return nil
}
for _, info := range colInfos {
if info.PartitionTag != partition || info.ChannelID != channelID {
for e := assignInfos.Front(); e != nil; e = e.Next() {
info := e.Value.(*assignInfo)
if info.partitionTag != partition || info.channelID != channelID {
continue
}
return info
@ -151,19 +182,29 @@ 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]
newAssign := &assignInfo{
SegIDAssignment: *info,
expireTime: expiredTime,
lastInsertTime: now,
colInfos, ok := sa.assignInfos[info.CollName]
if !ok {
colInfos = list.New()
}
colInfos = append(colInfos, newAssign)
segInfo := make(map[UniqueID]uint32)
segInfo[info.SegID] = info.Count
newAssign := &assignInfo{
collName: info.CollName,
partitionTag: info.PartitionTag,
channelID: info.ChannelID,
segInfo: segInfo,
}
colInfos.PushBack(newAssign)
sa.assignInfos[info.CollName] = colInfos
} else {
assign.SegIDAssignment = *info
assign.segInfo[info.SegID] = info.Count
assign.expireTime = expiredTime
assign.lastInsertTime = now
}
@ -181,13 +222,38 @@ func (sa *SegIDAssigner) processFunc(req request) error {
if assign == nil {
return errors.New("Failed to GetSegmentID")
}
segRequest.segID = assign.SegID
assign.Count -= segRequest.count
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
fmt.Println("process segmentID")
return nil
}
func (sa *SegIDAssigner) GetSegmentID(colName, partition string, channelID int32, count uint32) (UniqueID, error) {
func (sa *SegIDAssigner) GetSegmentID(colName, partition string, channelID int32, count uint32) (map[UniqueID]uint32, error) {
req := &segRequest{
baseRequest: baseRequest{done: make(chan error), valid: false},
colName: colName,
@ -199,7 +265,7 @@ func (sa *SegIDAssigner) GetSegmentID(colName, partition string, channelID int32
req.Wait()
if !req.IsValid() {
return 0, errors.New("GetSegmentID Failed")
return nil, errors.New("GetSegmentID Failed")
}
return req.segID, nil
return req.segInfo, nil
}

View File

@ -13,7 +13,7 @@ import (
type Timestamp = typeutil.Timestamp
const (
tsCountPerRPC = 2 << 18 * 10
tsCountPerRPC = 2 << 15
)
type TimestampAllocator struct {
@ -37,6 +37,7 @@ 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
}

View File

@ -65,7 +65,8 @@ StructuredIndexFlat<T>::NotIn(const size_t n, const T* values) {
if (!is_built_) {
build();
}
TargetBitmapPtr bitset = std::make_unique<TargetBitmap>(data_.size(), true);
TargetBitmapPtr bitset = std::make_unique<TargetBitmap>(data_.size());
bitset->set();
for (size_t i = 0; i < n; ++i) {
for (const auto& index : data_) {
if (index->a_ == *(values + i)) {

View File

@ -120,7 +120,8 @@ StructuredIndexSort<T>::NotIn(const size_t n, const T* values) {
if (!is_built_) {
build();
}
TargetBitmapPtr bitset = std::make_unique<TargetBitmap>(data_.size(), true);
TargetBitmapPtr bitset = std::make_unique<TargetBitmap>(data_.size());
bitset->set();
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)));

File diff suppressed because it is too large Load Diff

View File

@ -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[17]
static const ::PROTOBUF_NAMESPACE_ID::internal::ParseTable schema[15]
PROTOBUF_SECTION_VARIABLE(protodesc_cold);
static const ::PROTOBUF_NAMESPACE_ID::internal::FieldMetadata field_metadata[];
static const ::PROTOBUF_NAMESPACE_ID::internal::SerializationTable serialization_table[];
@ -99,18 +99,12 @@ extern QueryResultDefaultTypeInternal _QueryResult_default_instance_;
class RowBatch;
class RowBatchDefaultTypeInternal;
extern RowBatchDefaultTypeInternal _RowBatch_default_instance_;
class Score;
class ScoreDefaultTypeInternal;
extern ScoreDefaultTypeInternal _Score_default_instance_;
class StringListResponse;
class StringListResponseDefaultTypeInternal;
extern StringListResponseDefaultTypeInternal _StringListResponse_default_instance_;
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,10 +122,8 @@ template<> ::milvus::proto::service::PlaceholderValue* Arena::CreateMaybeMessage
template<> ::milvus::proto::service::Query* Arena::CreateMaybeMessage<::milvus::proto::service::Query>(Arena*);
template<> ::milvus::proto::service::QueryResult* Arena::CreateMaybeMessage<::milvus::proto::service::QueryResult>(Arena*);
template<> ::milvus::proto::service::RowBatch* Arena::CreateMaybeMessage<::milvus::proto::service::RowBatch>(Arena*);
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 {
@ -2162,329 +2154,6 @@ 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:
Score();
virtual ~Score();
Score(const Score& from);
Score(Score&& from) noexcept
: Score() {
*this = ::std::move(from);
}
inline Score& operator=(const Score& from) {
CopyFrom(from);
return *this;
}
inline Score& operator=(Score&& 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 Score& default_instance();
static void InitAsDefaultInstance(); // FOR INTERNAL USE ONLY
static inline const Score* internal_default_instance() {
return reinterpret_cast<const Score*>(
&_Score_default_instance_);
}
static constexpr int kIndexInFileMessages =
14;
friend void swap(Score& a, Score& b) {
a.Swap(&b);
}
inline void Swap(Score* other) {
if (other == this) return;
InternalSwap(other);
}
// implements Message ----------------------------------------------
inline Score* New() const final {
return CreateMaybeMessage<Score>(nullptr);
}
Score* New(::PROTOBUF_NAMESPACE_ID::Arena* arena) const final {
return CreateMaybeMessage<Score>(arena);
}
void CopyFrom(const ::PROTOBUF_NAMESPACE_ID::Message& from) final;
void MergeFrom(const ::PROTOBUF_NAMESPACE_ID::Message& from) final;
void CopyFrom(const Score& from);
void MergeFrom(const Score& 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(Score* other);
friend class ::PROTOBUF_NAMESPACE_ID::internal::AnyMetadata;
static ::PROTOBUF_NAMESPACE_ID::StringPiece FullMessageName() {
return "milvus.proto.service.Score";
}
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 {
kValuesFieldNumber = 2,
kTagFieldNumber = 1,
};
// repeated float values = 2;
int values_size() const;
void clear_values();
float values(int index) const;
void set_values(int index, float value);
void add_values(float value);
const ::PROTOBUF_NAMESPACE_ID::RepeatedField< float >&
values() const;
::PROTOBUF_NAMESPACE_ID::RepeatedField< float >*
mutable_values();
// string tag = 1;
void clear_tag();
const std::string& tag() const;
void set_tag(const std::string& value);
void set_tag(std::string&& value);
void set_tag(const char* value);
void set_tag(const char* value, size_t size);
std::string* mutable_tag();
std::string* release_tag();
void set_allocated_tag(std::string* tag);
// @@protoc_insertion_point(class_scope:milvus.proto.service.Score)
private:
class _Internal;
::PROTOBUF_NAMESPACE_ID::internal::InternalMetadataWithArena _internal_metadata_;
::PROTOBUF_NAMESPACE_ID::RepeatedField< float > values_;
mutable std::atomic<int> _values_cached_byte_size_;
::PROTOBUF_NAMESPACE_ID::internal::ArenaStringPtr tag_;
mutable ::PROTOBUF_NAMESPACE_ID::internal::CachedSize _cached_size_;
friend struct ::TableStruct_service_5fmsg_2eproto;
};
// -------------------------------------------------------------------
class Hits :
public ::PROTOBUF_NAMESPACE_ID::Message /* @@protoc_insertion_point(class_definition:milvus.proto.service.Hits) */ {
public:
@ -2527,7 +2196,7 @@ class Hits :
&_Hits_default_instance_);
}
static constexpr int kIndexInFileMessages =
15;
13;
friend void swap(Hits& a, Hits& b) {
a.Swap(&b);
@ -2613,27 +2282,33 @@ class Hits :
::PROTOBUF_NAMESPACE_ID::RepeatedField< ::PROTOBUF_NAMESPACE_ID::int64 >*
mutable_ids();
// repeated .milvus.proto.common.Blob row_data = 2;
// repeated bytes row_data = 2;
int row_data_size() const;
void clear_row_data();
::milvus::proto::common::Blob* mutable_row_data(int index);
::PROTOBUF_NAMESPACE_ID::RepeatedPtrField< ::milvus::proto::common::Blob >*
mutable_row_data();
const ::milvus::proto::common::Blob& row_data(int index) const;
::milvus::proto::common::Blob* add_row_data();
const ::PROTOBUF_NAMESPACE_ID::RepeatedPtrField< ::milvus::proto::common::Blob >&
row_data() const;
const std::string& row_data(int index) const;
std::string* mutable_row_data(int index);
void set_row_data(int index, const std::string& value);
void set_row_data(int index, std::string&& value);
void set_row_data(int index, const char* value);
void set_row_data(int index, const void* value, size_t size);
std::string* add_row_data();
void add_row_data(const std::string& value);
void add_row_data(std::string&& value);
void add_row_data(const char* value);
void add_row_data(const void* value, size_t size);
const ::PROTOBUF_NAMESPACE_ID::RepeatedPtrField<std::string>& row_data() const;
::PROTOBUF_NAMESPACE_ID::RepeatedPtrField<std::string>* mutable_row_data();
// repeated .milvus.proto.service.Score scores = 3;
// repeated float scores = 3;
int scores_size() const;
void clear_scores();
::milvus::proto::service::Score* mutable_scores(int index);
::PROTOBUF_NAMESPACE_ID::RepeatedPtrField< ::milvus::proto::service::Score >*
mutable_scores();
const ::milvus::proto::service::Score& scores(int index) const;
::milvus::proto::service::Score* add_scores();
const ::PROTOBUF_NAMESPACE_ID::RepeatedPtrField< ::milvus::proto::service::Score >&
float scores(int index) const;
void set_scores(int index, float value);
void add_scores(float value);
const ::PROTOBUF_NAMESPACE_ID::RepeatedField< float >&
scores() const;
::PROTOBUF_NAMESPACE_ID::RepeatedField< float >*
mutable_scores();
// @@protoc_insertion_point(class_scope:milvus.proto.service.Hits)
private:
@ -2642,8 +2317,9 @@ class Hits :
::PROTOBUF_NAMESPACE_ID::internal::InternalMetadataWithArena _internal_metadata_;
::PROTOBUF_NAMESPACE_ID::RepeatedField< ::PROTOBUF_NAMESPACE_ID::int64 > ids_;
mutable std::atomic<int> _ids_cached_byte_size_;
::PROTOBUF_NAMESPACE_ID::RepeatedPtrField< ::milvus::proto::common::Blob > row_data_;
::PROTOBUF_NAMESPACE_ID::RepeatedPtrField< ::milvus::proto::service::Score > scores_;
::PROTOBUF_NAMESPACE_ID::RepeatedPtrField<std::string> row_data_;
::PROTOBUF_NAMESPACE_ID::RepeatedField< float > scores_;
mutable std::atomic<int> _scores_cached_byte_size_;
mutable ::PROTOBUF_NAMESPACE_ID::internal::CachedSize _cached_size_;
friend struct ::TableStruct_service_5fmsg_2eproto;
};
@ -2691,7 +2367,7 @@ class QueryResult :
&_QueryResult_default_instance_);
}
static constexpr int kIndexInFileMessages =
16;
14;
friend void swap(QueryResult& a, QueryResult& b) {
a.Swap(&b);
@ -2765,16 +2441,22 @@ class QueryResult :
kHitsFieldNumber = 2,
kStatusFieldNumber = 1,
};
// repeated .milvus.proto.service.Hits hits = 2;
// repeated bytes hits = 2;
int hits_size() const;
void clear_hits();
::milvus::proto::service::Hits* mutable_hits(int index);
::PROTOBUF_NAMESPACE_ID::RepeatedPtrField< ::milvus::proto::service::Hits >*
mutable_hits();
const ::milvus::proto::service::Hits& hits(int index) const;
::milvus::proto::service::Hits* add_hits();
const ::PROTOBUF_NAMESPACE_ID::RepeatedPtrField< ::milvus::proto::service::Hits >&
hits() const;
const std::string& hits(int index) const;
std::string* mutable_hits(int index);
void set_hits(int index, const std::string& value);
void set_hits(int index, std::string&& value);
void set_hits(int index, const char* value);
void set_hits(int index, const void* value, size_t size);
std::string* add_hits();
void add_hits(const std::string& value);
void add_hits(std::string&& value);
void add_hits(const char* value);
void add_hits(const void* value, size_t size);
const ::PROTOBUF_NAMESPACE_ID::RepeatedPtrField<std::string>& hits() const;
::PROTOBUF_NAMESPACE_ID::RepeatedPtrField<std::string>* mutable_hits();
// .milvus.proto.common.Status status = 1;
bool has_status() const;
@ -2789,7 +2471,7 @@ class QueryResult :
class _Internal;
::PROTOBUF_NAMESPACE_ID::internal::InternalMetadataWithArena _internal_metadata_;
::PROTOBUF_NAMESPACE_ID::RepeatedPtrField< ::milvus::proto::service::Hits > hits_;
::PROTOBUF_NAMESPACE_ID::RepeatedPtrField<std::string> hits_;
::milvus::proto::common::Status* status_;
mutable ::PROTOBUF_NAMESPACE_ID::internal::CachedSize _cached_size_;
friend struct ::TableStruct_service_5fmsg_2eproto;
@ -4198,270 +3880,6 @@ 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;
inline void Score::clear_tag() {
tag_.ClearToEmptyNoArena(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited());
}
inline const std::string& Score::tag() const {
// @@protoc_insertion_point(field_get:milvus.proto.service.Score.tag)
return tag_.GetNoArena();
}
inline void Score::set_tag(const std::string& value) {
tag_.SetNoArena(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited(), value);
// @@protoc_insertion_point(field_set:milvus.proto.service.Score.tag)
}
inline void Score::set_tag(std::string&& value) {
tag_.SetNoArena(
&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited(), ::std::move(value));
// @@protoc_insertion_point(field_set_rvalue:milvus.proto.service.Score.tag)
}
inline void Score::set_tag(const char* value) {
GOOGLE_DCHECK(value != nullptr);
tag_.SetNoArena(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited(), ::std::string(value));
// @@protoc_insertion_point(field_set_char:milvus.proto.service.Score.tag)
}
inline void Score::set_tag(const char* value, size_t size) {
tag_.SetNoArena(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited(),
::std::string(reinterpret_cast<const char*>(value), size));
// @@protoc_insertion_point(field_set_pointer:milvus.proto.service.Score.tag)
}
inline std::string* Score::mutable_tag() {
// @@protoc_insertion_point(field_mutable:milvus.proto.service.Score.tag)
return tag_.MutableNoArena(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited());
}
inline std::string* Score::release_tag() {
// @@protoc_insertion_point(field_release:milvus.proto.service.Score.tag)
return tag_.ReleaseNoArena(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited());
}
inline void Score::set_allocated_tag(std::string* tag) {
if (tag != nullptr) {
} else {
}
tag_.SetAllocatedNoArena(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited(), tag);
// @@protoc_insertion_point(field_set_allocated:milvus.proto.service.Score.tag)
}
// repeated float values = 2;
inline int Score::values_size() const {
return values_.size();
}
inline void Score::clear_values() {
values_.Clear();
}
inline float Score::values(int index) const {
// @@protoc_insertion_point(field_get:milvus.proto.service.Score.values)
return values_.Get(index);
}
inline void Score::set_values(int index, float value) {
values_.Set(index, value);
// @@protoc_insertion_point(field_set:milvus.proto.service.Score.values)
}
inline void Score::add_values(float value) {
values_.Add(value);
// @@protoc_insertion_point(field_add:milvus.proto.service.Score.values)
}
inline const ::PROTOBUF_NAMESPACE_ID::RepeatedField< float >&
Score::values() const {
// @@protoc_insertion_point(field_list:milvus.proto.service.Score.values)
return values_;
}
inline ::PROTOBUF_NAMESPACE_ID::RepeatedField< float >*
Score::mutable_values() {
// @@protoc_insertion_point(field_mutable_list:milvus.proto.service.Score.values)
return &values_;
}
// -------------------------------------------------------------------
// Hits
// repeated int64 IDs = 1;
@ -4494,62 +3912,100 @@ Hits::mutable_ids() {
return &ids_;
}
// repeated .milvus.proto.common.Blob row_data = 2;
// repeated bytes row_data = 2;
inline int Hits::row_data_size() const {
return row_data_.size();
}
inline ::milvus::proto::common::Blob* Hits::mutable_row_data(int index) {
// @@protoc_insertion_point(field_mutable:milvus.proto.service.Hits.row_data)
return row_data_.Mutable(index);
inline void Hits::clear_row_data() {
row_data_.Clear();
}
inline ::PROTOBUF_NAMESPACE_ID::RepeatedPtrField< ::milvus::proto::common::Blob >*
Hits::mutable_row_data() {
// @@protoc_insertion_point(field_mutable_list:milvus.proto.service.Hits.row_data)
return &row_data_;
}
inline const ::milvus::proto::common::Blob& Hits::row_data(int index) const {
inline const std::string& Hits::row_data(int index) const {
// @@protoc_insertion_point(field_get:milvus.proto.service.Hits.row_data)
return row_data_.Get(index);
}
inline ::milvus::proto::common::Blob* Hits::add_row_data() {
// @@protoc_insertion_point(field_add:milvus.proto.service.Hits.row_data)
inline std::string* Hits::mutable_row_data(int index) {
// @@protoc_insertion_point(field_mutable:milvus.proto.service.Hits.row_data)
return row_data_.Mutable(index);
}
inline void Hits::set_row_data(int index, const std::string& value) {
// @@protoc_insertion_point(field_set:milvus.proto.service.Hits.row_data)
row_data_.Mutable(index)->assign(value);
}
inline void Hits::set_row_data(int index, std::string&& value) {
// @@protoc_insertion_point(field_set:milvus.proto.service.Hits.row_data)
row_data_.Mutable(index)->assign(std::move(value));
}
inline void Hits::set_row_data(int index, const char* value) {
GOOGLE_DCHECK(value != nullptr);
row_data_.Mutable(index)->assign(value);
// @@protoc_insertion_point(field_set_char:milvus.proto.service.Hits.row_data)
}
inline void Hits::set_row_data(int index, const void* value, size_t size) {
row_data_.Mutable(index)->assign(
reinterpret_cast<const char*>(value), size);
// @@protoc_insertion_point(field_set_pointer:milvus.proto.service.Hits.row_data)
}
inline std::string* Hits::add_row_data() {
// @@protoc_insertion_point(field_add_mutable:milvus.proto.service.Hits.row_data)
return row_data_.Add();
}
inline const ::PROTOBUF_NAMESPACE_ID::RepeatedPtrField< ::milvus::proto::common::Blob >&
inline void Hits::add_row_data(const std::string& value) {
row_data_.Add()->assign(value);
// @@protoc_insertion_point(field_add:milvus.proto.service.Hits.row_data)
}
inline void Hits::add_row_data(std::string&& value) {
row_data_.Add(std::move(value));
// @@protoc_insertion_point(field_add:milvus.proto.service.Hits.row_data)
}
inline void Hits::add_row_data(const char* value) {
GOOGLE_DCHECK(value != nullptr);
row_data_.Add()->assign(value);
// @@protoc_insertion_point(field_add_char:milvus.proto.service.Hits.row_data)
}
inline void Hits::add_row_data(const void* value, size_t size) {
row_data_.Add()->assign(reinterpret_cast<const char*>(value), size);
// @@protoc_insertion_point(field_add_pointer:milvus.proto.service.Hits.row_data)
}
inline const ::PROTOBUF_NAMESPACE_ID::RepeatedPtrField<std::string>&
Hits::row_data() const {
// @@protoc_insertion_point(field_list:milvus.proto.service.Hits.row_data)
return row_data_;
}
inline ::PROTOBUF_NAMESPACE_ID::RepeatedPtrField<std::string>*
Hits::mutable_row_data() {
// @@protoc_insertion_point(field_mutable_list:milvus.proto.service.Hits.row_data)
return &row_data_;
}
// repeated .milvus.proto.service.Score scores = 3;
// repeated float scores = 3;
inline int Hits::scores_size() const {
return scores_.size();
}
inline void Hits::clear_scores() {
scores_.Clear();
}
inline ::milvus::proto::service::Score* Hits::mutable_scores(int index) {
// @@protoc_insertion_point(field_mutable:milvus.proto.service.Hits.scores)
return scores_.Mutable(index);
}
inline ::PROTOBUF_NAMESPACE_ID::RepeatedPtrField< ::milvus::proto::service::Score >*
Hits::mutable_scores() {
// @@protoc_insertion_point(field_mutable_list:milvus.proto.service.Hits.scores)
return &scores_;
}
inline const ::milvus::proto::service::Score& Hits::scores(int index) const {
inline float Hits::scores(int index) const {
// @@protoc_insertion_point(field_get:milvus.proto.service.Hits.scores)
return scores_.Get(index);
}
inline ::milvus::proto::service::Score* Hits::add_scores() {
// @@protoc_insertion_point(field_add:milvus.proto.service.Hits.scores)
return scores_.Add();
inline void Hits::set_scores(int index, float value) {
scores_.Set(index, value);
// @@protoc_insertion_point(field_set:milvus.proto.service.Hits.scores)
}
inline const ::PROTOBUF_NAMESPACE_ID::RepeatedPtrField< ::milvus::proto::service::Score >&
inline void Hits::add_scores(float value) {
scores_.Add(value);
// @@protoc_insertion_point(field_add:milvus.proto.service.Hits.scores)
}
inline const ::PROTOBUF_NAMESPACE_ID::RepeatedField< float >&
Hits::scores() const {
// @@protoc_insertion_point(field_list:milvus.proto.service.Hits.scores)
return scores_;
}
inline ::PROTOBUF_NAMESPACE_ID::RepeatedField< float >*
Hits::mutable_scores() {
// @@protoc_insertion_point(field_mutable_list:milvus.proto.service.Hits.scores)
return &scores_;
}
// -------------------------------------------------------------------
@ -4600,35 +4056,70 @@ inline void QueryResult::set_allocated_status(::milvus::proto::common::Status* s
// @@protoc_insertion_point(field_set_allocated:milvus.proto.service.QueryResult.status)
}
// repeated .milvus.proto.service.Hits hits = 2;
// repeated bytes hits = 2;
inline int QueryResult::hits_size() const {
return hits_.size();
}
inline void QueryResult::clear_hits() {
hits_.Clear();
}
inline ::milvus::proto::service::Hits* QueryResult::mutable_hits(int index) {
// @@protoc_insertion_point(field_mutable:milvus.proto.service.QueryResult.hits)
return hits_.Mutable(index);
}
inline ::PROTOBUF_NAMESPACE_ID::RepeatedPtrField< ::milvus::proto::service::Hits >*
QueryResult::mutable_hits() {
// @@protoc_insertion_point(field_mutable_list:milvus.proto.service.QueryResult.hits)
return &hits_;
}
inline const ::milvus::proto::service::Hits& QueryResult::hits(int index) const {
inline const std::string& QueryResult::hits(int index) const {
// @@protoc_insertion_point(field_get:milvus.proto.service.QueryResult.hits)
return hits_.Get(index);
}
inline ::milvus::proto::service::Hits* QueryResult::add_hits() {
// @@protoc_insertion_point(field_add:milvus.proto.service.QueryResult.hits)
inline std::string* QueryResult::mutable_hits(int index) {
// @@protoc_insertion_point(field_mutable:milvus.proto.service.QueryResult.hits)
return hits_.Mutable(index);
}
inline void QueryResult::set_hits(int index, const std::string& value) {
// @@protoc_insertion_point(field_set:milvus.proto.service.QueryResult.hits)
hits_.Mutable(index)->assign(value);
}
inline void QueryResult::set_hits(int index, std::string&& value) {
// @@protoc_insertion_point(field_set:milvus.proto.service.QueryResult.hits)
hits_.Mutable(index)->assign(std::move(value));
}
inline void QueryResult::set_hits(int index, const char* value) {
GOOGLE_DCHECK(value != nullptr);
hits_.Mutable(index)->assign(value);
// @@protoc_insertion_point(field_set_char:milvus.proto.service.QueryResult.hits)
}
inline void QueryResult::set_hits(int index, const void* value, size_t size) {
hits_.Mutable(index)->assign(
reinterpret_cast<const char*>(value), size);
// @@protoc_insertion_point(field_set_pointer:milvus.proto.service.QueryResult.hits)
}
inline std::string* QueryResult::add_hits() {
// @@protoc_insertion_point(field_add_mutable:milvus.proto.service.QueryResult.hits)
return hits_.Add();
}
inline const ::PROTOBUF_NAMESPACE_ID::RepeatedPtrField< ::milvus::proto::service::Hits >&
inline void QueryResult::add_hits(const std::string& value) {
hits_.Add()->assign(value);
// @@protoc_insertion_point(field_add:milvus.proto.service.QueryResult.hits)
}
inline void QueryResult::add_hits(std::string&& value) {
hits_.Add(std::move(value));
// @@protoc_insertion_point(field_add:milvus.proto.service.QueryResult.hits)
}
inline void QueryResult::add_hits(const char* value) {
GOOGLE_DCHECK(value != nullptr);
hits_.Add()->assign(value);
// @@protoc_insertion_point(field_add_char:milvus.proto.service.QueryResult.hits)
}
inline void QueryResult::add_hits(const void* value, size_t size) {
hits_.Add()->assign(reinterpret_cast<const char*>(value), size);
// @@protoc_insertion_point(field_add_pointer:milvus.proto.service.QueryResult.hits)
}
inline const ::PROTOBUF_NAMESPACE_ID::RepeatedPtrField<std::string>&
QueryResult::hits() const {
// @@protoc_insertion_point(field_list:milvus.proto.service.QueryResult.hits)
return hits_;
}
inline ::PROTOBUF_NAMESPACE_ID::RepeatedPtrField<std::string>*
QueryResult::mutable_hits() {
// @@protoc_insertion_point(field_mutable_list:milvus.proto.service.QueryResult.hits)
return &hits_;
}
#ifdef __GNUC__
#pragma GCC diagnostic pop
@ -4661,10 +4152,6 @@ QueryResult::hits() const {
// -------------------------------------------------------------------
// -------------------------------------------------------------------
// -------------------------------------------------------------------
// @@protoc_insertion_point(namespace_scope)

View File

@ -130,13 +130,7 @@ ExecExprVisitor::ExecRangeVisitorDispatcher(RangeExpr& expr_raw) -> RetType {
}
case OpType::NotEqual: {
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;
};
auto index_func = [val](Index* index) { return index->NotIn(1, &val); };
return ExecRangeVisitorImpl(expr, index_func, [val](T x) { return !(x != val); });
}

View File

@ -13,9 +13,9 @@
extern "C" {
#endif
#include "segcore/collection_c.h"
#include <stdbool.h>
#include <stdint.h>
#include "segcore/collection_c.h"
typedef void* CPlan;
typedef void* CPlaceholderGroup;

View File

@ -9,11 +9,183 @@
// is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
// or implied. See the License for the specific language governing permissions and limitations under the License
#include <vector>
#include <utils/EasyAssert.h>
#include "segcore/reduce_c.h"
#include "segcore/Reduce.h"
#include "utils/Types.h"
#include "pb/service_msg.pb.h"
using SearchResult = milvus::engine::QueryResult;
int
MergeInto(int64_t num_queries, int64_t topk, float* distances, int64_t* uids, float* new_distances, int64_t* new_uids) {
auto status = milvus::segcore::merge_into(num_queries, topk, distances, uids, new_distances, new_uids);
return status.code();
}
struct MarshaledHitsPeerGroup {
std::vector<std::string> hits_;
std::vector<int64_t> blob_length_;
};
struct MarshaledHits {
explicit MarshaledHits(int64_t num_group) {
marshaled_hits_.resize(num_group);
}
int
get_num_group() {
return marshaled_hits_.size();
}
std::vector<MarshaledHitsPeerGroup> marshaled_hits_;
};
void
DeleteMarshaledHits(CMarshaledHits c_marshaled_hits) {
auto hits = (MarshaledHits*)c_marshaled_hits;
delete hits;
}
struct SearchResultPair {
uint64_t id_;
float distance_;
int64_t segment_id_;
SearchResultPair(uint64_t id, float distance, int64_t segment_id)
: id_(id), distance_(distance), segment_id_(segment_id) {
}
bool
operator<(const SearchResultPair& pair) const {
return (distance_ < pair.distance_);
}
};
void
GetResultData(std::vector<SearchResult*>& search_results,
SearchResult& final_result,
int64_t query_offset,
int64_t topk) {
auto num_segments = search_results.size();
std::map<int, int> iter_loc_peer_result;
std::vector<SearchResultPair> result_pairs;
for (int j = 0; j < num_segments; ++j) {
auto id = search_results[j]->result_ids_[query_offset];
auto distance = search_results[j]->result_distances_[query_offset];
result_pairs.push_back(SearchResultPair(id, distance, j));
iter_loc_peer_result[j] = query_offset;
}
std::sort(result_pairs.begin(), result_pairs.end());
final_result.result_ids_.push_back(result_pairs[0].id_);
final_result.result_distances_.push_back(result_pairs[0].distance_);
for (int i = 1; i < topk; ++i) {
auto segment_id = result_pairs[0].segment_id_;
auto query_offset = ++(iter_loc_peer_result[segment_id]);
auto id = search_results[segment_id]->result_ids_[query_offset];
auto distance = search_results[segment_id]->result_distances_[query_offset];
result_pairs[0] = SearchResultPair(id, distance, segment_id);
std::sort(result_pairs.begin(), result_pairs.end());
final_result.result_ids_.push_back(result_pairs[0].id_);
final_result.result_distances_.push_back(result_pairs[0].distance_);
}
}
CQueryResult
ReduceQueryResults(CQueryResult* query_results, int64_t num_segments) {
std::vector<SearchResult*> search_results;
for (int i = 0; i < num_segments; ++i) {
search_results.push_back((SearchResult*)query_results[i]);
}
auto topk = search_results[0]->topK_;
auto num_queries = search_results[0]->num_queries_;
auto final_result = std::make_unique<SearchResult>();
int64_t query_offset = 0;
for (int j = 0; j < num_queries; ++j) {
GetResultData(search_results, *final_result, query_offset, topk);
query_offset += topk;
}
return (CQueryResult)final_result.release();
}
CMarshaledHits
ReorganizeQueryResults(CQueryResult c_query_result,
CPlan c_plan,
CPlaceholderGroup* c_placeholder_groups,
int64_t num_groups) {
auto marshaledHits = std::make_unique<MarshaledHits>(num_groups);
auto search_result = (milvus::engine::QueryResult*)c_query_result;
auto& result_ids = search_result->result_ids_;
auto& result_distances = search_result->result_distances_;
auto topk = GetTopK(c_plan);
int64_t queries_offset = 0;
for (int i = 0; i < num_groups; i++) {
auto num_queries = GetNumOfQueries(c_placeholder_groups[i]);
MarshaledHitsPeerGroup& hits_peer_group = (*marshaledHits).marshaled_hits_[i];
for (int j = 0; j < num_queries; j++) {
auto index = topk * queries_offset++;
milvus::proto::service::Hits hits;
for (int k = index; k < index + topk; k++) {
hits.add_ids(result_ids[k]);
hits.add_scores(result_distances[k]);
}
auto blob = hits.SerializeAsString();
hits_peer_group.hits_.push_back(blob);
hits_peer_group.blob_length_.push_back(blob.size());
}
}
return (CMarshaledHits)marshaledHits.release();
}
int64_t
GetHitsBlobSize(CMarshaledHits c_marshaled_hits) {
int64_t total_size = 0;
auto marshaled_hits = (MarshaledHits*)c_marshaled_hits;
auto num_group = marshaled_hits->get_num_group();
for (int i = 0; i < num_group; i++) {
auto& length_vector = marshaled_hits->marshaled_hits_[i].blob_length_;
for (int j = 0; j < length_vector.size(); j++) {
total_size += length_vector[j];
}
}
return total_size;
}
void
GetHitsBlob(CMarshaledHits c_marshaled_hits, const void* hits) {
auto byte_hits = (char*)hits;
auto marshaled_hits = (MarshaledHits*)c_marshaled_hits;
auto num_group = marshaled_hits->get_num_group();
int offset = 0;
for (int i = 0; i < num_group; i++) {
auto& hits = marshaled_hits->marshaled_hits_[i];
auto num_queries = hits.hits_.size();
for (int j = 0; j < num_queries; j++) {
auto blob_size = hits.blob_length_[j];
memcpy(byte_hits + offset, hits.hits_[j].data(), blob_size);
offset += blob_size;
}
}
}
int64_t
GetNumQueriesPeerGroup(CMarshaledHits c_marshaled_hits, int64_t group_index) {
auto marshaled_hits = (MarshaledHits*)c_marshaled_hits;
auto& hits = marshaled_hits->marshaled_hits_[group_index].hits_;
return hits.size();
}
void
GetHitSizePeerQueries(CMarshaledHits c_marshaled_hits, int64_t group_index, int64_t* hit_size_peer_query) {
auto marshaled_hits = (MarshaledHits*)c_marshaled_hits;
auto& blob_lens = marshaled_hits->marshaled_hits_[group_index].blob_length_;
for (int i = 0; i < blob_lens.size(); i++) {
hit_size_peer_query[i] = blob_lens[i];
}
}

View File

@ -15,10 +15,37 @@ extern "C" {
#include <stdbool.h>
#include <stdint.h>
#include "segcore/segment_c.h"
typedef void* CMarshaledHits;
void
DeleteMarshaledHits(CMarshaledHits c_marshaled_hits);
int
MergeInto(int64_t num_queries, int64_t topk, float* distances, int64_t* uids, float* new_distances, int64_t* new_uids);
CQueryResult
ReduceQueryResults(CQueryResult* query_results, int64_t num_segments);
CMarshaledHits
ReorganizeQueryResults(CQueryResult query_result,
CPlan c_plan,
CPlaceholderGroup* c_placeholder_groups,
int64_t num_groups);
int64_t
GetHitsBlobSize(CMarshaledHits c_marshaled_hits);
void
GetHitsBlob(CMarshaledHits c_marshaled_hits, const void* hits);
int64_t
GetNumQueriesPeerGroup(CMarshaledHits c_marshaled_hits, int64_t group_index);
void
GetHitSizePeerQueries(CMarshaledHits c_marshaled_hits, int64_t group_index, int64_t* hit_size_peer_query);
#ifdef __cplusplus
}
#endif

View File

@ -18,6 +18,7 @@
#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) {
@ -39,9 +40,15 @@ DeleteSegment(CSegmentBase segment) {
delete s;
}
void
DeleteQueryResult(CQueryResult query_result) {
auto res = (milvus::segcore::QueryResult*)query_result;
delete res;
}
//////////////////////////////////////////////////////////////////
int
CStatus
Insert(CSegmentBase c_segment,
int64_t reserved_offset,
int64_t size,
@ -57,11 +64,22 @@ Insert(CSegmentBase c_segment,
dataChunk.sizeof_per_row = sizeof_per_row;
dataChunk.count = count;
auto res = segment->Insert(reserved_offset, size, row_ids, timestamps, dataChunk);
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;
}
// TODO: delete print
// std::cout << "do segment insert, sizeof_per_row = " << sizeof_per_row << std::endl;
return res.code();
}
int64_t
@ -73,13 +91,24 @@ PreInsert(CSegmentBase c_segment, int64_t size) {
return segment->PreInsert(size);
}
int
CStatus
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;
auto res = segment->Delete(reserved_offset, size, row_ids, timestamps);
return res.code();
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;
}
}
int64_t
@ -91,30 +120,39 @@ PreDelete(CSegmentBase c_segment, int64_t size) {
return segment->PreDelete(size);
}
int
CStatus
Search(CSegmentBase c_segment,
CPlan c_plan,
CPlaceholderGroup* c_placeholder_groups,
uint64_t* timestamps,
int num_groups,
int64_t* result_ids,
float* result_distances) {
CQueryResult* result) {
auto segment = (milvus::segcore::SegmentBase*)c_segment;
auto plan = (milvus::query::Plan*)c_plan;
std::vector<const milvus::query::PlaceholderGroup*> placeholder_groups;
for (int i = 0; i < num_groups; ++i) {
placeholder_groups.push_back((const milvus::query::PlaceholderGroup*)c_placeholder_groups[i]);
}
milvus::segcore::QueryResult query_result;
auto res = segment->Search(plan, placeholder_groups.data(), timestamps, num_groups, query_result);
auto query_result = std::make_unique<milvus::segcore::QueryResult>();
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());
}
*result = query_result.release();
// 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));
// memcpy(result_ids, query_result.result_ids_.data(), query_result.get_row_count() * sizeof(long int));
// memcpy(result_distances, query_result.result_distances_.data(), query_result.get_row_count() * sizeof(float));
return res.code();
return status;
}
//////////////////////////////////////////////////////////////////

View File

@ -14,11 +14,24 @@ extern "C" {
#endif
#include <stdbool.h>
#include "segcore/collection_c.h"
#include "segcore/plan_c.h"
#include <stdlib.h>
#include <stdint.h>
#include "segcore/collection_c.h"
#include "segcore/plan_c.h"
typedef void* CSegmentBase;
typedef void* CQueryResult;
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);
@ -26,9 +39,12 @@ NewSegment(CCollection collection, uint64_t segment_id);
void
DeleteSegment(CSegmentBase segment);
void
DeleteQueryResult(CQueryResult query_result);
//////////////////////////////////////////////////////////////////
int
CStatus
Insert(CSegmentBase c_segment,
int64_t reserved_offset,
int64_t size,
@ -41,21 +57,20 @@ Insert(CSegmentBase c_segment,
int64_t
PreInsert(CSegmentBase c_segment, int64_t size);
int
CStatus
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);
int
CStatus
Search(CSegmentBase c_segment,
CPlan plan,
CPlaceholderGroup* placeholder_groups,
uint64_t* timestamps,
int num_groups,
int64_t* result_ids,
float* result_distances);
CQueryResult* result);
//////////////////////////////////////////////////////////////////

View File

@ -15,7 +15,6 @@
#include <gtest/gtest.h>
#include "segcore/collection_c.h"
#include "segcore/segment_c.h"
#include "pb/service_msg.pb.h"
#include "segcore/reduce_c.h"
@ -65,7 +64,7 @@ TEST(CApiTest, InsertTest) {
auto res = Insert(segment, offset, N, uids.data(), timestamps.data(), raw_data.data(), (int)line_sizeof, N);
assert(res == 0);
assert(res.error_code == Success);
DeleteCollection(collection);
DeleteSegment(segment);
@ -82,7 +81,7 @@ TEST(CApiTest, DeleteTest) {
auto offset = PreDelete(segment, 3);
auto del_res = Delete(segment, offset, 3, delete_row_ids, delete_timestamps);
assert(del_res == 0);
assert(del_res.error_code == Success);
DeleteCollection(collection);
DeleteSegment(segment);
@ -116,7 +115,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 == 0);
assert(ins_res.error_code == Success);
const char* dsl_string = R"(
{
@ -159,105 +158,100 @@ TEST(CApiTest, SearchTest) {
timestamps.clear();
timestamps.push_back(1);
long result_ids[100];
float result_distances[100];
auto sea_res = Search(segment, plan, placeholderGroups.data(), timestamps.data(), 1, result_ids, result_distances);
assert(sea_res == 0);
CQueryResult search_result;
auto res = Search(segment, plan, placeholderGroups.data(), timestamps.data(), 1, &search_result);
assert(res.error_code == Success);
DeletePlan(plan);
DeletePlaceholderGroup(placeholderGroup);
DeleteQueryResult(search_result);
DeleteCollection(collection);
DeleteSegment(segment);
}
TEST(CApiTest, BuildIndexTest) {
auto schema_tmp_conf = "";
auto collection = NewCollection(schema_tmp_conf);
auto segment = NewSegment(collection, 0);
std::vector<char> raw_data;
std::vector<uint64_t> timestamps;
std::vector<int64_t> uids;
int N = 10000;
std::default_random_engine e(67);
for (int i = 0; i < N; ++i) {
uids.push_back(100000 + i);
timestamps.push_back(0);
// append vec
float vec[16];
for (auto& x : vec) {
x = e() % 2000 * 0.001 - 1.0;
}
raw_data.insert(raw_data.end(), (const char*)std::begin(vec), (const char*)std::end(vec));
int age = e() % 100;
raw_data.insert(raw_data.end(), (const char*)&age, ((const char*)&age) + sizeof(age));
}
auto line_sizeof = (sizeof(int) + sizeof(float) * 16);
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 == 0);
// TODO: add index ptr
Close(segment);
// BuildIndex(collection, segment);
const char* dsl_string = R"(
{
"bool": {
"vector": {
"fakevec": {
"metric_type": "L2",
"params": {
"nprobe": 10
},
"query": "$0",
"topk": 10
}
}
}
})";
namespace ser = milvus::proto::service;
int num_queries = 10;
int dim = 16;
std::normal_distribution<double> dis(0, 1);
ser::PlaceholderGroup raw_group;
auto value = raw_group.add_placeholders();
value->set_tag("$0");
value->set_type(ser::PlaceholderType::VECTOR_FLOAT);
for (int i = 0; i < num_queries; ++i) {
std::vector<float> vec;
for (int d = 0; d < dim; ++d) {
vec.push_back(dis(e));
}
// std::string line((char*)vec.data(), (char*)vec.data() + vec.size() * sizeof(float));
value->add_values(vec.data(), vec.size() * sizeof(float));
}
auto blob = raw_group.SerializeAsString();
auto plan = CreatePlan(collection, dsl_string);
auto placeholderGroup = ParsePlaceholderGroup(plan, blob.data(), blob.length());
std::vector<CPlaceholderGroup> placeholderGroups;
placeholderGroups.push_back(placeholderGroup);
timestamps.clear();
timestamps.push_back(1);
long result_ids[100];
float result_distances[100];
auto sea_res = Search(segment, plan, placeholderGroups.data(), timestamps.data(), 1, result_ids, result_distances);
assert(sea_res == 0);
DeletePlan(plan);
DeletePlaceholderGroup(placeholderGroup);
DeleteCollection(collection);
DeleteSegment(segment);
}
// TEST(CApiTest, BuildIndexTest) {
// auto schema_tmp_conf = "";
// auto collection = NewCollection(schema_tmp_conf);
// auto segment = NewSegment(collection, 0);
//
// std::vector<char> raw_data;
// std::vector<uint64_t> timestamps;
// std::vector<int64_t> uids;
// int N = 10000;
// std::default_random_engine e(67);
// for (int i = 0; i < N; ++i) {
// uids.push_back(100000 + i);
// timestamps.push_back(0);
// // append vec
// float vec[16];
// for (auto& x : vec) {
// x = e() % 2000 * 0.001 - 1.0;
// }
// raw_data.insert(raw_data.end(), (const char*)std::begin(vec), (const char*)std::end(vec));
// int age = e() % 100;
// raw_data.insert(raw_data.end(), (const char*)&age, ((const char*)&age) + sizeof(age));
// }
//
// auto line_sizeof = (sizeof(int) + sizeof(float) * 16);
//
// 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 == 0);
//
// // TODO: add index ptr
// Close(segment);
// BuildIndex(collection, segment);
//
// const char* dsl_string = R"(
// {
// "bool": {
// "vector": {
// "fakevec": {
// "metric_type": "L2",
// "params": {
// "nprobe": 10
// },
// "query": "$0",
// "topk": 10
// }
// }
// }
// })";
//
// namespace ser = milvus::proto::service;
// int num_queries = 10;
// int dim = 16;
// std::normal_distribution<double> dis(0, 1);
// ser::PlaceholderGroup raw_group;
// auto value = raw_group.add_placeholders();
// value->set_tag("$0");
// value->set_type(ser::PlaceholderType::VECTOR_FLOAT);
// for (int i = 0; i < num_queries; ++i) {
// std::vector<float> vec;
// for (int d = 0; d < dim; ++d) {
// vec.push_back(dis(e));
// }
// // std::string line((char*)vec.data(), (char*)vec.data() + vec.size() * sizeof(float));
// value->add_values(vec.data(), vec.size() * sizeof(float));
// }
// auto blob = raw_group.SerializeAsString();
//
// auto plan = CreatePlan(collection, dsl_string);
// auto placeholderGroup = ParsePlaceholderGroup(plan, blob.data(), blob.length());
// std::vector<CPlaceholderGroup> placeholderGroups;
// placeholderGroups.push_back(placeholderGroup);
// timestamps.clear();
// timestamps.push_back(1);
//
// auto search_res = Search(segment, plan, placeholderGroups.data(), timestamps.data(), 1);
//
// DeletePlan(plan);
// DeletePlaceholderGroup(placeholderGroup);
// DeleteQueryResult(search_res);
// DeleteCollection(collection);
// DeleteSegment(segment);
//}
TEST(CApiTest, IsOpenedTest) {
auto schema_tmp_conf = "";
@ -315,7 +309,7 @@ TEST(CApiTest, GetMemoryUsageInBytesTest) {
auto res = Insert(segment, offset, N, uids.data(), timestamps.data(), raw_data.data(), (int)line_sizeof, N);
assert(res == 0);
assert(res.error_code == Success);
auto memory_usage_size = GetMemoryUsageInBytes(segment);
@ -482,7 +476,7 @@ TEST(CApiTest, GetDeletedCountTest) {
auto offset = PreDelete(segment, 3);
auto del_res = Delete(segment, offset, 3, delete_row_ids, delete_timestamps);
assert(del_res == 0);
assert(del_res.error_code == Success);
// TODO: assert(deleted_count == len(delete_row_ids))
auto deleted_count = GetDeletedCount(segment);
@ -502,7 +496,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 == 0);
assert(res.error_code == Success);
auto row_count = GetRowCount(segment);
assert(row_count == N);
@ -552,3 +546,109 @@ TEST(CApiTest, MergeInto) {
ASSERT_EQ(uids[1], 1);
ASSERT_EQ(distance[1], 5);
}
TEST(CApiTest, Reduce) {
auto schema_tmp_conf = "";
auto collection = NewCollection(schema_tmp_conf);
auto segment = NewSegment(collection, 0);
std::vector<char> raw_data;
std::vector<uint64_t> timestamps;
std::vector<int64_t> uids;
int N = 10000;
std::default_random_engine e(67);
for (int i = 0; i < N; ++i) {
uids.push_back(100000 + i);
timestamps.push_back(0);
// append vec
float vec[16];
for (auto& x : vec) {
x = e() % 2000 * 0.001 - 1.0;
}
raw_data.insert(raw_data.end(), (const char*)std::begin(vec), (const char*)std::end(vec));
int age = e() % 100;
raw_data.insert(raw_data.end(), (const char*)&age, ((const char*)&age) + sizeof(age));
}
auto line_sizeof = (sizeof(int) + sizeof(float) * 16);
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);
const char* dsl_string = R"(
{
"bool": {
"vector": {
"fakevec": {
"metric_type": "L2",
"params": {
"nprobe": 10
},
"query": "$0",
"topk": 10
}
}
}
})";
namespace ser = milvus::proto::service;
int num_queries = 10;
int dim = 16;
std::normal_distribution<double> dis(0, 1);
ser::PlaceholderGroup raw_group;
auto value = raw_group.add_placeholders();
value->set_tag("$0");
value->set_type(ser::PlaceholderType::VECTOR_FLOAT);
for (int i = 0; i < num_queries; ++i) {
std::vector<float> vec;
for (int d = 0; d < dim; ++d) {
vec.push_back(dis(e));
}
// std::string line((char*)vec.data(), (char*)vec.data() + vec.size() * sizeof(float));
value->add_values(vec.data(), vec.size() * sizeof(float));
}
auto blob = raw_group.SerializeAsString();
auto plan = CreatePlan(collection, dsl_string);
auto placeholderGroup = ParsePlaceholderGroup(plan, blob.data(), blob.length());
std::vector<CPlaceholderGroup> placeholderGroups;
placeholderGroups.push_back(placeholderGroup);
timestamps.clear();
timestamps.push_back(1);
std::vector<CQueryResult> results;
CQueryResult res1;
CQueryResult res2;
auto res = Search(segment, plan, placeholderGroups.data(), timestamps.data(), 1, &res1);
assert(res.error_code == Success);
res = Search(segment, plan, placeholderGroups.data(), timestamps.data(), 1, &res2);
assert(res.error_code == Success);
results.push_back(res1);
results.push_back(res2);
auto reduced_search_result = ReduceQueryResults(results.data(), 2);
auto reorganize_search_result = ReorganizeQueryResults(reduced_search_result, plan, placeholderGroups.data(), 1);
auto hits_blob_size = GetHitsBlobSize(reorganize_search_result);
assert(hits_blob_size > 0);
std::vector<char> hits_blob;
hits_blob.resize(hits_blob_size);
GetHitsBlob(reorganize_search_result, hits_blob.data());
assert(hits_blob.data() != nullptr);
auto num_queries_group = GetNumQueriesPeerGroup(reorganize_search_result, 0);
assert(num_queries_group == 10);
std::vector<int64_t> hit_size_peer_query;
hit_size_peer_query.resize(num_queries_group);
GetHitSizePeerQueries(reorganize_search_result, 0, hit_size_peer_query.data());
assert(hit_size_peer_query[0] > 0);
DeletePlan(plan);
DeletePlaceholderGroup(placeholderGroup);
DeleteQueryResult(res1);
DeleteQueryResult(res2);
DeleteQueryResult(reduced_search_result);
DeleteMarshaledHits(reorganize_search_result);
DeleteCollection(collection);
DeleteSegment(segment);
}

View File

@ -1,79 +0,0 @@
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
}

View File

@ -1,150 +0,0 @@
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()))
})
}

View File

@ -1,111 +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.
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

View File

@ -359,43 +359,6 @@ 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) {

View File

@ -52,7 +52,7 @@ func (mt *metaTable) reloadFromKV() error {
for _, value := range values {
tenantMeta := pb.TenantMeta{}
err := proto.Unmarshal([]byte(value), &tenantMeta)
err := proto.UnmarshalText(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.Unmarshal([]byte(value), &proxyMeta)
err = proto.UnmarshalText(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.Unmarshal([]byte(value), &collectionMeta)
err = proto.UnmarshalText(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.Unmarshal([]byte(value), &segmentMeta)
err = proto.UnmarshalText(value, &segmentMeta)
if err != nil {
return err
}
@ -107,10 +107,7 @@ func (mt *metaTable) reloadFromKV() error {
// metaTable.ddLock.Lock() before call this function
func (mt *metaTable) saveCollectionMeta(coll *pb.CollectionMeta) error {
collBytes, err := proto.Marshal(coll)
if err != nil {
return err
}
collBytes := proto.MarshalTextString(coll)
mt.collID2Meta[coll.ID] = *coll
mt.collName2ID[coll.Schema.Name] = coll.ID
return mt.client.Save("/collection/"+strconv.FormatInt(coll.ID, 10), string(collBytes))
@ -118,10 +115,7 @@ func (mt *metaTable) saveCollectionMeta(coll *pb.CollectionMeta) error {
// metaTable.ddLock.Lock() before call this function
func (mt *metaTable) saveSegmentMeta(seg *pb.SegmentMeta) error {
segBytes, err := proto.Marshal(seg)
if err != nil {
return err
}
segBytes := proto.MarshalTextString(seg)
mt.segID2Meta[seg.SegmentID] = *seg
@ -136,10 +130,7 @@ func (mt *metaTable) saveCollectionAndDeleteSegmentsMeta(coll *pb.CollectionMeta
}
kvs := make(map[string]string)
collStrs, err := proto.Marshal(coll)
if err != nil {
return err
}
collStrs := proto.MarshalTextString(coll)
kvs["/collection/"+strconv.FormatInt(coll.ID, 10)] = string(collStrs)
@ -159,19 +150,15 @@ 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, err := proto.Marshal(coll)
if err != nil {
return err
}
collBytes := proto.MarshalTextString(coll)
kvs["/collection/"+strconv.FormatInt(coll.ID, 10)] = string(collBytes)
mt.collID2Meta[coll.ID] = *coll
mt.collName2ID[coll.Schema.Name] = coll.ID
segBytes, err := proto.Marshal(seg)
if err != nil {
return err
}
segBytes := proto.MarshalTextString(seg)
kvs["/segment/"+strconv.FormatInt(seg.SegmentID, 10)] = string(segBytes)
mt.segID2Meta[seg.SegmentID] = *seg
@ -220,7 +207,7 @@ func (mt *metaTable) AddCollection(coll *pb.CollectionMeta) error {
}
if len(coll.PartitionTags) == 0 {
coll.PartitionTags = append(coll.PartitionTags, "default")
coll.PartitionTags = append(coll.PartitionTags, Params.DefaultPartitionTag)
}
_, ok := mt.collName2ID[coll.Schema.Name]
if ok {
@ -292,6 +279,10 @@ 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.")
@ -326,17 +317,29 @@ 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
}

View File

@ -3,6 +3,7 @@ package master
import (
"context"
"reflect"
"strconv"
"testing"
"github.com/stretchr/testify/assert"
@ -238,6 +239,10 @@ 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) {
@ -366,3 +371,39 @@ 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)
}

View File

@ -43,6 +43,9 @@ type ParamTable struct {
K2SChannelNames []string
QueryNodeStatsChannelName string
MsgChannelSubName string
MaxPartitionNum int64
DefaultPartitionTag string
}
var Params ParamTable
@ -62,6 +65,10 @@ func (p *ParamTable) Init() {
if err != nil {
panic(err)
}
err = p.LoadYaml("advanced/common.yaml")
if err != nil {
panic(err)
}
// set members
p.initAddress()
@ -91,6 +98,8 @@ func (p *ParamTable) Init() {
p.initK2SChannelNames()
p.initQueryNodeStatsChannelName()
p.initMsgChannelSubName()
p.initMaxPartitionNum()
p.initDefaultPartitionTag()
}
func (p *ParamTable) initAddress() {
@ -360,18 +369,33 @@ func (p *ParamTable) initInsertChannelNames() {
if err != nil {
log.Fatal(err)
}
id, err := p.Load("nodeID.queryNodeIDList")
channelRange, err := p.Load("msgChannel.channelRange.insert")
if err != nil {
log.Panicf("load query node id list error, %s", err.Error())
panic(err)
}
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)
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)
}
p.InsertChannelNames = channels
}
@ -396,3 +420,24 @@ 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
}

View File

@ -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, 15)
assert.Equal(t, num, 1)
}
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, 2)
assert.Equal(t, num, 1)
}
func TestParamTable_QueryNodeStatsChannelName(t *testing.T) {
@ -85,17 +85,15 @@ func TestParamTable_QueryNodeStatsChannelName(t *testing.T) {
func TestParamTable_ProxyIDList(t *testing.T) {
Params.Init()
ids := Params.ProxyIDList
assert.Equal(t, len(ids), 2)
assert.Equal(t, len(ids), 1)
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), 2)
assert.Equal(t, len(names), 1)
assert.Equal(t, names[0], "proxyTimeTick-1")
assert.Equal(t, names[1], "proxyTimeTick-2")
}
func TestParamTable_MsgChannelSubName(t *testing.T) {
@ -113,31 +111,27 @@ func TestParamTable_SoftTimeTickBarrierInterval(t *testing.T) {
func TestParamTable_WriteNodeIDList(t *testing.T) {
Params.Init()
ids := Params.WriteNodeIDList
assert.Equal(t, len(ids), 2)
assert.Equal(t, ids[0], int64(5))
assert.Equal(t, ids[1], int64(6))
assert.Equal(t, len(ids), 1)
assert.Equal(t, ids[0], int64(3))
}
func TestParamTable_WriteNodeTimeTickChannelNames(t *testing.T) {
Params.Init()
names := Params.WriteNodeTimeTickChannelNames
assert.Equal(t, len(names), 2)
assert.Equal(t, names[0], "writeNodeTimeTick-5")
assert.Equal(t, names[1], "writeNodeTimeTick-6")
assert.Equal(t, len(names), 1)
assert.Equal(t, names[0], "writeNodeTimeTick-3")
}
func TestParamTable_InsertChannelNames(t *testing.T) {
Params.Init()
names := Params.InsertChannelNames
assert.Equal(t, len(names), 2)
assert.Equal(t, names[0], "insert-3")
assert.Equal(t, names[1], "insert-4")
assert.Equal(t, len(names), 1)
assert.Equal(t, names[0], "insert-0")
}
func TestParamTable_K2SChannelNames(t *testing.T) {
Params.Init()
names := Params.K2SChannelNames
assert.Equal(t, len(names), 2)
assert.Equal(t, names[0], "k2s-5")
assert.Equal(t, names[1], "k2s-6")
assert.Equal(t, len(names), 1)
assert.Equal(t, names[0], "k2s-3")
}

View File

@ -191,10 +191,12 @@ func (t *showPartitionTask) Execute() error {
return errors.New("null request")
}
partitions := make([]string, 0)
for _, collection := range t.mt.collID2Meta {
partitions = append(partitions, collection.PartitionTags...)
collMeta, err := t.mt.GetCollectionByName(t.req.CollectionName.CollectionName)
if err != nil {
return err
}
partitions := make([]string, 0)
partitions = append(partitions, collMeta.PartitionTags...)
stringListResponse := servicepb.StringListResponse{
Status: &commonpb.Status{

View File

@ -60,6 +60,9 @@ 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)
@ -212,7 +215,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,
@ -224,7 +227,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,

View File

@ -261,6 +261,9 @@ func startupMaster() {
K2SChannelNames: []string{"k2s0", "k2s1"},
QueryNodeStatsChannelName: "statistic",
MsgChannelSubName: Params.MsgChannelSubName,
MaxPartitionNum: int64(4096),
DefaultPartitionTag: "_default",
}
master, err = CreateServer(ctx)

View File

@ -1,117 +0,0 @@
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
}

View File

@ -1,209 +0,0 @@
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)
})
}

View File

@ -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 reader producer %s, error = %v", channels[i], err)
log.Printf("Failed to create querynode producer %s, error = %v", channels[i], err)
}
ms.producers = append(ms.producers, &pp)
}
@ -141,6 +141,15 @@ func (ms *PulsarMsgStream) Produce(msgPack *MsgPack) error {
hashValues := tsMsg.HashKeys()
bucketValues := make([]int32, len(hashValues))
for index, hashValue := range hashValues {
if tsMsg.Type() == internalPb.MsgType_kSearchResult {
searchResult := tsMsg.(*SearchResultMsg)
channelID := int32(searchResult.ResultChannelID)
if channelID >= int32(len(ms.producers)) {
return errors.New("Failed to produce pulsar msg to unKnow channel")
}
bucketValues[index] = channelID
continue
}
bucketValues[index] = hashValue % int32(len(ms.producers))
}
reBucketValues[channelID] = bucketValues

View File

@ -50,7 +50,7 @@ func getTsMsg(msgType MsgType, reqID UniqueID, hashValue int32) TsMsg {
CollectionName: "Collection",
PartitionTag: "Partition",
SegmentID: 1,
ChannelID: 1,
ChannelID: 0,
ProxyID: 1,
Timestamps: []Timestamp{1},
RowIDs: []int64{1},
@ -82,7 +82,7 @@ func getTsMsg(msgType MsgType, reqID UniqueID, hashValue int32) TsMsg {
ReqID: reqID,
ProxyID: 1,
Timestamp: 1,
ResultChannelID: 1,
ResultChannelID: 0,
}
searchMsg := &SearchMsg{
BaseMsg: baseMsg,
@ -97,7 +97,7 @@ func getTsMsg(msgType MsgType, reqID UniqueID, hashValue int32) TsMsg {
ProxyID: 1,
QueryNodeID: 1,
Timestamp: 1,
ResultChannelID: 1,
ResultChannelID: 0,
}
searchResultMsg := &SearchResultMsg{
BaseMsg: baseMsg,

View File

@ -14,7 +14,6 @@ enum MsgType {
kHasCollection = 102;
kDescribeCollection = 103;
kShowCollections = 104;
kGetSysConfigs = 105;
/* Definition Requests: partition */
kCreatePartition = 200;
@ -34,7 +33,6 @@ enum MsgType {
/* System Control */
kTimeTick = 1200;
kQueryNodeSegStats = 1201;
}
enum PeerRole {
@ -225,19 +223,6 @@ 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;
@ -246,7 +231,7 @@ message SearchResult {
int64 query_nodeID = 5;
uint64 timestamp = 6;
int64 result_channelID = 7;
repeated service.Hits hits = 8;
repeated bytes hits = 8;
}
message TimeTickMsg {
@ -281,4 +266,4 @@ message QueryNodeSegStats {
MsgType msg_type = 1;
int64 peerID = 2;
repeated SegmentStats seg_stats = 3;
}
}

View File

@ -32,7 +32,6 @@ 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
@ -57,7 +56,6 @@ var MsgType_name = map[int32]string{
102: "kHasCollection",
103: "kDescribeCollection",
104: "kShowCollections",
105: "kGetSysConfigs",
200: "kCreatePartition",
201: "kDropPartition",
202: "kHasPartition",
@ -78,7 +76,6 @@ var MsgType_value = map[string]int32{
"kHasCollection": 102,
"kDescribeCollection": 103,
"kShowCollections": 104,
"kGetSysConfigs": 105,
"kCreatePartition": 200,
"kDropPartition": 201,
"kHasPartition": 202,
@ -1582,106 +1579,25 @@ 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"`
ReqID int64 `protobuf:"varint,3,opt,name=reqID,proto3" json:"reqID,omitempty"`
ProxyID int64 `protobuf:"varint,4,opt,name=proxyID,proto3" json:"proxyID,omitempty"`
QueryNodeID int64 `protobuf:"varint,5,opt,name=query_nodeID,json=queryNodeID,proto3" json:"query_nodeID,omitempty"`
Timestamp uint64 `protobuf:"varint,6,opt,name=timestamp,proto3" json:"timestamp,omitempty"`
ResultChannelID int64 `protobuf:"varint,7,opt,name=result_channelID,json=resultChannelID,proto3" json:"result_channelID,omitempty"`
Hits []*servicepb.Hits `protobuf:"bytes,8,rep,name=hits,proto3" json:"hits,omitempty"`
XXX_NoUnkeyedLiteral struct{} `json:"-"`
XXX_unrecognized []byte `json:"-"`
XXX_sizecache int32 `json:"-"`
MsgType MsgType `protobuf:"varint,1,opt,name=msg_type,json=msgType,proto3,enum=milvus.proto.internal.MsgType" json:"msg_type,omitempty"`
Status *commonpb.Status `protobuf:"bytes,2,opt,name=status,proto3" json:"status,omitempty"`
ReqID int64 `protobuf:"varint,3,opt,name=reqID,proto3" json:"reqID,omitempty"`
ProxyID int64 `protobuf:"varint,4,opt,name=proxyID,proto3" json:"proxyID,omitempty"`
QueryNodeID int64 `protobuf:"varint,5,opt,name=query_nodeID,json=queryNodeID,proto3" json:"query_nodeID,omitempty"`
Timestamp uint64 `protobuf:"varint,6,opt,name=timestamp,proto3" json:"timestamp,omitempty"`
ResultChannelID int64 `protobuf:"varint,7,opt,name=result_channelID,json=resultChannelID,proto3" json:"result_channelID,omitempty"`
Hits [][]byte `protobuf:"bytes,8,rep,name=hits,proto3" json:"hits,omitempty"`
XXX_NoUnkeyedLiteral struct{} `json:"-"`
XXX_unrecognized []byte `json:"-"`
XXX_sizecache int32 `json:"-"`
}
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{22}
return fileDescriptor_7eb37f6b80b23116, []int{21}
}
func (m *SearchResult) XXX_Unmarshal(b []byte) error {
@ -1751,7 +1667,7 @@ func (m *SearchResult) GetResultChannelID() int64 {
return 0
}
func (m *SearchResult) GetHits() []*servicepb.Hits {
func (m *SearchResult) GetHits() [][]byte {
if m != nil {
return m.Hits
}
@ -1771,7 +1687,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{23}
return fileDescriptor_7eb37f6b80b23116, []int{22}
}
func (m *TimeTickMsg) XXX_Unmarshal(b []byte) error {
@ -1828,7 +1744,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{24}
return fileDescriptor_7eb37f6b80b23116, []int{23}
}
func (m *Key2Seg) XXX_Unmarshal(b []byte) error {
@ -1896,7 +1812,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{25}
return fileDescriptor_7eb37f6b80b23116, []int{24}
}
func (m *Key2SegMsg) XXX_Unmarshal(b []byte) error {
@ -1945,7 +1861,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{26}
return fileDescriptor_7eb37f6b80b23116, []int{25}
}
func (m *SegmentStats) XXX_Unmarshal(b []byte) error {
@ -2007,7 +1923,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{27}
return fileDescriptor_7eb37f6b80b23116, []int{26}
}
func (m *QueryNodeSegStats) XXX_Unmarshal(b []byte) error {
@ -2073,7 +1989,6 @@ 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")
@ -2085,98 +2000,94 @@ func init() {
func init() { proto.RegisterFile("internal_msg.proto", fileDescriptor_7eb37f6b80b23116) }
var fileDescriptor_7eb37f6b80b23116 = []byte{
// 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,
// 1416 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe4, 0x58, 0x4b, 0x6f, 0x1c, 0xc5,
0x13, 0x4f, 0xef, 0xac, 0xf7, 0x51, 0x6b, 0xaf, 0xc7, 0x6d, 0x3b, 0xd9, 0x24, 0x7f, 0x25, 0xce,
0xe4, 0x2f, 0x62, 0x82, 0xb0, 0x85, 0xc3, 0x81, 0xdc, 0x20, 0xde, 0x43, 0x96, 0xc8, 0x51, 0x18,
0x5b, 0x20, 0xa1, 0x48, 0xa3, 0xf1, 0x6e, 0x65, 0x77, 0x34, 0x4f, 0x77, 0xcf, 0xda, 0x59, 0x1f,
0x38, 0xe5, 0x03, 0xc0, 0x81, 0x03, 0x07, 0x24, 0x8e, 0x9c, 0x22, 0xf8, 0x16, 0xbc, 0xae, 0x1c,
0xf8, 0x0a, 0x20, 0x88, 0x04, 0xe1, 0x8e, 0xba, 0x7b, 0x1e, 0x3b, 0x7e, 0x46, 0x4a, 0x0c, 0x96,
0x7c, 0x9b, 0xaa, 0xe9, 0xe9, 0xaa, 0xfa, 0xfd, 0xaa, 0x6b, 0xaa, 0x1a, 0xa8, 0x13, 0xc4, 0xc8,
0x02, 0xdb, 0xb3, 0x7c, 0xde, 0x5f, 0x8a, 0x58, 0x18, 0x87, 0x74, 0xde, 0x77, 0xbc, 0xed, 0x21,
0x57, 0xd2, 0x52, 0xba, 0xe0, 0xd2, 0x64, 0x37, 0xf4, 0xfd, 0x30, 0x50, 0xea, 0x4b, 0x33, 0x1c,
0xd9, 0xb6, 0xd3, 0xc5, 0xfc, 0x3b, 0x23, 0x80, 0x7a, 0xa7, 0x6d, 0xe2, 0xd6, 0x10, 0x79, 0x4c,
0xcf, 0x43, 0x25, 0x42, 0x64, 0x9d, 0x76, 0x8b, 0x2c, 0x90, 0x45, 0xcd, 0x4c, 0x24, 0x7a, 0x0b,
0xca, 0x2c, 0xf4, 0xb0, 0x55, 0x5a, 0x20, 0x8b, 0xcd, 0x95, 0xab, 0x4b, 0x07, 0xda, 0x5a, 0x7a,
0x80, 0xc8, 0xcc, 0xd0, 0x43, 0x53, 0x2e, 0xa6, 0x73, 0x30, 0xd1, 0x0d, 0x87, 0x41, 0xdc, 0xd2,
0x16, 0xc8, 0xe2, 0x94, 0xa9, 0x04, 0xa3, 0x0f, 0x20, 0xec, 0xf1, 0x28, 0x0c, 0x38, 0xd2, 0x5b,
0x50, 0xe1, 0xb1, 0x1d, 0x0f, 0xb9, 0x34, 0xd8, 0x58, 0xb9, 0x5c, 0xdc, 0x3a, 0x71, 0x7e, 0x5d,
0x2e, 0x31, 0x93, 0xa5, 0xb4, 0x09, 0xa5, 0x4e, 0x5b, 0xfa, 0xa2, 0x99, 0xa5, 0x4e, 0xfb, 0x10,
0x43, 0x21, 0xc0, 0x06, 0x0f, 0xff, 0xc5, 0xc8, 0xb6, 0xa1, 0x21, 0x0d, 0xbe, 0x4c, 0x68, 0xff,
0x83, 0x7a, 0xec, 0xf8, 0xc8, 0x63, 0xdb, 0x8f, 0xa4, 0x4f, 0x65, 0x33, 0x57, 0x1c, 0x62, 0xf7,
0x09, 0x81, 0xc9, 0x75, 0xec, 0xe7, 0x2c, 0x66, 0xcb, 0xc8, 0xd8, 0x32, 0xb1, 0x75, 0x77, 0x60,
0x07, 0x01, 0x7a, 0x09, 0x78, 0x13, 0x66, 0xae, 0xa0, 0x97, 0xa1, 0xde, 0x0d, 0x3d, 0xcf, 0x0a,
0x6c, 0x1f, 0xe5, 0xf6, 0x75, 0xb3, 0x26, 0x14, 0xf7, 0x6d, 0x1f, 0xe9, 0x75, 0x98, 0x8a, 0x6c,
0x16, 0x3b, 0xb1, 0x13, 0x06, 0x56, 0x6c, 0xf7, 0x5b, 0x65, 0xb9, 0x60, 0x32, 0x53, 0x6e, 0xd8,
0x7d, 0xe3, 0x29, 0x01, 0xfa, 0x1e, 0xe7, 0x4e, 0x3f, 0x28, 0x38, 0xf3, 0x4a, 0x81, 0xbf, 0x07,
0xd3, 0x11, 0x32, 0x2b, 0x71, 0xdb, 0x62, 0xb8, 0xd5, 0xd2, 0x16, 0xb4, 0xc5, 0xc6, 0xca, 0xf5,
0x43, 0xbe, 0x1f, 0x77, 0xc5, 0x9c, 0x8a, 0x90, 0xad, 0xaa, 0x4f, 0x4d, 0xdc, 0x32, 0xbe, 0x24,
0x30, 0x2d, 0xdf, 0x2b, 0xaf, 0x7d, 0x0c, 0x24, 0x74, 0x5c, 0xa8, 0x12, 0x67, 0x95, 0x70, 0x0c,
0x74, 0x07, 0xb2, 0x52, 0x04, 0xb4, 0x7c, 0x1c, 0xa0, 0x13, 0x07, 0x00, 0xfa, 0x8c, 0xc0, 0x6c,
0x01, 0xd0, 0x93, 0x4b, 0xac, 0x1b, 0x30, 0x8d, 0x8f, 0x23, 0x87, 0xa1, 0xd5, 0x1b, 0x32, 0x5b,
0x38, 0x20, 0x83, 0x29, 0x9b, 0x4d, 0xa5, 0x6e, 0x27, 0x5a, 0xfa, 0x10, 0xce, 0x8f, 0x13, 0x60,
0x67, 0xc8, 0xb5, 0xca, 0x92, 0x87, 0xd7, 0x8e, 0xe2, 0x21, 0xc7, 0xd9, 0x9c, 0xcb, 0xa9, 0xc8,
0xb5, 0xc6, 0xcf, 0x04, 0x2e, 0xac, 0x32, 0xb4, 0x63, 0x5c, 0x0d, 0x3d, 0x0f, 0xbb, 0xc2, 0x64,
0x9a, 0x47, 0xb7, 0xa1, 0xe6, 0xf3, 0xbe, 0x15, 0x8f, 0x22, 0x94, 0x71, 0x37, 0x57, 0xae, 0x1c,
0x62, 0x6b, 0x8d, 0xf7, 0x37, 0x46, 0x11, 0x9a, 0x55, 0x5f, 0x3d, 0x08, 0x82, 0x18, 0x6e, 0x65,
0x25, 0x43, 0x09, 0x45, 0x44, 0xb4, 0xbd, 0x88, 0xb4, 0xa0, 0x1a, 0xb1, 0xf0, 0xf1, 0xa8, 0xd3,
0x96, 0xe4, 0x69, 0x66, 0x2a, 0xd2, 0xb7, 0xa0, 0xc2, 0xbb, 0x03, 0xf4, 0x6d, 0x49, 0x5a, 0x63,
0xe5, 0xe2, 0x81, 0xf0, 0xdf, 0xf1, 0xc2, 0x4d, 0x33, 0x59, 0x28, 0x98, 0x9c, 0x6f, 0xb3, 0x30,
0x3a, 0xc5, 0x51, 0xad, 0xc1, 0x74, 0x37, 0xf3, 0x4e, 0x25, 0xad, 0x0a, 0xef, 0xff, 0x45, 0x7f,
0x92, 0x1f, 0xc8, 0x52, 0x1e, 0x8a, 0x48, 0x68, 0xb3, 0xd9, 0x2d, 0xc8, 0xc6, 0x1f, 0x04, 0xe6,
0xee, 0xda, 0xfc, 0xec, 0x04, 0xfc, 0x17, 0x81, 0x8b, 0x6d, 0xe4, 0x5d, 0xe6, 0x6c, 0xe2, 0xd9,
0x89, 0xfa, 0x2b, 0x02, 0xf3, 0xeb, 0x83, 0x70, 0xe7, 0xf4, 0x46, 0x6c, 0xfc, 0x4e, 0xe0, 0xbc,
0xaa, 0x29, 0x0f, 0xd2, 0xe2, 0x7a, 0xea, 0x58, 0x79, 0x1f, 0x9a, 0xf9, 0xef, 0x60, 0x8c, 0x94,
0xeb, 0x07, 0x93, 0x92, 0x05, 0x22, 0x39, 0xc9, 0xff, 0x24, 0x92, 0x92, 0xdf, 0x08, 0xcc, 0x89,
0x5a, 0x73, 0x36, 0xa2, 0xfd, 0x95, 0xc0, 0xec, 0x5d, 0x9b, 0x9f, 0x8d, 0x60, 0x9f, 0x11, 0x68,
0xa5, 0x35, 0xe6, 0x6c, 0x44, 0x2c, 0x7e, 0x23, 0xa2, 0xbe, 0x9c, 0xde, 0x68, 0x5f, 0x71, 0x41,
0xfd, 0xb3, 0x04, 0x53, 0x9d, 0x80, 0x23, 0x8b, 0x4f, 0x2c, 0xd2, 0x1b, 0xfb, 0x3d, 0x56, 0xfd,
0xfe, 0x1e, 0x5f, 0x5e, 0xa8, 0xeb, 0x17, 0xb8, 0x71, 0xec, 0x8b, 0xee, 0xad, 0xd3, 0x96, 0x91,
0x6b, 0x66, 0xae, 0x28, 0x36, 0xce, 0x15, 0xf5, 0x36, 0x6f, 0x9c, 0xc7, 0x50, 0xad, 0x16, 0x51,
0xbd, 0x02, 0x90, 0x81, 0xcf, 0x5b, 0xb5, 0x05, 0x6d, 0xb1, 0x6c, 0x8e, 0x69, 0xc4, 0x50, 0xc1,
0xc2, 0x9d, 0x4e, 0x9b, 0xb7, 0xea, 0x0b, 0x9a, 0x18, 0x2a, 0x94, 0x44, 0xdf, 0x86, 0x1a, 0x0b,
0x77, 0xac, 0x9e, 0x1d, 0xdb, 0x2d, 0x90, 0x0d, 0xe9, 0x11, 0xdd, 0x59, 0x95, 0x85, 0x3b, 0x6d,
0x3b, 0xb6, 0x8d, 0x27, 0x25, 0x98, 0x6a, 0xa3, 0x87, 0x31, 0xfe, 0xf7, 0xa0, 0x17, 0x10, 0x2b,
0x1f, 0x81, 0xd8, 0xc4, 0x51, 0x88, 0x55, 0xf6, 0x21, 0x76, 0x0d, 0x26, 0x23, 0xe6, 0xf8, 0x36,
0x1b, 0x59, 0x2e, 0x8e, 0x78, 0xab, 0x2a, 0x71, 0x6b, 0x24, 0xba, 0x7b, 0x38, 0xe2, 0xc6, 0x73,
0x02, 0x53, 0xeb, 0x68, 0xb3, 0xee, 0xe0, 0xc4, 0x60, 0x18, 0xf3, 0x5f, 0x2b, 0xfa, 0x5f, 0x38,
0x7f, 0xe5, 0xbd, 0xe7, 0xef, 0x75, 0xd0, 0x19, 0xf2, 0xa1, 0x17, 0x5b, 0x39, 0x38, 0x0a, 0x80,
0x69, 0xa5, 0x5f, 0xcd, 0x20, 0x5a, 0x86, 0x89, 0xad, 0x21, 0xb2, 0x91, 0x4c, 0xb7, 0x23, 0xf9,
0x57, 0xeb, 0x8c, 0xa7, 0x25, 0x31, 0x3e, 0xab, 0xb0, 0xc5, 0x56, 0x2f, 0x13, 0x75, 0x3e, 0x9a,
0x95, 0x5e, 0x7c, 0x34, 0xcb, 0xa0, 0xd2, 0x0e, 0x81, 0x6a, 0x4f, 0xc9, 0xb9, 0x06, 0x93, 0xd2,
0x73, 0x2b, 0x08, 0x7b, 0x98, 0x01, 0xd1, 0x90, 0xba, 0xfb, 0x52, 0x55, 0x44, 0xb3, 0xf2, 0x22,
0x68, 0x56, 0x0f, 0x46, 0x93, 0x42, 0x79, 0xe0, 0xc4, 0xea, 0x08, 0x4e, 0x9a, 0xf2, 0xd9, 0xf8,
0x04, 0x1a, 0x1b, 0x8e, 0x8f, 0x1b, 0x4e, 0xd7, 0x5d, 0xe3, 0xfd, 0x97, 0x81, 0x2b, 0xbf, 0x1b,
0x28, 0x15, 0xee, 0x06, 0x8e, 0x2c, 0xc6, 0xc6, 0x17, 0x04, 0xaa, 0xf7, 0x70, 0xb4, 0xb2, 0x8e,
0x7d, 0x89, 0x9d, 0x38, 0xfa, 0xe9, 0xbc, 0x2e, 0x05, 0x7a, 0x15, 0x1a, 0x63, 0xc9, 0x9e, 0x6c,
0x0e, 0x79, 0xae, 0x1f, 0x53, 0xed, 0x2f, 0x42, 0xcd, 0xe1, 0xd6, 0xb6, 0xed, 0x39, 0x3d, 0x89,
0x7d, 0xcd, 0xac, 0x3a, 0xfc, 0x43, 0x21, 0x8a, 0x63, 0x96, 0x55, 0x37, 0xde, 0x9a, 0x90, 0x87,
0x68, 0x4c, 0x63, 0x3c, 0x04, 0x48, 0x5c, 0x13, 0xd0, 0x64, 0xcc, 0x92, 0x71, 0x66, 0xdf, 0x81,
0xaa, 0x8b, 0xa3, 0x15, 0x8e, 0xfd, 0x56, 0x49, 0xd6, 0xa8, 0xc3, 0xf0, 0x4a, 0x76, 0x32, 0xd3,
0xe5, 0xc6, 0xe7, 0xea, 0xa6, 0x47, 0x18, 0x13, 0x39, 0xc4, 0x8b, 0xd5, 0x97, 0xec, 0xad, 0xbe,
0x57, 0xa1, 0xe1, 0xa3, 0x1f, 0xb2, 0x91, 0xc5, 0x9d, 0x5d, 0x4c, 0x61, 0x50, 0xaa, 0x75, 0x67,
0x17, 0x45, 0xa0, 0xc1, 0xd0, 0xb7, 0x58, 0xb8, 0xc3, 0xd3, 0xf3, 0x18, 0x0c, 0x7d, 0x33, 0xdc,
0xe1, 0xf4, 0x0d, 0x98, 0x61, 0xd8, 0xc5, 0x20, 0xf6, 0x46, 0x96, 0x1f, 0xf6, 0x9c, 0x47, 0x0e,
0xa6, 0x60, 0xe8, 0xe9, 0x8b, 0xb5, 0x44, 0x6f, 0x7c, 0x4d, 0x60, 0xe6, 0x83, 0x34, 0xfd, 0xd6,
0xb1, 0xaf, 0x9c, 0x3b, 0x81, 0xc4, 0x78, 0x57, 0xc6, 0x6b, 0x89, 0x83, 0xc3, 0x8f, 0xbf, 0xf9,
0xc9, 0x70, 0x32, 0x6b, 0x3c, 0x71, 0xea, 0xe6, 0x2f, 0x25, 0xa8, 0x26, 0xe6, 0x68, 0x1d, 0x26,
0xdc, 0xfb, 0x61, 0x80, 0xfa, 0x39, 0x3a, 0x0f, 0x33, 0xee, 0xde, 0x9b, 0x07, 0xbd, 0x47, 0x67,
0x61, 0xda, 0x2d, 0x0e, 0xee, 0x3a, 0x52, 0x0a, 0x4d, 0xb7, 0x30, 0xdb, 0xea, 0x8f, 0xe8, 0x05,
0x98, 0x75, 0xf7, 0x8f, 0x7f, 0xba, 0x48, 0x01, 0xdd, 0x2d, 0x4e, 0x48, 0x5c, 0x1f, 0xd0, 0x79,
0xd0, 0xdd, 0x3d, 0x43, 0x89, 0xfe, 0x1d, 0xa1, 0xb3, 0xd0, 0x74, 0x0b, 0xbd, 0xbb, 0xfe, 0x3d,
0xa1, 0x14, 0xa6, 0xdc, 0xf1, 0x16, 0x57, 0xff, 0x81, 0xd0, 0x0b, 0x40, 0xdd, 0x7d, 0x9d, 0xa0,
0xfe, 0x23, 0xa1, 0x73, 0x30, 0xed, 0x16, 0x1a, 0x26, 0xae, 0xff, 0x44, 0xe8, 0x24, 0x54, 0x5d,
0xd5, 0x55, 0xe8, 0x9f, 0x6a, 0x52, 0x52, 0xbf, 0x3b, 0xfd, 0x33, 0x25, 0xa9, 0xf2, 0xa7, 0x3f,
0xd7, 0xa4, 0xb1, 0xf1, 0x62, 0xa8, 0xff, 0xad, 0xd1, 0x26, 0xd4, 0xdd, 0xf4, 0xc0, 0xeb, 0xdf,
0xd4, 0xa5, 0xf1, 0x7d, 0x6c, 0xeb, 0xdf, 0xd6, 0x6f, 0xde, 0x86, 0x5a, 0x7a, 0x5f, 0x47, 0x01,
0x2a, 0x6b, 0x36, 0x8f, 0x91, 0xe9, 0xe7, 0xc4, 0xb3, 0x89, 0x76, 0x0f, 0x99, 0x4e, 0xc4, 0xf3,
0x47, 0xcc, 0x11, 0xfa, 0x92, 0xc0, 0xff, 0x81, 0xa8, 0x69, 0xba, 0x76, 0xa7, 0xfd, 0xf1, 0x9d,
0xbe, 0x13, 0x0f, 0x86, 0x9b, 0xa2, 0x46, 0x2e, 0xef, 0x3a, 0x9e, 0xe7, 0xec, 0xc6, 0xd8, 0x1d,
0x2c, 0x2b, 0x72, 0xdf, 0xec, 0x39, 0x3c, 0x66, 0xce, 0xe6, 0x30, 0xc6, 0xde, 0x72, 0x4a, 0xf1,
0xb2, 0x64, 0x3c, 0x13, 0xa3, 0xcd, 0xcd, 0x8a, 0xd4, 0xdc, 0xfa, 0x27, 0x00, 0x00, 0xff, 0xff,
0x56, 0x57, 0x32, 0x28, 0x20, 0x17, 0x00, 0x00,
}

View File

@ -89,15 +89,6 @@ 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) {}

View File

@ -30,38 +30,36 @@ const _ = proto.ProtoPackageIsVersion3 // please upgrade the proto package
func init() { proto.RegisterFile("master.proto", fileDescriptor_f9c348dec43a6705) }
var fileDescriptor_f9c348dec43a6705 = []byte{
// 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,
// 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,
}
// Reference imports to suppress errors if they are not otherwise used.
@ -136,13 +134,6 @@ 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)
@ -246,15 +237,6 @@ 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...)
@ -344,13 +326,6 @@ 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)
@ -390,9 +365,6 @@ 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")
}
@ -587,24 +559,6 @@ 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 {
@ -703,10 +657,6 @@ var _Master_serviceDesc = grpc.ServiceDesc{
MethodName: "ShowPartitions",
Handler: _Master_ShowPartitions_Handler,
},
{
MethodName: "GetSysConfigs",
Handler: _Master_GetSysConfigs_Handler,
},
{
MethodName: "AllocTimestamp",
Handler: _Master_AllocTimestamp_Handler,

View File

@ -135,33 +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.
* The default value of tag is "root".
* It corresponds to the final score of each hit.
*/
message Score {
string tag = 1;
repeated float values = 2;
}
/**
* @brief Entities hit by query
*/
message Hits {
repeated int64 IDs = 1;
repeated common.Blob row_data = 2;
repeated Score scores = 3;
repeated bytes row_data = 2;
repeated float scores = 3;
}
@ -170,6 +151,6 @@ message Hits {
*/
message QueryResult {
common.Status status = 1;
repeated Hits hits = 2;
repeated bytes hits = 2;
}

View File

@ -738,129 +738,21 @@ func (m *PartitionDescription) GetStatistics() []*commonpb.KeyValuePair {
}
//*
// @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".
// It corresponds to the final score of each hit.
type Score struct {
Tag string `protobuf:"bytes,1,opt,name=tag,proto3" json:"tag,omitempty"`
Values []float32 `protobuf:"fixed32,2,rep,packed,name=values,proto3" json:"values,omitempty"`
// @brief Entities hit by query
type Hits struct {
IDs []int64 `protobuf:"varint,1,rep,packed,name=IDs,proto3" json:"IDs,omitempty"`
RowData [][]byte `protobuf:"bytes,2,rep,name=row_data,json=rowData,proto3" json:"row_data,omitempty"`
Scores []float32 `protobuf:"fixed32,3,rep,packed,name=scores,proto3" json:"scores,omitempty"`
XXX_NoUnkeyedLiteral struct{} `json:"-"`
XXX_unrecognized []byte `json:"-"`
XXX_sizecache int32 `json:"-"`
}
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{14}
}
func (m *Score) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_Score.Unmarshal(m, b)
}
func (m *Score) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
return xxx_messageInfo_Score.Marshal(b, m, deterministic)
}
func (m *Score) XXX_Merge(src proto.Message) {
xxx_messageInfo_Score.Merge(m, src)
}
func (m *Score) XXX_Size() int {
return xxx_messageInfo_Score.Size(m)
}
func (m *Score) XXX_DiscardUnknown() {
xxx_messageInfo_Score.DiscardUnknown(m)
}
var xxx_messageInfo_Score proto.InternalMessageInfo
func (m *Score) GetTag() string {
if m != nil {
return m.Tag
}
return ""
}
func (m *Score) GetValues() []float32 {
if m != nil {
return m.Values
}
return nil
}
//*
// @brief Entities hit by query
type Hits struct {
IDs []int64 `protobuf:"varint,1,rep,packed,name=IDs,proto3" json:"IDs,omitempty"`
RowData []*commonpb.Blob `protobuf:"bytes,2,rep,name=row_data,json=rowData,proto3" json:"row_data,omitempty"`
Scores []*Score `protobuf:"bytes,3,rep,name=scores,proto3" json:"scores,omitempty"`
XXX_NoUnkeyedLiteral struct{} `json:"-"`
XXX_unrecognized []byte `json:"-"`
XXX_sizecache int32 `json:"-"`
}
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{15}
return fileDescriptor_b4b40b84dd2f74cb, []int{13}
}
func (m *Hits) XXX_Unmarshal(b []byte) error {
@ -888,14 +780,14 @@ func (m *Hits) GetIDs() []int64 {
return nil
}
func (m *Hits) GetRowData() []*commonpb.Blob {
func (m *Hits) GetRowData() [][]byte {
if m != nil {
return m.RowData
}
return nil
}
func (m *Hits) GetScores() []*Score {
func (m *Hits) GetScores() []float32 {
if m != nil {
return m.Scores
}
@ -906,7 +798,7 @@ func (m *Hits) GetScores() []*Score {
// @brief Query result
type QueryResult struct {
Status *commonpb.Status `protobuf:"bytes,1,opt,name=status,proto3" json:"status,omitempty"`
Hits []*Hits `protobuf:"bytes,2,rep,name=hits,proto3" json:"hits,omitempty"`
Hits [][]byte `protobuf:"bytes,2,rep,name=hits,proto3" json:"hits,omitempty"`
XXX_NoUnkeyedLiteral struct{} `json:"-"`
XXX_unrecognized []byte `json:"-"`
XXX_sizecache int32 `json:"-"`
@ -916,7 +808,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{16}
return fileDescriptor_b4b40b84dd2f74cb, []int{14}
}
func (m *QueryResult) XXX_Unmarshal(b []byte) error {
@ -944,7 +836,7 @@ func (m *QueryResult) GetStatus() *commonpb.Status {
return nil
}
func (m *QueryResult) GetHits() []*Hits {
func (m *QueryResult) GetHits() [][]byte {
if m != nil {
return m.Hits
}
@ -966,8 +858,6 @@ 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")
}
@ -975,55 +865,52 @@ func init() {
func init() { proto.RegisterFile("service_msg.proto", fileDescriptor_b4b40b84dd2f74cb) }
var fileDescriptor_b4b40b84dd2f74cb = []byte{
// 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,
// 739 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xac, 0x54, 0xdd, 0x4e, 0xdb, 0x4a,
0x10, 0x3e, 0x8e, 0x43, 0x4e, 0x98, 0x38, 0x21, 0xec, 0xc9, 0x41, 0x06, 0x6e, 0x72, 0x8c, 0x38,
0x8d, 0x5a, 0x35, 0x91, 0xa0, 0x52, 0xc5, 0x45, 0xa5, 0x26, 0x40, 0x5b, 0x7e, 0x14, 0xe8, 0x12,
0x21, 0xd1, 0x4a, 0x8d, 0x36, 0xf6, 0xca, 0x5e, 0xd5, 0xf1, 0x5a, 0xde, 0x35, 0x51, 0x78, 0x90,
0xbe, 0x44, 0x1f, 0xa4, 0x77, 0x7d, 0xa6, 0xca, 0x6b, 0x93, 0x1f, 0x4a, 0x55, 0x0a, 0xdc, 0xcd,
0xcc, 0xee, 0xcc, 0x37, 0xbf, 0x1f, 0x2c, 0x0b, 0x1a, 0x5d, 0x32, 0x9b, 0xf6, 0x87, 0xc2, 0x6d,
0x86, 0x11, 0x97, 0x1c, 0xd5, 0x86, 0xcc, 0xbf, 0x8c, 0x45, 0xaa, 0x35, 0xb3, 0xf7, 0x35, 0xc3,
0xe6, 0xc3, 0x21, 0x0f, 0x52, 0xeb, 0x9a, 0x21, 0x6c, 0x8f, 0x0e, 0x49, 0xaa, 0x59, 0x3b, 0x50,
0xd9, 0xe5, 0xbe, 0x4f, 0x6d, 0xc9, 0x78, 0xd0, 0x25, 0x43, 0x8a, 0x9e, 0xc0, 0x92, 0x3d, 0xb1,
0xf4, 0x03, 0x32, 0xa4, 0xa6, 0x56, 0xd7, 0x1a, 0x8b, 0xb8, 0x62, 0xcf, 0x7d, 0xb4, 0x0e, 0xa1,
0x7c, 0x4a, 0x22, 0xc9, 0xfe, 0xd8, 0x13, 0x55, 0x41, 0x97, 0xc4, 0x35, 0x73, 0xea, 0x31, 0x11,
0xad, 0xaf, 0x1a, 0x14, 0x31, 0x1f, 0x75, 0x88, 0xb4, 0xbd, 0xbb, 0xc7, 0xd9, 0x80, 0x72, 0x78,
0x9d, 0x41, 0x7f, 0x1a, 0xd1, 0x98, 0x18, 0x7b, 0xc4, 0x45, 0x2f, 0xa0, 0x18, 0xf1, 0x51, 0xdf,
0x21, 0x92, 0x98, 0x7a, 0x5d, 0x6f, 0x94, 0xb6, 0x56, 0x9b, 0x73, 0x6d, 0xca, 0xba, 0xd3, 0xf1,
0xf9, 0x00, 0xff, 0x1d, 0xf1, 0xd1, 0x1e, 0x91, 0x04, 0xad, 0xc3, 0xa2, 0x47, 0x84, 0xd7, 0xff,
0x4c, 0xc7, 0xc2, 0xcc, 0xd7, 0xf5, 0xc6, 0x02, 0x2e, 0x26, 0x86, 0x23, 0x3a, 0x16, 0xd6, 0x08,
0xaa, 0xa7, 0x3e, 0xb1, 0xa9, 0xc7, 0x7d, 0x87, 0x46, 0xe7, 0xc4, 0x8f, 0x27, 0x35, 0x69, 0x93,
0x9a, 0xd0, 0x0e, 0xe4, 0xe5, 0x38, 0xa4, 0x2a, 0xa9, 0xca, 0xd6, 0x66, 0xf3, 0xb6, 0xd9, 0x34,
0x67, 0xe2, 0xf4, 0xc6, 0x21, 0xc5, 0xca, 0x05, 0xad, 0x40, 0xe1, 0x32, 0x89, 0x2a, 0x54, 0xc6,
0x06, 0xce, 0x34, 0xeb, 0xd3, 0x1c, 0xf0, 0xdb, 0x88, 0xc7, 0x21, 0x3a, 0x04, 0x23, 0x9c, 0xda,
0x84, 0xa9, 0xa9, 0x1a, 0xff, 0xff, 0x2d, 0x9c, 0x4a, 0x1b, 0xcf, 0xf9, 0x5a, 0x5f, 0x34, 0x58,
0x78, 0x1f, 0xd3, 0x68, 0x7c, 0xf7, 0x19, 0x6c, 0x42, 0x65, 0x6e, 0x06, 0xc2, 0xcc, 0xd5, 0xf5,
0xc6, 0x22, 0x2e, 0xcf, 0x0e, 0x41, 0x24, 0xed, 0x71, 0x84, 0x6f, 0xea, 0x69, 0x7b, 0x1c, 0xe1,
0xa3, 0x67, 0xb0, 0x3c, 0x83, 0xdd, 0x77, 0x93, 0x62, 0xcc, 0x7c, 0x5d, 0x6b, 0x18, 0xb8, 0x1a,
0xde, 0x28, 0xd2, 0xfa, 0x08, 0x95, 0x33, 0x19, 0xb1, 0xc0, 0xc5, 0x54, 0x84, 0x3c, 0x10, 0x14,
0x6d, 0x43, 0x41, 0x48, 0x22, 0x63, 0xa1, 0xf2, 0x2a, 0x6d, 0xad, 0xdf, 0x3a, 0xd4, 0x33, 0xf5,
0x05, 0x67, 0x5f, 0x51, 0x0d, 0x16, 0x54, 0x27, 0xb3, 0x45, 0x49, 0x15, 0xeb, 0x02, 0x8c, 0x0e,
0xe7, 0xfe, 0x23, 0x86, 0x2e, 0x5e, 0x87, 0x26, 0x80, 0xd2, 0xbc, 0x8f, 0x99, 0x90, 0x0f, 0x03,
0x98, 0xee, 0x44, 0xda, 0xe0, 0xeb, 0x9d, 0x18, 0xc0, 0x3f, 0x07, 0x81, 0xa4, 0x2e, 0x8d, 0x1e,
0x1b, 0x43, 0x9f, 0x60, 0x08, 0xa8, 0x65, 0x18, 0x98, 0x04, 0x2e, 0x7d, 0x70, 0xa7, 0x06, 0xd4,
0x65, 0x81, 0xea, 0x94, 0x8e, 0x53, 0x25, 0x59, 0x10, 0x1a, 0x38, 0x6a, 0x41, 0x74, 0x9c, 0x88,
0xd6, 0x77, 0x0d, 0xfe, 0x9d, 0x72, 0xd3, 0x1e, 0x15, 0x76, 0xc4, 0xc2, 0x44, 0xbc, 0x1f, 0xec,
0x2b, 0x28, 0xa4, 0xcc, 0xa7, 0x70, 0x4b, 0x3f, 0x1d, 0x64, 0xca, 0x8a, 0x53, 0xc0, 0x33, 0x65,
0xc0, 0x99, 0x13, 0x6a, 0x03, 0x24, 0x81, 0x98, 0x90, 0xcc, 0x16, 0x19, 0x91, 0xfc, 0x77, 0x2b,
0xee, 0x11, 0x1d, 0xab, 0xdb, 0x3a, 0x25, 0x2c, 0xc2, 0x33, 0x4e, 0xd6, 0x37, 0x0d, 0x6a, 0x13,
0xc6, 0x7c, 0x70, 0x3d, 0x2f, 0x21, 0xaf, 0xce, 0x32, 0xad, 0x66, 0xe3, 0x17, 0xf7, 0x3e, 0x4b,
0xd0, 0x58, 0x39, 0x3c, 0x46, 0x25, 0x47, 0x90, 0x7f, 0xc7, 0xa4, 0xba, 0xea, 0x83, 0xbd, 0x94,
0x72, 0x74, 0x9c, 0x88, 0x68, 0x75, 0x86, 0x6d, 0x73, 0x8a, 0xbb, 0x26, 0x94, 0xba, 0x92, 0x0c,
0x80, 0x47, 0x19, 0xa9, 0xe5, 0x70, 0xa6, 0x59, 0xe7, 0x50, 0x52, 0x9c, 0x83, 0xa9, 0x88, 0x7d,
0x79, 0xbf, 0x66, 0x20, 0xc8, 0x7b, 0x4c, 0x8a, 0x0c, 0x52, 0xc9, 0x4f, 0x5f, 0xc3, 0xd2, 0x0d,
0x76, 0x45, 0x45, 0xc8, 0x77, 0x4f, 0xba, 0xfb, 0xd5, 0xbf, 0xd0, 0x32, 0x94, 0xcf, 0xf7, 0x77,
0x7b, 0x27, 0xb8, 0xdf, 0x39, 0xe8, 0xb6, 0xf1, 0x45, 0xd5, 0x41, 0x55, 0x30, 0x32, 0xd3, 0x9b,
0xe3, 0x93, 0x76, 0xaf, 0x4a, 0x3b, 0xbb, 0x1f, 0xda, 0x2e, 0x93, 0x5e, 0x3c, 0x48, 0x50, 0x5b,
0x57, 0xcc, 0xf7, 0xd9, 0x95, 0xa4, 0xb6, 0xd7, 0x4a, 0x33, 0x7a, 0xee, 0x30, 0x21, 0x23, 0x36,
0x88, 0x25, 0x75, 0x5a, 0x2c, 0x90, 0x34, 0x0a, 0x88, 0xdf, 0x52, 0x69, 0xb6, 0xb2, 0x01, 0x84,
0x83, 0x41, 0x41, 0x19, 0xb6, 0x7f, 0x04, 0x00, 0x00, 0xff, 0xff, 0x33, 0xc8, 0x08, 0xe2, 0xaf,
0x07, 0x00, 0x00,
}

View File

@ -36,6 +36,9 @@ 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)
@ -82,9 +85,8 @@ 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()
@ -125,6 +127,7 @@ 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)

View File

@ -31,6 +31,10 @@ 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 == "" {
@ -96,6 +100,27 @@ 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 {
@ -322,3 +347,143 @@ 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
}

View File

@ -69,3 +69,8 @@ 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)
}

View File

@ -55,12 +55,11 @@ 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)
@ -84,7 +83,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)
}

View File

@ -229,7 +229,7 @@ func TestProxy_Insert(t *testing.T) {
collectionName := "CreateCollection" + strconv.FormatInt(int64(i), 10)
req := &servicepb.RowBatch{
CollectionName: collectionName,
PartitionTag: "",
PartitionTag: "haha",
RowData: make([]*commonpb.Blob, 0),
HashKeys: make([]int32, 0),
}
@ -237,6 +237,7 @@ 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)

View File

@ -1,6 +1,9 @@
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"
@ -15,6 +18,9 @@ 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"))
@ -23,8 +29,8 @@ func insertRepackFunc(tsMsgs []msgstream.TsMsg,
if !ok {
return nil, errors.New(string("msg's must be Insert"))
}
keys := hashKeys[i]
keys := hashKeys[i]
timestampLen := len(insertRequest.Timestamps)
rowIDLen := len(insertRequest.RowIDs)
rowDataLen := len(insertRequest.RowData)
@ -34,10 +40,84 @@ 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]
@ -48,13 +128,14 @@ 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: 0, // will be assigned later if together
ChannelID: channelID,
SegmentID: segmentID,
ChannelID: int64(key),
ProxyID: proxyID,
Timestamps: []uint64{ts},
RowIDs: []int64{rowID},
@ -73,25 +154,10 @@ 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
}

View File

@ -4,12 +4,17 @@ import (
"context"
"errors"
"log"
"math"
"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"
)
@ -32,7 +37,6 @@ type BaseInsertTask = msgstream.InsertMsg
type InsertTask struct {
BaseInsertTask
Condition
ts Timestamp
result *servicepb.IntegerRangeResponse
manipulationMsgStream *msgstream.PulsarMsgStream
ctx context.Context
@ -44,15 +48,21 @@ func (it *InsertTask) SetID(uid UniqueID) {
}
func (it *InsertTask) SetTs(ts Timestamp) {
it.ts = ts
rowNum := len(it.RowData)
it.Timestamps = make([]uint64, rowNum)
for index := range it.Timestamps {
it.Timestamps[index] = ts
}
it.BeginTimestamp = ts
it.EndTimestamp = ts
}
func (it *InsertTask) BeginTs() Timestamp {
return it.ts
return it.BeginTimestamp
}
func (it *InsertTask) EndTs() Timestamp {
return it.ts
return it.EndTimestamp
}
func (it *InsertTask) ID() UniqueID {
@ -64,6 +74,15 @@ 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
}
@ -120,6 +139,7 @@ type CreateCollectionTask struct {
masterClient masterpb.MasterClient
result *commonpb.Status
ctx context.Context
schema *schemapb.CollectionSchema
}
func (cct *CreateCollectionTask) ID() UniqueID {
@ -147,10 +167,28 @@ 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)
@ -201,6 +239,9 @@ func (dct *DropCollectionTask) SetTs(ts Timestamp) {
}
func (dct *DropCollectionTask) PreExecute() error {
if err := ValidateCollectionName(dct.CollectionName.CollectionName); err != nil {
return err
}
return nil
}
@ -229,6 +270,7 @@ type QueryTask struct {
resultBuf chan []*internalpb.SearchResult
result *servicepb.QueryResult
ctx context.Context
query *servicepb.Query
}
func (qt *QueryTask) ID() UniqueID {
@ -256,6 +298,15 @@ 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
}
@ -294,7 +345,17 @@ func (qt *QueryTask) PostExecute() error {
qt.result = &servicepb.QueryResult{}
return nil
}
k := len(searchResults[0].Hits[0].IDs) // k
var hits [][]*servicepb.Hits = make([][]*servicepb.Hits, rlen)
for i, sr := range searchResults {
hits[i] = make([]*servicepb.Hits, n)
for j, bs := range sr.Hits {
err := proto.Unmarshal(bs, hits[i][j])
if err != nil {
return err
}
}
}
k := len(hits[0][0].IDs)
queryResult := &servicepb.QueryResult{
Status: &commonpb.Status{
ErrorCode: 0,
@ -307,26 +368,27 @@ func (qt *QueryTask) PostExecute() error {
// len(queryResult.Hits[i].Ids) == k for i in range(n)
for i := 0; i < n; n++ { // n
locs := make([]int, rlen)
hits := &servicepb.Hits{}
reducedHits := &servicepb.Hits{}
for j := 0; j < k; j++ { // k
choice, maxScore := 0, float32(0)
choice, minDistance := 0, float32(math.MaxFloat32)
for q, loc := range locs { // query num, the number of ways to merge
score := func(score *servicepb.Score) float32 {
// TODO: get score of root
return 0.0
}(searchResults[q].Hits[i].Scores[loc])
if score > maxScore {
distance := hits[q][i].Scores[loc]
if distance < minDistance {
choice = q
maxScore = score
minDistance = distance
}
}
choiceOffset := locs[choice]
hits.IDs = append(hits.IDs, searchResults[choice].Hits[i].IDs[choiceOffset])
hits.RowData = append(hits.RowData, searchResults[choice].Hits[i].RowData[choiceOffset])
hits.Scores = append(hits.Scores, searchResults[choice].Hits[i].Scores[choiceOffset])
reducedHits.IDs = append(reducedHits.IDs, hits[choice][i].IDs[choiceOffset])
reducedHits.RowData = append(reducedHits.RowData, hits[choice][i].RowData[choiceOffset])
reducedHits.Scores = append(reducedHits.Scores, hits[choice][i].Scores[choiceOffset])
locs[choice]++
}
queryResult.Hits = append(queryResult.Hits, hits)
reducedHitsBs, err := proto.Marshal(reducedHits)
if err != nil {
return err
}
queryResult.Hits = append(queryResult.Hits, reducedHitsBs)
}
qt.result = queryResult
}
@ -367,6 +429,9 @@ func (hct *HasCollectionTask) SetTs(ts Timestamp) {
}
func (hct *HasCollectionTask) PreExecute() error {
if err := ValidateCollectionName(hct.CollectionName.CollectionName); err != nil {
return err
}
return nil
}
@ -424,6 +489,9 @@ func (dct *DescribeCollectionTask) SetTs(ts Timestamp) {
}
func (dct *DescribeCollectionTask) PreExecute() error {
if err := ValidateCollectionName(dct.CollectionName.CollectionName); err != nil {
return err
}
return nil
}
@ -532,6 +600,16 @@ 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
}
@ -577,6 +655,16 @@ 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
}
@ -622,6 +710,15 @@ 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
}
@ -667,6 +764,15 @@ 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
}
@ -712,6 +818,9 @@ func (spt *ShowPartitionsTask) SetTs(ts Timestamp) {
}
func (spt *ShowPartitionsTask) PreExecute() error {
if err := ValidateCollectionName(spt.CollectionName.CollectionName); err != nil {
return err
}
return nil
}

View File

@ -186,16 +186,7 @@ type DqTaskQueue struct {
func (queue *DdTaskQueue) Enqueue(t task) error {
queue.lock.Lock()
defer queue.lock.Unlock()
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)
return queue.BaseTaskQueue.Enqueue(t)
}
func NewDdTaskQueue(sched *TaskScheduler) *DdTaskQueue {
@ -369,14 +360,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()
@ -401,8 +392,7 @@ func (sched *TaskScheduler) queryResultLoop() {
queryResultBuf[reqID] = make([]*internalpb.SearchResult, 0)
}
queryResultBuf[reqID] = append(queryResultBuf[reqID], &searchResultMsg.SearchResult)
if len(queryResultBuf[reqID]) == 4 {
// TODO: use the number of query node instead
if len(queryResultBuf[reqID]) == queryNodeNum {
t := sched.getTaskByReqID(reqID)
if t != nil {
qt, ok := t.(*QueryTask)

View File

@ -0,0 +1,118 @@
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
}

View File

@ -0,0 +1,84 @@
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))
}
}

View File

@ -1,4 +1,4 @@
package reader
package querynode
/*

View File

@ -1,4 +1,4 @@
package reader
package querynode
/*

View File

@ -1,4 +1,4 @@
package reader
package querynode
import (
"context"

View File

@ -1,4 +1,4 @@
package reader
package querynode
import (
"context"

View File

@ -1,4 +1,4 @@
package reader
package querynode
import (
"context"

View File

@ -1,4 +1,4 @@
package reader
package querynode
import (
"context"
@ -175,7 +175,7 @@ func TestDataSyncService_Start(t *testing.T) {
// pulsar produce
const receiveBufSize = 1024
producerChannels := []string{"insert"}
producerChannels := Params.insertChannelNames()
insertStream := msgstream.NewPulsarMsgStream(ctx, receiveBufSize)
insertStream.SetPulsarClient(pulsarURL)

View File

@ -1,4 +1,4 @@
package reader
package querynode
type deleteNode struct {
BaseNode

View File

@ -1,4 +1,4 @@
package reader
package querynode
import (
"log"
@ -29,8 +29,6 @@ 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{

View File

@ -1,4 +1,4 @@
package reader
package querynode
import (
"fmt"
@ -106,6 +106,7 @@ 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
}
@ -116,8 +117,9 @@ func (iNode *insertNode) insert(insertData *InsertData, segmentID int64, wg *syn
err = targetSegment.segmentInsert(offsets, &ids, &timestamps, &records)
if err != nil {
log.Println("insert failed")
log.Println(err)
// TODO: add error handling
wg.Done()
return
}

View File

@ -1,4 +1,4 @@
package reader
package querynode
type key2SegNode struct {
BaseNode

View File

@ -1,4 +1,4 @@
package reader
package querynode
import (
"github.com/zilliztech/milvus-distributed/internal/msgstream"

View File

@ -1,4 +1,4 @@
package reader
package querynode
import (
"context"
@ -9,16 +9,16 @@ import (
)
func newDmInputNode(ctx context.Context) *flowgraph.InputNode {
receiveBufSize := Params.dmReceiveBufSize()
pulsarBufSize := Params.dmPulsarBufSize()
receiveBufSize := Params.insertReceiveBufSize()
pulsarBufSize := Params.insertPulsarBufSize()
msgStreamURL, err := Params.pulsarAddress()
if err != nil {
log.Fatal(err)
}
consumeChannels := []string{"insert"}
consumeSubName := "insertSub"
consumeChannels := Params.insertChannelNames()
consumeSubName := Params.msgChannelSubName()
insertStream := msgstream.NewPulsarTtMsgStream(ctx, receiveBufSize)
insertStream.SetPulsarClient(msgStreamURL)

View File

@ -1,4 +1,4 @@
package reader
package querynode
import "github.com/zilliztech/milvus-distributed/internal/util/flowgraph"

View File

@ -1,4 +1,4 @@
package reader
package querynode
type schemaUpdateNode struct {
BaseNode

View File

@ -1,4 +1,4 @@
package reader
package querynode
import (
"log"

View File

@ -1,4 +1,4 @@
package reader
package querynode
/*

View File

@ -1,4 +1,4 @@
package reader
package querynode
import (
"context"
@ -111,8 +111,8 @@ func isSegmentChannelRangeInQueryNodeChannelRange(segment *etcdpb.SegmentMeta) b
}
Params.Init()
var queryNodeChannelStart = Params.topicStart()
var queryNodeChannelEnd = Params.topicEnd()
var queryNodeChannelStart = Params.insertChannelRange()[0]
var queryNodeChannelEnd = Params.insertChannelRange()[1]
if segment.ChannelStart >= int32(queryNodeChannelStart) && segment.ChannelEnd <= int32(queryNodeChannelEnd) {
return true
@ -167,6 +167,7 @@ func (mService *metaService) processSegmentCreate(id string, value string) {
seg := mService.segmentUnmarshal(value)
if !isSegmentChannelRangeInQueryNodeChannelRange(seg) {
log.Println("Illegal segment channel range")
return
}

View File

@ -1,4 +1,4 @@
package reader
package querynode
import (
"context"
@ -93,7 +93,7 @@ func TestMetaService_isSegmentChannelRangeInQueryNodeChannelRange(t *testing.T)
CollectionID: UniqueID(0),
PartitionTag: "partition0",
ChannelStart: 0,
ChannelEnd: 128,
ChannelEnd: 1,
OpenTime: Timestamp(0),
CloseTime: Timestamp(math.MaxUint64),
NumRows: UniqueID(0),
@ -264,10 +264,9 @@ func TestMetaService_processSegmentCreate(t *testing.T) {
PartitionTags: []string{"default"},
}
colMetaBlob, err := proto.Marshal(&collectionMeta)
assert.NoError(t, err)
colMetaBlob := proto.MarshalTextString(&collectionMeta)
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")
@ -276,7 +275,7 @@ func TestMetaService_processSegmentCreate(t *testing.T) {
id := "0"
value := `partition_tag: "default"
channel_start: 0
channel_end: 128
channel_end: 1
close_time: 18446744073709551615
`
@ -331,7 +330,7 @@ func TestMetaService_processCreate(t *testing.T) {
key2 := "by-dev/segment/0"
msg2 := `partition_tag: "default"
channel_start: 0
channel_end: 128
channel_end: 1
close_time: 18446744073709551615
`
@ -388,10 +387,9 @@ func TestMetaService_processSegmentModify(t *testing.T) {
PartitionTags: []string{"default"},
}
colMetaBlob, err := proto.Marshal(&collectionMeta)
assert.NoError(t, err)
colMetaBlob := proto.MarshalTextString(&collectionMeta)
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")
@ -400,7 +398,7 @@ func TestMetaService_processSegmentModify(t *testing.T) {
id := "0"
value := `partition_tag: "default"
channel_start: 0
channel_end: 128
channel_end: 1
close_time: 18446744073709551615
`
@ -411,7 +409,7 @@ func TestMetaService_processSegmentModify(t *testing.T) {
newValue := `partition_tag: "default"
channel_start: 0
channel_end: 128
channel_end: 1
close_time: 18446744073709551615
`
@ -581,7 +579,7 @@ func TestMetaService_processModify(t *testing.T) {
key2 := "by-dev/segment/0"
msg2 := `partition_tag: "p1"
channel_start: 0
channel_end: 128
channel_end: 1
close_time: 18446744073709551615
`
@ -637,7 +635,7 @@ func TestMetaService_processModify(t *testing.T) {
msg4 := `partition_tag: "p1"
channel_start: 0
channel_end: 128
channel_end: 1
close_time: 18446744073709551615
`
@ -694,10 +692,9 @@ func TestMetaService_processSegmentDelete(t *testing.T) {
PartitionTags: []string{"default"},
}
colMetaBlob, err := proto.Marshal(&collectionMeta)
assert.NoError(t, err)
colMetaBlob := proto.MarshalTextString(&collectionMeta)
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")
@ -706,7 +703,7 @@ func TestMetaService_processSegmentDelete(t *testing.T) {
id := "0"
value := `partition_tag: "default"
channel_start: 0
channel_end: 128
channel_end: 1
close_time: 18446744073709551615
`
@ -810,7 +807,7 @@ func TestMetaService_processDelete(t *testing.T) {
key2 := "by-dev/segment/0"
msg2 := `partition_tag: "default"
channel_start: 0
channel_end: 128
channel_end: 1
close_time: 18446744073709551615
`

View File

@ -0,0 +1,345 @@
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("milvus.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) gracefulTime() int64 {
gracefulTime, err := p.Load("queryNode.gracefulTime")
if err != nil {
panic(err)
}
time, err := strconv.Atoi(gracefulTime)
if err != nil {
panic(err)
}
return int64(time)
}
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
}

View File

@ -1,4 +1,4 @@
package reader
package querynode
import (
"strings"
@ -26,16 +26,12 @@ func TestParamTable_QueryNodeID(t *testing.T) {
assert.Equal(t, id, 0)
}
func TestParamTable_TopicStart(t *testing.T) {
func TestParamTable_insertChannelRange(t *testing.T) {
Params.Init()
topicStart := Params.topicStart()
assert.Equal(t, topicStart, 0)
}
func TestParamTable_TopicEnd(t *testing.T) {
Params.Init()
topicEnd := Params.topicEnd()
assert.Equal(t, topicEnd, 128)
channelRange := Params.insertChannelRange()
assert.Equal(t, len(channelRange), 2)
assert.Equal(t, channelRange[0], 0)
assert.Equal(t, channelRange[1], 1)
}
func TestParamTable_statsServiceTimeInterval(t *testing.T) {
@ -50,9 +46,9 @@ func TestParamTable_statsMsgStreamReceiveBufSize(t *testing.T) {
assert.Equal(t, bufSize, int64(64))
}
func TestParamTable_dmMsgStreamReceiveBufSize(t *testing.T) {
func TestParamTable_insertMsgStreamReceiveBufSize(t *testing.T) {
Params.Init()
bufSize := Params.dmReceiveBufSize()
bufSize := Params.insertReceiveBufSize()
assert.Equal(t, bufSize, int64(1024))
}
@ -74,9 +70,9 @@ func TestParamTable_searchPulsarBufSize(t *testing.T) {
assert.Equal(t, bufSize, int64(512))
}
func TestParamTable_dmPulsarBufSize(t *testing.T) {
func TestParamTable_insertPulsarBufSize(t *testing.T) {
Params.Init()
bufSize := Params.dmPulsarBufSize()
bufSize := Params.insertPulsarBufSize()
assert.Equal(t, bufSize, int64(1024))
}
@ -91,3 +87,36 @@ 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")
}

View File

@ -1,4 +1,4 @@
package reader
package querynode
/*

View File

@ -1,4 +1,4 @@
package reader
package querynode
import (
"context"

View File

@ -1,8 +1,9 @@
package reader
package querynode
/*
#cgo CFLAGS: -I${SRCDIR}/../core/output/include
#cgo LDFLAGS: -L${SRCDIR}/../core/output/lib -lmilvus_segcore -Wl,-rpath=${SRCDIR}/../core/output/lib
#include "segcore/collection_c.h"
#include "segcore/segment_c.h"
#include "segcore/plan_c.h"
@ -16,19 +17,19 @@ type Plan struct {
cPlan C.CPlan
}
func CreatePlan(col Collection, dsl string) *Plan {
func createPlan(col Collection, dsl string) *Plan {
cDsl := C.CString(dsl)
cPlan := C.CreatePlan(col.collectionPtr, cDsl)
var newPlan = &Plan{cPlan: cPlan}
return newPlan
}
func (plan *Plan) GetTopK() int64 {
func (plan *Plan) getTopK() int64 {
topK := C.GetTopK(plan.cPlan)
return int64(topK)
}
func (plan *Plan) Delete() {
func (plan *Plan) delete() {
C.DeletePlan(plan.cPlan)
}
@ -36,7 +37,7 @@ type PlaceholderGroup struct {
cPlaceholderGroup C.CPlaceholderGroup
}
func ParserPlaceholderGroup(plan *Plan, placeHolderBlob []byte) *PlaceholderGroup {
func parserPlaceholderGroup(plan *Plan, placeHolderBlob []byte) *PlaceholderGroup {
var blobPtr = unsafe.Pointer(&placeHolderBlob[0])
blobSize := C.long(len(placeHolderBlob))
cPlaceholderGroup := C.ParsePlaceholderGroup(plan.cPlan, blobPtr, blobSize)
@ -44,11 +45,11 @@ func ParserPlaceholderGroup(plan *Plan, placeHolderBlob []byte) *PlaceholderGrou
return newPlaceholderGroup
}
func (pg *PlaceholderGroup) GetNumOfQuery() int64 {
func (pg *PlaceholderGroup) getNumOfQuery() int64 {
numQueries := C.GetNumOfQueries(pg.cPlaceholderGroup)
return int64(numQueries)
}
func (pg *PlaceholderGroup) Delete() {
func (pg *PlaceholderGroup) delete() {
C.DeletePlaceholderGroup(pg.cPlaceholderGroup)
}

View File

@ -0,0 +1,156 @@
package querynode
import (
"encoding/binary"
"math"
"testing"
"github.com/golang/protobuf/proto"
"github.com/stretchr/testify/assert"
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
"github.com/zilliztech/milvus-distributed/internal/proto/etcdpb"
"github.com/zilliztech/milvus-distributed/internal/proto/schemapb"
"github.com/zilliztech/milvus-distributed/internal/proto/servicepb"
)
func TestPlan_Plan(t *testing.T) {
fieldVec := schemapb.FieldSchema{
Name: "vec",
DataType: schemapb.DataType_VECTOR_FLOAT,
TypeParams: []*commonpb.KeyValuePair{
{
Key: "dim",
Value: "16",
},
},
}
fieldInt := schemapb.FieldSchema{
Name: "age",
DataType: schemapb.DataType_INT32,
TypeParams: []*commonpb.KeyValuePair{
{
Key: "dim",
Value: "1",
},
},
}
schema := schemapb.CollectionSchema{
Name: "collection0",
Fields: []*schemapb.FieldSchema{
&fieldVec, &fieldInt,
},
}
collectionMeta := etcdpb.CollectionMeta{
ID: UniqueID(0),
Schema: &schema,
CreateTime: Timestamp(0),
SegmentIDs: []UniqueID{0},
PartitionTags: []string{"default"},
}
collectionMetaBlob := proto.MarshalTextString(&collectionMeta)
assert.NotEqual(t, "", collectionMetaBlob)
collection := newCollection(&collectionMeta, collectionMetaBlob)
assert.Equal(t, collection.meta.Schema.Name, "collection0")
assert.Equal(t, collection.meta.ID, UniqueID(0))
dslString := "{\"bool\": { \n\"vector\": {\n \"vec\": {\n \"metric_type\": \"L2\", \n \"params\": {\n \"nprobe\": 10 \n},\n \"query\": \"$0\",\"topk\": 10 \n } \n } \n } \n }"
plan := createPlan(*collection, dslString)
assert.NotEqual(t, plan, nil)
topk := plan.getTopK()
assert.Equal(t, int(topk), 10)
plan.delete()
deleteCollection(collection)
}
func TestPlan_PlaceholderGroup(t *testing.T) {
fieldVec := schemapb.FieldSchema{
Name: "vec",
DataType: schemapb.DataType_VECTOR_FLOAT,
TypeParams: []*commonpb.KeyValuePair{
{
Key: "dim",
Value: "16",
},
},
}
fieldInt := schemapb.FieldSchema{
Name: "age",
DataType: schemapb.DataType_INT32,
TypeParams: []*commonpb.KeyValuePair{
{
Key: "dim",
Value: "1",
},
},
}
schema := schemapb.CollectionSchema{
Name: "collection0",
Fields: []*schemapb.FieldSchema{
&fieldVec, &fieldInt,
},
}
collectionMeta := etcdpb.CollectionMeta{
ID: UniqueID(0),
Schema: &schema,
CreateTime: Timestamp(0),
SegmentIDs: []UniqueID{0},
PartitionTags: []string{"default"},
}
collectionMetaBlob := proto.MarshalTextString(&collectionMeta)
assert.NotEqual(t, "", collectionMetaBlob)
collection := newCollection(&collectionMeta, collectionMetaBlob)
assert.Equal(t, collection.meta.Schema.Name, "collection0")
assert.Equal(t, collection.meta.ID, UniqueID(0))
dslString := "{\"bool\": { \n\"vector\": {\n \"vec\": {\n \"metric_type\": \"L2\", \n \"params\": {\n \"nprobe\": 10 \n},\n \"query\": \"$0\",\"topk\": 10 \n } \n } \n } \n }"
plan := createPlan(*collection, dslString)
assert.NotNil(t, plan)
var searchRawData1 []byte
var searchRawData2 []byte
const DIM = 16
var vec = [DIM]float32{1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16}
for i, ele := range vec {
buf := make([]byte, 4)
binary.LittleEndian.PutUint32(buf, math.Float32bits(ele+float32(i*2)))
searchRawData1 = append(searchRawData1, buf...)
}
for i, ele := range vec {
buf := make([]byte, 4)
binary.LittleEndian.PutUint32(buf, math.Float32bits(ele+float32(i*4)))
searchRawData2 = append(searchRawData2, buf...)
}
placeholderValue := servicepb.PlaceholderValue{
Tag: "$0",
Type: servicepb.PlaceholderType_VECTOR_FLOAT,
Values: [][]byte{searchRawData1, searchRawData2},
}
placeholderGroup := servicepb.PlaceholderGroup{
Placeholders: []*servicepb.PlaceholderValue{&placeholderValue},
}
placeGroupByte, err := proto.Marshal(&placeholderGroup)
assert.Nil(t, err)
holder := parserPlaceholderGroup(plan, placeGroupByte)
assert.NotNil(t, holder)
numQueries := holder.getNumOfQuery()
assert.Equal(t, int(numQueries), 2)
plan.delete()
holder.delete()
deleteCollection(collection)
}

View File

@ -1,4 +1,4 @@
package reader
package querynode
/*

View File

@ -1,4 +1,4 @@
package reader
package querynode
import (
"context"
@ -6,7 +6,7 @@ import (
"time"
)
const ctxTimeInMillisecond = 2000
const ctxTimeInMillisecond = 200
const closeWithDeadline = true
// NOTE: start pulsar and etcd before test

View File

@ -1,4 +1,4 @@
package reader
package querynode
import (
"context"

View File

@ -0,0 +1,76 @@
package querynode
/*
#cgo CFLAGS: -I${SRCDIR}/../core/output/include
#cgo LDFLAGS: -L${SRCDIR}/../core/output/lib -lmilvus_segcore -Wl,-rpath=${SRCDIR}/../core/output/lib
#include "segcore/plan_c.h"
#include "segcore/reduce_c.h"
*/
import "C"
import (
"unsafe"
)
type SearchResult struct {
cQueryResult C.CQueryResult
}
type MarshaledHits struct {
cMarshaledHits C.CMarshaledHits
}
func reduceSearchResults(searchResults []*SearchResult, numSegments int64) *SearchResult {
cSearchResults := make([]C.CQueryResult, 0)
for _, res := range searchResults {
cSearchResults = append(cSearchResults, res.cQueryResult)
}
cSearchResultPtr := (*C.CQueryResult)(&cSearchResults[0])
cNumSegments := C.long(numSegments)
res := C.ReduceQueryResults(cSearchResultPtr, cNumSegments)
return &SearchResult{cQueryResult: res}
}
func (sr *SearchResult) reorganizeQueryResults(plan *Plan, placeholderGroups []*PlaceholderGroup) *MarshaledHits {
cPlaceholderGroups := make([]C.CPlaceholderGroup, 0)
for _, pg := range placeholderGroups {
cPlaceholderGroups = append(cPlaceholderGroups, (*pg).cPlaceholderGroup)
}
cNumGroup := (C.long)(len(placeholderGroups))
var cPlaceHolder = (*C.CPlaceholderGroup)(&cPlaceholderGroups[0])
res := C.ReorganizeQueryResults(sr.cQueryResult, plan.cPlan, cPlaceHolder, cNumGroup)
return &MarshaledHits{cMarshaledHits: res}
}
func (mh *MarshaledHits) getHitsBlobSize() int64 {
res := C.GetHitsBlobSize(mh.cMarshaledHits)
return int64(res)
}
func (mh *MarshaledHits) getHitsBlob() ([]byte, error) {
byteSize := mh.getHitsBlobSize()
result := make([]byte, byteSize)
cResultPtr := unsafe.Pointer(&result[0])
C.GetHitsBlob(mh.cMarshaledHits, cResultPtr)
return result, nil
}
func (mh *MarshaledHits) hitBlobSizeInGroup(groupOffset int64) ([]int64, error) {
cGroupOffset := (C.long)(groupOffset)
numQueries := C.GetNumQueriesPeerGroup(mh.cMarshaledHits, cGroupOffset)
result := make([]int64, int64(numQueries))
cResult := (*C.long)(&result[0])
C.GetHitSizePeerQueries(mh.cMarshaledHits, cGroupOffset, cResult)
return result, nil
}
func deleteMarshaledHits(hits *MarshaledHits) {
C.DeleteMarshaledHits(hits.cMarshaledHits)
}
func deleteSearchResults(results []*SearchResult) {
for _, result := range results {
C.DeleteQueryResult(result.cQueryResult)
}
}

View File

@ -0,0 +1,139 @@
package querynode
import (
"encoding/binary"
"log"
"math"
"testing"
"github.com/golang/protobuf/proto"
"github.com/stretchr/testify/assert"
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
"github.com/zilliztech/milvus-distributed/internal/proto/etcdpb"
"github.com/zilliztech/milvus-distributed/internal/proto/schemapb"
"github.com/zilliztech/milvus-distributed/internal/proto/servicepb"
)
func TestReduce_AllFunc(t *testing.T) {
fieldVec := schemapb.FieldSchema{
Name: "vec",
DataType: schemapb.DataType_VECTOR_FLOAT,
TypeParams: []*commonpb.KeyValuePair{
{
Key: "dim",
Value: "16",
},
},
}
fieldInt := schemapb.FieldSchema{
Name: "age",
DataType: schemapb.DataType_INT32,
TypeParams: []*commonpb.KeyValuePair{
{
Key: "dim",
Value: "1",
},
},
}
schema := schemapb.CollectionSchema{
Name: "collection0",
Fields: []*schemapb.FieldSchema{
&fieldVec, &fieldInt,
},
}
collectionMeta := etcdpb.CollectionMeta{
ID: UniqueID(0),
Schema: &schema,
CreateTime: Timestamp(0),
SegmentIDs: []UniqueID{0},
PartitionTags: []string{"default"},
}
collectionMetaBlob := proto.MarshalTextString(&collectionMeta)
assert.NotEqual(t, "", collectionMetaBlob)
collection := newCollection(&collectionMeta, collectionMetaBlob)
assert.Equal(t, collection.meta.Schema.Name, "collection0")
assert.Equal(t, collection.meta.ID, UniqueID(0))
segmentID := UniqueID(0)
segment := newSegment(collection, segmentID)
assert.Equal(t, segmentID, segment.segmentID)
const DIM = 16
var vec = [DIM]float32{1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16}
// start search service
dslString := "{\"bool\": { \n\"vector\": {\n \"vec\": {\n \"metric_type\": \"L2\", \n \"params\": {\n \"nprobe\": 10 \n},\n \"query\": \"$0\",\"topk\": 10 \n } \n } \n } \n }"
var searchRawData1 []byte
var searchRawData2 []byte
for i, ele := range vec {
buf := make([]byte, 4)
binary.LittleEndian.PutUint32(buf, math.Float32bits(ele+float32(i*2)))
searchRawData1 = append(searchRawData1, buf...)
}
for i, ele := range vec {
buf := make([]byte, 4)
binary.LittleEndian.PutUint32(buf, math.Float32bits(ele+float32(i*4)))
searchRawData2 = append(searchRawData2, buf...)
}
placeholderValue := servicepb.PlaceholderValue{
Tag: "$0",
Type: servicepb.PlaceholderType_VECTOR_FLOAT,
Values: [][]byte{searchRawData1, searchRawData2},
}
placeholderGroup := servicepb.PlaceholderGroup{
Placeholders: []*servicepb.PlaceholderValue{&placeholderValue},
}
placeGroupByte, err := proto.Marshal(&placeholderGroup)
if err != nil {
log.Print("marshal placeholderGroup failed")
}
plan := createPlan(*collection, dslString)
holder := parserPlaceholderGroup(plan, placeGroupByte)
placeholderGroups := make([]*PlaceholderGroup, 0)
placeholderGroups = append(placeholderGroups, holder)
searchResults := make([]*SearchResult, 0)
searchResult, err := segment.segmentSearch(plan, placeholderGroups, []Timestamp{0})
assert.Nil(t, err)
searchResults = append(searchResults, searchResult)
reducedSearchResults := reduceSearchResults(searchResults, 1)
assert.NotNil(t, reducedSearchResults)
marshaledHits := reducedSearchResults.reorganizeQueryResults(plan, placeholderGroups)
assert.NotNil(t, marshaledHits)
hitsBlob, err := marshaledHits.getHitsBlob()
assert.Nil(t, err)
var offset int64 = 0
for index := range placeholderGroups {
hitBolbSizePeerQuery, err := marshaledHits.hitBlobSizeInGroup(int64(index))
assert.Nil(t, err)
for _, len := range hitBolbSizePeerQuery {
marshaledHit := hitsBlob[offset : offset+len]
unMarshaledHit := servicepb.Hits{}
err = proto.Unmarshal(marshaledHit, &unMarshaledHit)
assert.Nil(t, err)
log.Println("hits msg = ", unMarshaledHit)
offset += len
}
}
plan.delete()
holder.delete()
deleteSearchResults(searchResults)
deleteSearchResults([]*SearchResult{reducedSearchResults})
deleteMarshaledHits(marshaledHits)
deleteSegment(segment)
deleteCollection(collection)
}

View File

@ -0,0 +1,339 @@
package querynode
import "C"
import (
"context"
"errors"
"log"
"sync"
"github.com/golang/protobuf/proto"
"github.com/zilliztech/milvus-distributed/internal/msgstream"
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb"
"github.com/zilliztech/milvus-distributed/internal/proto/servicepb"
)
type searchService struct {
ctx context.Context
wait sync.WaitGroup
cancel context.CancelFunc
replica *collectionReplica
tSafeWatcher *tSafeWatcher
serviceableTime Timestamp
serviceableTimeMutex sync.Mutex
msgBuffer chan msgstream.TsMsg
unsolvedMsg []msgstream.TsMsg
searchMsgStream *msgstream.MsgStream
searchResultMsgStream *msgstream.MsgStream
}
type ResultEntityIds []UniqueID
func newSearchService(ctx context.Context, replica *collectionReplica) *searchService {
receiveBufSize := Params.searchReceiveBufSize()
pulsarBufSize := Params.searchPulsarBufSize()
msgStreamURL, err := Params.pulsarAddress()
if err != nil {
log.Fatal(err)
}
consumeChannels := Params.searchChannelNames()
consumeSubName := Params.msgChannelSubName()
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()
searchResultStream := msgstream.NewPulsarMsgStream(ctx, receiveBufSize)
searchResultStream.SetPulsarClient(msgStreamURL)
searchResultStream.CreatePulsarProducers(producerChannels)
var outputStream msgstream.MsgStream = searchResultStream
searchServiceCtx, searchServiceCancel := context.WithCancel(ctx)
msgBuffer := make(chan msgstream.TsMsg, receiveBufSize)
unsolvedMsg := make([]msgstream.TsMsg, 0)
return &searchService{
ctx: searchServiceCtx,
cancel: searchServiceCancel,
serviceableTime: Timestamp(0),
msgBuffer: msgBuffer,
unsolvedMsg: unsolvedMsg,
replica: replica,
tSafeWatcher: newTSafeWatcher(),
searchMsgStream: &inputStream,
searchResultMsgStream: &outputStream,
}
}
func (ss *searchService) start() {
(*ss.searchMsgStream).Start()
(*ss.searchResultMsgStream).Start()
ss.register()
ss.wait.Add(2)
go ss.receiveSearchMsg()
go ss.doUnsolvedMsgSearch()
ss.wait.Wait()
}
func (ss *searchService) close() {
(*ss.searchMsgStream).Close()
(*ss.searchResultMsgStream).Close()
ss.cancel()
}
func (ss *searchService) register() {
tSafe := (*(ss.replica)).getTSafe()
(*tSafe).registerTSafeWatcher(ss.tSafeWatcher)
}
func (ss *searchService) waitNewTSafe() Timestamp {
// block until dataSyncService updating tSafe
ss.tSafeWatcher.hasUpdate()
timestamp := (*(*ss.replica).getTSafe()).get()
return timestamp
}
func (ss *searchService) getServiceableTime() Timestamp {
ss.serviceableTimeMutex.Lock()
defer ss.serviceableTimeMutex.Unlock()
return ss.serviceableTime
}
func (ss *searchService) setServiceableTime(t Timestamp) {
ss.serviceableTimeMutex.Lock()
// TODO:: add gracefulTime
ss.serviceableTime = t
ss.serviceableTimeMutex.Unlock()
}
func (ss *searchService) receiveSearchMsg() {
defer ss.wait.Done()
for {
select {
case <-ss.ctx.Done():
return
default:
msgPack := (*ss.searchMsgStream).Consume()
if msgPack == nil || len(msgPack.Msgs) <= 0 {
continue
}
searchMsg := make([]msgstream.TsMsg, 0)
serverTime := ss.getServiceableTime()
for i := range msgPack.Msgs {
if msgPack.Msgs[i].BeginTs() > serverTime {
ss.msgBuffer <- msgPack.Msgs[i]
continue
}
searchMsg = append(searchMsg, msgPack.Msgs[i])
}
for _, msg := range searchMsg {
err := ss.search(msg)
if err != nil {
log.Println("search Failed, error msg type: ", msg.Type())
}
err = ss.publishFailedSearchResult(msg)
if err != nil {
log.Println("publish FailedSearchResult failed, error message: ", err)
}
}
log.Println("Do search done, num of searchMsg = ", len(searchMsg))
}
}
}
func (ss *searchService) doUnsolvedMsgSearch() {
defer ss.wait.Done()
for {
select {
case <-ss.ctx.Done():
return
default:
serviceTime := ss.waitNewTSafe()
ss.setServiceableTime(serviceTime)
searchMsg := make([]msgstream.TsMsg, 0)
tempMsg := make([]msgstream.TsMsg, 0)
tempMsg = append(tempMsg, ss.unsolvedMsg...)
ss.unsolvedMsg = ss.unsolvedMsg[:0]
for _, msg := range tempMsg {
if msg.EndTs() <= serviceTime {
searchMsg = append(searchMsg, msg)
continue
}
ss.unsolvedMsg = append(ss.unsolvedMsg, msg)
}
msgBufferLength := len(ss.msgBuffer)
for i := 0; i < msgBufferLength; i++ {
msg := <-ss.msgBuffer
if msg.EndTs() <= serviceTime {
searchMsg = append(searchMsg, msg)
continue
}
ss.unsolvedMsg = append(ss.unsolvedMsg, msg)
}
if len(searchMsg) <= 0 {
continue
}
for _, msg := range searchMsg {
err := ss.search(msg)
if err != nil {
log.Println("search Failed, error msg type: ", msg.Type())
}
err = ss.publishFailedSearchResult(msg)
if err != nil {
log.Println("publish FailedSearchResult failed, error message: ", err)
}
}
log.Println("Do search done, num of searchMsg = ", len(searchMsg))
}
}
}
// TODO:: cache map[dsl]plan
// TODO: reBatched search requests
func (ss *searchService) search(msg msgstream.TsMsg) error {
searchMsg, ok := msg.(*msgstream.SearchMsg)
if !ok {
return errors.New("invalid request type = " + string(msg.Type()))
}
searchTimestamp := searchMsg.Timestamp
var queryBlob = searchMsg.Query.Value
query := servicepb.Query{}
err := proto.Unmarshal(queryBlob, &query)
if err != nil {
return errors.New("unmarshal query failed")
}
collectionName := query.CollectionName
partitionTags := query.PartitionTags
collection, err := (*ss.replica).getCollectionByName(collectionName)
if err != nil {
return err
}
collectionID := collection.ID()
dsl := query.Dsl
plan := createPlan(*collection, dsl)
placeHolderGroupBlob := query.PlaceholderGroup
placeholderGroup := parserPlaceholderGroup(plan, placeHolderGroupBlob)
placeholderGroups := make([]*PlaceholderGroup, 0)
placeholderGroups = append(placeholderGroups, placeholderGroup)
searchResults := make([]*SearchResult, 0)
for _, partitionTag := range partitionTags {
partition, err := (*ss.replica).getPartitionByTag(collectionID, partitionTag)
if err != nil {
return err
}
for _, segment := range partition.segments {
searchResult, err := segment.segmentSearch(plan, placeholderGroups, []Timestamp{searchTimestamp})
if err != nil {
return err
}
searchResults = append(searchResults, searchResult)
}
}
reducedSearchResult := reduceSearchResults(searchResults, int64(len(searchResults)))
marshaledHits := reducedSearchResult.reorganizeQueryResults(plan, placeholderGroups)
hitsBlob, err := marshaledHits.getHitsBlob()
if err != nil {
return err
}
var offset int64 = 0
for index := range placeholderGroups {
hitBolbSizePeerQuery, err := marshaledHits.hitBlobSizeInGroup(int64(index))
if err != nil {
return err
}
hits := make([][]byte, 0)
for _, len := range hitBolbSizePeerQuery {
hits = append(hits, hitsBlob[offset:offset+len])
//test code to checkout marshaled hits
//marshaledHit := hitsBlob[offset:offset+len]
//unMarshaledHit := servicepb.Hits{}
//err = proto.Unmarshal(marshaledHit, &unMarshaledHit)
//if err != nil {
// return err
//}
//fmt.Println("hits msg = ", unMarshaledHit)
offset += len
}
var results = internalpb.SearchResult{
MsgType: internalpb.MsgType_kSearchResult,
Status: &commonpb.Status{ErrorCode: commonpb.ErrorCode_SUCCESS},
ReqID: searchMsg.ReqID,
ProxyID: searchMsg.ProxyID,
QueryNodeID: searchMsg.ProxyID,
Timestamp: searchTimestamp,
ResultChannelID: searchMsg.ResultChannelID,
Hits: hits,
}
searchResultMsg := &msgstream.SearchResultMsg{
BaseMsg: msgstream.BaseMsg{HashValues: []int32{0}},
SearchResult: results,
}
err = ss.publishSearchResult(searchResultMsg)
if err != nil {
return err
}
}
deleteSearchResults(searchResults)
deleteSearchResults([]*SearchResult{reducedSearchResult})
deleteMarshaledHits(marshaledHits)
plan.delete()
placeholderGroup.delete()
return nil
}
func (ss *searchService) publishSearchResult(msg msgstream.TsMsg) error {
msgPack := msgstream.MsgPack{}
msgPack.Msgs = append(msgPack.Msgs, msg)
err := (*ss.searchResultMsgStream).Produce(&msgPack)
if err != nil {
return err
}
return nil
}
func (ss *searchService) publishFailedSearchResult(msg msgstream.TsMsg) error {
msgPack := msgstream.MsgPack{}
searchMsg, ok := msg.(*msgstream.SearchMsg)
if !ok {
return errors.New("invalid request type = " + string(msg.Type()))
}
var results = internalpb.SearchResult{
MsgType: internalpb.MsgType_kSearchResult,
Status: &commonpb.Status{ErrorCode: commonpb.ErrorCode_UNEXPECTED_ERROR},
ReqID: searchMsg.ReqID,
ProxyID: searchMsg.ProxyID,
QueryNodeID: searchMsg.ProxyID,
Timestamp: searchMsg.Timestamp,
ResultChannelID: searchMsg.ResultChannelID,
Hits: [][]byte{},
}
tsMsg := &msgstream.SearchResultMsg{
BaseMsg: msgstream.BaseMsg{HashValues: []int32{0}},
SearchResult: results,
}
msgPack.Msgs = append(msgPack.Msgs, tsMsg)
err := (*ss.searchResultMsgStream).Produce(&msgPack)
if err != nil {
return err
}
return nil
}

View File

@ -1,9 +1,8 @@
package reader
package querynode
import (
"context"
"encoding/binary"
"log"
"math"
"testing"
@ -88,103 +87,30 @@ func TestSearch_Search(t *testing.T) {
// test data generate
const msgLength = 10
const DIM = 16
const N = 10
var vec = [DIM]float32{1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16}
var rawData []byte
for _, ele := range vec {
buf := make([]byte, 4)
binary.LittleEndian.PutUint32(buf, math.Float32bits(ele))
rawData = append(rawData, buf...)
}
bs := make([]byte, 4)
binary.LittleEndian.PutUint32(bs, 1)
rawData = append(rawData, bs...)
var records []*commonpb.Blob
for i := 0; i < N; i++ {
blob := &commonpb.Blob{
Value: rawData,
}
records = append(records, blob)
}
timeRange := TimeRange{
timestampMin: 0,
timestampMax: math.MaxUint64,
}
// messages generate
insertMessages := make([]msgstream.TsMsg, 0)
for i := 0; i < msgLength; i++ {
var msg msgstream.TsMsg = &msgstream.InsertMsg{
BaseMsg: msgstream.BaseMsg{
HashValues: []int32{
int32(i), int32(i),
},
},
InsertRequest: internalpb.InsertRequest{
MsgType: internalpb.MsgType_kInsert,
ReqID: int64(i),
CollectionName: "collection0",
PartitionTag: "default",
SegmentID: int64(0),
ChannelID: int64(0),
ProxyID: int64(0),
Timestamps: []uint64{uint64(i + 1000), uint64(i + 1000)},
RowIDs: []int64{int64(i * 2), int64(i*2 + 1)},
RowData: []*commonpb.Blob{
{Value: rawData},
{Value: rawData},
},
},
}
insertMessages = append(insertMessages, msg)
}
msgPack := msgstream.MsgPack{
BeginTs: timeRange.timestampMin,
EndTs: timeRange.timestampMax,
Msgs: insertMessages,
}
// pulsar produce
const receiveBufSize = 1024
insertProducerChannels := []string{"insert"}
insertStream := msgstream.NewPulsarMsgStream(ctx, receiveBufSize)
insertStream.SetPulsarClient(pulsarURL)
insertStream.CreatePulsarProducers(insertProducerChannels)
var insertMsgStream msgstream.MsgStream = insertStream
insertMsgStream.Start()
err = insertMsgStream.Produce(&msgPack)
assert.NoError(t, err)
// dataSync
node.dataSyncService = newDataSyncService(node.ctx, node.replica)
go node.dataSyncService.start()
time.Sleep(2 * time.Second)
const DIM = 16
insertProducerChannels := Params.insertChannelNames()
searchProducerChannels := Params.searchChannelNames()
var vec = [DIM]float32{1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16}
// start search service
dslString := "{\"bool\": { \n\"vector\": {\n \"vec\": {\n \"metric_type\": \"L2\", \n \"params\": {\n \"nprobe\": 10 \n},\n \"query\": \"$0\",\"topk\": 10 \n } \n } \n } \n }"
searchProducerChannels := []string{"search"}
searchStream := msgstream.NewPulsarMsgStream(ctx, receiveBufSize)
searchStream.SetPulsarClient(pulsarURL)
searchStream.CreatePulsarProducers(searchProducerChannels)
var vecSearch = [DIM]float32{1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 17}
var searchRawData []byte
for _, ele := range vecSearch {
var searchRawData1 []byte
var searchRawData2 []byte
for i, ele := range vec {
buf := make([]byte, 4)
binary.LittleEndian.PutUint32(buf, math.Float32bits(ele))
searchRawData = append(searchRawData, buf...)
binary.LittleEndian.PutUint32(buf, math.Float32bits(ele+float32(i*2)))
searchRawData1 = append(searchRawData1, buf...)
}
for i, ele := range vec {
buf := make([]byte, 4)
binary.LittleEndian.PutUint32(buf, math.Float32bits(ele+float32(i*4)))
searchRawData2 = append(searchRawData2, buf...)
}
placeholderValue := servicepb.PlaceholderValue{
Tag: "$0",
Type: servicepb.PlaceholderType_VECTOR_FLOAT,
Values: [][]byte{searchRawData},
Values: [][]byte{searchRawData1, searchRawData2},
}
placeholderGroup := servicepb.PlaceholderGroup{
@ -220,8 +146,8 @@ func TestSearch_Search(t *testing.T) {
MsgType: internalpb.MsgType_kSearch,
ReqID: int64(1),
ProxyID: int64(1),
Timestamp: uint64(20 + 1000),
ResultChannelID: int64(1),
Timestamp: uint64(10 + 1000),
ResultChannelID: int64(0),
Query: &blob,
},
}
@ -229,15 +155,97 @@ func TestSearch_Search(t *testing.T) {
msgPackSearch := msgstream.MsgPack{}
msgPackSearch.Msgs = append(msgPackSearch.Msgs, searchMsg)
var searchMsgStream msgstream.MsgStream = searchStream
searchMsgStream.Start()
err = searchMsgStream.Produce(&msgPackSearch)
searchStream := msgstream.NewPulsarMsgStream(ctx, receiveBufSize)
searchStream.SetPulsarClient(pulsarURL)
searchStream.CreatePulsarProducers(searchProducerChannels)
searchStream.Start()
err = searchStream.Produce(&msgPackSearch)
assert.NoError(t, err)
node.searchService = newSearchService(node.ctx, node.replica)
go node.searchService.start()
time.Sleep(2 * time.Second)
// start insert
timeRange := TimeRange{
timestampMin: 0,
timestampMax: math.MaxUint64,
}
insertMessages := make([]msgstream.TsMsg, 0)
for i := 0; i < msgLength; i++ {
var rawData []byte
for _, ele := range vec {
buf := make([]byte, 4)
binary.LittleEndian.PutUint32(buf, math.Float32bits(ele+float32(i*2)))
rawData = append(rawData, buf...)
}
bs := make([]byte, 4)
binary.LittleEndian.PutUint32(bs, 1)
rawData = append(rawData, bs...)
var msg msgstream.TsMsg = &msgstream.InsertMsg{
BaseMsg: msgstream.BaseMsg{
HashValues: []int32{
int32(i),
},
},
InsertRequest: internalpb.InsertRequest{
MsgType: internalpb.MsgType_kInsert,
ReqID: int64(i),
CollectionName: "collection0",
PartitionTag: "default",
SegmentID: int64(0),
ChannelID: int64(0),
ProxyID: int64(0),
Timestamps: []uint64{uint64(i + 1000)},
RowIDs: []int64{int64(i)},
RowData: []*commonpb.Blob{
{Value: rawData},
},
},
}
insertMessages = append(insertMessages, msg)
}
msgPack := msgstream.MsgPack{
BeginTs: timeRange.timestampMin,
EndTs: timeRange.timestampMax,
Msgs: insertMessages,
}
// generate timeTick
timeTickMsgPack := msgstream.MsgPack{}
baseMsg := msgstream.BaseMsg{
BeginTimestamp: 0,
EndTimestamp: 0,
HashValues: []int32{0},
}
timeTickResult := internalpb.TimeTickMsg{
MsgType: internalpb.MsgType_kTimeTick,
PeerID: UniqueID(0),
Timestamp: math.MaxUint64,
}
timeTickMsg := &msgstream.TimeTickMsg{
BaseMsg: baseMsg,
TimeTickMsg: timeTickResult,
}
timeTickMsgPack.Msgs = append(timeTickMsgPack.Msgs, timeTickMsg)
// pulsar produce
insertStream := msgstream.NewPulsarMsgStream(ctx, receiveBufSize)
insertStream.SetPulsarClient(pulsarURL)
insertStream.CreatePulsarProducers(insertProducerChannels)
insertStream.Start()
err = insertStream.Produce(&msgPack)
assert.NoError(t, err)
err = insertStream.Broadcast(&timeTickMsgPack)
assert.NoError(t, err)
// dataSync
node.dataSyncService = newDataSyncService(node.ctx, node.replica)
go node.dataSyncService.start()
time.Sleep(1 * time.Second)
cancel()
node.Close()

View File

@ -1,4 +1,4 @@
package reader
package querynode
/*
@ -7,7 +7,6 @@ package reader
#cgo LDFLAGS: -L${SRCDIR}/../core/output/lib -lmilvus_segcore -Wl,-rpath=${SRCDIR}/../core/output/lib
#include "segcore/collection_c.h"
#include "segcore/segment_c.h"
#include "segcore/plan_c.h"
#include "segcore/reduce_c.h"
@ -109,7 +108,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 {
/*
int
CStatus
Insert(CSegmentBase c_segment,
long int reserved_offset,
signed long int size,
@ -148,8 +147,12 @@ func (s *Segment) segmentInsert(offset int64, entityIDs *[]UniqueID, timestamps
cSizeofPerRow,
cNumOfRows)
if status != 0 {
return errors.New("Insert failed, error code = " + strconv.Itoa(int(status)))
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)
}
s.recentlyModified = true
@ -158,7 +161,7 @@ func (s *Segment) segmentInsert(offset int64, entityIDs *[]UniqueID, timestamps
func (s *Segment) segmentDelete(offset int64, entityIDs *[]UniqueID, timestamps *[]Timestamp) error {
/*
int
CStatus
Delete(CSegmentBase c_segment,
long int reserved_offset,
long size,
@ -172,8 +175,12 @@ func (s *Segment) segmentDelete(offset int64, entityIDs *[]UniqueID, timestamps
var status = C.Delete(s.segmentPtr, cOffset, cSize, cEntityIdsPtr, cTimestampsPtr)
if status != 0 {
return errors.New("Delete failed, error code = " + strconv.Itoa(int(status)))
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)
}
return nil
@ -181,13 +188,10 @@ func (s *Segment) segmentDelete(offset int64, entityIDs *[]UniqueID, timestamps
func (s *Segment) segmentSearch(plan *Plan,
placeHolderGroups []*PlaceholderGroup,
timestamp []Timestamp,
resultIds []IntPrimaryKey,
resultDistances []float32,
numQueries int64,
topK int64) error {
timestamp []Timestamp) (*SearchResult, error) {
/*
void* Search(void* plan,
CStatus
Search(void* plan,
void* placeholder_groups,
uint64_t* timestamps,
int num_groups,
@ -195,32 +199,25 @@ func (s *Segment) segmentSearch(plan *Plan,
float* result_distances);
*/
newResultIds := make([]IntPrimaryKey, topK*numQueries)
NewResultDistances := make([]float32, topK*numQueries)
cPlaceholderGroups := make([]C.CPlaceholderGroup, 0)
for _, pg := range placeHolderGroups {
cPlaceholderGroups = append(cPlaceholderGroups, (*pg).cPlaceholderGroup)
}
var searchResult SearchResult
var cTimestamp = (*C.ulong)(&timestamp[0])
var cResultIds = (*C.long)(&resultIds[0])
var cResultDistances = (*C.float)(&resultDistances[0])
var cNewResultIds = (*C.long)(&newResultIds[0])
var cNewResultDistances = (*C.float)(&NewResultDistances[0])
var cPlaceHolder = (*C.CPlaceholderGroup)(&cPlaceholderGroups[0])
var cNumGroups = C.int(len(placeHolderGroups))
cQueryResult := (*C.CQueryResult)(&searchResult.cQueryResult)
var status = C.Search(s.segmentPtr, plan.cPlan, cPlaceHolder, cTimestamp, cNumGroups, cNewResultIds, cNewResultDistances)
if status != 0 {
return errors.New("search failed, error code = " + strconv.Itoa(int(status)))
var status = C.Search(s.segmentPtr, plan.cPlan, cPlaceHolder, cTimestamp, cNumGroups, cQueryResult)
errorCode := status.error_code
if errorCode != 0 {
errorMsg := C.GoString(status.error_msg)
defer C.free(unsafe.Pointer(status.error_msg))
return nil, errors.New("Search failed, C runtime error detected, error code = " + strconv.Itoa(int(errorCode)) + ", error msg = " + errorMsg)
}
cNumQueries := C.long(numQueries)
cTopK := C.long(topK)
// reduce search result
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
return &searchResult, nil
}

View File

@ -1,4 +1,4 @@
package reader
package querynode
import (
"context"
@ -463,7 +463,6 @@ func TestSegment_segmentInsert(t *testing.T) {
err := segment.segmentInsert(offset, &ids, &timestamps, &records)
assert.NoError(t, err)
deleteSegment(segment)
deleteCollection(collection)
}
@ -640,7 +639,7 @@ func TestSegment_segmentSearch(t *testing.T) {
pulsarURL, _ := Params.pulsarAddress()
const receiveBufSize = 1024
searchProducerChannels := []string{"search"}
searchProducerChannels := Params.searchChannelNames()
searchStream := msgstream.NewPulsarMsgStream(ctx, receiveBufSize)
searchStream.SetPulsarClient(pulsarURL)
searchStream.CreatePulsarProducers(searchProducerChannels)
@ -667,25 +666,16 @@ func TestSegment_segmentSearch(t *testing.T) {
}
searchTimestamp := Timestamp(1020)
cPlan := CreatePlan(*collection, dslString)
topK := cPlan.GetTopK()
cPlaceholderGroup := ParserPlaceholderGroup(cPlan, placeHolderGroupBlob)
plan := createPlan(*collection, dslString)
holder := parserPlaceholderGroup(plan, placeHolderGroupBlob)
placeholderGroups := make([]*PlaceholderGroup, 0)
placeholderGroups = append(placeholderGroups, cPlaceholderGroup)
placeholderGroups = append(placeholderGroups, holder)
var numQueries int64 = 0
for _, pg := range placeholderGroups {
numQueries += pg.GetNumOfQuery()
}
resultIds := make([]IntPrimaryKey, topK*numQueries)
resultDistances := make([]float32, topK*numQueries)
for i := range resultDistances {
resultDistances[i] = math.MaxFloat32
}
err = segment.segmentSearch(cPlan, placeholderGroups, []Timestamp{searchTimestamp}, resultIds, resultDistances, numQueries, topK)
assert.NoError(t, err)
_, err = segment.segmentSearch(plan, placeholderGroups, []Timestamp{searchTimestamp})
assert.Nil(t, err)
plan.delete()
holder.delete()
deleteSegment(segment)
deleteCollection(collection)
}

View File

@ -1,4 +1,4 @@
package reader
package querynode
import (
"context"
@ -36,7 +36,7 @@ func (sService *statsService) start() {
if err != nil {
log.Fatal(err)
}
producerChannels := []string{"statistic"}
producerChannels := []string{Params.statsChannelName()}
statsStream := msgstream.NewPulsarMsgStream(sService.ctx, receiveBufSize)
statsStream.SetPulsarClient(msgStreamURL)

View File

@ -1,4 +1,4 @@
package reader
package querynode
import (
"context"
@ -171,7 +171,7 @@ func TestSegmentManagement_SegmentStatisticService(t *testing.T) {
const receiveBufSize = 1024
// start pulsar
producerChannels := []string{"statistic"}
producerChannels := []string{Params.statsChannelName()}
statsStream := msgstream.NewPulsarMsgStream(ctx, receiveBufSize)
statsStream.SetPulsarClient(pulsarURL)

View File

@ -1,4 +1,4 @@
package reader
package querynode
import (
"sync"

View File

@ -1,4 +1,4 @@
package reader
package querynode
import (
"testing"

View File

@ -1,4 +1,4 @@
package reader
package querynode
import "github.com/zilliztech/milvus-distributed/internal/util/typeutil"

View File

@ -1,195 +0,0 @@
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
}

View File

@ -1,281 +0,0 @@
package reader
import "C"
import (
"context"
"errors"
"fmt"
"log"
"math"
"sync"
"github.com/golang/protobuf/proto"
"github.com/zilliztech/milvus-distributed/internal/msgstream"
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb"
"github.com/zilliztech/milvus-distributed/internal/proto/servicepb"
)
type searchService struct {
ctx context.Context
wait sync.WaitGroup
cancel context.CancelFunc
msgBuffer chan msgstream.TsMsg
unsolvedMsg []msgstream.TsMsg
replica *collectionReplica
tSafeWatcher *tSafeWatcher
searchMsgStream *msgstream.MsgStream
searchResultMsgStream *msgstream.MsgStream
}
type ResultEntityIds []UniqueID
func newSearchService(ctx context.Context, replica *collectionReplica) *searchService {
receiveBufSize := Params.searchReceiveBufSize()
pulsarBufSize := Params.searchPulsarBufSize()
msgStreamURL, err := Params.pulsarAddress()
if err != nil {
log.Fatal(err)
}
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 := []string{"searchResult"}
searchResultStream := msgstream.NewPulsarMsgStream(ctx, receiveBufSize)
searchResultStream.SetPulsarClient(msgStreamURL)
searchResultStream.CreatePulsarProducers(producerChannels)
var outputStream msgstream.MsgStream = searchResultStream
searchServiceCtx, searchServiceCancel := context.WithCancel(ctx)
msgBuffer := make(chan msgstream.TsMsg, receiveBufSize)
unsolvedMsg := make([]msgstream.TsMsg, 0)
return &searchService{
ctx: searchServiceCtx,
cancel: searchServiceCancel,
msgBuffer: msgBuffer,
unsolvedMsg: unsolvedMsg,
replica: replica,
tSafeWatcher: newTSafeWatcher(),
searchMsgStream: &inputStream,
searchResultMsgStream: &outputStream,
}
}
func (ss *searchService) start() {
(*ss.searchMsgStream).Start()
(*ss.searchResultMsgStream).Start()
ss.wait.Add(2)
go ss.receiveSearchMsg()
go ss.startSearchService()
ss.wait.Wait()
}
func (ss *searchService) close() {
(*ss.searchMsgStream).Close()
(*ss.searchResultMsgStream).Close()
ss.cancel()
}
func (ss *searchService) register() {
tSafe := (*(ss.replica)).getTSafe()
(*tSafe).registerTSafeWatcher(ss.tSafeWatcher)
}
func (ss *searchService) waitNewTSafe() Timestamp {
// block until dataSyncService updating tSafe
ss.tSafeWatcher.hasUpdate()
timestamp := (*(*ss.replica).getTSafe()).get()
return timestamp
}
func (ss *searchService) receiveSearchMsg() {
defer ss.wait.Done()
for {
select {
case <-ss.ctx.Done():
return
default:
msgPack := (*ss.searchMsgStream).Consume()
if msgPack == nil || len(msgPack.Msgs) <= 0 {
continue
}
for i := range msgPack.Msgs {
ss.msgBuffer <- msgPack.Msgs[i]
//fmt.Println("receive a search msg")
}
}
}
}
func (ss *searchService) startSearchService() {
defer ss.wait.Done()
for {
select {
case <-ss.ctx.Done():
return
default:
serviceTimestamp := (*(*ss.replica).getTSafe()).get()
searchMsg := make([]msgstream.TsMsg, 0)
tempMsg := make([]msgstream.TsMsg, 0)
tempMsg = append(tempMsg, ss.unsolvedMsg...)
ss.unsolvedMsg = ss.unsolvedMsg[:0]
for _, msg := range tempMsg {
if msg.BeginTs() > serviceTimestamp {
searchMsg = append(searchMsg, msg)
continue
}
ss.unsolvedMsg = append(ss.unsolvedMsg, msg)
}
msgBufferLength := len(ss.msgBuffer)
for i := 0; i < msgBufferLength; i++ {
msg := <-ss.msgBuffer
if msg.BeginTs() > serviceTimestamp {
searchMsg = append(searchMsg, msg)
continue
}
ss.unsolvedMsg = append(ss.unsolvedMsg, msg)
}
if len(searchMsg) <= 0 {
continue
}
err := ss.search(searchMsg)
if err != nil {
fmt.Println("search Failed")
ss.publishFailedSearchResult()
}
fmt.Println("Do search done")
}
}
}
func (ss *searchService) search(searchMessages []msgstream.TsMsg) error {
// TODO:: cache map[dsl]plan
// TODO: reBatched search requests
for _, msg := range searchMessages {
searchMsg, ok := msg.(*msgstream.SearchMsg)
if !ok {
return errors.New("invalid request type = " + string(msg.Type()))
}
searchTimestamp := searchMsg.Timestamp
var queryBlob = searchMsg.Query.Value
query := servicepb.Query{}
err := proto.Unmarshal(queryBlob, &query)
if err != nil {
return errors.New("unmarshal query failed")
}
collectionName := query.CollectionName
partitionTags := query.PartitionTags
collection, err := (*ss.replica).getCollectionByName(collectionName)
if err != nil {
return err
}
collectionID := collection.ID()
dsl := query.Dsl
plan := CreatePlan(*collection, dsl)
topK := plan.GetTopK()
placeHolderGroupBlob := query.PlaceholderGroup
group := servicepb.PlaceholderGroup{}
err = proto.Unmarshal(placeHolderGroupBlob, &group)
if err != nil {
return err
}
placeholderGroup := ParserPlaceholderGroup(plan, placeHolderGroupBlob)
placeholderGroups := make([]*PlaceholderGroup, 0)
placeholderGroups = append(placeholderGroups, placeholderGroup)
// 2d slice for receiving multiple queries's results
var numQueries int64 = 0
for _, pg := range placeholderGroups {
numQueries += pg.GetNumOfQuery()
}
resultIds := make([]IntPrimaryKey, topK*numQueries)
resultDistances := make([]float32, topK*numQueries)
for i := range resultDistances {
resultDistances[i] = math.MaxFloat32
}
// 3. Do search in all segments
for _, partitionTag := range partitionTags {
partition, err := (*ss.replica).getPartitionByTag(collectionID, partitionTag)
if err != nil {
return err
}
for _, segment := range partition.segments {
err := segment.segmentSearch(plan,
placeholderGroups,
[]Timestamp{searchTimestamp},
resultIds,
resultDistances,
numQueries,
topK)
if err != nil {
return err
}
}
}
// 4. return results
hits := make([]*servicepb.Hits, 0)
for i := int64(0); i < numQueries; i++ {
hit := servicepb.Hits{}
score := servicepb.Score{}
for j := i * topK; j < (i+1)*topK; j++ {
hit.IDs = append(hit.IDs, resultIds[j])
score.Values = append(score.Values, resultDistances[j])
}
hit.Scores = append(hit.Scores, &score)
hits = append(hits, &hit)
}
var results = internalpb.SearchResult{
MsgType: internalpb.MsgType_kSearchResult,
Status: &commonpb.Status{ErrorCode: commonpb.ErrorCode_SUCCESS},
ReqID: searchMsg.ReqID,
ProxyID: searchMsg.ProxyID,
QueryNodeID: searchMsg.ProxyID,
Timestamp: searchTimestamp,
ResultChannelID: searchMsg.ResultChannelID,
Hits: hits,
}
var tsMsg msgstream.TsMsg = &msgstream.SearchResultMsg{SearchResult: results}
ss.publishSearchResult(tsMsg)
plan.Delete()
placeholderGroup.Delete()
}
return nil
}
func (ss *searchService) publishSearchResult(res msgstream.TsMsg) {
msgPack := msgstream.MsgPack{}
msgPack.Msgs = append(msgPack.Msgs, res)
(*ss.searchResultMsgStream).Produce(&msgPack)
}
func (ss *searchService) publishFailedSearchResult() {
var errorResults = internalpb.SearchResult{
MsgType: internalpb.MsgType_kSearchResult,
Status: &commonpb.Status{ErrorCode: commonpb.ErrorCode_UNEXPECTED_ERROR},
}
var tsMsg msgstream.TsMsg = &msgstream.SearchResultMsg{SearchResult: errorResults}
msgPack := msgstream.MsgPack{}
msgPack.Msgs = append(msgPack.Msgs, tsMsg)
(*ss.searchResultMsgStream).Produce(&msgPack)
}

View File

@ -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}/reader/..." "${MILVUS_DIR}/proxy/..." -failfast
go test -cover "${MILVUS_DIR}/kv/..." "${MILVUS_DIR}/msgstream/..." "${MILVUS_DIR}/master/..." "${MILVUS_DIR}/reader/..." -failfast
#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