mirror of https://github.com/milvus-io/milvus.git
Add LoadIndexService in query node
Signed-off-by: xige-16 <xi.ge@zilliz.com>pull/4973/head^2
parent
add31188ba
commit
f5198bcaee
|
@ -5,4 +5,4 @@ TARGET_TAG=latest
|
|||
PULSAR_ADDRESS=pulsar://pulsar:6650
|
||||
ETCD_ADDRESS=etcd:2379
|
||||
MASTER_ADDRESS=master:53100
|
||||
MINIO_ADDRESS=minio:9000
|
||||
MINIO_ADDRESS=minio:9000
|
||||
|
|
|
@ -22,6 +22,8 @@ msgChannel:
|
|||
writeNodeTimeTick: "writeNodeTimeTick"
|
||||
# old name: statsChannels: "statistic"
|
||||
queryNodeStats: "query-node-stats"
|
||||
# cmd for loadIndex, flush, etc...
|
||||
cmd: "cmd"
|
||||
|
||||
# sub name generation rule: ${subNamePrefix}-${NodeID}
|
||||
subNamePrefix:
|
||||
|
|
|
@ -42,3 +42,7 @@ queryNode:
|
|||
|
||||
stats:
|
||||
recvBufSize: 64
|
||||
|
||||
loadIndex:
|
||||
recvBufSize: 512
|
||||
pulsarBufSize: 512
|
||||
|
|
|
@ -0,0 +1,23 @@
|
|||
// Copyright (C) 2019-2020 Zilliz. All rights reserved.
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance
|
||||
// with the License. You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software distributed under the License
|
||||
// is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
|
||||
// or implied. See the License for the specific language governing permissions and limitations under the License
|
||||
|
||||
#pragma once
|
||||
#include <string>
|
||||
#include <map>
|
||||
|
||||
#include "../index/knowhere/knowhere/index/vector_index/VecIndex.h"
|
||||
|
||||
struct LoadIndexInfo {
|
||||
std::string field_name;
|
||||
int64_t field_id;
|
||||
std::map<std::string, std::string> index_params;
|
||||
milvus::knowhere::VecIndexPtr index;
|
||||
};
|
|
@ -11,7 +11,8 @@ set(SEGCORE_FILES
|
|||
InsertRecord.cpp
|
||||
Reduce.cpp
|
||||
plan_c.cpp
|
||||
reduce_c.cpp)
|
||||
reduce_c.cpp
|
||||
load_index_c.cpp)
|
||||
add_library(milvus_segcore SHARED
|
||||
${SEGCORE_FILES}
|
||||
)
|
||||
|
|
|
@ -0,0 +1,139 @@
|
|||
// Copyright (C) 2019-2020 Zilliz. All rights reserved.
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance
|
||||
// with the License. You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software distributed under the License
|
||||
// is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
|
||||
// or implied. See the License for the specific language governing permissions and limitations under the License
|
||||
|
||||
#include "index/knowhere/knowhere/common/BinarySet.h"
|
||||
#include "index/knowhere/knowhere/index/vector_index/VecIndexFactory.h"
|
||||
#include "segcore/load_index_c.h"
|
||||
#include "common/LoadIndex.h"
|
||||
#include "utils/EasyAssert.h"
|
||||
|
||||
CStatus
|
||||
NewLoadIndexInfo(CLoadIndexInfo* c_load_index_info) {
|
||||
try {
|
||||
auto load_index_info = std::make_unique<LoadIndexInfo>();
|
||||
*c_load_index_info = load_index_info.release();
|
||||
auto status = CStatus();
|
||||
status.error_code = Success;
|
||||
status.error_msg = "";
|
||||
return status;
|
||||
} catch (std::exception& e) {
|
||||
auto status = CStatus();
|
||||
status.error_code = UnexpectedException;
|
||||
status.error_msg = strdup(e.what());
|
||||
return status;
|
||||
}
|
||||
}
|
||||
|
||||
CStatus
|
||||
AppendIndexParam(CLoadIndexInfo c_load_index_info, const char* c_index_key, const char* c_index_value) {
|
||||
try {
|
||||
auto load_index_info = (LoadIndexInfo*)c_load_index_info;
|
||||
std::string index_key(c_index_key);
|
||||
std::string index_value(c_index_value);
|
||||
load_index_info->index_params[index_key] = index_value;
|
||||
|
||||
auto status = CStatus();
|
||||
status.error_code = Success;
|
||||
status.error_msg = "";
|
||||
return status;
|
||||
} catch (std::exception& e) {
|
||||
auto status = CStatus();
|
||||
status.error_code = UnexpectedException;
|
||||
status.error_msg = strdup(e.what());
|
||||
return status;
|
||||
}
|
||||
}
|
||||
|
||||
CStatus
|
||||
AppendFieldInfo(CLoadIndexInfo c_load_index_info, const char* c_field_name, int64_t field_id) {
|
||||
try {
|
||||
auto load_index_info = (LoadIndexInfo*)c_load_index_info;
|
||||
std::string field_name(c_field_name);
|
||||
load_index_info->field_name = field_name;
|
||||
load_index_info->field_id = field_id;
|
||||
|
||||
auto status = CStatus();
|
||||
status.error_code = Success;
|
||||
status.error_msg = "";
|
||||
return status;
|
||||
} catch (std::exception& e) {
|
||||
auto status = CStatus();
|
||||
status.error_code = UnexpectedException;
|
||||
status.error_msg = strdup(e.what());
|
||||
return status;
|
||||
}
|
||||
}
|
||||
|
||||
CStatus
|
||||
AppendIndex(CLoadIndexInfo c_load_index_info, CBinarySet c_binary_set) {
|
||||
try {
|
||||
auto load_index_info = (LoadIndexInfo*)c_load_index_info;
|
||||
auto binary_set = (milvus::knowhere::BinarySet*)c_binary_set;
|
||||
auto& index_params = load_index_info->index_params;
|
||||
bool find_index_type = index_params.count("index_type") > 0 ? true : false;
|
||||
bool find_index_mode = index_params.count("index_mode") > 0 ? true : false;
|
||||
Assert(find_index_mode == true);
|
||||
Assert(find_index_type == true);
|
||||
auto mode = index_params["index_mode"] == "CPU" ? milvus::knowhere::IndexMode::MODE_CPU
|
||||
: milvus::knowhere::IndexMode::MODE_GPU;
|
||||
load_index_info->index =
|
||||
milvus::knowhere::VecIndexFactory::GetInstance().CreateVecIndex(index_params["index_type"], mode);
|
||||
load_index_info->index->Load(*binary_set);
|
||||
|
||||
auto status = CStatus();
|
||||
status.error_code = Success;
|
||||
status.error_msg = "";
|
||||
return status;
|
||||
} catch (std::exception& e) {
|
||||
auto status = CStatus();
|
||||
status.error_code = UnexpectedException;
|
||||
status.error_msg = strdup(e.what());
|
||||
return status;
|
||||
}
|
||||
}
|
||||
|
||||
CStatus
|
||||
NewBinarySet(CBinarySet* c_binary_set) {
|
||||
try {
|
||||
auto binary_set = std::make_unique<milvus::knowhere::BinarySet>();
|
||||
*c_binary_set = binary_set.release();
|
||||
auto status = CStatus();
|
||||
status.error_code = Success;
|
||||
status.error_msg = "";
|
||||
return status;
|
||||
} catch (std::exception& e) {
|
||||
auto status = CStatus();
|
||||
status.error_code = UnexpectedException;
|
||||
status.error_msg = strdup(e.what());
|
||||
return status;
|
||||
}
|
||||
}
|
||||
|
||||
CStatus
|
||||
AppendBinaryIndex(CBinarySet c_binary_set, void* index_binary, int64_t index_size, const char* c_index_key) {
|
||||
try {
|
||||
auto binary_set = (milvus::knowhere::BinarySet*)c_binary_set;
|
||||
std::string index_key(c_index_key);
|
||||
uint8_t* index = (uint8_t*)index_binary;
|
||||
std::shared_ptr<uint8_t[]> data(index);
|
||||
binary_set->Append(index_key, data, index_size);
|
||||
|
||||
auto status = CStatus();
|
||||
status.error_code = Success;
|
||||
status.error_msg = "";
|
||||
return status;
|
||||
} catch (std::exception& e) {
|
||||
auto status = CStatus();
|
||||
status.error_code = UnexpectedException;
|
||||
status.error_msg = strdup(e.what());
|
||||
return status;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,45 @@
|
|||
// Copyright (C) 2019-2020 Zilliz. All rights reserved.
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance
|
||||
// with the License. You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software distributed under the License
|
||||
// is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
|
||||
// or implied. See the License for the specific language governing permissions and limitations under the License
|
||||
|
||||
#ifdef __cplusplus
|
||||
extern "C" {
|
||||
#endif
|
||||
|
||||
#include <stdbool.h>
|
||||
#include <stdlib.h>
|
||||
#include <stdint.h>
|
||||
|
||||
#include "segcore/collection_c.h"
|
||||
|
||||
typedef void* CLoadIndexInfo;
|
||||
typedef void* CBinarySet;
|
||||
|
||||
CStatus
|
||||
NewLoadIndexInfo(CLoadIndexInfo* c_load_index_info);
|
||||
|
||||
CStatus
|
||||
AppendIndexParam(CLoadIndexInfo c_load_index_info, const char* index_key, const char* index_value);
|
||||
|
||||
CStatus
|
||||
AppendFieldInfo(CLoadIndexInfo c_load_index_info, const char* field_name, int64_t field_id);
|
||||
|
||||
CStatus
|
||||
AppendIndex(CLoadIndexInfo c_load_index_info, CBinarySet c_binary_set);
|
||||
|
||||
CStatus
|
||||
NewBinarySet(CBinarySet* c_binary_set);
|
||||
|
||||
CStatus
|
||||
AppendBinaryIndex(CBinarySet c_binary_set, void* index_binary, int64_t index_size, const char* c_index_key);
|
||||
|
||||
#ifdef __cplusplus
|
||||
}
|
||||
#endif
|
|
@ -19,6 +19,7 @@
|
|||
#include <knowhere/index/vector_index/VecIndexFactory.h>
|
||||
#include <cstdint>
|
||||
#include <boost/concept_check.hpp>
|
||||
#include "common/LoadIndex.h"
|
||||
|
||||
CSegmentBase
|
||||
NewSegment(CCollection collection, uint64_t segment_id) {
|
||||
|
@ -173,6 +174,22 @@ FillTargetEntry(CSegmentBase c_segment, CPlan c_plan, CQueryResult c_result) {
|
|||
return status;
|
||||
}
|
||||
|
||||
CStatus
|
||||
UpdateSegmentIndex(CSegmentBase c_segment, CLoadIndexInfo c_load_index_info) {
|
||||
auto load_index_info = (LoadIndexInfo*)c_load_index_info;
|
||||
try {
|
||||
auto status = CStatus();
|
||||
status.error_code = Success;
|
||||
status.error_msg = "";
|
||||
return status;
|
||||
} catch (std::exception& e) {
|
||||
auto status = CStatus();
|
||||
status.error_code = UnexpectedException;
|
||||
status.error_msg = strdup(e.what());
|
||||
return status;
|
||||
}
|
||||
}
|
||||
|
||||
//////////////////////////////////////////////////////////////////
|
||||
|
||||
int
|
||||
|
|
|
@ -18,6 +18,7 @@ extern "C" {
|
|||
#include <stdint.h>
|
||||
|
||||
#include "segcore/plan_c.h"
|
||||
#include "segcore/load_index_c.h"
|
||||
|
||||
typedef void* CSegmentBase;
|
||||
typedef void* CQueryResult;
|
||||
|
@ -64,6 +65,8 @@ Search(CSegmentBase c_segment,
|
|||
CStatus
|
||||
FillTargetEntry(CSegmentBase c_segment, CPlan c_plan, CQueryResult result);
|
||||
|
||||
CStatus
|
||||
UpdateSegmentIndex(CSegmentBase c_segment, CLoadIndexInfo c_load_index_info);
|
||||
//////////////////////////////////////////////////////////////////
|
||||
|
||||
int
|
||||
|
|
|
@ -13,11 +13,17 @@
|
|||
#include <string>
|
||||
#include <random>
|
||||
#include <gtest/gtest.h>
|
||||
#include <chrono>
|
||||
|
||||
#include "pb/service_msg.pb.h"
|
||||
#include "segcore/reduce_c.h"
|
||||
|
||||
#include <chrono>
|
||||
#include <index/knowhere/knowhere/index/vector_index/helpers/IndexParameter.h>
|
||||
#include <index/knowhere/knowhere/index/vector_index/adapter/VectorAdapter.h>
|
||||
#include <index/knowhere/knowhere/index/vector_index/VecIndexFactory.h>
|
||||
#include <index/knowhere/knowhere/index/vector_index/IndexIVFPQ.h>
|
||||
#include <common/LoadIndex.h>
|
||||
|
||||
namespace chrono = std::chrono;
|
||||
|
||||
TEST(CApiTest, CollectionTest) {
|
||||
|
@ -338,7 +344,7 @@ TEST(CApiTest, GetMemoryUsageInBytesTest) {
|
|||
namespace {
|
||||
auto
|
||||
generate_data(int N) {
|
||||
std::vector<char> raw_data;
|
||||
std::vector<float> raw_data;
|
||||
std::vector<uint64_t> timestamps;
|
||||
std::vector<int64_t> uids;
|
||||
std::default_random_engine er(42);
|
||||
|
@ -352,7 +358,7 @@ generate_data(int N) {
|
|||
for (auto& x : vec) {
|
||||
x = distribution(er);
|
||||
}
|
||||
raw_data.insert(raw_data.end(), (const char*)std::begin(vec), (const char*)std::end(vec));
|
||||
raw_data.insert(raw_data.end(), std::begin(vec), std::end(vec));
|
||||
int age = ei() % 100;
|
||||
raw_data.insert(raw_data.end(), (const char*)&age, ((const char*)&age) + sizeof(age));
|
||||
}
|
||||
|
@ -678,3 +684,53 @@ TEST(CApiTest, Reduce) {
|
|||
DeleteCollection(collection);
|
||||
DeleteSegment(segment);
|
||||
}
|
||||
|
||||
TEST(CApiTest, LoadIndex_Search) {
|
||||
// generator index
|
||||
constexpr auto DIM = 16;
|
||||
constexpr auto K = 10;
|
||||
|
||||
auto N = 1024 * 1024 * 10;
|
||||
auto num_query = 100;
|
||||
auto [raw_data, timestamps, uids] = generate_data(N);
|
||||
auto indexing = std::make_shared<milvus::knowhere::IVFPQ>();
|
||||
auto conf = milvus::knowhere::Config{{milvus::knowhere::meta::DIM, DIM},
|
||||
{milvus::knowhere::meta::TOPK, K},
|
||||
{milvus::knowhere::IndexParams::nlist, 100},
|
||||
{milvus::knowhere::IndexParams::nprobe, 4},
|
||||
{milvus::knowhere::IndexParams::m, 4},
|
||||
{milvus::knowhere::IndexParams::nbits, 8},
|
||||
{milvus::knowhere::Metric::TYPE, milvus::knowhere::Metric::L2},
|
||||
{milvus::knowhere::meta::DEVICEID, 0}};
|
||||
|
||||
auto database = milvus::knowhere::GenDataset(N, DIM, raw_data.data());
|
||||
indexing->Train(database, conf);
|
||||
indexing->AddWithoutIds(database, conf);
|
||||
|
||||
EXPECT_EQ(indexing->Count(), N);
|
||||
EXPECT_EQ(indexing->Dim(), DIM);
|
||||
|
||||
// serializ index to binarySet
|
||||
auto binary_set = indexing->Serialize(conf);
|
||||
|
||||
// fill loadIndexInfo
|
||||
LoadIndexInfo load_index_info;
|
||||
auto& index_params = load_index_info.index_params;
|
||||
index_params["index_type"] = "IVF_PQ";
|
||||
index_params["index_mode"] = "CPU";
|
||||
auto mode = milvus::knowhere::IndexMode::MODE_CPU;
|
||||
load_index_info.index =
|
||||
milvus::knowhere::VecIndexFactory::GetInstance().CreateVecIndex(index_params["index_type"], mode);
|
||||
load_index_info.index->Load(binary_set);
|
||||
|
||||
// search
|
||||
auto query_dataset = milvus::knowhere::GenDataset(num_query, DIM, raw_data.data() + DIM * 4200);
|
||||
|
||||
auto result = indexing->Query(query_dataset, conf, nullptr);
|
||||
|
||||
auto ids = result->Get<int64_t*>(milvus::knowhere::meta::IDS);
|
||||
auto dis = result->Get<float*>(milvus::knowhere::meta::DISTANCE);
|
||||
for (int i = 0; i < std::min(num_query * K, 100); ++i) {
|
||||
std::cout << ids[i] << "->" << dis[i] << std::endl;
|
||||
}
|
||||
}
|
|
@ -443,39 +443,8 @@ func (s *Master) AssignSegmentID(ctx context.Context, request *internalpb.Assign
|
|||
}, nil
|
||||
}
|
||||
|
||||
func (s *Master) CreateIndex(ctx context.Context, req *internalpb.CreateIndexRequest) (*commonpb.Status, error) {
|
||||
task := &createIndexTask{
|
||||
baseTask: baseTask{
|
||||
sch: s.scheduler,
|
||||
mt: s.metaTable,
|
||||
cv: make(chan error),
|
||||
},
|
||||
req: req,
|
||||
indexBuildScheduler: s.indexBuildSch,
|
||||
indexLoadScheduler: s.indexLoadSch,
|
||||
segManager: s.segmentManager,
|
||||
}
|
||||
|
||||
err := s.scheduler.Enqueue(task)
|
||||
if err != nil {
|
||||
return &commonpb.Status{
|
||||
ErrorCode: commonpb.ErrorCode_UNEXPECTED_ERROR,
|
||||
Reason: "Enqueue failed: " + err.Error(),
|
||||
}, nil
|
||||
}
|
||||
|
||||
err = task.WaitToFinish(ctx)
|
||||
if err != nil {
|
||||
return &commonpb.Status{
|
||||
ErrorCode: commonpb.ErrorCode_UNEXPECTED_ERROR,
|
||||
Reason: "Create Index error: " + err.Error(),
|
||||
}, nil
|
||||
}
|
||||
|
||||
return &commonpb.Status{
|
||||
ErrorCode: commonpb.ErrorCode_SUCCESS,
|
||||
Reason: "",
|
||||
}, nil
|
||||
func (s *Master) CreateIndex(context.Context, *internalpb.CreateIndexRequest) (*commonpb.Status, error) {
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
func (s *Master) DescribeIndex(context.Context, *internalpb.DescribeIndexRequest) (*servicepb.DescribeIndexResponse, error) {
|
||||
|
|
|
@ -1,95 +0,0 @@
|
|||
package master
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb"
|
||||
)
|
||||
|
||||
type createIndexTask struct {
|
||||
baseTask
|
||||
req *internalpb.CreateIndexRequest
|
||||
indexBuildScheduler *IndexBuildScheduler
|
||||
indexLoadScheduler *IndexLoadScheduler
|
||||
segManager *SegmentManager
|
||||
}
|
||||
|
||||
func (task *createIndexTask) Type() internalpb.MsgType {
|
||||
return internalpb.MsgType_kCreateIndex
|
||||
}
|
||||
|
||||
func (task *createIndexTask) Ts() (Timestamp, error) {
|
||||
return task.req.Timestamp, nil
|
||||
}
|
||||
|
||||
func (task *createIndexTask) Execute() error {
|
||||
// modify schema
|
||||
if err := task.mt.UpdateFieldIndexParams(task.req.CollectionName, task.req.FieldName, task.req.ExtraParams); err != nil {
|
||||
return err
|
||||
}
|
||||
// check if closed segment has the same index build history
|
||||
collMeta, err := task.mt.GetCollectionByName(task.req.CollectionName)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
var fieldID int64 = -1
|
||||
for _, fieldSchema := range collMeta.Schema.Fields {
|
||||
if fieldSchema.Name == task.req.FieldName {
|
||||
fieldID = fieldSchema.FieldID
|
||||
break
|
||||
}
|
||||
}
|
||||
if fieldID == -1 {
|
||||
return fmt.Errorf("can not find field name %s", task.req.FieldName)
|
||||
}
|
||||
|
||||
for _, segID := range collMeta.SegmentIDs {
|
||||
segMeta, err := task.mt.GetSegmentByID(segID)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
if segMeta.CloseTime == 0 {
|
||||
continue
|
||||
}
|
||||
hasIndexMeta, err := task.mt.HasFieldIndexMeta(segID, fieldID, task.req.ExtraParams)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
if hasIndexMeta {
|
||||
// load index
|
||||
indexMeta, err := task.mt.GetFieldIndexMeta(segID, fieldID, task.req.ExtraParams)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
err = task.indexLoadScheduler.Enqueue(&IndexLoadInfo{
|
||||
segmentID: segID,
|
||||
fieldID: fieldID,
|
||||
fieldName: task.req.FieldName,
|
||||
indexFilePaths: indexMeta.IndexFilePaths,
|
||||
})
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
} else {
|
||||
// create index
|
||||
for _, kv := range segMeta.BinlogFilePaths {
|
||||
if kv.FieldID != fieldID {
|
||||
continue
|
||||
}
|
||||
err := task.indexBuildScheduler.Enqueue(&IndexBuildInfo{
|
||||
segmentID: segID,
|
||||
fieldID: fieldID,
|
||||
binlogFilePath: kv.BinlogFiles,
|
||||
})
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
break
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// close unfilled segment
|
||||
return task.segManager.ForceClose(collMeta.ID)
|
||||
}
|
|
@ -150,7 +150,7 @@ func CreateServer(ctx context.Context) (*Master, error) {
|
|||
// stats msg stream
|
||||
statsMs := ms.NewPulsarMsgStream(ctx, 1024)
|
||||
statsMs.SetPulsarClient(pulsarAddr)
|
||||
statsMs.CreatePulsarConsumers([]string{Params.QueryNodeStatsChannelName}, "SegmentStats", ms.NewUnmarshalDispatcher(), 1024)
|
||||
statsMs.CreatePulsarConsumers([]string{Params.QueryNodeStatsChannelName}, Params.MsgChannelSubName, ms.NewUnmarshalDispatcher(), 1024)
|
||||
statsMs.Start()
|
||||
|
||||
m := &Master{
|
||||
|
|
|
@ -524,7 +524,7 @@ func (mt *metaTable) saveFieldIndexMetaToEtcd(meta *pb.FieldIndexMeta) error {
|
|||
return mt.client.Save(key, marshaledMeta)
|
||||
}
|
||||
|
||||
func (mt *metaTable) DeleteFieldIndexMeta(segID UniqueID, fieldID UniqueID, indexParams []*commonpb.KeyValuePair) error {
|
||||
func (mt *metaTable) DeleteFieldIndexMeta(segID UniqueID, fieldID UniqueID, indexType string, indexParams []*commonpb.KeyValuePair) error {
|
||||
mt.indexLock.Lock()
|
||||
defer mt.indexLock.Unlock()
|
||||
|
||||
|
@ -568,22 +568,6 @@ func (mt *metaTable) HasFieldIndexMeta(segID UniqueID, fieldID UniqueID, indexPa
|
|||
return false, nil
|
||||
}
|
||||
|
||||
func (mt *metaTable) GetFieldIndexMeta(segID UniqueID, fieldID UniqueID, indexParams []*commonpb.KeyValuePair) (*pb.FieldIndexMeta, error) {
|
||||
mt.indexLock.RLock()
|
||||
defer mt.indexLock.RUnlock()
|
||||
|
||||
if _, ok := mt.segID2IndexMetas[segID]; !ok {
|
||||
return nil, fmt.Errorf("can not find segment %d", segID)
|
||||
}
|
||||
|
||||
for _, v := range mt.segID2IndexMetas[segID] {
|
||||
if v.FieldID == fieldID && typeutil.CompareIndexParams(v.IndexParams, indexParams) {
|
||||
return &v, nil
|
||||
}
|
||||
}
|
||||
return nil, fmt.Errorf("can not find field %d", fieldID)
|
||||
}
|
||||
|
||||
func (mt *metaTable) UpdateFieldIndexMeta(meta *pb.FieldIndexMeta) error {
|
||||
mt.indexLock.Lock()
|
||||
defer mt.indexLock.Unlock()
|
||||
|
@ -651,30 +635,3 @@ func (mt *metaTable) GetFieldIndexParams(collID UniqueID, fieldID UniqueID) ([]*
|
|||
}
|
||||
return nil, fmt.Errorf("can not find field %d in collection %d", fieldID, collID)
|
||||
}
|
||||
|
||||
func (mt *metaTable) UpdateFieldIndexParams(collName string, fieldName string, indexParams []*commonpb.KeyValuePair) error {
|
||||
mt.ddLock.Lock()
|
||||
defer mt.ddLock.Unlock()
|
||||
|
||||
vid, ok := mt.collName2ID[collName]
|
||||
if !ok {
|
||||
return errors.Errorf("can't find collection: " + collName)
|
||||
}
|
||||
meta, ok := mt.collID2Meta[vid]
|
||||
if !ok {
|
||||
return errors.Errorf("can't find collection: " + collName)
|
||||
}
|
||||
|
||||
for _, fieldSchema := range meta.Schema.Fields {
|
||||
if fieldSchema.Name == fieldName {
|
||||
fieldSchema.IndexParams = indexParams
|
||||
if err := mt.saveCollectionMeta(&meta); err != nil {
|
||||
_ = mt.reloadFromKV()
|
||||
return err
|
||||
}
|
||||
return nil
|
||||
}
|
||||
}
|
||||
|
||||
return fmt.Errorf("can not find field with id %s", fieldName)
|
||||
}
|
||||
|
|
|
@ -497,7 +497,7 @@ func TestMetaTable_IndexMeta(t *testing.T) {
|
|||
})
|
||||
assert.Nil(t, err)
|
||||
assert.EqualValues(t, indexbuilderpb.IndexStatus_FINISHED, meta.segID2IndexMetas[1][0].Status)
|
||||
err = meta.DeleteFieldIndexMeta(1, 100, []*commonpb.KeyValuePair{{Key: "k1", Value: "v1"}})
|
||||
err = meta.DeleteFieldIndexMeta(1, 100, "type1", []*commonpb.KeyValuePair{{Key: "k1", Value: "v1"}})
|
||||
assert.Nil(t, err)
|
||||
res, err = meta.HasFieldIndexMeta(1, 100, []*commonpb.KeyValuePair{{Key: "k1", Value: "v1"}})
|
||||
assert.Nil(t, err)
|
||||
|
|
|
@ -358,20 +358,6 @@ func (manager *SegmentManager) initChannelRanges() error {
|
|||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// ForceClose set segments of collection with collID closable, segment will be closed after the assignments of it has expired
|
||||
func (manager *SegmentManager) ForceClose(collID UniqueID) error {
|
||||
status, ok := manager.collStatus[collID]
|
||||
if !ok {
|
||||
return nil
|
||||
}
|
||||
|
||||
for _, segStatus := range status.segments {
|
||||
segStatus.closable = true
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func NewSegmentManager(ctx context.Context,
|
||||
meta *metaTable,
|
||||
globalIDAllocator func() (UniqueID, error),
|
||||
|
|
|
@ -34,6 +34,7 @@ func (dispatcher *UnmarshalDispatcher) addDefaultMsgTemplates() {
|
|||
dropCollectionMsg := DropCollectionMsg{}
|
||||
createPartitionMsg := CreatePartitionMsg{}
|
||||
dropPartitionMsg := DropPartitionMsg{}
|
||||
loadIndexMsg := LoadIndexMsg{}
|
||||
|
||||
queryNodeSegStatsMsg := QueryNodeStatsMsg{}
|
||||
dispatcher.tempMap = make(map[internalPb.MsgType]UnmarshalFunc)
|
||||
|
@ -47,6 +48,7 @@ func (dispatcher *UnmarshalDispatcher) addDefaultMsgTemplates() {
|
|||
dispatcher.tempMap[internalPb.MsgType_kDropCollection] = dropCollectionMsg.Unmarshal
|
||||
dispatcher.tempMap[internalPb.MsgType_kCreatePartition] = createPartitionMsg.Unmarshal
|
||||
dispatcher.tempMap[internalPb.MsgType_kDropPartition] = dropPartitionMsg.Unmarshal
|
||||
dispatcher.tempMap[internalPb.MsgType_kLoadIndex] = loadIndexMsg.Unmarshal
|
||||
|
||||
}
|
||||
|
||||
|
|
|
@ -291,8 +291,10 @@ message Key2SegMsg {
|
|||
message LoadIndex {
|
||||
MsgType msg_type = 1;
|
||||
int64 segmentID = 2;
|
||||
int64 fieldID = 3;
|
||||
repeated string index_paths = 4;
|
||||
string fieldName = 3;
|
||||
int64 fieldID = 4;
|
||||
repeated string index_paths = 5;
|
||||
repeated common.KeyValuePair index_params = 6;
|
||||
}
|
||||
|
||||
message IndexStats {
|
||||
|
|
|
@ -2043,13 +2043,15 @@ func (m *Key2SegMsg) GetKey2Seg() []*Key2Seg {
|
|||
}
|
||||
|
||||
type LoadIndex struct {
|
||||
MsgType MsgType `protobuf:"varint,1,opt,name=msg_type,json=msgType,proto3,enum=milvus.proto.internal.MsgType" json:"msg_type,omitempty"`
|
||||
SegmentID int64 `protobuf:"varint,2,opt,name=segmentID,proto3" json:"segmentID,omitempty"`
|
||||
FieldID int64 `protobuf:"varint,3,opt,name=fieldID,proto3" json:"fieldID,omitempty"`
|
||||
IndexPaths []string `protobuf:"bytes,4,rep,name=index_paths,json=indexPaths,proto3" json:"index_paths,omitempty"`
|
||||
XXX_NoUnkeyedLiteral struct{} `json:"-"`
|
||||
XXX_unrecognized []byte `json:"-"`
|
||||
XXX_sizecache int32 `json:"-"`
|
||||
MsgType MsgType `protobuf:"varint,1,opt,name=msg_type,json=msgType,proto3,enum=milvus.proto.internal.MsgType" json:"msg_type,omitempty"`
|
||||
SegmentID int64 `protobuf:"varint,2,opt,name=segmentID,proto3" json:"segmentID,omitempty"`
|
||||
FieldName string `protobuf:"bytes,3,opt,name=fieldName,proto3" json:"fieldName,omitempty"`
|
||||
FieldID int64 `protobuf:"varint,4,opt,name=fieldID,proto3" json:"fieldID,omitempty"`
|
||||
IndexPaths []string `protobuf:"bytes,5,rep,name=index_paths,json=indexPaths,proto3" json:"index_paths,omitempty"`
|
||||
IndexParams []*commonpb.KeyValuePair `protobuf:"bytes,6,rep,name=index_params,json=indexParams,proto3" json:"index_params,omitempty"`
|
||||
XXX_NoUnkeyedLiteral struct{} `json:"-"`
|
||||
XXX_unrecognized []byte `json:"-"`
|
||||
XXX_sizecache int32 `json:"-"`
|
||||
}
|
||||
|
||||
func (m *LoadIndex) Reset() { *m = LoadIndex{} }
|
||||
|
@ -2091,6 +2093,13 @@ func (m *LoadIndex) GetSegmentID() int64 {
|
|||
return 0
|
||||
}
|
||||
|
||||
func (m *LoadIndex) GetFieldName() string {
|
||||
if m != nil {
|
||||
return m.FieldName
|
||||
}
|
||||
return ""
|
||||
}
|
||||
|
||||
func (m *LoadIndex) GetFieldID() int64 {
|
||||
if m != nil {
|
||||
return m.FieldID
|
||||
|
@ -2105,6 +2114,13 @@ func (m *LoadIndex) GetIndexPaths() []string {
|
|||
return nil
|
||||
}
|
||||
|
||||
func (m *LoadIndex) GetIndexParams() []*commonpb.KeyValuePair {
|
||||
if m != nil {
|
||||
return m.IndexParams
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
type IndexStats struct {
|
||||
IndexParams []*commonpb.KeyValuePair `protobuf:"bytes,1,rep,name=index_params,json=indexParams,proto3" json:"index_params,omitempty"`
|
||||
NumRelatedSegments int64 `protobuf:"varint,2,opt,name=num_related_segments,json=numRelatedSegments,proto3" json:"num_related_segments,omitempty"`
|
||||
|
@ -2637,120 +2653,120 @@ func init() {
|
|||
func init() { proto.RegisterFile("internal_msg.proto", fileDescriptor_7eb37f6b80b23116) }
|
||||
|
||||
var fileDescriptor_7eb37f6b80b23116 = []byte{
|
||||
// 1831 bytes of a gzipped FileDescriptorProto
|
||||
// 1837 bytes of a gzipped FileDescriptorProto
|
||||
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xec, 0x59, 0xcd, 0x6f, 0x23, 0x49,
|
||||
0x15, 0xdf, 0xee, 0xf6, 0x47, 0xfc, 0xec, 0x38, 0x3d, 0x95, 0x64, 0xc6, 0xb3, 0xb3, 0xec, 0x64,
|
||||
0x7a, 0x10, 0x1b, 0x16, 0x91, 0x40, 0x86, 0x03, 0x7b, 0x83, 0xc4, 0x5a, 0xd6, 0x0c, 0x19, 0x85,
|
||||
0x4e, 0xb4, 0x48, 0x68, 0xa5, 0x56, 0xc7, 0x7e, 0xb1, 0x4b, 0xfd, 0xe5, 0x54, 0xb5, 0x27, 0xf1,
|
||||
0x1c, 0x90, 0x90, 0xe6, 0x8c, 0xf8, 0x10, 0x07, 0x4e, 0x70, 0x87, 0x1d, 0xb1, 0x20, 0xfe, 0x07,
|
||||
0x3e, 0x2f, 0xfc, 0x17, 0x70, 0x00, 0x89, 0x5d, 0x0e, 0xdc, 0x50, 0x55, 0xf5, 0x87, 0x3b, 0xb1,
|
||||
0x9d, 0x88, 0x64, 0x96, 0x59, 0xed, 0xdc, 0xaa, 0x5e, 0x97, 0xab, 0xde, 0xef, 0xf7, 0x5e, 0xbd,
|
||||
0x7a, 0xef, 0x19, 0x08, 0x0d, 0x63, 0x64, 0xa1, 0xeb, 0x3b, 0x01, 0xef, 0x6f, 0x0c, 0x59, 0x14,
|
||||
0x47, 0x64, 0x35, 0xa0, 0xfe, 0xe3, 0x11, 0x57, 0xb3, 0x8d, 0x74, 0xc1, 0xab, 0x8d, 0x6e, 0x14,
|
||||
0x04, 0x51, 0xa8, 0xc4, 0xaf, 0xde, 0xe0, 0xc8, 0x1e, 0xd3, 0x2e, 0xe6, 0xbf, 0xb3, 0x42, 0xa8,
|
||||
0x75, 0xda, 0x36, 0x1e, 0x8f, 0x90, 0xc7, 0xe4, 0x26, 0x54, 0x86, 0x88, 0xac, 0xd3, 0x6e, 0x69,
|
||||
0x6b, 0xda, 0xba, 0x61, 0x27, 0x33, 0xf2, 0x00, 0x4a, 0x2c, 0xf2, 0xb1, 0xa5, 0xaf, 0x69, 0xeb,
|
||||
0xcd, 0xad, 0xbb, 0x1b, 0x53, 0xcf, 0xda, 0xd8, 0x43, 0x64, 0x76, 0xe4, 0xa3, 0x2d, 0x17, 0x93,
|
||||
0x15, 0x28, 0x77, 0xa3, 0x51, 0x18, 0xb7, 0x8c, 0x35, 0x6d, 0x7d, 0xd1, 0x56, 0x13, 0xab, 0x0f,
|
||||
0x20, 0xce, 0xe3, 0xc3, 0x28, 0xe4, 0x48, 0x1e, 0x40, 0x85, 0xc7, 0x6e, 0x3c, 0xe2, 0xf2, 0xc0,
|
||||
0xfa, 0xd6, 0x9d, 0xe2, 0xd6, 0x89, 0xf2, 0xfb, 0x72, 0x89, 0x9d, 0x2c, 0x25, 0x4d, 0xd0, 0x3b,
|
||||
0x6d, 0xa9, 0x8b, 0x61, 0xeb, 0x9d, 0xf6, 0x8c, 0x83, 0x22, 0x80, 0x03, 0x1e, 0x7d, 0x8c, 0xc8,
|
||||
0x1e, 0x43, 0x5d, 0x1e, 0x78, 0x15, 0x68, 0xaf, 0x41, 0x2d, 0xa6, 0x01, 0xf2, 0xd8, 0x0d, 0x86,
|
||||
0x52, 0xa7, 0x92, 0x9d, 0x0b, 0x66, 0x9c, 0xfb, 0x54, 0x83, 0xc6, 0x3e, 0xf6, 0x73, 0x2b, 0x66,
|
||||
0xcb, 0xb4, 0x89, 0x65, 0x62, 0xeb, 0xee, 0xc0, 0x0d, 0x43, 0xf4, 0x13, 0xf2, 0xca, 0x76, 0x2e,
|
||||
0x20, 0x77, 0xa0, 0xd6, 0x8d, 0x7c, 0xdf, 0x09, 0xdd, 0x00, 0xe5, 0xf6, 0x35, 0x7b, 0x41, 0x08,
|
||||
0x1e, 0xb9, 0x01, 0x92, 0xfb, 0xb0, 0x38, 0x74, 0x59, 0x4c, 0x63, 0x1a, 0x85, 0x4e, 0xec, 0xf6,
|
||||
0x5b, 0x25, 0xb9, 0xa0, 0x91, 0x09, 0x0f, 0xdc, 0xbe, 0xf5, 0x4c, 0x03, 0xf2, 0x75, 0xce, 0x69,
|
||||
0x3f, 0x2c, 0x28, 0x73, 0xad, 0xc4, 0x3f, 0x84, 0xa5, 0x21, 0x32, 0x27, 0x51, 0xdb, 0x61, 0x78,
|
||||
0xdc, 0x32, 0xd6, 0x8c, 0xf5, 0xfa, 0xd6, 0xfd, 0x19, 0xbf, 0x9f, 0x54, 0xc5, 0x5e, 0x1c, 0x22,
|
||||
0xdb, 0x51, 0x3f, 0xb5, 0xf1, 0xd8, 0xfa, 0x50, 0x83, 0x25, 0xf9, 0x5d, 0x69, 0x1d, 0x60, 0x28,
|
||||
0xa9, 0xe3, 0x42, 0x94, 0x28, 0xab, 0x26, 0x17, 0x50, 0x37, 0xd5, 0x2a, 0x45, 0x42, 0x4b, 0x17,
|
||||
0x11, 0x5a, 0x3e, 0x4f, 0x28, 0xb9, 0x0b, 0x75, 0x3c, 0x1d, 0x52, 0x86, 0x8e, 0xf0, 0x80, 0x56,
|
||||
0x45, 0x7a, 0x03, 0x28, 0xd1, 0x01, 0x0d, 0x26, 0x3d, 0xac, 0x7a, 0x69, 0x0f, 0xb3, 0x38, 0x2c,
|
||||
0x17, 0xac, 0x94, 0x78, 0xeb, 0x7b, 0x70, 0x73, 0x92, 0x59, 0x37, 0xa3, 0xa4, 0xa5, 0x49, 0x82,
|
||||
0x3f, 0x37, 0x8f, 0xe0, 0x9c, 0x40, 0x7b, 0x25, 0xe7, 0x38, 0x97, 0x5a, 0xff, 0xd1, 0xe0, 0xd6,
|
||||
0x0e, 0x43, 0x37, 0xc6, 0x9d, 0xc8, 0xf7, 0xb1, 0x2b, 0x20, 0xa6, 0x0e, 0xf2, 0x16, 0x2c, 0x04,
|
||||
0xbc, 0xef, 0xc4, 0xe3, 0x21, 0x4a, 0xd6, 0x9b, 0x5b, 0xaf, 0xcf, 0x38, 0x6b, 0x97, 0xf7, 0x0f,
|
||||
0xc6, 0x43, 0xb4, 0xab, 0x81, 0x1a, 0x10, 0x0b, 0x1a, 0xdd, 0x6c, 0xbf, 0x2c, 0x24, 0x14, 0x64,
|
||||
0xc2, 0x3a, 0x0c, 0x8f, 0x3b, 0x6d, 0x69, 0x1d, 0xc3, 0x56, 0x93, 0xe2, 0x3d, 0x2b, 0x9d, 0xbd,
|
||||
0x67, 0x2d, 0xa8, 0x0e, 0x59, 0x74, 0x3a, 0xee, 0xb4, 0xa5, 0x61, 0x0c, 0x3b, 0x9d, 0x92, 0x2f,
|
||||
0x43, 0x85, 0x77, 0x07, 0x18, 0xb8, 0xd2, 0x1c, 0xf5, 0xad, 0xdb, 0x53, 0x29, 0xdf, 0xf6, 0xa3,
|
||||
0x43, 0x3b, 0x59, 0x68, 0xfd, 0x44, 0x87, 0xd5, 0x36, 0x8b, 0x86, 0x9f, 0x70, 0xe4, 0xbb, 0xb0,
|
||||
0x94, 0xef, 0xae, 0xbc, 0x5a, 0x51, 0xf0, 0xd9, 0xa2, 0xce, 0xc9, 0x0b, 0xb3, 0x91, 0xc3, 0x15,
|
||||
0x1e, 0x6f, 0x37, 0xbb, 0x85, 0xb9, 0xf5, 0x0f, 0x0d, 0x56, 0xde, 0x71, 0xf9, 0xb5, 0x92, 0x92,
|
||||
0x01, 0xd6, 0x67, 0x02, 0x36, 0xe6, 0x00, 0x2e, 0x5d, 0x08, 0xb8, 0x7c, 0x05, 0xc0, 0x1f, 0x6a,
|
||||
0x70, 0xbb, 0x8d, 0xbc, 0xcb, 0xe8, 0x21, 0x7e, 0x7a, 0x50, 0xff, 0x42, 0x83, 0xd5, 0xfd, 0x41,
|
||||
0x74, 0xf2, 0xe2, 0x22, 0xb6, 0x7e, 0xab, 0xc3, 0x4d, 0x15, 0x9b, 0xf6, 0xd2, 0xe8, 0xfb, 0x31,
|
||||
0x5d, 0xd0, 0x35, 0xa8, 0x67, 0x01, 0x3f, 0xbb, 0xa6, 0x93, 0xa2, 0x1c, 0x69, 0x69, 0x26, 0xd2,
|
||||
0xf2, 0x1c, 0xa4, 0x95, 0xa2, 0x6d, 0xbf, 0x09, 0xcd, 0xfc, 0xd5, 0x91, 0xa6, 0x55, 0xef, 0xc6,
|
||||
0xfd, 0xe9, 0xa6, 0xcd, 0xe8, 0x90, 0x96, 0xcd, 0x1f, 0x2c, 0x69, 0xd8, 0x0f, 0x74, 0x58, 0x11,
|
||||
0x51, 0xed, 0x25, 0x67, 0x97, 0xe7, 0xec, 0xef, 0x1a, 0x2c, 0xbf, 0xe3, 0xf2, 0xeb, 0xa4, 0xec,
|
||||
0x7a, 0x2f, 0xff, 0x79, 0xb0, 0xe5, 0xff, 0x19, 0xec, 0x3f, 0x35, 0x68, 0xa5, 0xf1, 0xee, 0xd3,
|
||||
0x81, 0x58, 0x3c, 0x69, 0x22, 0xd6, 0xbd, 0xb8, 0x68, 0xaf, 0x39, 0xb8, 0xff, 0x4b, 0x87, 0xc5,
|
||||
0x4e, 0xc8, 0x91, 0xc5, 0xcf, 0x0d, 0xe9, 0x1b, 0xe7, 0x35, 0x56, 0xc5, 0xc9, 0x19, 0x5d, 0x2e,
|
||||
0x55, 0xa2, 0x08, 0xde, 0x38, 0xf6, 0x45, 0x46, 0x9a, 0xe5, 0x37, 0xb9, 0xa0, 0x98, 0xe5, 0xab,
|
||||
0x30, 0x30, 0x91, 0xe5, 0x4f, 0xb0, 0x5a, 0x2d, 0xb2, 0xfa, 0x3a, 0x40, 0x46, 0x3e, 0x6f, 0x2d,
|
||||
0xac, 0x19, 0x22, 0x4d, 0xcf, 0x25, 0xa2, 0x02, 0x62, 0xd1, 0x49, 0xa7, 0xcd, 0x5b, 0xb5, 0x35,
|
||||
0x43, 0x54, 0x40, 0x6a, 0x46, 0xbe, 0x02, 0x0b, 0x2c, 0x3a, 0x71, 0x7a, 0x6e, 0xec, 0xb6, 0x40,
|
||||
0x26, 0xd9, 0x73, 0xb2, 0xc9, 0x2a, 0x8b, 0x4e, 0xda, 0x6e, 0xec, 0x5a, 0x4f, 0x75, 0x58, 0x6c,
|
||||
0xa3, 0x8f, 0x31, 0xfe, 0xff, 0x49, 0x2f, 0x30, 0x56, 0x9a, 0xc3, 0x58, 0x79, 0x1e, 0x63, 0x95,
|
||||
0x73, 0x8c, 0xdd, 0x83, 0xc6, 0x90, 0xd1, 0xc0, 0x65, 0x63, 0xc7, 0xc3, 0xb1, 0x28, 0x6f, 0x0c,
|
||||
0x19, 0xe5, 0x95, 0xec, 0x21, 0x8e, 0xb9, 0xf5, 0x91, 0x06, 0x8b, 0xfb, 0xe8, 0xb2, 0xee, 0xe0,
|
||||
0xb9, 0xd1, 0x30, 0xa1, 0xbf, 0x51, 0xd4, 0x7f, 0x7e, 0x0e, 0xfd, 0x79, 0x30, 0x19, 0xf2, 0x91,
|
||||
0x1f, 0x3b, 0x39, 0x39, 0x8a, 0x80, 0x25, 0x25, 0xdf, 0xc9, 0x28, 0xda, 0x84, 0xf2, 0xf1, 0x08,
|
||||
0xd9, 0xf8, 0xe2, 0x6a, 0x42, 0xad, 0xb3, 0xfe, 0xaa, 0x81, 0xb9, 0x3f, 0xe6, 0x3b, 0x51, 0x78,
|
||||
0x44, 0xfb, 0x2f, 0x1c, 0x72, 0x02, 0x25, 0x69, 0xaf, 0xf2, 0x9a, 0xb1, 0x5e, 0xb3, 0xe5, 0x58,
|
||||
0xd8, 0xd2, 0xc3, 0xb1, 0x33, 0x64, 0x78, 0x44, 0x4f, 0x51, 0x59, 0xbb, 0x66, 0xd7, 0x3d, 0x1c,
|
||||
0xef, 0x25, 0x22, 0xeb, 0x99, 0x0e, 0x8d, 0xd4, 0x96, 0x82, 0x9f, 0xab, 0x00, 0xca, 0x6b, 0x62,
|
||||
0xfd, 0xf2, 0x5d, 0x97, 0xe9, 0x95, 0xd2, 0xec, 0x38, 0x7a, 0x0f, 0x1a, 0xd2, 0x1c, 0x4e, 0x18,
|
||||
0xf5, 0x30, 0xb3, 0x6e, 0x5d, 0xca, 0x1e, 0x49, 0x51, 0x91, 0xa8, 0xca, 0x65, 0x5c, 0xa4, 0x3a,
|
||||
0xdd, 0x45, 0x08, 0x94, 0x06, 0x34, 0x56, 0x71, 0xa5, 0x61, 0xcb, 0xb1, 0xf5, 0x3d, 0xa8, 0x1f,
|
||||
0xd0, 0x00, 0x0f, 0x68, 0xd7, 0xdb, 0xe5, 0xfd, 0xab, 0xd0, 0x95, 0x77, 0x67, 0xf4, 0x42, 0x77,
|
||||
0x66, 0xee, 0x0b, 0x63, 0x7d, 0x5f, 0x83, 0x85, 0xb7, 0xfd, 0x11, 0x1f, 0x5c, 0xf1, 0xf4, 0x42,
|
||||
0x3c, 0xd6, 0xa7, 0xc4, 0xe3, 0x39, 0x3a, 0xfc, 0x4c, 0x83, 0xea, 0x43, 0x1c, 0x6f, 0xed, 0x63,
|
||||
0x5f, 0xda, 0x4f, 0xc4, 0xd4, 0xb4, 0x6b, 0x23, 0x27, 0xe4, 0x2e, 0xd4, 0x27, 0xa2, 0x48, 0xb2,
|
||||
0x3f, 0xe4, 0x41, 0xe4, 0x82, 0x67, 0xf4, 0x36, 0x2c, 0x50, 0xee, 0x3c, 0x76, 0x7d, 0xda, 0x93,
|
||||
0xf6, 0x5f, 0xb0, 0xab, 0x94, 0xbf, 0x2b, 0xa6, 0x22, 0x7e, 0x65, 0x6a, 0x2a, 0x6f, 0x37, 0xec,
|
||||
0x09, 0x89, 0xf5, 0x1e, 0x40, 0xa2, 0x9a, 0x20, 0x28, 0xf3, 0x2e, 0x6d, 0xd2, 0xbb, 0xbe, 0x0a,
|
||||
0x55, 0x0f, 0xc7, 0x5b, 0x1c, 0xfb, 0x2d, 0x5d, 0x06, 0xff, 0x59, 0xac, 0x25, 0x3b, 0xd9, 0xe9,
|
||||
0x72, 0xeb, 0xe7, 0x1a, 0xd4, 0xbe, 0x15, 0xb9, 0xbd, 0x4e, 0xd8, 0xc3, 0xd3, 0xe7, 0x47, 0x7f,
|
||||
0x0b, 0xaa, 0x47, 0x14, 0xfd, 0x5e, 0x1e, 0x04, 0x92, 0xa9, 0x20, 0x96, 0x8a, 0xb3, 0x9d, 0xa1,
|
||||
0x1b, 0x0f, 0x78, 0xab, 0x24, 0x6f, 0x34, 0x48, 0xd1, 0x9e, 0x90, 0x58, 0x4f, 0x35, 0x00, 0xa9,
|
||||
0x9d, 0xb8, 0x67, 0x9c, 0xb4, 0xa1, 0x91, 0xae, 0x67, 0x6e, 0xc0, 0x93, 0x8e, 0xd2, 0xbd, 0xa9,
|
||||
0x37, 0xf3, 0x21, 0x8e, 0xdf, 0x75, 0xfd, 0x11, 0xee, 0xb9, 0x94, 0xd9, 0xf5, 0x64, 0x4f, 0xf1,
|
||||
0x2b, 0xf2, 0x25, 0x58, 0x09, 0x47, 0x81, 0xc3, 0xd0, 0x77, 0x63, 0xec, 0x39, 0x89, 0xa2, 0x3c,
|
||||
0x51, 0x9c, 0x84, 0xa3, 0xc0, 0x56, 0x9f, 0xf6, 0x93, 0x2f, 0xd6, 0x0f, 0x34, 0x80, 0xb7, 0x85,
|
||||
0xce, 0x4a, 0x8d, 0xb3, 0xe5, 0x85, 0x36, 0xa5, 0xbc, 0x98, 0x00, 0xad, 0x17, 0x41, 0x6f, 0xa7,
|
||||
0xa0, 0x45, 0xcc, 0xe0, 0x49, 0xdb, 0xf1, 0xde, 0x0c, 0xaa, 0x73, 0xf0, 0x09, 0x2f, 0x72, 0x6c,
|
||||
0xfd, 0x54, 0x75, 0x6a, 0x85, 0x76, 0x4a, 0xa5, 0x82, 0x05, 0xb4, 0xb3, 0x16, 0xb8, 0x0b, 0xf5,
|
||||
0x00, 0x83, 0x88, 0x8d, 0x1d, 0x4e, 0x9f, 0x60, 0xea, 0xc0, 0x4a, 0xb4, 0x4f, 0x9f, 0xa0, 0x70,
|
||||
0x51, 0x49, 0x49, 0x74, 0xc2, 0x53, 0x1b, 0x09, 0x1a, 0xa2, 0x13, 0x4e, 0xbe, 0x00, 0x37, 0x18,
|
||||
0x76, 0x31, 0x8c, 0xfd, 0xb1, 0x13, 0x44, 0x3d, 0x7a, 0x44, 0x31, 0x75, 0x63, 0x33, 0xfd, 0xb0,
|
||||
0x9b, 0xc8, 0xad, 0xbf, 0x69, 0xd0, 0xfc, 0x76, 0x1a, 0xbc, 0x94, 0x66, 0xcf, 0x21, 0xa6, 0x7c,
|
||||
0x4d, 0x82, 0x2d, 0xf0, 0x37, 0xa7, 0x6d, 0x9b, 0x91, 0x64, 0x2f, 0x70, 0xec, 0x2b, 0xa5, 0xb6,
|
||||
0xa1, 0x2e, 0xcd, 0x91, 0xec, 0x51, 0x9a, 0x6b, 0x83, 0xdc, 0xf2, 0x36, 0x1c, 0x65, 0x63, 0xeb,
|
||||
0x97, 0x3a, 0x10, 0x55, 0xee, 0x4b, 0x23, 0xbd, 0x70, 0x39, 0xfa, 0x1b, 0xd3, 0x73, 0xf4, 0xf3,
|
||||
0xc9, 0xd7, 0x67, 0x40, 0xc1, 0xca, 0x7b, 0x71, 0x35, 0xbb, 0x26, 0x25, 0xf2, 0x73, 0x1b, 0x1a,
|
||||
0x78, 0x1a, 0x33, 0x37, 0xbd, 0x74, 0xd5, 0x4b, 0x5f, 0x3a, 0xf9, 0x33, 0x75, 0xe9, 0xac, 0xf7,
|
||||
0x45, 0x99, 0x9f, 0x54, 0x71, 0x2f, 0xf9, 0xba, 0x98, 0xaf, 0xdf, 0xe9, 0xf0, 0x5a, 0x81, 0xaf,
|
||||
0x3d, 0x16, 0xf5, 0x19, 0x72, 0xfe, 0x92, 0xb7, 0x79, 0xbc, 0xbd, 0xf9, 0x17, 0x03, 0xaa, 0x09,
|
||||
0x60, 0x52, 0x83, 0xb2, 0xf7, 0x28, 0x0a, 0xd1, 0x7c, 0x85, 0xac, 0xc2, 0x0d, 0xef, 0xec, 0xff,
|
||||
0x06, 0x66, 0x8f, 0x2c, 0xc3, 0x92, 0x57, 0x6c, 0xa9, 0x9b, 0x48, 0x08, 0x34, 0xbd, 0x42, 0x47,
|
||||
0xd9, 0x3c, 0x22, 0xb7, 0x60, 0xd9, 0x3b, 0xdf, 0x74, 0x35, 0xc5, 0x9b, 0x6c, 0x7a, 0xc5, 0xbe,
|
||||
0x24, 0x37, 0x07, 0x72, 0x8b, 0x6f, 0x60, 0x9c, 0x25, 0xd8, 0xdc, 0xa4, 0x64, 0x15, 0x4c, 0xef,
|
||||
0x4c, 0x7b, 0xd0, 0xfc, 0xbd, 0x46, 0x96, 0xa1, 0xe9, 0x15, 0xfa, 0x5f, 0xe6, 0x1f, 0x34, 0x42,
|
||||
0x60, 0xd1, 0x9b, 0x6c, 0xf0, 0x98, 0x7f, 0xd4, 0xc8, 0x2d, 0x20, 0xde, 0xb9, 0x3e, 0x88, 0xf9,
|
||||
0x27, 0x8d, 0xac, 0xc0, 0x92, 0x57, 0x68, 0x17, 0x70, 0xf3, 0xcf, 0x1a, 0xb9, 0x01, 0x0d, 0x6f,
|
||||
0x22, 0x3c, 0x99, 0xbf, 0xd2, 0xd5, 0x51, 0x93, 0x3e, 0x65, 0xbe, 0xaf, 0x93, 0x3b, 0x70, 0xd3,
|
||||
0x9b, 0xea, 0x68, 0xe6, 0x33, 0x9d, 0x34, 0xa0, 0xea, 0xa9, 0xc2, 0xdc, 0xfc, 0xa1, 0x21, 0x67,
|
||||
0xaa, 0x62, 0x34, 0x7f, 0x64, 0x90, 0x3a, 0x54, 0x3c, 0x99, 0xbb, 0x99, 0x3f, 0x56, 0x9f, 0x54,
|
||||
0xe6, 0x6d, 0x7e, 0x64, 0x48, 0xf5, 0x27, 0xf3, 0x70, 0xf3, 0xdf, 0x06, 0x69, 0x42, 0xcd, 0x4b,
|
||||
0x73, 0x4d, 0xf3, 0xd7, 0x35, 0xa9, 0x75, 0xf1, 0xa9, 0x30, 0x3f, 0xa8, 0x91, 0x25, 0x00, 0x2f,
|
||||
0x4b, 0x49, 0xcc, 0xdf, 0xd4, 0xde, 0x7c, 0x0b, 0x16, 0xd2, 0xbf, 0xee, 0x08, 0x40, 0x65, 0xd7,
|
||||
0xe5, 0x31, 0x32, 0xf3, 0x15, 0x31, 0xb6, 0xd1, 0xed, 0x21, 0x33, 0x35, 0x31, 0xfe, 0x0e, 0xa3,
|
||||
0x42, 0xae, 0x0b, 0x9b, 0xef, 0x09, 0xc7, 0x34, 0x8d, 0xed, 0xf6, 0x77, 0xb7, 0xfb, 0x34, 0x1e,
|
||||
0x8c, 0x0e, 0x85, 0xd7, 0x6c, 0x3e, 0xa1, 0xbe, 0x4f, 0x9f, 0xc4, 0xd8, 0x1d, 0x6c, 0x2a, 0x8f,
|
||||
0xfa, 0x62, 0x8f, 0xf2, 0x98, 0xd1, 0xc3, 0x51, 0x8c, 0xbd, 0xcd, 0xf4, 0xb2, 0x6c, 0x4a, 0x37,
|
||||
0xcb, 0xa6, 0xc3, 0xc3, 0xc3, 0x8a, 0x94, 0x3c, 0xf8, 0x6f, 0x00, 0x00, 0x00, 0xff, 0xff, 0x40,
|
||||
0xb8, 0xbd, 0x0b, 0x2b, 0x1f, 0x00, 0x00,
|
||||
0x1c, 0x90, 0x10, 0x73, 0x46, 0x7c, 0x88, 0x03, 0x37, 0xee, 0xb0, 0x23, 0x16, 0xc4, 0xff, 0xc0,
|
||||
0xe7, 0x85, 0xff, 0x02, 0x0e, 0x20, 0xb1, 0xcb, 0x81, 0x1b, 0xaa, 0xaa, 0xfe, 0x70, 0x27, 0xb6,
|
||||
0x13, 0x6d, 0x32, 0xcb, 0xa0, 0x9d, 0x5b, 0xbd, 0xe7, 0xea, 0xaa, 0xf7, 0x7e, 0xef, 0xa3, 0xde,
|
||||
0x7b, 0x06, 0x42, 0xc3, 0x18, 0x59, 0xe8, 0xfa, 0x4e, 0xc0, 0xfb, 0x1b, 0x43, 0x16, 0xc5, 0x11,
|
||||
0x59, 0x0d, 0xa8, 0xff, 0x78, 0xc4, 0x15, 0xb5, 0x91, 0x6e, 0x78, 0xb5, 0xd1, 0x8d, 0x82, 0x20,
|
||||
0x0a, 0x15, 0xfb, 0xd5, 0x1b, 0x1c, 0xd9, 0x63, 0xda, 0xc5, 0xfc, 0x3b, 0x2b, 0x84, 0x5a, 0xa7,
|
||||
0x6d, 0xe3, 0xf1, 0x08, 0x79, 0x4c, 0x6e, 0x42, 0x65, 0x88, 0xc8, 0x3a, 0xed, 0x96, 0xb6, 0xa6,
|
||||
0xad, 0x1b, 0x76, 0x42, 0x91, 0x07, 0x50, 0x62, 0x91, 0x8f, 0x2d, 0x7d, 0x4d, 0x5b, 0x6f, 0x6e,
|
||||
0xdd, 0xdd, 0x98, 0x7a, 0xd7, 0xc6, 0x1e, 0x22, 0xb3, 0x23, 0x1f, 0x6d, 0xb9, 0x99, 0xac, 0x40,
|
||||
0xb9, 0x1b, 0x8d, 0xc2, 0xb8, 0x65, 0xac, 0x69, 0xeb, 0x8b, 0xb6, 0x22, 0xac, 0x3e, 0x80, 0xb8,
|
||||
0x8f, 0x0f, 0xa3, 0x90, 0x23, 0x79, 0x00, 0x15, 0x1e, 0xbb, 0xf1, 0x88, 0xcb, 0x0b, 0xeb, 0x5b,
|
||||
0x77, 0x8a, 0x47, 0x27, 0xc2, 0xef, 0xcb, 0x2d, 0x76, 0xb2, 0x95, 0x34, 0x41, 0xef, 0xb4, 0xa5,
|
||||
0x2c, 0x86, 0xad, 0x77, 0xda, 0x33, 0x2e, 0x8a, 0x00, 0x0e, 0x78, 0xf4, 0x09, 0x6a, 0xf6, 0x18,
|
||||
0xea, 0xf2, 0xc2, 0xab, 0xa8, 0xf6, 0x1a, 0xd4, 0x62, 0x1a, 0x20, 0x8f, 0xdd, 0x60, 0x28, 0x65,
|
||||
0x2a, 0xd9, 0x39, 0x63, 0xc6, 0xbd, 0x4f, 0x35, 0x68, 0xec, 0x63, 0x3f, 0xb7, 0x62, 0xb6, 0x4d,
|
||||
0x9b, 0xd8, 0x26, 0x8e, 0xee, 0x0e, 0xdc, 0x30, 0x44, 0x3f, 0x01, 0xaf, 0x6c, 0xe7, 0x0c, 0x72,
|
||||
0x07, 0x6a, 0xdd, 0xc8, 0xf7, 0x9d, 0xd0, 0x0d, 0x50, 0x1e, 0x5f, 0xb3, 0x17, 0x04, 0xe3, 0x91,
|
||||
0x1b, 0x20, 0xb9, 0x0f, 0x8b, 0x43, 0x97, 0xc5, 0x34, 0xa6, 0x51, 0xe8, 0xc4, 0x6e, 0xbf, 0x55,
|
||||
0x92, 0x1b, 0x1a, 0x19, 0xf3, 0xc0, 0xed, 0x5b, 0xcf, 0x34, 0x20, 0x5f, 0xe7, 0x9c, 0xf6, 0xc3,
|
||||
0x82, 0x30, 0xd7, 0x0a, 0xfc, 0x43, 0x58, 0x1a, 0x22, 0x73, 0x12, 0xb1, 0x1d, 0x86, 0xc7, 0x2d,
|
||||
0x63, 0xcd, 0x58, 0xaf, 0x6f, 0xdd, 0x9f, 0xf1, 0xfd, 0xa4, 0x28, 0xf6, 0xe2, 0x10, 0xd9, 0x8e,
|
||||
0xfa, 0xd4, 0xc6, 0x63, 0xeb, 0x43, 0x0d, 0x96, 0xe4, 0xef, 0x4a, 0xea, 0x00, 0x43, 0x09, 0x1d,
|
||||
0x17, 0xac, 0x44, 0x58, 0x45, 0x5c, 0x00, 0xdd, 0x54, 0xab, 0x14, 0x01, 0x2d, 0x5d, 0x04, 0x68,
|
||||
0xf9, 0x3c, 0xa0, 0xe4, 0x2e, 0xd4, 0xf1, 0x74, 0x48, 0x19, 0x3a, 0xc2, 0x03, 0x5a, 0x15, 0xe9,
|
||||
0x0d, 0xa0, 0x58, 0x07, 0x34, 0x98, 0xf4, 0xb0, 0xea, 0xa5, 0x3d, 0xcc, 0xe2, 0xb0, 0x5c, 0xb0,
|
||||
0x52, 0xe2, 0xad, 0xef, 0xc1, 0xcd, 0x49, 0x64, 0xdd, 0x0c, 0x92, 0x96, 0x26, 0x01, 0xfe, 0xdc,
|
||||
0x3c, 0x80, 0x73, 0x00, 0xed, 0x95, 0x1c, 0xe3, 0x9c, 0x6b, 0xfd, 0x47, 0x83, 0x5b, 0x3b, 0x0c,
|
||||
0xdd, 0x18, 0x77, 0x22, 0xdf, 0xc7, 0xae, 0x50, 0x31, 0x75, 0x90, 0xb7, 0x60, 0x21, 0xe0, 0x7d,
|
||||
0x27, 0x1e, 0x0f, 0x51, 0xa2, 0xde, 0xdc, 0x7a, 0x7d, 0xc6, 0x5d, 0xbb, 0xbc, 0x7f, 0x30, 0x1e,
|
||||
0xa2, 0x5d, 0x0d, 0xd4, 0x82, 0x58, 0xd0, 0xe8, 0x66, 0xe7, 0x65, 0x29, 0xa1, 0xc0, 0x13, 0xd6,
|
||||
0x61, 0x78, 0xdc, 0x69, 0x4b, 0xeb, 0x18, 0xb6, 0x22, 0x8a, 0x71, 0x56, 0x3a, 0x1b, 0x67, 0x2d,
|
||||
0xa8, 0x0e, 0x59, 0x74, 0x3a, 0xee, 0xb4, 0xa5, 0x61, 0x0c, 0x3b, 0x25, 0xc9, 0x97, 0xa1, 0xc2,
|
||||
0xbb, 0x03, 0x0c, 0x5c, 0x69, 0x8e, 0xfa, 0xd6, 0xed, 0xa9, 0x90, 0x6f, 0xfb, 0xd1, 0xa1, 0x9d,
|
||||
0x6c, 0xb4, 0x7e, 0xaa, 0xc3, 0x6a, 0x9b, 0x45, 0xc3, 0xff, 0x73, 0xcd, 0x77, 0x61, 0x29, 0x3f,
|
||||
0x5d, 0x79, 0xb5, 0x82, 0xe0, 0xb3, 0x45, 0x99, 0x93, 0x17, 0x66, 0x23, 0x57, 0x57, 0x78, 0xbc,
|
||||
0xdd, 0xec, 0x16, 0x68, 0xeb, 0x1f, 0x1a, 0xac, 0xbc, 0xe3, 0xf2, 0x6b, 0x05, 0x25, 0x53, 0x58,
|
||||
0x9f, 0xa9, 0xb0, 0x31, 0x47, 0xe1, 0xd2, 0x85, 0x0a, 0x97, 0xaf, 0xa0, 0xf0, 0x87, 0x1a, 0xdc,
|
||||
0x6e, 0x23, 0xef, 0x32, 0x7a, 0x88, 0x9f, 0x1e, 0xad, 0x7f, 0xa1, 0xc1, 0xea, 0xfe, 0x20, 0x3a,
|
||||
0x79, 0x71, 0x35, 0xb6, 0x7e, 0xab, 0xc3, 0x4d, 0x95, 0x9b, 0xf6, 0xd2, 0xec, 0xfb, 0x09, 0x05,
|
||||
0xe8, 0x1a, 0xd4, 0xb3, 0x84, 0x9f, 0x85, 0xe9, 0x24, 0x2b, 0xd7, 0xb4, 0x34, 0x53, 0xd3, 0xf2,
|
||||
0x1c, 0x4d, 0x2b, 0x45, 0xdb, 0x7e, 0x13, 0x9a, 0xf9, 0xab, 0x23, 0x4d, 0xab, 0xde, 0x8d, 0xfb,
|
||||
0xd3, 0x4d, 0x9b, 0xc1, 0x21, 0x2d, 0x9b, 0x3f, 0x58, 0xd2, 0xb0, 0x1f, 0xe8, 0xb0, 0x22, 0xb2,
|
||||
0xda, 0x4b, 0xcc, 0x2e, 0x8f, 0xd9, 0xdf, 0x35, 0x58, 0x7e, 0xc7, 0xe5, 0xd7, 0x09, 0xd9, 0xf5,
|
||||
0x06, 0xff, 0x79, 0x65, 0xcb, 0x1f, 0x5b, 0xd9, 0x7f, 0x6a, 0xd0, 0x4a, 0xf3, 0xdd, 0xa7, 0x43,
|
||||
0x63, 0xf1, 0xa4, 0x89, 0x5c, 0xf7, 0xe2, 0x6a, 0x7b, 0xcd, 0xc9, 0xfd, 0x5f, 0x3a, 0x2c, 0x76,
|
||||
0x42, 0x8e, 0x2c, 0x7e, 0x6e, 0x9a, 0xbe, 0x71, 0x5e, 0x62, 0xd5, 0x9c, 0x9c, 0x91, 0xe5, 0x52,
|
||||
0x2d, 0x8a, 0xc0, 0x8d, 0x63, 0x5f, 0x54, 0xa4, 0x59, 0x7d, 0x93, 0x33, 0x8a, 0x55, 0xbe, 0x4a,
|
||||
0x03, 0x13, 0x55, 0xfe, 0x04, 0xaa, 0xd5, 0x22, 0xaa, 0xaf, 0x03, 0x64, 0xe0, 0xf3, 0xd6, 0xc2,
|
||||
0x9a, 0x21, 0xca, 0xf4, 0x9c, 0x23, 0x3a, 0x20, 0x16, 0x9d, 0x74, 0xda, 0xbc, 0x55, 0x5b, 0x33,
|
||||
0x44, 0x07, 0xa4, 0x28, 0xf2, 0x15, 0x58, 0x60, 0xd1, 0x89, 0xd3, 0x73, 0x63, 0xb7, 0x05, 0xb2,
|
||||
0xc8, 0x9e, 0x53, 0x4d, 0x56, 0x59, 0x74, 0xd2, 0x76, 0x63, 0xd7, 0x7a, 0xaa, 0xc3, 0x62, 0x1b,
|
||||
0x7d, 0x8c, 0xf1, 0x7f, 0x0f, 0x7a, 0x01, 0xb1, 0xd2, 0x1c, 0xc4, 0xca, 0xf3, 0x10, 0xab, 0x9c,
|
||||
0x43, 0xec, 0x1e, 0x34, 0x86, 0x8c, 0x06, 0x2e, 0x1b, 0x3b, 0x1e, 0x8e, 0x45, 0x7b, 0x63, 0xc8,
|
||||
0x2c, 0xaf, 0x78, 0x0f, 0x71, 0xcc, 0xad, 0x8f, 0x34, 0x58, 0xdc, 0x47, 0x97, 0x75, 0x07, 0xcf,
|
||||
0x0d, 0x86, 0x09, 0xf9, 0x8d, 0xa2, 0xfc, 0xf3, 0x6b, 0xe8, 0xcf, 0x83, 0xc9, 0x90, 0x8f, 0xfc,
|
||||
0xd8, 0xc9, 0xc1, 0x51, 0x00, 0x2c, 0x29, 0xfe, 0x4e, 0x06, 0xd1, 0x26, 0x94, 0x8f, 0x47, 0xc8,
|
||||
0xc6, 0x17, 0x77, 0x13, 0x6a, 0x9f, 0xf5, 0x57, 0x0d, 0xcc, 0xfd, 0x31, 0xdf, 0x89, 0xc2, 0x23,
|
||||
0xda, 0x7f, 0xe1, 0x34, 0x27, 0x50, 0x92, 0xf6, 0x2a, 0xaf, 0x19, 0xeb, 0x35, 0x5b, 0xae, 0x85,
|
||||
0x2d, 0x3d, 0x1c, 0x3b, 0x43, 0x86, 0x47, 0xf4, 0x14, 0x95, 0xb5, 0x6b, 0x76, 0xdd, 0xc3, 0xf1,
|
||||
0x5e, 0xc2, 0xb2, 0x9e, 0xe9, 0xd0, 0x48, 0x6d, 0x29, 0xf0, 0xb9, 0x8a, 0x42, 0x79, 0x4f, 0xac,
|
||||
0x5f, 0x7e, 0xea, 0x32, 0xbd, 0x53, 0x9a, 0x9d, 0x47, 0xef, 0x41, 0x43, 0x9a, 0xc3, 0x09, 0xa3,
|
||||
0x1e, 0x66, 0xd6, 0xad, 0x4b, 0xde, 0x23, 0xc9, 0x2a, 0x02, 0x55, 0xb9, 0x8c, 0x8b, 0x54, 0xa7,
|
||||
0xbb, 0x08, 0x81, 0xd2, 0x80, 0xc6, 0x2a, 0xaf, 0x34, 0x6c, 0xb9, 0xb6, 0xbe, 0x07, 0xf5, 0x03,
|
||||
0x1a, 0xe0, 0x01, 0xed, 0x7a, 0xbb, 0xbc, 0x7f, 0x15, 0xb8, 0xf2, 0xe9, 0x8c, 0x5e, 0x98, 0xce,
|
||||
0xcc, 0x7d, 0x61, 0xac, 0xef, 0x6b, 0xb0, 0xf0, 0xb6, 0x3f, 0xe2, 0x83, 0x2b, 0xde, 0x5e, 0xc8,
|
||||
0xc7, 0xfa, 0x94, 0x7c, 0x3c, 0x47, 0x86, 0x9f, 0x6b, 0x50, 0x7d, 0x88, 0xe3, 0xad, 0x7d, 0xec,
|
||||
0x4b, 0xfb, 0x89, 0x9c, 0x9a, 0x4e, 0x6d, 0x24, 0x41, 0xee, 0x42, 0x7d, 0x22, 0x8b, 0x24, 0xe7,
|
||||
0x43, 0x9e, 0x44, 0x2e, 0x78, 0x46, 0x6f, 0xc3, 0x02, 0xe5, 0xce, 0x63, 0xd7, 0xa7, 0x3d, 0x69,
|
||||
0xff, 0x05, 0xbb, 0x4a, 0xf9, 0xbb, 0x82, 0x14, 0xf9, 0x2b, 0x13, 0x53, 0x79, 0xbb, 0x61, 0x4f,
|
||||
0x70, 0xac, 0xf7, 0x00, 0x12, 0xd1, 0x04, 0x40, 0x99, 0x77, 0x69, 0x93, 0xde, 0xf5, 0x55, 0xa8,
|
||||
0x7a, 0x38, 0xde, 0xe2, 0xd8, 0x6f, 0xe9, 0x32, 0xf9, 0xcf, 0x42, 0x2d, 0x39, 0xc9, 0x4e, 0xb7,
|
||||
0x5b, 0x3f, 0xd0, 0xa1, 0xf6, 0xad, 0xc8, 0xed, 0x75, 0xc2, 0x1e, 0x9e, 0x3e, 0x57, 0xf8, 0x8f,
|
||||
0x28, 0xfa, 0xbd, 0x47, 0x79, 0xfe, 0xcf, 0x19, 0x22, 0x38, 0x24, 0x91, 0x07, 0x47, 0x42, 0x0a,
|
||||
0xd8, 0xa9, 0x90, 0xcc, 0x19, 0xba, 0xf1, 0x20, 0xcd, 0x05, 0x20, 0x59, 0x7b, 0x82, 0x43, 0xda,
|
||||
0xd0, 0x48, 0x37, 0x30, 0x37, 0x50, 0x19, 0xa1, 0xbe, 0x75, 0x6f, 0x6a, 0xa0, 0x3e, 0xc4, 0xf1,
|
||||
0xbb, 0xae, 0x3f, 0xc2, 0x3d, 0x97, 0x32, 0xbb, 0x9e, 0x1c, 0x22, 0xbe, 0xb2, 0x9e, 0x6a, 0x00,
|
||||
0x12, 0x01, 0x11, 0xcb, 0xe7, 0x0f, 0xd5, 0x3e, 0xce, 0xa1, 0xe4, 0x4b, 0xb0, 0x12, 0x8e, 0x02,
|
||||
0x87, 0xa1, 0xef, 0xc6, 0xd8, 0x73, 0x12, 0x30, 0x78, 0x02, 0x0e, 0x09, 0x47, 0x81, 0xad, 0x7e,
|
||||
0xda, 0x4f, 0x7e, 0xb1, 0x7e, 0xa8, 0x01, 0xbc, 0x2d, 0x34, 0x57, 0x62, 0x9c, 0x6d, 0x61, 0xb4,
|
||||
0x29, 0x2d, 0xcc, 0x04, 0x74, 0x7a, 0x11, 0xba, 0xed, 0x14, 0x3a, 0x91, 0x97, 0x78, 0x32, 0xda,
|
||||
0xbc, 0x37, 0xc3, 0x9c, 0xb9, 0xf2, 0x09, 0xba, 0x72, 0x6d, 0xfd, 0x4c, 0x4d, 0x83, 0x85, 0x74,
|
||||
0x4a, 0xa4, 0x82, 0x95, 0xb5, 0xb3, 0x56, 0xbe, 0x0b, 0xf5, 0x00, 0x83, 0x88, 0x8d, 0x1d, 0x4e,
|
||||
0x9f, 0x60, 0x1a, 0x24, 0x8a, 0xb5, 0x4f, 0x9f, 0xa0, 0x08, 0x03, 0x09, 0x49, 0x74, 0xc2, 0xd3,
|
||||
0xc7, 0x40, 0xc0, 0x10, 0x9d, 0x70, 0xf2, 0x05, 0xb8, 0xc1, 0xb0, 0x8b, 0x61, 0xec, 0x8f, 0x9d,
|
||||
0x20, 0xea, 0xd1, 0x23, 0x8a, 0x69, 0xa8, 0x98, 0xe9, 0x0f, 0xbb, 0x09, 0xdf, 0xfa, 0x9b, 0x06,
|
||||
0xcd, 0x6f, 0xa7, 0x09, 0x52, 0x49, 0xf6, 0x1c, 0xf2, 0xd6, 0xd7, 0xa4, 0xb2, 0x05, 0xfc, 0xe6,
|
||||
0x8c, 0x86, 0x33, 0x90, 0xec, 0x05, 0x8e, 0x7d, 0x25, 0xd4, 0x36, 0xd4, 0xa5, 0x39, 0x92, 0x33,
|
||||
0x4a, 0x73, 0x6d, 0x90, 0x5b, 0xde, 0x86, 0xa3, 0x6c, 0x6d, 0xfd, 0x52, 0x07, 0xa2, 0x46, 0x0a,
|
||||
0xd2, 0x48, 0x2f, 0x5c, 0x1f, 0xf0, 0xc6, 0xf4, 0x3e, 0xe0, 0x7c, 0x81, 0xf7, 0x19, 0x50, 0x6a,
|
||||
0xe5, 0xf3, 0xbe, 0x42, 0x12, 0x68, 0x43, 0x03, 0x4f, 0x63, 0xe6, 0xa6, 0x41, 0x57, 0xbd, 0x74,
|
||||
0xd0, 0xc9, 0xcf, 0x92, 0x48, 0x7e, 0x5f, 0x87, 0x95, 0xb4, 0x53, 0x7c, 0x89, 0xd7, 0xc5, 0x78,
|
||||
0xfd, 0x4e, 0x87, 0xd7, 0x0a, 0x78, 0xed, 0xb1, 0xa8, 0xcf, 0x90, 0xf3, 0x97, 0xb8, 0xcd, 0xc3,
|
||||
0xed, 0xcd, 0xbf, 0x18, 0x50, 0x4d, 0x14, 0x26, 0x35, 0x28, 0x7b, 0x8f, 0xa2, 0x10, 0xcd, 0x57,
|
||||
0xc8, 0x2a, 0xdc, 0xf0, 0xce, 0xfe, 0x37, 0x61, 0xf6, 0xc8, 0x32, 0x2c, 0x79, 0xc5, 0xb1, 0xbd,
|
||||
0x89, 0x84, 0x40, 0xd3, 0x2b, 0x4c, 0xad, 0xcd, 0x23, 0x72, 0x0b, 0x96, 0xbd, 0xf3, 0x83, 0x5d,
|
||||
0x53, 0xbc, 0xfb, 0xa6, 0x57, 0x9c, 0x7d, 0x72, 0x73, 0x20, 0x8f, 0xf8, 0x06, 0xc6, 0x59, 0x11,
|
||||
0xcf, 0x4d, 0x4a, 0x56, 0xc1, 0xf4, 0xce, 0x8c, 0x20, 0xcd, 0xdf, 0x6b, 0x64, 0x19, 0x9a, 0x5e,
|
||||
0x61, 0xc6, 0x66, 0xfe, 0x41, 0x23, 0x04, 0x16, 0xbd, 0xc9, 0x21, 0x92, 0xf9, 0x47, 0x8d, 0xdc,
|
||||
0x02, 0xe2, 0x9d, 0x9b, 0xb5, 0x98, 0x7f, 0xd2, 0xc8, 0x0a, 0x2c, 0x79, 0x85, 0x91, 0x04, 0x37,
|
||||
0xff, 0xac, 0x91, 0x1b, 0xd0, 0xf0, 0x26, 0xd2, 0x93, 0xf9, 0x2b, 0x5d, 0x5d, 0x35, 0xe9, 0x53,
|
||||
0xe6, 0xfb, 0x3a, 0xb9, 0x03, 0x37, 0xbd, 0xa9, 0x8e, 0x66, 0x3e, 0xd3, 0x49, 0x03, 0xaa, 0x9e,
|
||||
0x6a, 0xfe, 0xcd, 0x1f, 0x19, 0x92, 0x52, 0x5d, 0xa9, 0xf9, 0x63, 0x83, 0xd4, 0xa1, 0xe2, 0xc9,
|
||||
0xfa, 0xd0, 0xfc, 0x89, 0xfa, 0x49, 0x55, 0xf7, 0xe6, 0x47, 0x86, 0x14, 0x7f, 0xb2, 0xd6, 0x37,
|
||||
0xff, 0x6d, 0x90, 0x26, 0xd4, 0xbc, 0xb4, 0x9e, 0x35, 0x7f, 0x5d, 0x93, 0x52, 0x17, 0x9f, 0x0a,
|
||||
0xf3, 0x83, 0x1a, 0x59, 0x02, 0xf0, 0xb2, 0xb2, 0xc7, 0xfc, 0x4d, 0xed, 0xcd, 0xb7, 0x60, 0x21,
|
||||
0xfd, 0x7b, 0x90, 0x00, 0x54, 0x76, 0x5d, 0x1e, 0x23, 0x33, 0x5f, 0x11, 0x6b, 0x1b, 0xdd, 0x1e,
|
||||
0x32, 0x53, 0x13, 0xeb, 0xef, 0x30, 0x2a, 0xf8, 0xba, 0xb0, 0xf9, 0x9e, 0x70, 0x4c, 0xd3, 0xd8,
|
||||
0x6e, 0x7f, 0x77, 0xbb, 0x4f, 0xe3, 0xc1, 0xe8, 0x50, 0x78, 0xcd, 0xe6, 0x13, 0xea, 0xfb, 0xf4,
|
||||
0x49, 0x8c, 0xdd, 0xc1, 0xa6, 0xf2, 0xa8, 0x2f, 0xf6, 0x28, 0x8f, 0x19, 0x3d, 0x1c, 0xc5, 0xd8,
|
||||
0xdb, 0x4c, 0x83, 0x65, 0x53, 0xba, 0x59, 0x46, 0x0e, 0x0f, 0x0f, 0x2b, 0x92, 0xf3, 0xe0, 0xbf,
|
||||
0x01, 0x00, 0x00, 0xff, 0xff, 0xb2, 0xdd, 0xfb, 0x16, 0x8f, 0x1f, 0x00, 0x00,
|
||||
}
|
||||
|
|
|
@ -21,7 +21,8 @@ func NewLoadIndexClient(ctx context.Context, pulsarAddress string, loadIndexChan
|
|||
}
|
||||
}
|
||||
|
||||
func (lic *LoadIndexClient) LoadIndex(indexPaths []string, segmentID int64, fieldID int64) error {
|
||||
func (lic *LoadIndexClient) LoadIndex(indexPaths []string, segmentID int64, fieldID int64, indexParam map[string]string) error {
|
||||
// TODO:: add indexParam to proto
|
||||
baseMsg := msgstream.BaseMsg{
|
||||
BeginTimestamp: 0,
|
||||
EndTimestamp: 0,
|
||||
|
|
|
@ -54,7 +54,7 @@ type collectionReplica interface {
|
|||
|
||||
// segment
|
||||
getSegmentNum() int
|
||||
getSegmentStatistics() *internalpb.QueryNodeStats
|
||||
getSegmentStatistics() []*internalpb.SegmentStats
|
||||
addSegment(segmentID UniqueID, partitionTag string, collectionID UniqueID) error
|
||||
removeSegment(segmentID UniqueID) error
|
||||
getSegmentByID(segmentID UniqueID) (*Segment, error)
|
||||
|
@ -317,7 +317,7 @@ func (colReplica *collectionReplicaImpl) getSegmentNum() int {
|
|||
return len(colReplica.segments)
|
||||
}
|
||||
|
||||
func (colReplica *collectionReplicaImpl) getSegmentStatistics() *internalpb.QueryNodeStats {
|
||||
func (colReplica *collectionReplicaImpl) getSegmentStatistics() []*internalpb.SegmentStats {
|
||||
colReplica.mu.RLock()
|
||||
defer colReplica.mu.RUnlock()
|
||||
|
||||
|
@ -339,10 +339,7 @@ func (colReplica *collectionReplicaImpl) getSegmentStatistics() *internalpb.Quer
|
|||
segment.recentlyModified = false
|
||||
}
|
||||
|
||||
return &internalpb.QueryNodeStats{
|
||||
MsgType: internalpb.MsgType_kQueryNodeStats,
|
||||
SegStats: statisticData,
|
||||
}
|
||||
return statisticData
|
||||
}
|
||||
|
||||
func (colReplica *collectionReplicaImpl) addSegment(segmentID UniqueID, partitionTag string, collectionID UniqueID) error {
|
||||
|
@ -359,7 +356,7 @@ func (colReplica *collectionReplicaImpl) addSegment(segmentID UniqueID, partitio
|
|||
colReplica.mu.Lock()
|
||||
defer colReplica.mu.Unlock()
|
||||
|
||||
var newSegment = newSegment(collection, segmentID)
|
||||
var newSegment = newSegment(collection, segmentID, partitionTag, collectionID)
|
||||
|
||||
colReplica.segments[segmentID] = newSegment
|
||||
*partition.Segments() = append(*partition.Segments(), newSegment)
|
||||
|
|
|
@ -1,41 +0,0 @@
|
|||
package querynode
|
||||
|
||||
import (
|
||||
"context"
|
||||
|
||||
"github.com/minio/minio-go/v7"
|
||||
"github.com/zilliztech/milvus-distributed/internal/msgstream"
|
||||
)
|
||||
|
||||
type LoadIndex struct {
|
||||
ctx context.Context
|
||||
cancel context.CancelFunc
|
||||
client *minio.Client
|
||||
|
||||
replica collectionReplica
|
||||
numCompletedSegmentsToFieldID map[int64]int64
|
||||
|
||||
msgBuffer chan msgstream.TsMsg
|
||||
unsolvedMsg []msgstream.TsMsg
|
||||
loadIndexMsgStream msgstream.MsgStream
|
||||
|
||||
queryNodeID UniqueID
|
||||
}
|
||||
|
||||
func (li *LoadIndex) loadIndex(indexKey []string) [][]byte {
|
||||
// TODO:: load dataStore client interface to load builtIndex according index key
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
func (li *LoadIndex) updateSegmentIndex(bytesIndex [][]byte, segmentID UniqueID) error {
|
||||
// TODO:: dataStore return bytes index, load index to c++ segment
|
||||
// TODO: how to deserialize bytes to segment index?
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
func (li *LoadIndex) sendQueryNodeStats() error {
|
||||
// TODO:: update segment index type in replica, and publish queryNode segmentStats
|
||||
return nil
|
||||
}
|
|
@ -0,0 +1,98 @@
|
|||
package querynode
|
||||
|
||||
/*
|
||||
#cgo CFLAGS: -I${SRCDIR}/../core/output/include
|
||||
#cgo LDFLAGS: -L${SRCDIR}/../core/output/lib -lmilvus_segcore -Wl,-rpath=${SRCDIR}/../core/output/lib
|
||||
|
||||
#include "segcore/load_index_c.h"
|
||||
|
||||
*/
|
||||
import "C"
|
||||
import (
|
||||
"errors"
|
||||
"strconv"
|
||||
"unsafe"
|
||||
)
|
||||
|
||||
type LoadIndexInfo struct {
|
||||
cLoadIndexInfo C.CLoadIndexInfo
|
||||
}
|
||||
|
||||
func NewLoadIndexInfo() (*LoadIndexInfo, error) {
|
||||
var cLoadIndexInfo C.CLoadIndexInfo
|
||||
status := C.NewLoadIndexInfo(&cLoadIndexInfo)
|
||||
errorCode := status.error_code
|
||||
|
||||
if errorCode != 0 {
|
||||
errorMsg := C.GoString(status.error_msg)
|
||||
defer C.free(unsafe.Pointer(status.error_msg))
|
||||
return nil, errors.New("NewLoadIndexInfo failed, C runtime error detected, error code = " + strconv.Itoa(int(errorCode)) + ", error msg = " + errorMsg)
|
||||
}
|
||||
return &LoadIndexInfo{cLoadIndexInfo: cLoadIndexInfo}, nil
|
||||
}
|
||||
|
||||
func (li *LoadIndexInfo) AppendIndexParam(indexKey string, indexValue string) error {
|
||||
cIndexKey := C.CString(indexKey)
|
||||
cIndexValue := C.CString(indexValue)
|
||||
status := C.AppendIndexParam(li.cLoadIndexInfo, cIndexKey, cIndexValue)
|
||||
errorCode := status.error_code
|
||||
|
||||
if errorCode != 0 {
|
||||
errorMsg := C.GoString(status.error_msg)
|
||||
defer C.free(unsafe.Pointer(status.error_msg))
|
||||
return errors.New("AppendIndexParam failed, C runtime error detected, error code = " + strconv.Itoa(int(errorCode)) + ", error msg = " + errorMsg)
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (li *LoadIndexInfo) AppendFieldInfo(fieldName string, fieldID int64) error {
|
||||
cFieldName := C.CString(fieldName)
|
||||
cFieldID := C.long(fieldID)
|
||||
status := C.AppendFieldInfo(li.cLoadIndexInfo, cFieldName, cFieldID)
|
||||
errorCode := status.error_code
|
||||
|
||||
if errorCode != 0 {
|
||||
errorMsg := C.GoString(status.error_msg)
|
||||
defer C.free(unsafe.Pointer(status.error_msg))
|
||||
return errors.New("AppendFieldInfo failed, C runtime error detected, error code = " + strconv.Itoa(int(errorCode)) + ", error msg = " + errorMsg)
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (li *LoadIndexInfo) AppendIndex(bytesIndex [][]byte, indexKeys []string) error {
|
||||
var cBinarySet C.CBinarySet
|
||||
status := C.NewBinarySet(&cBinarySet)
|
||||
|
||||
errorCode := status.error_code
|
||||
if errorCode != 0 {
|
||||
errorMsg := C.GoString(status.error_msg)
|
||||
defer C.free(unsafe.Pointer(status.error_msg))
|
||||
return errors.New("newBinarySet failed, C runtime error detected, error code = " + strconv.Itoa(int(errorCode)) + ", error msg = " + errorMsg)
|
||||
}
|
||||
|
||||
for i, byteIndex := range bytesIndex {
|
||||
indexPtr := unsafe.Pointer(&byteIndex[0])
|
||||
indexLen := C.long(len(byteIndex))
|
||||
indexKey := C.CString(indexKeys[i])
|
||||
status = C.AppendBinaryIndex(cBinarySet, indexPtr, indexLen, indexKey)
|
||||
errorCode = status.error_code
|
||||
if errorCode != 0 {
|
||||
break
|
||||
}
|
||||
}
|
||||
if errorCode != 0 {
|
||||
errorMsg := C.GoString(status.error_msg)
|
||||
defer C.free(unsafe.Pointer(status.error_msg))
|
||||
return errors.New("AppendBinaryIndex failed, C runtime error detected, error code = " + strconv.Itoa(int(errorCode)) + ", error msg = " + errorMsg)
|
||||
}
|
||||
|
||||
status = C.AppendIndex(li.cLoadIndexInfo, cBinarySet)
|
||||
errorCode = status.error_code
|
||||
if errorCode != 0 {
|
||||
errorMsg := C.GoString(status.error_msg)
|
||||
defer C.free(unsafe.Pointer(status.error_msg))
|
||||
return errors.New("AppendIndex failed, C runtime error detected, error code = " + strconv.Itoa(int(errorCode)) + ", error msg = " + errorMsg)
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
|
@ -0,0 +1,286 @@
|
|||
package querynode
|
||||
|
||||
import (
|
||||
"context"
|
||||
"errors"
|
||||
"fmt"
|
||||
"log"
|
||||
"path/filepath"
|
||||
"sort"
|
||||
"strconv"
|
||||
"strings"
|
||||
|
||||
"github.com/minio/minio-go/v7"
|
||||
"github.com/minio/minio-go/v7/pkg/credentials"
|
||||
|
||||
minioKV "github.com/zilliztech/milvus-distributed/internal/kv/minio"
|
||||
"github.com/zilliztech/milvus-distributed/internal/msgstream"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
|
||||
internalPb "github.com/zilliztech/milvus-distributed/internal/proto/internalpb"
|
||||
)
|
||||
|
||||
type loadIndexService struct {
|
||||
ctx context.Context
|
||||
cancel context.CancelFunc
|
||||
client *minioKV.MinIOKV
|
||||
|
||||
replica collectionReplica
|
||||
|
||||
fieldIndexes map[string][]*internalPb.IndexStats
|
||||
fieldStatsChan chan []*internalPb.FieldStats
|
||||
|
||||
msgBuffer chan msgstream.TsMsg
|
||||
unsolvedMsg []msgstream.TsMsg
|
||||
loadIndexMsgStream msgstream.MsgStream
|
||||
|
||||
queryNodeID UniqueID
|
||||
}
|
||||
|
||||
func newLoadIndexService(ctx context.Context, replica collectionReplica) *loadIndexService {
|
||||
ctx1, cancel := context.WithCancel(ctx)
|
||||
|
||||
// init minio
|
||||
minioClient, err := minio.New(Params.MinioEndPoint, &minio.Options{
|
||||
Creds: credentials.NewStaticV4(Params.MinioAccessKeyID, Params.MinioSecretAccessKey, ""),
|
||||
Secure: Params.MinioUseSSLStr,
|
||||
})
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
|
||||
// TODO: load bucketName from config
|
||||
bucketName := "query-node-load-index-service-minio"
|
||||
MinioKV, err := minioKV.NewMinIOKV(ctx1, minioClient, bucketName)
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
|
||||
// init msgStream
|
||||
receiveBufSize := Params.LoadIndexReceiveBufSize
|
||||
pulsarBufSize := Params.LoadIndexPulsarBufSize
|
||||
|
||||
msgStreamURL := Params.PulsarAddress
|
||||
|
||||
consumeChannels := Params.LoadIndexChannelNames
|
||||
consumeSubName := Params.MsgChannelSubName
|
||||
|
||||
loadIndexStream := msgstream.NewPulsarMsgStream(ctx, receiveBufSize)
|
||||
loadIndexStream.SetPulsarClient(msgStreamURL)
|
||||
unmarshalDispatcher := msgstream.NewUnmarshalDispatcher()
|
||||
loadIndexStream.CreatePulsarConsumers(consumeChannels, consumeSubName, unmarshalDispatcher, pulsarBufSize)
|
||||
|
||||
var stream msgstream.MsgStream = loadIndexStream
|
||||
|
||||
return &loadIndexService{
|
||||
ctx: ctx1,
|
||||
cancel: cancel,
|
||||
client: MinioKV,
|
||||
|
||||
replica: replica,
|
||||
fieldIndexes: make(map[string][]*internalPb.IndexStats),
|
||||
fieldStatsChan: make(chan []*internalPb.FieldStats, 1),
|
||||
|
||||
msgBuffer: make(chan msgstream.TsMsg, 1),
|
||||
unsolvedMsg: make([]msgstream.TsMsg, 0),
|
||||
loadIndexMsgStream: stream,
|
||||
|
||||
queryNodeID: Params.QueryNodeID,
|
||||
}
|
||||
}
|
||||
|
||||
func (lis *loadIndexService) start() {
|
||||
lis.loadIndexMsgStream.Start()
|
||||
|
||||
for {
|
||||
select {
|
||||
case <-lis.ctx.Done():
|
||||
return
|
||||
default:
|
||||
messages := lis.loadIndexMsgStream.Consume()
|
||||
if messages == nil || len(messages.Msgs) <= 0 {
|
||||
log.Println("null msg pack")
|
||||
continue
|
||||
}
|
||||
for _, msg := range messages.Msgs {
|
||||
indexMsg, ok := msg.(*msgstream.LoadIndexMsg)
|
||||
if !ok {
|
||||
log.Println("type assertion failed for LoadIndexMsg")
|
||||
continue
|
||||
}
|
||||
/* TODO: debug
|
||||
// 1. use msg's index paths to get index bytes
|
||||
indexBuffer := lis.loadIndex(indexMsg.IndexPaths)
|
||||
// 2. use index bytes and index path to update segment
|
||||
err := lis.updateSegmentIndex(indexBuffer, indexMsg.IndexPaths, indexMsg.SegmentID)
|
||||
if err != nil {
|
||||
log.Println(err)
|
||||
continue
|
||||
}
|
||||
*/
|
||||
// 3. update segment index stats
|
||||
err := lis.updateSegmentIndexStats(indexMsg)
|
||||
if err != nil {
|
||||
log.Println(err)
|
||||
continue
|
||||
}
|
||||
}
|
||||
|
||||
// sendQueryNodeStats
|
||||
err := lis.sendQueryNodeStats()
|
||||
if err != nil {
|
||||
log.Println(err)
|
||||
continue
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (lis *loadIndexService) printIndexParams(index []*commonpb.KeyValuePair) {
|
||||
fmt.Println("=================================================")
|
||||
for i := 0; i < len(index); i++ {
|
||||
fmt.Println(index[i])
|
||||
}
|
||||
}
|
||||
|
||||
func (lis *loadIndexService) indexParamsEqual(index1 []*commonpb.KeyValuePair, index2 []*commonpb.KeyValuePair) bool {
|
||||
if len(index1) != len(index2) {
|
||||
return false
|
||||
}
|
||||
|
||||
for i := 0; i < len(index1); i++ {
|
||||
kv1 := *index1[i]
|
||||
kv2 := *index2[i]
|
||||
if kv1.Key != kv2.Key || kv1.Value != kv2.Value {
|
||||
return false
|
||||
}
|
||||
}
|
||||
|
||||
return true
|
||||
}
|
||||
|
||||
func (lis *loadIndexService) fieldsStatsIDs2Key(collectionID UniqueID, fieldID UniqueID) string {
|
||||
return strconv.FormatInt(collectionID, 10) + "/" + strconv.FormatInt(fieldID, 10)
|
||||
}
|
||||
|
||||
func (lis *loadIndexService) fieldsStatsKey2IDs(key string) (UniqueID, UniqueID, error) {
|
||||
ids := strings.Split(key, "/")
|
||||
if len(ids) != 2 {
|
||||
return 0, 0, errors.New("illegal fieldsStatsKey")
|
||||
}
|
||||
collectionID, err := strconv.ParseInt(ids[0], 10, 64)
|
||||
if err != nil {
|
||||
return 0, 0, err
|
||||
}
|
||||
fieldID, err := strconv.ParseInt(ids[1], 10, 64)
|
||||
if err != nil {
|
||||
return 0, 0, err
|
||||
}
|
||||
return collectionID, fieldID, nil
|
||||
}
|
||||
|
||||
func (lis *loadIndexService) updateSegmentIndexStats(indexMsg *msgstream.LoadIndexMsg) error {
|
||||
targetSegment, err := lis.replica.getSegmentByID(indexMsg.SegmentID)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
fieldStatsKey := lis.fieldsStatsIDs2Key(targetSegment.collectionID, indexMsg.FieldID)
|
||||
_, ok := lis.fieldIndexes[fieldStatsKey]
|
||||
newIndexParams := indexMsg.IndexParams
|
||||
// sort index params by key
|
||||
sort.Slice(newIndexParams, func(i, j int) bool { return newIndexParams[i].Key < newIndexParams[j].Key })
|
||||
if !ok {
|
||||
lis.fieldIndexes[fieldStatsKey] = make([]*internalPb.IndexStats, 0)
|
||||
lis.fieldIndexes[fieldStatsKey] = append(lis.fieldIndexes[fieldStatsKey],
|
||||
&internalPb.IndexStats{
|
||||
IndexParams: newIndexParams,
|
||||
NumRelatedSegments: 1,
|
||||
})
|
||||
} else {
|
||||
isNewIndex := true
|
||||
for _, index := range lis.fieldIndexes[fieldStatsKey] {
|
||||
if lis.indexParamsEqual(newIndexParams, index.IndexParams) {
|
||||
index.NumRelatedSegments++
|
||||
isNewIndex = false
|
||||
}
|
||||
}
|
||||
if isNewIndex {
|
||||
lis.fieldIndexes[fieldStatsKey] = append(lis.fieldIndexes[fieldStatsKey],
|
||||
&internalPb.IndexStats{
|
||||
IndexParams: newIndexParams,
|
||||
NumRelatedSegments: 1,
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
func (lis *loadIndexService) loadIndex(indexPath []string) [][]byte {
|
||||
index := make([][]byte, 0)
|
||||
|
||||
for _, path := range indexPath {
|
||||
// get binarySetKey from indexPath
|
||||
binarySetKey := filepath.Base(path)
|
||||
indexPiece, err := (*lis.client).Load(binarySetKey)
|
||||
if err != nil {
|
||||
log.Println(err)
|
||||
return nil
|
||||
}
|
||||
index = append(index, []byte(indexPiece))
|
||||
}
|
||||
|
||||
return index
|
||||
}
|
||||
|
||||
func (lis *loadIndexService) updateSegmentIndex(bytesIndex [][]byte, loadIndexMsg *msgstream.LoadIndexMsg) error {
|
||||
segment, err := lis.replica.getSegmentByID(loadIndexMsg.SegmentID)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
loadIndexInfo, err := NewLoadIndexInfo()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
err = loadIndexInfo.AppendFieldInfo(loadIndexMsg.FieldName, loadIndexMsg.FieldID)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
for _, indexParam := range loadIndexMsg.IndexParams {
|
||||
err = loadIndexInfo.AppendIndexParam(indexParam.Key, indexParam.Value)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
err = loadIndexInfo.AppendIndex(bytesIndex, loadIndexMsg.IndexPaths)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
err = segment.updateSegmentIndex(loadIndexInfo)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
func (lis *loadIndexService) sendQueryNodeStats() error {
|
||||
resultFieldsStats := make([]*internalPb.FieldStats, 0)
|
||||
for fieldStatsKey, indexStats := range lis.fieldIndexes {
|
||||
colID, fieldID, err := lis.fieldsStatsKey2IDs(fieldStatsKey)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
fieldStats := internalPb.FieldStats{
|
||||
CollectionID: colID,
|
||||
FieldID: fieldID,
|
||||
IndexStats: indexStats,
|
||||
}
|
||||
resultFieldsStats = append(resultFieldsStats, &fieldStats)
|
||||
}
|
||||
|
||||
lis.fieldStatsChan <- resultFieldsStats
|
||||
fmt.Println("sent field stats")
|
||||
return nil
|
||||
}
|
|
@ -0,0 +1,148 @@
|
|||
package querynode
|
||||
|
||||
import (
|
||||
"math"
|
||||
"math/rand"
|
||||
"sort"
|
||||
"testing"
|
||||
|
||||
"github.com/stretchr/testify/assert"
|
||||
|
||||
"github.com/zilliztech/milvus-distributed/internal/msgstream"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
|
||||
internalPb "github.com/zilliztech/milvus-distributed/internal/proto/internalpb"
|
||||
)
|
||||
|
||||
func TestLoadIndexService_PulsarAddress(t *testing.T) {
|
||||
node := newQueryNode()
|
||||
collectionID := rand.Int63n(1000000)
|
||||
segmentID := rand.Int63n(1000000)
|
||||
fieldID := rand.Int63n(1000000)
|
||||
initTestMeta(t, node, "collection0", collectionID, segmentID)
|
||||
|
||||
// loadIndexService and statsService
|
||||
node.loadIndexService = newLoadIndexService(node.queryNodeLoopCtx, node.replica)
|
||||
go node.loadIndexService.start()
|
||||
node.statsService = newStatsService(node.queryNodeLoopCtx, node.replica, node.loadIndexService.fieldStatsChan)
|
||||
go node.statsService.start()
|
||||
|
||||
// gen load index message pack
|
||||
const msgLength = 10
|
||||
indexParams := make([]*commonpb.KeyValuePair, 0)
|
||||
// init IVF_FLAT index params
|
||||
const (
|
||||
KeyDim = "dim"
|
||||
KeyTopK = "k"
|
||||
KeyNList = "nlist"
|
||||
KeyNProbe = "nprobe"
|
||||
KeyMetricType = "metric_type"
|
||||
KeySliceSize = "SLICE_SIZE"
|
||||
KeyDeviceID = "gpu_id"
|
||||
)
|
||||
const (
|
||||
ValueDim = "128"
|
||||
ValueTopK = "10"
|
||||
ValueNList = "100"
|
||||
ValueNProbe = "4"
|
||||
ValueMetricType = "L2"
|
||||
ValueSliceSize = "4"
|
||||
ValueDeviceID = "0"
|
||||
)
|
||||
|
||||
indexParams = append(indexParams, &commonpb.KeyValuePair{
|
||||
Key: KeyDim,
|
||||
Value: ValueDim,
|
||||
})
|
||||
indexParams = append(indexParams, &commonpb.KeyValuePair{
|
||||
Key: KeyTopK,
|
||||
Value: ValueTopK,
|
||||
})
|
||||
indexParams = append(indexParams, &commonpb.KeyValuePair{
|
||||
Key: KeyNList,
|
||||
Value: ValueNList,
|
||||
})
|
||||
indexParams = append(indexParams, &commonpb.KeyValuePair{
|
||||
Key: KeyNProbe,
|
||||
Value: ValueNProbe,
|
||||
})
|
||||
indexParams = append(indexParams, &commonpb.KeyValuePair{
|
||||
Key: KeyMetricType,
|
||||
Value: ValueMetricType,
|
||||
})
|
||||
indexParams = append(indexParams, &commonpb.KeyValuePair{
|
||||
Key: KeySliceSize,
|
||||
Value: ValueSliceSize,
|
||||
})
|
||||
indexParams = append(indexParams, &commonpb.KeyValuePair{
|
||||
Key: KeyDeviceID,
|
||||
Value: ValueDeviceID,
|
||||
})
|
||||
|
||||
loadIndex := internalPb.LoadIndex{
|
||||
MsgType: internalPb.MsgType_kLoadIndex,
|
||||
SegmentID: segmentID,
|
||||
FieldID: fieldID,
|
||||
IndexPaths: []string{"tmp/index"}, // TODO:
|
||||
IndexParams: indexParams,
|
||||
}
|
||||
|
||||
loadIndexMsg := msgstream.LoadIndexMsg{
|
||||
BaseMsg: msgstream.BaseMsg{
|
||||
HashValues: []uint32{uint32(0)},
|
||||
},
|
||||
LoadIndex: loadIndex,
|
||||
}
|
||||
|
||||
messages := make([]msgstream.TsMsg, 0)
|
||||
for i := 0; i < msgLength; i++ {
|
||||
var msg msgstream.TsMsg = &loadIndexMsg
|
||||
messages = append(messages, msg)
|
||||
}
|
||||
|
||||
msgPack := msgstream.MsgPack{
|
||||
BeginTs: 0,
|
||||
EndTs: math.MaxUint64,
|
||||
Msgs: messages,
|
||||
}
|
||||
|
||||
// init message stream producer
|
||||
loadIndexChannelNames := Params.LoadIndexChannelNames
|
||||
pulsarURL := Params.PulsarAddress
|
||||
|
||||
loadIndexStream := msgstream.NewPulsarMsgStream(node.queryNodeLoopCtx, Params.LoadIndexReceiveBufSize)
|
||||
loadIndexStream.SetPulsarClient(pulsarURL)
|
||||
loadIndexStream.CreatePulsarProducers(loadIndexChannelNames)
|
||||
|
||||
var loadIndexMsgStream msgstream.MsgStream = loadIndexStream
|
||||
loadIndexMsgStream.Start()
|
||||
|
||||
err := loadIndexMsgStream.Produce(&msgPack)
|
||||
assert.NoError(t, err)
|
||||
|
||||
// init message stream consumer and do checks
|
||||
statsMs := msgstream.NewPulsarMsgStream(node.queryNodeLoopCtx, Params.StatsReceiveBufSize)
|
||||
statsMs.SetPulsarClient(pulsarURL)
|
||||
statsMs.CreatePulsarConsumers([]string{Params.StatsChannelName}, Params.MsgChannelSubName, msgstream.NewUnmarshalDispatcher(), Params.StatsReceiveBufSize)
|
||||
statsMs.Start()
|
||||
|
||||
receiveMsg := msgstream.MsgStream(statsMs).Consume()
|
||||
assert.NotNil(t, receiveMsg)
|
||||
assert.NotEqual(t, len(receiveMsg.Msgs), 0)
|
||||
statsMsg, ok := receiveMsg.Msgs[0].(*msgstream.QueryNodeStatsMsg)
|
||||
assert.Equal(t, ok, true)
|
||||
assert.Equal(t, len(statsMsg.FieldStats), 1)
|
||||
fieldStats0 := statsMsg.FieldStats[0]
|
||||
assert.Equal(t, fieldStats0.FieldID, fieldID)
|
||||
assert.Equal(t, fieldStats0.CollectionID, collectionID)
|
||||
assert.Equal(t, len(fieldStats0.IndexStats), 1)
|
||||
indexStats0 := fieldStats0.IndexStats[0]
|
||||
|
||||
params := indexStats0.IndexParams
|
||||
// sort index params by key
|
||||
sort.Slice(indexParams, func(i, j int) bool { return indexParams[i].Key < indexParams[j].Key })
|
||||
indexEqual := node.loadIndexService.indexParamsEqual(params, indexParams)
|
||||
assert.Equal(t, indexEqual, true)
|
||||
|
||||
<-node.queryNodeLoopCtx.Done()
|
||||
node.Close()
|
||||
}
|
|
@ -21,6 +21,12 @@ type ParamTable struct {
|
|||
FlowGraphMaxQueueLength int32
|
||||
FlowGraphMaxParallelism int32
|
||||
|
||||
// minio
|
||||
MinioEndPoint string
|
||||
MinioAccessKeyID string
|
||||
MinioSecretAccessKey string
|
||||
MinioUseSSLStr bool
|
||||
|
||||
// dm
|
||||
InsertChannelNames []string
|
||||
InsertChannelRange []int
|
||||
|
@ -44,6 +50,11 @@ type ParamTable struct {
|
|||
StatsChannelName string
|
||||
StatsReceiveBufSize int64
|
||||
|
||||
// load index
|
||||
LoadIndexChannelNames []string
|
||||
LoadIndexReceiveBufSize int64
|
||||
LoadIndexPulsarBufSize int64
|
||||
|
||||
GracefulTime int64
|
||||
MsgChannelSubName string
|
||||
DefaultPartitionTag string
|
||||
|
@ -59,6 +70,11 @@ func (p *ParamTable) Init() {
|
|||
panic(err)
|
||||
}
|
||||
|
||||
err = p.LoadYaml("milvus.yaml")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
|
||||
queryNodeIDStr := os.Getenv("QUERY_NODE_ID")
|
||||
if queryNodeIDStr == "" {
|
||||
queryNodeIDList := p.QueryNodeIDList()
|
||||
|
@ -78,6 +94,11 @@ func (p *ParamTable) Init() {
|
|||
panic(err)
|
||||
}
|
||||
|
||||
p.initMinioEndPoint()
|
||||
p.initMinioAccessKeyID()
|
||||
p.initMinioSecretAccessKey()
|
||||
p.initMinioUseSSLStr()
|
||||
|
||||
p.initPulsarAddress()
|
||||
p.initETCDAddress()
|
||||
p.initMetaRootPath()
|
||||
|
@ -111,6 +132,46 @@ func (p *ParamTable) Init() {
|
|||
p.initStatsPublishInterval()
|
||||
p.initStatsChannelName()
|
||||
p.initStatsReceiveBufSize()
|
||||
|
||||
p.initLoadIndexChannelNames()
|
||||
p.initLoadIndexReceiveBufSize()
|
||||
p.initLoadIndexPulsarBufSize()
|
||||
}
|
||||
|
||||
func (p *ParamTable) initMinioEndPoint() {
|
||||
url, err := p.Load("_MinioAddress")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
p.MinioEndPoint = url
|
||||
}
|
||||
|
||||
func (p *ParamTable) initMinioAccessKeyID() {
|
||||
id, err := p.Load("minio.accessKeyID")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
p.MinioAccessKeyID = id
|
||||
}
|
||||
|
||||
func (p *ParamTable) initMinioSecretAccessKey() {
|
||||
key, err := p.Load("minio.secretAccessKey")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
p.MinioSecretAccessKey = key
|
||||
}
|
||||
|
||||
func (p *ParamTable) initMinioUseSSLStr() {
|
||||
ssl, err := p.Load("minio.useSSL")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
sslBoolean, err := strconv.ParseBool(ssl)
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
p.MinioUseSSLStr = sslBoolean
|
||||
}
|
||||
|
||||
func (p *ParamTable) initPulsarAddress() {
|
||||
|
@ -358,3 +419,19 @@ func (p *ParamTable) initSliceIndex() {
|
|||
func (p *ParamTable) initQueryNodeNum() {
|
||||
p.QueryNodeNum = len(p.QueryNodeIDList())
|
||||
}
|
||||
|
||||
func (p *ParamTable) initLoadIndexChannelNames() {
|
||||
loadIndexChannelName, err := p.Load("msgChannel.chanNamePrefix.cmd")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
p.LoadIndexChannelNames = []string{loadIndexChannelName}
|
||||
}
|
||||
|
||||
func (p *ParamTable) initLoadIndexReceiveBufSize() {
|
||||
p.LoadIndexReceiveBufSize = p.ParseInt64("queryNode.msgStream.loadIndex.recvBufSize")
|
||||
}
|
||||
|
||||
func (p *ParamTable) initLoadIndexPulsarBufSize() {
|
||||
p.LoadIndexPulsarBufSize = p.ParseInt64("queryNode.msgStream.loadIndex.pulsarBufSize")
|
||||
}
|
||||
|
|
|
@ -15,6 +15,47 @@ func TestParamTable_PulsarAddress(t *testing.T) {
|
|||
assert.Equal(t, "6650", split[len(split)-1])
|
||||
}
|
||||
|
||||
func TestParamTable_minio(t *testing.T) {
|
||||
t.Run("Test endPoint", func(t *testing.T) {
|
||||
endPoint := Params.MinioEndPoint
|
||||
equal := endPoint == "localhost:9000" || endPoint == "minio:9000"
|
||||
assert.Equal(t, equal, true)
|
||||
})
|
||||
|
||||
t.Run("Test accessKeyID", func(t *testing.T) {
|
||||
accessKeyID := Params.MinioAccessKeyID
|
||||
assert.Equal(t, accessKeyID, "minioadmin")
|
||||
})
|
||||
|
||||
t.Run("Test secretAccessKey", func(t *testing.T) {
|
||||
secretAccessKey := Params.MinioSecretAccessKey
|
||||
assert.Equal(t, secretAccessKey, "minioadmin")
|
||||
})
|
||||
|
||||
t.Run("Test useSSL", func(t *testing.T) {
|
||||
useSSL := Params.MinioUseSSLStr
|
||||
assert.Equal(t, useSSL, false)
|
||||
})
|
||||
}
|
||||
|
||||
func TestParamTable_LoadIndex(t *testing.T) {
|
||||
t.Run("Test channel names", func(t *testing.T) {
|
||||
names := Params.LoadIndexChannelNames
|
||||
assert.Equal(t, len(names), 1)
|
||||
assert.Contains(t, names[0], "cmd")
|
||||
})
|
||||
|
||||
t.Run("Test recvBufSize", func(t *testing.T) {
|
||||
size := Params.LoadIndexReceiveBufSize
|
||||
assert.Equal(t, size, int64(512))
|
||||
})
|
||||
|
||||
t.Run("Test pulsarBufSize", func(t *testing.T) {
|
||||
size := Params.LoadIndexPulsarBufSize
|
||||
assert.Equal(t, size, int64(512))
|
||||
})
|
||||
}
|
||||
|
||||
func TestParamTable_QueryNodeID(t *testing.T) {
|
||||
id := Params.QueryNodeID
|
||||
assert.Contains(t, Params.QueryNodeIDList(), id)
|
||||
|
|
|
@ -24,10 +24,12 @@ type QueryNode struct {
|
|||
|
||||
replica collectionReplica
|
||||
|
||||
dataSyncService *dataSyncService
|
||||
metaService *metaService
|
||||
searchService *searchService
|
||||
statsService *statsService
|
||||
// services
|
||||
dataSyncService *dataSyncService
|
||||
metaService *metaService
|
||||
searchService *searchService
|
||||
loadIndexService *loadIndexService
|
||||
statsService *statsService
|
||||
}
|
||||
|
||||
func Init() {
|
||||
|
@ -69,11 +71,13 @@ func (node *QueryNode) Start() error {
|
|||
node.dataSyncService = newDataSyncService(node.queryNodeLoopCtx, node.replica)
|
||||
node.searchService = newSearchService(node.queryNodeLoopCtx, node.replica)
|
||||
node.metaService = newMetaService(node.queryNodeLoopCtx, node.replica)
|
||||
node.statsService = newStatsService(node.queryNodeLoopCtx, node.replica)
|
||||
node.loadIndexService = newLoadIndexService(node.queryNodeLoopCtx, node.replica)
|
||||
node.statsService = newStatsService(node.queryNodeLoopCtx, node.replica, node.loadIndexService.fieldStatsChan)
|
||||
|
||||
go node.dataSyncService.start()
|
||||
go node.searchService.start()
|
||||
go node.metaService.start()
|
||||
go node.loadIndexService.start()
|
||||
go node.statsService.start()
|
||||
|
||||
<-node.queryNodeLoopCtx.Done()
|
||||
|
|
|
@ -26,6 +26,7 @@ func setup() {
|
|||
|
||||
func genTestCollectionMeta(collectionName string, collectionID UniqueID) *etcdpb.CollectionMeta {
|
||||
fieldVec := schemapb.FieldSchema{
|
||||
FieldID: UniqueID(0),
|
||||
Name: "vec",
|
||||
IsPrimaryKey: false,
|
||||
DataType: schemapb.DataType_VECTOR_FLOAT,
|
||||
|
@ -44,6 +45,7 @@ func genTestCollectionMeta(collectionName string, collectionID UniqueID) *etcdpb
|
|||
}
|
||||
|
||||
fieldInt := schemapb.FieldSchema{
|
||||
FieldID: UniqueID(1),
|
||||
Name: "age",
|
||||
IsPrimaryKey: false,
|
||||
DataType: schemapb.DataType_INT32,
|
||||
|
@ -119,12 +121,13 @@ func makeNewChannelNames(names []string, suffix string) []string {
|
|||
}
|
||||
|
||||
func refreshChannelNames() {
|
||||
suffix := "-test-query-node" + strconv.FormatInt(rand.Int63n(100), 10)
|
||||
suffix := "-test-query-node" + strconv.FormatInt(rand.Int63n(1000000), 10)
|
||||
Params.DDChannelNames = makeNewChannelNames(Params.DDChannelNames, suffix)
|
||||
Params.InsertChannelNames = makeNewChannelNames(Params.InsertChannelNames, suffix)
|
||||
Params.SearchChannelNames = makeNewChannelNames(Params.SearchChannelNames, suffix)
|
||||
Params.SearchResultChannelNames = makeNewChannelNames(Params.SearchResultChannelNames, suffix)
|
||||
Params.StatsChannelName = Params.StatsChannelName + suffix
|
||||
Params.LoadIndexChannelNames = makeNewChannelNames(Params.LoadIndexChannelNames, suffix)
|
||||
}
|
||||
|
||||
func TestMain(m *testing.M) {
|
||||
|
|
|
@ -21,7 +21,7 @@ func TestReduce_AllFunc(t *testing.T) {
|
|||
assert.NotEqual(t, "", schemaBlob)
|
||||
|
||||
collection := newCollection(collectionMeta.ID, schemaBlob)
|
||||
segment := newSegment(collection, segmentID)
|
||||
segment := newSegment(collection, segmentID, Params.DefaultPartitionTag, collectionID)
|
||||
|
||||
const DIM = 16
|
||||
var vec = [DIM]float32{1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16}
|
||||
|
|
|
@ -9,7 +9,6 @@ package querynode
|
|||
#include "segcore/collection_c.h"
|
||||
#include "segcore/plan_c.h"
|
||||
#include "segcore/reduce_c.h"
|
||||
|
||||
*/
|
||||
import "C"
|
||||
import (
|
||||
|
@ -25,6 +24,8 @@ import (
|
|||
type Segment struct {
|
||||
segmentPtr C.CSegmentBase
|
||||
segmentID UniqueID
|
||||
partitionTag string // TODO: use partitionID
|
||||
collectionID UniqueID
|
||||
lastMemSize int64
|
||||
lastRowCount int64
|
||||
recentlyModified bool
|
||||
|
@ -35,13 +36,18 @@ func (s *Segment) ID() UniqueID {
|
|||
}
|
||||
|
||||
//-------------------------------------------------------------------------------------- constructor and destructor
|
||||
func newSegment(collection *Collection, segmentID int64) *Segment {
|
||||
func newSegment(collection *Collection, segmentID int64, partitionTag string, collectionID UniqueID) *Segment {
|
||||
/*
|
||||
CSegmentBase
|
||||
newSegment(CPartition partition, unsigned long segment_id);
|
||||
*/
|
||||
segmentPtr := C.NewSegment(collection.collectionPtr, C.ulong(segmentID))
|
||||
var newSegment = &Segment{segmentPtr: segmentPtr, segmentID: segmentID}
|
||||
var newSegment = &Segment{
|
||||
segmentPtr: segmentPtr,
|
||||
segmentID: segmentID,
|
||||
partitionTag: partitionTag,
|
||||
collectionID: collectionID,
|
||||
}
|
||||
|
||||
return newSegment
|
||||
}
|
||||
|
@ -236,3 +242,17 @@ func (s *Segment) fillTargetEntry(plan *Plan,
|
|||
|
||||
return nil
|
||||
}
|
||||
|
||||
// segment, err := loadIndexService.replica.getSegmentByID(segmentID)
|
||||
func (s *Segment) updateSegmentIndex(loadIndexInfo *LoadIndexInfo) error {
|
||||
status := C.UpdateSegmentIndex(s.segmentPtr, loadIndexInfo.cLoadIndexInfo)
|
||||
errorCode := status.error_code
|
||||
|
||||
if errorCode != 0 {
|
||||
errorMsg := C.GoString(status.error_msg)
|
||||
defer C.free(unsafe.Pointer(status.error_msg))
|
||||
return errors.New("updateSegmentIndex failed, C runtime error detected, error code = " + strconv.Itoa(int(errorCode)) + ", error msg = " + errorMsg)
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
|
|
@ -26,7 +26,7 @@ func TestSegment_newSegment(t *testing.T) {
|
|||
assert.Equal(t, collection.ID(), collectionID)
|
||||
|
||||
segmentID := UniqueID(0)
|
||||
segment := newSegment(collection, segmentID)
|
||||
segment := newSegment(collection, segmentID, Params.DefaultPartitionTag, collectionID)
|
||||
assert.Equal(t, segmentID, segment.segmentID)
|
||||
deleteSegment(segment)
|
||||
deleteCollection(collection)
|
||||
|
@ -44,7 +44,7 @@ func TestSegment_deleteSegment(t *testing.T) {
|
|||
assert.Equal(t, collection.ID(), collectionID)
|
||||
|
||||
segmentID := UniqueID(0)
|
||||
segment := newSegment(collection, segmentID)
|
||||
segment := newSegment(collection, segmentID, Params.DefaultPartitionTag, collectionID)
|
||||
assert.Equal(t, segmentID, segment.segmentID)
|
||||
|
||||
deleteSegment(segment)
|
||||
|
@ -64,7 +64,7 @@ func TestSegment_getRowCount(t *testing.T) {
|
|||
assert.Equal(t, collection.ID(), collectionID)
|
||||
|
||||
segmentID := UniqueID(0)
|
||||
segment := newSegment(collection, segmentID)
|
||||
segment := newSegment(collection, segmentID, Params.DefaultPartitionTag, collectionID)
|
||||
assert.Equal(t, segmentID, segment.segmentID)
|
||||
|
||||
ids := []int64{1, 2, 3}
|
||||
|
@ -115,7 +115,7 @@ func TestSegment_getDeletedCount(t *testing.T) {
|
|||
assert.Equal(t, collection.ID(), collectionID)
|
||||
|
||||
segmentID := UniqueID(0)
|
||||
segment := newSegment(collection, segmentID)
|
||||
segment := newSegment(collection, segmentID, Params.DefaultPartitionTag, collectionID)
|
||||
assert.Equal(t, segmentID, segment.segmentID)
|
||||
|
||||
ids := []int64{1, 2, 3}
|
||||
|
@ -172,7 +172,7 @@ func TestSegment_getMemSize(t *testing.T) {
|
|||
assert.Equal(t, collection.ID(), collectionID)
|
||||
|
||||
segmentID := UniqueID(0)
|
||||
segment := newSegment(collection, segmentID)
|
||||
segment := newSegment(collection, segmentID, Params.DefaultPartitionTag, collectionID)
|
||||
assert.Equal(t, segmentID, segment.segmentID)
|
||||
|
||||
ids := []int64{1, 2, 3}
|
||||
|
@ -223,7 +223,7 @@ func TestSegment_segmentInsert(t *testing.T) {
|
|||
assert.Equal(t, collection.Name(), collectionName)
|
||||
assert.Equal(t, collection.ID(), collectionID)
|
||||
segmentID := UniqueID(0)
|
||||
segment := newSegment(collection, segmentID)
|
||||
segment := newSegment(collection, segmentID, Params.DefaultPartitionTag, collectionID)
|
||||
assert.Equal(t, segmentID, segment.segmentID)
|
||||
|
||||
ids := []int64{1, 2, 3}
|
||||
|
@ -270,7 +270,7 @@ func TestSegment_segmentDelete(t *testing.T) {
|
|||
assert.Equal(t, collection.ID(), collectionID)
|
||||
|
||||
segmentID := UniqueID(0)
|
||||
segment := newSegment(collection, segmentID)
|
||||
segment := newSegment(collection, segmentID, Params.DefaultPartitionTag, collectionID)
|
||||
assert.Equal(t, segmentID, segment.segmentID)
|
||||
|
||||
ids := []int64{1, 2, 3}
|
||||
|
@ -323,7 +323,7 @@ func TestSegment_segmentSearch(t *testing.T) {
|
|||
assert.Equal(t, collection.ID(), collectionID)
|
||||
|
||||
segmentID := UniqueID(0)
|
||||
segment := newSegment(collection, segmentID)
|
||||
segment := newSegment(collection, segmentID, Params.DefaultPartitionTag, collectionID)
|
||||
assert.Equal(t, segmentID, segment.segmentID)
|
||||
|
||||
ids := []int64{1, 2, 3}
|
||||
|
@ -408,7 +408,7 @@ func TestSegment_segmentPreInsert(t *testing.T) {
|
|||
assert.Equal(t, collection.ID(), collectionID)
|
||||
|
||||
segmentID := UniqueID(0)
|
||||
segment := newSegment(collection, segmentID)
|
||||
segment := newSegment(collection, segmentID, Params.DefaultPartitionTag, collectionID)
|
||||
assert.Equal(t, segmentID, segment.segmentID)
|
||||
|
||||
const DIM = 16
|
||||
|
@ -450,7 +450,7 @@ func TestSegment_segmentPreDelete(t *testing.T) {
|
|||
assert.Equal(t, collection.ID(), collectionID)
|
||||
|
||||
segmentID := UniqueID(0)
|
||||
segment := newSegment(collection, segmentID)
|
||||
segment := newSegment(collection, segmentID, Params.DefaultPartitionTag, collectionID)
|
||||
assert.Equal(t, segmentID, segment.segmentID)
|
||||
|
||||
ids := []int64{1, 2, 3}
|
||||
|
|
|
@ -12,17 +12,23 @@ import (
|
|||
)
|
||||
|
||||
type statsService struct {
|
||||
ctx context.Context
|
||||
statsStream msgstream.MsgStream
|
||||
replica collectionReplica
|
||||
ctx context.Context
|
||||
|
||||
replica collectionReplica
|
||||
|
||||
fieldStatsChan chan []*internalpb.FieldStats
|
||||
statsStream msgstream.MsgStream
|
||||
}
|
||||
|
||||
func newStatsService(ctx context.Context, replica collectionReplica) *statsService {
|
||||
func newStatsService(ctx context.Context, replica collectionReplica, fieldStatsChan chan []*internalpb.FieldStats) *statsService {
|
||||
|
||||
return &statsService{
|
||||
ctx: ctx,
|
||||
statsStream: nil,
|
||||
replica: replica,
|
||||
ctx: ctx,
|
||||
|
||||
replica: replica,
|
||||
|
||||
fieldStatsChan: fieldStatsChan,
|
||||
statsStream: nil,
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -50,7 +56,9 @@ func (sService *statsService) start() {
|
|||
case <-sService.ctx.Done():
|
||||
return
|
||||
case <-time.After(time.Duration(sleepTimeInterval) * time.Millisecond):
|
||||
sService.sendSegmentStatistic()
|
||||
sService.publicStatistic(nil)
|
||||
case fieldStats := <-sService.fieldStatsChan:
|
||||
sService.publicStatistic(fieldStats)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -61,20 +69,21 @@ func (sService *statsService) close() {
|
|||
}
|
||||
}
|
||||
|
||||
func (sService *statsService) sendSegmentStatistic() {
|
||||
statisticData := sService.replica.getSegmentStatistics()
|
||||
func (sService *statsService) publicStatistic(fieldStats []*internalpb.FieldStats) {
|
||||
segStats := sService.replica.getSegmentStatistics()
|
||||
|
||||
// fmt.Println("Publish segment statistic")
|
||||
// fmt.Println(statisticData)
|
||||
sService.publicStatistic(statisticData)
|
||||
}
|
||||
queryNodeStats := internalpb.QueryNodeStats{
|
||||
MsgType: internalpb.MsgType_kQueryNodeStats,
|
||||
PeerID: Params.QueryNodeID,
|
||||
SegStats: segStats,
|
||||
FieldStats: fieldStats,
|
||||
}
|
||||
|
||||
func (sService *statsService) publicStatistic(statistic *internalpb.QueryNodeStats) {
|
||||
var msg msgstream.TsMsg = &msgstream.QueryNodeStatsMsg{
|
||||
BaseMsg: msgstream.BaseMsg{
|
||||
HashValues: []uint32{0},
|
||||
},
|
||||
QueryNodeStats: *statistic,
|
||||
QueryNodeStats: queryNodeStats,
|
||||
}
|
||||
|
||||
var msgPack = msgstream.MsgPack{
|
||||
|
|
|
@ -10,7 +10,7 @@ import (
|
|||
func TestStatsService_start(t *testing.T) {
|
||||
node := newQueryNode()
|
||||
initTestMeta(t, node, "collection0", 0, 0)
|
||||
node.statsService = newStatsService(node.queryNodeLoopCtx, node.replica)
|
||||
node.statsService = newStatsService(node.queryNodeLoopCtx, node.replica, nil)
|
||||
node.statsService.start()
|
||||
node.Close()
|
||||
}
|
||||
|
@ -32,11 +32,11 @@ func TestSegmentManagement_sendSegmentStatistic(t *testing.T) {
|
|||
|
||||
var statsMsgStream msgstream.MsgStream = statsStream
|
||||
|
||||
node.statsService = newStatsService(node.queryNodeLoopCtx, node.replica)
|
||||
node.statsService = newStatsService(node.queryNodeLoopCtx, node.replica, nil)
|
||||
node.statsService.statsStream = statsMsgStream
|
||||
node.statsService.statsStream.Start()
|
||||
|
||||
// send stats
|
||||
node.statsService.sendSegmentStatistic()
|
||||
node.statsService.publicStatistic(nil)
|
||||
node.Close()
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue