From 88cb2746442b56abb18fdf948b2fa71dd3803a06 Mon Sep 17 00:00:00 2001 From: XuanYang-cn Date: Fri, 13 Nov 2020 12:46:24 +0800 Subject: [PATCH] Refactor master scheduler: drop_collection Signed-off-by: XuanYang-cn --- internal/master/collection/collection.go | 2 +- internal/master/collection_task.go | 14 +- internal/master/collection_task_test.go | 186 +++++++++++++++++++ internal/master/grpc_service.go | 45 ++--- internal/master/grpc_service_test.go | 2 +- internal/master/master.go | 2 +- internal/master/meta_table.go | 24 +-- internal/master/meta_table_test.go | 36 ++-- internal/master/mock/grpc_service.go | 2 +- internal/master/partition_task.go | 8 +- internal/proto/etcdpb/etcd_meta.pb.go | 17 +- internal/proto/internalpb/internal_msg.pb.go | 73 ++++---- internal/proto/servicepb/service_msg.pb.go | 180 +++++++----------- internal/proxy/proxy.go | 2 +- internal/reader/col_seg_container_test.go | 48 ++--- internal/reader/collection.go | 2 +- internal/reader/collection_test.go | 12 +- internal/reader/data_sync_service_test.go | 4 +- internal/reader/partition_test.go | 4 +- internal/reader/segment_test.go | 36 ++-- scripts/README.md | 2 +- 21 files changed, 414 insertions(+), 287 deletions(-) create mode 100644 internal/master/collection_task_test.go diff --git a/internal/master/collection/collection.go b/internal/master/collection/collection.go index e38cdad8f4..bcbebc892e 100644 --- a/internal/master/collection/collection.go +++ b/internal/master/collection/collection.go @@ -50,7 +50,7 @@ func GrpcMarshal(c *Collection) *Collection { } pbSchema.Fields = schemaSlice grpcCollection := &etcdpb.CollectionMeta{ - ID: c.ID, + Id: c.ID, Schema: pbSchema, CreateTime: c.CreateTime, SegmentIds: c.SegmentIDs, diff --git a/internal/master/collection_task.go b/internal/master/collection_task.go index 0c0d682baa..a28e890a46 100644 --- a/internal/master/collection_task.go +++ b/internal/master/collection_task.go @@ -3,7 +3,6 @@ package master import ( "errors" "log" - "strconv" "github.com/golang/protobuf/proto" "github.com/zilliztech/milvus-distributed/internal/util/typeutil" @@ -85,7 +84,7 @@ func (t *createCollectionTask) Execute() error { } collection := etcdpb.CollectionMeta{ - ID: collectionID, + Id: collectionID, Schema: &schema, CreateTime: ts, // TODO: initial segment? @@ -124,16 +123,9 @@ func (t *dropCollectionTask) Execute() error { return err } - collectionID := collectionMeta.ID + collectionID := collectionMeta.Id - err = (*t.kvBase).Remove(collectionMetaPrefix + strconv.FormatInt(collectionID, 10)) - if err != nil { - return err - } - - delete(t.mt.collID2Meta, collectionID) - - return nil + return t.mt.DeleteCollection(collectionID) } ////////////////////////////////////////////////////////////////////////// diff --git a/internal/master/collection_task_test.go b/internal/master/collection_task_test.go new file mode 100644 index 0000000000..069ffc7ddf --- /dev/null +++ b/internal/master/collection_task_test.go @@ -0,0 +1,186 @@ +package master + +import ( + "context" + "log" + "strconv" + "testing" + + "github.com/golang/protobuf/proto" + "github.com/stretchr/testify/assert" + "github.com/zilliztech/milvus-distributed/internal/conf" + "github.com/zilliztech/milvus-distributed/internal/proto/commonpb" + "github.com/zilliztech/milvus-distributed/internal/proto/internalpb" + "github.com/zilliztech/milvus-distributed/internal/proto/masterpb" + "github.com/zilliztech/milvus-distributed/internal/proto/schemapb" + "github.com/zilliztech/milvus-distributed/internal/proto/servicepb" + "go.etcd.io/etcd/clientv3" + "google.golang.org/grpc" +) + +func TestMaster_CreateCollectionTask(t *testing.T) { + conf.LoadConfig("config.yaml") + ctx, cancel := context.WithCancel(context.TODO()) + defer cancel() + + etcdPort := strconv.Itoa(int(conf.Config.Etcd.Port)) + etcdAddr := "127.0.0.1:" + etcdPort + + etcdCli, err := clientv3.New(clientv3.Config{Endpoints: []string{etcdAddr}}) + assert.Nil(t, err) + _, err = etcdCli.Delete(ctx, "/test/root", clientv3.WithPrefix()) + assert.Nil(t, err) + + svr, err := CreateServer(ctx, "/test/root/kv", "/test/root/meta", "/test/root/meta/tso", []string{etcdAddr}) + assert.Nil(t, err) + err = svr.Run(10002) + assert.Nil(t, err) + + conn, err := grpc.DialContext(ctx, "127.0.0.1:10002", grpc.WithInsecure(), grpc.WithBlock()) + assert.Nil(t, err) + defer conn.Close() + + cli := masterpb.NewMasterClient(conn) + sch := schemapb.CollectionSchema{ + Name: "col1", + Description: "test collection", + AutoId: false, + Fields: []*schemapb.FieldSchema{ + { + Name: "col1_f1", + Description: "test collection filed 1", + DataType: schemapb.DataType_VECTOR_FLOAT, + TypeParams: []*commonpb.KeyValuePair{ + { + Key: "col1_f1_tk1", + Value: "col1_f1_tv1", + }, + { + Key: "col1_f1_tk2", + Value: "col1_f1_tv2", + }, + }, + IndexParams: []*commonpb.KeyValuePair{ + { + Key: "col1_f1_ik1", + Value: "col1_f1_iv1", + }, + { + Key: "col1_f1_ik2", + Value: "col1_f1_iv2", + }, + }, + }, + { + Name: "col1_f2", + Description: "test collection filed 2", + DataType: schemapb.DataType_VECTOR_BINARY, + TypeParams: []*commonpb.KeyValuePair{ + { + Key: "col1_f2_tk1", + Value: "col1_f2_tv1", + }, + { + Key: "col1_f2_tk2", + Value: "col1_f2_tv2", + }, + }, + IndexParams: []*commonpb.KeyValuePair{ + { + Key: "col1_f2_ik1", + Value: "col1_f2_iv1", + }, + { + Key: "col1_f2_ik2", + Value: "col1_f2_iv2", + }, + }, + }, + }, + } + schemaBytes, err := proto.Marshal(&sch) + assert.Nil(t, err) + + req := internalpb.CreateCollectionRequest{ + MsgType: internalpb.MsgType_kCreateCollection, + ReqId: 1, + Timestamp: 11, + ProxyId: 1, + Schema: &commonpb.Blob{Value: schemaBytes}, + } + log.Printf("... [Create] collection col1\n") + st, err := cli.CreateCollection(ctx, &req) + assert.Nil(t, err) + assert.Equal(t, st.ErrorCode, commonpb.ErrorCode_SUCCESS) + + collMeta, err := svr.mt.GetCollectionByName(sch.Name) + assert.Nil(t, err) + t.Logf("collection id = %d", collMeta.ID) + assert.Equal(t, collMeta.CreateTime, uint64(11)) + assert.Equal(t, collMeta.Schema.Name, "col1") + assert.Equal(t, collMeta.Schema.AutoId, false) + assert.Equal(t, len(collMeta.Schema.Fields), 2) + assert.Equal(t, collMeta.Schema.Fields[0].Name, "col1_f1") + assert.Equal(t, collMeta.Schema.Fields[1].Name, "col1_f2") + assert.Equal(t, collMeta.Schema.Fields[0].DataType, schemapb.DataType_VECTOR_FLOAT) + assert.Equal(t, collMeta.Schema.Fields[1].DataType, schemapb.DataType_VECTOR_BINARY) + assert.Equal(t, len(collMeta.Schema.Fields[0].TypeParams), 2) + assert.Equal(t, len(collMeta.Schema.Fields[0].IndexParams), 2) + assert.Equal(t, len(collMeta.Schema.Fields[1].TypeParams), 2) + assert.Equal(t, len(collMeta.Schema.Fields[1].IndexParams), 2) + assert.Equal(t, collMeta.Schema.Fields[0].TypeParams[0].Key, "col1_f1_tk1") + assert.Equal(t, collMeta.Schema.Fields[0].TypeParams[1].Key, "col1_f1_tk2") + assert.Equal(t, collMeta.Schema.Fields[0].TypeParams[0].Value, "col1_f1_tv1") + assert.Equal(t, collMeta.Schema.Fields[0].TypeParams[1].Value, "col1_f1_tv2") + assert.Equal(t, collMeta.Schema.Fields[0].IndexParams[0].Key, "col1_f1_ik1") + assert.Equal(t, collMeta.Schema.Fields[0].IndexParams[1].Key, "col1_f1_ik2") + assert.Equal(t, collMeta.Schema.Fields[0].IndexParams[0].Value, "col1_f1_iv1") + assert.Equal(t, collMeta.Schema.Fields[0].IndexParams[1].Value, "col1_f1_iv2") + + assert.Equal(t, collMeta.Schema.Fields[1].TypeParams[0].Key, "col1_f2_tk1") + assert.Equal(t, collMeta.Schema.Fields[1].TypeParams[1].Key, "col1_f2_tk2") + assert.Equal(t, collMeta.Schema.Fields[1].TypeParams[0].Value, "col1_f2_tv1") + assert.Equal(t, collMeta.Schema.Fields[1].TypeParams[1].Value, "col1_f2_tv2") + assert.Equal(t, collMeta.Schema.Fields[1].IndexParams[0].Key, "col1_f2_ik1") + assert.Equal(t, collMeta.Schema.Fields[1].IndexParams[1].Key, "col1_f2_ik2") + assert.Equal(t, collMeta.Schema.Fields[1].IndexParams[0].Value, "col1_f2_iv1") + assert.Equal(t, collMeta.Schema.Fields[1].IndexParams[1].Value, "col1_f2_iv2") + + req.Timestamp = Timestamp(10) + st, err = cli.CreateCollection(ctx, &req) + assert.Nil(t, err) + assert.NotEqual(t, st.ErrorCode, commonpb.ErrorCode_SUCCESS) + // ------------------------------DropCollectionTask--------------------------- + + log.Printf("... [Drop] collection col1\n") + ser := servicepb.CollectionName{CollectionName: "col1"} + + reqDrop := internalpb.DropCollectionRequest{ + MsgType: internalpb.MsgType_kDropCollection, + ReqId: 1, + Timestamp: 11, + ProxyId: 1, + CollectionName: &ser, + } + + st, err = cli.DropCollection(ctx, &reqDrop) + assert.Nil(t, err) + assert.Equal(t, st.ErrorCode, commonpb.ErrorCode_SUCCESS) + + collMeta, err = svr.mt.GetCollectionByName(sch.Name) + assert.NotNil(t, err) + + // Drop again + st, err = cli.DropCollection(ctx, &reqDrop) + assert.Nil(t, err) + assert.NotEqual(t, st.ErrorCode, commonpb.ErrorCode_SUCCESS) + + // Create + req.Timestamp = Timestamp(11) + st, err = cli.CreateCollection(ctx, &req) + assert.Nil(t, err) + log.Printf(st.Reason) + assert.Equal(t, st.ErrorCode, commonpb.ErrorCode_SUCCESS) + + svr.Close() +} diff --git a/internal/master/grpc_service.go b/internal/master/grpc_service.go index 096c03af59..edb62e3377 100644 --- a/internal/master/grpc_service.go +++ b/internal/master/grpc_service.go @@ -57,20 +57,18 @@ func (s *Master) DropCollection(ctx context.Context, in *internalpb.DropCollecti var err = s.scheduler.Enqueue(t) if err != nil { - err := errors.New("Enqueue failed") return &commonpb.Status{ ErrorCode: commonpb.ErrorCode_UNEXPECTED_ERROR, Reason: "Enqueue failed", - }, err + }, nil } err = t.WaitToFinish(ctx) if err != nil { - err := errors.New("WaitToFinish failed") return &commonpb.Status{ ErrorCode: commonpb.ErrorCode_UNEXPECTED_ERROR, Reason: "WaitToFinish failed", - }, err + }, nil } return &commonpb.Status{ @@ -89,36 +87,29 @@ func (s *Master) HasCollection(ctx context.Context, in *internalpb.HasCollection hasCollection: false, } + st := &commonpb.Status{ + ErrorCode: commonpb.ErrorCode_UNEXPECTED_ERROR, + } + + response := &servicepb.BoolResponse{ + Status: st, + Value: t.(*hasCollectionTask).hasCollection, + } + var err = s.scheduler.Enqueue(t) if err != nil { - err := errors.New("Enqueue failed") - return &servicepb.BoolResponse{ - Status: &commonpb.Status{ - ErrorCode: commonpb.ErrorCode_UNEXPECTED_ERROR, - Reason: "Enqueue failed", - }, - Value: t.(*hasCollectionTask).hasCollection, - }, err + st.Reason = "Enqueue failed" + return response, nil } err = t.WaitToFinish(ctx) if err != nil { - err := errors.New("WaitToFinish failed") - return &servicepb.BoolResponse{ - Status: &commonpb.Status{ - ErrorCode: commonpb.ErrorCode_UNEXPECTED_ERROR, - Reason: "WaitToFinish failed", - }, - Value: t.(*hasCollectionTask).hasCollection, - }, err + st.Reason = "WaitToFinish failed" + return response, nil } - return &servicepb.BoolResponse{ - Status: &commonpb.Status{ - ErrorCode: commonpb.ErrorCode_SUCCESS, - }, - Value: t.(*hasCollectionTask).hasCollection, - }, nil + st.ErrorCode = commonpb.ErrorCode_SUCCESS + return response, nil } func (s *Master) DescribeCollection(ctx context.Context, in *internalpb.DescribeCollectionRequest) (*servicepb.CollectionDescription, error) { @@ -368,7 +359,7 @@ func (s *Master) AllocId(ctx context.Context, request *internalpb.IdRequest) (*i response := &internalpb.IdResponse{ Status: &commonpb.Status{ErrorCode: commonpb.ErrorCode_UNEXPECTED_ERROR}, - ID: ts, + Id: ts, Count: count, } diff --git a/internal/master/grpc_service_test.go b/internal/master/grpc_service_test.go index 112e0c916e..488a1a2e95 100644 --- a/internal/master/grpc_service_test.go +++ b/internal/master/grpc_service_test.go @@ -112,7 +112,7 @@ func TestMaster_CreateCollection(t *testing.T) { collMeta, err := svr.mt.GetCollectionByName(sch.Name) assert.Nil(t, err) - t.Logf("collection id = %d", collMeta.ID) + t.Logf("collection id = %d", collMeta.Id) assert.Equal(t, collMeta.CreateTime, uint64(11)) assert.Equal(t, collMeta.Schema.Name, "col1") assert.Equal(t, collMeta.Schema.AutoId, false) diff --git a/internal/master/master.go b/internal/master/master.go index 132fd91ca0..4a45ec1b6c 100644 --- a/internal/master/master.go +++ b/internal/master/master.go @@ -194,7 +194,7 @@ func (s *Master) startServerLoop(ctx context.Context, grpcPort int64) { func (s *Master) stopServerLoop() { if s.grpcServer != nil { s.grpcServer.GracefulStop() - log.Printf("server is cloded, exit grpc server") + log.Printf("server is closed, exit grpc server") } s.serverLoopCancel() s.serverLoopWg.Wait() diff --git a/internal/master/meta_table.go b/internal/master/meta_table.go index 1b95d56aa2..e166fcc35a 100644 --- a/internal/master/meta_table.go +++ b/internal/master/meta_table.go @@ -60,7 +60,7 @@ func (mt *metaTable) reloadFromKV() error { if err != nil { return err } - mt.tenantID2Meta[tenantMeta.ID] = tenantMeta + mt.tenantID2Meta[tenantMeta.Id] = tenantMeta } _, values, err = mt.client.LoadWithPrefix("proxy") @@ -74,7 +74,7 @@ func (mt *metaTable) reloadFromKV() error { if err != nil { return err } - mt.proxyID2Meta[proxyMeta.ID] = proxyMeta + mt.proxyID2Meta[proxyMeta.Id] = proxyMeta } _, values, err = mt.client.LoadWithPrefix("collection") @@ -88,8 +88,8 @@ func (mt *metaTable) reloadFromKV() error { if err != nil { return err } - mt.collID2Meta[collectionMeta.ID] = collectionMeta - mt.collName2ID[collectionMeta.Schema.Name] = collectionMeta.ID + mt.collID2Meta[collectionMeta.Id] = collectionMeta + mt.collName2ID[collectionMeta.Schema.Name] = collectionMeta.Id } _, values, err = mt.client.LoadWithPrefix("segment") @@ -115,9 +115,9 @@ func (mt *metaTable) saveCollectionMeta(coll *pb.CollectionMeta) error { if err != nil { return err } - mt.collID2Meta[coll.ID] = *coll - mt.collName2ID[coll.Schema.Name] = coll.ID - return mt.client.Save("/collection/"+strconv.FormatInt(coll.ID, 10), string(collBytes)) + mt.collID2Meta[coll.Id] = *coll + mt.collName2ID[coll.Schema.Name] = coll.Id + return mt.client.Save("/collection/"+strconv.FormatInt(coll.Id, 10), string(collBytes)) } // mt.ddLock.Lock() before call this function @@ -156,7 +156,7 @@ func (mt *metaTable) saveCollectionAndDeleteSegmentsMeta(coll *pb.CollectionMeta return err } - kvs["/collection/"+strconv.FormatInt(coll.ID, 10)] = string(collStrs) + kvs["/collection/"+strconv.FormatInt(coll.Id, 10)] = string(collStrs) for _, segID := range segIDs { _, ok := mt.segID2Meta[segID] @@ -166,7 +166,7 @@ func (mt *metaTable) saveCollectionAndDeleteSegmentsMeta(coll *pb.CollectionMeta } } - mt.collID2Meta[coll.ID] = *coll + mt.collID2Meta[coll.Id] = *coll return mt.client.MultiSaveAndRemove(kvs, segIDStrs) } @@ -178,10 +178,10 @@ func (mt *metaTable) saveCollectionsAndSegmentsMeta(coll *pb.CollectionMeta, seg if err != nil { return err } - kvs["/collection/"+strconv.FormatInt(coll.ID, 10)] = string(collBytes) + kvs["/collection/"+strconv.FormatInt(coll.Id, 10)] = string(collBytes) - mt.collID2Meta[coll.ID] = *coll - mt.collName2ID[coll.Schema.Name] = coll.ID + mt.collID2Meta[coll.Id] = *coll + mt.collName2ID[coll.Schema.Name] = coll.Id segBytes, err := proto.Marshal(seg) if err != nil { diff --git a/internal/master/meta_table_test.go b/internal/master/meta_table_test.go index b7cf6fb3ae..fd63b09ecc 100644 --- a/internal/master/meta_table_test.go +++ b/internal/master/meta_table_test.go @@ -28,7 +28,7 @@ func TestMetaTable_Collection(t *testing.T) { defer meta.client.Close() colMeta := pb.CollectionMeta{ - ID: 100, + Id: 100, Schema: &schemapb.CollectionSchema{ Name: "coll1", }, @@ -37,7 +37,7 @@ func TestMetaTable_Collection(t *testing.T) { PartitionTags: []string{}, } colMeta2 := pb.CollectionMeta{ - ID: 50, + Id: 50, Schema: &schemapb.CollectionSchema{ Name: "coll1", }, @@ -46,7 +46,7 @@ func TestMetaTable_Collection(t *testing.T) { PartitionTags: []string{}, } colMeta3 := pb.CollectionMeta{ - ID: 30, + Id: 30, Schema: &schemapb.CollectionSchema{ Name: "coll2", }, @@ -55,7 +55,7 @@ func TestMetaTable_Collection(t *testing.T) { PartitionTags: []string{}, } colMeta4 := pb.CollectionMeta{ - ID: 30, + Id: 30, Schema: &schemapb.CollectionSchema{ Name: "coll2", }, @@ -64,7 +64,7 @@ func TestMetaTable_Collection(t *testing.T) { PartitionTags: []string{}, } colMeta5 := pb.CollectionMeta{ - ID: 30, + Id: 30, Schema: &schemapb.CollectionSchema{ Name: "coll2", }, @@ -97,11 +97,11 @@ func TestMetaTable_Collection(t *testing.T) { assert.NotNil(t, err) err = meta.AddCollection(&colMeta5) assert.NotNil(t, err) - hasCollection := meta.HasCollection(colMeta.ID) + hasCollection := meta.HasCollection(colMeta.Id) assert.True(t, hasCollection) - err = meta.AddPartition(colMeta.ID, "p1") + err = meta.AddPartition(colMeta.Id, "p1") assert.Nil(t, err) - err = meta.AddPartition(colMeta.ID, "p2") + err = meta.AddPartition(colMeta.Id, "p2") assert.Nil(t, err) err = meta.AddSegment(&segID1) assert.Nil(t, err) @@ -112,9 +112,9 @@ func TestMetaTable_Collection(t *testing.T) { getColMeta, err := meta.GetCollectionByName(colMeta.Schema.Name) assert.Nil(t, err) assert.Equal(t, 3, len(getColMeta.SegmentIds)) - err = meta.DeleteCollection(colMeta.ID) + err = meta.DeleteCollection(colMeta.Id) assert.Nil(t, err) - hasCollection = meta.HasCollection(colMeta.ID) + hasCollection = meta.HasCollection(colMeta.Id) assert.False(t, hasCollection) _, err = meta.GetSegmentByID(segID1.SegmentId) assert.NotNil(t, err) @@ -132,7 +132,7 @@ func TestMetaTable_Collection(t *testing.T) { assert.Equal(t, 1, len(meta.collID2Meta)) assert.Equal(t, 0, len(meta.segID2Meta)) - err = meta.DeleteCollection(colMeta3.ID) + err = meta.DeleteCollection(colMeta3.Id) assert.Nil(t, err) } @@ -151,7 +151,7 @@ func TestMetaTable_DeletePartition(t *testing.T) { defer meta.client.Close() colMeta := pb.CollectionMeta{ - ID: 100, + Id: 100, Schema: &schemapb.CollectionSchema{ Name: "coll1", }, @@ -176,9 +176,9 @@ func TestMetaTable_DeletePartition(t *testing.T) { } err = meta.AddCollection(&colMeta) assert.Nil(t, err) - err = meta.AddPartition(colMeta.ID, "p1") + err = meta.AddPartition(colMeta.Id, "p1") assert.Nil(t, err) - err = meta.AddPartition(colMeta.ID, "p2") + err = meta.AddPartition(colMeta.Id, "p2") assert.Nil(t, err) err = meta.AddSegment(&segID1) assert.Nil(t, err) @@ -196,9 +196,9 @@ func TestMetaTable_DeletePartition(t *testing.T) { assert.Nil(t, err) assert.Equal(t, 1, len(afterCollMeta.PartitionTags)) assert.Equal(t, 1, len(afterCollMeta.SegmentIds)) - hasPartition := meta.HasPartition(colMeta.ID, "p1") + hasPartition := meta.HasPartition(colMeta.Id, "p1") assert.False(t, hasPartition) - hasPartition = meta.HasPartition(colMeta.ID, "p2") + hasPartition = meta.HasPartition(colMeta.Id, "p2") assert.True(t, hasPartition) _, err = meta.GetSegmentByID(segID1.SegmentId) assert.NotNil(t, err) @@ -239,7 +239,7 @@ func TestMetaTable_Segment(t *testing.T) { assert.Nil(t, err) colMeta := pb.CollectionMeta{ - ID: 100, + Id: 100, Schema: &schemapb.CollectionSchema{ Name: "coll1", }, @@ -254,7 +254,7 @@ func TestMetaTable_Segment(t *testing.T) { } err = meta.AddCollection(&colMeta) assert.Nil(t, err) - err = meta.AddPartition(colMeta.ID, "p1") + err = meta.AddPartition(colMeta.Id, "p1") assert.Nil(t, err) err = meta.AddSegment(&segMeta) assert.Nil(t, err) diff --git a/internal/master/mock/grpc_service.go b/internal/master/mock/grpc_service.go index a31c20ef3e..79a7edbc8f 100644 --- a/internal/master/mock/grpc_service.go +++ b/internal/master/mock/grpc_service.go @@ -102,7 +102,7 @@ func (s *Master) AllocId(ctx context.Context, request *internalpb.IdRequest) (*i response := &internalpb.IdResponse{ Status: &commonpb.Status{ErrorCode: commonpb.ErrorCode_UNEXPECTED_ERROR}, - ID: ts, + Id: ts, Count: count, } diff --git a/internal/master/partition_task.go b/internal/master/partition_task.go index 2dc57c409f..0a1505d361 100644 --- a/internal/master/partition_task.go +++ b/internal/master/partition_task.go @@ -76,7 +76,7 @@ func (t *createPartitionTask) Execute() error { return err } - collectionID := collectionMeta.ID + collectionID := collectionMeta.Id err = (*t.kvBase).Save(partitionMetaPrefix+strconv.FormatInt(collectionID, 10), string(collectionJSON)) if err != nil { return err @@ -113,7 +113,7 @@ func (t *dropPartitionTask) Execute() error { return err } - err = t.mt.DeletePartition(collectionMeta.ID, partitionName.Tag) + err = t.mt.DeletePartition(collectionMeta.Id, partitionName.Tag) if err != nil { return err } @@ -123,7 +123,7 @@ func (t *dropPartitionTask) Execute() error { return err } - collectionID := collectionMeta.ID + collectionID := collectionMeta.Id err = (*t.kvBase).Save(partitionMetaPrefix+strconv.FormatInt(collectionID, 10), string(collectionJSON)) if err != nil { return err @@ -160,7 +160,7 @@ func (t *hasPartitionTask) Execute() error { return err } - t.hasPartition = t.mt.HasPartition(collectionMeta.ID, partitionName.Tag) + t.hasPartition = t.mt.HasPartition(collectionMeta.Id, partitionName.Tag) return nil } diff --git a/internal/proto/etcdpb/etcd_meta.pb.go b/internal/proto/etcdpb/etcd_meta.pb.go index 1987b104e5..cecb028791 100644 --- a/internal/proto/etcdpb/etcd_meta.pb.go +++ b/internal/proto/etcdpb/etcd_meta.pb.go @@ -5,11 +5,10 @@ package etcdpb import ( fmt "fmt" - math "math" - proto "github.com/golang/protobuf/proto" commonpb "github.com/zilliztech/milvus-distributed/internal/proto/commonpb" schemapb "github.com/zilliztech/milvus-distributed/internal/proto/schemapb" + math "math" ) // Reference imports to suppress errors if they are not otherwise used. @@ -24,7 +23,7 @@ var _ = math.Inf const _ = proto.ProtoPackageIsVersion3 // please upgrade the proto package type TenantMeta struct { - ID int64 `protobuf:"varint,1,opt,name=id,proto3" json:"id,omitempty"` + Id int64 `protobuf:"varint,1,opt,name=id,proto3" json:"id,omitempty"` NumQueryNodes int64 `protobuf:"varint,2,opt,name=num_query_nodes,json=numQueryNodes,proto3" json:"num_query_nodes,omitempty"` InsertChannelIds []string `protobuf:"bytes,3,rep,name=insert_channel_ids,json=insertChannelIds,proto3" json:"insert_channel_ids,omitempty"` QueryChannelId string `protobuf:"bytes,4,opt,name=query_channel_id,json=queryChannelId,proto3" json:"query_channel_id,omitempty"` @@ -60,7 +59,7 @@ var xxx_messageInfo_TenantMeta proto.InternalMessageInfo func (m *TenantMeta) GetId() int64 { if m != nil { - return m.ID + return m.Id } return 0 } @@ -87,7 +86,7 @@ func (m *TenantMeta) GetQueryChannelId() string { } type ProxyMeta struct { - ID int64 `protobuf:"varint,1,opt,name=id,proto3" json:"id,omitempty"` + Id int64 `protobuf:"varint,1,opt,name=id,proto3" json:"id,omitempty"` Address *commonpb.Address `protobuf:"bytes,2,opt,name=address,proto3" json:"address,omitempty"` ResultChannelIds []string `protobuf:"bytes,3,rep,name=result_channel_ids,json=resultChannelIds,proto3" json:"result_channel_ids,omitempty"` XXX_NoUnkeyedLiteral struct{} `json:"-"` @@ -122,7 +121,7 @@ var xxx_messageInfo_ProxyMeta proto.InternalMessageInfo func (m *ProxyMeta) GetId() int64 { if m != nil { - return m.ID + return m.Id } return 0 } @@ -142,7 +141,7 @@ func (m *ProxyMeta) GetResultChannelIds() []string { } type CollectionMeta struct { - ID int64 `protobuf:"varint,1,opt,name=id,proto3" json:"id,omitempty"` + Id int64 `protobuf:"varint,1,opt,name=id,proto3" json:"id,omitempty"` Schema *schemapb.CollectionSchema `protobuf:"bytes,2,opt,name=schema,proto3" json:"schema,omitempty"` CreateTime uint64 `protobuf:"varint,3,opt,name=create_time,json=createTime,proto3" json:"create_time,omitempty"` SegmentIds []int64 `protobuf:"varint,4,rep,packed,name=segment_ids,json=segmentIds,proto3" json:"segment_ids,omitempty"` @@ -179,7 +178,7 @@ var xxx_messageInfo_CollectionMeta proto.InternalMessageInfo func (m *CollectionMeta) GetId() int64 { if m != nil { - return m.ID + return m.Id } return 0 } @@ -214,7 +213,7 @@ func (m *CollectionMeta) GetPartitionTags() []string { type SegmentMeta struct { SegmentId int64 `protobuf:"varint,1,opt,name=segment_id,json=segmentId,proto3" json:"segment_id,omitempty"` - CollectionId int64 `protobuf:"varint,2,opt,name=collection_id,json=collectionID,proto3" json:"collection_id,omitempty"` + CollectionId int64 `protobuf:"varint,2,opt,name=collection_id,json=collectionId,proto3" json:"collection_id,omitempty"` PartitionTag string `protobuf:"bytes,3,opt,name=partition_tag,json=partitionTag,proto3" json:"partition_tag,omitempty"` ChannelStart int32 `protobuf:"varint,4,opt,name=channel_start,json=channelStart,proto3" json:"channel_start,omitempty"` ChannelEnd int32 `protobuf:"varint,5,opt,name=channel_end,json=channelEnd,proto3" json:"channel_end,omitempty"` diff --git a/internal/proto/internalpb/internal_msg.pb.go b/internal/proto/internalpb/internal_msg.pb.go index 42667f44c9..a755c0c822 100644 --- a/internal/proto/internalpb/internal_msg.pb.go +++ b/internal/proto/internalpb/internal_msg.pb.go @@ -5,11 +5,10 @@ package internalpb import ( fmt "fmt" - math "math" - proto "github.com/golang/protobuf/proto" commonpb "github.com/zilliztech/milvus-distributed/internal/proto/commonpb" servicepb "github.com/zilliztech/milvus-distributed/internal/proto/servicepb" + math "math" ) // Reference imports to suppress errors if they are not otherwise used. @@ -127,7 +126,7 @@ func (PeerRole) EnumDescriptor() ([]byte, []int) { } type IdRequest struct { - PeerId int64 `protobuf:"varint,1,opt,name=peerID,json=peerId,proto3" json:"peerID,omitempty"` + PeerId int64 `protobuf:"varint,1,opt,name=peer_id,json=peerId,proto3" json:"peer_id,omitempty"` Role PeerRole `protobuf:"varint,2,opt,name=role,proto3,enum=milvus.proto.internal.PeerRole" json:"role,omitempty"` Count uint32 `protobuf:"varint,3,opt,name=count,proto3" json:"count,omitempty"` XXX_NoUnkeyedLiteral struct{} `json:"-"` @@ -183,7 +182,7 @@ func (m *IdRequest) GetCount() uint32 { type IdResponse struct { Status *commonpb.Status `protobuf:"bytes,1,opt,name=status,proto3" json:"status,omitempty"` - ID int64 `protobuf:"varint,2,opt,name=id,proto3" json:"id,omitempty"` + Id int64 `protobuf:"varint,2,opt,name=id,proto3" json:"id,omitempty"` Count uint32 `protobuf:"varint,3,opt,name=count,proto3" json:"count,omitempty"` XXX_NoUnkeyedLiteral struct{} `json:"-"` XXX_unrecognized []byte `json:"-"` @@ -224,7 +223,7 @@ func (m *IdResponse) GetStatus() *commonpb.Status { func (m *IdResponse) GetId() int64 { if m != nil { - return m.ID + return m.Id } return 0 } @@ -237,7 +236,7 @@ func (m *IdResponse) GetCount() uint32 { } type TsoRequest struct { - PeerId int64 `protobuf:"varint,1,opt,name=peerID,json=peerId,proto3" json:"peerID,omitempty"` + PeerId int64 `protobuf:"varint,1,opt,name=peer_id,json=peerId,proto3" json:"peer_id,omitempty"` Role PeerRole `protobuf:"varint,2,opt,name=role,proto3,enum=milvus.proto.internal.PeerRole" json:"role,omitempty"` Count uint32 `protobuf:"varint,3,opt,name=count,proto3" json:"count,omitempty"` XXX_NoUnkeyedLiteral struct{} `json:"-"` @@ -348,7 +347,7 @@ func (m *TsoResponse) GetCount() uint32 { type CreateCollectionRequest struct { MsgType MsgType `protobuf:"varint,1,opt,name=msg_type,json=msgType,proto3,enum=milvus.proto.internal.MsgType" json:"msg_type,omitempty"` - ReqId int64 `protobuf:"varint,2,opt,name=req_id,json=reqID,proto3" json:"req_id,omitempty"` + ReqId int64 `protobuf:"varint,2,opt,name=req_id,json=reqId,proto3" json:"req_id,omitempty"` Timestamp uint64 `protobuf:"varint,3,opt,name=timestamp,proto3" json:"timestamp,omitempty"` ProxyId int64 `protobuf:"varint,4,opt,name=proxy_id,json=proxyId,proto3" json:"proxy_id,omitempty"` Schema *commonpb.Blob `protobuf:"bytes,5,opt,name=schema,proto3" json:"schema,omitempty"` @@ -389,7 +388,7 @@ func (m *CreateCollectionRequest) GetMsgType() MsgType { return MsgType_kNone } -func (m *CreateCollectionRequest) GetReqID() int64 { +func (m *CreateCollectionRequest) GetReqId() int64 { if m != nil { return m.ReqId } @@ -419,7 +418,7 @@ func (m *CreateCollectionRequest) GetSchema() *commonpb.Blob { type DropCollectionRequest struct { MsgType MsgType `protobuf:"varint,1,opt,name=msg_type,json=msgType,proto3,enum=milvus.proto.internal.MsgType" json:"msg_type,omitempty"` - ReqId int64 `protobuf:"varint,2,opt,name=req_id,json=reqID,proto3" json:"req_id,omitempty"` + ReqId int64 `protobuf:"varint,2,opt,name=req_id,json=reqId,proto3" json:"req_id,omitempty"` Timestamp uint64 `protobuf:"varint,3,opt,name=timestamp,proto3" json:"timestamp,omitempty"` ProxyId int64 `protobuf:"varint,4,opt,name=proxy_id,json=proxyId,proto3" json:"proxy_id,omitempty"` CollectionName *servicepb.CollectionName `protobuf:"bytes,5,opt,name=collection_name,json=collectionName,proto3" json:"collection_name,omitempty"` @@ -460,7 +459,7 @@ func (m *DropCollectionRequest) GetMsgType() MsgType { return MsgType_kNone } -func (m *DropCollectionRequest) GetReqID() int64 { +func (m *DropCollectionRequest) GetReqId() int64 { if m != nil { return m.ReqId } @@ -490,7 +489,7 @@ func (m *DropCollectionRequest) GetCollectionName() *servicepb.CollectionName { type HasCollectionRequest struct { MsgType MsgType `protobuf:"varint,1,opt,name=msg_type,json=msgType,proto3,enum=milvus.proto.internal.MsgType" json:"msg_type,omitempty"` - ReqId int64 `protobuf:"varint,2,opt,name=req_id,json=reqID,proto3" json:"req_id,omitempty"` + ReqId int64 `protobuf:"varint,2,opt,name=req_id,json=reqId,proto3" json:"req_id,omitempty"` Timestamp uint64 `protobuf:"varint,3,opt,name=timestamp,proto3" json:"timestamp,omitempty"` ProxyId int64 `protobuf:"varint,4,opt,name=proxy_id,json=proxyId,proto3" json:"proxy_id,omitempty"` CollectionName *servicepb.CollectionName `protobuf:"bytes,5,opt,name=collection_name,json=collectionName,proto3" json:"collection_name,omitempty"` @@ -531,7 +530,7 @@ func (m *HasCollectionRequest) GetMsgType() MsgType { return MsgType_kNone } -func (m *HasCollectionRequest) GetReqID() int64 { +func (m *HasCollectionRequest) GetReqId() int64 { if m != nil { return m.ReqId } @@ -561,7 +560,7 @@ func (m *HasCollectionRequest) GetCollectionName() *servicepb.CollectionName { type DescribeCollectionRequest struct { MsgType MsgType `protobuf:"varint,1,opt,name=msg_type,json=msgType,proto3,enum=milvus.proto.internal.MsgType" json:"msg_type,omitempty"` - ReqId int64 `protobuf:"varint,2,opt,name=req_id,json=reqID,proto3" json:"req_id,omitempty"` + ReqId int64 `protobuf:"varint,2,opt,name=req_id,json=reqId,proto3" json:"req_id,omitempty"` Timestamp uint64 `protobuf:"varint,3,opt,name=timestamp,proto3" json:"timestamp,omitempty"` ProxyId int64 `protobuf:"varint,4,opt,name=proxy_id,json=proxyId,proto3" json:"proxy_id,omitempty"` CollectionName *servicepb.CollectionName `protobuf:"bytes,5,opt,name=collection_name,json=collectionName,proto3" json:"collection_name,omitempty"` @@ -602,7 +601,7 @@ func (m *DescribeCollectionRequest) GetMsgType() MsgType { return MsgType_kNone } -func (m *DescribeCollectionRequest) GetReqID() int64 { +func (m *DescribeCollectionRequest) GetReqId() int64 { if m != nil { return m.ReqId } @@ -632,7 +631,7 @@ func (m *DescribeCollectionRequest) GetCollectionName() *servicepb.CollectionNam type ShowCollectionRequest struct { MsgType MsgType `protobuf:"varint,1,opt,name=msg_type,json=msgType,proto3,enum=milvus.proto.internal.MsgType" json:"msg_type,omitempty"` - ReqId int64 `protobuf:"varint,2,opt,name=req_id,json=reqID,proto3" json:"req_id,omitempty"` + ReqId int64 `protobuf:"varint,2,opt,name=req_id,json=reqId,proto3" json:"req_id,omitempty"` Timestamp uint64 `protobuf:"varint,3,opt,name=timestamp,proto3" json:"timestamp,omitempty"` ProxyId int64 `protobuf:"varint,4,opt,name=proxy_id,json=proxyId,proto3" json:"proxy_id,omitempty"` XXX_NoUnkeyedLiteral struct{} `json:"-"` @@ -672,7 +671,7 @@ func (m *ShowCollectionRequest) GetMsgType() MsgType { return MsgType_kNone } -func (m *ShowCollectionRequest) GetReqID() int64 { +func (m *ShowCollectionRequest) GetReqId() int64 { if m != nil { return m.ReqId } @@ -695,7 +694,7 @@ func (m *ShowCollectionRequest) GetProxyId() int64 { type CreatePartitionRequest struct { MsgType MsgType `protobuf:"varint,1,opt,name=msg_type,json=msgType,proto3,enum=milvus.proto.internal.MsgType" json:"msg_type,omitempty"` - ReqId int64 `protobuf:"varint,2,opt,name=req_id,json=reqID,proto3" json:"req_id,omitempty"` + ReqId int64 `protobuf:"varint,2,opt,name=req_id,json=reqId,proto3" json:"req_id,omitempty"` Timestamp uint64 `protobuf:"varint,3,opt,name=timestamp,proto3" json:"timestamp,omitempty"` ProxyId int64 `protobuf:"varint,4,opt,name=proxy_id,json=proxyId,proto3" json:"proxy_id,omitempty"` PartitionName *servicepb.PartitionName `protobuf:"bytes,5,opt,name=partition_name,json=partitionName,proto3" json:"partition_name,omitempty"` @@ -736,7 +735,7 @@ func (m *CreatePartitionRequest) GetMsgType() MsgType { return MsgType_kNone } -func (m *CreatePartitionRequest) GetReqID() int64 { +func (m *CreatePartitionRequest) GetReqId() int64 { if m != nil { return m.ReqId } @@ -766,7 +765,7 @@ func (m *CreatePartitionRequest) GetPartitionName() *servicepb.PartitionName { type DropPartitionRequest struct { MsgType MsgType `protobuf:"varint,1,opt,name=msg_type,json=msgType,proto3,enum=milvus.proto.internal.MsgType" json:"msg_type,omitempty"` - ReqId int64 `protobuf:"varint,2,opt,name=req_id,json=reqID,proto3" json:"req_id,omitempty"` + ReqId int64 `protobuf:"varint,2,opt,name=req_id,json=reqId,proto3" json:"req_id,omitempty"` Timestamp uint64 `protobuf:"varint,3,opt,name=timestamp,proto3" json:"timestamp,omitempty"` ProxyId int64 `protobuf:"varint,4,opt,name=proxy_id,json=proxyId,proto3" json:"proxy_id,omitempty"` PartitionName *servicepb.PartitionName `protobuf:"bytes,5,opt,name=partition_name,json=partitionName,proto3" json:"partition_name,omitempty"` @@ -807,7 +806,7 @@ func (m *DropPartitionRequest) GetMsgType() MsgType { return MsgType_kNone } -func (m *DropPartitionRequest) GetReqID() int64 { +func (m *DropPartitionRequest) GetReqId() int64 { if m != nil { return m.ReqId } @@ -837,7 +836,7 @@ func (m *DropPartitionRequest) GetPartitionName() *servicepb.PartitionName { type HasPartitionRequest struct { MsgType MsgType `protobuf:"varint,1,opt,name=msg_type,json=msgType,proto3,enum=milvus.proto.internal.MsgType" json:"msg_type,omitempty"` - ReqId int64 `protobuf:"varint,2,opt,name=req_id,json=reqID,proto3" json:"req_id,omitempty"` + ReqId int64 `protobuf:"varint,2,opt,name=req_id,json=reqId,proto3" json:"req_id,omitempty"` Timestamp uint64 `protobuf:"varint,3,opt,name=timestamp,proto3" json:"timestamp,omitempty"` ProxyId int64 `protobuf:"varint,4,opt,name=proxy_id,json=proxyId,proto3" json:"proxy_id,omitempty"` PartitionName *servicepb.PartitionName `protobuf:"bytes,5,opt,name=partition_name,json=partitionName,proto3" json:"partition_name,omitempty"` @@ -878,7 +877,7 @@ func (m *HasPartitionRequest) GetMsgType() MsgType { return MsgType_kNone } -func (m *HasPartitionRequest) GetReqID() int64 { +func (m *HasPartitionRequest) GetReqId() int64 { if m != nil { return m.ReqId } @@ -908,7 +907,7 @@ func (m *HasPartitionRequest) GetPartitionName() *servicepb.PartitionName { type DescribePartitionRequest struct { MsgType MsgType `protobuf:"varint,1,opt,name=msg_type,json=msgType,proto3,enum=milvus.proto.internal.MsgType" json:"msg_type,omitempty"` - ReqId int64 `protobuf:"varint,2,opt,name=req_id,json=reqID,proto3" json:"req_id,omitempty"` + ReqId int64 `protobuf:"varint,2,opt,name=req_id,json=reqId,proto3" json:"req_id,omitempty"` Timestamp uint64 `protobuf:"varint,3,opt,name=timestamp,proto3" json:"timestamp,omitempty"` ProxyId int64 `protobuf:"varint,4,opt,name=proxy_id,json=proxyId,proto3" json:"proxy_id,omitempty"` PartitionName *servicepb.PartitionName `protobuf:"bytes,5,opt,name=partition_name,json=partitionName,proto3" json:"partition_name,omitempty"` @@ -949,7 +948,7 @@ func (m *DescribePartitionRequest) GetMsgType() MsgType { return MsgType_kNone } -func (m *DescribePartitionRequest) GetReqID() int64 { +func (m *DescribePartitionRequest) GetReqId() int64 { if m != nil { return m.ReqId } @@ -979,7 +978,7 @@ func (m *DescribePartitionRequest) GetPartitionName() *servicepb.PartitionName { type ShowPartitionRequest struct { MsgType MsgType `protobuf:"varint,1,opt,name=msg_type,json=msgType,proto3,enum=milvus.proto.internal.MsgType" json:"msg_type,omitempty"` - ReqId int64 `protobuf:"varint,2,opt,name=req_id,json=reqID,proto3" json:"req_id,omitempty"` + ReqId int64 `protobuf:"varint,2,opt,name=req_id,json=reqId,proto3" json:"req_id,omitempty"` Timestamp uint64 `protobuf:"varint,3,opt,name=timestamp,proto3" json:"timestamp,omitempty"` ProxyId int64 `protobuf:"varint,4,opt,name=proxy_id,json=proxyId,proto3" json:"proxy_id,omitempty"` CollectionName *servicepb.CollectionName `protobuf:"bytes,5,opt,name=collection_name,json=collectionName,proto3" json:"collection_name,omitempty"` @@ -1020,7 +1019,7 @@ func (m *ShowPartitionRequest) GetMsgType() MsgType { return MsgType_kNone } -func (m *ShowPartitionRequest) GetReqID() int64 { +func (m *ShowPartitionRequest) GetReqId() int64 { if m != nil { return m.ReqId } @@ -1050,7 +1049,7 @@ func (m *ShowPartitionRequest) GetCollectionName() *servicepb.CollectionName { type InsertRequest struct { MsgType MsgType `protobuf:"varint,1,opt,name=msg_type,json=msgType,proto3,enum=milvus.proto.internal.MsgType" json:"msg_type,omitempty"` - ReqId int64 `protobuf:"varint,2,opt,name=req_id,json=reqID,proto3" json:"req_id,omitempty"` + ReqId int64 `protobuf:"varint,2,opt,name=req_id,json=reqId,proto3" json:"req_id,omitempty"` CollectionName string `protobuf:"bytes,3,opt,name=collection_name,json=collectionName,proto3" json:"collection_name,omitempty"` PartitionTag string `protobuf:"bytes,4,opt,name=partition_tag,json=partitionTag,proto3" json:"partition_tag,omitempty"` SegmentId int64 `protobuf:"varint,5,opt,name=segment_id,json=segmentId,proto3" json:"segment_id,omitempty"` @@ -1096,7 +1095,7 @@ func (m *InsertRequest) GetMsgType() MsgType { return MsgType_kNone } -func (m *InsertRequest) GetReqID() int64 { +func (m *InsertRequest) GetReqId() int64 { if m != nil { return m.ReqId } @@ -1161,7 +1160,7 @@ func (m *InsertRequest) GetRowData() []*commonpb.Blob { type DeleteRequest struct { MsgType MsgType `protobuf:"varint,1,opt,name=msg_type,json=msgType,proto3,enum=milvus.proto.internal.MsgType" json:"msg_type,omitempty"` - ReqId int64 `protobuf:"varint,2,opt,name=req_id,json=reqID,proto3" json:"req_id,omitempty"` + ReqId int64 `protobuf:"varint,2,opt,name=req_id,json=reqId,proto3" json:"req_id,omitempty"` CollectionName string `protobuf:"bytes,3,opt,name=collection_name,json=collectionName,proto3" json:"collection_name,omitempty"` ChannelId int64 `protobuf:"varint,4,opt,name=channel_id,json=channelId,proto3" json:"channel_id,omitempty"` ProxyId int64 `protobuf:"varint,5,opt,name=proxy_id,json=proxyId,proto3" json:"proxy_id,omitempty"` @@ -1204,7 +1203,7 @@ func (m *DeleteRequest) GetMsgType() MsgType { return MsgType_kNone } -func (m *DeleteRequest) GetReqID() int64 { +func (m *DeleteRequest) GetReqId() int64 { if m != nil { return m.ReqId } @@ -1248,7 +1247,7 @@ func (m *DeleteRequest) GetPrimaryKeys() []int64 { type SearchRequest struct { MsgType MsgType `protobuf:"varint,1,opt,name=msg_type,json=msgType,proto3,enum=milvus.proto.internal.MsgType" json:"msg_type,omitempty"` - ReqId int64 `protobuf:"varint,2,opt,name=req_id,json=reqID,proto3" json:"req_id,omitempty"` + ReqId int64 `protobuf:"varint,2,opt,name=req_id,json=reqId,proto3" json:"req_id,omitempty"` ProxyId int64 `protobuf:"varint,3,opt,name=proxy_id,json=proxyId,proto3" json:"proxy_id,omitempty"` Timestamp uint64 `protobuf:"varint,4,opt,name=timestamp,proto3" json:"timestamp,omitempty"` ResultChannelId int64 `protobuf:"varint,5,opt,name=result_channel_id,json=resultChannelId,proto3" json:"result_channel_id,omitempty"` @@ -1290,7 +1289,7 @@ func (m *SearchRequest) GetMsgType() MsgType { return MsgType_kNone } -func (m *SearchRequest) GetReqID() int64 { +func (m *SearchRequest) GetReqId() int64 { if m != nil { return m.ReqId } @@ -1328,7 +1327,7 @@ func (m *SearchRequest) GetQuery() *commonpb.Blob { type SearchResult struct { MsgType MsgType `protobuf:"varint,1,opt,name=msg_type,json=msgType,proto3,enum=milvus.proto.internal.MsgType" json:"msg_type,omitempty"` Status *commonpb.Status `protobuf:"bytes,2,opt,name=status,proto3" json:"status,omitempty"` - ReqId int64 `protobuf:"varint,3,opt,name=req_id,json=reqID,proto3" json:"req_id,omitempty"` + ReqId int64 `protobuf:"varint,3,opt,name=req_id,json=reqId,proto3" json:"req_id,omitempty"` ProxyId int64 `protobuf:"varint,4,opt,name=proxy_id,json=proxyId,proto3" json:"proxy_id,omitempty"` QueryNodeId int64 `protobuf:"varint,5,opt,name=query_node_id,json=queryNodeId,proto3" json:"query_node_id,omitempty"` Timestamp uint64 `protobuf:"varint,6,opt,name=timestamp,proto3" json:"timestamp,omitempty"` @@ -1378,7 +1377,7 @@ func (m *SearchResult) GetStatus() *commonpb.Status { return nil } -func (m *SearchResult) GetReqID() int64 { +func (m *SearchResult) GetReqId() int64 { if m != nil { return m.ReqId } @@ -1422,7 +1421,7 @@ func (m *SearchResult) GetHits() []*servicepb.Hits { type TimeTickMsg struct { MsgType MsgType `protobuf:"varint,1,opt,name=msg_type,json=msgType,proto3,enum=milvus.proto.internal.MsgType" json:"msg_type,omitempty"` - PeerId int64 `protobuf:"varint,2,opt,name=peerID,json=peerId,proto3" json:"peerID,omitempty"` + PeerId int64 `protobuf:"varint,2,opt,name=peer_id,json=peerId,proto3" json:"peer_id,omitempty"` Timestamp uint64 `protobuf:"varint,3,opt,name=timestamp,proto3" json:"timestamp,omitempty"` XXX_NoUnkeyedLiteral struct{} `json:"-"` XXX_unrecognized []byte `json:"-"` @@ -1547,7 +1546,7 @@ func (m *Key2Seg) GetSegmentIds() []int64 { } type Key2SegMsg struct { - ReqId int64 `protobuf:"varint,1,opt,name=req_id,json=reqID,proto3" json:"req_id,omitempty"` + ReqId int64 `protobuf:"varint,1,opt,name=req_id,json=reqId,proto3" json:"req_id,omitempty"` Key2Seg []*Key2Seg `protobuf:"bytes,2,rep,name=key2seg,proto3" json:"key2seg,omitempty"` XXX_NoUnkeyedLiteral struct{} `json:"-"` XXX_unrecognized []byte `json:"-"` @@ -1579,7 +1578,7 @@ func (m *Key2SegMsg) XXX_DiscardUnknown() { var xxx_messageInfo_Key2SegMsg proto.InternalMessageInfo -func (m *Key2SegMsg) GetReqID() int64 { +func (m *Key2SegMsg) GetReqId() int64 { if m != nil { return m.ReqId } diff --git a/internal/proto/servicepb/service_msg.pb.go b/internal/proto/servicepb/service_msg.pb.go index 7dec0284af..eb03afe1b6 100644 --- a/internal/proto/servicepb/service_msg.pb.go +++ b/internal/proto/servicepb/service_msg.pb.go @@ -207,52 +207,13 @@ func (m *RowBatch) GetHashKeys() []int32 { return nil } -type VectorValues struct { - Value []*commonpb.Blob `protobuf:"bytes,1,rep,name=value,proto3" json:"value,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -} - -func (m *VectorValues) Reset() { *m = VectorValues{} } -func (m *VectorValues) String() string { return proto.CompactTextString(m) } -func (*VectorValues) ProtoMessage() {} -func (*VectorValues) Descriptor() ([]byte, []int) { - return fileDescriptor_b4b40b84dd2f74cb, []int{3} -} - -func (m *VectorValues) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_VectorValues.Unmarshal(m, b) -} -func (m *VectorValues) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_VectorValues.Marshal(b, m, deterministic) -} -func (m *VectorValues) XXX_Merge(src proto.Message) { - xxx_messageInfo_VectorValues.Merge(m, src) -} -func (m *VectorValues) XXX_Size() int { - return xxx_messageInfo_VectorValues.Size(m) -} -func (m *VectorValues) XXX_DiscardUnknown() { - xxx_messageInfo_VectorValues.DiscardUnknown(m) -} - -var xxx_messageInfo_VectorValues proto.InternalMessageInfo - -func (m *VectorValues) GetValue() []*commonpb.Blob { - if m != nil { - return m.Value - } - return nil -} - //* // @brief Placeholder value in DSL type PlaceholderValue struct { Tag string `protobuf:"bytes,1,opt,name=tag,proto3" json:"tag,omitempty"` Type PlaceholderType `protobuf:"varint,2,opt,name=type,proto3,enum=milvus.proto.service.PlaceholderType" json:"type,omitempty"` - // value contains the serialized VectorValues - Value []byte `protobuf:"bytes,3,opt,name=value,proto3" json:"value,omitempty"` + // values is a 2d-array, every array contains a vector + Values [][]byte `protobuf:"bytes,3,rep,name=values,proto3" json:"values,omitempty"` XXX_NoUnkeyedLiteral struct{} `json:"-"` XXX_unrecognized []byte `json:"-"` XXX_sizecache int32 `json:"-"` @@ -262,7 +223,7 @@ func (m *PlaceholderValue) Reset() { *m = PlaceholderValue{} } func (m *PlaceholderValue) String() string { return proto.CompactTextString(m) } func (*PlaceholderValue) ProtoMessage() {} func (*PlaceholderValue) Descriptor() ([]byte, []int) { - return fileDescriptor_b4b40b84dd2f74cb, []int{4} + return fileDescriptor_b4b40b84dd2f74cb, []int{3} } func (m *PlaceholderValue) XXX_Unmarshal(b []byte) error { @@ -297,9 +258,9 @@ func (m *PlaceholderValue) GetType() PlaceholderType { return PlaceholderType_NONE } -func (m *PlaceholderValue) GetValue() []byte { +func (m *PlaceholderValue) GetValues() [][]byte { if m != nil { - return m.Value + return m.Values } return nil } @@ -315,7 +276,7 @@ func (m *PlaceholderGroup) Reset() { *m = PlaceholderGroup{} } func (m *PlaceholderGroup) String() string { return proto.CompactTextString(m) } func (*PlaceholderGroup) ProtoMessage() {} func (*PlaceholderGroup) Descriptor() ([]byte, []int) { - return fileDescriptor_b4b40b84dd2f74cb, []int{5} + return fileDescriptor_b4b40b84dd2f74cb, []int{4} } func (m *PlaceholderGroup) XXX_Unmarshal(b []byte) error { @@ -346,9 +307,10 @@ func (m *PlaceholderGroup) GetPlaceholders() []*PlaceholderValue { //* // @brief Query for Search call type Query struct { - CollectionName string `protobuf:"bytes,1,opt,name=collection_name,json=collectionName,proto3" json:"collection_name,omitempty"` - PartitionTags []string `protobuf:"bytes,2,rep,name=partition_tags,json=partitionTags,proto3" json:"partition_tags,omitempty"` - Dsl string `protobuf:"bytes,3,opt,name=dsl,proto3" json:"dsl,omitempty"` + CollectionName string `protobuf:"bytes,1,opt,name=collection_name,json=collectionName,proto3" json:"collection_name,omitempty"` + PartitionTags []string `protobuf:"bytes,2,rep,name=partition_tags,json=partitionTags,proto3" json:"partition_tags,omitempty"` + Dsl string `protobuf:"bytes,3,opt,name=dsl,proto3" json:"dsl,omitempty"` + // placeholder_group contains the serialized PlaceholderGroup PlaceholderGroup []byte `protobuf:"bytes,4,opt,name=placeholder_group,json=placeholderGroup,proto3" json:"placeholder_group,omitempty"` XXX_NoUnkeyedLiteral struct{} `json:"-"` XXX_unrecognized []byte `json:"-"` @@ -359,7 +321,7 @@ func (m *Query) Reset() { *m = Query{} } func (m *Query) String() string { return proto.CompactTextString(m) } func (*Query) ProtoMessage() {} func (*Query) Descriptor() ([]byte, []int) { - return fileDescriptor_b4b40b84dd2f74cb, []int{6} + return fileDescriptor_b4b40b84dd2f74cb, []int{5} } func (m *Query) XXX_Unmarshal(b []byte) error { @@ -422,7 +384,7 @@ func (m *StringResponse) Reset() { *m = StringResponse{} } func (m *StringResponse) String() string { return proto.CompactTextString(m) } func (*StringResponse) ProtoMessage() {} func (*StringResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_b4b40b84dd2f74cb, []int{7} + return fileDescriptor_b4b40b84dd2f74cb, []int{6} } func (m *StringResponse) XXX_Unmarshal(b []byte) error { @@ -471,7 +433,7 @@ func (m *BoolResponse) Reset() { *m = BoolResponse{} } func (m *BoolResponse) String() string { return proto.CompactTextString(m) } func (*BoolResponse) ProtoMessage() {} func (*BoolResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_b4b40b84dd2f74cb, []int{8} + return fileDescriptor_b4b40b84dd2f74cb, []int{7} } func (m *BoolResponse) XXX_Unmarshal(b []byte) error { @@ -520,7 +482,7 @@ func (m *StringListResponse) Reset() { *m = StringListResponse{} } func (m *StringListResponse) String() string { return proto.CompactTextString(m) } func (*StringListResponse) ProtoMessage() {} func (*StringListResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_b4b40b84dd2f74cb, []int{9} + return fileDescriptor_b4b40b84dd2f74cb, []int{8} } func (m *StringListResponse) XXX_Unmarshal(b []byte) error { @@ -569,7 +531,7 @@ func (m *IntegerListResponse) Reset() { *m = IntegerListResponse{} } func (m *IntegerListResponse) String() string { return proto.CompactTextString(m) } func (*IntegerListResponse) ProtoMessage() {} func (*IntegerListResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_b4b40b84dd2f74cb, []int{10} + return fileDescriptor_b4b40b84dd2f74cb, []int{9} } func (m *IntegerListResponse) XXX_Unmarshal(b []byte) error { @@ -619,7 +581,7 @@ func (m *IntegerRangeResponse) Reset() { *m = IntegerRangeResponse{} } func (m *IntegerRangeResponse) String() string { return proto.CompactTextString(m) } func (*IntegerRangeResponse) ProtoMessage() {} func (*IntegerRangeResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_b4b40b84dd2f74cb, []int{11} + return fileDescriptor_b4b40b84dd2f74cb, []int{10} } func (m *IntegerRangeResponse) XXX_Unmarshal(b []byte) error { @@ -676,7 +638,7 @@ func (m *CollectionDescription) Reset() { *m = CollectionDescription{} } func (m *CollectionDescription) String() string { return proto.CompactTextString(m) } func (*CollectionDescription) ProtoMessage() {} func (*CollectionDescription) Descriptor() ([]byte, []int) { - return fileDescriptor_b4b40b84dd2f74cb, []int{12} + return fileDescriptor_b4b40b84dd2f74cb, []int{11} } func (m *CollectionDescription) XXX_Unmarshal(b []byte) error { @@ -733,7 +695,7 @@ func (m *PartitionDescription) Reset() { *m = PartitionDescription{} } func (m *PartitionDescription) String() string { return proto.CompactTextString(m) } func (*PartitionDescription) ProtoMessage() {} func (*PartitionDescription) Descriptor() ([]byte, []int) { - return fileDescriptor_b4b40b84dd2f74cb, []int{13} + return fileDescriptor_b4b40b84dd2f74cb, []int{12} } func (m *PartitionDescription) XXX_Unmarshal(b []byte) error { @@ -791,7 +753,7 @@ func (m *Score) Reset() { *m = Score{} } func (m *Score) String() string { return proto.CompactTextString(m) } func (*Score) ProtoMessage() {} func (*Score) Descriptor() ([]byte, []int) { - return fileDescriptor_b4b40b84dd2f74cb, []int{14} + return fileDescriptor_b4b40b84dd2f74cb, []int{13} } func (m *Score) XXX_Unmarshal(b []byte) error { @@ -841,7 +803,7 @@ func (m *Hits) Reset() { *m = Hits{} } func (m *Hits) String() string { return proto.CompactTextString(m) } func (*Hits) ProtoMessage() {} func (*Hits) Descriptor() ([]byte, []int) { - return fileDescriptor_b4b40b84dd2f74cb, []int{15} + return fileDescriptor_b4b40b84dd2f74cb, []int{14} } func (m *Hits) XXX_Unmarshal(b []byte) error { @@ -897,7 +859,7 @@ func (m *QueryResult) Reset() { *m = QueryResult{} } func (m *QueryResult) String() string { return proto.CompactTextString(m) } func (*QueryResult) ProtoMessage() {} func (*QueryResult) Descriptor() ([]byte, []int) { - return fileDescriptor_b4b40b84dd2f74cb, []int{16} + return fileDescriptor_b4b40b84dd2f74cb, []int{15} } func (m *QueryResult) XXX_Unmarshal(b []byte) error { @@ -937,7 +899,6 @@ func init() { proto.RegisterType((*CollectionName)(nil), "milvus.proto.service.CollectionName") proto.RegisterType((*PartitionName)(nil), "milvus.proto.service.PartitionName") proto.RegisterType((*RowBatch)(nil), "milvus.proto.service.RowBatch") - proto.RegisterType((*VectorValues)(nil), "milvus.proto.service.VectorValues") proto.RegisterType((*PlaceholderValue)(nil), "milvus.proto.service.PlaceholderValue") proto.RegisterType((*PlaceholderGroup)(nil), "milvus.proto.service.PlaceholderGroup") proto.RegisterType((*Query)(nil), "milvus.proto.service.Query") @@ -956,54 +917,53 @@ func init() { func init() { proto.RegisterFile("service_msg.proto", fileDescriptor_b4b40b84dd2f74cb) } var fileDescriptor_b4b40b84dd2f74cb = []byte{ - // 775 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xac, 0x54, 0xdd, 0x6e, 0xf3, 0x44, - 0x10, 0xc5, 0x71, 0x12, 0x92, 0x89, 0x93, 0xa6, 0x4b, 0x40, 0xa1, 0xbd, 0x09, 0xae, 0x0a, 0x11, - 0x88, 0x44, 0xb4, 0x48, 0xa8, 0x17, 0x08, 0x92, 0xb6, 0x40, 0x7f, 0x94, 0x96, 0x4d, 0x54, 0xa9, - 0x20, 0x11, 0x6d, 0xec, 0x95, 0xbd, 0xc2, 0xf1, 0x5a, 0xbb, 0xeb, 0x56, 0xe9, 0x2d, 0xef, 0xc0, - 0x4b, 0xf0, 0x20, 0xdc, 0xf1, 0x4c, 0xc8, 0x6b, 0xe7, 0xaf, 0x04, 0xb5, 0x5f, 0xdb, 0xbb, 0xf1, - 0x78, 0x66, 0xce, 0x9c, 0xb3, 0xbb, 0x07, 0xb6, 0x25, 0x15, 0x77, 0xcc, 0xa1, 0xe3, 0xa9, 0xf4, - 0x3a, 0x91, 0xe0, 0x8a, 0xa3, 0xc6, 0x94, 0x05, 0x77, 0xb1, 0x4c, 0xbf, 0x3a, 0xd9, 0xff, 0x1d, - 0xcb, 0xe1, 0xd3, 0x29, 0x0f, 0xd3, 0xec, 0x8e, 0x25, 0x1d, 0x9f, 0x4e, 0x49, 0xfa, 0x65, 0x1f, - 0x41, 0xed, 0x98, 0x07, 0x01, 0x75, 0x14, 0xe3, 0xe1, 0x80, 0x4c, 0x29, 0xfa, 0x0c, 0xb6, 0x9c, - 0x45, 0x66, 0x1c, 0x92, 0x29, 0x6d, 0x1a, 0x2d, 0xa3, 0x5d, 0xc6, 0x35, 0x67, 0xad, 0xd0, 0x3e, - 0x87, 0xea, 0x35, 0x11, 0x8a, 0xbd, 0x73, 0x27, 0xaa, 0x83, 0xa9, 0x88, 0xd7, 0xcc, 0xe9, 0x9f, - 0x49, 0x68, 0xff, 0x65, 0x40, 0x09, 0xf3, 0xfb, 0x3e, 0x51, 0x8e, 0xff, 0xfc, 0x39, 0x7b, 0x50, - 0x8d, 0xe6, 0x1b, 0x8c, 0x97, 0x13, 0xad, 0x45, 0x72, 0x44, 0x3c, 0xf4, 0x35, 0x94, 0x04, 0xbf, - 0x1f, 0xbb, 0x44, 0x91, 0xa6, 0xd9, 0x32, 0xdb, 0x95, 0x83, 0x8f, 0x3b, 0x6b, 0x32, 0x65, 0xea, - 0xf4, 0x03, 0x3e, 0xc1, 0xef, 0x0b, 0x7e, 0x7f, 0x42, 0x14, 0x41, 0xbb, 0x50, 0xf6, 0x89, 0xf4, - 0xc7, 0xbf, 0xd3, 0x99, 0x6c, 0xe6, 0x5b, 0x66, 0xbb, 0x80, 0x4b, 0x49, 0xe2, 0x82, 0xce, 0xa4, - 0xfd, 0x1d, 0x58, 0x37, 0xd4, 0x51, 0x5c, 0xdc, 0x90, 0x20, 0xa6, 0x12, 0x75, 0xa1, 0x70, 0x97, - 0x44, 0x4d, 0xe3, 0xa9, 0xf9, 0x69, 0x9d, 0x1d, 0x43, 0xfd, 0x3a, 0x20, 0x0e, 0xf5, 0x79, 0xe0, - 0xd2, 0x74, 0xca, 0x5c, 0x14, 0x63, 0x21, 0x0a, 0x3a, 0x82, 0xbc, 0x9a, 0x45, 0x54, 0xb3, 0xaa, - 0x1d, 0xec, 0x77, 0x36, 0x1d, 0x6e, 0x67, 0x65, 0xce, 0x68, 0x16, 0x51, 0xac, 0x5b, 0x50, 0x63, - 0xbe, 0x91, 0xd9, 0x32, 0xda, 0xd6, 0x1c, 0xf6, 0xb7, 0x35, 0xd8, 0x1f, 0x05, 0x8f, 0x23, 0x74, - 0x0e, 0x56, 0xb4, 0xcc, 0xc9, 0x8c, 0xc2, 0xa7, 0x4f, 0x82, 0xe9, 0xa5, 0xf1, 0x5a, 0xaf, 0xfd, - 0xa7, 0x01, 0x85, 0x9f, 0x63, 0x2a, 0x66, 0xcf, 0x3f, 0xc2, 0x7d, 0xa8, 0xad, 0x1d, 0xa1, 0x6c, - 0xe6, 0x5a, 0x66, 0xbb, 0x8c, 0xab, 0xab, 0x67, 0x28, 0x13, 0x71, 0x5c, 0x19, 0x68, 0x36, 0x65, - 0x9c, 0x84, 0xe8, 0x0b, 0xd8, 0x5e, 0xc1, 0x1e, 0x7b, 0x09, 0x99, 0x66, 0x5e, 0xb3, 0xad, 0x47, - 0x8f, 0x48, 0xda, 0xbf, 0x42, 0x6d, 0xa8, 0x04, 0x0b, 0x3d, 0x4c, 0x65, 0xc4, 0x43, 0x49, 0xd1, - 0x21, 0x14, 0xa5, 0x22, 0x2a, 0x96, 0x7a, 0xaf, 0xca, 0xc1, 0xee, 0xc6, 0x33, 0x1b, 0xea, 0x12, - 0x9c, 0x95, 0x2e, 0x55, 0x4d, 0xef, 0x59, 0xa6, 0xea, 0x2d, 0x58, 0x7d, 0xce, 0x83, 0x37, 0x1c, - 0x5d, 0x9a, 0x8f, 0x26, 0x80, 0xd2, 0xbd, 0x2f, 0x99, 0x54, 0xaf, 0x03, 0xf8, 0x08, 0x8a, 0x7a, - 0xe6, 0x5c, 0xe0, 0xec, 0xcb, 0x9e, 0xc0, 0x07, 0x67, 0xa1, 0xa2, 0x1e, 0x15, 0x6f, 0x8d, 0x61, - 0x2e, 0x30, 0x24, 0x34, 0x32, 0x0c, 0x4c, 0x42, 0x8f, 0xbe, 0x5a, 0xa9, 0x09, 0xf5, 0x58, 0xa8, - 0x95, 0x32, 0x71, 0xfa, 0x91, 0x5c, 0x10, 0x1a, 0xba, 0xfa, 0x82, 0x98, 0x38, 0x09, 0xed, 0x7f, - 0x0c, 0xf8, 0x70, 0x69, 0x6d, 0x27, 0x54, 0x3a, 0x82, 0x45, 0x49, 0xf8, 0x32, 0xd8, 0x6f, 0xa1, - 0x98, 0x1a, 0xa7, 0xc6, 0xad, 0xfc, 0xe7, 0x39, 0xa6, 0xa6, 0xba, 0x04, 0x1c, 0xea, 0x04, 0xce, - 0x9a, 0x50, 0x0f, 0x20, 0x19, 0xc4, 0xa4, 0x62, 0x8e, 0xcc, 0x7c, 0xe8, 0x93, 0x8d, 0xb8, 0x17, - 0x74, 0xa6, 0xdf, 0xd6, 0x35, 0x61, 0x02, 0xaf, 0x34, 0xd9, 0x7f, 0x1b, 0xd0, 0x58, 0x18, 0xee, - 0xab, 0xf9, 0x7c, 0x03, 0x79, 0xfd, 0x2c, 0x53, 0x36, 0x7b, 0xff, 0xf3, 0xde, 0x57, 0xfd, 0x1d, - 0xeb, 0x86, 0xb7, 0x60, 0xf2, 0x15, 0x14, 0x86, 0x0e, 0x17, 0x9b, 0x3c, 0x6f, 0xfd, 0x0a, 0xe5, - 0x16, 0x57, 0xe8, 0x0f, 0x03, 0xf2, 0x3f, 0x31, 0xa5, 0x9d, 0x80, 0xb9, 0xa9, 0x4d, 0x99, 0x38, - 0x09, 0xd7, 0x0c, 0x3e, 0xf7, 0x6c, 0x83, 0x4f, 0x44, 0x4b, 0x76, 0x98, 0x53, 0xd8, 0xdd, 0xac, - 0x80, 0xde, 0x13, 0x67, 0xa5, 0xb6, 0x80, 0x8a, 0xf6, 0x37, 0x4c, 0x65, 0x1c, 0xa8, 0x97, 0x09, - 0xdf, 0x81, 0xbc, 0xcf, 0x94, 0xcc, 0x56, 0xdd, 0xd9, 0x0c, 0x9b, 0x50, 0xc5, 0xba, 0xee, 0xf3, - 0xef, 0x61, 0xeb, 0x91, 0xc7, 0xa3, 0x12, 0xe4, 0x07, 0x57, 0x83, 0xd3, 0xfa, 0x7b, 0x68, 0x1b, - 0xaa, 0x37, 0xa7, 0xc7, 0xa3, 0x2b, 0x3c, 0xee, 0x9f, 0x0d, 0x7a, 0xf8, 0xb6, 0xee, 0xa2, 0x3a, - 0x58, 0x59, 0xea, 0x87, 0xcb, 0xab, 0xde, 0xa8, 0x4e, 0xfb, 0xc7, 0xbf, 0xf4, 0x3c, 0xa6, 0xfc, - 0x78, 0x92, 0x6c, 0xd4, 0x7d, 0x60, 0x41, 0xc0, 0x1e, 0x14, 0x75, 0xfc, 0x6e, 0x0a, 0xfd, 0xa5, - 0xcb, 0xa4, 0x12, 0x6c, 0x12, 0x2b, 0xea, 0x76, 0x59, 0xa8, 0xa8, 0x08, 0x49, 0xd0, 0xd5, 0xfb, - 0x74, 0xb3, 0x7d, 0xa2, 0xc9, 0xa4, 0xa8, 0x13, 0x87, 0xff, 0x06, 0x00, 0x00, 0xff, 0xff, 0x94, - 0xd2, 0x2e, 0x53, 0x76, 0x08, 0x00, 0x00, + // 762 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xac, 0x54, 0x5d, 0x6f, 0xf3, 0x34, + 0x14, 0x26, 0x4d, 0x5b, 0xba, 0xd3, 0xb4, 0x6f, 0x67, 0x0a, 0x0a, 0xdb, 0x4d, 0xc9, 0xab, 0x41, + 0x05, 0xa2, 0x15, 0x1b, 0x12, 0xda, 0x05, 0x12, 0xed, 0x36, 0x60, 0x1f, 0xea, 0x86, 0x5b, 0x4d, + 0x1a, 0x48, 0x54, 0x6e, 0x62, 0x25, 0x16, 0x69, 0x1c, 0xd9, 0xce, 0xaa, 0xee, 0x96, 0xff, 0xc0, + 0x9f, 0xe0, 0x87, 0x70, 0xc7, 0x6f, 0x42, 0x71, 0xd2, 0xaf, 0x51, 0xc4, 0xde, 0x6d, 0x77, 0xf6, + 0xb1, 0xcf, 0x79, 0x9e, 0xf3, 0xf8, 0xf8, 0x81, 0x5d, 0x49, 0xc5, 0x3d, 0x73, 0xe9, 0x78, 0x2a, + 0xfd, 0x4e, 0x2c, 0xb8, 0xe2, 0xa8, 0x39, 0x65, 0xe1, 0x7d, 0x22, 0xb3, 0x5d, 0x27, 0x3f, 0xdf, + 0xb3, 0x5c, 0x3e, 0x9d, 0xf2, 0x28, 0x8b, 0xee, 0x59, 0xd2, 0x0d, 0xe8, 0x94, 0x64, 0x3b, 0xe7, + 0x18, 0xea, 0x27, 0x3c, 0x0c, 0xa9, 0xab, 0x18, 0x8f, 0x06, 0x64, 0x4a, 0xd1, 0x67, 0xf0, 0xc6, + 0x5d, 0x46, 0xc6, 0x11, 0x99, 0x52, 0xdb, 0x68, 0x19, 0xed, 0x1d, 0x5c, 0x77, 0x37, 0x2e, 0x3a, + 0x17, 0x50, 0xbb, 0x21, 0x42, 0xb1, 0x77, 0xce, 0x44, 0x0d, 0x30, 0x15, 0xf1, 0xed, 0x82, 0x3e, + 0x4c, 0x97, 0xce, 0x9f, 0x06, 0x54, 0x30, 0x9f, 0xf5, 0x89, 0x72, 0x83, 0xa7, 0xd7, 0x79, 0x0b, + 0xb5, 0x78, 0xc1, 0x60, 0xbc, 0xaa, 0x68, 0x2d, 0x83, 0x23, 0xe2, 0xa3, 0xaf, 0xa1, 0x22, 0xf8, + 0x6c, 0xec, 0x11, 0x45, 0x6c, 0xb3, 0x65, 0xb6, 0xab, 0x87, 0x1f, 0x77, 0x36, 0x64, 0xca, 0xd5, + 0xe9, 0x87, 0x7c, 0x82, 0xdf, 0x17, 0x7c, 0x76, 0x4a, 0x14, 0x41, 0xfb, 0xb0, 0x13, 0x10, 0x19, + 0x8c, 0x7f, 0xa3, 0x73, 0x69, 0x17, 0x5b, 0x66, 0xbb, 0x84, 0x2b, 0x69, 0xe0, 0x92, 0xce, 0xa5, + 0x33, 0x83, 0xc6, 0x4d, 0x48, 0x5c, 0x1a, 0xf0, 0xd0, 0xa3, 0xe2, 0x96, 0x84, 0xc9, 0xb2, 0x27, + 0x63, 0xd9, 0x13, 0x3a, 0x86, 0xa2, 0x9a, 0xc7, 0x54, 0x93, 0xaa, 0x1f, 0x1e, 0x74, 0xb6, 0xbd, + 0x4d, 0x67, 0xad, 0xce, 0x68, 0x1e, 0x53, 0xac, 0x53, 0xd0, 0x47, 0x50, 0xbe, 0x4f, 0xab, 0x4a, + 0xcd, 0xd8, 0xc2, 0xf9, 0xce, 0xf9, 0x75, 0x03, 0xf8, 0x07, 0xc1, 0x93, 0x18, 0x5d, 0x80, 0x15, + 0xaf, 0x62, 0xd2, 0x36, 0x74, 0x8f, 0x9f, 0xfe, 0x2f, 0x9c, 0xa6, 0x8d, 0x37, 0x72, 0x9d, 0x3f, + 0x0c, 0x28, 0xfd, 0x94, 0x50, 0x31, 0x7f, 0xfa, 0x1b, 0x1c, 0x40, 0x7d, 0xe3, 0x0d, 0xa4, 0x5d, + 0x68, 0x99, 0xed, 0x1d, 0x5c, 0x5b, 0x7f, 0x04, 0x99, 0xca, 0xe3, 0xc9, 0xd0, 0x36, 0x33, 0x79, + 0x3c, 0x19, 0xa2, 0x2f, 0x60, 0x77, 0x0d, 0x7b, 0xec, 0xa7, 0xcd, 0xd8, 0xc5, 0x96, 0xd1, 0xb6, + 0x70, 0x23, 0x7e, 0xd4, 0xa4, 0xf3, 0x0b, 0xd4, 0x87, 0x4a, 0xb0, 0xc8, 0xc7, 0x54, 0xc6, 0x3c, + 0x92, 0x14, 0x1d, 0x41, 0x59, 0x2a, 0xa2, 0x12, 0xa9, 0x79, 0x55, 0x0f, 0xf7, 0xb7, 0x3e, 0xea, + 0x50, 0x5f, 0xc1, 0xf9, 0x55, 0xd4, 0x84, 0x92, 0x56, 0x32, 0x1f, 0x94, 0x6c, 0xe3, 0xdc, 0x81, + 0xd5, 0xe7, 0x3c, 0x7c, 0xc5, 0xd2, 0x95, 0x45, 0x69, 0x02, 0x28, 0xe3, 0x7d, 0xc5, 0xa4, 0x7a, + 0x19, 0xc0, 0x6a, 0x26, 0x32, 0x81, 0x17, 0x33, 0x31, 0x81, 0x0f, 0xce, 0x23, 0x45, 0x7d, 0x2a, + 0x5e, 0x1b, 0xc3, 0x5c, 0x62, 0x48, 0x68, 0xe6, 0x18, 0x98, 0x44, 0x3e, 0x7d, 0xb1, 0x52, 0x13, + 0xea, 0xb3, 0x48, 0x2b, 0x65, 0xe2, 0x6c, 0x93, 0x0e, 0x08, 0x8d, 0x3c, 0x3d, 0x20, 0x26, 0x4e, + 0x97, 0xce, 0xdf, 0x06, 0x7c, 0xb8, 0xf2, 0xa6, 0x53, 0x2a, 0x5d, 0xc1, 0xe2, 0x74, 0xf9, 0x3c, + 0xd8, 0x6f, 0xa1, 0x9c, 0x39, 0x9f, 0xc6, 0xad, 0xfe, 0xeb, 0x43, 0x66, 0xae, 0xb8, 0x02, 0x1c, + 0xea, 0x00, 0xce, 0x93, 0x50, 0x0f, 0x20, 0x2d, 0xc4, 0xa4, 0x62, 0xae, 0xcc, 0x8d, 0xe4, 0x93, + 0xad, 0xb8, 0x97, 0x74, 0xae, 0xff, 0xd6, 0x0d, 0x61, 0x02, 0xaf, 0x25, 0x39, 0x7f, 0x19, 0xd0, + 0x5c, 0x3a, 0xe6, 0x8b, 0xfb, 0xf9, 0x06, 0x8a, 0xfa, 0x5b, 0x66, 0xdd, 0xbc, 0xfd, 0x8f, 0xff, + 0xbe, 0x6e, 0xd0, 0x58, 0x27, 0xbc, 0x46, 0x27, 0x5f, 0x41, 0x69, 0xe8, 0x72, 0xb1, 0xcd, 0xf5, + 0x36, 0x47, 0xa8, 0xb0, 0x1c, 0xa1, 0xdf, 0x0d, 0x28, 0xfe, 0xc8, 0x94, 0x76, 0x02, 0xe6, 0x65, + 0x36, 0x65, 0xe2, 0x74, 0xb9, 0xe1, 0xd0, 0x85, 0x27, 0x3b, 0x74, 0x2a, 0x5a, 0xca, 0x61, 0xd1, + 0xc2, 0xfe, 0x76, 0x05, 0x34, 0x4f, 0x9c, 0x5f, 0x75, 0x04, 0x54, 0xb5, 0xbf, 0x61, 0x2a, 0x93, + 0x50, 0x3d, 0x4f, 0xf8, 0x0e, 0x14, 0x03, 0xa6, 0x64, 0x4e, 0x75, 0x6f, 0x3b, 0x6c, 0xda, 0x2a, + 0xd6, 0xf7, 0x3e, 0xff, 0x0e, 0xde, 0x3c, 0x72, 0x79, 0x54, 0x81, 0xe2, 0xe0, 0x7a, 0x70, 0xd6, + 0x78, 0x0f, 0xed, 0x42, 0xed, 0xf6, 0xec, 0x64, 0x74, 0x8d, 0xc7, 0xfd, 0xf3, 0x41, 0x0f, 0xdf, + 0x35, 0x3c, 0xd4, 0x00, 0x2b, 0x0f, 0x7d, 0x7f, 0x75, 0xdd, 0x1b, 0x35, 0x68, 0xff, 0xe4, 0xe7, + 0x9e, 0xcf, 0x54, 0x90, 0x4c, 0x52, 0x46, 0xdd, 0x07, 0x16, 0x86, 0xec, 0x41, 0x51, 0x37, 0xe8, + 0x66, 0xd0, 0x5f, 0x7a, 0x4c, 0x2a, 0xc1, 0x26, 0x89, 0xa2, 0x5e, 0x97, 0x45, 0x8a, 0x8a, 0x88, + 0x84, 0x5d, 0xcd, 0xa7, 0x9b, 0xf3, 0x89, 0x27, 0x93, 0xb2, 0x0e, 0x1c, 0xfd, 0x13, 0x00, 0x00, + 0xff, 0xff, 0x23, 0x87, 0xde, 0xce, 0x37, 0x08, 0x00, 0x00, } diff --git a/internal/proxy/proxy.go b/internal/proxy/proxy.go index fa243bcc37..22395db6b6 100644 --- a/internal/proxy/proxy.go +++ b/internal/proxy/proxy.go @@ -151,7 +151,7 @@ func (p *Proxy) queryResultLoop() { } tsMsg := msgPack.Msgs[0] searchResultMsg, _ := (*tsMsg).(*msgstream.SearchResultMsg) - reqID := searchResultMsg.GetReqID() + reqID := searchResultMsg.GetReqId() _, ok := queryResultBuf[reqID] if !ok { queryResultBuf[reqID] = make([]*internalpb.SearchResult, 0) diff --git a/internal/reader/col_seg_container_test.go b/internal/reader/col_seg_container_test.go index 458a03f606..527c940866 100644 --- a/internal/reader/col_seg_container_test.go +++ b/internal/reader/col_seg_container_test.go @@ -47,7 +47,7 @@ func TestColSegContainer_addCollection(t *testing.T) { } collectionMeta := etcdpb.CollectionMeta{ - ID: UniqueID(0), + Id: UniqueID(0), Schema: &schema, CreateTime: Timestamp(0), SegmentIds: []UniqueID{0}, @@ -60,7 +60,7 @@ func TestColSegContainer_addCollection(t *testing.T) { var collection = node.container.addCollection(&collectionMeta, collectionMetaBlob) assert.Equal(t, collection.meta.Schema.Name, "collection0") - assert.Equal(t, collection.meta.ID, UniqueID(0)) + assert.Equal(t, collection.meta.Id, UniqueID(0)) assert.Equal(t, len(node.container.collections), 1) } @@ -99,7 +99,7 @@ func TestColSegContainer_removeCollection(t *testing.T) { } collectionMeta := etcdpb.CollectionMeta{ - ID: UniqueID(0), + Id: UniqueID(0), Schema: &schema, CreateTime: Timestamp(0), SegmentIds: []UniqueID{0}, @@ -112,7 +112,7 @@ func TestColSegContainer_removeCollection(t *testing.T) { var collection = node.container.addCollection(&collectionMeta, collectionMetaBlob) assert.Equal(t, collection.meta.Schema.Name, "collection0") - assert.Equal(t, collection.meta.ID, UniqueID(0)) + assert.Equal(t, collection.meta.Id, UniqueID(0)) assert.Equal(t, len(node.container.collections), 1) err := node.container.removeCollection(collection) @@ -155,7 +155,7 @@ func TestColSegContainer_getCollectionByID(t *testing.T) { } collectionMeta := etcdpb.CollectionMeta{ - ID: UniqueID(0), + Id: UniqueID(0), Schema: &schema, CreateTime: Timestamp(0), SegmentIds: []UniqueID{0}, @@ -168,14 +168,14 @@ func TestColSegContainer_getCollectionByID(t *testing.T) { var collection = node.container.addCollection(&collectionMeta, collectionMetaBlob) assert.Equal(t, collection.meta.Schema.Name, "collection0") - assert.Equal(t, collection.meta.ID, UniqueID(0)) + assert.Equal(t, collection.meta.Id, UniqueID(0)) assert.Equal(t, len(node.container.collections), 1) targetCollection, err := node.container.getCollectionByID(UniqueID(0)) assert.NoError(t, err) assert.NotNil(t, targetCollection) assert.Equal(t, targetCollection.meta.Schema.Name, "collection0") - assert.Equal(t, targetCollection.meta.ID, UniqueID(0)) + assert.Equal(t, targetCollection.meta.Id, UniqueID(0)) } func TestColSegContainer_getCollectionByName(t *testing.T) { @@ -213,7 +213,7 @@ func TestColSegContainer_getCollectionByName(t *testing.T) { } collectionMeta := etcdpb.CollectionMeta{ - ID: UniqueID(0), + Id: UniqueID(0), Schema: &schema, CreateTime: Timestamp(0), SegmentIds: []UniqueID{0}, @@ -226,14 +226,14 @@ func TestColSegContainer_getCollectionByName(t *testing.T) { var collection = node.container.addCollection(&collectionMeta, collectionMetaBlob) assert.Equal(t, collection.meta.Schema.Name, "collection0") - assert.Equal(t, collection.meta.ID, UniqueID(0)) + assert.Equal(t, collection.meta.Id, UniqueID(0)) assert.Equal(t, len(node.container.collections), 1) targetCollection, err := node.container.getCollectionByName("collection0") assert.NoError(t, err) assert.NotNil(t, targetCollection) assert.Equal(t, targetCollection.meta.Schema.Name, "collection0") - assert.Equal(t, targetCollection.meta.ID, UniqueID(0)) + assert.Equal(t, targetCollection.meta.Id, UniqueID(0)) } //----------------------------------------------------------------------------------------------------- partition @@ -272,7 +272,7 @@ func TestColSegContainer_addPartition(t *testing.T) { } collectionMeta := etcdpb.CollectionMeta{ - ID: UniqueID(0), + Id: UniqueID(0), Schema: &schema, CreateTime: Timestamp(0), SegmentIds: []UniqueID{0}, @@ -285,7 +285,7 @@ func TestColSegContainer_addPartition(t *testing.T) { var collection = node.container.addCollection(&collectionMeta, collectionMetaBlob) assert.Equal(t, collection.meta.Schema.Name, "collection0") - assert.Equal(t, collection.meta.ID, UniqueID(0)) + assert.Equal(t, collection.meta.Id, UniqueID(0)) assert.Equal(t, len(node.container.collections), 1) for _, tag := range collectionMeta.PartitionTags { @@ -330,7 +330,7 @@ func TestColSegContainer_removePartition(t *testing.T) { } collectionMeta := etcdpb.CollectionMeta{ - ID: UniqueID(0), + Id: UniqueID(0), Schema: &schema, CreateTime: Timestamp(0), SegmentIds: []UniqueID{0}, @@ -343,7 +343,7 @@ func TestColSegContainer_removePartition(t *testing.T) { var collection = node.container.addCollection(&collectionMeta, collectionMetaBlob) assert.Equal(t, collection.meta.Schema.Name, "collection0") - assert.Equal(t, collection.meta.ID, UniqueID(0)) + assert.Equal(t, collection.meta.Id, UniqueID(0)) assert.Equal(t, len(node.container.collections), 1) for _, tag := range collectionMeta.PartitionTags { @@ -390,7 +390,7 @@ func TestColSegContainer_getPartitionByTag(t *testing.T) { } collectionMeta := etcdpb.CollectionMeta{ - ID: UniqueID(0), + Id: UniqueID(0), Schema: &schema, CreateTime: Timestamp(0), SegmentIds: []UniqueID{0}, @@ -403,7 +403,7 @@ func TestColSegContainer_getPartitionByTag(t *testing.T) { var collection = node.container.addCollection(&collectionMeta, collectionMetaBlob) assert.Equal(t, collection.meta.Schema.Name, "collection0") - assert.Equal(t, collection.meta.ID, UniqueID(0)) + assert.Equal(t, collection.meta.Id, UniqueID(0)) assert.Equal(t, len(node.container.collections), 1) for _, tag := range collectionMeta.PartitionTags { @@ -453,7 +453,7 @@ func TestColSegContainer_addSegment(t *testing.T) { } collectionMeta := etcdpb.CollectionMeta{ - ID: UniqueID(0), + Id: UniqueID(0), Schema: &schema, CreateTime: Timestamp(0), SegmentIds: []UniqueID{0}, @@ -466,7 +466,7 @@ func TestColSegContainer_addSegment(t *testing.T) { var collection = node.container.addCollection(&collectionMeta, collectionMetaBlob) assert.Equal(t, collection.meta.Schema.Name, "collection0") - assert.Equal(t, collection.meta.ID, UniqueID(0)) + assert.Equal(t, collection.meta.Id, UniqueID(0)) assert.Equal(t, len(node.container.collections), 1) partition, err := node.container.addPartition(collection, collectionMeta.PartitionTags[0]) @@ -515,7 +515,7 @@ func TestColSegContainer_removeSegment(t *testing.T) { } collectionMeta := etcdpb.CollectionMeta{ - ID: UniqueID(0), + Id: UniqueID(0), Schema: &schema, CreateTime: Timestamp(0), SegmentIds: []UniqueID{0}, @@ -528,7 +528,7 @@ func TestColSegContainer_removeSegment(t *testing.T) { var collection = node.container.addCollection(&collectionMeta, collectionMetaBlob) assert.Equal(t, collection.meta.Schema.Name, "collection0") - assert.Equal(t, collection.meta.ID, UniqueID(0)) + assert.Equal(t, collection.meta.Id, UniqueID(0)) assert.Equal(t, len(node.container.collections), 1) partition, err := node.container.addPartition(collection, collectionMeta.PartitionTags[0]) @@ -579,7 +579,7 @@ func TestColSegContainer_getSegmentByID(t *testing.T) { } collectionMeta := etcdpb.CollectionMeta{ - ID: UniqueID(0), + Id: UniqueID(0), Schema: &schema, CreateTime: Timestamp(0), SegmentIds: []UniqueID{0}, @@ -592,7 +592,7 @@ func TestColSegContainer_getSegmentByID(t *testing.T) { var collection = node.container.addCollection(&collectionMeta, collectionMetaBlob) assert.Equal(t, collection.meta.Schema.Name, "collection0") - assert.Equal(t, collection.meta.ID, UniqueID(0)) + assert.Equal(t, collection.meta.Id, UniqueID(0)) assert.Equal(t, len(node.container.collections), 1) partition, err := node.container.addPartition(collection, collectionMeta.PartitionTags[0]) @@ -644,7 +644,7 @@ func TestColSegContainer_hasSegment(t *testing.T) { } collectionMeta := etcdpb.CollectionMeta{ - ID: UniqueID(0), + Id: UniqueID(0), Schema: &schema, CreateTime: Timestamp(0), SegmentIds: []UniqueID{0}, @@ -657,7 +657,7 @@ func TestColSegContainer_hasSegment(t *testing.T) { var collection = node.container.addCollection(&collectionMeta, collectionMetaBlob) assert.Equal(t, collection.meta.Schema.Name, "collection0") - assert.Equal(t, collection.meta.ID, UniqueID(0)) + assert.Equal(t, collection.meta.Id, UniqueID(0)) assert.Equal(t, len(node.container.collections), 1) partition, err := node.container.addPartition(collection, collectionMeta.PartitionTags[0]) diff --git a/internal/reader/collection.go b/internal/reader/collection.go index 952429f70c..98039d84ae 100644 --- a/internal/reader/collection.go +++ b/internal/reader/collection.go @@ -26,7 +26,7 @@ func (c *Collection) Name() string { } func (c *Collection) ID() UniqueID { - return (*c.meta).ID + return (*c.meta).Id } func (c *Collection) Partitions() *[]*Partition { diff --git a/internal/reader/collection_test.go b/internal/reader/collection_test.go index 313784a62f..26bce61a49 100644 --- a/internal/reader/collection_test.go +++ b/internal/reader/collection_test.go @@ -46,7 +46,7 @@ func TestCollection_Partitions(t *testing.T) { } collectionMeta := etcdpb.CollectionMeta{ - ID: UniqueID(0), + Id: UniqueID(0), Schema: &schema, CreateTime: Timestamp(0), SegmentIds: []UniqueID{0}, @@ -59,7 +59,7 @@ func TestCollection_Partitions(t *testing.T) { var collection = node.container.addCollection(&collectionMeta, collectionMetaBlob) assert.Equal(t, collection.meta.Schema.Name, "collection0") - assert.Equal(t, collection.meta.ID, UniqueID(0)) + assert.Equal(t, collection.meta.Id, UniqueID(0)) assert.Equal(t, len(node.container.collections), 1) for _, tag := range collectionMeta.PartitionTags { @@ -102,7 +102,7 @@ func TestCollection_newCollection(t *testing.T) { } collectionMeta := etcdpb.CollectionMeta{ - ID: UniqueID(0), + Id: UniqueID(0), Schema: &schema, CreateTime: Timestamp(0), SegmentIds: []UniqueID{0}, @@ -114,7 +114,7 @@ func TestCollection_newCollection(t *testing.T) { collection := newCollection(&collectionMeta, collectionMetaBlob) assert.Equal(t, collection.meta.Schema.Name, "collection0") - assert.Equal(t, collection.meta.ID, UniqueID(0)) + assert.Equal(t, collection.meta.Id, UniqueID(0)) } func TestCollection_deleteCollection(t *testing.T) { @@ -148,7 +148,7 @@ func TestCollection_deleteCollection(t *testing.T) { } collectionMeta := etcdpb.CollectionMeta{ - ID: UniqueID(0), + Id: UniqueID(0), Schema: &schema, CreateTime: Timestamp(0), SegmentIds: []UniqueID{0}, @@ -160,7 +160,7 @@ func TestCollection_deleteCollection(t *testing.T) { collection := newCollection(&collectionMeta, collectionMetaBlob) assert.Equal(t, collection.meta.Schema.Name, "collection0") - assert.Equal(t, collection.meta.ID, UniqueID(0)) + assert.Equal(t, collection.meta.Id, UniqueID(0)) deleteCollection(collection) } diff --git a/internal/reader/data_sync_service_test.go b/internal/reader/data_sync_service_test.go index 6907ce3bca..1c725d7462 100644 --- a/internal/reader/data_sync_service_test.go +++ b/internal/reader/data_sync_service_test.go @@ -68,7 +68,7 @@ func TestManipulationService_Start(t *testing.T) { } collectionMeta := etcdpb.CollectionMeta{ - ID: UniqueID(0), + Id: UniqueID(0), Schema: &schema, CreateTime: Timestamp(0), SegmentIds: []UniqueID{0}, @@ -80,7 +80,7 @@ func TestManipulationService_Start(t *testing.T) { var collection = node.container.addCollection(&collectionMeta, collectionMetaBlob) assert.Equal(t, collection.meta.Schema.Name, "collection0") - assert.Equal(t, collection.meta.ID, UniqueID(0)) + assert.Equal(t, collection.meta.Id, UniqueID(0)) assert.Equal(t, len(node.container.collections), 1) partition, err := node.container.addPartition(collection, collectionMeta.PartitionTags[0]) diff --git a/internal/reader/partition_test.go b/internal/reader/partition_test.go index a21b5556a0..b5b5d791b9 100644 --- a/internal/reader/partition_test.go +++ b/internal/reader/partition_test.go @@ -46,7 +46,7 @@ func TestPartition_Segments(t *testing.T) { } collectionMeta := etcdpb.CollectionMeta{ - ID: UniqueID(0), + Id: UniqueID(0), Schema: &schema, CreateTime: Timestamp(0), SegmentIds: []UniqueID{0}, @@ -59,7 +59,7 @@ func TestPartition_Segments(t *testing.T) { var collection = node.container.addCollection(&collectionMeta, collectionMetaBlob) assert.Equal(t, collection.meta.Schema.Name, "collection0") - assert.Equal(t, collection.meta.ID, UniqueID(0)) + assert.Equal(t, collection.meta.Id, UniqueID(0)) assert.Equal(t, len(node.container.collections), 1) for _, tag := range collectionMeta.PartitionTags { diff --git a/internal/reader/segment_test.go b/internal/reader/segment_test.go index 96eaf71e83..9eb87701f5 100644 --- a/internal/reader/segment_test.go +++ b/internal/reader/segment_test.go @@ -46,7 +46,7 @@ func TestSegment_newSegment(t *testing.T) { } collectionMeta := etcdpb.CollectionMeta{ - ID: UniqueID(0), + Id: UniqueID(0), Schema: &schema, CreateTime: Timestamp(0), SegmentIds: []UniqueID{0}, @@ -58,7 +58,7 @@ func TestSegment_newSegment(t *testing.T) { collection := newCollection(&collectionMeta, collectionMetaBlob) assert.Equal(t, collection.meta.Schema.Name, "collection0") - assert.Equal(t, collection.meta.ID, UniqueID(0)) + assert.Equal(t, collection.meta.Id, UniqueID(0)) segmentID := UniqueID(0) segment := newSegment(collection, segmentID) @@ -96,7 +96,7 @@ func TestSegment_deleteSegment(t *testing.T) { } collectionMeta := etcdpb.CollectionMeta{ - ID: UniqueID(0), + Id: UniqueID(0), Schema: &schema, CreateTime: Timestamp(0), SegmentIds: []UniqueID{0}, @@ -108,7 +108,7 @@ func TestSegment_deleteSegment(t *testing.T) { collection := newCollection(&collectionMeta, collectionMetaBlob) assert.Equal(t, collection.meta.Schema.Name, "collection0") - assert.Equal(t, collection.meta.ID, UniqueID(0)) + assert.Equal(t, collection.meta.Id, UniqueID(0)) segmentID := UniqueID(0) segment := newSegment(collection, segmentID) @@ -149,7 +149,7 @@ func TestSegment_getRowCount(t *testing.T) { } collectionMeta := etcdpb.CollectionMeta{ - ID: UniqueID(0), + Id: UniqueID(0), Schema: &schema, CreateTime: Timestamp(0), SegmentIds: []UniqueID{0}, @@ -161,7 +161,7 @@ func TestSegment_getRowCount(t *testing.T) { collection := newCollection(&collectionMeta, collectionMetaBlob) assert.Equal(t, collection.meta.Schema.Name, "collection0") - assert.Equal(t, collection.meta.ID, UniqueID(0)) + assert.Equal(t, collection.meta.Id, UniqueID(0)) segmentID := UniqueID(0) segment := newSegment(collection, segmentID) @@ -231,7 +231,7 @@ func TestSegment_getDeletedCount(t *testing.T) { } collectionMeta := etcdpb.CollectionMeta{ - ID: UniqueID(0), + Id: UniqueID(0), Schema: &schema, CreateTime: Timestamp(0), SegmentIds: []UniqueID{0}, @@ -243,7 +243,7 @@ func TestSegment_getDeletedCount(t *testing.T) { collection := newCollection(&collectionMeta, collectionMetaBlob) assert.Equal(t, collection.meta.Schema.Name, "collection0") - assert.Equal(t, collection.meta.ID, UniqueID(0)) + assert.Equal(t, collection.meta.Id, UniqueID(0)) segmentID := UniqueID(0) segment := newSegment(collection, segmentID) @@ -320,7 +320,7 @@ func TestSegment_getMemSize(t *testing.T) { } collectionMeta := etcdpb.CollectionMeta{ - ID: UniqueID(0), + Id: UniqueID(0), Schema: &schema, CreateTime: Timestamp(0), SegmentIds: []UniqueID{0}, @@ -332,7 +332,7 @@ func TestSegment_getMemSize(t *testing.T) { collection := newCollection(&collectionMeta, collectionMetaBlob) assert.Equal(t, collection.meta.Schema.Name, "collection0") - assert.Equal(t, collection.meta.ID, UniqueID(0)) + assert.Equal(t, collection.meta.Id, UniqueID(0)) segmentID := UniqueID(0) segment := newSegment(collection, segmentID) @@ -403,7 +403,7 @@ func TestSegment_segmentInsert(t *testing.T) { } collectionMeta := etcdpb.CollectionMeta{ - ID: UniqueID(0), + Id: UniqueID(0), Schema: &schema, CreateTime: Timestamp(0), SegmentIds: []UniqueID{0}, @@ -415,7 +415,7 @@ func TestSegment_segmentInsert(t *testing.T) { collection := newCollection(&collectionMeta, collectionMetaBlob) assert.Equal(t, collection.meta.Schema.Name, "collection0") - assert.Equal(t, collection.meta.ID, UniqueID(0)) + assert.Equal(t, collection.meta.Id, UniqueID(0)) segmentID := UniqueID(0) segment := newSegment(collection, segmentID) @@ -482,7 +482,7 @@ func TestSegment_segmentDelete(t *testing.T) { } collectionMeta := etcdpb.CollectionMeta{ - ID: UniqueID(0), + Id: UniqueID(0), Schema: &schema, CreateTime: Timestamp(0), SegmentIds: []UniqueID{0}, @@ -494,7 +494,7 @@ func TestSegment_segmentDelete(t *testing.T) { collection := newCollection(&collectionMeta, collectionMetaBlob) assert.Equal(t, collection.meta.Schema.Name, "collection0") - assert.Equal(t, collection.meta.ID, UniqueID(0)) + assert.Equal(t, collection.meta.Id, UniqueID(0)) segmentID := UniqueID(0) segment := newSegment(collection, segmentID) @@ -648,7 +648,7 @@ func TestSegment_segmentPreInsert(t *testing.T) { } collectionMeta := etcdpb.CollectionMeta{ - ID: UniqueID(0), + Id: UniqueID(0), Schema: &schema, CreateTime: Timestamp(0), SegmentIds: []UniqueID{0}, @@ -660,7 +660,7 @@ func TestSegment_segmentPreInsert(t *testing.T) { collection := newCollection(&collectionMeta, collectionMetaBlob) assert.Equal(t, collection.meta.Schema.Name, "collection0") - assert.Equal(t, collection.meta.ID, UniqueID(0)) + assert.Equal(t, collection.meta.Id, UniqueID(0)) segmentID := UniqueID(0) segment := newSegment(collection, segmentID) @@ -721,7 +721,7 @@ func TestSegment_segmentPreDelete(t *testing.T) { } collectionMeta := etcdpb.CollectionMeta{ - ID: UniqueID(0), + Id: UniqueID(0), Schema: &schema, CreateTime: Timestamp(0), SegmentIds: []UniqueID{0}, @@ -733,7 +733,7 @@ func TestSegment_segmentPreDelete(t *testing.T) { collection := newCollection(&collectionMeta, collectionMetaBlob) assert.Equal(t, collection.meta.Schema.Name, "collection0") - assert.Equal(t, collection.meta.ID, UniqueID(0)) + assert.Equal(t, collection.meta.Id, UniqueID(0)) segmentID := UniqueID(0) segment := newSegment(collection, segmentID) diff --git a/scripts/README.md b/scripts/README.md index cdd5655119..c3475d6e15 100644 --- a/scripts/README.md +++ b/scripts/README.md @@ -38,7 +38,7 @@ pwd_dir=`pwd` export PATH=$PATH:$(go env GOPATH)/bin export protoc=${pwd_dir}/cmake_build/thirdparty/protobuf/protobuf-build/protoc - ./ci/scripts/proto_gen_go.sh + ./scripts/proto_gen_go.sh ``` #### Check code specifications