Add docker file for masterservice

Signed-off-by: neza2017 <yefu.chen@zilliz.com>
This commit is contained in:
neza2017 2021-01-25 15:58:04 +08:00 committed by yefu.chen
parent a8fd6e7686
commit 7f3aa92d10
24 changed files with 254 additions and 312 deletions

View File

@ -113,6 +113,8 @@ build-go: build-cpp
@echo "Building each component's binary to './bin'"
@echo "Building master ..."
@mkdir -p $(INSTALL_PATH) && go env -w CGO_ENABLED="0" && GO111MODULE=on $(GO) build -o $(INSTALL_PATH)/master $(PWD)/cmd/master/main.go 1>/dev/null
@echo "Building masterservice ..."
@mkdir -p $(INSTALL_PATH) && go env -w CGO_ENABLED="1" && GO111MODULE=on $(GO) build -o $(INSTALL_PATH)/masterservice $(PWD)/cmd/masterservice/main.go 1>/dev/null
@echo "Building proxy service ..."
@mkdir -p $(INSTALL_PATH) && go env -w CGO_ENABLED="0" && GO111MODULE=on $(GO) build -o $(INSTALL_PATH)/proxyservice $(PWD)/cmd/proxy/service/proxy_service.go 1>/dev/null
@echo "Building proxy node ..."
@ -132,6 +134,7 @@ build-go: build-cpp
@echo "Building distributed indexnode ..."
@mkdir -p $(INSTALL_PATH)/distributed && go env -w CGO_ENABLED="1" && GO111MODULE=on $(GO) build -o $(INSTALL_PATH)/distributed/indexnode $(PWD)/cmd/distributed/indexnode/main.go 1>/dev/null
build-cpp:
@(env bash $(PWD)/scripts/core_build.sh -f "$(CUSTOM_THIRDPARTY_PATH)")
@(env bash $(PWD)/scripts/cwrapper_build.sh -t Release -f "$(CUSTOM_THIRDPARTY_PATH)")

View File

@ -0,0 +1,41 @@
# 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.
FROM milvusdb/milvus-distributed-dev:amd64-ubuntu18.04-latest AS openblas
#FROM alpine
FROM ubuntu:bionic-20200921
RUN apt-get update && apt-get install -y --no-install-recommends libtbb-dev gfortran
#RUN echo "http://dl-cdn.alpinelinux.org/alpine/edge/testing" >> /etc/apk/repositories
#RUN sed -i "s/dl-cdn.alpinelinux.org/mirrors.aliyun.com/g" /etc/apk/repositories \
# && apk add --no-cache libtbb gfortran
COPY --from=openblas /usr/lib/libopenblas-r0.3.9.so /usr/lib/
RUN ln -s /usr/lib/libopenblas-r0.3.9.so /usr/lib/libopenblas.so.0 && \
ln -s /usr/lib/libopenblas.so.0 /usr/lib/libopenblas.so
COPY ./bin/masterservice /milvus-distributed/bin/masterservice
COPY ./configs/ /milvus-distributed/configs/
COPY ./lib/ /milvus-distributed/lib/
ENV LD_LIBRARY_PATH=/milvus-distributed/lib:$LD_LIBRARY_PATH:/usr/lib
WORKDIR /milvus-distributed/
CMD ["./bin/masterservice"]
EXPOSE 53100

View File

@ -24,13 +24,14 @@ const reTryCnt = 3
func main() {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
log.Printf("master service address : %s:%d", ms.Params.Address, ms.Params.Port)
svr, err := msc.NewGrpcServer(ctx)
if err != nil {
panic(err)
}
log.Printf("master service address : %s:%d", ms.Params.Address, ms.Params.Port)
psc.Params.Init()
log.Printf("proxy service address : %s", psc.Params.NetworkAddress())
//proxyService := psc.NewClient(ctx, psc.Params.NetworkAddress())
@ -40,6 +41,7 @@ func main() {
// panic(err)
//}
ds.Params.Init()
log.Printf("data service address : %s:%d", ds.Params.Address, ds.Params.Port)
dataService := dsc.NewClient(fmt.Sprintf("%s:%d", ds.Params.Address, ds.Params.Port))
if err = dataService.Init(); err != nil {
@ -52,9 +54,11 @@ func main() {
for cnt = 0; cnt < reTryCnt; cnt++ {
dsStates, err := dataService.GetComponentStates()
if err != nil {
log.Printf("retry cout = %d, error = %s", cnt, err.Error())
continue
}
if dsStates.Status.ErrorCode != commonpb.ErrorCode_SUCCESS {
log.Printf("retry cout = %d, error = %s", cnt, dsStates.Status.Reason)
continue
}
if dsStates.State.StateCode != internalpb2.StateCode_INITIALIZING && dsStates.State.StateCode != internalpb2.StateCode_HEALTHY {
@ -66,9 +70,9 @@ func main() {
panic("connect to data service failed")
}
//if err = svr.SetDataService(dataService); err != nil {
// panic(err)
//}
if err = svr.SetDataService(dataService); err != nil {
panic(err)
}
log.Printf("index service address : %s", is.Params.Address)
indexService := isc.NewClient(is.Params.Address)

View File

@ -8,8 +8,6 @@ set(MILVUS_QUERY_SRCS
visitors/ExecExprVisitor.cpp
visitors/VerifyPlanNodeVisitor.cpp
visitors/VerifyExprVisitor.cpp
visitors/ExtractInfoPlanNodeVisitor.cpp
visitors/ExtractInfoExprVisitor.cpp
Plan.cpp
SearchOnGrowing.cpp
SearchOnSealed.cpp

View File

@ -22,7 +22,6 @@
#include <boost/algorithm/string.hpp>
#include <algorithm>
#include "query/generated/VerifyPlanNodeVisitor.h"
#include "query/generated/ExtractInfoPlanNodeVisitor.h"
namespace milvus::query {
@ -143,14 +142,9 @@ Parser::CreatePlanImpl(const std::string& dsl_str) {
VerifyPlanNodeVisitor verifier;
vec_node->accept(verifier);
ExtractedPlanInfo plan_info(schema.size());
ExtractInfoPlanNodeVisitor extractor(plan_info);
vec_node->accept(extractor);
auto plan = std::make_unique<Plan>(schema);
plan->tag2field_ = std::move(tag2field_);
plan->plan_node_ = std::move(vec_node);
plan->extra_info_opt_ = std::move(plan_info);
return plan;
}

View File

@ -20,28 +20,11 @@
#include <map>
#include <string>
#include <vector>
#include <optional>
#include <boost/dynamic_bitset.hpp>
namespace milvus::query {
using Json = nlohmann::json;
// class definitions
struct ExtractedPlanInfo {
public:
explicit ExtractedPlanInfo(int64_t size) : involved_fields_(size) {
}
void
add_involved_field(FieldOffset field_offset) {
involved_fields_.set(field_offset.get());
}
public:
boost::dynamic_bitset<> involved_fields_;
};
struct Plan {
public:
explicit Plan(const Schema& schema) : schema_(schema) {
@ -52,9 +35,7 @@ struct Plan {
std::unique_ptr<VectorPlanNode> plan_node_;
std::map<std::string, FieldOffset> tag2field_; // PlaceholderName -> FieldOffset
std::vector<FieldOffset> target_entries_;
public:
std::optional<ExtractedPlanInfo> extra_info_opt_;
std::vector<FieldOffset> referred_fields_;
// TODO: add move extra info
};

View File

@ -1,3 +0,0 @@
!.gitignore
*PlanNodeVisitor.cpp
*ExprVisitor.cpp

View File

@ -9,26 +9,28 @@
// 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
// Generated File
// DO NOT EDIT
#include "query/Plan.h"
#include "PlanNodeVisitor.h"
#error TODO: copy this file out, and modify the content.
#include "query/generated/ExecExprVisitor.h"
namespace milvus::query {
class ExtractInfoPlanNodeVisitor : public PlanNodeVisitor {
public:
void
visit(FloatVectorANNS& node) override;
void
ExecExprVisitor::visit(BoolUnaryExpr& expr) {
// TODO
}
void
visit(BinaryVectorANNS& node) override;
void
ExecExprVisitor::visit(BoolBinaryExpr& expr) {
// TODO
}
public:
explicit ExtractInfoPlanNodeVisitor(ExtractedPlanInfo& plan_info) : plan_info_(plan_info) {
}
void
ExecExprVisitor::visit(TermExpr& expr) {
// TODO
}
void
ExecExprVisitor::visit(RangeExpr& expr) {
// TODO
}
private:
ExtractedPlanInfo& plan_info_;
};
} // namespace milvus::query

View File

@ -0,0 +1,26 @@
// 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
#error TODO: copy this file out, and modify the content.
#include "query/generated/ExecPlanNodeVisitor.h"
namespace milvus::query {
void
ExecPlanNodeVisitor::visit(FloatVectorANNS& node) {
// TODO
}
void
ExecPlanNodeVisitor::visit(BinaryVectorANNS& node) {
// TODO
}
} // namespace milvus::query

View File

@ -9,32 +9,28 @@
// 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
// Generated File
// DO NOT EDIT
#include "query/Plan.h"
#include "ExprVisitor.h"
#error TODO: copy this file out, and modify the content.
#include "query/generated/ShowExprVisitor.h"
namespace milvus::query {
class ExtractInfoExprVisitor : public ExprVisitor {
public:
void
visit(BoolUnaryExpr& expr) override;
void
ShowExprVisitor::visit(BoolUnaryExpr& expr) {
// TODO
}
void
visit(BoolBinaryExpr& expr) override;
void
ShowExprVisitor::visit(BoolBinaryExpr& expr) {
// TODO
}
void
visit(TermExpr& expr) override;
void
ShowExprVisitor::visit(TermExpr& expr) {
// TODO
}
void
visit(RangeExpr& expr) override;
void
ShowExprVisitor::visit(RangeExpr& expr) {
// TODO
}
public:
explicit ExtractInfoExprVisitor(ExtractedPlanInfo& plan_info) : plan_info_(plan_info) {
}
private:
ExtractedPlanInfo& plan_info_;
};
} // namespace milvus::query

View File

@ -0,0 +1,26 @@
// 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
#error TODO: copy this file out, and modify the content.
#include "query/generated/ShowPlanNodeVisitor.h"
namespace milvus::query {
void
ShowPlanNodeVisitor::visit(FloatVectorANNS& node) {
// TODO
}
void
ShowPlanNodeVisitor::visit(BinaryVectorANNS& node) {
// TODO
}
} // namespace milvus::query

View File

@ -0,0 +1,36 @@
// 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
#error TODO: copy this file out, and modify the content.
#include "query/generated/VerifyExprVisitor.h"
namespace milvus::query {
void
VerifyExprVisitor::visit(BoolUnaryExpr& expr) {
// TODO
}
void
VerifyExprVisitor::visit(BoolBinaryExpr& expr) {
// TODO
}
void
VerifyExprVisitor::visit(TermExpr& expr) {
// TODO
}
void
VerifyExprVisitor::visit(RangeExpr& expr) {
// TODO
}
} // namespace milvus::query

View File

@ -0,0 +1,26 @@
// 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
#error TODO: copy this file out, and modify the content.
#include "query/generated/VerifyPlanNodeVisitor.h"
namespace milvus::query {
void
VerifyPlanNodeVisitor::visit(FloatVectorANNS& node) {
// TODO
}
void
VerifyPlanNodeVisitor::visit(BinaryVectorANNS& node) {
// TODO
}
} // namespace milvus::query

View File

@ -1,52 +0,0 @@
// Copyright (C) 2019-2020 Zilliz. All rights reserved.
//
// Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software distributed under the License
// is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
// or implied. See the License for the specific language governing permissions and limitations under the License
#include "query/Plan.h"
#include "query/generated/ExtractInfoExprVisitor.h"
namespace milvus::query {
#if 1
namespace impl {
// THIS CONTAINS EXTRA BODY FOR VISITOR
// WILL BE USED BY GENERATOR UNDER suvlim/core_gen/
class ExtractInfoExprVisitor : ExprVisitor {
public:
explicit ExtractInfoExprVisitor(ExtractedPlanInfo& plan_info) : plan_info_(plan_info) {
}
private:
ExtractedPlanInfo& plan_info_;
};
} // namespace impl
#endif
void
ExtractInfoExprVisitor::visit(BoolUnaryExpr& expr) {
expr.child_->accept(*this);
}
void
ExtractInfoExprVisitor::visit(BoolBinaryExpr& expr) {
expr.left_->accept(*this);
expr.right_->accept(*this);
}
void
ExtractInfoExprVisitor::visit(TermExpr& expr) {
plan_info_.add_involved_field(expr.field_offset_);
}
void
ExtractInfoExprVisitor::visit(RangeExpr& expr) {
plan_info_.add_involved_field(expr.field_offset_);
}
} // namespace milvus::query

View File

@ -1,51 +0,0 @@
// Copyright (C) 2019-2020 Zilliz. All rights reserved.
//
// Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software distributed under the License
// is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
// or implied. See the License for the specific language governing permissions and limitations under the License
#include "query/Plan.h"
#include "query/generated/ExtractInfoPlanNodeVisitor.h"
#include "query/generated/ExtractInfoExprVisitor.h"
namespace milvus::query {
#if 1
namespace impl {
// THIS CONTAINS EXTRA BODY FOR VISITOR
// WILL BE USED BY GENERATOR UNDER suvlim/core_gen/
class ExtractInfoPlanNodeVisitor : PlanNodeVisitor {
public:
explicit ExtractInfoPlanNodeVisitor(ExtractedPlanInfo& plan_info) : plan_info_(plan_info) {
}
private:
ExtractedPlanInfo& plan_info_;
};
} // namespace impl
#endif
void
ExtractInfoPlanNodeVisitor::visit(FloatVectorANNS& node) {
plan_info_.add_involved_field(node.query_info_.field_offset_);
if (node.predicate_.has_value()) {
ExtractInfoExprVisitor expr_visitor(plan_info_);
node.predicate_.value()->accept(expr_visitor);
}
}
void
ExtractInfoPlanNodeVisitor::visit(BinaryVectorANNS& node) {
plan_info_.add_involved_field(node.query_info_.field_offset_);
if (node.predicate_.has_value()) {
ExtractInfoExprVisitor expr_visitor(plan_info_);
node.predicate_.value()->accept(expr_visitor);
}
}
} // namespace milvus::query

View File

@ -46,12 +46,6 @@ struct SealedIndexingRecord {
return field_indexings_.at(field_offset).get();
}
void
drop_field_indexing(FieldOffset field_offset) {
std::unique_lock lck(mutex_);
field_indexings_.erase(field_offset);
}
bool
is_ready(FieldOffset field_offset) const {
std::shared_lock lck(mutex_);

View File

@ -158,6 +158,9 @@ class SegmentGrowingImpl : public SegmentGrowing {
bulk_subscript_impl<int64_t>(*vec_ptr, seg_offsets, count, output);
}
int64_t
num_chunk() const override;
Status
LoadIndexing(const LoadIndexInfo& info) override;
@ -185,17 +188,9 @@ class SegmentGrowingImpl : public SegmentGrowing {
get_deleted_bitmap(int64_t del_barrier, Timestamp query_timestamp, int64_t insert_barrier, bool force = false);
protected:
int64_t
num_chunk() const override;
SpanBase
chunk_data_impl(FieldOffset field_offset, int64_t chunk_id) const override;
void
check_search(const query::Plan* plan) const override {
Assert(plan);
}
private:
int64_t size_per_chunk_;
SchemaPtr schema_;

View File

@ -48,7 +48,6 @@ SegmentInternalInterface::Search(const query::Plan* plan,
const Timestamp* timestamps,
int64_t num_groups) const {
std::shared_lock lck(mutex_);
check_search(plan);
Assert(num_groups == 1);
query::ExecPlanNodeVisitor visitor(*this, timestamps[0], *placeholder_groups[0]);
auto results = visitor.get_moved_result(*plan->plan_node_);

View File

@ -117,9 +117,6 @@ class SegmentInternalInterface : public SegmentInterface {
virtual void
bulk_subscript(FieldOffset field_offset, const int64_t* seg_offsets, int64_t count, void* output) const = 0;
virtual void
check_search(const query::Plan* plan) const = 0;
protected:
mutable std::shared_mutex mutex_;
};

View File

@ -31,7 +31,7 @@ SegmentSealedImpl::LoadIndex(const LoadIndexInfo& info) {
}
Assert(!vec_indexings_.is_ready(field_offset));
vec_indexings_.append_field_indexing(field_offset, GetMetricType(metric_type_str), info.index);
set_field_ready(field_offset, true);
++ready_count_;
}
void
@ -70,10 +70,10 @@ SegmentSealedImpl::LoadFieldData(const LoadFieldDataInfo& info) {
// write data under lock
std::unique_lock lck(mutex_);
update_row_count(info.row_count);
AssertInfo(field_datas_[field_offset.get()].empty(), "already exists");
field_datas_[field_offset.get()] = std::move(vec_data);
AssertInfo(columns_data_[field_offset.get()].empty(), "already exists");
columns_data_[field_offset.get()] = std::move(vec_data);
set_field_ready(field_offset, true);
++ready_count_;
}
}
@ -96,10 +96,10 @@ SegmentSealedImpl::size_per_chunk() const {
SpanBase
SegmentSealedImpl::chunk_data_impl(FieldOffset field_offset, int64_t chunk_id) const {
std::shared_lock lck(mutex_);
Assert(is_field_ready(field_offset));
auto& field_meta = schema_->operator[](field_offset);
auto element_sizeof = field_meta.get_sizeof();
SpanBase base(field_datas_[field_offset.get()].data(), row_count_opt_.value(), element_sizeof);
Assert(is_all_ready());
SpanBase base(columns_data_[field_offset.get()].data(), row_count_opt_.value(), element_sizeof);
return base;
}
@ -143,39 +143,13 @@ SegmentSealedImpl::vector_search(int64_t vec_count,
}
void
SegmentSealedImpl::DropFieldData(const FieldId field_id) {
if (SystemProperty::Instance().IsSystem(field_id)) {
auto system_field_type = SystemProperty::Instance().GetSystemFieldType(field_id);
Assert(system_field_type == SystemFieldType::RowId);
std::unique_lock lck(mutex_);
--system_ready_count_;
auto row_ids = std::move(row_ids_);
lck.unlock();
row_ids.clear();
} else {
auto field_offset = schema_->get_offset(field_id);
auto& field_meta = schema_->operator[](field_offset);
Assert(!field_meta.is_vector());
std::unique_lock lck(mutex_);
set_field_ready(field_offset, false);
auto vec = std::move(field_datas_[field_offset.get()]);
lck.unlock();
vec.clear();
}
PanicInfo("unimplemented");
}
void
SegmentSealedImpl::DropIndex(const FieldId field_id) {
Assert(!SystemProperty::Instance().IsSystem(field_id));
auto field_offset = schema_->get_offset(field_id);
auto& field_meta = schema_->operator[](field_offset);
Assert(field_meta.is_vector());
std::unique_lock lck(mutex_);
vec_indexings_.drop_field_indexing(field_offset);
PanicInfo("unimplemented");
}
SegmentSealedPtr

View File

@ -18,8 +18,7 @@
namespace milvus::segcore {
class SegmentSealedImpl : public SegmentSealed {
public:
explicit SegmentSealedImpl(SchemaPtr schema)
: schema_(schema), field_datas_(schema->size()), field_ready_bitset_(schema->size()) {
explicit SegmentSealedImpl(SchemaPtr schema) : schema_(schema), columns_data_(schema->size()) {
}
void
LoadIndex(const LoadIndexInfo& info) override;
@ -66,7 +65,7 @@ class SegmentSealedImpl : public SegmentSealed {
const int64_t* seg_offsets,
int64_t count,
void* output) const override {
Assert(is_system_field_ready());
Assert(is_all_ready());
Assert(system_type == SystemFieldType::RowId);
bulk_subscript_impl<int64_t>(row_ids_.data(), seg_offsets, count, output);
}
@ -75,29 +74,10 @@ class SegmentSealedImpl : public SegmentSealed {
// where Vec is determined from field_offset
void
bulk_subscript(FieldOffset field_offset, const int64_t* seg_offsets, int64_t count, void* output) const override {
Assert(is_field_ready(field_offset));
Assert(is_all_ready());
auto& field_meta = schema_->operator[](field_offset);
Assert(field_meta.get_data_type() == DataType::INT64);
bulk_subscript_impl<int64_t>(field_datas_[field_offset.get()].data(), seg_offsets, count, output);
}
void
check_search(const query::Plan* plan) const override {
Assert(plan);
Assert(plan->extra_info_opt_.has_value());
if (!is_system_field_ready()) {
PanicInfo("System Field RowID is not loaded");
}
auto& request_fields = plan->extra_info_opt_.value().involved_fields_;
Assert(request_fields.size() == field_ready_bitset_.size());
auto absent_fields = request_fields - field_ready_bitset_;
if (absent_fields.any()) {
auto field_offset = FieldOffset(absent_fields.find_first());
auto& field_meta = schema_->operator[](field_offset);
PanicInfo("User Field(" + field_meta.get_name().get() + ") is not loaded");
}
bulk_subscript_impl<int64_t>(columns_data_[field_offset.get()].data(), seg_offsets, count, output);
}
private:
@ -136,25 +116,25 @@ class SegmentSealedImpl : public SegmentSealed {
}
bool
is_field_ready(FieldOffset field_offset) const {
return field_ready_bitset_.test(field_offset.get());
is_all_ready() const {
// TODO: optimize here
// NOTE: including row_ids
if (!is_system_field_ready()) {
return false;
}
void
set_field_ready(FieldOffset field_offset, bool flag = true) {
field_ready_bitset_[field_offset.get()] = flag;
return ready_count_ == schema_->size();
}
private:
// segment loading state
boost::dynamic_bitset<> field_ready_bitset_;
std::atomic<int> ready_count_ = 0;
std::atomic<int> system_ready_count_ = 0;
// segment datas
// TODO: generate index for scalar
std::optional<int64_t> row_count_opt_;
std::map<FieldOffset, knowhere::IndexPtr> scalar_indexings_;
SealedIndexingRecord vec_indexings_;
std::vector<aligned_vector<char>> field_datas_;
std::vector<aligned_vector<char>> columns_data_;
aligned_vector<idx_t> row_ids_;
SchemaPtr schema_;
};

View File

@ -249,9 +249,8 @@ TEST(Sealed, LoadFieldData) {
auto metric_type = MetricType::METRIC_L2;
auto schema = std::make_shared<Schema>();
auto fakevec_id = schema->AddDebugField("fakevec", DataType::VECTOR_FLOAT, dim, metric_type);
auto counter_id = schema->AddDebugField("counter", DataType::INT64);
auto double_id = schema->AddDebugField("double", DataType::DOUBLE);
auto nothing_id = schema->AddDebugField("nothing", DataType::INT32);
schema->AddDebugField("counter", DataType::INT64);
schema->AddDebugField("double", DataType::DOUBLE);
auto dataset = DataGen(schema, N);
@ -269,6 +268,24 @@ TEST(Sealed, LoadFieldData) {
indexing->AddWithoutIds(database, conf);
auto segment = CreateSealedSegment(schema);
SealedLoader(dataset, *segment);
{
LoadIndexInfo vec_info;
vec_info.field_id = fakevec_id.get();
vec_info.field_name = "fakevec";
vec_info.index = indexing;
vec_info.index_params["metric_type"] = milvus::knowhere::Metric::L2;
segment->LoadIndex(vec_info);
}
ASSERT_EQ(segment->num_chunk(), 1);
auto chunk_span1 = segment->chunk_data<int64_t>(FieldOffset(1), 0);
auto chunk_span2 = segment->chunk_data<double>(FieldOffset(2), 0);
auto ref1 = dataset.get_col<int64_t>(1);
auto ref2 = dataset.get_col<double>(2);
for (int i = 0; i < N; ++i) {
ASSERT_EQ(chunk_span1[i], ref1[i]);
ASSERT_EQ(chunk_span2[i], ref2[i]);
}
std::string dsl = R"({
"bool": {
"must": [
@ -296,47 +313,14 @@ TEST(Sealed, LoadFieldData) {
}
})";
Timestamp time = 1000000;
auto plan = CreatePlan(*schema, dsl);
auto num_queries = 5;
auto ph_group_raw = CreatePlaceholderGroup(num_queries, 16, 1024);
auto ph_group = ParsePlaceholderGroup(plan.get(), ph_group_raw.SerializeAsString());
Timestamp time = 1000000;
std::vector<const PlaceholderGroup*> ph_group_arr = {ph_group.get()};
ASSERT_ANY_THROW(segment->Search(plan.get(), ph_group_arr.data(), &time, 1));
SealedLoader(dataset, *segment);
segment->DropFieldData(nothing_id);
ASSERT_ANY_THROW(segment->Search(plan.get(), ph_group_arr.data(), &time, 1));
LoadIndexInfo vec_info;
vec_info.field_id = fakevec_id.get();
vec_info.field_name = "fakevec";
vec_info.index = indexing;
vec_info.index_params["metric_type"] = milvus::knowhere::Metric::L2;
segment->LoadIndex(vec_info);
ASSERT_EQ(segment->num_chunk(), 1);
auto chunk_span1 = segment->chunk_data<int64_t>(FieldOffset(1), 0);
auto chunk_span2 = segment->chunk_data<double>(FieldOffset(2), 0);
auto ref1 = dataset.get_col<int64_t>(1);
auto ref2 = dataset.get_col<double>(2);
for (int i = 0; i < N; ++i) {
ASSERT_EQ(chunk_span1[i], ref1[i]);
ASSERT_EQ(chunk_span2[i], ref2[i]);
}
auto qr = segment->Search(plan.get(), ph_group_arr.data(), &time, 1);
auto json = QueryResultToJson(qr);
std::cout << json.dump(1);
segment->DropIndex(fakevec_id);
ASSERT_ANY_THROW(segment->Search(plan.get(), ph_group_arr.data(), &time, 1));
segment->LoadIndex(vec_info);
auto qr2 = segment->Search(plan.get(), ph_group_arr.data(), &time, 1);
auto json2 = QueryResultToJson(qr);
ASSERT_EQ(json.dump(-2), json2.dump(-2));
segment->DropFieldData(double_id);
ASSERT_ANY_THROW(segment->Search(plan.get(), ph_group_arr.data(), &time, 1));
}

View File

@ -55,11 +55,11 @@ func (c *Client) Stop() error {
}
func (c *Client) GetComponentStates() (*internalpb2.ComponentStates, error) {
return c.grpcClient.GetComponentStates(context.Background(), nil)
return c.grpcClient.GetComponentStates(context.Background(), &commonpb.Empty{})
}
func (c *Client) GetTimeTickChannel() (string, error) {
resp, err := c.grpcClient.GetTimeTickChannel(context.Background(), nil)
resp, err := c.grpcClient.GetTimeTickChannel(context.Background(), &commonpb.Empty{})
if err != nil {
return "", err
}
@ -70,7 +70,7 @@ func (c *Client) GetTimeTickChannel() (string, error) {
}
func (c *Client) GetStatisticsChannel() (string, error) {
resp, err := c.grpcClient.GetStatisticsChannel(context.Background(), nil)
resp, err := c.grpcClient.GetStatisticsChannel(context.Background(), &commonpb.Empty{})
if err != nil {
return "", err
}
@ -124,7 +124,7 @@ func (c *Client) GetPartitionStatistics(req *datapb.PartitionStatsRequest) (*dat
}
func (c *Client) GetSegmentInfoChannel() (string, error) {
resp, err := c.grpcClient.GetSegmentInfoChannel(context.Background(), nil)
resp, err := c.grpcClient.GetSegmentInfoChannel(context.Background(), &commonpb.Empty{})
if err != nil {
return "", err
}

View File

@ -62,10 +62,6 @@ if __name__ == "__main__":
'visitor_name': "VerifyExprVisitor",
"parameter_name": 'expr',
},
{
'visitor_name': "ExtractInfoExprVisitor",
"parameter_name": 'expr',
},
],
'PlanNode': [
{
@ -80,10 +76,6 @@ if __name__ == "__main__":
'visitor_name': "VerifyPlanNodeVisitor",
"parameter_name": 'node',
},
{
'visitor_name': "ExtractInfoPlanNodeVisitor",
"parameter_name": 'node',
},
]
}
extract_extra_body(visitor_info, query_path)