Skip to content
Snippets Groups Projects
Commit f5198bca authored by xige-16's avatar xige-16 Committed by yefu.chen
Browse files

Add LoadIndexService in query node


Signed-off-by: default avatarxige-16 <xi.ge@zilliz.com>
parent add31188
No related branches found
No related tags found
No related merge requests found
Showing
with 443 additions and 315 deletions
......@@ -5,4 +5,4 @@ TARGET_TAG=latest
PULSAR_ADDRESS=pulsar://pulsar:6650
ETCD_ADDRESS=etcd:2379
MASTER_ADDRESS=master:53100
MINIO_ADDRESS=minio:9000
\ No newline at end of file
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
// 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}
)
......
// 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;
}
}
// 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;
}
}
\ No newline at end of file
......@@ -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) {
......
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 {
......
This diff is collapsed.
......@@ -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,
......
0% or .
You are about to add 0 people to the discussion. Proceed with caution.
Finish editing this message first!
Please register or to comment