mirror of
https://gitee.com/milvus-io/milvus.git
synced 2025-12-06 17:18:35 +08:00
enhance: [AddField] Broadcast schema change via WAL (#41373)
Related to #39718 Add Broadcast logic for collection schema change and notifies: - Streamnode - Delegator - Streamnode - Flush component - QueryNodes via grpc --------- Signed-off-by: Congqi Xia <congqi.xia@zilliz.com>
This commit is contained in:
parent
110c5aaaf4
commit
b36c88f3c8
@ -64,4 +64,16 @@ Collection::parseIndexMeta(const void* index_proto, const int64_t length) {
|
|||||||
LOG_INFO("index meta info: {}", index_meta_->ToString());
|
LOG_INFO("index meta info: {}", index_meta_->ToString());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
void
|
||||||
|
Collection::parse_schema(const void* schema_proto_blob, const int64_t length) {
|
||||||
|
Assert(schema_proto_blob != nullptr);
|
||||||
|
|
||||||
|
milvus::proto::schema::CollectionSchema collection_schema;
|
||||||
|
auto suc = collection_schema.ParseFromArray(schema_proto_blob, length);
|
||||||
|
|
||||||
|
AssertInfo(suc, "parse schema proto failed");
|
||||||
|
|
||||||
|
schema_ = Schema::ParseFrom(collection_schema);
|
||||||
|
}
|
||||||
|
|
||||||
} // namespace milvus::segcore
|
} // namespace milvus::segcore
|
||||||
|
|||||||
@ -28,6 +28,9 @@ class Collection {
|
|||||||
void
|
void
|
||||||
parseIndexMeta(const void* index_meta_proto_blob, const int64_t length);
|
parseIndexMeta(const void* index_meta_proto_blob, const int64_t length);
|
||||||
|
|
||||||
|
void
|
||||||
|
parse_schema(const void* schema_proto_blob, const int64_t length);
|
||||||
|
|
||||||
public:
|
public:
|
||||||
SchemaPtr&
|
SchemaPtr&
|
||||||
get_schema() {
|
get_schema() {
|
||||||
|
|||||||
@ -9,6 +9,8 @@
|
|||||||
// is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
|
// 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
|
// or implied. See the License for the specific language governing permissions and limitations under the License
|
||||||
|
|
||||||
|
#include <exception>
|
||||||
|
#include "common/EasyAssert.h"
|
||||||
#include "common/type_c.h"
|
#include "common/type_c.h"
|
||||||
#ifdef __linux__
|
#ifdef __linux__
|
||||||
#include <malloc.h>
|
#include <malloc.h>
|
||||||
@ -32,6 +34,20 @@ NewCollection(const void* schema_proto_blob,
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
CStatus
|
||||||
|
UpdateSchema(CCollection collection,
|
||||||
|
const void* proto_blob,
|
||||||
|
const int64_t length) {
|
||||||
|
try {
|
||||||
|
auto col = static_cast<milvus::segcore::Collection*>(collection);
|
||||||
|
|
||||||
|
col->parse_schema(proto_blob, length);
|
||||||
|
return milvus::SuccessCStatus();
|
||||||
|
} catch (std::exception& e) {
|
||||||
|
return milvus::FailureCStatus(&e);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
CStatus
|
CStatus
|
||||||
SetIndexMeta(CCollection collection,
|
SetIndexMeta(CCollection collection,
|
||||||
const void* proto_blob,
|
const void* proto_blob,
|
||||||
|
|||||||
@ -25,6 +25,11 @@ NewCollection(const void* schema_proto_blob,
|
|||||||
const int64_t length,
|
const int64_t length,
|
||||||
CCollection* collection);
|
CCollection* collection);
|
||||||
|
|
||||||
|
CStatus
|
||||||
|
UpdateSchema(CCollection collection,
|
||||||
|
const void* proto_blob,
|
||||||
|
const int64_t length);
|
||||||
|
|
||||||
CStatus
|
CStatus
|
||||||
SetIndexMeta(CCollection collection,
|
SetIndexMeta(CCollection collection,
|
||||||
const void* proto_blob,
|
const void* proto_blob,
|
||||||
|
|||||||
@ -30,7 +30,9 @@
|
|||||||
#include "index/IndexFactory.h"
|
#include "index/IndexFactory.h"
|
||||||
#include "knowhere/comp/index_param.h"
|
#include "knowhere/comp/index_param.h"
|
||||||
#include "pb/plan.pb.h"
|
#include "pb/plan.pb.h"
|
||||||
|
#include "pb/schema.pb.h"
|
||||||
#include "segcore/Collection.h"
|
#include "segcore/Collection.h"
|
||||||
|
#include "segcore/collection_c.h"
|
||||||
#include "segcore/reduce/Reduce.h"
|
#include "segcore/reduce/Reduce.h"
|
||||||
#include "segcore/reduce_c.h"
|
#include "segcore/reduce_c.h"
|
||||||
#include "segcore/segment_c.h"
|
#include "segcore/segment_c.h"
|
||||||
@ -239,6 +241,71 @@ TEST(CApiTest, CollectionTest) {
|
|||||||
DeleteCollection(collection);
|
DeleteCollection(collection);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
TEST(CApiTest, UpdateSchemaTest) {
|
||||||
|
std::string schema_string = generate_collection_schema<milvus::FloatVector>(
|
||||||
|
knowhere::metric::L2, DIM);
|
||||||
|
// CCollection
|
||||||
|
auto collection = NewCollection(schema_string.c_str());
|
||||||
|
|
||||||
|
// create updated schema with extra field
|
||||||
|
namespace schema = milvus::proto::schema;
|
||||||
|
schema::CollectionSchema collection_schema;
|
||||||
|
collection_schema.set_name("collection_test");
|
||||||
|
|
||||||
|
auto vec_field_schema = collection_schema.add_fields();
|
||||||
|
vec_field_schema->set_name("fakevec");
|
||||||
|
vec_field_schema->set_fieldid(100);
|
||||||
|
vec_field_schema->set_data_type(schema::DataType::FloatVector);
|
||||||
|
auto metric_type_param = vec_field_schema->add_index_params();
|
||||||
|
metric_type_param->set_key("metric_type");
|
||||||
|
metric_type_param->set_value(knowhere::metric::L2);
|
||||||
|
auto dim_param = vec_field_schema->add_type_params();
|
||||||
|
dim_param->set_key("dim");
|
||||||
|
dim_param->set_value(std::to_string(DIM));
|
||||||
|
|
||||||
|
auto other_field_schema = collection_schema.add_fields();
|
||||||
|
other_field_schema->set_name("counter");
|
||||||
|
other_field_schema->set_fieldid(101);
|
||||||
|
other_field_schema->set_data_type(schema::DataType::Int64);
|
||||||
|
other_field_schema->set_is_primary_key(true);
|
||||||
|
|
||||||
|
auto other_field_schema2 = collection_schema.add_fields();
|
||||||
|
other_field_schema2->set_name("doubleField");
|
||||||
|
other_field_schema2->set_fieldid(102);
|
||||||
|
other_field_schema2->set_data_type(schema::DataType::Double);
|
||||||
|
|
||||||
|
auto added_field = collection_schema.add_fields();
|
||||||
|
added_field->set_name("added_field");
|
||||||
|
added_field->set_fieldid(103);
|
||||||
|
added_field->set_data_type(schema::DataType::Int64);
|
||||||
|
added_field->set_nullable(true);
|
||||||
|
|
||||||
|
std::string updated_schema_string;
|
||||||
|
auto marshal = collection_schema.SerializeToString(&updated_schema_string);
|
||||||
|
ASSERT_TRUE(marshal);
|
||||||
|
|
||||||
|
// Call UpdateSchema CApi here
|
||||||
|
// UpdateSchema(CCollection, const void*, const int64_t)
|
||||||
|
auto status = UpdateSchema(collection,
|
||||||
|
updated_schema_string.c_str(),
|
||||||
|
updated_schema_string.length());
|
||||||
|
ASSERT_EQ(status.error_code, Success);
|
||||||
|
|
||||||
|
auto col = static_cast<milvus::segcore::Collection*>(collection);
|
||||||
|
auto updated_schema = col->get_schema();
|
||||||
|
auto add_field = updated_schema->operator[](FieldId(103));
|
||||||
|
|
||||||
|
ASSERT_EQ(add_field.get_name().get(), "added_field");
|
||||||
|
|
||||||
|
// Test failure case, no panicking with failure code
|
||||||
|
status = UpdateSchema(collection, nullptr, 0);
|
||||||
|
ASSERT_NE(status.error_code, Success);
|
||||||
|
|
||||||
|
DeleteCollection(collection);
|
||||||
|
// free error msg, which shall be responsible for go side to call C.free()
|
||||||
|
free(const_cast<char*>(status.error_msg));
|
||||||
|
}
|
||||||
|
|
||||||
TEST(CApiTest, LoadInfoTest) {
|
TEST(CApiTest, LoadInfoTest) {
|
||||||
auto load_info = std::make_shared<LoadFieldDataInfo>();
|
auto load_info = std::make_shared<LoadFieldDataInfo>();
|
||||||
auto c_load_info = reinterpret_cast<CLoadFieldDataInfo*>(load_info.get());
|
auto c_load_info = reinterpret_cast<CLoadFieldDataInfo*>(load_info.get());
|
||||||
@ -4231,7 +4298,6 @@ TEST(CApiTest, GrowingSegment_Load_Field_Data_Lack_Binlog_Rows) {
|
|||||||
DeleteSegment(segment);
|
DeleteSegment(segment);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
TEST(CApiTest, DISABLED_SealedSegment_Load_Field_Data_Lack_Binlog_Rows) {
|
TEST(CApiTest, DISABLED_SealedSegment_Load_Field_Data_Lack_Binlog_Rows) {
|
||||||
double double_default_value = 20;
|
double double_default_value = 20;
|
||||||
auto schema = std::make_shared<Schema>();
|
auto schema = std::make_shared<Schema>();
|
||||||
|
|||||||
@ -45,7 +45,6 @@
|
|||||||
#include "storage/Util.h"
|
#include "storage/Util.h"
|
||||||
#include "milvus-storage/common/constants.h"
|
#include "milvus-storage/common/constants.h"
|
||||||
|
|
||||||
|
|
||||||
using boost::algorithm::starts_with;
|
using boost::algorithm::starts_with;
|
||||||
|
|
||||||
namespace milvus::segcore {
|
namespace milvus::segcore {
|
||||||
|
|||||||
@ -32,6 +32,7 @@ import (
|
|||||||
"github.com/milvus-io/milvus/pkg/v2/log"
|
"github.com/milvus-io/milvus/pkg/v2/log"
|
||||||
"github.com/milvus-io/milvus/pkg/v2/proto/internalpb"
|
"github.com/milvus-io/milvus/pkg/v2/proto/internalpb"
|
||||||
"github.com/milvus-io/milvus/pkg/v2/streaming/util/message"
|
"github.com/milvus-io/milvus/pkg/v2/streaming/util/message"
|
||||||
|
"github.com/milvus-io/milvus/pkg/v2/streaming/util/message/adaptor"
|
||||||
"github.com/milvus-io/milvus/pkg/v2/streaming/util/types"
|
"github.com/milvus-io/milvus/pkg/v2/streaming/util/types"
|
||||||
"github.com/milvus-io/milvus/pkg/v2/util/funcutil"
|
"github.com/milvus-io/milvus/pkg/v2/util/funcutil"
|
||||||
"github.com/milvus-io/milvus/pkg/v2/util/merr"
|
"github.com/milvus-io/milvus/pkg/v2/util/merr"
|
||||||
@ -89,6 +90,13 @@ func (m *msgHandlerImpl) HandleImport(ctx context.Context, vchannel string, impo
|
|||||||
}, retry.AttemptAlways())
|
}, retry.AttemptAlways())
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func (impl *msgHandlerImpl) HandleSchemaChange(ctx context.Context, vchannel string, msg *adaptor.SchemaChangeMessageBody) error {
|
||||||
|
return streaming.WAL().Broadcast().Ack(ctx, types.BroadcastAckRequest{
|
||||||
|
BroadcastID: msg.BroadcastID,
|
||||||
|
VChannel: vchannel,
|
||||||
|
})
|
||||||
|
}
|
||||||
|
|
||||||
func NewMsgHandlerImpl(broker broker.Broker) *msgHandlerImpl {
|
func NewMsgHandlerImpl(broker broker.Broker) *msgHandlerImpl {
|
||||||
return &msgHandlerImpl{
|
return &msgHandlerImpl{
|
||||||
broker: broker,
|
broker: broker,
|
||||||
|
|||||||
@ -360,3 +360,14 @@ func (c *Client) DeleteBatch(ctx context.Context, req *querypb.DeleteBatchReques
|
|||||||
return client.DeleteBatch(ctx, req)
|
return client.DeleteBatch(ctx, req)
|
||||||
})
|
})
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func (c *Client) UpdateSchema(ctx context.Context, req *querypb.UpdateSchemaRequest, _ ...grpc.CallOption) (*commonpb.Status, error) {
|
||||||
|
req = typeutil.Clone(req)
|
||||||
|
commonpbutil.UpdateMsgBase(
|
||||||
|
req.GetBase(),
|
||||||
|
commonpbutil.FillMsgBaseFromClient(c.nodeID),
|
||||||
|
)
|
||||||
|
return wrapGrpcCall(ctx, c, func(client querypb.QueryNodeClient) (*commonpb.Status, error) {
|
||||||
|
return client.UpdateSchema(ctx, req)
|
||||||
|
})
|
||||||
|
}
|
||||||
|
|||||||
@ -394,3 +394,7 @@ func (s *Server) Delete(ctx context.Context, req *querypb.DeleteRequest) (*commo
|
|||||||
func (s *Server) DeleteBatch(ctx context.Context, req *querypb.DeleteBatchRequest) (*querypb.DeleteBatchResponse, error) {
|
func (s *Server) DeleteBatch(ctx context.Context, req *querypb.DeleteBatchRequest) (*querypb.DeleteBatchResponse, error) {
|
||||||
return s.querynode.DeleteBatch(ctx, req)
|
return s.querynode.DeleteBatch(ctx, req)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func (s *Server) UpdateSchema(ctx context.Context, req *querypb.UpdateSchemaRequest) (*commonpb.Status, error) {
|
||||||
|
return s.querynode.UpdateSchema(ctx, req)
|
||||||
|
}
|
||||||
|
|||||||
@ -54,6 +54,8 @@ type MetaCache interface {
|
|||||||
DetectMissingSegments(segments map[int64]struct{}) []int64
|
DetectMissingSegments(segments map[int64]struct{}) []int64
|
||||||
// UpdateSegmentView updates the segments BF from datacoord view.
|
// UpdateSegmentView updates the segments BF from datacoord view.
|
||||||
UpdateSegmentView(partitionID int64, newSegments []*datapb.SyncSegmentInfo, newSegmentsBF []*pkoracle.BloomFilterSet, allSegments map[int64]struct{})
|
UpdateSegmentView(partitionID int64, newSegments []*datapb.SyncSegmentInfo, newSegmentsBF []*pkoracle.BloomFilterSet, allSegments map[int64]struct{})
|
||||||
|
// UpdateSchema update the latest collection schema
|
||||||
|
UpdateSchema(updatedSchema *schemapb.CollectionSchema)
|
||||||
}
|
}
|
||||||
|
|
||||||
var _ MetaCache = (*metaCacheImpl)(nil)
|
var _ MetaCache = (*metaCacheImpl)(nil)
|
||||||
@ -316,3 +318,7 @@ func (c *metaCacheImpl) UpdateSegmentView(partitionID int64,
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func (c *metaCacheImpl) UpdateSchema(updatedSchema *schemapb.CollectionSchema) {
|
||||||
|
c.schema = updatedSchema
|
||||||
|
}
|
||||||
|
|||||||
@ -545,6 +545,39 @@ func (_c *MockMetaCache_Schema_Call) RunAndReturn(run func() *schemapb.Collectio
|
|||||||
return _c
|
return _c
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// UpdateSchema provides a mock function with given fields: updatedSchema
|
||||||
|
func (_m *MockMetaCache) UpdateSchema(updatedSchema *schemapb.CollectionSchema) {
|
||||||
|
_m.Called(updatedSchema)
|
||||||
|
}
|
||||||
|
|
||||||
|
// MockMetaCache_UpdateSchema_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'UpdateSchema'
|
||||||
|
type MockMetaCache_UpdateSchema_Call struct {
|
||||||
|
*mock.Call
|
||||||
|
}
|
||||||
|
|
||||||
|
// UpdateSchema is a helper method to define mock.On call
|
||||||
|
// - updatedSchema *schemapb.CollectionSchema
|
||||||
|
func (_e *MockMetaCache_Expecter) UpdateSchema(updatedSchema interface{}) *MockMetaCache_UpdateSchema_Call {
|
||||||
|
return &MockMetaCache_UpdateSchema_Call{Call: _e.mock.On("UpdateSchema", updatedSchema)}
|
||||||
|
}
|
||||||
|
|
||||||
|
func (_c *MockMetaCache_UpdateSchema_Call) Run(run func(updatedSchema *schemapb.CollectionSchema)) *MockMetaCache_UpdateSchema_Call {
|
||||||
|
_c.Call.Run(func(args mock.Arguments) {
|
||||||
|
run(args[0].(*schemapb.CollectionSchema))
|
||||||
|
})
|
||||||
|
return _c
|
||||||
|
}
|
||||||
|
|
||||||
|
func (_c *MockMetaCache_UpdateSchema_Call) Return() *MockMetaCache_UpdateSchema_Call {
|
||||||
|
_c.Call.Return()
|
||||||
|
return _c
|
||||||
|
}
|
||||||
|
|
||||||
|
func (_c *MockMetaCache_UpdateSchema_Call) RunAndReturn(run func(*schemapb.CollectionSchema)) *MockMetaCache_UpdateSchema_Call {
|
||||||
|
_c.Call.Return(run)
|
||||||
|
return _c
|
||||||
|
}
|
||||||
|
|
||||||
// UpdateSegmentView provides a mock function with given fields: partitionID, newSegments, newSegmentsBF, allSegments
|
// UpdateSegmentView provides a mock function with given fields: partitionID, newSegments, newSegmentsBF, allSegments
|
||||||
func (_m *MockMetaCache) UpdateSegmentView(partitionID int64, newSegments []*datapb.SyncSegmentInfo, newSegmentsBF []*pkoracle.BloomFilterSet, allSegments map[int64]struct{}) {
|
func (_m *MockMetaCache) UpdateSegmentView(partitionID int64, newSegments []*datapb.SyncSegmentInfo, newSegmentsBF []*pkoracle.BloomFilterSet, allSegments map[int64]struct{}) {
|
||||||
_m.Called(partitionID, newSegments, newSegmentsBF, allSegments)
|
_m.Called(partitionID, newSegments, newSegmentsBF, allSegments)
|
||||||
|
|||||||
@ -296,6 +296,21 @@ func (ddn *ddNode) Operate(in []Msg) []Msg {
|
|||||||
} else {
|
} else {
|
||||||
logger.Info("handle import message success")
|
logger.Info("handle import message success")
|
||||||
}
|
}
|
||||||
|
case commonpb.MsgType_AddCollectionField:
|
||||||
|
schemaMsg := msg.(*adaptor.SchemaChangeMessageBody)
|
||||||
|
header := schemaMsg.SchemaChangeMessage.Header()
|
||||||
|
if header.GetCollectionId() != ddn.collectionID {
|
||||||
|
continue
|
||||||
|
}
|
||||||
|
logger := log.With(zap.String("vchannel", ddn.Name()))
|
||||||
|
logger.Info("receive schema change message")
|
||||||
|
body, err := schemaMsg.SchemaChangeMessage.Body()
|
||||||
|
if err != nil {
|
||||||
|
logger.Warn("failed to unmarshal schema change message body", zap.Error(err))
|
||||||
|
continue
|
||||||
|
}
|
||||||
|
fgMsg.updatedSchema = body.GetSchema()
|
||||||
|
ddn.msgHandler.HandleSchemaChange(ddn.ctx, ddn.vChannelName, schemaMsg)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|||||||
@ -18,6 +18,7 @@ package pipeline
|
|||||||
|
|
||||||
import (
|
import (
|
||||||
"github.com/milvus-io/milvus-proto/go-api/v2/msgpb"
|
"github.com/milvus-io/milvus-proto/go-api/v2/msgpb"
|
||||||
|
"github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
|
||||||
"github.com/milvus-io/milvus/internal/flushcommon/util"
|
"github.com/milvus-io/milvus/internal/flushcommon/util"
|
||||||
"github.com/milvus-io/milvus/internal/flushcommon/writebuffer"
|
"github.com/milvus-io/milvus/internal/flushcommon/writebuffer"
|
||||||
"github.com/milvus-io/milvus/internal/storage"
|
"github.com/milvus-io/milvus/internal/storage"
|
||||||
@ -60,6 +61,8 @@ type FlowGraphMsg struct {
|
|||||||
segmentsToSync []typeutil.UniqueID
|
segmentsToSync []typeutil.UniqueID
|
||||||
dropCollection bool
|
dropCollection bool
|
||||||
dropPartitions []typeutil.UniqueID
|
dropPartitions []typeutil.UniqueID
|
||||||
|
|
||||||
|
updatedSchema *schemapb.CollectionSchema
|
||||||
}
|
}
|
||||||
|
|
||||||
func (fgMsg *FlowGraphMsg) TimeTick() typeutil.Timestamp {
|
func (fgMsg *FlowGraphMsg) TimeTick() typeutil.Timestamp {
|
||||||
|
|||||||
@ -115,6 +115,11 @@ func (wNode *writeNode) Operate(in []Msg) []Msg {
|
|||||||
wNode.updater.Update(wNode.channelName, end.GetTimestamp(), stats)
|
wNode.updater.Update(wNode.channelName, end.GetTimestamp(), stats)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// update schema after all data processed
|
||||||
|
if fgMsg.updatedSchema != nil {
|
||||||
|
wNode.metacache.UpdateSchema(fgMsg.updatedSchema)
|
||||||
|
}
|
||||||
|
|
||||||
res := FlowGraphMsg{
|
res := FlowGraphMsg{
|
||||||
TimeRange: fgMsg.TimeRange,
|
TimeRange: fgMsg.TimeRange,
|
||||||
StartPositions: fgMsg.StartPositions,
|
StartPositions: fgMsg.StartPositions,
|
||||||
|
|||||||
@ -22,6 +22,7 @@ import (
|
|||||||
"github.com/milvus-io/milvus-proto/go-api/v2/msgpb"
|
"github.com/milvus-io/milvus-proto/go-api/v2/msgpb"
|
||||||
"github.com/milvus-io/milvus/pkg/v2/proto/internalpb"
|
"github.com/milvus-io/milvus/pkg/v2/proto/internalpb"
|
||||||
"github.com/milvus-io/milvus/pkg/v2/streaming/util/message"
|
"github.com/milvus-io/milvus/pkg/v2/streaming/util/message"
|
||||||
|
"github.com/milvus-io/milvus/pkg/v2/streaming/util/message/adaptor"
|
||||||
)
|
)
|
||||||
|
|
||||||
type MsgHandler interface {
|
type MsgHandler interface {
|
||||||
@ -32,6 +33,8 @@ type MsgHandler interface {
|
|||||||
HandleManualFlush(vchannel string, flushMsg message.ImmutableManualFlushMessageV2) error
|
HandleManualFlush(vchannel string, flushMsg message.ImmutableManualFlushMessageV2) error
|
||||||
|
|
||||||
HandleImport(ctx context.Context, vchannel string, importMsg *msgpb.ImportMsg) error
|
HandleImport(ctx context.Context, vchannel string, importMsg *msgpb.ImportMsg) error
|
||||||
|
|
||||||
|
HandleSchemaChange(ctx context.Context, vchannel string, msg *adaptor.SchemaChangeMessageBody) error
|
||||||
}
|
}
|
||||||
|
|
||||||
func ConvertInternalImportFile(file *msgpb.ImportFile, _ int) *internalpb.ImportFile {
|
func ConvertInternalImportFile(file *msgpb.ImportFile, _ int) *internalpb.ImportFile {
|
||||||
|
|||||||
@ -5,6 +5,8 @@ package mock_util
|
|||||||
import (
|
import (
|
||||||
context "context"
|
context "context"
|
||||||
|
|
||||||
|
adaptor "github.com/milvus-io/milvus/pkg/v2/streaming/util/message/adaptor"
|
||||||
|
|
||||||
message "github.com/milvus-io/milvus/pkg/v2/streaming/util/message"
|
message "github.com/milvus-io/milvus/pkg/v2/streaming/util/message"
|
||||||
|
|
||||||
mock "github.com/stretchr/testify/mock"
|
mock "github.com/stretchr/testify/mock"
|
||||||
@ -215,6 +217,54 @@ func (_c *MockMsgHandler_HandleManualFlush_Call) RunAndReturn(run func(string, m
|
|||||||
return _c
|
return _c
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// HandleSchemaChange provides a mock function with given fields: ctx, vchannel, msg
|
||||||
|
func (_m *MockMsgHandler) HandleSchemaChange(ctx context.Context, vchannel string, msg *adaptor.SchemaChangeMessageBody) error {
|
||||||
|
ret := _m.Called(ctx, vchannel, msg)
|
||||||
|
|
||||||
|
if len(ret) == 0 {
|
||||||
|
panic("no return value specified for HandleSchemaChange")
|
||||||
|
}
|
||||||
|
|
||||||
|
var r0 error
|
||||||
|
if rf, ok := ret.Get(0).(func(context.Context, string, *adaptor.SchemaChangeMessageBody) error); ok {
|
||||||
|
r0 = rf(ctx, vchannel, msg)
|
||||||
|
} else {
|
||||||
|
r0 = ret.Error(0)
|
||||||
|
}
|
||||||
|
|
||||||
|
return r0
|
||||||
|
}
|
||||||
|
|
||||||
|
// MockMsgHandler_HandleSchemaChange_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'HandleSchemaChange'
|
||||||
|
type MockMsgHandler_HandleSchemaChange_Call struct {
|
||||||
|
*mock.Call
|
||||||
|
}
|
||||||
|
|
||||||
|
// HandleSchemaChange is a helper method to define mock.On call
|
||||||
|
// - ctx context.Context
|
||||||
|
// - vchannel string
|
||||||
|
// - msg *adaptor.SchemaChangeMessageBody
|
||||||
|
func (_e *MockMsgHandler_Expecter) HandleSchemaChange(ctx interface{}, vchannel interface{}, msg interface{}) *MockMsgHandler_HandleSchemaChange_Call {
|
||||||
|
return &MockMsgHandler_HandleSchemaChange_Call{Call: _e.mock.On("HandleSchemaChange", ctx, vchannel, msg)}
|
||||||
|
}
|
||||||
|
|
||||||
|
func (_c *MockMsgHandler_HandleSchemaChange_Call) Run(run func(ctx context.Context, vchannel string, msg *adaptor.SchemaChangeMessageBody)) *MockMsgHandler_HandleSchemaChange_Call {
|
||||||
|
_c.Call.Run(func(args mock.Arguments) {
|
||||||
|
run(args[0].(context.Context), args[1].(string), args[2].(*adaptor.SchemaChangeMessageBody))
|
||||||
|
})
|
||||||
|
return _c
|
||||||
|
}
|
||||||
|
|
||||||
|
func (_c *MockMsgHandler_HandleSchemaChange_Call) Return(_a0 error) *MockMsgHandler_HandleSchemaChange_Call {
|
||||||
|
_c.Call.Return(_a0)
|
||||||
|
return _c
|
||||||
|
}
|
||||||
|
|
||||||
|
func (_c *MockMsgHandler_HandleSchemaChange_Call) RunAndReturn(run func(context.Context, string, *adaptor.SchemaChangeMessageBody) error) *MockMsgHandler_HandleSchemaChange_Call {
|
||||||
|
_c.Call.Return(run)
|
||||||
|
return _c
|
||||||
|
}
|
||||||
|
|
||||||
// NewMockMsgHandler creates a new instance of MockMsgHandler. It also registers a testing interface on the mock and a cleanup function to assert the mocks expectations.
|
// NewMockMsgHandler creates a new instance of MockMsgHandler. It also registers a testing interface on the mock and a cleanup function to assert the mocks expectations.
|
||||||
// The first argument is typically a *testing.T value.
|
// The first argument is typically a *testing.T value.
|
||||||
func NewMockMsgHandler(t interface {
|
func NewMockMsgHandler(t interface {
|
||||||
|
|||||||
@ -1758,6 +1758,65 @@ func (_c *MockQueryNode_UnsubDmChannel_Call) RunAndReturn(run func(context.Conte
|
|||||||
return _c
|
return _c
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// UpdateSchema provides a mock function with given fields: _a0, _a1
|
||||||
|
func (_m *MockQueryNode) UpdateSchema(_a0 context.Context, _a1 *querypb.UpdateSchemaRequest) (*commonpb.Status, error) {
|
||||||
|
ret := _m.Called(_a0, _a1)
|
||||||
|
|
||||||
|
if len(ret) == 0 {
|
||||||
|
panic("no return value specified for UpdateSchema")
|
||||||
|
}
|
||||||
|
|
||||||
|
var r0 *commonpb.Status
|
||||||
|
var r1 error
|
||||||
|
if rf, ok := ret.Get(0).(func(context.Context, *querypb.UpdateSchemaRequest) (*commonpb.Status, error)); ok {
|
||||||
|
return rf(_a0, _a1)
|
||||||
|
}
|
||||||
|
if rf, ok := ret.Get(0).(func(context.Context, *querypb.UpdateSchemaRequest) *commonpb.Status); ok {
|
||||||
|
r0 = rf(_a0, _a1)
|
||||||
|
} else {
|
||||||
|
if ret.Get(0) != nil {
|
||||||
|
r0 = ret.Get(0).(*commonpb.Status)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
if rf, ok := ret.Get(1).(func(context.Context, *querypb.UpdateSchemaRequest) error); ok {
|
||||||
|
r1 = rf(_a0, _a1)
|
||||||
|
} else {
|
||||||
|
r1 = ret.Error(1)
|
||||||
|
}
|
||||||
|
|
||||||
|
return r0, r1
|
||||||
|
}
|
||||||
|
|
||||||
|
// MockQueryNode_UpdateSchema_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'UpdateSchema'
|
||||||
|
type MockQueryNode_UpdateSchema_Call struct {
|
||||||
|
*mock.Call
|
||||||
|
}
|
||||||
|
|
||||||
|
// UpdateSchema is a helper method to define mock.On call
|
||||||
|
// - _a0 context.Context
|
||||||
|
// - _a1 *querypb.UpdateSchemaRequest
|
||||||
|
func (_e *MockQueryNode_Expecter) UpdateSchema(_a0 interface{}, _a1 interface{}) *MockQueryNode_UpdateSchema_Call {
|
||||||
|
return &MockQueryNode_UpdateSchema_Call{Call: _e.mock.On("UpdateSchema", _a0, _a1)}
|
||||||
|
}
|
||||||
|
|
||||||
|
func (_c *MockQueryNode_UpdateSchema_Call) Run(run func(_a0 context.Context, _a1 *querypb.UpdateSchemaRequest)) *MockQueryNode_UpdateSchema_Call {
|
||||||
|
_c.Call.Run(func(args mock.Arguments) {
|
||||||
|
run(args[0].(context.Context), args[1].(*querypb.UpdateSchemaRequest))
|
||||||
|
})
|
||||||
|
return _c
|
||||||
|
}
|
||||||
|
|
||||||
|
func (_c *MockQueryNode_UpdateSchema_Call) Return(_a0 *commonpb.Status, _a1 error) *MockQueryNode_UpdateSchema_Call {
|
||||||
|
_c.Call.Return(_a0, _a1)
|
||||||
|
return _c
|
||||||
|
}
|
||||||
|
|
||||||
|
func (_c *MockQueryNode_UpdateSchema_Call) RunAndReturn(run func(context.Context, *querypb.UpdateSchemaRequest) (*commonpb.Status, error)) *MockQueryNode_UpdateSchema_Call {
|
||||||
|
_c.Call.Return(run)
|
||||||
|
return _c
|
||||||
|
}
|
||||||
|
|
||||||
// UpdateStateCode provides a mock function with given fields: stateCode
|
// UpdateStateCode provides a mock function with given fields: stateCode
|
||||||
func (_m *MockQueryNode) UpdateStateCode(stateCode commonpb.StateCode) {
|
func (_m *MockQueryNode) UpdateStateCode(stateCode commonpb.StateCode) {
|
||||||
_m.Called(stateCode)
|
_m.Called(stateCode)
|
||||||
|
|||||||
@ -1852,6 +1852,80 @@ func (_c *MockQueryNodeClient_UnsubDmChannel_Call) RunAndReturn(run func(context
|
|||||||
return _c
|
return _c
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// UpdateSchema provides a mock function with given fields: ctx, in, opts
|
||||||
|
func (_m *MockQueryNodeClient) UpdateSchema(ctx context.Context, in *querypb.UpdateSchemaRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
|
||||||
|
_va := make([]interface{}, len(opts))
|
||||||
|
for _i := range opts {
|
||||||
|
_va[_i] = opts[_i]
|
||||||
|
}
|
||||||
|
var _ca []interface{}
|
||||||
|
_ca = append(_ca, ctx, in)
|
||||||
|
_ca = append(_ca, _va...)
|
||||||
|
ret := _m.Called(_ca...)
|
||||||
|
|
||||||
|
if len(ret) == 0 {
|
||||||
|
panic("no return value specified for UpdateSchema")
|
||||||
|
}
|
||||||
|
|
||||||
|
var r0 *commonpb.Status
|
||||||
|
var r1 error
|
||||||
|
if rf, ok := ret.Get(0).(func(context.Context, *querypb.UpdateSchemaRequest, ...grpc.CallOption) (*commonpb.Status, error)); ok {
|
||||||
|
return rf(ctx, in, opts...)
|
||||||
|
}
|
||||||
|
if rf, ok := ret.Get(0).(func(context.Context, *querypb.UpdateSchemaRequest, ...grpc.CallOption) *commonpb.Status); ok {
|
||||||
|
r0 = rf(ctx, in, opts...)
|
||||||
|
} else {
|
||||||
|
if ret.Get(0) != nil {
|
||||||
|
r0 = ret.Get(0).(*commonpb.Status)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
if rf, ok := ret.Get(1).(func(context.Context, *querypb.UpdateSchemaRequest, ...grpc.CallOption) error); ok {
|
||||||
|
r1 = rf(ctx, in, opts...)
|
||||||
|
} else {
|
||||||
|
r1 = ret.Error(1)
|
||||||
|
}
|
||||||
|
|
||||||
|
return r0, r1
|
||||||
|
}
|
||||||
|
|
||||||
|
// MockQueryNodeClient_UpdateSchema_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'UpdateSchema'
|
||||||
|
type MockQueryNodeClient_UpdateSchema_Call struct {
|
||||||
|
*mock.Call
|
||||||
|
}
|
||||||
|
|
||||||
|
// UpdateSchema is a helper method to define mock.On call
|
||||||
|
// - ctx context.Context
|
||||||
|
// - in *querypb.UpdateSchemaRequest
|
||||||
|
// - opts ...grpc.CallOption
|
||||||
|
func (_e *MockQueryNodeClient_Expecter) UpdateSchema(ctx interface{}, in interface{}, opts ...interface{}) *MockQueryNodeClient_UpdateSchema_Call {
|
||||||
|
return &MockQueryNodeClient_UpdateSchema_Call{Call: _e.mock.On("UpdateSchema",
|
||||||
|
append([]interface{}{ctx, in}, opts...)...)}
|
||||||
|
}
|
||||||
|
|
||||||
|
func (_c *MockQueryNodeClient_UpdateSchema_Call) Run(run func(ctx context.Context, in *querypb.UpdateSchemaRequest, opts ...grpc.CallOption)) *MockQueryNodeClient_UpdateSchema_Call {
|
||||||
|
_c.Call.Run(func(args mock.Arguments) {
|
||||||
|
variadicArgs := make([]grpc.CallOption, len(args)-2)
|
||||||
|
for i, a := range args[2:] {
|
||||||
|
if a != nil {
|
||||||
|
variadicArgs[i] = a.(grpc.CallOption)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
run(args[0].(context.Context), args[1].(*querypb.UpdateSchemaRequest), variadicArgs...)
|
||||||
|
})
|
||||||
|
return _c
|
||||||
|
}
|
||||||
|
|
||||||
|
func (_c *MockQueryNodeClient_UpdateSchema_Call) Return(_a0 *commonpb.Status, _a1 error) *MockQueryNodeClient_UpdateSchema_Call {
|
||||||
|
_c.Call.Return(_a0, _a1)
|
||||||
|
return _c
|
||||||
|
}
|
||||||
|
|
||||||
|
func (_c *MockQueryNodeClient_UpdateSchema_Call) RunAndReturn(run func(context.Context, *querypb.UpdateSchemaRequest, ...grpc.CallOption) (*commonpb.Status, error)) *MockQueryNodeClient_UpdateSchema_Call {
|
||||||
|
_c.Call.Return(run)
|
||||||
|
return _c
|
||||||
|
}
|
||||||
|
|
||||||
// WatchDmChannels provides a mock function with given fields: ctx, in, opts
|
// WatchDmChannels provides a mock function with given fields: ctx, in, opts
|
||||||
func (_m *MockQueryNodeClient) WatchDmChannels(ctx context.Context, in *querypb.WatchDmChannelsRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
|
func (_m *MockQueryNodeClient) WatchDmChannels(ctx context.Context, in *querypb.WatchDmChannelsRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
|
||||||
_va := make([]interface{}, len(opts))
|
_va := make([]interface{}, len(opts))
|
||||||
|
|||||||
@ -1421,6 +1421,65 @@ func (_c *MockQueryNodeServer_UnsubDmChannel_Call) RunAndReturn(run func(context
|
|||||||
return _c
|
return _c
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// UpdateSchema provides a mock function with given fields: _a0, _a1
|
||||||
|
func (_m *MockQueryNodeServer) UpdateSchema(_a0 context.Context, _a1 *querypb.UpdateSchemaRequest) (*commonpb.Status, error) {
|
||||||
|
ret := _m.Called(_a0, _a1)
|
||||||
|
|
||||||
|
if len(ret) == 0 {
|
||||||
|
panic("no return value specified for UpdateSchema")
|
||||||
|
}
|
||||||
|
|
||||||
|
var r0 *commonpb.Status
|
||||||
|
var r1 error
|
||||||
|
if rf, ok := ret.Get(0).(func(context.Context, *querypb.UpdateSchemaRequest) (*commonpb.Status, error)); ok {
|
||||||
|
return rf(_a0, _a1)
|
||||||
|
}
|
||||||
|
if rf, ok := ret.Get(0).(func(context.Context, *querypb.UpdateSchemaRequest) *commonpb.Status); ok {
|
||||||
|
r0 = rf(_a0, _a1)
|
||||||
|
} else {
|
||||||
|
if ret.Get(0) != nil {
|
||||||
|
r0 = ret.Get(0).(*commonpb.Status)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
if rf, ok := ret.Get(1).(func(context.Context, *querypb.UpdateSchemaRequest) error); ok {
|
||||||
|
r1 = rf(_a0, _a1)
|
||||||
|
} else {
|
||||||
|
r1 = ret.Error(1)
|
||||||
|
}
|
||||||
|
|
||||||
|
return r0, r1
|
||||||
|
}
|
||||||
|
|
||||||
|
// MockQueryNodeServer_UpdateSchema_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'UpdateSchema'
|
||||||
|
type MockQueryNodeServer_UpdateSchema_Call struct {
|
||||||
|
*mock.Call
|
||||||
|
}
|
||||||
|
|
||||||
|
// UpdateSchema is a helper method to define mock.On call
|
||||||
|
// - _a0 context.Context
|
||||||
|
// - _a1 *querypb.UpdateSchemaRequest
|
||||||
|
func (_e *MockQueryNodeServer_Expecter) UpdateSchema(_a0 interface{}, _a1 interface{}) *MockQueryNodeServer_UpdateSchema_Call {
|
||||||
|
return &MockQueryNodeServer_UpdateSchema_Call{Call: _e.mock.On("UpdateSchema", _a0, _a1)}
|
||||||
|
}
|
||||||
|
|
||||||
|
func (_c *MockQueryNodeServer_UpdateSchema_Call) Run(run func(_a0 context.Context, _a1 *querypb.UpdateSchemaRequest)) *MockQueryNodeServer_UpdateSchema_Call {
|
||||||
|
_c.Call.Run(func(args mock.Arguments) {
|
||||||
|
run(args[0].(context.Context), args[1].(*querypb.UpdateSchemaRequest))
|
||||||
|
})
|
||||||
|
return _c
|
||||||
|
}
|
||||||
|
|
||||||
|
func (_c *MockQueryNodeServer_UpdateSchema_Call) Return(_a0 *commonpb.Status, _a1 error) *MockQueryNodeServer_UpdateSchema_Call {
|
||||||
|
_c.Call.Return(_a0, _a1)
|
||||||
|
return _c
|
||||||
|
}
|
||||||
|
|
||||||
|
func (_c *MockQueryNodeServer_UpdateSchema_Call) RunAndReturn(run func(context.Context, *querypb.UpdateSchemaRequest) (*commonpb.Status, error)) *MockQueryNodeServer_UpdateSchema_Call {
|
||||||
|
_c.Call.Return(run)
|
||||||
|
return _c
|
||||||
|
}
|
||||||
|
|
||||||
// WatchDmChannels provides a mock function with given fields: _a0, _a1
|
// WatchDmChannels provides a mock function with given fields: _a0, _a1
|
||||||
func (_m *MockQueryNodeServer) WatchDmChannels(_a0 context.Context, _a1 *querypb.WatchDmChannelsRequest) (*commonpb.Status, error) {
|
func (_m *MockQueryNodeServer) WatchDmChannels(_a0 context.Context, _a1 *querypb.WatchDmChannelsRequest) (*commonpb.Status, error) {
|
||||||
ret := _m.Called(_a0, _a1)
|
ret := _m.Called(_a0, _a1)
|
||||||
|
|||||||
@ -5,7 +5,10 @@ package cluster
|
|||||||
import (
|
import (
|
||||||
context "context"
|
context "context"
|
||||||
|
|
||||||
|
commonpb "github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
|
||||||
|
|
||||||
internalpb "github.com/milvus-io/milvus/pkg/v2/proto/internalpb"
|
internalpb "github.com/milvus-io/milvus/pkg/v2/proto/internalpb"
|
||||||
|
|
||||||
mock "github.com/stretchr/testify/mock"
|
mock "github.com/stretchr/testify/mock"
|
||||||
|
|
||||||
querypb "github.com/milvus-io/milvus/pkg/v2/proto/querypb"
|
querypb "github.com/milvus-io/milvus/pkg/v2/proto/querypb"
|
||||||
@ -528,6 +531,65 @@ func (_c *MockWorker_Stop_Call) RunAndReturn(run func()) *MockWorker_Stop_Call {
|
|||||||
return _c
|
return _c
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// UpdateSchema provides a mock function with given fields: ctx, req
|
||||||
|
func (_m *MockWorker) UpdateSchema(ctx context.Context, req *querypb.UpdateSchemaRequest) (*commonpb.Status, error) {
|
||||||
|
ret := _m.Called(ctx, req)
|
||||||
|
|
||||||
|
if len(ret) == 0 {
|
||||||
|
panic("no return value specified for UpdateSchema")
|
||||||
|
}
|
||||||
|
|
||||||
|
var r0 *commonpb.Status
|
||||||
|
var r1 error
|
||||||
|
if rf, ok := ret.Get(0).(func(context.Context, *querypb.UpdateSchemaRequest) (*commonpb.Status, error)); ok {
|
||||||
|
return rf(ctx, req)
|
||||||
|
}
|
||||||
|
if rf, ok := ret.Get(0).(func(context.Context, *querypb.UpdateSchemaRequest) *commonpb.Status); ok {
|
||||||
|
r0 = rf(ctx, req)
|
||||||
|
} else {
|
||||||
|
if ret.Get(0) != nil {
|
||||||
|
r0 = ret.Get(0).(*commonpb.Status)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
if rf, ok := ret.Get(1).(func(context.Context, *querypb.UpdateSchemaRequest) error); ok {
|
||||||
|
r1 = rf(ctx, req)
|
||||||
|
} else {
|
||||||
|
r1 = ret.Error(1)
|
||||||
|
}
|
||||||
|
|
||||||
|
return r0, r1
|
||||||
|
}
|
||||||
|
|
||||||
|
// MockWorker_UpdateSchema_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'UpdateSchema'
|
||||||
|
type MockWorker_UpdateSchema_Call struct {
|
||||||
|
*mock.Call
|
||||||
|
}
|
||||||
|
|
||||||
|
// UpdateSchema is a helper method to define mock.On call
|
||||||
|
// - ctx context.Context
|
||||||
|
// - req *querypb.UpdateSchemaRequest
|
||||||
|
func (_e *MockWorker_Expecter) UpdateSchema(ctx interface{}, req interface{}) *MockWorker_UpdateSchema_Call {
|
||||||
|
return &MockWorker_UpdateSchema_Call{Call: _e.mock.On("UpdateSchema", ctx, req)}
|
||||||
|
}
|
||||||
|
|
||||||
|
func (_c *MockWorker_UpdateSchema_Call) Run(run func(ctx context.Context, req *querypb.UpdateSchemaRequest)) *MockWorker_UpdateSchema_Call {
|
||||||
|
_c.Call.Run(func(args mock.Arguments) {
|
||||||
|
run(args[0].(context.Context), args[1].(*querypb.UpdateSchemaRequest))
|
||||||
|
})
|
||||||
|
return _c
|
||||||
|
}
|
||||||
|
|
||||||
|
func (_c *MockWorker_UpdateSchema_Call) Return(_a0 *commonpb.Status, _a1 error) *MockWorker_UpdateSchema_Call {
|
||||||
|
_c.Call.Return(_a0, _a1)
|
||||||
|
return _c
|
||||||
|
}
|
||||||
|
|
||||||
|
func (_c *MockWorker_UpdateSchema_Call) RunAndReturn(run func(context.Context, *querypb.UpdateSchemaRequest) (*commonpb.Status, error)) *MockWorker_UpdateSchema_Call {
|
||||||
|
_c.Call.Return(run)
|
||||||
|
return _c
|
||||||
|
}
|
||||||
|
|
||||||
// NewMockWorker creates a new instance of MockWorker. It also registers a testing interface on the mock and a cleanup function to assert the mocks expectations.
|
// NewMockWorker creates a new instance of MockWorker. It also registers a testing interface on the mock and a cleanup function to assert the mocks expectations.
|
||||||
// The first argument is typically a *testing.T value.
|
// The first argument is typically a *testing.T value.
|
||||||
func NewMockWorker(t interface {
|
func NewMockWorker(t interface {
|
||||||
|
|||||||
@ -25,6 +25,7 @@ import (
|
|||||||
"go.uber.org/atomic"
|
"go.uber.org/atomic"
|
||||||
"go.uber.org/zap"
|
"go.uber.org/zap"
|
||||||
|
|
||||||
|
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
|
||||||
"github.com/milvus-io/milvus/internal/types"
|
"github.com/milvus-io/milvus/internal/types"
|
||||||
"github.com/milvus-io/milvus/internal/util/streamrpc"
|
"github.com/milvus-io/milvus/internal/util/streamrpc"
|
||||||
"github.com/milvus-io/milvus/pkg/v2/log"
|
"github.com/milvus-io/milvus/pkg/v2/log"
|
||||||
@ -44,6 +45,7 @@ type Worker interface {
|
|||||||
QuerySegments(ctx context.Context, req *querypb.QueryRequest) (*internalpb.RetrieveResults, error)
|
QuerySegments(ctx context.Context, req *querypb.QueryRequest) (*internalpb.RetrieveResults, error)
|
||||||
QueryStreamSegments(ctx context.Context, req *querypb.QueryRequest, srv streamrpc.QueryStreamServer) error
|
QueryStreamSegments(ctx context.Context, req *querypb.QueryRequest, srv streamrpc.QueryStreamServer) error
|
||||||
GetStatistics(ctx context.Context, req *querypb.GetStatisticsRequest) (*internalpb.GetStatisticsResponse, error)
|
GetStatistics(ctx context.Context, req *querypb.GetStatisticsRequest) (*internalpb.GetStatisticsResponse, error)
|
||||||
|
UpdateSchema(ctx context.Context, req *querypb.UpdateSchemaRequest) (*commonpb.Status, error)
|
||||||
|
|
||||||
IsHealthy() bool
|
IsHealthy() bool
|
||||||
Stop()
|
Stop()
|
||||||
@ -247,6 +249,11 @@ func (w *remoteWorker) GetStatistics(ctx context.Context, req *querypb.GetStatis
|
|||||||
return client.GetStatistics(ctx, req)
|
return client.GetStatistics(ctx, req)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func (w *remoteWorker) UpdateSchema(ctx context.Context, req *querypb.UpdateSchemaRequest) (*commonpb.Status, error) {
|
||||||
|
client := w.getClient()
|
||||||
|
return client.UpdateSchema(ctx, req)
|
||||||
|
}
|
||||||
|
|
||||||
func (w *remoteWorker) IsHealthy() bool {
|
func (w *remoteWorker) IsHealthy() bool {
|
||||||
return true
|
return true
|
||||||
}
|
}
|
||||||
|
|||||||
@ -52,6 +52,7 @@ import (
|
|||||||
"github.com/milvus-io/milvus/pkg/v2/mq/msgstream"
|
"github.com/milvus-io/milvus/pkg/v2/mq/msgstream"
|
||||||
"github.com/milvus-io/milvus/pkg/v2/proto/internalpb"
|
"github.com/milvus-io/milvus/pkg/v2/proto/internalpb"
|
||||||
"github.com/milvus-io/milvus/pkg/v2/proto/querypb"
|
"github.com/milvus-io/milvus/pkg/v2/proto/querypb"
|
||||||
|
"github.com/milvus-io/milvus/pkg/v2/util/commonpbutil"
|
||||||
"github.com/milvus-io/milvus/pkg/v2/util/conc"
|
"github.com/milvus-io/milvus/pkg/v2/util/conc"
|
||||||
"github.com/milvus-io/milvus/pkg/v2/util/funcutil"
|
"github.com/milvus-io/milvus/pkg/v2/util/funcutil"
|
||||||
"github.com/milvus-io/milvus/pkg/v2/util/lifetime"
|
"github.com/milvus-io/milvus/pkg/v2/util/lifetime"
|
||||||
@ -76,6 +77,7 @@ type ShardDelegator interface {
|
|||||||
Query(ctx context.Context, req *querypb.QueryRequest) ([]*internalpb.RetrieveResults, error)
|
Query(ctx context.Context, req *querypb.QueryRequest) ([]*internalpb.RetrieveResults, error)
|
||||||
QueryStream(ctx context.Context, req *querypb.QueryRequest, srv streamrpc.QueryStreamServer) error
|
QueryStream(ctx context.Context, req *querypb.QueryRequest, srv streamrpc.QueryStreamServer) error
|
||||||
GetStatistics(ctx context.Context, req *querypb.GetStatisticsRequest) ([]*internalpb.GetStatisticsResponse, error)
|
GetStatistics(ctx context.Context, req *querypb.GetStatisticsRequest) ([]*internalpb.GetStatisticsResponse, error)
|
||||||
|
UpdateSchema(ctx context.Context, sch *schemapb.CollectionSchema) error
|
||||||
|
|
||||||
// data
|
// data
|
||||||
ProcessInsert(insertRecords map[int64]*InsertData)
|
ProcessInsert(insertRecords map[int64]*InsertData)
|
||||||
@ -856,6 +858,51 @@ func (sd *shardDelegator) GetTSafe() uint64 {
|
|||||||
return sd.latestTsafe.Load()
|
return sd.latestTsafe.Load()
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func (sd *shardDelegator) UpdateSchema(ctx context.Context, schema *schemapb.CollectionSchema) error {
|
||||||
|
log := sd.getLogger(ctx)
|
||||||
|
if err := sd.lifetime.Add(lifetime.IsWorking); err != nil {
|
||||||
|
return err
|
||||||
|
}
|
||||||
|
defer sd.lifetime.Done()
|
||||||
|
|
||||||
|
log.Info("delegator received update schema event")
|
||||||
|
|
||||||
|
sealed, growing, version, err := sd.distribution.PinReadableSegments()
|
||||||
|
if err != nil {
|
||||||
|
log.Warn("delegator failed to query, current distribution is not serviceable", zap.Error(err))
|
||||||
|
return err
|
||||||
|
}
|
||||||
|
defer sd.distribution.Unpin(version)
|
||||||
|
|
||||||
|
tasks, err := organizeSubTask(ctx, &querypb.UpdateSchemaRequest{
|
||||||
|
Base: commonpbutil.NewMsgBase(
|
||||||
|
commonpbutil.WithSourceID(paramtable.GetNodeID()),
|
||||||
|
),
|
||||||
|
CollectionID: sd.collectionID,
|
||||||
|
Schema: schema,
|
||||||
|
}, sealed, growing, sd, func(req *querypb.UpdateSchemaRequest, scope querypb.DataScope, segmentIDs []int64, targetID int64) *querypb.UpdateSchemaRequest {
|
||||||
|
nodeReq := typeutil.Clone(req)
|
||||||
|
nodeReq.GetBase().TargetID = targetID
|
||||||
|
return nodeReq
|
||||||
|
})
|
||||||
|
if err != nil {
|
||||||
|
return err
|
||||||
|
}
|
||||||
|
|
||||||
|
_, err = executeSubTasks(ctx, tasks, func(ctx context.Context, req *querypb.UpdateSchemaRequest, worker cluster.Worker) (*StatusWrapper, error) {
|
||||||
|
status, err := worker.UpdateSchema(ctx, req)
|
||||||
|
return (*StatusWrapper)(status), err
|
||||||
|
}, "UpdateSchema", log)
|
||||||
|
|
||||||
|
return err
|
||||||
|
}
|
||||||
|
|
||||||
|
type StatusWrapper commonpb.Status
|
||||||
|
|
||||||
|
func (w *StatusWrapper) GetStatus() *commonpb.Status {
|
||||||
|
return (*commonpb.Status)(w)
|
||||||
|
}
|
||||||
|
|
||||||
// Close closes the delegator.
|
// Close closes the delegator.
|
||||||
func (sd *shardDelegator) Close() {
|
func (sd *shardDelegator) Close() {
|
||||||
sd.lifetime.SetState(lifetime.Stopped)
|
sd.lifetime.SetState(lifetime.Stopped)
|
||||||
|
|||||||
@ -1189,6 +1189,99 @@ func (s *DelegatorSuite) TestGetStats() {
|
|||||||
})
|
})
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func (s *DelegatorSuite) TestUpdateSchema() {
|
||||||
|
s.delegator.Start()
|
||||||
|
paramtable.SetNodeID(1)
|
||||||
|
s.initSegments()
|
||||||
|
|
||||||
|
s.Run("normal", func() {
|
||||||
|
workers := make(map[int64]*cluster.MockWorker)
|
||||||
|
worker1 := cluster.NewMockWorker(s.T())
|
||||||
|
worker2 := cluster.NewMockWorker(s.T())
|
||||||
|
|
||||||
|
workers[1] = worker1
|
||||||
|
workers[2] = worker2
|
||||||
|
|
||||||
|
worker1.EXPECT().UpdateSchema(mock.Anything, mock.AnythingOfType("*querypb.UpdateSchemaRequest")).RunAndReturn(func(ctx context.Context, usr *querypb.UpdateSchemaRequest) (*commonpb.Status, error) {
|
||||||
|
return merr.Success(), nil
|
||||||
|
}).Twice()
|
||||||
|
|
||||||
|
worker2.EXPECT().UpdateSchema(mock.Anything, mock.AnythingOfType("*querypb.UpdateSchemaRequest")).RunAndReturn(func(ctx context.Context, usr *querypb.UpdateSchemaRequest) (*commonpb.Status, error) {
|
||||||
|
return merr.Success(), nil
|
||||||
|
}).Once()
|
||||||
|
|
||||||
|
s.workerManager.EXPECT().GetWorker(mock.Anything, mock.AnythingOfType("int64")).Call.Return(func(_ context.Context, nodeID int64) cluster.Worker {
|
||||||
|
return workers[nodeID]
|
||||||
|
}, nil).Times(3) // currently node 1 will be called twice for growing & sealed
|
||||||
|
|
||||||
|
ctx, cancel := context.WithCancel(context.Background())
|
||||||
|
defer cancel()
|
||||||
|
|
||||||
|
err := s.delegator.UpdateSchema(ctx, &schemapb.CollectionSchema{})
|
||||||
|
s.NoError(err)
|
||||||
|
})
|
||||||
|
|
||||||
|
s.Run("worker_return_error", func() {
|
||||||
|
workers := make(map[int64]*cluster.MockWorker)
|
||||||
|
worker1 := cluster.NewMockWorker(s.T())
|
||||||
|
worker2 := cluster.NewMockWorker(s.T())
|
||||||
|
|
||||||
|
workers[1] = worker1
|
||||||
|
workers[2] = worker2
|
||||||
|
|
||||||
|
worker1.EXPECT().UpdateSchema(mock.Anything, mock.AnythingOfType("*querypb.UpdateSchemaRequest")).RunAndReturn(func(ctx context.Context, usr *querypb.UpdateSchemaRequest) (*commonpb.Status, error) {
|
||||||
|
return merr.Status(merr.WrapErrServiceInternal("mocked")), merr.WrapErrServiceInternal("mocked")
|
||||||
|
}).Maybe()
|
||||||
|
|
||||||
|
worker2.EXPECT().UpdateSchema(mock.Anything, mock.AnythingOfType("*querypb.UpdateSchemaRequest")).RunAndReturn(func(ctx context.Context, usr *querypb.UpdateSchemaRequest) (*commonpb.Status, error) {
|
||||||
|
return merr.Success(), nil
|
||||||
|
}).Maybe()
|
||||||
|
|
||||||
|
s.workerManager.EXPECT().GetWorker(mock.Anything, mock.AnythingOfType("int64")).Call.Return(func(_ context.Context, nodeID int64) cluster.Worker {
|
||||||
|
return workers[nodeID]
|
||||||
|
}, nil).Times(3)
|
||||||
|
|
||||||
|
ctx, cancel := context.WithCancel(context.Background())
|
||||||
|
defer cancel()
|
||||||
|
|
||||||
|
err := s.delegator.UpdateSchema(ctx, &schemapb.CollectionSchema{})
|
||||||
|
s.Error(err)
|
||||||
|
})
|
||||||
|
|
||||||
|
s.Run("worker_manager_error", func() {
|
||||||
|
s.workerManager.EXPECT().GetWorker(mock.Anything, mock.AnythingOfType("int64")).RunAndReturn(func(ctx context.Context, i int64) (cluster.Worker, error) {
|
||||||
|
return nil, merr.WrapErrServiceInternal("mocked")
|
||||||
|
}).Once()
|
||||||
|
ctx, cancel := context.WithCancel(context.Background())
|
||||||
|
defer cancel()
|
||||||
|
|
||||||
|
err := s.delegator.UpdateSchema(ctx, &schemapb.CollectionSchema{})
|
||||||
|
s.Error(err)
|
||||||
|
})
|
||||||
|
|
||||||
|
s.Run("distribution_not_serviceable", func() {
|
||||||
|
ctx, cancel := context.WithCancel(context.Background())
|
||||||
|
defer cancel()
|
||||||
|
|
||||||
|
sd, ok := s.delegator.(*shardDelegator)
|
||||||
|
s.Require().True(ok)
|
||||||
|
sd.distribution.AddOfflines(1001)
|
||||||
|
|
||||||
|
err := s.delegator.UpdateSchema(ctx, &schemapb.CollectionSchema{})
|
||||||
|
s.Error(err)
|
||||||
|
})
|
||||||
|
|
||||||
|
s.Run("cluster_not_serviceable", func() {
|
||||||
|
s.delegator.Close()
|
||||||
|
|
||||||
|
ctx, cancel := context.WithCancel(context.Background())
|
||||||
|
defer cancel()
|
||||||
|
|
||||||
|
err := s.delegator.UpdateSchema(ctx, &schemapb.CollectionSchema{})
|
||||||
|
s.Error(err)
|
||||||
|
})
|
||||||
|
}
|
||||||
|
|
||||||
func TestDelegatorSuite(t *testing.T) {
|
func TestDelegatorSuite(t *testing.T) {
|
||||||
suite.Run(t, new(DelegatorSuite))
|
suite.Run(t, new(DelegatorSuite))
|
||||||
}
|
}
|
||||||
|
|||||||
@ -12,6 +12,8 @@ import (
|
|||||||
|
|
||||||
querypb "github.com/milvus-io/milvus/pkg/v2/proto/querypb"
|
querypb "github.com/milvus-io/milvus/pkg/v2/proto/querypb"
|
||||||
|
|
||||||
|
schemapb "github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
|
||||||
|
|
||||||
streamrpc "github.com/milvus-io/milvus/internal/util/streamrpc"
|
streamrpc "github.com/milvus-io/milvus/internal/util/streamrpc"
|
||||||
)
|
)
|
||||||
|
|
||||||
@ -1105,6 +1107,53 @@ func (_c *MockShardDelegator_TryCleanExcludedSegments_Call) RunAndReturn(run fun
|
|||||||
return _c
|
return _c
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// UpdateSchema provides a mock function with given fields: ctx, sch
|
||||||
|
func (_m *MockShardDelegator) UpdateSchema(ctx context.Context, sch *schemapb.CollectionSchema) error {
|
||||||
|
ret := _m.Called(ctx, sch)
|
||||||
|
|
||||||
|
if len(ret) == 0 {
|
||||||
|
panic("no return value specified for UpdateSchema")
|
||||||
|
}
|
||||||
|
|
||||||
|
var r0 error
|
||||||
|
if rf, ok := ret.Get(0).(func(context.Context, *schemapb.CollectionSchema) error); ok {
|
||||||
|
r0 = rf(ctx, sch)
|
||||||
|
} else {
|
||||||
|
r0 = ret.Error(0)
|
||||||
|
}
|
||||||
|
|
||||||
|
return r0
|
||||||
|
}
|
||||||
|
|
||||||
|
// MockShardDelegator_UpdateSchema_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'UpdateSchema'
|
||||||
|
type MockShardDelegator_UpdateSchema_Call struct {
|
||||||
|
*mock.Call
|
||||||
|
}
|
||||||
|
|
||||||
|
// UpdateSchema is a helper method to define mock.On call
|
||||||
|
// - ctx context.Context
|
||||||
|
// - sch *schemapb.CollectionSchema
|
||||||
|
func (_e *MockShardDelegator_Expecter) UpdateSchema(ctx interface{}, sch interface{}) *MockShardDelegator_UpdateSchema_Call {
|
||||||
|
return &MockShardDelegator_UpdateSchema_Call{Call: _e.mock.On("UpdateSchema", ctx, sch)}
|
||||||
|
}
|
||||||
|
|
||||||
|
func (_c *MockShardDelegator_UpdateSchema_Call) Run(run func(ctx context.Context, sch *schemapb.CollectionSchema)) *MockShardDelegator_UpdateSchema_Call {
|
||||||
|
_c.Call.Run(func(args mock.Arguments) {
|
||||||
|
run(args[0].(context.Context), args[1].(*schemapb.CollectionSchema))
|
||||||
|
})
|
||||||
|
return _c
|
||||||
|
}
|
||||||
|
|
||||||
|
func (_c *MockShardDelegator_UpdateSchema_Call) Return(_a0 error) *MockShardDelegator_UpdateSchema_Call {
|
||||||
|
_c.Call.Return(_a0)
|
||||||
|
return _c
|
||||||
|
}
|
||||||
|
|
||||||
|
func (_c *MockShardDelegator_UpdateSchema_Call) RunAndReturn(run func(context.Context, *schemapb.CollectionSchema) error) *MockShardDelegator_UpdateSchema_Call {
|
||||||
|
_c.Call.Return(run)
|
||||||
|
return _c
|
||||||
|
}
|
||||||
|
|
||||||
// UpdateTSafe provides a mock function with given fields: ts
|
// UpdateTSafe provides a mock function with given fields: ts
|
||||||
func (_m *MockShardDelegator) UpdateTSafe(ts uint64) {
|
func (_m *MockShardDelegator) UpdateTSafe(ts uint64) {
|
||||||
_m.Called(ts)
|
_m.Called(ts)
|
||||||
|
|||||||
@ -19,6 +19,7 @@ package querynodev2
|
|||||||
import (
|
import (
|
||||||
"context"
|
"context"
|
||||||
|
|
||||||
|
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
|
||||||
"github.com/milvus-io/milvus/internal/querynodev2/cluster"
|
"github.com/milvus-io/milvus/internal/querynodev2/cluster"
|
||||||
"github.com/milvus-io/milvus/internal/util/streamrpc"
|
"github.com/milvus-io/milvus/internal/util/streamrpc"
|
||||||
"github.com/milvus-io/milvus/pkg/v2/proto/internalpb"
|
"github.com/milvus-io/milvus/pkg/v2/proto/internalpb"
|
||||||
@ -73,6 +74,10 @@ func (w *LocalWorker) GetStatistics(ctx context.Context, req *querypb.GetStatist
|
|||||||
return w.node.GetStatistics(ctx, req)
|
return w.node.GetStatistics(ctx, req)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func (w *LocalWorker) UpdateSchema(ctx context.Context, req *querypb.UpdateSchemaRequest) (*commonpb.Status, error) {
|
||||||
|
return w.node.UpdateSchema(ctx, req)
|
||||||
|
}
|
||||||
|
|
||||||
func (w *LocalWorker) IsHealthy() bool {
|
func (w *LocalWorker) IsHealthy() bool {
|
||||||
return true
|
return true
|
||||||
}
|
}
|
||||||
|
|||||||
@ -17,6 +17,7 @@
|
|||||||
package pipeline
|
package pipeline
|
||||||
|
|
||||||
import (
|
import (
|
||||||
|
"context"
|
||||||
"fmt"
|
"fmt"
|
||||||
|
|
||||||
"github.com/samber/lo"
|
"github.com/samber/lo"
|
||||||
@ -76,6 +77,10 @@ func (dNode *deleteNode) Operate(in Msg) Msg {
|
|||||||
dNode.delegator.ProcessDelete(lo.Values(deleteDatas), nodeMsg.timeRange.timestampMax)
|
dNode.delegator.ProcessDelete(lo.Values(deleteDatas), nodeMsg.timeRange.timestampMax)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
if nodeMsg.schema != nil {
|
||||||
|
dNode.delegator.UpdateSchema(context.Background(), nodeMsg.schema)
|
||||||
|
}
|
||||||
|
|
||||||
// update tSafe
|
// update tSafe
|
||||||
dNode.delegator.UpdateTSafe(nodeMsg.timeRange.timestampMax)
|
dNode.delegator.UpdateTSafe(nodeMsg.timeRange.timestampMax)
|
||||||
return nil
|
return nil
|
||||||
|
|||||||
@ -29,6 +29,7 @@ import (
|
|||||||
"github.com/milvus-io/milvus/pkg/v2/log"
|
"github.com/milvus-io/milvus/pkg/v2/log"
|
||||||
"github.com/milvus-io/milvus/pkg/v2/metrics"
|
"github.com/milvus-io/milvus/pkg/v2/metrics"
|
||||||
"github.com/milvus-io/milvus/pkg/v2/mq/msgstream"
|
"github.com/milvus-io/milvus/pkg/v2/mq/msgstream"
|
||||||
|
"github.com/milvus-io/milvus/pkg/v2/streaming/util/message/adaptor"
|
||||||
"github.com/milvus-io/milvus/pkg/v2/util/merr"
|
"github.com/milvus-io/milvus/pkg/v2/util/merr"
|
||||||
"github.com/milvus-io/milvus/pkg/v2/util/paramtable"
|
"github.com/milvus-io/milvus/pkg/v2/util/paramtable"
|
||||||
"github.com/milvus-io/milvus/pkg/v2/util/tsoutil"
|
"github.com/milvus-io/milvus/pkg/v2/util/tsoutil"
|
||||||
@ -134,6 +135,13 @@ func (fNode *filterNode) filtrate(c *Collection, msg msgstream.TsMsg) error {
|
|||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
case commonpb.MsgType_AddCollectionField:
|
||||||
|
schemaMsg := msg.(*adaptor.SchemaChangeMessageBody)
|
||||||
|
header := schemaMsg.SchemaChangeMessage.Header()
|
||||||
|
if header.GetCollectionId() != fNode.collectionID {
|
||||||
|
return merr.WrapErrCollectionNotFound(header.GetCollectionId())
|
||||||
|
}
|
||||||
|
return nil
|
||||||
default:
|
default:
|
||||||
return merr.WrapErrParameterInvalid("msgType is Insert or Delete", "not")
|
return merr.WrapErrParameterInvalid("msgType is Insert or Delete", "not")
|
||||||
}
|
}
|
||||||
|
|||||||
@ -118,6 +118,7 @@ func (iNode *insertNode) Operate(in Msg) Msg {
|
|||||||
return &deleteNodeMsg{
|
return &deleteNodeMsg{
|
||||||
deleteMsgs: nodeMsg.deleteMsgs,
|
deleteMsgs: nodeMsg.deleteMsgs,
|
||||||
timeRange: nodeMsg.timeRange,
|
timeRange: nodeMsg.timeRange,
|
||||||
|
schema: nodeMsg.schema,
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|||||||
@ -18,9 +18,11 @@ package pipeline
|
|||||||
|
|
||||||
import (
|
import (
|
||||||
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
|
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
|
||||||
|
"github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
|
||||||
"github.com/milvus-io/milvus/internal/querynodev2/collector"
|
"github.com/milvus-io/milvus/internal/querynodev2/collector"
|
||||||
"github.com/milvus-io/milvus/internal/querynodev2/delegator"
|
"github.com/milvus-io/milvus/internal/querynodev2/delegator"
|
||||||
"github.com/milvus-io/milvus/pkg/v2/mq/msgstream"
|
"github.com/milvus-io/milvus/pkg/v2/mq/msgstream"
|
||||||
|
"github.com/milvus-io/milvus/pkg/v2/streaming/util/message/adaptor"
|
||||||
"github.com/milvus-io/milvus/pkg/v2/util/merr"
|
"github.com/milvus-io/milvus/pkg/v2/util/merr"
|
||||||
"github.com/milvus-io/milvus/pkg/v2/util/metricsinfo"
|
"github.com/milvus-io/milvus/pkg/v2/util/metricsinfo"
|
||||||
)
|
)
|
||||||
@ -30,11 +32,13 @@ type insertNodeMsg struct {
|
|||||||
deleteMsgs []*DeleteMsg
|
deleteMsgs []*DeleteMsg
|
||||||
insertDatas map[int64]*delegator.InsertData
|
insertDatas map[int64]*delegator.InsertData
|
||||||
timeRange TimeRange
|
timeRange TimeRange
|
||||||
|
schema *schemapb.CollectionSchema
|
||||||
}
|
}
|
||||||
|
|
||||||
type deleteNodeMsg struct {
|
type deleteNodeMsg struct {
|
||||||
deleteMsgs []*DeleteMsg
|
deleteMsgs []*DeleteMsg
|
||||||
timeRange TimeRange
|
timeRange TimeRange
|
||||||
|
schema *schemapb.CollectionSchema
|
||||||
}
|
}
|
||||||
|
|
||||||
func (msg *insertNodeMsg) append(taskMsg msgstream.TsMsg) error {
|
func (msg *insertNodeMsg) append(taskMsg msgstream.TsMsg) error {
|
||||||
@ -47,6 +51,13 @@ func (msg *insertNodeMsg) append(taskMsg msgstream.TsMsg) error {
|
|||||||
deleteMsg := taskMsg.(*DeleteMsg)
|
deleteMsg := taskMsg.(*DeleteMsg)
|
||||||
msg.deleteMsgs = append(msg.deleteMsgs, deleteMsg)
|
msg.deleteMsgs = append(msg.deleteMsgs, deleteMsg)
|
||||||
collector.Rate.Add(metricsinfo.DeleteConsumeThroughput, float64(deleteMsg.Size()))
|
collector.Rate.Add(metricsinfo.DeleteConsumeThroughput, float64(deleteMsg.Size()))
|
||||||
|
case commonpb.MsgType_AddCollectionField:
|
||||||
|
schemaMsg := taskMsg.(*adaptor.SchemaChangeMessageBody)
|
||||||
|
body, err := schemaMsg.SchemaChangeMessage.Body()
|
||||||
|
if err != nil {
|
||||||
|
return err
|
||||||
|
}
|
||||||
|
msg.schema = body.GetSchema()
|
||||||
default:
|
default:
|
||||||
return merr.WrapErrParameterInvalid("msgType is Insert or Delete", "not")
|
return merr.WrapErrParameterInvalid("msgType is Insert or Delete", "not")
|
||||||
}
|
}
|
||||||
|
|||||||
@ -33,6 +33,7 @@ import (
|
|||||||
"github.com/milvus-io/milvus/pkg/v2/metrics"
|
"github.com/milvus-io/milvus/pkg/v2/metrics"
|
||||||
"github.com/milvus-io/milvus/pkg/v2/proto/querypb"
|
"github.com/milvus-io/milvus/pkg/v2/proto/querypb"
|
||||||
"github.com/milvus-io/milvus/pkg/v2/proto/segcorepb"
|
"github.com/milvus-io/milvus/pkg/v2/proto/segcorepb"
|
||||||
|
"github.com/milvus-io/milvus/pkg/v2/util/merr"
|
||||||
"github.com/milvus-io/milvus/pkg/v2/util/paramtable"
|
"github.com/milvus-io/milvus/pkg/v2/util/paramtable"
|
||||||
"github.com/milvus-io/milvus/pkg/v2/util/typeutil"
|
"github.com/milvus-io/milvus/pkg/v2/util/typeutil"
|
||||||
)
|
)
|
||||||
@ -47,6 +48,8 @@ type CollectionManager interface {
|
|||||||
// returns true if the collection ref count goes 0, or the collection not exists,
|
// returns true if the collection ref count goes 0, or the collection not exists,
|
||||||
// return false otherwise
|
// return false otherwise
|
||||||
Unref(collectionID int64, count uint32) bool
|
Unref(collectionID int64, count uint32) bool
|
||||||
|
// UpdateSchema update the underlying collection schema of the provided collection.
|
||||||
|
UpdateSchema(collectionID int64, schema *schemapb.CollectionSchema) error
|
||||||
}
|
}
|
||||||
|
|
||||||
type collectionManager struct {
|
type collectionManager struct {
|
||||||
@ -106,6 +109,22 @@ func (m *collectionManager) PutOrRef(collectionID int64, schema *schemapb.Collec
|
|||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func (m *collectionManager) UpdateSchema(collectionID int64, schema *schemapb.CollectionSchema) error {
|
||||||
|
m.mut.Lock()
|
||||||
|
defer m.mut.Unlock()
|
||||||
|
|
||||||
|
collection, ok := m.collections[collectionID]
|
||||||
|
if !ok {
|
||||||
|
return merr.WrapErrCollectionNotFound(collectionID, "collection not found in querynode collection manager")
|
||||||
|
}
|
||||||
|
|
||||||
|
if err := collection.ccollection.UpdateSchema(schema); err != nil {
|
||||||
|
return err
|
||||||
|
}
|
||||||
|
collection.schema.Store(schema)
|
||||||
|
return nil
|
||||||
|
}
|
||||||
|
|
||||||
func (m *collectionManager) updateMetric() {
|
func (m *collectionManager) updateMetric() {
|
||||||
metrics.QueryNodeNumCollections.WithLabelValues(fmt.Sprint(paramtable.GetNodeID())).Set(float64(len(m.collections)))
|
metrics.QueryNodeNumCollections.WithLabelValues(fmt.Sprint(paramtable.GetNodeID())).Set(float64(len(m.collections)))
|
||||||
}
|
}
|
||||||
|
|||||||
93
internal/querynodev2/segments/collection_test.go
Normal file
93
internal/querynodev2/segments/collection_test.go
Normal file
@ -0,0 +1,93 @@
|
|||||||
|
// Licensed to the LF AI & Data foundation under one
|
||||||
|
// or more contributor license agreements. See the NOTICE file
|
||||||
|
// distributed with this work for additional information
|
||||||
|
// regarding copyright ownership. The ASF licenses this file
|
||||||
|
// to you 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.
|
||||||
|
|
||||||
|
package segments
|
||||||
|
|
||||||
|
import (
|
||||||
|
"testing"
|
||||||
|
|
||||||
|
"github.com/stretchr/testify/suite"
|
||||||
|
|
||||||
|
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
|
||||||
|
"github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
|
||||||
|
"github.com/milvus-io/milvus/internal/mocks/util/mock_segcore"
|
||||||
|
"github.com/milvus-io/milvus/pkg/v2/common"
|
||||||
|
"github.com/milvus-io/milvus/pkg/v2/proto/querypb"
|
||||||
|
"github.com/milvus-io/milvus/pkg/v2/util/paramtable"
|
||||||
|
)
|
||||||
|
|
||||||
|
type CollectionManagerSuite struct {
|
||||||
|
suite.Suite
|
||||||
|
cm *collectionManager
|
||||||
|
}
|
||||||
|
|
||||||
|
func (s *CollectionManagerSuite) SetupSuite() {
|
||||||
|
paramtable.Init()
|
||||||
|
}
|
||||||
|
|
||||||
|
func (s *CollectionManagerSuite) SetupTest() {
|
||||||
|
s.cm = NewCollectionManager()
|
||||||
|
schema := mock_segcore.GenTestCollectionSchema("collection_1", schemapb.DataType_Int64, false)
|
||||||
|
err := s.cm.PutOrRef(1, schema, mock_segcore.GenTestIndexMeta(1, schema), &querypb.LoadMetaInfo{
|
||||||
|
LoadType: querypb.LoadType_LoadCollection,
|
||||||
|
DbProperties: []*commonpb.KeyValuePair{
|
||||||
|
{
|
||||||
|
Key: common.ReplicateIDKey,
|
||||||
|
Value: "local-test",
|
||||||
|
},
|
||||||
|
},
|
||||||
|
})
|
||||||
|
s.Require().NoError(err)
|
||||||
|
}
|
||||||
|
|
||||||
|
func (s *CollectionManagerSuite) TestUpdateSchema() {
|
||||||
|
s.Run("normal_case", func() {
|
||||||
|
schema := mock_segcore.GenTestCollectionSchema("collection_1", schemapb.DataType_Int64, false)
|
||||||
|
schema.Fields = append(schema.Fields, &schemapb.FieldSchema{
|
||||||
|
FieldID: common.StartOfUserFieldID + int64(len(schema.Fields)),
|
||||||
|
Name: "added_field",
|
||||||
|
DataType: schemapb.DataType_Bool,
|
||||||
|
Nullable: true,
|
||||||
|
})
|
||||||
|
|
||||||
|
err := s.cm.UpdateSchema(1, schema)
|
||||||
|
s.NoError(err)
|
||||||
|
})
|
||||||
|
|
||||||
|
s.Run("not_exist_collection", func() {
|
||||||
|
schema := mock_segcore.GenTestCollectionSchema("collection_1", schemapb.DataType_Int64, false)
|
||||||
|
schema.Fields = append(schema.Fields, &schemapb.FieldSchema{
|
||||||
|
FieldID: common.StartOfUserFieldID + int64(len(schema.Fields)),
|
||||||
|
Name: "added_field",
|
||||||
|
DataType: schemapb.DataType_Bool,
|
||||||
|
Nullable: true,
|
||||||
|
})
|
||||||
|
|
||||||
|
err := s.cm.UpdateSchema(2, schema)
|
||||||
|
s.Error(err)
|
||||||
|
})
|
||||||
|
|
||||||
|
s.Run("nil_schema", func() {
|
||||||
|
s.NotPanics(func() {
|
||||||
|
err := s.cm.UpdateSchema(1, nil)
|
||||||
|
s.Error(err)
|
||||||
|
})
|
||||||
|
})
|
||||||
|
}
|
||||||
|
|
||||||
|
func TestCollectionManager(t *testing.T) {
|
||||||
|
suite.Run(t, new(CollectionManagerSuite))
|
||||||
|
}
|
||||||
@ -308,6 +308,53 @@ func (_c *MockCollectionManager_Unref_Call) RunAndReturn(run func(int64, uint32)
|
|||||||
return _c
|
return _c
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// UpdateSchema provides a mock function with given fields: collectionID, schema
|
||||||
|
func (_m *MockCollectionManager) UpdateSchema(collectionID int64, schema *schemapb.CollectionSchema) error {
|
||||||
|
ret := _m.Called(collectionID, schema)
|
||||||
|
|
||||||
|
if len(ret) == 0 {
|
||||||
|
panic("no return value specified for UpdateSchema")
|
||||||
|
}
|
||||||
|
|
||||||
|
var r0 error
|
||||||
|
if rf, ok := ret.Get(0).(func(int64, *schemapb.CollectionSchema) error); ok {
|
||||||
|
r0 = rf(collectionID, schema)
|
||||||
|
} else {
|
||||||
|
r0 = ret.Error(0)
|
||||||
|
}
|
||||||
|
|
||||||
|
return r0
|
||||||
|
}
|
||||||
|
|
||||||
|
// MockCollectionManager_UpdateSchema_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'UpdateSchema'
|
||||||
|
type MockCollectionManager_UpdateSchema_Call struct {
|
||||||
|
*mock.Call
|
||||||
|
}
|
||||||
|
|
||||||
|
// UpdateSchema is a helper method to define mock.On call
|
||||||
|
// - collectionID int64
|
||||||
|
// - schema *schemapb.CollectionSchema
|
||||||
|
func (_e *MockCollectionManager_Expecter) UpdateSchema(collectionID interface{}, schema interface{}) *MockCollectionManager_UpdateSchema_Call {
|
||||||
|
return &MockCollectionManager_UpdateSchema_Call{Call: _e.mock.On("UpdateSchema", collectionID, schema)}
|
||||||
|
}
|
||||||
|
|
||||||
|
func (_c *MockCollectionManager_UpdateSchema_Call) Run(run func(collectionID int64, schema *schemapb.CollectionSchema)) *MockCollectionManager_UpdateSchema_Call {
|
||||||
|
_c.Call.Run(func(args mock.Arguments) {
|
||||||
|
run(args[0].(int64), args[1].(*schemapb.CollectionSchema))
|
||||||
|
})
|
||||||
|
return _c
|
||||||
|
}
|
||||||
|
|
||||||
|
func (_c *MockCollectionManager_UpdateSchema_Call) Return(_a0 error) *MockCollectionManager_UpdateSchema_Call {
|
||||||
|
_c.Call.Return(_a0)
|
||||||
|
return _c
|
||||||
|
}
|
||||||
|
|
||||||
|
func (_c *MockCollectionManager_UpdateSchema_Call) RunAndReturn(run func(int64, *schemapb.CollectionSchema) error) *MockCollectionManager_UpdateSchema_Call {
|
||||||
|
_c.Call.Return(run)
|
||||||
|
return _c
|
||||||
|
}
|
||||||
|
|
||||||
// NewMockCollectionManager creates a new instance of MockCollectionManager. It also registers a testing interface on the mock and a cleanup function to assert the mocks expectations.
|
// NewMockCollectionManager creates a new instance of MockCollectionManager. It also registers a testing interface on the mock and a cleanup function to assert the mocks expectations.
|
||||||
// The first argument is typically a *testing.T value.
|
// The first argument is typically a *testing.T value.
|
||||||
func NewMockCollectionManager(t interface {
|
func NewMockCollectionManager(t interface {
|
||||||
|
|||||||
@ -505,6 +505,28 @@ func (node *QueryNode) LoadSegments(ctx context.Context, req *querypb.LoadSegmen
|
|||||||
return merr.Success(), nil
|
return merr.Success(), nil
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// UpdateSchema updates the schema of the collection on the querynode.
|
||||||
|
func (node *QueryNode) UpdateSchema(ctx context.Context, req *querypb.UpdateSchemaRequest) (*commonpb.Status, error) {
|
||||||
|
defer node.updateDistributionModifyTS()
|
||||||
|
|
||||||
|
// check node healthy
|
||||||
|
if err := node.lifetime.Add(merr.IsHealthy); err != nil {
|
||||||
|
return merr.Status(err), nil
|
||||||
|
}
|
||||||
|
defer node.lifetime.Done()
|
||||||
|
|
||||||
|
log := log.Ctx(ctx).With(
|
||||||
|
zap.Int64("collectionID", req.GetCollectionID()),
|
||||||
|
)
|
||||||
|
|
||||||
|
err := node.manager.Collection.UpdateSchema(req.GetCollectionID(), req.GetSchema())
|
||||||
|
if err != nil {
|
||||||
|
log.Warn("failed to update schema", zap.Error(err))
|
||||||
|
}
|
||||||
|
|
||||||
|
return merr.Status(err), nil
|
||||||
|
}
|
||||||
|
|
||||||
// ReleaseCollection clears all data related to this collection on the querynode
|
// ReleaseCollection clears all data related to this collection on the querynode
|
||||||
func (node *QueryNode) ReleaseCollection(ctx context.Context, in *querypb.ReleaseCollectionRequest) (*commonpb.Status, error) {
|
func (node *QueryNode) ReleaseCollection(ctx context.Context, in *querypb.ReleaseCollectionRequest) (*commonpb.Status, error) {
|
||||||
if err := node.lifetime.Add(merr.IsHealthyOrStopping); err != nil {
|
if err := node.lifetime.Add(merr.IsHealthyOrStopping); err != nil {
|
||||||
|
|||||||
@ -2334,6 +2334,44 @@ func (suite *ServiceSuite) TestLoadPartition() {
|
|||||||
suite.Equal(commonpb.ErrorCode_Success, status.ErrorCode)
|
suite.Equal(commonpb.ErrorCode_Success, status.ErrorCode)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func (suite *ServiceSuite) TestUpdateSchema() {
|
||||||
|
ctx, cancel := context.WithCancel(context.Background())
|
||||||
|
defer cancel()
|
||||||
|
|
||||||
|
req := &querypb.UpdateSchemaRequest{
|
||||||
|
CollectionID: suite.collectionID,
|
||||||
|
Schema: suite.schema,
|
||||||
|
}
|
||||||
|
manager := suite.node.manager.Collection
|
||||||
|
// reset manager to align default teardown logic
|
||||||
|
defer func() {
|
||||||
|
suite.node.manager.Collection = manager
|
||||||
|
}()
|
||||||
|
mockManager := segments.NewMockCollectionManager(suite.T())
|
||||||
|
suite.node.manager.Collection = mockManager
|
||||||
|
|
||||||
|
suite.Run("normal", func() {
|
||||||
|
mockManager.EXPECT().UpdateSchema(suite.collectionID, suite.schema).Return(nil).Once()
|
||||||
|
|
||||||
|
status, err := suite.node.UpdateSchema(ctx, req)
|
||||||
|
suite.NoError(merr.CheckRPCCall(status, err))
|
||||||
|
})
|
||||||
|
|
||||||
|
suite.Run("manager_returns_error", func() {
|
||||||
|
mockManager.EXPECT().UpdateSchema(suite.collectionID, suite.schema).Return(merr.WrapErrServiceInternal("mocked")).Once()
|
||||||
|
|
||||||
|
status, err := suite.node.UpdateSchema(ctx, req)
|
||||||
|
suite.Error(merr.CheckRPCCall(status, err))
|
||||||
|
})
|
||||||
|
|
||||||
|
suite.Run("abonormal_node", func() {
|
||||||
|
suite.node.UpdateStateCode(commonpb.StateCode_Abnormal)
|
||||||
|
defer suite.node.UpdateStateCode(commonpb.StateCode_Healthy)
|
||||||
|
status, err := suite.node.UpdateSchema(ctx, req)
|
||||||
|
suite.Error(merr.CheckRPCCall(status, err))
|
||||||
|
})
|
||||||
|
}
|
||||||
|
|
||||||
func TestQueryNodeService(t *testing.T) {
|
func TestQueryNodeService(t *testing.T) {
|
||||||
suite.Run(t, new(ServiceSuite))
|
suite.Run(t, new(ServiceSuite))
|
||||||
}
|
}
|
||||||
|
|||||||
@ -88,8 +88,9 @@ func executeAddCollectionFieldTaskSteps(ctx context.Context,
|
|||||||
req *milvuspb.AddCollectionFieldRequest,
|
req *milvuspb.AddCollectionFieldRequest,
|
||||||
ts Timestamp,
|
ts Timestamp,
|
||||||
) error {
|
) error {
|
||||||
oldColl := col.Clone()
|
|
||||||
redoTask := newBaseRedoTask(core.stepExecutor)
|
redoTask := newBaseRedoTask(core.stepExecutor)
|
||||||
|
|
||||||
|
oldColl := col.Clone()
|
||||||
redoTask.AddSyncStep(&AddCollectionFieldStep{
|
redoTask.AddSyncStep(&AddCollectionFieldStep{
|
||||||
baseStep: baseStep{core: core},
|
baseStep: baseStep{core: core},
|
||||||
oldColl: oldColl,
|
oldColl: oldColl,
|
||||||
@ -97,6 +98,14 @@ func executeAddCollectionFieldTaskSteps(ctx context.Context,
|
|||||||
ts: ts,
|
ts: ts,
|
||||||
})
|
})
|
||||||
|
|
||||||
|
updatedCollection := col.Clone()
|
||||||
|
updatedCollection.Fields = append(updatedCollection.Fields, newField)
|
||||||
|
redoTask.AddSyncStep(&WriteSchemaChangeWALStep{
|
||||||
|
baseStep: baseStep{core: core},
|
||||||
|
collection: updatedCollection,
|
||||||
|
ts: ts,
|
||||||
|
})
|
||||||
|
|
||||||
req.CollectionID = oldColl.CollectionID
|
req.CollectionID = oldColl.CollectionID
|
||||||
redoTask.AddSyncStep(&BroadcastAlteredCollectionStep{
|
redoTask.AddSyncStep(&BroadcastAlteredCollectionStep{
|
||||||
baseStep: baseStep{core: core},
|
baseStep: baseStep{core: core},
|
||||||
|
|||||||
@ -28,8 +28,11 @@ import (
|
|||||||
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
|
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
|
||||||
"github.com/milvus-io/milvus-proto/go-api/v2/milvuspb"
|
"github.com/milvus-io/milvus-proto/go-api/v2/milvuspb"
|
||||||
"github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
|
"github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
|
||||||
|
"github.com/milvus-io/milvus/internal/distributed/streaming"
|
||||||
"github.com/milvus-io/milvus/internal/metastore/model"
|
"github.com/milvus-io/milvus/internal/metastore/model"
|
||||||
|
"github.com/milvus-io/milvus/internal/mocks/distributed/mock_streaming"
|
||||||
mockrootcoord "github.com/milvus-io/milvus/internal/rootcoord/mocks"
|
mockrootcoord "github.com/milvus-io/milvus/internal/rootcoord/mocks"
|
||||||
|
"github.com/milvus-io/milvus/pkg/v2/streaming/util/types"
|
||||||
"github.com/milvus-io/milvus/pkg/v2/util/merr"
|
"github.com/milvus-io/milvus/pkg/v2/util/merr"
|
||||||
)
|
)
|
||||||
|
|
||||||
@ -74,7 +77,29 @@ func Test_AddCollectionFieldTask_Prepare(t *testing.T) {
|
|||||||
}
|
}
|
||||||
|
|
||||||
func Test_AddCollectionFieldTask_Execute(t *testing.T) {
|
func Test_AddCollectionFieldTask_Execute(t *testing.T) {
|
||||||
t.Run("failed to get collection", func(t *testing.T) {
|
b := mock_streaming.NewMockBroadcast(t)
|
||||||
|
wal := mock_streaming.NewMockWALAccesser(t)
|
||||||
|
wal.EXPECT().Broadcast().Return(b).Maybe()
|
||||||
|
streaming.SetWALForTest(wal)
|
||||||
|
|
||||||
|
testCollection := &model.Collection{
|
||||||
|
CollectionID: int64(1),
|
||||||
|
Fields: []*model.Field{
|
||||||
|
{
|
||||||
|
Name: "pk",
|
||||||
|
IsPrimaryKey: true,
|
||||||
|
DataType: schemapb.DataType_Int64,
|
||||||
|
},
|
||||||
|
{
|
||||||
|
Name: "vec",
|
||||||
|
DataType: schemapb.DataType_FloatVector,
|
||||||
|
},
|
||||||
|
},
|
||||||
|
PhysicalChannelNames: []string{"dml_ch_01", "dml_ch_02"},
|
||||||
|
VirtualChannelNames: []string{"dml_ch_01", "dml_ch_02"},
|
||||||
|
}
|
||||||
|
|
||||||
|
t.Run("failed_to_get_collection", func(t *testing.T) {
|
||||||
metaTable := mockrootcoord.NewIMetaTable(t)
|
metaTable := mockrootcoord.NewIMetaTable(t)
|
||||||
metaTable.EXPECT().GetCollectionByName(mock.Anything, mock.Anything, "not_existed_coll", mock.Anything).Return(nil, merr.WrapErrCollectionNotFound("not_existed_coll"))
|
metaTable.EXPECT().GetCollectionByName(mock.Anything, mock.Anything, "not_existed_coll", mock.Anything).Return(nil, merr.WrapErrCollectionNotFound("not_existed_coll"))
|
||||||
core := newTestCore(withMeta(metaTable))
|
core := newTestCore(withMeta(metaTable))
|
||||||
@ -86,34 +111,24 @@ func Test_AddCollectionFieldTask_Execute(t *testing.T) {
|
|||||||
},
|
},
|
||||||
}
|
}
|
||||||
err := task.Execute(context.Background())
|
err := task.Execute(context.Background())
|
||||||
assert.Error(t, err)
|
assert.Error(t, err, "error shall be return when get collection failed")
|
||||||
})
|
})
|
||||||
|
|
||||||
t.Run("add field step failed", func(t *testing.T) {
|
t.Run("add field step failed", func(t *testing.T) {
|
||||||
meta := mockrootcoord.NewIMetaTable(t)
|
meta := mockrootcoord.NewIMetaTable(t)
|
||||||
meta.On("GetCollectionByName",
|
meta.EXPECT().GetCollectionByName(
|
||||||
mock.Anything,
|
mock.Anything,
|
||||||
mock.Anything,
|
mock.Anything,
|
||||||
mock.Anything,
|
mock.Anything,
|
||||||
mock.Anything,
|
mock.Anything,
|
||||||
).Return(&model.Collection{CollectionID: int64(1), Fields: []*model.Field{
|
).Return(testCollection, nil)
|
||||||
{
|
meta.EXPECT().AlterCollection(
|
||||||
Name: "pk",
|
|
||||||
IsPrimaryKey: true,
|
|
||||||
DataType: schemapb.DataType_Int64,
|
|
||||||
},
|
|
||||||
{
|
|
||||||
Name: "vec",
|
|
||||||
DataType: schemapb.DataType_FloatVector,
|
|
||||||
},
|
|
||||||
}}, nil)
|
|
||||||
meta.On("AlterCollection",
|
|
||||||
mock.Anything,
|
mock.Anything,
|
||||||
mock.Anything,
|
mock.Anything,
|
||||||
mock.Anything,
|
mock.Anything,
|
||||||
mock.Anything,
|
mock.Anything,
|
||||||
).Return(errors.New("mock"))
|
).Return(errors.New("mock"))
|
||||||
meta.On("ListAliasesByID", mock.Anything, mock.Anything).Return([]string{})
|
meta.EXPECT().ListAliasesByID(mock.Anything, mock.Anything).Return([]string{})
|
||||||
alloc := newMockIDAllocator()
|
alloc := newMockIDAllocator()
|
||||||
core := newTestCore(withValidProxyManager(), withMeta(meta), withIDAllocator(alloc))
|
core := newTestCore(withValidProxyManager(), withMeta(meta), withIDAllocator(alloc))
|
||||||
task := &addCollectionFieldTask{
|
task := &addCollectionFieldTask{
|
||||||
@ -132,31 +147,74 @@ func Test_AddCollectionFieldTask_Execute(t *testing.T) {
|
|||||||
assert.Error(t, err)
|
assert.Error(t, err)
|
||||||
})
|
})
|
||||||
|
|
||||||
t.Run("broadcast add field step failed", func(t *testing.T) {
|
t.Run("write_wal_fail", func(t *testing.T) {
|
||||||
meta := mockrootcoord.NewIMetaTable(t)
|
meta := mockrootcoord.NewIMetaTable(t)
|
||||||
meta.On("GetCollectionByName",
|
meta.EXPECT().GetCollectionByName(
|
||||||
mock.Anything,
|
mock.Anything,
|
||||||
mock.Anything,
|
mock.Anything,
|
||||||
mock.Anything,
|
mock.Anything,
|
||||||
mock.Anything,
|
mock.Anything,
|
||||||
).Return(&model.Collection{CollectionID: int64(1), Fields: []*model.Field{
|
).Return(testCollection, nil)
|
||||||
{
|
meta.EXPECT().AlterCollection(
|
||||||
Name: "pk",
|
|
||||||
IsPrimaryKey: true,
|
|
||||||
DataType: schemapb.DataType_Int64,
|
|
||||||
},
|
|
||||||
{
|
|
||||||
Name: "vec",
|
|
||||||
DataType: schemapb.DataType_FloatVector,
|
|
||||||
},
|
|
||||||
}}, nil)
|
|
||||||
meta.On("AlterCollection",
|
|
||||||
mock.Anything,
|
mock.Anything,
|
||||||
mock.Anything,
|
mock.Anything,
|
||||||
mock.Anything,
|
mock.Anything,
|
||||||
mock.Anything,
|
mock.Anything,
|
||||||
).Return(nil)
|
).Return(nil)
|
||||||
meta.On("ListAliasesByID", mock.Anything, mock.Anything).Return([]string{})
|
meta.EXPECT().ListAliasesByID(mock.Anything, mock.Anything).Return([]string{})
|
||||||
|
|
||||||
|
broker := newMockBroker()
|
||||||
|
broker.BroadcastAlteredCollectionFunc = func(ctx context.Context, req *milvuspb.AlterCollectionRequest) error {
|
||||||
|
return errors.New("mock")
|
||||||
|
}
|
||||||
|
alloc := newMockIDAllocator()
|
||||||
|
core := newTestCore(withValidProxyManager(), withMeta(meta), withBroker(broker), withIDAllocator(alloc))
|
||||||
|
task := &addCollectionFieldTask{
|
||||||
|
baseTask: newBaseTask(context.Background(), core),
|
||||||
|
Req: &milvuspb.AddCollectionFieldRequest{
|
||||||
|
Base: &commonpb.MsgBase{MsgType: commonpb.MsgType_AlterAlias},
|
||||||
|
CollectionName: "coll",
|
||||||
|
},
|
||||||
|
fieldSchema: &schemapb.FieldSchema{
|
||||||
|
Name: "fid",
|
||||||
|
DataType: schemapb.DataType_Bool,
|
||||||
|
Nullable: true,
|
||||||
|
},
|
||||||
|
}
|
||||||
|
|
||||||
|
t.Run("write_schema_change_fail", func(t *testing.T) {
|
||||||
|
b.EXPECT().Append(mock.Anything, mock.Anything).Return(nil, merr.WrapErrServiceInternal("mocked")).Once()
|
||||||
|
|
||||||
|
err := task.Execute(context.Background())
|
||||||
|
assert.Error(t, err)
|
||||||
|
})
|
||||||
|
|
||||||
|
t.Run("write_flush_failed", func(t *testing.T) {
|
||||||
|
b.EXPECT().Append(mock.Anything, mock.Anything).Return(&types.BroadcastAppendResult{}, nil).Once()
|
||||||
|
b.EXPECT().Append(mock.Anything, mock.Anything).Return(nil, merr.WrapErrServiceInternal("mocked")).Once()
|
||||||
|
|
||||||
|
err := task.Execute(context.Background())
|
||||||
|
assert.Error(t, err)
|
||||||
|
})
|
||||||
|
})
|
||||||
|
|
||||||
|
t.Run("broadcast add field step failed", func(t *testing.T) {
|
||||||
|
b.EXPECT().Append(mock.Anything, mock.Anything).Return(&types.BroadcastAppendResult{}, nil).Times(2)
|
||||||
|
|
||||||
|
meta := mockrootcoord.NewIMetaTable(t)
|
||||||
|
meta.EXPECT().GetCollectionByName(
|
||||||
|
mock.Anything,
|
||||||
|
mock.Anything,
|
||||||
|
mock.Anything,
|
||||||
|
mock.Anything,
|
||||||
|
).Return(testCollection, nil)
|
||||||
|
meta.EXPECT().AlterCollection(
|
||||||
|
mock.Anything,
|
||||||
|
mock.Anything,
|
||||||
|
mock.Anything,
|
||||||
|
mock.Anything,
|
||||||
|
).Return(nil)
|
||||||
|
meta.EXPECT().ListAliasesByID(mock.Anything, mock.Anything).Return([]string{})
|
||||||
|
|
||||||
broker := newMockBroker()
|
broker := newMockBroker()
|
||||||
broker.BroadcastAlteredCollectionFunc = func(ctx context.Context, req *milvuspb.AlterCollectionRequest) error {
|
broker.BroadcastAlteredCollectionFunc = func(ctx context.Context, req *milvuspb.AlterCollectionRequest) error {
|
||||||
@ -181,30 +239,22 @@ func Test_AddCollectionFieldTask_Execute(t *testing.T) {
|
|||||||
})
|
})
|
||||||
|
|
||||||
t.Run("expire cache failed", func(t *testing.T) {
|
t.Run("expire cache failed", func(t *testing.T) {
|
||||||
|
b.EXPECT().Append(mock.Anything, mock.Anything).Return(&types.BroadcastAppendResult{}, nil).Times(2)
|
||||||
|
|
||||||
meta := mockrootcoord.NewIMetaTable(t)
|
meta := mockrootcoord.NewIMetaTable(t)
|
||||||
meta.On("GetCollectionByName",
|
meta.EXPECT().GetCollectionByName(
|
||||||
mock.Anything,
|
mock.Anything,
|
||||||
mock.Anything,
|
mock.Anything,
|
||||||
mock.Anything,
|
mock.Anything,
|
||||||
mock.Anything,
|
mock.Anything,
|
||||||
).Return(&model.Collection{CollectionID: int64(1), Fields: []*model.Field{
|
).Return(testCollection, nil)
|
||||||
{
|
meta.EXPECT().AlterCollection(
|
||||||
Name: "pk",
|
|
||||||
IsPrimaryKey: true,
|
|
||||||
DataType: schemapb.DataType_Int64,
|
|
||||||
},
|
|
||||||
{
|
|
||||||
Name: "vec",
|
|
||||||
DataType: schemapb.DataType_FloatVector,
|
|
||||||
},
|
|
||||||
}}, nil)
|
|
||||||
meta.On("AlterCollection",
|
|
||||||
mock.Anything,
|
mock.Anything,
|
||||||
mock.Anything,
|
mock.Anything,
|
||||||
mock.Anything,
|
mock.Anything,
|
||||||
mock.Anything,
|
mock.Anything,
|
||||||
).Return(nil)
|
).Return(nil)
|
||||||
meta.On("ListAliasesByID", mock.Anything, mock.Anything).Return([]string{})
|
meta.EXPECT().ListAliasesByID(mock.Anything, mock.Anything).Return([]string{})
|
||||||
|
|
||||||
broker := newMockBroker()
|
broker := newMockBroker()
|
||||||
broker.BroadcastAlteredCollectionFunc = func(ctx context.Context, req *milvuspb.AlterCollectionRequest) error {
|
broker.BroadcastAlteredCollectionFunc = func(ctx context.Context, req *milvuspb.AlterCollectionRequest) error {
|
||||||
@ -230,30 +280,22 @@ func Test_AddCollectionFieldTask_Execute(t *testing.T) {
|
|||||||
})
|
})
|
||||||
|
|
||||||
t.Run("normal case", func(t *testing.T) {
|
t.Run("normal case", func(t *testing.T) {
|
||||||
|
b.EXPECT().Append(mock.Anything, mock.Anything).Return(&types.BroadcastAppendResult{}, nil).Times(2)
|
||||||
|
|
||||||
meta := mockrootcoord.NewIMetaTable(t)
|
meta := mockrootcoord.NewIMetaTable(t)
|
||||||
meta.On("GetCollectionByName",
|
meta.EXPECT().GetCollectionByName(
|
||||||
mock.Anything,
|
mock.Anything,
|
||||||
mock.Anything,
|
mock.Anything,
|
||||||
mock.Anything,
|
mock.Anything,
|
||||||
mock.Anything,
|
mock.Anything,
|
||||||
).Return(&model.Collection{CollectionID: int64(1), Fields: []*model.Field{
|
).Return(testCollection, nil)
|
||||||
{
|
meta.EXPECT().AlterCollection(
|
||||||
Name: "pk",
|
|
||||||
IsPrimaryKey: true,
|
|
||||||
DataType: schemapb.DataType_Int64,
|
|
||||||
},
|
|
||||||
{
|
|
||||||
Name: "vec",
|
|
||||||
DataType: schemapb.DataType_FloatVector,
|
|
||||||
},
|
|
||||||
}}, nil)
|
|
||||||
meta.On("AlterCollection",
|
|
||||||
mock.Anything,
|
mock.Anything,
|
||||||
mock.Anything,
|
mock.Anything,
|
||||||
mock.Anything,
|
mock.Anything,
|
||||||
mock.Anything,
|
mock.Anything,
|
||||||
).Return(nil)
|
).Return(nil)
|
||||||
meta.On("ListAliasesByID", mock.Anything, mock.Anything).Return([]string{})
|
meta.EXPECT().ListAliasesByID(mock.Anything, mock.Anything).Return([]string{})
|
||||||
|
|
||||||
broker := newMockBroker()
|
broker := newMockBroker()
|
||||||
broker.BroadcastAlteredCollectionFunc = func(ctx context.Context, req *milvuspb.AlterCollectionRequest) error {
|
broker.BroadcastAlteredCollectionFunc = func(ctx context.Context, req *milvuspb.AlterCollectionRequest) error {
|
||||||
@ -264,7 +306,7 @@ func Test_AddCollectionFieldTask_Execute(t *testing.T) {
|
|||||||
task := &addCollectionFieldTask{
|
task := &addCollectionFieldTask{
|
||||||
baseTask: newBaseTask(context.Background(), core),
|
baseTask: newBaseTask(context.Background(), core),
|
||||||
Req: &milvuspb.AddCollectionFieldRequest{
|
Req: &milvuspb.AddCollectionFieldRequest{
|
||||||
Base: &commonpb.MsgBase{MsgType: commonpb.MsgType_AlterAlias},
|
Base: &commonpb.MsgBase{MsgType: commonpb.MsgType_AddCollectionField},
|
||||||
CollectionName: "coll",
|
CollectionName: "coll",
|
||||||
},
|
},
|
||||||
fieldSchema: &schemapb.FieldSchema{
|
fieldSchema: &schemapb.FieldSchema{
|
||||||
|
|||||||
@ -26,11 +26,13 @@ import (
|
|||||||
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
|
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
|
||||||
"github.com/milvus-io/milvus-proto/go-api/v2/milvuspb"
|
"github.com/milvus-io/milvus-proto/go-api/v2/milvuspb"
|
||||||
"github.com/milvus-io/milvus-proto/go-api/v2/msgpb"
|
"github.com/milvus-io/milvus-proto/go-api/v2/msgpb"
|
||||||
|
"github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
|
||||||
"github.com/milvus-io/milvus/internal/distributed/streaming"
|
"github.com/milvus-io/milvus/internal/distributed/streaming"
|
||||||
"github.com/milvus-io/milvus/internal/metastore/model"
|
"github.com/milvus-io/milvus/internal/metastore/model"
|
||||||
"github.com/milvus-io/milvus/internal/util/proxyutil"
|
"github.com/milvus-io/milvus/internal/util/proxyutil"
|
||||||
"github.com/milvus-io/milvus/pkg/v2/log"
|
"github.com/milvus-io/milvus/pkg/v2/log"
|
||||||
pb "github.com/milvus-io/milvus/pkg/v2/proto/etcdpb"
|
pb "github.com/milvus-io/milvus/pkg/v2/proto/etcdpb"
|
||||||
|
"github.com/milvus-io/milvus/pkg/v2/proto/messagespb"
|
||||||
"github.com/milvus-io/milvus/pkg/v2/streaming/util/message"
|
"github.com/milvus-io/milvus/pkg/v2/streaming/util/message"
|
||||||
"github.com/milvus-io/milvus/pkg/v2/util/commonpbutil"
|
"github.com/milvus-io/milvus/pkg/v2/util/commonpbutil"
|
||||||
)
|
)
|
||||||
@ -514,6 +516,77 @@ func (a *AddCollectionFieldStep) Desc() string {
|
|||||||
return fmt.Sprintf("add field, collectionID: %d, fieldID: %d, ts: %d", a.oldColl.CollectionID, a.newField.FieldID, a.ts)
|
return fmt.Sprintf("add field, collectionID: %d, fieldID: %d, ts: %d", a.oldColl.CollectionID, a.newField.FieldID, a.ts)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
type WriteSchemaChangeWALStep struct {
|
||||||
|
baseStep
|
||||||
|
collection *model.Collection
|
||||||
|
ts Timestamp
|
||||||
|
}
|
||||||
|
|
||||||
|
func (s *WriteSchemaChangeWALStep) Execute(ctx context.Context) ([]nestedStep, error) {
|
||||||
|
vchannels := s.collection.VirtualChannelNames
|
||||||
|
schema := &schemapb.CollectionSchema{
|
||||||
|
Name: s.collection.Name,
|
||||||
|
Description: s.collection.Description,
|
||||||
|
AutoID: s.collection.AutoID,
|
||||||
|
Fields: model.MarshalFieldModels(s.collection.Fields),
|
||||||
|
Functions: model.MarshalFunctionModels(s.collection.Functions),
|
||||||
|
EnableDynamicField: s.collection.EnableDynamicField,
|
||||||
|
Properties: s.collection.Properties,
|
||||||
|
}
|
||||||
|
|
||||||
|
schemaMsg, err := message.NewSchemaChangeMessageBuilderV2().
|
||||||
|
WithBroadcast(vchannels).
|
||||||
|
WithHeader(&message.SchemaChangeMessageHeader{
|
||||||
|
CollectionId: s.collection.CollectionID,
|
||||||
|
}).
|
||||||
|
WithBody(&message.SchemaChangeMessageBody{
|
||||||
|
Schema: schema,
|
||||||
|
ModifyTs: s.ts,
|
||||||
|
}).BuildBroadcast()
|
||||||
|
if err != nil {
|
||||||
|
return nil, err
|
||||||
|
}
|
||||||
|
|
||||||
|
resp, err := streaming.WAL().Broadcast().Append(ctx, schemaMsg)
|
||||||
|
if err != nil {
|
||||||
|
return nil, err
|
||||||
|
}
|
||||||
|
|
||||||
|
log.Ctx(ctx).Info(
|
||||||
|
"broadcast schema change success",
|
||||||
|
zap.Uint64("broadcastID", resp.BroadcastID),
|
||||||
|
zap.Any("appendResults", resp.AppendResults),
|
||||||
|
)
|
||||||
|
|
||||||
|
flushMsg, err := message.NewManualFlushMessageBuilderV2().
|
||||||
|
WithBroadcast(vchannels).
|
||||||
|
WithHeader(&messagespb.ManualFlushMessageHeader{
|
||||||
|
CollectionId: s.collection.CollectionID,
|
||||||
|
FlushTs: s.ts,
|
||||||
|
}).
|
||||||
|
WithBody(&message.ManualFlushMessageBody{}).BuildBroadcast()
|
||||||
|
if err != nil {
|
||||||
|
return nil, err
|
||||||
|
}
|
||||||
|
|
||||||
|
resp, err = streaming.WAL().Broadcast().Append(ctx, flushMsg)
|
||||||
|
if err != nil {
|
||||||
|
return nil, err
|
||||||
|
}
|
||||||
|
|
||||||
|
log.Ctx(ctx).Info(
|
||||||
|
"broadcast schema change success",
|
||||||
|
zap.Uint64("broadcastID", resp.BroadcastID),
|
||||||
|
zap.Any("appendResults", resp.AppendResults),
|
||||||
|
)
|
||||||
|
|
||||||
|
return nil, nil
|
||||||
|
}
|
||||||
|
|
||||||
|
func (s *WriteSchemaChangeWALStep) Desc() string {
|
||||||
|
return fmt.Sprintf("write schema change WALcollectionID: %d, ts: %d", s.collection.CollectionID, s.ts)
|
||||||
|
}
|
||||||
|
|
||||||
type AlterDatabaseStep struct {
|
type AlterDatabaseStep struct {
|
||||||
baseStep
|
baseStep
|
||||||
oldDB *model.Database
|
oldDB *model.Database
|
||||||
|
|||||||
@ -31,6 +31,7 @@ import (
|
|||||||
"github.com/milvus-io/milvus/pkg/v2/log"
|
"github.com/milvus-io/milvus/pkg/v2/log"
|
||||||
"github.com/milvus-io/milvus/pkg/v2/proto/internalpb"
|
"github.com/milvus-io/milvus/pkg/v2/proto/internalpb"
|
||||||
"github.com/milvus-io/milvus/pkg/v2/streaming/util/message"
|
"github.com/milvus-io/milvus/pkg/v2/streaming/util/message"
|
||||||
|
"github.com/milvus-io/milvus/pkg/v2/streaming/util/message/adaptor"
|
||||||
"github.com/milvus-io/milvus/pkg/v2/streaming/util/types"
|
"github.com/milvus-io/milvus/pkg/v2/streaming/util/types"
|
||||||
"github.com/milvus-io/milvus/pkg/v2/util/funcutil"
|
"github.com/milvus-io/milvus/pkg/v2/util/funcutil"
|
||||||
"github.com/milvus-io/milvus/pkg/v2/util/merr"
|
"github.com/milvus-io/milvus/pkg/v2/util/merr"
|
||||||
@ -84,8 +85,22 @@ func (impl *msgHandlerImpl) HandleManualFlush(vchannel string, flushMsg message.
|
|||||||
if err := impl.wbMgr.FlushChannel(context.Background(), vchannel, flushMsg.Header().GetFlushTs()); err != nil {
|
if err := impl.wbMgr.FlushChannel(context.Background(), vchannel, flushMsg.Header().GetFlushTs()); err != nil {
|
||||||
return errors.Wrap(err, "failed to flush channel")
|
return errors.Wrap(err, "failed to flush channel")
|
||||||
}
|
}
|
||||||
|
broadcastID := flushMsg.BroadcastHeader().BroadcastID
|
||||||
|
if broadcastID == 0 {
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
return streaming.WAL().Broadcast().Ack(context.Background(), types.BroadcastAckRequest{
|
||||||
|
BroadcastID: flushMsg.BroadcastHeader().BroadcastID,
|
||||||
|
VChannel: vchannel,
|
||||||
|
})
|
||||||
|
}
|
||||||
|
|
||||||
|
func (impl *msgHandlerImpl) HandleSchemaChange(ctx context.Context, vchannel string, msg *adaptor.SchemaChangeMessageBody) error {
|
||||||
|
return streaming.WAL().Broadcast().Ack(ctx, types.BroadcastAckRequest{
|
||||||
|
BroadcastID: msg.BroadcastID,
|
||||||
|
VChannel: vchannel,
|
||||||
|
})
|
||||||
|
}
|
||||||
|
|
||||||
func (impl *msgHandlerImpl) HandleImport(ctx context.Context, vchannel string, importMsg *msgpb.ImportMsg) error {
|
func (impl *msgHandlerImpl) HandleImport(ctx context.Context, vchannel string, importMsg *msgpb.ImportMsg) error {
|
||||||
return retry.Do(ctx, func() (err error) {
|
return retry.Do(ctx, func() (err error) {
|
||||||
|
|||||||
@ -29,7 +29,7 @@ func buildInterceptorParams(ctx context.Context, underlyingWALImpls walimpls.WAL
|
|||||||
|
|
||||||
capacity := int(paramtable.Get().StreamingCfg.WALWriteAheadBufferCapacity.GetAsSize())
|
capacity := int(paramtable.Get().StreamingCfg.WALWriteAheadBufferCapacity.GetAsSize())
|
||||||
keepalive := paramtable.Get().StreamingCfg.WALWriteAheadBufferKeepalive.GetAsDurationByParse()
|
keepalive := paramtable.Get().StreamingCfg.WALWriteAheadBufferKeepalive.GetAsDurationByParse()
|
||||||
writeAheadBuffer := wab.NewWirteAheadBuffer(
|
writeAheadBuffer := wab.NewWriteAheadBuffer(
|
||||||
underlyingWALImpls.Channel().Name,
|
underlyingWALImpls.Channel().Name,
|
||||||
resource.Resource().Logger().With(),
|
resource.Resource().Logger().With(),
|
||||||
capacity,
|
capacity,
|
||||||
|
|||||||
@ -48,7 +48,7 @@ func TestTimeTickSyncOperator(t *testing.T) {
|
|||||||
WAL: walFuture,
|
WAL: walFuture,
|
||||||
InitializedTimeTick: ts,
|
InitializedTimeTick: ts,
|
||||||
InitializedMessageID: msgID,
|
InitializedMessageID: msgID,
|
||||||
WriteAheadBuffer: wab.NewWirteAheadBuffer(
|
WriteAheadBuffer: wab.NewWriteAheadBuffer(
|
||||||
channel.Name,
|
channel.Name,
|
||||||
resource.Resource().Logger().With(),
|
resource.Resource().Logger().With(),
|
||||||
1024,
|
1024,
|
||||||
|
|||||||
@ -24,7 +24,7 @@ type ROWriteAheadBuffer interface {
|
|||||||
}
|
}
|
||||||
|
|
||||||
// NewWriteAheadBuffer creates a new WriteAheadBuffer.
|
// NewWriteAheadBuffer creates a new WriteAheadBuffer.
|
||||||
func NewWirteAheadBuffer(
|
func NewWriteAheadBuffer(
|
||||||
pchannel string,
|
pchannel string,
|
||||||
logger *log.MLogger,
|
logger *log.MLogger,
|
||||||
capacity int,
|
capacity int,
|
||||||
|
|||||||
@ -17,7 +17,7 @@ import (
|
|||||||
|
|
||||||
func TestWriteAheadBufferWithOnlyTrivialTimeTick(t *testing.T) {
|
func TestWriteAheadBufferWithOnlyTrivialTimeTick(t *testing.T) {
|
||||||
ctx := context.Background()
|
ctx := context.Background()
|
||||||
wb := NewWirteAheadBuffer("pchannel", log.With(), 5*1024*1024, 30*time.Second, createTimeTickMessage(0))
|
wb := NewWriteAheadBuffer("pchannel", log.With(), 5*1024*1024, 30*time.Second, createTimeTickMessage(0))
|
||||||
|
|
||||||
// Test timeout
|
// Test timeout
|
||||||
ctx, cancel := context.WithTimeout(ctx, 1*time.Millisecond)
|
ctx, cancel := context.WithTimeout(ctx, 1*time.Millisecond)
|
||||||
@ -81,7 +81,7 @@ func TestWriteAheadBufferWithOnlyTrivialTimeTick(t *testing.T) {
|
|||||||
func TestWriteAheadBuffer(t *testing.T) {
|
func TestWriteAheadBuffer(t *testing.T) {
|
||||||
// Concurrent add message into bufffer and make syncup.
|
// Concurrent add message into bufffer and make syncup.
|
||||||
// The reader should never lost any message if no eviction happen.
|
// The reader should never lost any message if no eviction happen.
|
||||||
wb := NewWirteAheadBuffer("pchannel", log.With(), 5*1024*1024, 30*time.Second, createTimeTickMessage(1))
|
wb := NewWriteAheadBuffer("pchannel", log.With(), 5*1024*1024, 30*time.Second, createTimeTickMessage(1))
|
||||||
expectedLastTimeTick := uint64(10000)
|
expectedLastTimeTick := uint64(10000)
|
||||||
ch := make(chan struct{})
|
ch := make(chan struct{})
|
||||||
totalCnt := 0
|
totalCnt := 0
|
||||||
@ -183,7 +183,7 @@ func TestWriteAheadBuffer(t *testing.T) {
|
|||||||
}
|
}
|
||||||
|
|
||||||
func TestWriteAheadBufferEviction(t *testing.T) {
|
func TestWriteAheadBufferEviction(t *testing.T) {
|
||||||
wb := NewWirteAheadBuffer("pchannel", log.With(), 5*1024*1024, 50*time.Millisecond, createTimeTickMessage(0))
|
wb := NewWriteAheadBuffer("pchannel", log.With(), 5*1024*1024, 50*time.Millisecond, createTimeTickMessage(0))
|
||||||
|
|
||||||
msgs := make([]message.ImmutableMessage, 0)
|
msgs := make([]message.ImmutableMessage, 0)
|
||||||
for i := 1; i < 100; i++ {
|
for i := 1; i < 100; i++ {
|
||||||
|
|||||||
@ -134,6 +134,10 @@ func (m *GrpcQueryNodeClient) DeleteBatch(ctx context.Context, in *querypb.Delet
|
|||||||
return &querypb.DeleteBatchResponse{}, m.Err
|
return &querypb.DeleteBatchResponse{}, m.Err
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func (m *GrpcQueryNodeClient) UpdateSchema(ctx context.Context, in *querypb.UpdateSchemaRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
|
||||||
|
return &commonpb.Status{}, m.Err
|
||||||
|
}
|
||||||
|
|
||||||
func (m *GrpcQueryNodeClient) Close() error {
|
func (m *GrpcQueryNodeClient) Close() error {
|
||||||
return m.Err
|
return m.Err
|
||||||
}
|
}
|
||||||
|
|||||||
@ -114,7 +114,8 @@ func (p *streamPipeline) ConsumeMsgStream(ctx context.Context, position *msgpb.M
|
|||||||
// only consume messages with timestamp >= position timestamp
|
// only consume messages with timestamp >= position timestamp
|
||||||
options.DeliverFilterTimeTickGTE(position.GetTimestamp()),
|
options.DeliverFilterTimeTickGTE(position.GetTimestamp()),
|
||||||
// only consume insert and delete messages
|
// only consume insert and delete messages
|
||||||
options.DeliverFilterMessageType(message.MessageTypeInsert, message.MessageTypeDelete),
|
// also schema change message to notify schema change events
|
||||||
|
options.DeliverFilterMessageType(message.MessageTypeInsert, message.MessageTypeDelete, message.MessageTypeSchemaChange),
|
||||||
},
|
},
|
||||||
MessageHandler: handler,
|
MessageHandler: handler,
|
||||||
})
|
})
|
||||||
|
|||||||
@ -16,6 +16,7 @@ import (
|
|||||||
|
|
||||||
"github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
|
"github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
|
||||||
"github.com/milvus-io/milvus/pkg/v2/proto/segcorepb"
|
"github.com/milvus-io/milvus/pkg/v2/proto/segcorepb"
|
||||||
|
"github.com/milvus-io/milvus/pkg/v2/util/merr"
|
||||||
)
|
)
|
||||||
|
|
||||||
// CreateCCollectionRequest is a request to create a CCollection.
|
// CreateCCollectionRequest is a request to create a CCollection.
|
||||||
@ -85,6 +86,20 @@ func (c *CCollection) IndexMeta() *segcorepb.CollectionIndexMeta {
|
|||||||
return c.indexMeta
|
return c.indexMeta
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func (c *CCollection) UpdateSchema(sch *schemapb.CollectionSchema) error {
|
||||||
|
if sch == nil {
|
||||||
|
return merr.WrapErrServiceInternal("update collection schema with nil")
|
||||||
|
}
|
||||||
|
|
||||||
|
schemaBlob, err := proto.Marshal(sch)
|
||||||
|
if err != nil {
|
||||||
|
return err
|
||||||
|
}
|
||||||
|
|
||||||
|
status := C.UpdateSchema(c.ptr, unsafe.Pointer(&schemaBlob[0]), (C.int64_t)(len(schemaBlob)))
|
||||||
|
return ConsumeCStatusIntoError(&status)
|
||||||
|
}
|
||||||
|
|
||||||
// Release releases the underlying collection
|
// Release releases the underlying collection
|
||||||
func (c *CCollection) Release() {
|
func (c *CCollection) Release() {
|
||||||
C.DeleteCollection(c.ptr)
|
C.DeleteCollection(c.ptr)
|
||||||
|
|||||||
@ -152,6 +152,10 @@ func (qn *qnServerWrapper) DeleteBatch(ctx context.Context, in *querypb.DeleteBa
|
|||||||
return qn.QueryNode.DeleteBatch(ctx, in)
|
return qn.QueryNode.DeleteBatch(ctx, in)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func (qn *qnServerWrapper) UpdateSchema(ctx context.Context, in *querypb.UpdateSchemaRequest, _ ...grpc.CallOption) (*commonpb.Status, error) {
|
||||||
|
return qn.QueryNode.UpdateSchema(ctx, in)
|
||||||
|
}
|
||||||
|
|
||||||
func WrapQueryNodeServerAsClient(qn types.QueryNode) types.QueryNodeClient {
|
func WrapQueryNodeServerAsClient(qn types.QueryNode) types.QueryNodeClient {
|
||||||
return &qnServerWrapper{
|
return &qnServerWrapper{
|
||||||
QueryNode: qn,
|
QueryNode: qn,
|
||||||
|
|||||||
@ -4,6 +4,8 @@ package milvus.proto.messages;
|
|||||||
|
|
||||||
option go_package = "github.com/milvus-io/milvus/pkg/v2/proto/messagespb";
|
option go_package = "github.com/milvus-io/milvus/pkg/v2/proto/messagespb";
|
||||||
|
|
||||||
|
import "schema.proto";
|
||||||
|
|
||||||
// MessageID is the unique identifier of a message.
|
// MessageID is the unique identifier of a message.
|
||||||
message MessageID {
|
message MessageID {
|
||||||
string id = 1;
|
string id = 1;
|
||||||
@ -36,6 +38,7 @@ enum MessageType {
|
|||||||
ManualFlush = 9;
|
ManualFlush = 9;
|
||||||
CreateSegment = 10;
|
CreateSegment = 10;
|
||||||
Import = 11;
|
Import = 11;
|
||||||
|
SchemaChange = 12;
|
||||||
// begin transaction message is only used for transaction, once a begin
|
// begin transaction message is only used for transaction, once a begin
|
||||||
// transaction message is received, all messages combined with the
|
// transaction message is received, all messages combined with the
|
||||||
// transaction message cannot be consumed until a CommitTxn message
|
// transaction message cannot be consumed until a CommitTxn message
|
||||||
@ -208,6 +211,17 @@ message TxnMessageHeader {}
|
|||||||
|
|
||||||
message ImportMessageHeader {}
|
message ImportMessageHeader {}
|
||||||
|
|
||||||
|
// SchemaChangeMessageHeader is the header of CollectionSchema update message.
|
||||||
|
message SchemaChangeMessageHeader{
|
||||||
|
int64 collection_id = 1;
|
||||||
|
}
|
||||||
|
|
||||||
|
// SchemaChangeMessageBody is the body of CollectionSchema update message.
|
||||||
|
message SchemaChangeMessageBody{
|
||||||
|
schema.CollectionSchema schema = 1;
|
||||||
|
uint64 modify_ts = 2;
|
||||||
|
}
|
||||||
|
|
||||||
///
|
///
|
||||||
/// Message Extra Response
|
/// Message Extra Response
|
||||||
/// Used to add extra information when response to the client.
|
/// Used to add extra information when response to the client.
|
||||||
|
|||||||
@ -7,6 +7,7 @@
|
|||||||
package messagespb
|
package messagespb
|
||||||
|
|
||||||
import (
|
import (
|
||||||
|
schemapb "github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
|
||||||
protoreflect "google.golang.org/protobuf/reflect/protoreflect"
|
protoreflect "google.golang.org/protobuf/reflect/protoreflect"
|
||||||
protoimpl "google.golang.org/protobuf/runtime/protoimpl"
|
protoimpl "google.golang.org/protobuf/runtime/protoimpl"
|
||||||
reflect "reflect"
|
reflect "reflect"
|
||||||
@ -36,6 +37,7 @@ const (
|
|||||||
MessageType_ManualFlush MessageType = 9
|
MessageType_ManualFlush MessageType = 9
|
||||||
MessageType_CreateSegment MessageType = 10
|
MessageType_CreateSegment MessageType = 10
|
||||||
MessageType_Import MessageType = 11
|
MessageType_Import MessageType = 11
|
||||||
|
MessageType_SchemaChange MessageType = 12
|
||||||
// begin transaction message is only used for transaction, once a begin
|
// begin transaction message is only used for transaction, once a begin
|
||||||
// transaction message is received, all messages combined with the
|
// transaction message is received, all messages combined with the
|
||||||
// transaction message cannot be consumed until a CommitTxn message
|
// transaction message cannot be consumed until a CommitTxn message
|
||||||
@ -74,6 +76,7 @@ var (
|
|||||||
9: "ManualFlush",
|
9: "ManualFlush",
|
||||||
10: "CreateSegment",
|
10: "CreateSegment",
|
||||||
11: "Import",
|
11: "Import",
|
||||||
|
12: "SchemaChange",
|
||||||
900: "BeginTxn",
|
900: "BeginTxn",
|
||||||
901: "CommitTxn",
|
901: "CommitTxn",
|
||||||
902: "RollbackTxn",
|
902: "RollbackTxn",
|
||||||
@ -92,6 +95,7 @@ var (
|
|||||||
"ManualFlush": 9,
|
"ManualFlush": 9,
|
||||||
"CreateSegment": 10,
|
"CreateSegment": 10,
|
||||||
"Import": 11,
|
"Import": 11,
|
||||||
|
"SchemaChange": 12,
|
||||||
"BeginTxn": 900,
|
"BeginTxn": 900,
|
||||||
"CommitTxn": 901,
|
"CommitTxn": 901,
|
||||||
"RollbackTxn": 902,
|
"RollbackTxn": 902,
|
||||||
@ -1623,6 +1627,110 @@ func (*ImportMessageHeader) Descriptor() ([]byte, []int) {
|
|||||||
return file_messages_proto_rawDescGZIP(), []int{27}
|
return file_messages_proto_rawDescGZIP(), []int{27}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// SchemaChangeMessageHeader is the header of CollectionSchema update message.
|
||||||
|
type SchemaChangeMessageHeader struct {
|
||||||
|
state protoimpl.MessageState
|
||||||
|
sizeCache protoimpl.SizeCache
|
||||||
|
unknownFields protoimpl.UnknownFields
|
||||||
|
|
||||||
|
CollectionId int64 `protobuf:"varint,1,opt,name=collection_id,json=collectionId,proto3" json:"collection_id,omitempty"`
|
||||||
|
}
|
||||||
|
|
||||||
|
func (x *SchemaChangeMessageHeader) Reset() {
|
||||||
|
*x = SchemaChangeMessageHeader{}
|
||||||
|
if protoimpl.UnsafeEnabled {
|
||||||
|
mi := &file_messages_proto_msgTypes[28]
|
||||||
|
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||||
|
ms.StoreMessageInfo(mi)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
func (x *SchemaChangeMessageHeader) String() string {
|
||||||
|
return protoimpl.X.MessageStringOf(x)
|
||||||
|
}
|
||||||
|
|
||||||
|
func (*SchemaChangeMessageHeader) ProtoMessage() {}
|
||||||
|
|
||||||
|
func (x *SchemaChangeMessageHeader) ProtoReflect() protoreflect.Message {
|
||||||
|
mi := &file_messages_proto_msgTypes[28]
|
||||||
|
if protoimpl.UnsafeEnabled && x != nil {
|
||||||
|
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||||
|
if ms.LoadMessageInfo() == nil {
|
||||||
|
ms.StoreMessageInfo(mi)
|
||||||
|
}
|
||||||
|
return ms
|
||||||
|
}
|
||||||
|
return mi.MessageOf(x)
|
||||||
|
}
|
||||||
|
|
||||||
|
// Deprecated: Use SchemaChangeMessageHeader.ProtoReflect.Descriptor instead.
|
||||||
|
func (*SchemaChangeMessageHeader) Descriptor() ([]byte, []int) {
|
||||||
|
return file_messages_proto_rawDescGZIP(), []int{28}
|
||||||
|
}
|
||||||
|
|
||||||
|
func (x *SchemaChangeMessageHeader) GetCollectionId() int64 {
|
||||||
|
if x != nil {
|
||||||
|
return x.CollectionId
|
||||||
|
}
|
||||||
|
return 0
|
||||||
|
}
|
||||||
|
|
||||||
|
// SchemaChangeMessageBody is the body of CollectionSchema update message.
|
||||||
|
type SchemaChangeMessageBody struct {
|
||||||
|
state protoimpl.MessageState
|
||||||
|
sizeCache protoimpl.SizeCache
|
||||||
|
unknownFields protoimpl.UnknownFields
|
||||||
|
|
||||||
|
Schema *schemapb.CollectionSchema `protobuf:"bytes,1,opt,name=schema,proto3" json:"schema,omitempty"`
|
||||||
|
ModifyTs uint64 `protobuf:"varint,2,opt,name=modify_ts,json=modifyTs,proto3" json:"modify_ts,omitempty"`
|
||||||
|
}
|
||||||
|
|
||||||
|
func (x *SchemaChangeMessageBody) Reset() {
|
||||||
|
*x = SchemaChangeMessageBody{}
|
||||||
|
if protoimpl.UnsafeEnabled {
|
||||||
|
mi := &file_messages_proto_msgTypes[29]
|
||||||
|
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||||
|
ms.StoreMessageInfo(mi)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
func (x *SchemaChangeMessageBody) String() string {
|
||||||
|
return protoimpl.X.MessageStringOf(x)
|
||||||
|
}
|
||||||
|
|
||||||
|
func (*SchemaChangeMessageBody) ProtoMessage() {}
|
||||||
|
|
||||||
|
func (x *SchemaChangeMessageBody) ProtoReflect() protoreflect.Message {
|
||||||
|
mi := &file_messages_proto_msgTypes[29]
|
||||||
|
if protoimpl.UnsafeEnabled && x != nil {
|
||||||
|
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||||
|
if ms.LoadMessageInfo() == nil {
|
||||||
|
ms.StoreMessageInfo(mi)
|
||||||
|
}
|
||||||
|
return ms
|
||||||
|
}
|
||||||
|
return mi.MessageOf(x)
|
||||||
|
}
|
||||||
|
|
||||||
|
// Deprecated: Use SchemaChangeMessageBody.ProtoReflect.Descriptor instead.
|
||||||
|
func (*SchemaChangeMessageBody) Descriptor() ([]byte, []int) {
|
||||||
|
return file_messages_proto_rawDescGZIP(), []int{29}
|
||||||
|
}
|
||||||
|
|
||||||
|
func (x *SchemaChangeMessageBody) GetSchema() *schemapb.CollectionSchema {
|
||||||
|
if x != nil {
|
||||||
|
return x.Schema
|
||||||
|
}
|
||||||
|
return nil
|
||||||
|
}
|
||||||
|
|
||||||
|
func (x *SchemaChangeMessageBody) GetModifyTs() uint64 {
|
||||||
|
if x != nil {
|
||||||
|
return x.ModifyTs
|
||||||
|
}
|
||||||
|
return 0
|
||||||
|
}
|
||||||
|
|
||||||
// ManualFlushExtraResponse is the extra response of manual flush message.
|
// ManualFlushExtraResponse is the extra response of manual flush message.
|
||||||
type ManualFlushExtraResponse struct {
|
type ManualFlushExtraResponse struct {
|
||||||
state protoimpl.MessageState
|
state protoimpl.MessageState
|
||||||
@ -1635,7 +1743,7 @@ type ManualFlushExtraResponse struct {
|
|||||||
func (x *ManualFlushExtraResponse) Reset() {
|
func (x *ManualFlushExtraResponse) Reset() {
|
||||||
*x = ManualFlushExtraResponse{}
|
*x = ManualFlushExtraResponse{}
|
||||||
if protoimpl.UnsafeEnabled {
|
if protoimpl.UnsafeEnabled {
|
||||||
mi := &file_messages_proto_msgTypes[28]
|
mi := &file_messages_proto_msgTypes[30]
|
||||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||||
ms.StoreMessageInfo(mi)
|
ms.StoreMessageInfo(mi)
|
||||||
}
|
}
|
||||||
@ -1648,7 +1756,7 @@ func (x *ManualFlushExtraResponse) String() string {
|
|||||||
func (*ManualFlushExtraResponse) ProtoMessage() {}
|
func (*ManualFlushExtraResponse) ProtoMessage() {}
|
||||||
|
|
||||||
func (x *ManualFlushExtraResponse) ProtoReflect() protoreflect.Message {
|
func (x *ManualFlushExtraResponse) ProtoReflect() protoreflect.Message {
|
||||||
mi := &file_messages_proto_msgTypes[28]
|
mi := &file_messages_proto_msgTypes[30]
|
||||||
if protoimpl.UnsafeEnabled && x != nil {
|
if protoimpl.UnsafeEnabled && x != nil {
|
||||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||||
if ms.LoadMessageInfo() == nil {
|
if ms.LoadMessageInfo() == nil {
|
||||||
@ -1661,7 +1769,7 @@ func (x *ManualFlushExtraResponse) ProtoReflect() protoreflect.Message {
|
|||||||
|
|
||||||
// Deprecated: Use ManualFlushExtraResponse.ProtoReflect.Descriptor instead.
|
// Deprecated: Use ManualFlushExtraResponse.ProtoReflect.Descriptor instead.
|
||||||
func (*ManualFlushExtraResponse) Descriptor() ([]byte, []int) {
|
func (*ManualFlushExtraResponse) Descriptor() ([]byte, []int) {
|
||||||
return file_messages_proto_rawDescGZIP(), []int{28}
|
return file_messages_proto_rawDescGZIP(), []int{30}
|
||||||
}
|
}
|
||||||
|
|
||||||
func (x *ManualFlushExtraResponse) GetSegmentIds() []int64 {
|
func (x *ManualFlushExtraResponse) GetSegmentIds() []int64 {
|
||||||
@ -1689,7 +1797,7 @@ type TxnContext struct {
|
|||||||
func (x *TxnContext) Reset() {
|
func (x *TxnContext) Reset() {
|
||||||
*x = TxnContext{}
|
*x = TxnContext{}
|
||||||
if protoimpl.UnsafeEnabled {
|
if protoimpl.UnsafeEnabled {
|
||||||
mi := &file_messages_proto_msgTypes[29]
|
mi := &file_messages_proto_msgTypes[31]
|
||||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||||
ms.StoreMessageInfo(mi)
|
ms.StoreMessageInfo(mi)
|
||||||
}
|
}
|
||||||
@ -1702,7 +1810,7 @@ func (x *TxnContext) String() string {
|
|||||||
func (*TxnContext) ProtoMessage() {}
|
func (*TxnContext) ProtoMessage() {}
|
||||||
|
|
||||||
func (x *TxnContext) ProtoReflect() protoreflect.Message {
|
func (x *TxnContext) ProtoReflect() protoreflect.Message {
|
||||||
mi := &file_messages_proto_msgTypes[29]
|
mi := &file_messages_proto_msgTypes[31]
|
||||||
if protoimpl.UnsafeEnabled && x != nil {
|
if protoimpl.UnsafeEnabled && x != nil {
|
||||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||||
if ms.LoadMessageInfo() == nil {
|
if ms.LoadMessageInfo() == nil {
|
||||||
@ -1715,7 +1823,7 @@ func (x *TxnContext) ProtoReflect() protoreflect.Message {
|
|||||||
|
|
||||||
// Deprecated: Use TxnContext.ProtoReflect.Descriptor instead.
|
// Deprecated: Use TxnContext.ProtoReflect.Descriptor instead.
|
||||||
func (*TxnContext) Descriptor() ([]byte, []int) {
|
func (*TxnContext) Descriptor() ([]byte, []int) {
|
||||||
return file_messages_proto_rawDescGZIP(), []int{29}
|
return file_messages_proto_rawDescGZIP(), []int{31}
|
||||||
}
|
}
|
||||||
|
|
||||||
func (x *TxnContext) GetTxnId() int64 {
|
func (x *TxnContext) GetTxnId() int64 {
|
||||||
@ -1745,7 +1853,7 @@ type RMQMessageLayout struct {
|
|||||||
func (x *RMQMessageLayout) Reset() {
|
func (x *RMQMessageLayout) Reset() {
|
||||||
*x = RMQMessageLayout{}
|
*x = RMQMessageLayout{}
|
||||||
if protoimpl.UnsafeEnabled {
|
if protoimpl.UnsafeEnabled {
|
||||||
mi := &file_messages_proto_msgTypes[30]
|
mi := &file_messages_proto_msgTypes[32]
|
||||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||||
ms.StoreMessageInfo(mi)
|
ms.StoreMessageInfo(mi)
|
||||||
}
|
}
|
||||||
@ -1758,7 +1866,7 @@ func (x *RMQMessageLayout) String() string {
|
|||||||
func (*RMQMessageLayout) ProtoMessage() {}
|
func (*RMQMessageLayout) ProtoMessage() {}
|
||||||
|
|
||||||
func (x *RMQMessageLayout) ProtoReflect() protoreflect.Message {
|
func (x *RMQMessageLayout) ProtoReflect() protoreflect.Message {
|
||||||
mi := &file_messages_proto_msgTypes[30]
|
mi := &file_messages_proto_msgTypes[32]
|
||||||
if protoimpl.UnsafeEnabled && x != nil {
|
if protoimpl.UnsafeEnabled && x != nil {
|
||||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||||
if ms.LoadMessageInfo() == nil {
|
if ms.LoadMessageInfo() == nil {
|
||||||
@ -1771,7 +1879,7 @@ func (x *RMQMessageLayout) ProtoReflect() protoreflect.Message {
|
|||||||
|
|
||||||
// Deprecated: Use RMQMessageLayout.ProtoReflect.Descriptor instead.
|
// Deprecated: Use RMQMessageLayout.ProtoReflect.Descriptor instead.
|
||||||
func (*RMQMessageLayout) Descriptor() ([]byte, []int) {
|
func (*RMQMessageLayout) Descriptor() ([]byte, []int) {
|
||||||
return file_messages_proto_rawDescGZIP(), []int{30}
|
return file_messages_proto_rawDescGZIP(), []int{32}
|
||||||
}
|
}
|
||||||
|
|
||||||
func (x *RMQMessageLayout) GetPayload() []byte {
|
func (x *RMQMessageLayout) GetPayload() []byte {
|
||||||
@ -1802,7 +1910,7 @@ type BroadcastHeader struct {
|
|||||||
func (x *BroadcastHeader) Reset() {
|
func (x *BroadcastHeader) Reset() {
|
||||||
*x = BroadcastHeader{}
|
*x = BroadcastHeader{}
|
||||||
if protoimpl.UnsafeEnabled {
|
if protoimpl.UnsafeEnabled {
|
||||||
mi := &file_messages_proto_msgTypes[31]
|
mi := &file_messages_proto_msgTypes[33]
|
||||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||||
ms.StoreMessageInfo(mi)
|
ms.StoreMessageInfo(mi)
|
||||||
}
|
}
|
||||||
@ -1815,7 +1923,7 @@ func (x *BroadcastHeader) String() string {
|
|||||||
func (*BroadcastHeader) ProtoMessage() {}
|
func (*BroadcastHeader) ProtoMessage() {}
|
||||||
|
|
||||||
func (x *BroadcastHeader) ProtoReflect() protoreflect.Message {
|
func (x *BroadcastHeader) ProtoReflect() protoreflect.Message {
|
||||||
mi := &file_messages_proto_msgTypes[31]
|
mi := &file_messages_proto_msgTypes[33]
|
||||||
if protoimpl.UnsafeEnabled && x != nil {
|
if protoimpl.UnsafeEnabled && x != nil {
|
||||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||||
if ms.LoadMessageInfo() == nil {
|
if ms.LoadMessageInfo() == nil {
|
||||||
@ -1828,7 +1936,7 @@ func (x *BroadcastHeader) ProtoReflect() protoreflect.Message {
|
|||||||
|
|
||||||
// Deprecated: Use BroadcastHeader.ProtoReflect.Descriptor instead.
|
// Deprecated: Use BroadcastHeader.ProtoReflect.Descriptor instead.
|
||||||
func (*BroadcastHeader) Descriptor() ([]byte, []int) {
|
func (*BroadcastHeader) Descriptor() ([]byte, []int) {
|
||||||
return file_messages_proto_rawDescGZIP(), []int{31}
|
return file_messages_proto_rawDescGZIP(), []int{33}
|
||||||
}
|
}
|
||||||
|
|
||||||
func (x *BroadcastHeader) GetBroadcastId() uint64 {
|
func (x *BroadcastHeader) GetBroadcastId() uint64 {
|
||||||
@ -1867,7 +1975,7 @@ type ResourceKey struct {
|
|||||||
func (x *ResourceKey) Reset() {
|
func (x *ResourceKey) Reset() {
|
||||||
*x = ResourceKey{}
|
*x = ResourceKey{}
|
||||||
if protoimpl.UnsafeEnabled {
|
if protoimpl.UnsafeEnabled {
|
||||||
mi := &file_messages_proto_msgTypes[32]
|
mi := &file_messages_proto_msgTypes[34]
|
||||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||||
ms.StoreMessageInfo(mi)
|
ms.StoreMessageInfo(mi)
|
||||||
}
|
}
|
||||||
@ -1880,7 +1988,7 @@ func (x *ResourceKey) String() string {
|
|||||||
func (*ResourceKey) ProtoMessage() {}
|
func (*ResourceKey) ProtoMessage() {}
|
||||||
|
|
||||||
func (x *ResourceKey) ProtoReflect() protoreflect.Message {
|
func (x *ResourceKey) ProtoReflect() protoreflect.Message {
|
||||||
mi := &file_messages_proto_msgTypes[32]
|
mi := &file_messages_proto_msgTypes[34]
|
||||||
if protoimpl.UnsafeEnabled && x != nil {
|
if protoimpl.UnsafeEnabled && x != nil {
|
||||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||||
if ms.LoadMessageInfo() == nil {
|
if ms.LoadMessageInfo() == nil {
|
||||||
@ -1893,7 +2001,7 @@ func (x *ResourceKey) ProtoReflect() protoreflect.Message {
|
|||||||
|
|
||||||
// Deprecated: Use ResourceKey.ProtoReflect.Descriptor instead.
|
// Deprecated: Use ResourceKey.ProtoReflect.Descriptor instead.
|
||||||
func (*ResourceKey) Descriptor() ([]byte, []int) {
|
func (*ResourceKey) Descriptor() ([]byte, []int) {
|
||||||
return file_messages_proto_rawDescGZIP(), []int{32}
|
return file_messages_proto_rawDescGZIP(), []int{34}
|
||||||
}
|
}
|
||||||
|
|
||||||
func (x *ResourceKey) GetDomain() ResourceDomain {
|
func (x *ResourceKey) GetDomain() ResourceDomain {
|
||||||
@ -1924,7 +2032,7 @@ type CipherHeader struct {
|
|||||||
func (x *CipherHeader) Reset() {
|
func (x *CipherHeader) Reset() {
|
||||||
*x = CipherHeader{}
|
*x = CipherHeader{}
|
||||||
if protoimpl.UnsafeEnabled {
|
if protoimpl.UnsafeEnabled {
|
||||||
mi := &file_messages_proto_msgTypes[33]
|
mi := &file_messages_proto_msgTypes[35]
|
||||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||||
ms.StoreMessageInfo(mi)
|
ms.StoreMessageInfo(mi)
|
||||||
}
|
}
|
||||||
@ -1937,7 +2045,7 @@ func (x *CipherHeader) String() string {
|
|||||||
func (*CipherHeader) ProtoMessage() {}
|
func (*CipherHeader) ProtoMessage() {}
|
||||||
|
|
||||||
func (x *CipherHeader) ProtoReflect() protoreflect.Message {
|
func (x *CipherHeader) ProtoReflect() protoreflect.Message {
|
||||||
mi := &file_messages_proto_msgTypes[33]
|
mi := &file_messages_proto_msgTypes[35]
|
||||||
if protoimpl.UnsafeEnabled && x != nil {
|
if protoimpl.UnsafeEnabled && x != nil {
|
||||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||||
if ms.LoadMessageInfo() == nil {
|
if ms.LoadMessageInfo() == nil {
|
||||||
@ -1950,7 +2058,7 @@ func (x *CipherHeader) ProtoReflect() protoreflect.Message {
|
|||||||
|
|
||||||
// Deprecated: Use CipherHeader.ProtoReflect.Descriptor instead.
|
// Deprecated: Use CipherHeader.ProtoReflect.Descriptor instead.
|
||||||
func (*CipherHeader) Descriptor() ([]byte, []int) {
|
func (*CipherHeader) Descriptor() ([]byte, []int) {
|
||||||
return file_messages_proto_rawDescGZIP(), []int{33}
|
return file_messages_proto_rawDescGZIP(), []int{35}
|
||||||
}
|
}
|
||||||
|
|
||||||
func (x *CipherHeader) GetEzId() int64 {
|
func (x *CipherHeader) GetEzId() int64 {
|
||||||
@ -1979,223 +2087,237 @@ var File_messages_proto protoreflect.FileDescriptor
|
|||||||
var file_messages_proto_rawDesc = []byte{
|
var file_messages_proto_rawDesc = []byte{
|
||||||
0x0a, 0x0e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f,
|
0x0a, 0x0e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f,
|
||||||
0x12, 0x15, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d,
|
0x12, 0x15, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d,
|
||||||
0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x22, 0x1b, 0x0a, 0x09, 0x4d, 0x65, 0x73, 0x73, 0x61,
|
0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x1a, 0x0c, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x2e,
|
||||||
0x67, 0x65, 0x49, 0x44, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09,
|
0x70, 0x72, 0x6f, 0x74, 0x6f, 0x22, 0x1b, 0x0a, 0x09, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65,
|
||||||
0x52, 0x02, 0x69, 0x64, 0x22, 0xb2, 0x01, 0x0a, 0x07, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65,
|
0x49, 0x44, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x02,
|
||||||
0x12, 0x18, 0x0a, 0x07, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28,
|
0x69, 0x64, 0x22, 0xb2, 0x01, 0x0a, 0x07, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x18,
|
||||||
0x0c, 0x52, 0x07, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x12, 0x4e, 0x0a, 0x0a, 0x70, 0x72,
|
0x0a, 0x07, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x52,
|
||||||
0x6f, 0x70, 0x65, 0x72, 0x74, 0x69, 0x65, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x2e,
|
0x07, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x12, 0x4e, 0x0a, 0x0a, 0x70, 0x72, 0x6f, 0x70,
|
||||||
0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x65,
|
0x65, 0x72, 0x74, 0x69, 0x65, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x2e, 0x2e, 0x6d,
|
||||||
0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x2e, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x2e, 0x50,
|
0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x65, 0x73, 0x73,
|
||||||
0x72, 0x6f, 0x70, 0x65, 0x72, 0x74, 0x69, 0x65, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0a,
|
0x61, 0x67, 0x65, 0x73, 0x2e, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x2e, 0x50, 0x72, 0x6f,
|
||||||
0x70, 0x72, 0x6f, 0x70, 0x65, 0x72, 0x74, 0x69, 0x65, 0x73, 0x1a, 0x3d, 0x0a, 0x0f, 0x50, 0x72,
|
0x70, 0x65, 0x72, 0x74, 0x69, 0x65, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0a, 0x70, 0x72,
|
||||||
0x6f, 0x70, 0x65, 0x72, 0x74, 0x69, 0x65, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a,
|
0x6f, 0x70, 0x65, 0x72, 0x74, 0x69, 0x65, 0x73, 0x1a, 0x3d, 0x0a, 0x0f, 0x50, 0x72, 0x6f, 0x70,
|
||||||
0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12,
|
0x65, 0x72, 0x74, 0x69, 0x65, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b,
|
||||||
0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05,
|
0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a,
|
||||||
0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0xf6, 0x01, 0x0a, 0x10, 0x49, 0x6d,
|
0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61,
|
||||||
0x6d, 0x75, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x30,
|
0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0xf6, 0x01, 0x0a, 0x10, 0x49, 0x6d, 0x6d, 0x75,
|
||||||
0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x6d, 0x69, 0x6c,
|
0x74, 0x61, 0x62, 0x6c, 0x65, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x30, 0x0a, 0x02,
|
||||||
0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67,
|
0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75,
|
||||||
0x65, 0x73, 0x2e, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x49, 0x44, 0x52, 0x02, 0x69, 0x64,
|
|
||||||
0x12, 0x18, 0x0a, 0x07, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28,
|
|
||||||
0x0c, 0x52, 0x07, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x12, 0x57, 0x0a, 0x0a, 0x70, 0x72,
|
|
||||||
0x6f, 0x70, 0x65, 0x72, 0x74, 0x69, 0x65, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x37,
|
|
||||||
0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x65,
|
|
||||||
0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x2e, 0x49, 0x6d, 0x6d, 0x75, 0x74, 0x61, 0x62, 0x6c, 0x65,
|
|
||||||
0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x2e, 0x50, 0x72, 0x6f, 0x70, 0x65, 0x72, 0x74, 0x69,
|
|
||||||
0x65, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0a, 0x70, 0x72, 0x6f, 0x70, 0x65, 0x72, 0x74,
|
|
||||||
0x69, 0x65, 0x73, 0x1a, 0x3d, 0x0a, 0x0f, 0x50, 0x72, 0x6f, 0x70, 0x65, 0x72, 0x74, 0x69, 0x65,
|
|
||||||
0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20,
|
|
||||||
0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75,
|
|
||||||
0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02,
|
|
||||||
0x38, 0x01, 0x22, 0x56, 0x0a, 0x10, 0x46, 0x6c, 0x75, 0x73, 0x68, 0x4d, 0x65, 0x73, 0x73, 0x61,
|
|
||||||
0x67, 0x65, 0x42, 0x6f, 0x64, 0x79, 0x12, 0x23, 0x0a, 0x0d, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63,
|
|
||||||
0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63,
|
|
||||||
0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x12, 0x1d, 0x0a, 0x0a, 0x73,
|
|
||||||
0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x03, 0x28, 0x03, 0x52,
|
|
||||||
0x09, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x64, 0x22, 0x18, 0x0a, 0x16, 0x4d, 0x61,
|
|
||||||
0x6e, 0x75, 0x61, 0x6c, 0x46, 0x6c, 0x75, 0x73, 0x68, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65,
|
|
||||||
0x42, 0x6f, 0x64, 0x79, 0x22, 0x85, 0x01, 0x0a, 0x18, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x53,
|
|
||||||
0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x42, 0x6f, 0x64,
|
|
||||||
0x79, 0x12, 0x23, 0x0a, 0x0d, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f,
|
|
||||||
0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63,
|
|
||||||
0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x12, 0x44, 0x0a, 0x08, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e,
|
|
||||||
0x74, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x28, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75,
|
|
||||||
0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73,
|
0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73,
|
||||||
0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x6e,
|
0x2e, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x49, 0x44, 0x52, 0x02, 0x69, 0x64, 0x12, 0x18,
|
||||||
0x66, 0x6f, 0x52, 0x08, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x22, 0x7e, 0x0a, 0x11,
|
0x0a, 0x07, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x52,
|
||||||
0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x6e, 0x66,
|
0x07, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x12, 0x57, 0x0a, 0x0a, 0x70, 0x72, 0x6f, 0x70,
|
||||||
0x6f, 0x12, 0x21, 0x0a, 0x0c, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69,
|
0x65, 0x72, 0x74, 0x69, 0x65, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x37, 0x2e, 0x6d,
|
||||||
0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0b, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69,
|
0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x65, 0x73, 0x73,
|
||||||
0x6f, 0x6e, 0x49, 0x64, 0x12, 0x1d, 0x0a, 0x0a, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x5f,
|
0x61, 0x67, 0x65, 0x73, 0x2e, 0x49, 0x6d, 0x6d, 0x75, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x4d, 0x65,
|
||||||
0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e,
|
0x73, 0x73, 0x61, 0x67, 0x65, 0x2e, 0x50, 0x72, 0x6f, 0x70, 0x65, 0x72, 0x74, 0x69, 0x65, 0x73,
|
||||||
0x74, 0x49, 0x64, 0x12, 0x27, 0x0a, 0x0f, 0x73, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x5f, 0x76,
|
0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0a, 0x70, 0x72, 0x6f, 0x70, 0x65, 0x72, 0x74, 0x69, 0x65,
|
||||||
0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0e, 0x73, 0x74,
|
0x73, 0x1a, 0x3d, 0x0a, 0x0f, 0x50, 0x72, 0x6f, 0x70, 0x65, 0x72, 0x74, 0x69, 0x65, 0x73, 0x45,
|
||||||
0x6f, 0x72, 0x61, 0x67, 0x65, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x22, 0x15, 0x0a, 0x13,
|
0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28,
|
||||||
0x42, 0x65, 0x67, 0x69, 0x6e, 0x54, 0x78, 0x6e, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x42,
|
0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18,
|
||||||
0x6f, 0x64, 0x79, 0x22, 0x16, 0x0a, 0x14, 0x43, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x54, 0x78, 0x6e,
|
0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01,
|
||||||
0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x42, 0x6f, 0x64, 0x79, 0x22, 0x18, 0x0a, 0x16, 0x52,
|
0x22, 0x56, 0x0a, 0x10, 0x46, 0x6c, 0x75, 0x73, 0x68, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65,
|
||||||
0x6f, 0x6c, 0x6c, 0x62, 0x61, 0x63, 0x6b, 0x54, 0x78, 0x6e, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67,
|
0x42, 0x6f, 0x64, 0x79, 0x12, 0x23, 0x0a, 0x0d, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69,
|
||||||
0x65, 0x42, 0x6f, 0x64, 0x79, 0x22, 0x4c, 0x0a, 0x0e, 0x54, 0x78, 0x6e, 0x4d, 0x65, 0x73, 0x73,
|
|
||||||
0x61, 0x67, 0x65, 0x42, 0x6f, 0x64, 0x79, 0x12, 0x3a, 0x0a, 0x08, 0x6d, 0x65, 0x73, 0x73, 0x61,
|
|
||||||
0x67, 0x65, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x6d, 0x69, 0x6c, 0x76,
|
|
||||||
0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65,
|
|
||||||
0x73, 0x2e, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x52, 0x08, 0x6d, 0x65, 0x73, 0x73, 0x61,
|
|
||||||
0x67, 0x65, 0x73, 0x22, 0x17, 0x0a, 0x15, 0x54, 0x69, 0x6d, 0x65, 0x54, 0x69, 0x63, 0x6b, 0x4d,
|
|
||||||
0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x22, 0x8d, 0x01, 0x0a,
|
|
||||||
0x13, 0x49, 0x6e, 0x73, 0x65, 0x72, 0x74, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x48, 0x65,
|
|
||||||
0x61, 0x64, 0x65, 0x72, 0x12, 0x23, 0x0a, 0x0d, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69,
|
|
||||||
0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c,
|
0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c,
|
||||||
0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x12, 0x51, 0x0a, 0x0a, 0x70, 0x61, 0x72,
|
0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x12, 0x1d, 0x0a, 0x0a, 0x73, 0x65, 0x67,
|
||||||
0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x31, 0x2e,
|
0x6d, 0x65, 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x03, 0x28, 0x03, 0x52, 0x09, 0x73,
|
||||||
|
0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x64, 0x22, 0x18, 0x0a, 0x16, 0x4d, 0x61, 0x6e, 0x75,
|
||||||
|
0x61, 0x6c, 0x46, 0x6c, 0x75, 0x73, 0x68, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x42, 0x6f,
|
||||||
|
0x64, 0x79, 0x22, 0x85, 0x01, 0x0a, 0x18, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x53, 0x65, 0x67,
|
||||||
|
0x6d, 0x65, 0x6e, 0x74, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x42, 0x6f, 0x64, 0x79, 0x12,
|
||||||
|
0x23, 0x0a, 0x0d, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64,
|
||||||
|
0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69,
|
||||||
|
0x6f, 0x6e, 0x49, 0x64, 0x12, 0x44, 0x0a, 0x08, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73,
|
||||||
|
0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x28, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e,
|
||||||
|
0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x2e, 0x43,
|
||||||
|
0x72, 0x65, 0x61, 0x74, 0x65, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x6e, 0x66, 0x6f,
|
||||||
|
0x52, 0x08, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x22, 0x7e, 0x0a, 0x11, 0x43, 0x72,
|
||||||
|
0x65, 0x61, 0x74, 0x65, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x12,
|
||||||
|
0x21, 0x0a, 0x0c, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18,
|
||||||
|
0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0b, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e,
|
||||||
|
0x49, 0x64, 0x12, 0x1d, 0x0a, 0x0a, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x5f, 0x69, 0x64,
|
||||||
|
0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49,
|
||||||
|
0x64, 0x12, 0x27, 0x0a, 0x0f, 0x73, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x5f, 0x76, 0x65, 0x72,
|
||||||
|
0x73, 0x69, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0e, 0x73, 0x74, 0x6f, 0x72,
|
||||||
|
0x61, 0x67, 0x65, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x22, 0x15, 0x0a, 0x13, 0x42, 0x65,
|
||||||
|
0x67, 0x69, 0x6e, 0x54, 0x78, 0x6e, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x42, 0x6f, 0x64,
|
||||||
|
0x79, 0x22, 0x16, 0x0a, 0x14, 0x43, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x54, 0x78, 0x6e, 0x4d, 0x65,
|
||||||
|
0x73, 0x73, 0x61, 0x67, 0x65, 0x42, 0x6f, 0x64, 0x79, 0x22, 0x18, 0x0a, 0x16, 0x52, 0x6f, 0x6c,
|
||||||
|
0x6c, 0x62, 0x61, 0x63, 0x6b, 0x54, 0x78, 0x6e, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x42,
|
||||||
|
0x6f, 0x64, 0x79, 0x22, 0x4c, 0x0a, 0x0e, 0x54, 0x78, 0x6e, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67,
|
||||||
|
0x65, 0x42, 0x6f, 0x64, 0x79, 0x12, 0x3a, 0x0a, 0x08, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65,
|
||||||
|
0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73,
|
||||||
|
0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x2e,
|
||||||
|
0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x52, 0x08, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65,
|
||||||
|
0x73, 0x22, 0x17, 0x0a, 0x15, 0x54, 0x69, 0x6d, 0x65, 0x54, 0x69, 0x63, 0x6b, 0x4d, 0x65, 0x73,
|
||||||
|
0x73, 0x61, 0x67, 0x65, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x22, 0x8d, 0x01, 0x0a, 0x13, 0x49,
|
||||||
|
0x6e, 0x73, 0x65, 0x72, 0x74, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x48, 0x65, 0x61, 0x64,
|
||||||
|
0x65, 0x72, 0x12, 0x23, 0x0a, 0x0d, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e,
|
||||||
|
0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65,
|
||||||
|
0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x12, 0x51, 0x0a, 0x0a, 0x70, 0x61, 0x72, 0x74, 0x69,
|
||||||
|
0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x31, 0x2e, 0x6d, 0x69,
|
||||||
|
0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61,
|
||||||
|
0x67, 0x65, 0x73, 0x2e, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x65, 0x67,
|
||||||
|
0x6d, 0x65, 0x6e, 0x74, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x52, 0x0a,
|
||||||
|
0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x22, 0xcd, 0x01, 0x0a, 0x1a, 0x50,
|
||||||
|
0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x41,
|
||||||
|
0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x12, 0x21, 0x0a, 0x0c, 0x70, 0x61, 0x72,
|
||||||
|
0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52,
|
||||||
|
0x0b, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x12, 0x12, 0x0a, 0x04,
|
||||||
|
0x72, 0x6f, 0x77, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 0x52, 0x04, 0x72, 0x6f, 0x77, 0x73,
|
||||||
|
0x12, 0x1f, 0x0a, 0x0b, 0x62, 0x69, 0x6e, 0x61, 0x72, 0x79, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x18,
|
||||||
|
0x03, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0a, 0x62, 0x69, 0x6e, 0x61, 0x72, 0x79, 0x53, 0x69, 0x7a,
|
||||||
|
0x65, 0x12, 0x57, 0x0a, 0x12, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x5f, 0x61, 0x73, 0x73,
|
||||||
|
0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x28, 0x2e,
|
||||||
0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x65, 0x73,
|
0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x65, 0x73,
|
||||||
0x73, 0x61, 0x67, 0x65, 0x73, 0x2e, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x53,
|
0x73, 0x61, 0x67, 0x65, 0x73, 0x2e, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x41, 0x73, 0x73,
|
||||||
0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74,
|
0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x52, 0x11, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74,
|
||||||
0x52, 0x0a, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x22, 0xcd, 0x01, 0x0a,
|
0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x22, 0x32, 0x0a, 0x11, 0x53, 0x65,
|
||||||
0x1a, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e,
|
0x67, 0x6d, 0x65, 0x6e, 0x74, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x12,
|
||||||
0x74, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x12, 0x21, 0x0a, 0x0c, 0x70,
|
0x1d, 0x0a, 0x0a, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20,
|
||||||
0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28,
|
0x01, 0x28, 0x03, 0x52, 0x09, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x64, 0x22, 0x3a,
|
||||||
0x03, 0x52, 0x0b, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x12, 0x12,
|
0x0a, 0x13, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x48,
|
||||||
0x0a, 0x04, 0x72, 0x6f, 0x77, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 0x52, 0x04, 0x72, 0x6f,
|
0x65, 0x61, 0x64, 0x65, 0x72, 0x12, 0x23, 0x0a, 0x0d, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74,
|
||||||
0x77, 0x73, 0x12, 0x1f, 0x0a, 0x0b, 0x62, 0x69, 0x6e, 0x61, 0x72, 0x79, 0x5f, 0x73, 0x69, 0x7a,
|
0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f,
|
||||||
0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0a, 0x62, 0x69, 0x6e, 0x61, 0x72, 0x79, 0x53,
|
0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x22, 0x14, 0x0a, 0x12, 0x46, 0x6c,
|
||||||
0x69, 0x7a, 0x65, 0x12, 0x57, 0x0a, 0x12, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x5f, 0x61,
|
0x75, 0x73, 0x68, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72,
|
||||||
0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32,
|
0x22, 0x1c, 0x0a, 0x1a, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e,
|
||||||
0x28, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d,
|
0x74, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x22, 0x5a,
|
||||||
0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x2e, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x41,
|
0x0a, 0x18, 0x4d, 0x61, 0x6e, 0x75, 0x61, 0x6c, 0x46, 0x6c, 0x75, 0x73, 0x68, 0x4d, 0x65, 0x73,
|
||||||
0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x52, 0x11, 0x73, 0x65, 0x67, 0x6d, 0x65,
|
0x73, 0x61, 0x67, 0x65, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x12, 0x23, 0x0a, 0x0d, 0x63, 0x6f,
|
||||||
0x6e, 0x74, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x22, 0x32, 0x0a, 0x11,
|
0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28,
|
||||||
0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e,
|
0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x12,
|
||||||
0x74, 0x12, 0x1d, 0x0a, 0x0a, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x18,
|
0x19, 0x0a, 0x08, 0x66, 0x6c, 0x75, 0x73, 0x68, 0x5f, 0x74, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28,
|
||||||
0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x64,
|
0x04, 0x52, 0x07, 0x66, 0x6c, 0x75, 0x73, 0x68, 0x54, 0x73, 0x22, 0x69, 0x0a, 0x1d, 0x43, 0x72,
|
||||||
0x22, 0x3a, 0x0a, 0x13, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67,
|
0x65, 0x61, 0x74, 0x65, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x4d, 0x65,
|
||||||
0x65, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x12, 0x23, 0x0a, 0x0d, 0x63, 0x6f, 0x6c, 0x6c, 0x65,
|
|
||||||
0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c,
|
|
||||||
0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x22, 0x14, 0x0a, 0x12,
|
|
||||||
0x46, 0x6c, 0x75, 0x73, 0x68, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x48, 0x65, 0x61, 0x64,
|
|
||||||
0x65, 0x72, 0x22, 0x1c, 0x0a, 0x1a, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x53, 0x65, 0x67, 0x6d,
|
|
||||||
0x65, 0x6e, 0x74, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72,
|
|
||||||
0x22, 0x5a, 0x0a, 0x18, 0x4d, 0x61, 0x6e, 0x75, 0x61, 0x6c, 0x46, 0x6c, 0x75, 0x73, 0x68, 0x4d,
|
|
||||||
0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x12, 0x23, 0x0a, 0x0d,
|
|
||||||
0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20,
|
|
||||||
0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49,
|
|
||||||
0x64, 0x12, 0x19, 0x0a, 0x08, 0x66, 0x6c, 0x75, 0x73, 0x68, 0x5f, 0x74, 0x73, 0x18, 0x02, 0x20,
|
|
||||||
0x01, 0x28, 0x04, 0x52, 0x07, 0x66, 0x6c, 0x75, 0x73, 0x68, 0x54, 0x73, 0x22, 0x69, 0x0a, 0x1d,
|
|
||||||
0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e,
|
|
||||||
0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x12, 0x23, 0x0a,
|
|
||||||
0x0d, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x01,
|
|
||||||
0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e,
|
|
||||||
0x49, 0x64, 0x12, 0x23, 0x0a, 0x0d, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x5f,
|
|
||||||
0x69, 0x64, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x03, 0x52, 0x0c, 0x70, 0x61, 0x72, 0x74, 0x69,
|
|
||||||
0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x73, 0x22, 0x42, 0x0a, 0x1b, 0x44, 0x72, 0x6f, 0x70, 0x43,
|
|
||||||
0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65,
|
|
||||||
0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x12, 0x23, 0x0a, 0x0d, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63,
|
|
||||||
0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63,
|
|
||||||
0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x22, 0x66, 0x0a, 0x1c, 0x43,
|
|
||||||
0x72, 0x65, 0x61, 0x74, 0x65, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x4d, 0x65,
|
|
||||||
0x73, 0x73, 0x61, 0x67, 0x65, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x12, 0x23, 0x0a, 0x0d, 0x63,
|
0x73, 0x73, 0x61, 0x67, 0x65, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x12, 0x23, 0x0a, 0x0d, 0x63,
|
||||||
0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01,
|
0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01,
|
||||||
0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64,
|
0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64,
|
||||||
0x12, 0x21, 0x0a, 0x0c, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64,
|
0x12, 0x23, 0x0a, 0x0d, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64,
|
||||||
0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0b, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f,
|
0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x03, 0x52, 0x0c, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69,
|
||||||
0x6e, 0x49, 0x64, 0x22, 0x64, 0x0a, 0x1a, 0x44, 0x72, 0x6f, 0x70, 0x50, 0x61, 0x72, 0x74, 0x69,
|
0x6f, 0x6e, 0x49, 0x64, 0x73, 0x22, 0x42, 0x0a, 0x1b, 0x44, 0x72, 0x6f, 0x70, 0x43, 0x6f, 0x6c,
|
||||||
0x74, 0x69, 0x6f, 0x6e, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x48, 0x65, 0x61, 0x64, 0x65,
|
0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x48, 0x65,
|
||||||
0x72, 0x12, 0x23, 0x0a, 0x0d, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f,
|
0x61, 0x64, 0x65, 0x72, 0x12, 0x23, 0x0a, 0x0d, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69,
|
||||||
0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63,
|
0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c,
|
||||||
0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x12, 0x21, 0x0a, 0x0c, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74,
|
0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x22, 0x66, 0x0a, 0x1c, 0x43, 0x72, 0x65,
|
||||||
0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0b, 0x70, 0x61,
|
0x61, 0x74, 0x65, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x4d, 0x65, 0x73, 0x73,
|
||||||
0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x22, 0x4e, 0x0a, 0x15, 0x42, 0x65, 0x67,
|
0x61, 0x67, 0x65, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x12, 0x23, 0x0a, 0x0d, 0x63, 0x6f, 0x6c,
|
||||||
0x69, 0x6e, 0x54, 0x78, 0x6e, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x48, 0x65, 0x61, 0x64,
|
0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03,
|
||||||
0x65, 0x72, 0x12, 0x35, 0x0a, 0x16, 0x6b, 0x65, 0x65, 0x70, 0x61, 0x6c, 0x69, 0x76, 0x65, 0x5f,
|
0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x12, 0x21,
|
||||||
0x6d, 0x69, 0x6c, 0x6c, 0x69, 0x73, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x18, 0x01, 0x20, 0x01,
|
0x0a, 0x0c, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x02,
|
||||||
0x28, 0x03, 0x52, 0x15, 0x6b, 0x65, 0x65, 0x70, 0x61, 0x6c, 0x69, 0x76, 0x65, 0x4d, 0x69, 0x6c,
|
0x20, 0x01, 0x28, 0x03, 0x52, 0x0b, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49,
|
||||||
0x6c, 0x69, 0x73, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x22, 0x18, 0x0a, 0x16, 0x43, 0x6f, 0x6d,
|
0x64, 0x22, 0x64, 0x0a, 0x1a, 0x44, 0x72, 0x6f, 0x70, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69,
|
||||||
0x6d, 0x69, 0x74, 0x54, 0x78, 0x6e, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x48, 0x65, 0x61,
|
0x6f, 0x6e, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x12,
|
||||||
0x64, 0x65, 0x72, 0x22, 0x1a, 0x0a, 0x18, 0x52, 0x6f, 0x6c, 0x6c, 0x62, 0x61, 0x63, 0x6b, 0x54,
|
0x23, 0x0a, 0x0d, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64,
|
||||||
0x78, 0x6e, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x22,
|
0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69,
|
||||||
0x12, 0x0a, 0x10, 0x54, 0x78, 0x6e, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x48, 0x65, 0x61,
|
0x6f, 0x6e, 0x49, 0x64, 0x12, 0x21, 0x0a, 0x0c, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f,
|
||||||
0x64, 0x65, 0x72, 0x22, 0x15, 0x0a, 0x13, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x4d, 0x65, 0x73,
|
0x6e, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0b, 0x70, 0x61, 0x72, 0x74,
|
||||||
0x73, 0x61, 0x67, 0x65, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x22, 0x3b, 0x0a, 0x18, 0x4d, 0x61,
|
0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x22, 0x4e, 0x0a, 0x15, 0x42, 0x65, 0x67, 0x69, 0x6e,
|
||||||
0x6e, 0x75, 0x61, 0x6c, 0x46, 0x6c, 0x75, 0x73, 0x68, 0x45, 0x78, 0x74, 0x72, 0x61, 0x52, 0x65,
|
0x54, 0x78, 0x6e, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72,
|
||||||
0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x1f, 0x0a, 0x0b, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e,
|
0x12, 0x35, 0x0a, 0x16, 0x6b, 0x65, 0x65, 0x70, 0x61, 0x6c, 0x69, 0x76, 0x65, 0x5f, 0x6d, 0x69,
|
||||||
0x74, 0x5f, 0x69, 0x64, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x03, 0x52, 0x0a, 0x73, 0x65, 0x67,
|
0x6c, 0x6c, 0x69, 0x73, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03,
|
||||||
0x6d, 0x65, 0x6e, 0x74, 0x49, 0x64, 0x73, 0x22, 0x5a, 0x0a, 0x0a, 0x54, 0x78, 0x6e, 0x43, 0x6f,
|
0x52, 0x15, 0x6b, 0x65, 0x65, 0x70, 0x61, 0x6c, 0x69, 0x76, 0x65, 0x4d, 0x69, 0x6c, 0x6c, 0x69,
|
||||||
0x6e, 0x74, 0x65, 0x78, 0x74, 0x12, 0x15, 0x0a, 0x06, 0x74, 0x78, 0x6e, 0x5f, 0x69, 0x64, 0x18,
|
0x73, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x22, 0x18, 0x0a, 0x16, 0x43, 0x6f, 0x6d, 0x6d, 0x69,
|
||||||
0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x05, 0x74, 0x78, 0x6e, 0x49, 0x64, 0x12, 0x35, 0x0a, 0x16,
|
0x74, 0x54, 0x78, 0x6e, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x48, 0x65, 0x61, 0x64, 0x65,
|
||||||
0x6b, 0x65, 0x65, 0x70, 0x61, 0x6c, 0x69, 0x76, 0x65, 0x5f, 0x6d, 0x69, 0x6c, 0x6c, 0x69, 0x73,
|
0x72, 0x22, 0x1a, 0x0a, 0x18, 0x52, 0x6f, 0x6c, 0x6c, 0x62, 0x61, 0x63, 0x6b, 0x54, 0x78, 0x6e,
|
||||||
0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x15, 0x6b, 0x65,
|
0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x22, 0x12, 0x0a,
|
||||||
0x65, 0x70, 0x61, 0x6c, 0x69, 0x76, 0x65, 0x4d, 0x69, 0x6c, 0x6c, 0x69, 0x73, 0x65, 0x63, 0x6f,
|
0x10, 0x54, 0x78, 0x6e, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x48, 0x65, 0x61, 0x64, 0x65,
|
||||||
0x6e, 0x64, 0x73, 0x22, 0xc4, 0x01, 0x0a, 0x10, 0x52, 0x4d, 0x51, 0x4d, 0x65, 0x73, 0x73, 0x61,
|
0x72, 0x22, 0x15, 0x0a, 0x13, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x4d, 0x65, 0x73, 0x73, 0x61,
|
||||||
0x67, 0x65, 0x4c, 0x61, 0x79, 0x6f, 0x75, 0x74, 0x12, 0x18, 0x0a, 0x07, 0x70, 0x61, 0x79, 0x6c,
|
0x67, 0x65, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x22, 0x40, 0x0a, 0x19, 0x53, 0x63, 0x68, 0x65,
|
||||||
0x6f, 0x61, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x07, 0x70, 0x61, 0x79, 0x6c, 0x6f,
|
0x6d, 0x61, 0x43, 0x68, 0x61, 0x6e, 0x67, 0x65, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x48,
|
||||||
0x61, 0x64, 0x12, 0x57, 0x0a, 0x0a, 0x70, 0x72, 0x6f, 0x70, 0x65, 0x72, 0x74, 0x69, 0x65, 0x73,
|
0x65, 0x61, 0x64, 0x65, 0x72, 0x12, 0x23, 0x0a, 0x0d, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74,
|
||||||
0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x37, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e,
|
0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f,
|
||||||
0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x2e, 0x52,
|
0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x22, 0x75, 0x0a, 0x17, 0x53, 0x63,
|
||||||
0x4d, 0x51, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x4c, 0x61, 0x79, 0x6f, 0x75, 0x74, 0x2e,
|
0x68, 0x65, 0x6d, 0x61, 0x43, 0x68, 0x61, 0x6e, 0x67, 0x65, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67,
|
||||||
0x50, 0x72, 0x6f, 0x70, 0x65, 0x72, 0x74, 0x69, 0x65, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52,
|
0x65, 0x42, 0x6f, 0x64, 0x79, 0x12, 0x3d, 0x0a, 0x06, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x18,
|
||||||
0x0a, 0x70, 0x72, 0x6f, 0x70, 0x65, 0x72, 0x74, 0x69, 0x65, 0x73, 0x1a, 0x3d, 0x0a, 0x0f, 0x50,
|
0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x25, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70,
|
||||||
0x72, 0x6f, 0x70, 0x65, 0x72, 0x74, 0x69, 0x65, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10,
|
0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x2e, 0x43, 0x6f, 0x6c, 0x6c,
|
||||||
0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79,
|
0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x06, 0x73, 0x63,
|
||||||
0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52,
|
0x68, 0x65, 0x6d, 0x61, 0x12, 0x1b, 0x0a, 0x09, 0x6d, 0x6f, 0x64, 0x69, 0x66, 0x79, 0x5f, 0x74,
|
||||||
0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x9b, 0x01, 0x0a, 0x0f, 0x42,
|
0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 0x52, 0x08, 0x6d, 0x6f, 0x64, 0x69, 0x66, 0x79, 0x54,
|
||||||
0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x12, 0x21,
|
0x73, 0x22, 0x3b, 0x0a, 0x18, 0x4d, 0x61, 0x6e, 0x75, 0x61, 0x6c, 0x46, 0x6c, 0x75, 0x73, 0x68,
|
||||||
0x0a, 0x0c, 0x62, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01,
|
0x45, 0x78, 0x74, 0x72, 0x61, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x1f, 0x0a,
|
||||||
0x20, 0x01, 0x28, 0x04, 0x52, 0x0b, 0x62, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x49,
|
0x0b, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x73, 0x18, 0x01, 0x20, 0x03,
|
||||||
0x64, 0x12, 0x1c, 0x0a, 0x09, 0x76, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x73, 0x18, 0x02,
|
0x28, 0x03, 0x52, 0x0a, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x64, 0x73, 0x22, 0x5a,
|
||||||
0x20, 0x03, 0x28, 0x09, 0x52, 0x09, 0x76, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x73, 0x12,
|
0x0a, 0x0a, 0x54, 0x78, 0x6e, 0x43, 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74, 0x12, 0x15, 0x0a, 0x06,
|
||||||
0x47, 0x0a, 0x0d, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x6b, 0x65, 0x79, 0x73,
|
0x74, 0x78, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x05, 0x74, 0x78,
|
||||||
0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x22, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e,
|
0x6e, 0x49, 0x64, 0x12, 0x35, 0x0a, 0x16, 0x6b, 0x65, 0x65, 0x70, 0x61, 0x6c, 0x69, 0x76, 0x65,
|
||||||
0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x2e, 0x52,
|
0x5f, 0x6d, 0x69, 0x6c, 0x6c, 0x69, 0x73, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x18, 0x02, 0x20,
|
||||||
0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x4b, 0x65, 0x79, 0x52, 0x0c, 0x52, 0x65, 0x73, 0x6f,
|
0x01, 0x28, 0x03, 0x52, 0x15, 0x6b, 0x65, 0x65, 0x70, 0x61, 0x6c, 0x69, 0x76, 0x65, 0x4d, 0x69,
|
||||||
0x75, 0x72, 0x63, 0x65, 0x4b, 0x65, 0x79, 0x73, 0x22, 0x5e, 0x0a, 0x0b, 0x52, 0x65, 0x73, 0x6f,
|
0x6c, 0x6c, 0x69, 0x73, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x22, 0xc4, 0x01, 0x0a, 0x10, 0x52,
|
||||||
0x75, 0x72, 0x63, 0x65, 0x4b, 0x65, 0x79, 0x12, 0x3d, 0x0a, 0x06, 0x64, 0x6f, 0x6d, 0x61, 0x69,
|
0x4d, 0x51, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x4c, 0x61, 0x79, 0x6f, 0x75, 0x74, 0x12,
|
||||||
0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x25, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73,
|
0x18, 0x0a, 0x07, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c,
|
||||||
0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x2e,
|
0x52, 0x07, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x12, 0x57, 0x0a, 0x0a, 0x70, 0x72, 0x6f,
|
||||||
0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x44, 0x6f, 0x6d, 0x61, 0x69, 0x6e, 0x52, 0x06,
|
0x70, 0x65, 0x72, 0x74, 0x69, 0x65, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x37, 0x2e,
|
||||||
0x64, 0x6f, 0x6d, 0x61, 0x69, 0x6e, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x02, 0x20,
|
0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x65, 0x73,
|
||||||
0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x22, 0x63, 0x0a, 0x0c, 0x43, 0x69, 0x70, 0x68,
|
0x73, 0x61, 0x67, 0x65, 0x73, 0x2e, 0x52, 0x4d, 0x51, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65,
|
||||||
0x65, 0x72, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x12, 0x13, 0x0a, 0x05, 0x65, 0x7a, 0x5f, 0x69,
|
0x4c, 0x61, 0x79, 0x6f, 0x75, 0x74, 0x2e, 0x50, 0x72, 0x6f, 0x70, 0x65, 0x72, 0x74, 0x69, 0x65,
|
||||||
0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x04, 0x65, 0x7a, 0x49, 0x64, 0x12, 0x19, 0x0a,
|
0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0a, 0x70, 0x72, 0x6f, 0x70, 0x65, 0x72, 0x74, 0x69,
|
||||||
0x08, 0x73, 0x61, 0x66, 0x65, 0x5f, 0x6b, 0x65, 0x79, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x52,
|
0x65, 0x73, 0x1a, 0x3d, 0x0a, 0x0f, 0x50, 0x72, 0x6f, 0x70, 0x65, 0x72, 0x74, 0x69, 0x65, 0x73,
|
||||||
0x07, 0x73, 0x61, 0x66, 0x65, 0x4b, 0x65, 0x79, 0x12, 0x23, 0x0a, 0x0d, 0x70, 0x61, 0x79, 0x6c,
|
0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01,
|
||||||
0x6f, 0x61, 0x64, 0x5f, 0x62, 0x79, 0x74, 0x65, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52,
|
0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65,
|
||||||
0x0c, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x42, 0x79, 0x74, 0x65, 0x73, 0x2a, 0x88, 0x02,
|
0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38,
|
||||||
0x0a, 0x0b, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x54, 0x79, 0x70, 0x65, 0x12, 0x0b, 0x0a,
|
0x01, 0x22, 0x9b, 0x01, 0x0a, 0x0f, 0x42, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x48,
|
||||||
0x07, 0x55, 0x6e, 0x6b, 0x6e, 0x6f, 0x77, 0x6e, 0x10, 0x00, 0x12, 0x0c, 0x0a, 0x08, 0x54, 0x69,
|
0x65, 0x61, 0x64, 0x65, 0x72, 0x12, 0x21, 0x0a, 0x0c, 0x62, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61,
|
||||||
0x6d, 0x65, 0x54, 0x69, 0x63, 0x6b, 0x10, 0x01, 0x12, 0x0a, 0x0a, 0x06, 0x49, 0x6e, 0x73, 0x65,
|
0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0b, 0x62, 0x72, 0x6f,
|
||||||
0x72, 0x74, 0x10, 0x02, 0x12, 0x0a, 0x0a, 0x06, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x10, 0x03,
|
0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x49, 0x64, 0x12, 0x1c, 0x0a, 0x09, 0x76, 0x63, 0x68, 0x61,
|
||||||
0x12, 0x09, 0x0a, 0x05, 0x46, 0x6c, 0x75, 0x73, 0x68, 0x10, 0x04, 0x12, 0x14, 0x0a, 0x10, 0x43,
|
0x6e, 0x6e, 0x65, 0x6c, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x09, 0x52, 0x09, 0x76, 0x63, 0x68,
|
||||||
0x72, 0x65, 0x61, 0x74, 0x65, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x10,
|
0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x73, 0x12, 0x47, 0x0a, 0x0d, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72,
|
||||||
0x05, 0x12, 0x12, 0x0a, 0x0e, 0x44, 0x72, 0x6f, 0x70, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74,
|
0x63, 0x65, 0x5f, 0x6b, 0x65, 0x79, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x22, 0x2e,
|
||||||
0x69, 0x6f, 0x6e, 0x10, 0x06, 0x12, 0x13, 0x0a, 0x0f, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x50,
|
0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x65, 0x73,
|
||||||
0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x10, 0x07, 0x12, 0x11, 0x0a, 0x0d, 0x44, 0x72,
|
0x73, 0x61, 0x67, 0x65, 0x73, 0x2e, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x4b, 0x65,
|
||||||
0x6f, 0x70, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x10, 0x08, 0x12, 0x0f, 0x0a,
|
0x79, 0x52, 0x0c, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x4b, 0x65, 0x79, 0x73, 0x22,
|
||||||
0x0b, 0x4d, 0x61, 0x6e, 0x75, 0x61, 0x6c, 0x46, 0x6c, 0x75, 0x73, 0x68, 0x10, 0x09, 0x12, 0x11,
|
0x5e, 0x0a, 0x0b, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x4b, 0x65, 0x79, 0x12, 0x3d,
|
||||||
0x0a, 0x0d, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x10,
|
0x0a, 0x06, 0x64, 0x6f, 0x6d, 0x61, 0x69, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x25,
|
||||||
0x0a, 0x12, 0x0a, 0x0a, 0x06, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x10, 0x0b, 0x12, 0x0d, 0x0a,
|
0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x65,
|
||||||
0x08, 0x42, 0x65, 0x67, 0x69, 0x6e, 0x54, 0x78, 0x6e, 0x10, 0x84, 0x07, 0x12, 0x0e, 0x0a, 0x09,
|
0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x2e, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x44,
|
||||||
0x43, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x54, 0x78, 0x6e, 0x10, 0x85, 0x07, 0x12, 0x10, 0x0a, 0x0b,
|
0x6f, 0x6d, 0x61, 0x69, 0x6e, 0x52, 0x06, 0x64, 0x6f, 0x6d, 0x61, 0x69, 0x6e, 0x12, 0x10, 0x0a,
|
||||||
0x52, 0x6f, 0x6c, 0x6c, 0x62, 0x61, 0x63, 0x6b, 0x54, 0x78, 0x6e, 0x10, 0x86, 0x07, 0x12, 0x08,
|
0x03, 0x6b, 0x65, 0x79, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x22,
|
||||||
0x0a, 0x03, 0x54, 0x78, 0x6e, 0x10, 0xe7, 0x07, 0x2a, 0x82, 0x01, 0x0a, 0x08, 0x54, 0x78, 0x6e,
|
0x63, 0x0a, 0x0c, 0x43, 0x69, 0x70, 0x68, 0x65, 0x72, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x12,
|
||||||
0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x0e, 0x0a, 0x0a, 0x54, 0x78, 0x6e, 0x55, 0x6e, 0x6b, 0x6e,
|
0x13, 0x0a, 0x05, 0x65, 0x7a, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x04,
|
||||||
0x6f, 0x77, 0x6e, 0x10, 0x00, 0x12, 0x0c, 0x0a, 0x08, 0x54, 0x78, 0x6e, 0x42, 0x65, 0x67, 0x69,
|
0x65, 0x7a, 0x49, 0x64, 0x12, 0x19, 0x0a, 0x08, 0x73, 0x61, 0x66, 0x65, 0x5f, 0x6b, 0x65, 0x79,
|
||||||
0x6e, 0x10, 0x01, 0x12, 0x0f, 0x0a, 0x0b, 0x54, 0x78, 0x6e, 0x49, 0x6e, 0x46, 0x6c, 0x69, 0x67,
|
0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x07, 0x73, 0x61, 0x66, 0x65, 0x4b, 0x65, 0x79, 0x12,
|
||||||
0x68, 0x74, 0x10, 0x02, 0x12, 0x0f, 0x0a, 0x0b, 0x54, 0x78, 0x6e, 0x4f, 0x6e, 0x43, 0x6f, 0x6d,
|
0x23, 0x0a, 0x0d, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x5f, 0x62, 0x79, 0x74, 0x65, 0x73,
|
||||||
0x6d, 0x69, 0x74, 0x10, 0x03, 0x12, 0x10, 0x0a, 0x0c, 0x54, 0x78, 0x6e, 0x43, 0x6f, 0x6d, 0x6d,
|
0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x42,
|
||||||
0x69, 0x74, 0x74, 0x65, 0x64, 0x10, 0x04, 0x12, 0x11, 0x0a, 0x0d, 0x54, 0x78, 0x6e, 0x4f, 0x6e,
|
0x79, 0x74, 0x65, 0x73, 0x2a, 0x9a, 0x02, 0x0a, 0x0b, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65,
|
||||||
0x52, 0x6f, 0x6c, 0x6c, 0x62, 0x61, 0x63, 0x6b, 0x10, 0x05, 0x12, 0x11, 0x0a, 0x0d, 0x54, 0x78,
|
0x54, 0x79, 0x70, 0x65, 0x12, 0x0b, 0x0a, 0x07, 0x55, 0x6e, 0x6b, 0x6e, 0x6f, 0x77, 0x6e, 0x10,
|
||||||
0x6e, 0x52, 0x6f, 0x6c, 0x6c, 0x62, 0x61, 0x63, 0x6b, 0x65, 0x64, 0x10, 0x06, 0x2a, 0x6c, 0x0a,
|
0x00, 0x12, 0x0c, 0x0a, 0x08, 0x54, 0x69, 0x6d, 0x65, 0x54, 0x69, 0x63, 0x6b, 0x10, 0x01, 0x12,
|
||||||
0x0e, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x44, 0x6f, 0x6d, 0x61, 0x69, 0x6e, 0x12,
|
0x0a, 0x0a, 0x06, 0x49, 0x6e, 0x73, 0x65, 0x72, 0x74, 0x10, 0x02, 0x12, 0x0a, 0x0a, 0x06, 0x44,
|
||||||
0x19, 0x0a, 0x15, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x44, 0x6f, 0x6d, 0x61, 0x69,
|
0x65, 0x6c, 0x65, 0x74, 0x65, 0x10, 0x03, 0x12, 0x09, 0x0a, 0x05, 0x46, 0x6c, 0x75, 0x73, 0x68,
|
||||||
0x6e, 0x55, 0x6e, 0x6b, 0x6e, 0x6f, 0x77, 0x6e, 0x10, 0x00, 0x12, 0x1d, 0x0a, 0x19, 0x52, 0x65,
|
0x10, 0x04, 0x12, 0x14, 0x0a, 0x10, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x43, 0x6f, 0x6c, 0x6c,
|
||||||
0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x44, 0x6f, 0x6d, 0x61, 0x69, 0x6e, 0x49, 0x6d, 0x70, 0x6f,
|
0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x10, 0x05, 0x12, 0x12, 0x0a, 0x0e, 0x44, 0x72, 0x6f, 0x70,
|
||||||
0x72, 0x74, 0x4a, 0x6f, 0x62, 0x49, 0x44, 0x10, 0x01, 0x12, 0x20, 0x0a, 0x1c, 0x52, 0x65, 0x73,
|
0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x10, 0x06, 0x12, 0x13, 0x0a, 0x0f,
|
||||||
0x6f, 0x75, 0x72, 0x63, 0x65, 0x44, 0x6f, 0x6d, 0x61, 0x69, 0x6e, 0x43, 0x6f, 0x6c, 0x6c, 0x65,
|
0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x10,
|
||||||
0x63, 0x74, 0x69, 0x6f, 0x6e, 0x4e, 0x61, 0x6d, 0x65, 0x10, 0x02, 0x42, 0x35, 0x5a, 0x33, 0x67,
|
0x07, 0x12, 0x11, 0x0a, 0x0d, 0x44, 0x72, 0x6f, 0x70, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69,
|
||||||
0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73,
|
0x6f, 0x6e, 0x10, 0x08, 0x12, 0x0f, 0x0a, 0x0b, 0x4d, 0x61, 0x6e, 0x75, 0x61, 0x6c, 0x46, 0x6c,
|
||||||
0x2d, 0x69, 0x6f, 0x2f, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2f, 0x70, 0x6b, 0x67, 0x2f, 0x76,
|
0x75, 0x73, 0x68, 0x10, 0x09, 0x12, 0x11, 0x0a, 0x0d, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x53,
|
||||||
0x32, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73,
|
0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x10, 0x0a, 0x12, 0x0a, 0x0a, 0x06, 0x49, 0x6d, 0x70, 0x6f,
|
||||||
0x70, 0x62, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33,
|
0x72, 0x74, 0x10, 0x0b, 0x12, 0x10, 0x0a, 0x0c, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x43, 0x68,
|
||||||
|
0x61, 0x6e, 0x67, 0x65, 0x10, 0x0c, 0x12, 0x0d, 0x0a, 0x08, 0x42, 0x65, 0x67, 0x69, 0x6e, 0x54,
|
||||||
|
0x78, 0x6e, 0x10, 0x84, 0x07, 0x12, 0x0e, 0x0a, 0x09, 0x43, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x54,
|
||||||
|
0x78, 0x6e, 0x10, 0x85, 0x07, 0x12, 0x10, 0x0a, 0x0b, 0x52, 0x6f, 0x6c, 0x6c, 0x62, 0x61, 0x63,
|
||||||
|
0x6b, 0x54, 0x78, 0x6e, 0x10, 0x86, 0x07, 0x12, 0x08, 0x0a, 0x03, 0x54, 0x78, 0x6e, 0x10, 0xe7,
|
||||||
|
0x07, 0x2a, 0x82, 0x01, 0x0a, 0x08, 0x54, 0x78, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x0e,
|
||||||
|
0x0a, 0x0a, 0x54, 0x78, 0x6e, 0x55, 0x6e, 0x6b, 0x6e, 0x6f, 0x77, 0x6e, 0x10, 0x00, 0x12, 0x0c,
|
||||||
|
0x0a, 0x08, 0x54, 0x78, 0x6e, 0x42, 0x65, 0x67, 0x69, 0x6e, 0x10, 0x01, 0x12, 0x0f, 0x0a, 0x0b,
|
||||||
|
0x54, 0x78, 0x6e, 0x49, 0x6e, 0x46, 0x6c, 0x69, 0x67, 0x68, 0x74, 0x10, 0x02, 0x12, 0x0f, 0x0a,
|
||||||
|
0x0b, 0x54, 0x78, 0x6e, 0x4f, 0x6e, 0x43, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x10, 0x03, 0x12, 0x10,
|
||||||
|
0x0a, 0x0c, 0x54, 0x78, 0x6e, 0x43, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x74, 0x65, 0x64, 0x10, 0x04,
|
||||||
|
0x12, 0x11, 0x0a, 0x0d, 0x54, 0x78, 0x6e, 0x4f, 0x6e, 0x52, 0x6f, 0x6c, 0x6c, 0x62, 0x61, 0x63,
|
||||||
|
0x6b, 0x10, 0x05, 0x12, 0x11, 0x0a, 0x0d, 0x54, 0x78, 0x6e, 0x52, 0x6f, 0x6c, 0x6c, 0x62, 0x61,
|
||||||
|
0x63, 0x6b, 0x65, 0x64, 0x10, 0x06, 0x2a, 0x6c, 0x0a, 0x0e, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72,
|
||||||
|
0x63, 0x65, 0x44, 0x6f, 0x6d, 0x61, 0x69, 0x6e, 0x12, 0x19, 0x0a, 0x15, 0x52, 0x65, 0x73, 0x6f,
|
||||||
|
0x75, 0x72, 0x63, 0x65, 0x44, 0x6f, 0x6d, 0x61, 0x69, 0x6e, 0x55, 0x6e, 0x6b, 0x6e, 0x6f, 0x77,
|
||||||
|
0x6e, 0x10, 0x00, 0x12, 0x1d, 0x0a, 0x19, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x44,
|
||||||
|
0x6f, 0x6d, 0x61, 0x69, 0x6e, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x4a, 0x6f, 0x62, 0x49, 0x44,
|
||||||
|
0x10, 0x01, 0x12, 0x20, 0x0a, 0x1c, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x44, 0x6f,
|
||||||
|
0x6d, 0x61, 0x69, 0x6e, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x4e, 0x61,
|
||||||
|
0x6d, 0x65, 0x10, 0x02, 0x42, 0x35, 0x5a, 0x33, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63,
|
||||||
|
0x6f, 0x6d, 0x2f, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2d, 0x69, 0x6f, 0x2f, 0x6d, 0x69, 0x6c,
|
||||||
|
0x76, 0x75, 0x73, 0x2f, 0x70, 0x6b, 0x67, 0x2f, 0x76, 0x32, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f,
|
||||||
|
0x2f, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x70, 0x62, 0x62, 0x06, 0x70, 0x72, 0x6f,
|
||||||
|
0x74, 0x6f, 0x33,
|
||||||
}
|
}
|
||||||
|
|
||||||
var (
|
var (
|
||||||
@ -2211,7 +2333,7 @@ func file_messages_proto_rawDescGZIP() []byte {
|
|||||||
}
|
}
|
||||||
|
|
||||||
var file_messages_proto_enumTypes = make([]protoimpl.EnumInfo, 3)
|
var file_messages_proto_enumTypes = make([]protoimpl.EnumInfo, 3)
|
||||||
var file_messages_proto_msgTypes = make([]protoimpl.MessageInfo, 37)
|
var file_messages_proto_msgTypes = make([]protoimpl.MessageInfo, 39)
|
||||||
var file_messages_proto_goTypes = []interface{}{
|
var file_messages_proto_goTypes = []interface{}{
|
||||||
(MessageType)(0), // 0: milvus.proto.messages.MessageType
|
(MessageType)(0), // 0: milvus.proto.messages.MessageType
|
||||||
(TxnState)(0), // 1: milvus.proto.messages.TxnState
|
(TxnState)(0), // 1: milvus.proto.messages.TxnState
|
||||||
@ -2244,32 +2366,36 @@ var file_messages_proto_goTypes = []interface{}{
|
|||||||
(*RollbackTxnMessageHeader)(nil), // 28: milvus.proto.messages.RollbackTxnMessageHeader
|
(*RollbackTxnMessageHeader)(nil), // 28: milvus.proto.messages.RollbackTxnMessageHeader
|
||||||
(*TxnMessageHeader)(nil), // 29: milvus.proto.messages.TxnMessageHeader
|
(*TxnMessageHeader)(nil), // 29: milvus.proto.messages.TxnMessageHeader
|
||||||
(*ImportMessageHeader)(nil), // 30: milvus.proto.messages.ImportMessageHeader
|
(*ImportMessageHeader)(nil), // 30: milvus.proto.messages.ImportMessageHeader
|
||||||
(*ManualFlushExtraResponse)(nil), // 31: milvus.proto.messages.ManualFlushExtraResponse
|
(*SchemaChangeMessageHeader)(nil), // 31: milvus.proto.messages.SchemaChangeMessageHeader
|
||||||
(*TxnContext)(nil), // 32: milvus.proto.messages.TxnContext
|
(*SchemaChangeMessageBody)(nil), // 32: milvus.proto.messages.SchemaChangeMessageBody
|
||||||
(*RMQMessageLayout)(nil), // 33: milvus.proto.messages.RMQMessageLayout
|
(*ManualFlushExtraResponse)(nil), // 33: milvus.proto.messages.ManualFlushExtraResponse
|
||||||
(*BroadcastHeader)(nil), // 34: milvus.proto.messages.BroadcastHeader
|
(*TxnContext)(nil), // 34: milvus.proto.messages.TxnContext
|
||||||
(*ResourceKey)(nil), // 35: milvus.proto.messages.ResourceKey
|
(*RMQMessageLayout)(nil), // 35: milvus.proto.messages.RMQMessageLayout
|
||||||
(*CipherHeader)(nil), // 36: milvus.proto.messages.CipherHeader
|
(*BroadcastHeader)(nil), // 36: milvus.proto.messages.BroadcastHeader
|
||||||
nil, // 37: milvus.proto.messages.Message.PropertiesEntry
|
(*ResourceKey)(nil), // 37: milvus.proto.messages.ResourceKey
|
||||||
nil, // 38: milvus.proto.messages.ImmutableMessage.PropertiesEntry
|
(*CipherHeader)(nil), // 38: milvus.proto.messages.CipherHeader
|
||||||
nil, // 39: milvus.proto.messages.RMQMessageLayout.PropertiesEntry
|
nil, // 39: milvus.proto.messages.Message.PropertiesEntry
|
||||||
|
nil, // 40: milvus.proto.messages.ImmutableMessage.PropertiesEntry
|
||||||
|
nil, // 41: milvus.proto.messages.RMQMessageLayout.PropertiesEntry
|
||||||
|
(*schemapb.CollectionSchema)(nil), // 42: milvus.proto.schema.CollectionSchema
|
||||||
}
|
}
|
||||||
var file_messages_proto_depIdxs = []int32{
|
var file_messages_proto_depIdxs = []int32{
|
||||||
37, // 0: milvus.proto.messages.Message.properties:type_name -> milvus.proto.messages.Message.PropertiesEntry
|
39, // 0: milvus.proto.messages.Message.properties:type_name -> milvus.proto.messages.Message.PropertiesEntry
|
||||||
3, // 1: milvus.proto.messages.ImmutableMessage.id:type_name -> milvus.proto.messages.MessageID
|
3, // 1: milvus.proto.messages.ImmutableMessage.id:type_name -> milvus.proto.messages.MessageID
|
||||||
38, // 2: milvus.proto.messages.ImmutableMessage.properties:type_name -> milvus.proto.messages.ImmutableMessage.PropertiesEntry
|
40, // 2: milvus.proto.messages.ImmutableMessage.properties:type_name -> milvus.proto.messages.ImmutableMessage.PropertiesEntry
|
||||||
9, // 3: milvus.proto.messages.CreateSegmentMessageBody.segments:type_name -> milvus.proto.messages.CreateSegmentInfo
|
9, // 3: milvus.proto.messages.CreateSegmentMessageBody.segments:type_name -> milvus.proto.messages.CreateSegmentInfo
|
||||||
4, // 4: milvus.proto.messages.TxnMessageBody.messages:type_name -> milvus.proto.messages.Message
|
4, // 4: milvus.proto.messages.TxnMessageBody.messages:type_name -> milvus.proto.messages.Message
|
||||||
16, // 5: milvus.proto.messages.InsertMessageHeader.partitions:type_name -> milvus.proto.messages.PartitionSegmentAssignment
|
16, // 5: milvus.proto.messages.InsertMessageHeader.partitions:type_name -> milvus.proto.messages.PartitionSegmentAssignment
|
||||||
17, // 6: milvus.proto.messages.PartitionSegmentAssignment.segment_assignment:type_name -> milvus.proto.messages.SegmentAssignment
|
17, // 6: milvus.proto.messages.PartitionSegmentAssignment.segment_assignment:type_name -> milvus.proto.messages.SegmentAssignment
|
||||||
39, // 7: milvus.proto.messages.RMQMessageLayout.properties:type_name -> milvus.proto.messages.RMQMessageLayout.PropertiesEntry
|
42, // 7: milvus.proto.messages.SchemaChangeMessageBody.schema:type_name -> milvus.proto.schema.CollectionSchema
|
||||||
35, // 8: milvus.proto.messages.BroadcastHeader.Resource_keys:type_name -> milvus.proto.messages.ResourceKey
|
41, // 8: milvus.proto.messages.RMQMessageLayout.properties:type_name -> milvus.proto.messages.RMQMessageLayout.PropertiesEntry
|
||||||
2, // 9: milvus.proto.messages.ResourceKey.domain:type_name -> milvus.proto.messages.ResourceDomain
|
37, // 9: milvus.proto.messages.BroadcastHeader.Resource_keys:type_name -> milvus.proto.messages.ResourceKey
|
||||||
10, // [10:10] is the sub-list for method output_type
|
2, // 10: milvus.proto.messages.ResourceKey.domain:type_name -> milvus.proto.messages.ResourceDomain
|
||||||
10, // [10:10] is the sub-list for method input_type
|
11, // [11:11] is the sub-list for method output_type
|
||||||
10, // [10:10] is the sub-list for extension type_name
|
11, // [11:11] is the sub-list for method input_type
|
||||||
10, // [10:10] is the sub-list for extension extendee
|
11, // [11:11] is the sub-list for extension type_name
|
||||||
0, // [0:10] is the sub-list for field type_name
|
11, // [11:11] is the sub-list for extension extendee
|
||||||
|
0, // [0:11] is the sub-list for field type_name
|
||||||
}
|
}
|
||||||
|
|
||||||
func init() { file_messages_proto_init() }
|
func init() { file_messages_proto_init() }
|
||||||
@ -2615,7 +2741,7 @@ func file_messages_proto_init() {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
file_messages_proto_msgTypes[28].Exporter = func(v interface{}, i int) interface{} {
|
file_messages_proto_msgTypes[28].Exporter = func(v interface{}, i int) interface{} {
|
||||||
switch v := v.(*ManualFlushExtraResponse); i {
|
switch v := v.(*SchemaChangeMessageHeader); i {
|
||||||
case 0:
|
case 0:
|
||||||
return &v.state
|
return &v.state
|
||||||
case 1:
|
case 1:
|
||||||
@ -2627,7 +2753,7 @@ func file_messages_proto_init() {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
file_messages_proto_msgTypes[29].Exporter = func(v interface{}, i int) interface{} {
|
file_messages_proto_msgTypes[29].Exporter = func(v interface{}, i int) interface{} {
|
||||||
switch v := v.(*TxnContext); i {
|
switch v := v.(*SchemaChangeMessageBody); i {
|
||||||
case 0:
|
case 0:
|
||||||
return &v.state
|
return &v.state
|
||||||
case 1:
|
case 1:
|
||||||
@ -2639,7 +2765,7 @@ func file_messages_proto_init() {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
file_messages_proto_msgTypes[30].Exporter = func(v interface{}, i int) interface{} {
|
file_messages_proto_msgTypes[30].Exporter = func(v interface{}, i int) interface{} {
|
||||||
switch v := v.(*RMQMessageLayout); i {
|
switch v := v.(*ManualFlushExtraResponse); i {
|
||||||
case 0:
|
case 0:
|
||||||
return &v.state
|
return &v.state
|
||||||
case 1:
|
case 1:
|
||||||
@ -2651,7 +2777,7 @@ func file_messages_proto_init() {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
file_messages_proto_msgTypes[31].Exporter = func(v interface{}, i int) interface{} {
|
file_messages_proto_msgTypes[31].Exporter = func(v interface{}, i int) interface{} {
|
||||||
switch v := v.(*BroadcastHeader); i {
|
switch v := v.(*TxnContext); i {
|
||||||
case 0:
|
case 0:
|
||||||
return &v.state
|
return &v.state
|
||||||
case 1:
|
case 1:
|
||||||
@ -2663,7 +2789,7 @@ func file_messages_proto_init() {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
file_messages_proto_msgTypes[32].Exporter = func(v interface{}, i int) interface{} {
|
file_messages_proto_msgTypes[32].Exporter = func(v interface{}, i int) interface{} {
|
||||||
switch v := v.(*ResourceKey); i {
|
switch v := v.(*RMQMessageLayout); i {
|
||||||
case 0:
|
case 0:
|
||||||
return &v.state
|
return &v.state
|
||||||
case 1:
|
case 1:
|
||||||
@ -2675,6 +2801,30 @@ func file_messages_proto_init() {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
file_messages_proto_msgTypes[33].Exporter = func(v interface{}, i int) interface{} {
|
file_messages_proto_msgTypes[33].Exporter = func(v interface{}, i int) interface{} {
|
||||||
|
switch v := v.(*BroadcastHeader); i {
|
||||||
|
case 0:
|
||||||
|
return &v.state
|
||||||
|
case 1:
|
||||||
|
return &v.sizeCache
|
||||||
|
case 2:
|
||||||
|
return &v.unknownFields
|
||||||
|
default:
|
||||||
|
return nil
|
||||||
|
}
|
||||||
|
}
|
||||||
|
file_messages_proto_msgTypes[34].Exporter = func(v interface{}, i int) interface{} {
|
||||||
|
switch v := v.(*ResourceKey); i {
|
||||||
|
case 0:
|
||||||
|
return &v.state
|
||||||
|
case 1:
|
||||||
|
return &v.sizeCache
|
||||||
|
case 2:
|
||||||
|
return &v.unknownFields
|
||||||
|
default:
|
||||||
|
return nil
|
||||||
|
}
|
||||||
|
}
|
||||||
|
file_messages_proto_msgTypes[35].Exporter = func(v interface{}, i int) interface{} {
|
||||||
switch v := v.(*CipherHeader); i {
|
switch v := v.(*CipherHeader); i {
|
||||||
case 0:
|
case 0:
|
||||||
return &v.state
|
return &v.state
|
||||||
@ -2693,7 +2843,7 @@ func file_messages_proto_init() {
|
|||||||
GoPackagePath: reflect.TypeOf(x{}).PkgPath(),
|
GoPackagePath: reflect.TypeOf(x{}).PkgPath(),
|
||||||
RawDescriptor: file_messages_proto_rawDesc,
|
RawDescriptor: file_messages_proto_rawDesc,
|
||||||
NumEnums: 3,
|
NumEnums: 3,
|
||||||
NumMessages: 37,
|
NumMessages: 39,
|
||||||
NumExtensions: 0,
|
NumExtensions: 0,
|
||||||
NumServices: 0,
|
NumServices: 0,
|
||||||
},
|
},
|
||||||
|
|||||||
@ -167,6 +167,7 @@ service QueryNode {
|
|||||||
// it's basically same as `Delete` but cost less memory pressure.
|
// it's basically same as `Delete` but cost less memory pressure.
|
||||||
rpc DeleteBatch(DeleteBatchRequest) returns (DeleteBatchResponse) {
|
rpc DeleteBatch(DeleteBatchRequest) returns (DeleteBatchResponse) {
|
||||||
}
|
}
|
||||||
|
rpc UpdateSchema(UpdateSchemaRequest) returns (common.Status) {}
|
||||||
}
|
}
|
||||||
|
|
||||||
// --------------------QueryCoord grpc request and response proto------------------
|
// --------------------QueryCoord grpc request and response proto------------------
|
||||||
@ -944,3 +945,9 @@ message UpdateLoadConfigRequest {
|
|||||||
int32 replica_number = 4;
|
int32 replica_number = 4;
|
||||||
repeated string resource_groups = 5;
|
repeated string resource_groups = 5;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
message UpdateSchemaRequest {
|
||||||
|
common.MsgBase base = 1;
|
||||||
|
int64 collectionID = 2;
|
||||||
|
schema.CollectionSchema schema = 3;
|
||||||
|
}
|
||||||
File diff suppressed because it is too large
Load Diff
@ -1436,6 +1436,7 @@ const (
|
|||||||
QueryNode_SyncDistribution_FullMethodName = "/milvus.proto.query.QueryNode/SyncDistribution"
|
QueryNode_SyncDistribution_FullMethodName = "/milvus.proto.query.QueryNode/SyncDistribution"
|
||||||
QueryNode_Delete_FullMethodName = "/milvus.proto.query.QueryNode/Delete"
|
QueryNode_Delete_FullMethodName = "/milvus.proto.query.QueryNode/Delete"
|
||||||
QueryNode_DeleteBatch_FullMethodName = "/milvus.proto.query.QueryNode/DeleteBatch"
|
QueryNode_DeleteBatch_FullMethodName = "/milvus.proto.query.QueryNode/DeleteBatch"
|
||||||
|
QueryNode_UpdateSchema_FullMethodName = "/milvus.proto.query.QueryNode/UpdateSchema"
|
||||||
)
|
)
|
||||||
|
|
||||||
// QueryNodeClient is the client API for QueryNode service.
|
// QueryNodeClient is the client API for QueryNode service.
|
||||||
@ -1470,6 +1471,7 @@ type QueryNodeClient interface {
|
|||||||
// DeleteBatch is the API to apply same delete data into multiple segments.
|
// DeleteBatch is the API to apply same delete data into multiple segments.
|
||||||
// it's basically same as `Delete` but cost less memory pressure.
|
// it's basically same as `Delete` but cost less memory pressure.
|
||||||
DeleteBatch(ctx context.Context, in *DeleteBatchRequest, opts ...grpc.CallOption) (*DeleteBatchResponse, error)
|
DeleteBatch(ctx context.Context, in *DeleteBatchRequest, opts ...grpc.CallOption) (*DeleteBatchResponse, error)
|
||||||
|
UpdateSchema(ctx context.Context, in *UpdateSchemaRequest, opts ...grpc.CallOption) (*commonpb.Status, error)
|
||||||
}
|
}
|
||||||
|
|
||||||
type queryNodeClient struct {
|
type queryNodeClient struct {
|
||||||
@ -1751,6 +1753,15 @@ func (c *queryNodeClient) DeleteBatch(ctx context.Context, in *DeleteBatchReques
|
|||||||
return out, nil
|
return out, nil
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func (c *queryNodeClient) UpdateSchema(ctx context.Context, in *UpdateSchemaRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
|
||||||
|
out := new(commonpb.Status)
|
||||||
|
err := c.cc.Invoke(ctx, QueryNode_UpdateSchema_FullMethodName, in, out, opts...)
|
||||||
|
if err != nil {
|
||||||
|
return nil, err
|
||||||
|
}
|
||||||
|
return out, nil
|
||||||
|
}
|
||||||
|
|
||||||
// QueryNodeServer is the server API for QueryNode service.
|
// QueryNodeServer is the server API for QueryNode service.
|
||||||
// All implementations should embed UnimplementedQueryNodeServer
|
// All implementations should embed UnimplementedQueryNodeServer
|
||||||
// for forward compatibility
|
// for forward compatibility
|
||||||
@ -1783,6 +1794,7 @@ type QueryNodeServer interface {
|
|||||||
// DeleteBatch is the API to apply same delete data into multiple segments.
|
// DeleteBatch is the API to apply same delete data into multiple segments.
|
||||||
// it's basically same as `Delete` but cost less memory pressure.
|
// it's basically same as `Delete` but cost less memory pressure.
|
||||||
DeleteBatch(context.Context, *DeleteBatchRequest) (*DeleteBatchResponse, error)
|
DeleteBatch(context.Context, *DeleteBatchRequest) (*DeleteBatchResponse, error)
|
||||||
|
UpdateSchema(context.Context, *UpdateSchemaRequest) (*commonpb.Status, error)
|
||||||
}
|
}
|
||||||
|
|
||||||
// UnimplementedQueryNodeServer should be embedded to have forward compatible implementations.
|
// UnimplementedQueryNodeServer should be embedded to have forward compatible implementations.
|
||||||
@ -1864,6 +1876,9 @@ func (UnimplementedQueryNodeServer) Delete(context.Context, *DeleteRequest) (*co
|
|||||||
func (UnimplementedQueryNodeServer) DeleteBatch(context.Context, *DeleteBatchRequest) (*DeleteBatchResponse, error) {
|
func (UnimplementedQueryNodeServer) DeleteBatch(context.Context, *DeleteBatchRequest) (*DeleteBatchResponse, error) {
|
||||||
return nil, status.Errorf(codes.Unimplemented, "method DeleteBatch not implemented")
|
return nil, status.Errorf(codes.Unimplemented, "method DeleteBatch not implemented")
|
||||||
}
|
}
|
||||||
|
func (UnimplementedQueryNodeServer) UpdateSchema(context.Context, *UpdateSchemaRequest) (*commonpb.Status, error) {
|
||||||
|
return nil, status.Errorf(codes.Unimplemented, "method UpdateSchema not implemented")
|
||||||
|
}
|
||||||
|
|
||||||
// UnsafeQueryNodeServer may be embedded to opt out of forward compatibility for this service.
|
// UnsafeQueryNodeServer may be embedded to opt out of forward compatibility for this service.
|
||||||
// Use of this interface is not recommended, as added methods to QueryNodeServer will
|
// Use of this interface is not recommended, as added methods to QueryNodeServer will
|
||||||
@ -2332,6 +2347,24 @@ func _QueryNode_DeleteBatch_Handler(srv interface{}, ctx context.Context, dec fu
|
|||||||
return interceptor(ctx, in, info, handler)
|
return interceptor(ctx, in, info, handler)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func _QueryNode_UpdateSchema_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
|
||||||
|
in := new(UpdateSchemaRequest)
|
||||||
|
if err := dec(in); err != nil {
|
||||||
|
return nil, err
|
||||||
|
}
|
||||||
|
if interceptor == nil {
|
||||||
|
return srv.(QueryNodeServer).UpdateSchema(ctx, in)
|
||||||
|
}
|
||||||
|
info := &grpc.UnaryServerInfo{
|
||||||
|
Server: srv,
|
||||||
|
FullMethod: QueryNode_UpdateSchema_FullMethodName,
|
||||||
|
}
|
||||||
|
handler := func(ctx context.Context, req interface{}) (interface{}, error) {
|
||||||
|
return srv.(QueryNodeServer).UpdateSchema(ctx, req.(*UpdateSchemaRequest))
|
||||||
|
}
|
||||||
|
return interceptor(ctx, in, info, handler)
|
||||||
|
}
|
||||||
|
|
||||||
// QueryNode_ServiceDesc is the grpc.ServiceDesc for QueryNode service.
|
// QueryNode_ServiceDesc is the grpc.ServiceDesc for QueryNode service.
|
||||||
// It's only intended for direct use with grpc.RegisterService,
|
// It's only intended for direct use with grpc.RegisterService,
|
||||||
// and not to be introspected or modified (even as a copy)
|
// and not to be introspected or modified (even as a copy)
|
||||||
@ -2431,6 +2464,10 @@ var QueryNode_ServiceDesc = grpc.ServiceDesc{
|
|||||||
MethodName: "DeleteBatch",
|
MethodName: "DeleteBatch",
|
||||||
Handler: _QueryNode_DeleteBatch_Handler,
|
Handler: _QueryNode_DeleteBatch_Handler,
|
||||||
},
|
},
|
||||||
|
{
|
||||||
|
MethodName: "UpdateSchema",
|
||||||
|
Handler: _QueryNode_UpdateSchema_Handler,
|
||||||
|
},
|
||||||
},
|
},
|
||||||
Streams: []grpc.StreamDesc{
|
Streams: []grpc.StreamDesc{
|
||||||
{
|
{
|
||||||
|
|||||||
@ -135,6 +135,8 @@ func fromMessageToTsMsgV2(msg message.ImmutableMessage) (msgstream.TsMsg, error)
|
|||||||
tsMsg, err = NewManualFlushMessageBody(msg)
|
tsMsg, err = NewManualFlushMessageBody(msg)
|
||||||
case message.MessageTypeCreateSegment:
|
case message.MessageTypeCreateSegment:
|
||||||
tsMsg, err = NewCreateSegmentMessageBody(msg)
|
tsMsg, err = NewCreateSegmentMessageBody(msg)
|
||||||
|
case message.MessageTypeSchemaChange:
|
||||||
|
tsMsg, err = NewSchemaChangeMessageBody(msg)
|
||||||
default:
|
default:
|
||||||
panic("unsupported message type")
|
panic("unsupported message type")
|
||||||
}
|
}
|
||||||
|
|||||||
@ -17,6 +17,7 @@ var messageTypeToCommonpbMsgType = map[message.MessageType]commonpb.MsgType{
|
|||||||
message.MessageTypeCreatePartition: commonpb.MsgType_CreatePartition,
|
message.MessageTypeCreatePartition: commonpb.MsgType_CreatePartition,
|
||||||
message.MessageTypeDropPartition: commonpb.MsgType_DropPartition,
|
message.MessageTypeDropPartition: commonpb.MsgType_DropPartition,
|
||||||
message.MessageTypeImport: commonpb.MsgType_Import,
|
message.MessageTypeImport: commonpb.MsgType_Import,
|
||||||
|
message.MessageTypeSchemaChange: commonpb.MsgType_AddCollectionField, // TODO change to schema change
|
||||||
}
|
}
|
||||||
|
|
||||||
// MustGetCommonpbMsgTypeFromMessageType returns the commonpb.MsgType from message.MessageType.
|
// MustGetCommonpbMsgTypeFromMessageType returns the commonpb.MsgType from message.MessageType.
|
||||||
|
|||||||
@ -121,3 +121,29 @@ func NewManualFlushMessageBody(msg message.ImmutableMessage) (msgstream.TsMsg, e
|
|||||||
ManualFlushMessage: flushMsg,
|
ManualFlushMessage: flushMsg,
|
||||||
}, nil
|
}, nil
|
||||||
}
|
}
|
||||||
|
|
||||||
|
type SchemaChangeMessageBody struct {
|
||||||
|
*tsMsgImpl
|
||||||
|
SchemaChangeMessage message.ImmutableSchemaChangeMessageV2
|
||||||
|
BroadcastID uint64
|
||||||
|
}
|
||||||
|
|
||||||
|
func NewSchemaChangeMessageBody(msg message.ImmutableMessage) (msgstream.TsMsg, error) {
|
||||||
|
schChgMsg, err := message.AsImmutableCollectionSchemaChangeV2(msg)
|
||||||
|
if err != nil {
|
||||||
|
return nil, err
|
||||||
|
}
|
||||||
|
return &SchemaChangeMessageBody{
|
||||||
|
tsMsgImpl: &tsMsgImpl{
|
||||||
|
BaseMsg: msgstream.BaseMsg{
|
||||||
|
BeginTimestamp: msg.TimeTick(),
|
||||||
|
EndTimestamp: msg.TimeTick(),
|
||||||
|
},
|
||||||
|
ts: msg.TimeTick(),
|
||||||
|
sz: msg.EstimateSize(),
|
||||||
|
msgType: MustGetCommonpbMsgTypeFromMessageType(msg.MessageType()),
|
||||||
|
},
|
||||||
|
SchemaChangeMessage: schChgMsg,
|
||||||
|
BroadcastID: msg.BroadcastHeader().BroadcastID,
|
||||||
|
}, nil
|
||||||
|
}
|
||||||
|
|||||||
@ -9,13 +9,13 @@ import (
|
|||||||
|
|
||||||
// newBroadcastHeaderFromProto creates a BroadcastHeader from proto.
|
// newBroadcastHeaderFromProto creates a BroadcastHeader from proto.
|
||||||
func newBroadcastHeaderFromProto(proto *messagespb.BroadcastHeader) *BroadcastHeader {
|
func newBroadcastHeaderFromProto(proto *messagespb.BroadcastHeader) *BroadcastHeader {
|
||||||
rks := make(typeutil.Set[ResourceKey], len(proto.ResourceKeys))
|
rks := make(typeutil.Set[ResourceKey], len(proto.GetResourceKeys()))
|
||||||
for _, key := range proto.ResourceKeys {
|
for _, key := range proto.GetResourceKeys() {
|
||||||
rks.Insert(NewResourceKeyFromProto(key))
|
rks.Insert(NewResourceKeyFromProto(key))
|
||||||
}
|
}
|
||||||
return &BroadcastHeader{
|
return &BroadcastHeader{
|
||||||
BroadcastID: proto.BroadcastId,
|
BroadcastID: proto.GetBroadcastId(),
|
||||||
VChannels: proto.Vchannels,
|
VChannels: proto.GetVchannels(),
|
||||||
ResourceKeys: rks,
|
ResourceKeys: rks,
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|||||||
@ -68,6 +68,7 @@ var (
|
|||||||
NewBeginTxnMessageBuilderV2 = createNewMessageBuilderV2[*BeginTxnMessageHeader, *BeginTxnMessageBody]()
|
NewBeginTxnMessageBuilderV2 = createNewMessageBuilderV2[*BeginTxnMessageHeader, *BeginTxnMessageBody]()
|
||||||
NewCommitTxnMessageBuilderV2 = createNewMessageBuilderV2[*CommitTxnMessageHeader, *CommitTxnMessageBody]()
|
NewCommitTxnMessageBuilderV2 = createNewMessageBuilderV2[*CommitTxnMessageHeader, *CommitTxnMessageBody]()
|
||||||
NewRollbackTxnMessageBuilderV2 = createNewMessageBuilderV2[*RollbackTxnMessageHeader, *RollbackTxnMessageBody]()
|
NewRollbackTxnMessageBuilderV2 = createNewMessageBuilderV2[*RollbackTxnMessageHeader, *RollbackTxnMessageBody]()
|
||||||
|
NewSchemaChangeMessageBuilderV2 = createNewMessageBuilderV2[*SchemaChangeMessageHeader, *SchemaChangeMessageBody]()
|
||||||
newTxnMessageBuilderV2 = createNewMessageBuilderV2[*TxnMessageHeader, *TxnMessageBody]()
|
newTxnMessageBuilderV2 = createNewMessageBuilderV2[*TxnMessageHeader, *TxnMessageBody]()
|
||||||
)
|
)
|
||||||
|
|
||||||
|
|||||||
@ -25,6 +25,7 @@ const (
|
|||||||
MessageTypeCommitTxn MessageType = MessageType(messagespb.MessageType_CommitTxn)
|
MessageTypeCommitTxn MessageType = MessageType(messagespb.MessageType_CommitTxn)
|
||||||
MessageTypeRollbackTxn MessageType = MessageType(messagespb.MessageType_RollbackTxn)
|
MessageTypeRollbackTxn MessageType = MessageType(messagespb.MessageType_RollbackTxn)
|
||||||
MessageTypeImport MessageType = MessageType(messagespb.MessageType_Import)
|
MessageTypeImport MessageType = MessageType(messagespb.MessageType_Import)
|
||||||
|
MessageTypeSchemaChange MessageType = MessageType(messagespb.MessageType_SchemaChange)
|
||||||
)
|
)
|
||||||
|
|
||||||
var messageTypeName = map[MessageType]string{
|
var messageTypeName = map[MessageType]string{
|
||||||
@ -44,6 +45,7 @@ var messageTypeName = map[MessageType]string{
|
|||||||
MessageTypeCommitTxn: "COMMIT_TXN",
|
MessageTypeCommitTxn: "COMMIT_TXN",
|
||||||
MessageTypeRollbackTxn: "ROLLBACK_TXN",
|
MessageTypeRollbackTxn: "ROLLBACK_TXN",
|
||||||
MessageTypeImport: "IMPORT",
|
MessageTypeImport: "IMPORT",
|
||||||
|
MessageTypeSchemaChange: "SCHEMA_CHANGE",
|
||||||
}
|
}
|
||||||
|
|
||||||
// String implements fmt.Stringer interface.
|
// String implements fmt.Stringer interface.
|
||||||
|
|||||||
@ -29,6 +29,7 @@ type (
|
|||||||
RollbackTxnMessageHeader = messagespb.RollbackTxnMessageHeader
|
RollbackTxnMessageHeader = messagespb.RollbackTxnMessageHeader
|
||||||
TxnMessageHeader = messagespb.TxnMessageHeader
|
TxnMessageHeader = messagespb.TxnMessageHeader
|
||||||
ImportMessageHeader = messagespb.ImportMessageHeader
|
ImportMessageHeader = messagespb.ImportMessageHeader
|
||||||
|
SchemaChangeMessageHeader = messagespb.SchemaChangeMessageHeader
|
||||||
)
|
)
|
||||||
|
|
||||||
type (
|
type (
|
||||||
@ -39,6 +40,7 @@ type (
|
|||||||
CommitTxnMessageBody = messagespb.CommitTxnMessageBody
|
CommitTxnMessageBody = messagespb.CommitTxnMessageBody
|
||||||
RollbackTxnMessageBody = messagespb.RollbackTxnMessageBody
|
RollbackTxnMessageBody = messagespb.RollbackTxnMessageBody
|
||||||
TxnMessageBody = messagespb.TxnMessageBody
|
TxnMessageBody = messagespb.TxnMessageBody
|
||||||
|
SchemaChangeMessageBody = messagespb.SchemaChangeMessageBody
|
||||||
)
|
)
|
||||||
|
|
||||||
type (
|
type (
|
||||||
@ -62,6 +64,7 @@ var messageTypeMap = map[reflect.Type]MessageType{
|
|||||||
reflect.TypeOf(&RollbackTxnMessageHeader{}): MessageTypeRollbackTxn,
|
reflect.TypeOf(&RollbackTxnMessageHeader{}): MessageTypeRollbackTxn,
|
||||||
reflect.TypeOf(&TxnMessageHeader{}): MessageTypeTxn,
|
reflect.TypeOf(&TxnMessageHeader{}): MessageTypeTxn,
|
||||||
reflect.TypeOf(&ImportMessageHeader{}): MessageTypeImport,
|
reflect.TypeOf(&ImportMessageHeader{}): MessageTypeImport,
|
||||||
|
reflect.TypeOf(&SchemaChangeMessageHeader{}): MessageTypeSchemaChange,
|
||||||
}
|
}
|
||||||
|
|
||||||
// A system preserved message, should not allowed to provide outside of the streaming system.
|
// A system preserved message, should not allowed to provide outside of the streaming system.
|
||||||
@ -93,6 +96,7 @@ type (
|
|||||||
MutableBeginTxnMessageV2 = specializedMutableMessage[*BeginTxnMessageHeader, *BeginTxnMessageBody]
|
MutableBeginTxnMessageV2 = specializedMutableMessage[*BeginTxnMessageHeader, *BeginTxnMessageBody]
|
||||||
MutableCommitTxnMessageV2 = specializedMutableMessage[*CommitTxnMessageHeader, *CommitTxnMessageBody]
|
MutableCommitTxnMessageV2 = specializedMutableMessage[*CommitTxnMessageHeader, *CommitTxnMessageBody]
|
||||||
MutableRollbackTxnMessageV2 = specializedMutableMessage[*RollbackTxnMessageHeader, *RollbackTxnMessageBody]
|
MutableRollbackTxnMessageV2 = specializedMutableMessage[*RollbackTxnMessageHeader, *RollbackTxnMessageBody]
|
||||||
|
MutableSchemaChangeMessageV2 = specializedMutableMessage[*SchemaChangeMessageHeader, *SchemaChangeMessageBody]
|
||||||
|
|
||||||
ImmutableTimeTickMessageV1 = specializedImmutableMessage[*TimeTickMessageHeader, *msgpb.TimeTickMsg]
|
ImmutableTimeTickMessageV1 = specializedImmutableMessage[*TimeTickMessageHeader, *msgpb.TimeTickMsg]
|
||||||
ImmutableInsertMessageV1 = specializedImmutableMessage[*InsertMessageHeader, *msgpb.InsertRequest]
|
ImmutableInsertMessageV1 = specializedImmutableMessage[*InsertMessageHeader, *msgpb.InsertRequest]
|
||||||
@ -108,6 +112,7 @@ type (
|
|||||||
ImmutableBeginTxnMessageV2 = specializedImmutableMessage[*BeginTxnMessageHeader, *BeginTxnMessageBody]
|
ImmutableBeginTxnMessageV2 = specializedImmutableMessage[*BeginTxnMessageHeader, *BeginTxnMessageBody]
|
||||||
ImmutableCommitTxnMessageV2 = specializedImmutableMessage[*CommitTxnMessageHeader, *CommitTxnMessageBody]
|
ImmutableCommitTxnMessageV2 = specializedImmutableMessage[*CommitTxnMessageHeader, *CommitTxnMessageBody]
|
||||||
ImmutableRollbackTxnMessageV2 = specializedImmutableMessage[*RollbackTxnMessageHeader, *RollbackTxnMessageBody]
|
ImmutableRollbackTxnMessageV2 = specializedImmutableMessage[*RollbackTxnMessageHeader, *RollbackTxnMessageBody]
|
||||||
|
ImmutableSchemaChangeMessageV2 = specializedImmutableMessage[*SchemaChangeMessageHeader, *SchemaChangeMessageBody]
|
||||||
)
|
)
|
||||||
|
|
||||||
// List all as functions for specialized messages.
|
// List all as functions for specialized messages.
|
||||||
@ -141,6 +146,7 @@ var (
|
|||||||
AsImmutableBeginTxnMessageV2 = asSpecializedImmutableMessage[*BeginTxnMessageHeader, *BeginTxnMessageBody]
|
AsImmutableBeginTxnMessageV2 = asSpecializedImmutableMessage[*BeginTxnMessageHeader, *BeginTxnMessageBody]
|
||||||
AsImmutableCommitTxnMessageV2 = asSpecializedImmutableMessage[*CommitTxnMessageHeader, *CommitTxnMessageBody]
|
AsImmutableCommitTxnMessageV2 = asSpecializedImmutableMessage[*CommitTxnMessageHeader, *CommitTxnMessageBody]
|
||||||
AsImmutableRollbackTxnMessageV2 = asSpecializedImmutableMessage[*RollbackTxnMessageHeader, *RollbackTxnMessageBody]
|
AsImmutableRollbackTxnMessageV2 = asSpecializedImmutableMessage[*RollbackTxnMessageHeader, *RollbackTxnMessageBody]
|
||||||
|
AsImmutableCollectionSchemaChangeV2 = asSpecializedImmutableMessage[*SchemaChangeMessageHeader, *SchemaChangeMessageBody]
|
||||||
AsImmutableTxnMessage = func(msg ImmutableMessage) ImmutableTxnMessage {
|
AsImmutableTxnMessage = func(msg ImmutableMessage) ImmutableTxnMessage {
|
||||||
underlying, ok := msg.(*immutableTxnMessageImpl)
|
underlying, ok := msg.(*immutableTxnMessageImpl)
|
||||||
if !ok {
|
if !ok {
|
||||||
|
|||||||
Loading…
x
Reference in New Issue
Block a user